From 7e33bb8a41e5925acb2692a3c77421a3bbc470f3 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Tue, 27 Jan 2015 17:11:56 +0100 Subject: [PATCH 001/153] 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 002/153] 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 003/153] 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 004/153] 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 005/153] 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 006/153] 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 007/153] 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 008/153] 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 009/153] 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 010/153] 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 011/153] 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 012/153] 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 013/153] 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; From ab8910eef5d68b32dca38a557c001c6b8847df70 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 3 Apr 2015 14:37:48 +0200 Subject: [PATCH 014/153] Updated README with the new tuple and subspace syntax --- README.md | 34 +++++++++++++--------------------- 1 file changed, 13 insertions(+), 21 deletions(-) diff --git a/README.md b/README.md index ee9775c6c..8e9694436 100644 --- a/README.md +++ b/README.md @@ -51,14 +51,6 @@ This code is licensed under the 3-clause BSD Licence. [![Build status](https://ci.appveyor.com/api/projects/status/83u4pd2ckevdtb57?svg=true)](https://ci.appveyor.com/project/KrzysFR/foundationdb-dotnet-client) -> __There is an incoming API change that may break a few things, regarding the use of Subspaces and Tuples__. The new API is currently in the `refac_tuples` branch, and will be merge soon in `master` for the 0.9 release. If you are just starting, you should probably use the new branch. If you already have existing code, there will be a document explaining the changes and how to fix your source code. __See [#42](https://github.com/Doxense/foundationdb-dotnet-client/pull/42) for more details.__ - -> __There is another incoming change, regarding the way FDB's Futures are implemented under the hood, which should fix some random crash issues under heavy load__. Some of the problems have already been fixed in master, but the new branch `future_ng` is a complete rework, in collaboration with the folks at FoundationDB. __See [#54](https://github.com/Doxense/foundationdb-dotnet-client/pull/54) for more details__ - -> Both these branches are already used in production. Once everything is merged and tested, we will be ready for 1.0. - -> The default API level selected on start is still 200 by default, but this will probably change to 300 by default for 1.0. If you are not doing it already, you should make sure to use `Fdb.UseApiVersion(200)` (or 300) if you want to lock your application to a specific level and be safe against future changes. - How to use ---------- @@ -80,7 +72,7 @@ using (var db = await Fdb.OpenAsync()) using (var trans = db.BeginTransaction(token)) { // For our convenience, we will use the Tuple Encoding format for our keys, - // which is accessible via the "location.Tuples" helper. We could have used + // which is accessible via the "location.Keys" helper. We could have used // any other encoding for the keys. Tuples are simple to use and have some // intereseting ordering properties that make it easy to work with. // => All our keys will be encoded as the packed tuple ({Test}, "foo"), @@ -89,25 +81,25 @@ using (var db = await Fdb.OpenAsync()) // Set "Hello" key to "World" trans.Set( - location.Tuples.EncodeKey("Hello"), + location.Keys.Encode("Hello"), Slice.FromString("World") // UTF-8 encoded string ); // Set "Count" key to 42 trans.Set( - location.Tuples.EncodeKey("Count"), + location.Keys.Encode("Count"), Slice.FromInt32(42) // 1 byte ); // Atomically add 123 to "Total" trans.AtomicAdd( - location.Tuples.EncodeKey("Total"), + location.Keys.Encode("Total"), Slice.FromFixed32(123) // 4 bytes, Little Endian ); // Set bits 3, 9 and 30 in the bit map stored in the key "Bitmap" trans.AtomicOr( - location.Tuples.EncodeKey("Bitmap"), + location.Keys.Encode("Bitmap"), Slice.FromFixed32((1 << 3) | (1 << 9) | (1 << 30)) // 4 bytes, Little Endian ); @@ -122,16 +114,16 @@ using (var db = await Fdb.OpenAsync()) using (var trans = db.BeginReadOnlyTransaction(token)) { // Read ("Test", "Hello", ) as a string - Slice value = await trans.GetAsync(location.Tuples.EncodeKey("Hello")); + Slice value = await trans.GetAsync(location.Keys.Encode("Hello")); Console.WriteLine(value.ToUnicode()); // -> World // Read ("Test", "Count", ) as an int - value = await trans.GetAsync(location.Tuples.EncodeKey("Count")); + value = await trans.GetAsync(location.Keys.Encode("Count")); Console.WriteLine(value.ToInt32()); // -> 42 // missing keys give a result of Slice.Nil, which is the equivalent // of "key not found". - value = await trans.GetAsync(location.Tuples.EncodeKey("NotFound")); + value = await trans.GetAsync(location.Keys.Encode("NotFound")); Console.WriteLine(value.HasValue); // -> false Console.WriteLine(value == Slice.Nil); // -> true // note: there is also Slice.Empty that is returned for existing keys @@ -154,9 +146,9 @@ using (var db = await Fdb.OpenAsync()) await db.WriteAsync((trans) => { // add some data to the list with the format: (..., index) = value - trans.Set(list.Tuples.EncodeKey(0), Slice.FromString("AAA")); - trans.Set(list.Tuples.EncodeKey(1), Slice.FromString("BBB")); - trans.Set(list.Tuples.EncodeKey(2), Slice.FromString("CCC")); + trans.Set(list.Keys.Encode(0), Slice.FromString("AAA")); + trans.Set(list.Keys.Encode(1), Slice.FromString("BBB")); + trans.Set(list.Keys.Encode(2), Slice.FromString("CCC")); // The actual keys will be a concatenation of the prefix of 'list', // and a packed tuple containing the index. Since we are using the // Directory Layer, this should still be fairly small (between 4 @@ -196,13 +188,13 @@ using (var db = await Fdb.OpenAsync()) // with the query itself. return trans // ask for all keys that are _inside_ our subspace - .GetRange(list.Tuples.ToRange()) + .GetRange(list.Keys.ToRange()) // transform the resultoing KeyValuePair into something // nicer to use, like a typed KeyValuePair .Select((kvp) => new KeyValuePair( // unpack the tuple and returns the last item as an int - list.Tuples.DecodeLast(kvp.Key), + list.Keys.DecodeLast(kvp.Key), // convert the value into an unicode string kvp.Value.ToUnicode() )) From 0766c1f5fb90d2f2160c4aab96461454c593a617 Mon Sep 17 00:00:00 2001 From: Alban Lecocq Date: Fri, 10 Apr 2015 17:40:56 +0200 Subject: [PATCH 015/153] Fix nullref --- .../Filters/Logging/FdbTransactionLog.Commands.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index db66ef323..0f33ea038 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -547,6 +547,7 @@ public override string GetArguments(KeyResolver resolver) public override string ToString(KeyResolver resolver) { + resolver = resolver ?? KeyResolver.Default; var arg = this.GetArguments(resolver); var sb = new StringBuilder(); if (this.Snapshot) sb.Append("Snapshot."); From d7f7ea3e8e5f46ab61415263e90e0003c5027f20 Mon Sep 17 00:00:00 2001 From: Alban Lecocq Date: Fri, 10 Apr 2015 17:44:17 +0200 Subject: [PATCH 016/153] Fix FdbTuple.Equals() : Equals() was comparing only first item --- FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs index 17e9b17df..2ab0936f2 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs @@ -1404,7 +1404,7 @@ internal static bool DeepEquals([NotNull] IFdbTuple x, [NotNull] IFdbTuple y, [N { if (!ys.MoveNext()) return false; - return comparer.Equals(xs.Current, ys.Current); + if (!comparer.Equals(xs.Current, ys.Current)) return false; } return !ys.MoveNext(); From 7ba2645e691ed3f526cc0d20f1e784a6e3c19481 Mon Sep 17 00:00:00 2001 From: qcastelain Date: Fri, 17 Apr 2015 14:04:25 +0200 Subject: [PATCH 017/153] bugfix ColaRangeDictionary --- .../Collections/ColaRangeDictionary.cs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs index 73c7be545..907b9025e 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs @@ -438,6 +438,8 @@ public void Mark(TKey begin, TKey end, TValue value) // = [==========).. = [============).. cursor.Set(entry); + //we keep the reference to cursor to be able to modify it later + entry = cursor; inserted = true; //TODO: need to propagate ! } @@ -560,6 +562,8 @@ public void Mark(TKey begin, TKey end, TValue value) if (!inserted) { // use that slot to insert ourselves cursor.Set(entry); + //get the reference to be able to eventually merge it afterwards + entry = cursor; inserted = true; } else From ae5beb065a73d161af1507bcd65e6a0366a26bff Mon Sep 17 00:00:00 2001 From: qcastelain Date: Mon, 20 Apr 2015 14:33:00 +0200 Subject: [PATCH 018/153] =?UTF-8?q?ColaRangeDictionary=20=3D>=20ajout=20de?= =?UTF-8?q?=20la=20methode=20delete=20qui=20supprime=20un=20interval=20et?= =?UTF-8?q?=20d=C3=A9calle=20le=20reste?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../Collections/ColaRangeDictionaryFacts.cs | 219 +++++++++++++ .../Collections/ColaRangeDictionary.cs | 302 ++++++++++++++++++ 2 files changed, 521 insertions(+) diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs index 7b080a1ff..0783a58d1 100644 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs +++ b/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs @@ -328,6 +328,225 @@ public void Test_RangeDictionary_Insert_Backwards() Console.WriteLine("Bounds = " + cola.Bounds); } + [Test] + public void Test_Can_Remove() + { + var dico = GetFilledRange(); + //on supprime tout + dico.Remove(0, 100, -100, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(0)); + + dico = GetFilledRange(); + //on ampute le premier range + dico.Remove(0, 12, -12, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(5)); + int i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 0, 3, true); + else if (i == 1) CompareEntries(entry, 8, 38, false); + else if (i == 2) CompareEntries(entry, 39, 50, true); + else if (i == 3) CompareEntries(entry, 51, 53, true); + else if (i == 4) CompareEntries(entry, 56, 63, false); + i++; + } + + //on supprime un truc a cheval sur plusieurs ranges + dico = GetFilledRange(); + dico.Remove(12, 55, -43, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(4)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 12, true); + if (i == 1) CompareEntries(entry, 12, 19, true); + else if (i == 2) CompareEntries(entry, 20, 22, true); + else if (i == 3) CompareEntries(entry, 25, 32, false); + i++; + } + + //on supprime avant le début + dico = GetFilledRange(); + dico.Remove(0, 8, -8, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(5)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 2, 7, true); + else if (i == 1) CompareEntries(entry, 12, 42, false); + else if (i == 2) CompareEntries(entry, 43, 54, true); + else if (i == 3) CompareEntries(entry, 55, 57, true); + else if (i == 4) CompareEntries(entry, 60, 67, false); + i++; + } + + //on supprimme exactement 2 ranges + dico = GetFilledRange(); + dico.Remove(20, 62, -42, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(3)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 15, true); + else if (i == 1) CompareEntries(entry, 21, 23, true); + else if (i == 2) CompareEntries(entry, 26, 33, false); + i++; + } + + //on supprime de maniere a ce que ca termine sur la fin d'un range + dico = GetFilledRange(); + dico.Remove(0, 50, -50, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(3)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 1, 12, true); + else if (i == 1) CompareEntries(entry, 13, 15, true); + else if (i == 2) CompareEntries(entry, 18, 25, false); + i++; + } + + //on supprimme jusqu'a la fin du premier + dico = GetFilledRange(); + dico.Remove(0, 15, -15, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(4)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 5, 35, false); + else if (i == 1) CompareEntries(entry, 36, 47, true); + else if (i == 2) CompareEntries(entry, 48, 50, true); + else if (i == 3) CompareEntries(entry, 53, 60, false); + i++; + } + + //on supprime jusqu'au milieu du 3e + dico = GetFilledRange(); + dico.Remove(0, 60, -60, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(3)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 0, 2, true); + else if (i == 1) CompareEntries(entry, 3, 5, true); + else if (i == 2) CompareEntries(entry, 8, 15, false); + i++; + } + + //on supprime jusqu'au debut du 3e + dico = GetFilledRange(); + dico.Remove(0, 51, -51, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(3)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 0, 11, true); + else if (i == 1) CompareEntries(entry, 12, 14, true); + else if (i == 2) CompareEntries(entry, 17, 24, false); + i++; + } + + //on supprime le début du 2e + dico = GetFilledRange(); + dico.Remove(20, 30, -10, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(5)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 15, true); + else if (i == 1) CompareEntries(entry, 20, 40, false); + else if (i == 2) CompareEntries(entry, 41, 52, true); + else if (i == 3) CompareEntries(entry, 53, 55, true); + else if (i == 4) CompareEntries(entry, 58, 65, false); + i++; + } + + //on supprime le 2e + dico = GetFilledRange(); + dico.Remove(20, 50, -30, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(4)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 15, true); + else if (i == 1) CompareEntries(entry, 21, 32, true); + else if (i == 2) CompareEntries(entry, 33, 35, true); + else if (i == 3) CompareEntries(entry, 38, 45, false); + i++; + } + + //on supprime le 1er et un bout du 2e + dico = GetFilledRange(); + dico.Remove(10, 30, -20, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(4)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 30, false); + else if (i == 1) CompareEntries(entry, 31, 42, true); + else if (i == 2) CompareEntries(entry, 43, 45, true); + else if (i == 3) CompareEntries(entry, 48, 55, false); + i++; + } + + //on supprime un morceau du second + dico = GetFilledRange(); + dico.Remove(30, 40, -10, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(6)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 15, true); + else if (i == 1) CompareEntries(entry, 20, 30, false); + else if (i == 2) CompareEntries(entry, 30, 40, false); + else if (i == 3) CompareEntries(entry, 41, 52, true); + else if (i == 4) CompareEntries(entry, 53, 55, true); + else if (i == 5) CompareEntries(entry, 58, 65, false); + i++; + } + + //on supprime la fin du second + dico = GetFilledRange(); + dico.Remove(30, 50, -20, (x, y) => x + y); + Assert.That(dico.Count, Is.EqualTo(5)); + i = 0; + foreach (var entry in dico) + { + if (i == 0) CompareEntries(entry, 10, 15, true); + else if (i == 1) CompareEntries(entry, 20, 30, false); + else if (i == 2) CompareEntries(entry, 31, 42, true); + else if (i == 3) CompareEntries(entry, 43, 45, true); + else if (i == 4) CompareEntries(entry, 48, 55, false); + i++; + } + } + + public void CompareEntries(ColaRangeDictionary.Entry entry, TKey begin, TKey end, TValue value) + { + Assert.That(entry.Begin, Is.EqualTo(begin)); + Assert.That(entry.End, Is.EqualTo(end)); + Assert.That(entry.Value, Is.EqualTo(value)); + } + + public ColaRangeDictionary GetFilledRange() + { + //returns a colaRange prefilled with this : + // {10-15, true} + // {20-50, false} + // {51,62, true} + // {63,65, true} + // {68,75, false} + + var cola = new ColaRangeDictionary(); + cola.Mark(10, 15, true); + cola.Mark(20, 50, false); + cola.Mark(51, 62, true); + cola.Mark(63, 65, true); + cola.Mark(68, 75, false); + + return cola; + } + enum RangeColor { Black, diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs index 907b9025e..fce553919 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs @@ -5,6 +5,8 @@ // enables consitency checks after each operation to the set #undef ENFORCE_INVARIANTS +using FoundationDB.Layers.Tuples; + namespace FoundationDB.Storage.Memory.Core { using System; @@ -218,6 +220,306 @@ public void Clear() CheckInvariants(); } + public void Remove(TKey begin, TKey end, TKey offset, Func applyOffset) + { + if (m_keyComparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); + + try + { + var entry = new Entry(begin, end, default(TValue)); + var iterator = m_items.GetIterator(); + var comparer = m_keyComparer; + if (!iterator.Seek(entry, true)) + { + //on ne trouve pas l'item exacte, on prends le premier. + iterator.SeekFirst(); + } + var cursor = iterator.Current; + var c1 = comparer.Compare(begin, cursor.Begin); + var c2 = comparer.Compare(end, cursor.End); + List toRemove = null; + //begin < cursor.Begin + if (c1 < 0) + { + var c3 = comparer.Compare(end, cursor.Begin); + //end <= cursor.Begin + // [++++++++++ + // ------[ + //ou + // [++++++ + // ------[ + if (c3 <= 0) + { + TranslateAfter(null, offset, applyOffset); + return; + } + //end < cursor.End + // [+++++++++++[ + //-----------[ + if (c2 < 0) + { + cursor.Begin = applyOffset(end, offset); + cursor.End = applyOffset(cursor.End, offset); + TranslateAfter(cursor, offset, applyOffset); + return; + } + //end == cursor.End + // [+++++++++[ + //---------------[ + if (c2 == 0) + { + m_items.RemoveItem(cursor); + TranslateAfter(cursor, offset, applyOffset); + return; + } + //end > cursor.End + // [+++++++++[ + //-------------------... + if (c2 > 0) + { + toRemove = new List(); + toRemove.Add(cursor); + while (iterator.Next()) + { + cursor = iterator.Current; + c2 = comparer.Compare(end, cursor.End); + c3 = comparer.Compare(end, cursor.Begin); + //end <= cursor.Begin + // [+++++ + // ----[ + //ou + // [+++++ + // ------[ + if (c3 <= 0) + { + //on set cursor pour que la translation soit faite correctement + cursor = entry; + break; + } + //end > cursor.Begin + if (c3 > 0) + { + //end < cursor.End + // [+++++++++++ + // ----------[ + if (c2 < 0) + { + cursor.Begin = begin; + cursor.End = applyOffset(cursor.End, offset); + break; + } + // end >= cursor.End + // [+++++++++[ + // ---------------[ + //ou + // [+++++++[ + // ----------------... + if (c2 >= 0) + { + toRemove.Add(cursor); + if (c2 == 0) break; + } + } + } + m_items.RemoveItems(toRemove); + TranslateAfter(cursor, offset, applyOffset); + return; + } + } + //begin == cursor.Begin + else if (c1 == 0) + { + //end < cursor.End + // [+++++++++[ + // [-----[ + if (c2 < 0) + { + cursor.Begin = begin; + cursor.End = applyOffset(cursor.End, offset); + TranslateAfter(cursor, offset, applyOffset); + return; + } + //end == cursor.End + // [++++++++[ + // [--------[ + else if (c2 == 0) + { + toRemove = new List(); + toRemove.Add(cursor); + } + // end > cursor.End + // [+++++++[ + // [-----------.... + else + { + toRemove = new List(); + toRemove.Add(cursor); + while (iterator.Next()) + { + cursor = iterator.Current; + var c3 = comparer.Compare(end, cursor.Begin); + c2 = comparer.Compare(end, cursor.End); + //end < cursor.Begin + // [++++++++[ + //---------[ + //ou + // [+++++++[ + //---------[ + if (c3 <= 0) + { + break; + } + else + { + //end < cursor.End + // [++++++++++++[ + //-----[ + if (c2 < 0) + { + cursor.Begin = begin; + cursor.End = applyOffset(cursor.End, offset); + break; + } + //end >= cursor.End + // [+++++++++[ + //---------------... + //ou + // [+++++++++[ + //-----------[ + if (c2 >= 0) + { + toRemove.Add(cursor); + if (c2 == 0) break; + } + } + } + } + m_items.RemoveItems(toRemove); + TranslateAfter(cursor, offset, applyOffset); + return; + } + //begin > cursor.Begin + else + { + //end < cursor.End + // [++++++++++++[ + // [----[ + // = [++[[++++[ + if (c2 < 0) + { + var oldEnd = cursor.End; + cursor.End = begin; + TranslateAfter(cursor, offset, applyOffset); + m_items.Insert(new Entry(begin, applyOffset(oldEnd, offset), cursor.Value)); + return; + } + //end == cursor.End + // [+++++++++++++[ + // [-------[ + if (c2 == 0) + { + cursor.End = begin; + TranslateAfter(cursor, offset, applyOffset); + return; + } + //end > cursor.End + // [+++++++++++++[ + // [------------- + else + { + cursor.End = begin; + while (iterator.Next()) + { + cursor = iterator.Current; + var c3 = comparer.Compare(end, cursor.Begin); + c2 = comparer.Compare(end, cursor.End); + //end <= cursor.Begin + // [++++++++++++[ + // --[ + //ou + // [++++++++++++[ + // -----[ + if (c3 <= 0) + { + break; + } + else + { + //end < cursor.End + // [+++++++++++++[ + // ------------[ + if (c2 < 0) + { + cursor.Begin = begin; + cursor.End = applyOffset(cursor.End, offset); + break; + } + //end >= cursor.End + // [+++++++++++[ + //--------------------... + //ou + // [+++++++++++[ + //---------------[ + else + { + toRemove = new List(); + toRemove.Add(cursor); + if (c2 == 0) break; + } + } + } + + if (toRemove != null) m_items.RemoveItems(toRemove); + TranslateAfter(cursor, offset, applyOffset); + return; + } + } + } + finally + { + CheckInvariants(); + } + } + + public void TranslateAfter(Entry lastOk, TKey offset, Func applyKeyOffset) + { + var iterator = m_items.GetIterator(); + //null il faut tout décaller + if (lastOk == null) + { + if (!iterator.SeekFirst()) return; + } + else + { + if (!iterator.Seek(lastOk, true)) + { + //l'element passé en parametre à été supprimé + //on cherche l'élément suivant + //si tout à été supprimé on sort. + if (!iterator.SeekFirst()) return; + var c = m_keyComparer.Compare(lastOk.End, iterator.Current.Begin); + while (c > 0 && iterator.Next()) + { + c = m_keyComparer.Compare(lastOk.End, iterator.Current.Begin); + } + } + //on veut décaller les suivants de celui passé en parametre + else iterator.Next(); + } + do + { + var cursor = iterator.Current; + //dans le cas ou tout à été supprimé après le lastOK l'iterator est déjà au bout quand on arrive ici... + if (cursor == null) break; + cursor.Begin = applyKeyOffset(cursor.Begin, offset); + cursor.End = applyKeyOffset(cursor.End, offset); + } + while (iterator.Next()); + //on décalle les bounds correctement + if (iterator.SeekFirst()) m_bounds.Begin = iterator.Current.Begin; + if (iterator.SeekLast()) m_bounds.End = iterator.Current.End; + } + public void Mark(TKey begin, TKey end, TValue value) { if (m_keyComparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); From 7450d9788c24d1b63312fd44a0bccb2af683b881 Mon Sep 17 00:00:00 2001 From: qcastelain Date: Mon, 20 Apr 2015 17:31:36 +0200 Subject: [PATCH 019/153] fix du delete --- FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs index fce553919..b0463ef75 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs @@ -462,7 +462,7 @@ public void Remove(TKey begin, TKey end, TKey offset, Func app //---------------[ else { - toRemove = new List(); + if(toRemove == null) toRemove = new List(); toRemove.Add(cursor); if (c2 == 0) break; } From b0600d5b9144e616335221aa226785f93b239963 Mon Sep 17 00:00:00 2001 From: qcastelain Date: Mon, 20 Apr 2015 17:39:53 +0200 Subject: [PATCH 020/153] xml doc on ColaRangeDictionary.Remove + TranslateAfter becomes private --- .../Collections/ColaRangeDictionary.cs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs index b0463ef75..099cd03aa 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs @@ -220,6 +220,13 @@ public void Clear() CheckInvariants(); } + /// + /// Removes everything between begin and end then translates everything + /// + /// begin key + /// end key + /// offset to apply + /// func to apply offset to a key public void Remove(TKey begin, TKey end, TKey offset, Func applyOffset) { if (m_keyComparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); @@ -481,7 +488,7 @@ public void Remove(TKey begin, TKey end, TKey offset, Func app } } - public void TranslateAfter(Entry lastOk, TKey offset, Func applyKeyOffset) + private void TranslateAfter(Entry lastOk, TKey offset, Func applyKeyOffset) { var iterator = m_items.GetIterator(); //null il faut tout décaller From 4b7c318eb91de86d4805545fb12f37c38cff60d1 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 7 May 2015 16:15:38 +0200 Subject: [PATCH 021/153] Fix potential issue in ComparisonHelper when comparing integers of different types - tracking a random issue where Equals((long) 5, (ulong) 7) would return true - if both TryAdaptToInteger(...) fail, then we have null == null and Equals(..) could return true --- .../Converters/ConversionHelper.cs | 82 +++++++++++-------- 1 file changed, 50 insertions(+), 32 deletions(-) diff --git a/FoundationDB.Client/Converters/ConversionHelper.cs b/FoundationDB.Client/Converters/ConversionHelper.cs index 6a5ad55e0..dd26ca9b7 100644 --- a/FoundationDB.Client/Converters/ConversionHelper.cs +++ b/FoundationDB.Client/Converters/ConversionHelper.cs @@ -40,7 +40,7 @@ internal static class ComparisonHelper { /// Pair of types that can be used as a key in a dictionary - internal struct TypePair + internal struct TypePair : IEquatable { public readonly Type Left; public readonly Type Right; @@ -54,8 +54,13 @@ public TypePair(Type left, Type right) public override bool Equals(object obj) { if (obj == null) return false; - TypePair other = (TypePair)obj; - return this.Left == other.Left && this.Right == other.Right; + return Equals((TypePair)obj); + } + + public bool Equals(TypePair other) + { + return this.Left == other.Left + && this.Right == other.Right; } public override int GetHashCode() @@ -89,7 +94,7 @@ public int GetHashCode(TypePair obj) /// Cache of all the comparison lambda for a pair of types /// Contains lambda that can compare two objects (of different types) for "similarity" - private static readonly ConcurrentDictionary> EqualityComparers = new ConcurrentDictionary>(ComparisonHelper.TypePairComparer.Default); + private static readonly ConcurrentDictionary> EqualityComparers = new ConcurrentDictionary>(TypePairComparer.Default); /// Tries to convert an object into an equivalent string representation (for equality comparison) /// Object to adapt @@ -117,49 +122,52 @@ internal static string TryAdaptToString(object value) /// Tries to convert an object into an equivalent double representation (for equality comparison) /// Object to adapt /// Type of the object to adapt - /// Double equivalent of the object - internal static double? TryAdaptToDecimal(object value, [NotNull] Type type) + /// Double equivalent of the object + /// True if is compatible with a decimal. False if the type is not compatible + internal static bool TryAdaptToDecimal(object value, [NotNull] Type type, out double result) { if (value != null) { switch (Type.GetTypeCode(type)) { - case TypeCode.Int16: return (short)value; - case TypeCode.UInt16: return (ushort)value; - case TypeCode.Int32: return (int)value; - case TypeCode.UInt32: return (uint)value; - case TypeCode.Int64: return (long)value; - case TypeCode.UInt64: return (ulong)value; - case TypeCode.Single: return (float)value; - case TypeCode.Double: return (double)value; + case TypeCode.Int16: { result = (short)value; return true; } + case TypeCode.UInt16: { result = (ushort)value; return true; } + case TypeCode.Int32: { result = (int)value; return true; } + case TypeCode.UInt32: { result = (uint)value; return true; } + case TypeCode.Int64: { result = (long)value; return true; } + case TypeCode.UInt64: { result = (ulong)value; return true; } + case TypeCode.Single: { result = (float)value; return true; } + case TypeCode.Double: { result = (double)value; return true; } //TODO: string? } } - return null; + result = 0; + return false; } /// Tries to convert an object into an equivalent Int64 representation (for equality comparison) /// Object to adapt /// Type of the object to adapt - /// Int64 equivalent of the object - internal static long? TryAdaptToInteger(object value, [NotNull] Type type) + /// Int64 equivalent of the object + /// True if is compatible with a decimal. False if the type is not compatible + internal static bool TryAdaptToInteger(object value, [NotNull] Type type, out long result) { if (value != null) { switch (Type.GetTypeCode(type)) { - case TypeCode.Int16: return (short)value; - case TypeCode.UInt16: return (ushort)value; - case TypeCode.Int32: return (int)value; - case TypeCode.UInt32: return (uint)value; - case TypeCode.Int64: return (long)value; - case TypeCode.UInt64: return (long?)(ulong)value; - case TypeCode.Single: return (long?)(float)value; - case TypeCode.Double: return (long?)(double)value; - //TODO: string? + case TypeCode.Int16: { result = (short)value; return true; } + case TypeCode.UInt16: { result = (ushort)value; return true; } + case TypeCode.Int32: { result = (int)value; return true; } + case TypeCode.UInt32: { result = (uint)value; return true; } + case TypeCode.Int64: { result = (long)value; return true; } + case TypeCode.UInt64: { result = (long)(ulong)value; return true; } + case TypeCode.Single: { result = (long)(float)value; return true; } + case TypeCode.Double: { result = (long)(double)value; return true; } } } - return null; + result = 0; + return false; } [NotNull] @@ -210,12 +218,21 @@ private static Func CreateTypeComparator([NotNull] Type t1 { if (IsDecimalType(t1) || IsDecimalType(t2)) { - return (x, y) => x == null ? y == null : y != null && TryAdaptToDecimal(x, t1) == TryAdaptToDecimal(y, t2); + return (x, y) => + { + double d1, d2; + return x == null ? y == null : y != null && TryAdaptToDecimal(x, t1, out d1) && TryAdaptToDecimal(y, t2, out d2) && d1 == d2; + }; + } + else + { + //TODO: handle UInt64 with values > long.MaxValue that will overflow to negative values when casted down to Int64 + return (x, y) => + { + long l1, l2; + return x == null ? y == null : y != null && TryAdaptToInteger(x, t1, out l1) && TryAdaptToInteger(y, t2, out l2) && l1 == l2; + }; } - - //TODO: handle UInt64 with values > long.MaxValue that will overflow to negative values when casted down to Int64 - - return (x, y) => x == null ? y == null : y != null && TryAdaptToInteger(x, t1) == TryAdaptToInteger(y, t2); } //TODO: some other way to compare ? @@ -317,6 +334,7 @@ private static bool IsNumericType([NotNull] Type t) private static bool IsDecimalType(Type t) { return t == typeof(double) || t == typeof(float); + //TODO: System.Decimal? } } From 8dcbf91368ce26adb205348e6c837caeadd6f1c4 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 13 May 2015 15:02:03 +0200 Subject: [PATCH 022/153] #if out the inside of [Conditional] herlper methods - even thoug they are not called, their body was still present in the assembly --- FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs | 4 ++++ FoundationDB.Client/Async/AsyncPump.cs | 2 ++ .../Linq/Iterators/FdbAsyncIteratorPump.cs | 2 ++ .../Linq/Iterators/FdbParallelSelectAsyncIterator.cs | 2 ++ FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs | 8 +++++++- .../API/MemoryTransactionHandler.cs | 5 ++++- .../Collections/ColaOrderedDictionary.cs | 2 ++ FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs | 2 ++ .../Collections/ColaRangeDictionary.cs | 5 ++++- FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs | 2 ++ FoundationDB.Storage.Memory/Collections/ColaStore.cs | 2 ++ FoundationDB.Storage.Memory/Collections/ColaStore`1.cs | 6 ++++-- FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs | 2 ++ FoundationDb.Client.sln.DotSettings | 1 + 14 files changed, 40 insertions(+), 5 deletions(-) diff --git a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs index 0ad6fb39e..83cdce814 100644 --- a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs +++ b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs @@ -145,13 +145,17 @@ public void Dispose() [Conditional("FULL_DEBUG")] protected void LogProducer(string msg, [CallerMemberName] string caller = null) { +#if FULL_DEBUG Console.WriteLine("@@@ [producer#{0}] {1} [{2}]", Thread.CurrentThread.ManagedThreadId, msg, caller); +#endif } [Conditional("FULL_DEBUG")] protected void LogConsumer(string msg, [CallerMemberName] string caller = null) { +#if FULL_DEBUG Console.WriteLine("@@@ [consumer#{0}] {1} [{2}]", Thread.CurrentThread.ManagedThreadId, msg, caller); +#endif } #endregion diff --git a/FoundationDB.Client/Async/AsyncPump.cs b/FoundationDB.Client/Async/AsyncPump.cs index b53a37a22..1c60840ae 100644 --- a/FoundationDB.Client/Async/AsyncPump.cs +++ b/FoundationDB.Client/Async/AsyncPump.cs @@ -189,7 +189,9 @@ public void Dispose() [Conditional("FULL_DEBUG")] private static void LogPump(string msg) { +#if FULL_DEBUG Console.WriteLine("[pump#{0}] {1}", Thread.CurrentThread.ManagedThreadId, msg); +#endif } #endregion diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs index 36dd46750..eef5c63f8 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs @@ -79,7 +79,9 @@ internal int State [Conditional("FULL_DEBUG")] private static void LogDebug(string msg) { +#if FULL_DEBUG Console.WriteLine("[pump] " + msg); +#endif } /// Run the pump until the inner iterator is done, an error occurs, or the cancellation token is fired diff --git a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs index 04748f9bd..0db5cbfdb 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs @@ -179,7 +179,9 @@ protected override void Dispose(bool disposing) [Conditional("FULL_DEBUG")] private static void LogDebug(string msg) { +#if FULL_DEBUG Console.WriteLine("[SelectAsync] " + msg); +#endif } } diff --git a/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs b/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs index 5daa29236..febfa1652 100644 --- a/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs +++ b/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs @@ -603,6 +603,7 @@ private void EnsureReadVersionNotInTheFuture_NeedsLocking(ulong readVersion) [Conditional("FULLDEBUG")] private unsafe static void DumpKey(string label, IntPtr userKey) { +#if FULLDEBUG var sb = new StringBuilder("(*) " + (label ?? "key") + " = "); if (userKey == IntPtr.Zero) { @@ -636,6 +637,7 @@ private unsafe static void DumpKey(string label, IntPtr userKey) } } Trace.WriteLine(sb.ToString()); +#endif } private unsafe bool TryGetValueAtVersion(USlice lookupKey, ulong sequence, out USlice result) @@ -1436,10 +1438,12 @@ public void MarkAsCancelled() } - [Conditional("FULL_DEBUG")] + [Conditional("FULLDEBUG")] private static void Log(string msg) { +#if FULLDEBUG Trace.WriteLine("MemoryDatabaseHandler[#" + Thread.CurrentThread.ManagedThreadId + "]: " + msg); +#endif } private const int STATE_IDLE = 0; @@ -1683,6 +1687,7 @@ private void ThrowDisposed() [Conditional("DEBUG")] public void Debug_Dump(bool detailed = false) { +#if DEBUG Debug.WriteLine("Dumping content of Database"); m_dataLock.EnterReadLock(); try @@ -1713,6 +1718,7 @@ public void Debug_Dump(bool detailed = false) { m_dataLock.ExitReadLock(); } +#endif } } diff --git a/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs b/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs index 3f0bf2b2e..e24260475 100644 --- a/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs +++ b/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs @@ -3,6 +3,7 @@ #endregion #undef DUMP_TRANSACTION_STATE +#undef FULLDEBUG namespace FoundationDB.Storage.Memory.API { @@ -1257,10 +1258,12 @@ public void Dispose() GC.SuppressFinalize(this); } - [Conditional("FULL_DEBUG")] + [Conditional("FULLDEBUG")] private static void Log(string msg) { +#if FULLDEBUG Trace.WriteLine("MemoryTransactionHandler[#" + Thread.CurrentThread.ManagedThreadId + "]: " + msg); +#endif } } diff --git a/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs index 0b8737052..ff467031d 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs @@ -382,8 +382,10 @@ private static void ThrowKeyAlreadyExists() [Conditional("DEBUG")] public void Debug_Dump() { +#if DEBUG Trace.WriteLine("Dumping ColaOrderedDictionary<" + typeof(TKey).Name + ", " + typeof(TValue).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); m_items.Debug_Dump(); +#endif } [StructLayout(LayoutKind.Sequential)] diff --git a/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs b/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs index bb08b076a..9582ce49c 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs @@ -224,8 +224,10 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() [Conditional("DEBUG")] public void Debug_Dump() { +#if DEBUG Trace.WriteLine("Dumping ColaOrderedSet<" + typeof(T).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); m_items.Debug_Dump(); +#endif } [StructLayout(LayoutKind.Sequential)] diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs index 73c7be545..7c7d9b224 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs @@ -124,6 +124,7 @@ public ColaRangeDictionary(int capacity, IComparer keyComparer, IComparer< [Conditional("ENFORCE_INVARIANTS")] private void CheckInvariants() { +#if ENFORCE_INVARIANTS Contract.Assert(m_bounds != null); Debug.WriteLine("INVARIANTS:" + this.ToString() + " <> " + m_bounds.ToString()); @@ -160,7 +161,7 @@ private void CheckInvariants() Contract.Assert(EqualityComparer.Default.Equals(m_bounds.Begin, first.Begin), String.Format("Min bound {0} does not match with {1}", m_bounds.Begin, first.Begin)); Contract.Assert(EqualityComparer.Default.Equals(m_bounds.End, previous.End), String.Format("Max bound {0} does not match with {1}", m_bounds.End, previous.End)); } - +#endif } public int Count { get { return m_items.Count; } } @@ -663,8 +664,10 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() //TODO: remove or set to internal ! public void Debug_Dump() { +#if DEBUG Debug.WriteLine("Dumping ColaRangeDictionary<" + typeof(TKey).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); m_items.Debug_Dump(); +#endif } public override string ToString() diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs index d8492d249..b1109b041 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs @@ -328,8 +328,10 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() //TODO: remove or set to internal ! public void Debug_Dump() { +#if DEBUG Console.WriteLine("Dumping ColaRangeSet<" + typeof(TKey).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); m_items.Debug_Dump(); +#endif } public override string ToString() diff --git a/FoundationDB.Storage.Memory/Collections/ColaStore.cs b/FoundationDB.Storage.Memory/Collections/ColaStore.cs index 96067d3af..d0f53fc3e 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaStore.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaStore.cs @@ -873,6 +873,7 @@ internal Iterator(T[][] levels, int count, IComparer comparer) [Conditional("FULLDEBUG")] private void Debug_Dump(string label = null) { +#if FULLDEBUG Trace.WriteLine("* Cursor State: " + label); for (int i = m_min; i < m_cursors.Length; i++) { @@ -886,6 +887,7 @@ private void Debug_Dump(string label = null) Trace.WriteLine(" - L" + i + ": " + p + " [" + (1 << i) + "] = " + (p < 0 ? "" : (p >= (1 << i)) ? "" : ("" + m_levels[i][p]))); } Trace.WriteLine(" > Current at " + m_currentLevel + " : " + m_current); +#endif } /// Set the cursor just before the first key in the store diff --git a/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs b/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs index 596e0df74..7e7c74a2d 100644 --- a/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs +++ b/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs @@ -143,6 +143,7 @@ public ColaStore(int capacity, IComparer comparer) [Conditional("ENFORCE_INVARIANTS")] private void CheckInvariants() { +#if ENFORCE_INVARIANTS Contract.Assert(m_count >= 0, "Count cannot be less than zero"); Contract.Assert(m_levels != null, "Storage array should not be null"); Contract.Assert(m_levels.Length > 0, "Storage array should always at least contain one level"); @@ -183,9 +184,7 @@ private void CheckInvariants() if (i < m_spares.Length) { -#if ENFORCE_INVARIANTS Contract.Assert(!m_spareUsed[i], "A spare level wasn't returned after being used!"); -#endif var spare = m_spares[i]; if (spare == null) continue; // All spare segments SHOULD be filled with default(T) @@ -200,6 +199,7 @@ private void CheckInvariants() } } +#endif } #endregion @@ -1093,6 +1093,7 @@ internal IEnumerable IterateUnordered() [Conditional("DEBUG")] public void Debug_Dump(Func dump = null) { +#if DEBUG Trace.WriteLine("> " + m_levels.Length + " levels:"); for(int i = 0; i < m_levels.Length; i++) { @@ -1108,6 +1109,7 @@ public void Debug_Dump(Func dump = null) } #endif Trace.WriteLine("> " + m_count + " items"); +#endif } } diff --git a/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs b/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs index 336ab1922..5a62577bf 100644 --- a/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs +++ b/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs @@ -136,6 +136,7 @@ protected virtual void Dispose(bool disposing) [Conditional("DEBUG")] public void Debug_Dump(bool detailed) { +#if DEBUG Debug.WriteLine("# Dumping {0} heap ({1:N0} pages in {2:N0} buckets)", this.GetType().Name, m_buckets.Sum(b => (long)b.Pages.Count), m_buckets.Length); //TODO: needs locking but should only be called from unit tests anyway... ulong entries = 0; @@ -163,6 +164,7 @@ public void Debug_Dump(bool detailed) } } Debug.WriteLine("# Found a total of {0:N0} entries using {1:N0} bytes out of {2:N0} bytes allocated", entries, used, allocated); +#endif } } diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index 47110b55c..6836c13c3 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -64,5 +64,6 @@ <Policy Inspect="True" Prefix="s_" Suffix="" Style="aaBb" /> <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 True \ No newline at end of file From 9b2d197073283f1cf328490cf6f45699d816ac81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gr=C3=A9goire=20Castre?= Date: Wed, 3 Jun 2015 16:05:54 +0200 Subject: [PATCH 023/153] Ajout du MaxValue sur un Uuid128 --- FoundationDB.Client/Utils/Uuid128.cs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Utils/Uuid128.cs index 8a7249fd1..dedf37cd4 100644 --- a/FoundationDB.Client/Utils/Uuid128.cs +++ b/FoundationDB.Client/Utils/Uuid128.cs @@ -48,7 +48,7 @@ public struct Uuid128 : IFormattable, IComparable, IEquatable, ICompara // Significant Byte first (known as network byte order). Note that the // field names, particularly for multiplexed fields, follow historical // practice. - + // 0 1 2 3 // 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ @@ -137,6 +137,8 @@ public static explicit operator Uuid128(Guid guid) public static readonly Uuid128 Empty = default(Uuid128); + public static readonly Uuid128 MaxValue = new Uuid128(uint.MaxValue, ushort.MaxValue, ushort.MaxValue, 255, 255, 255, 255, 255, 255, 255, 255); + public static Uuid128 NewUuid() { return new Uuid128(Guid.NewGuid()); From 0c3b736f017f61a061bc5bd5b1ed04ae675b2825 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 4 Jun 2015 11:14:26 +0200 Subject: [PATCH 024/153] Fixed FdbKeyRange.StartsWith(Slice.Empty) - return ('', '\xFF') for this case --- FoundationDB.Client/FdbKeyRange.cs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/FoundationDB.Client/FdbKeyRange.cs b/FoundationDB.Client/FdbKeyRange.cs index a66c45110..940704d30 100644 --- a/FoundationDB.Client/FdbKeyRange.cs +++ b/FoundationDB.Client/FdbKeyRange.cs @@ -85,6 +85,8 @@ public static FdbKeyRange Create(Slice a, Slice b) public static FdbKeyRange StartsWith(Slice prefix) { if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull("prefix"); + if (prefix.Count == 0) return new FdbKeyRange(Slice.Empty, FdbKey.MaxValue); + // prefix => [ prefix, prefix + 1 ) return new FdbKeyRange( From 039c3d927c6afffe28c5188efb65d2b166f6c537 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 4 Jun 2015 11:15:17 +0200 Subject: [PATCH 025/153] Better ToString() on Uuid64 - handles lower or upper hexadecimal formatting --- FoundationDB.Client/Utils/SliceReader.cs | 21 ++++++ FoundationDB.Client/Utils/Uuid64.cs | 83 +++++++++++++++++------- 2 files changed, 81 insertions(+), 23 deletions(-) diff --git a/FoundationDB.Client/Utils/SliceReader.cs b/FoundationDB.Client/Utils/SliceReader.cs index bb92ade0f..b86bae06f 100644 --- a/FoundationDB.Client/Utils/SliceReader.cs +++ b/FoundationDB.Client/Utils/SliceReader.cs @@ -62,6 +62,18 @@ public SliceReader(Slice buffer) this.Position = 0; } + public SliceReader(byte[] buffer) + { + m_buffer = new Slice(buffer, 0, buffer.Length); + this.Position = 0; + } + + public SliceReader(byte[] buffer, int offset, int count) + { + m_buffer = new Slice(buffer, offset, count); + this.Position = 0; + } + /// Returns true if there are more bytes to parse public bool HasMore { get { return this.Position < m_buffer.Count; } } @@ -256,6 +268,15 @@ public Slice ReadVarbytes() return ReadBytes((int)size); } + public Uuid128 ReadUuid128() + { + return ReadBytes(16).ToUuid128(); + } + + public Uuid64 ReadUuid64() + { + return ReadBytes(8).ToUuid64(); + } } } diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index 387eaa169..2a3125cde 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -256,9 +256,12 @@ public string ToString(string format, IFormatProvider formatProvider) switch(format) { case "D": + { // Default format is "xxxxxxxx-xxxxxxxx" + return Encode16(m_value, separator: true, quotes: false, upper: true); + } case "d": { // Default format is "xxxxxxxx-xxxxxxxx" - return Encode16(m_value, separator: true, quotes: false); + return Encode16(m_value, separator: true, quotes: false, upper: false); } case "C": @@ -278,16 +281,24 @@ public string ToString(string format, IFormatProvider formatProvider) return m_value.ToString(null, formatProvider ?? CultureInfo.InvariantCulture); } - case "X": - case "x": + case "X": //TODO: Guid.ToString("X") returns "{0x.....,0x.....,...}" + case "N": + { // "XXXXXXXXXXXXXXXX" + return Encode16(m_value, separator: false, quotes: false, upper: true); + } + case "x": //TODO: Guid.ToString("X") returns "{0x.....,0x.....,...}" + case "n": { // "xxxxxxxxxxxxxxxx" - return Encode16(m_value, separator: false, quotes: false); + return Encode16(m_value, separator: false, quotes: false, upper: false); } case "B": + { // "{xxxxxxxx-xxxxxxxx}" + return Encode16(m_value, separator: true, quotes: true, upper: true); + } case "b": { // "{xxxxxxxx-xxxxxxxx}" - return Encode16(m_value, separator: true, quotes: true); + return Encode16(m_value, separator: true, quotes: true, upper: false); } } throw new FormatException("Invalid Uuid64 format specification."); @@ -325,57 +336,83 @@ public int CompareTo(Uuid64 other) #region Base16 encoding... - private static char HexToChar(int a) + private static char HexToLowerChar(int a) { a &= 0xF; return a > 9 ? (char)(a - 10 + 'a') : (char)(a + '0'); } - private static unsafe char* HexsToChars(char* ptr, int a) + private static unsafe char* HexsToLowerChars(char* ptr, int a) + { + Contract.Requires(ptr != null); + ptr[0] = HexToLowerChar(a >> 28); + ptr[1] = HexToLowerChar(a >> 24); + ptr[2] = HexToLowerChar(a >> 20); + ptr[3] = HexToLowerChar(a >> 16); + ptr[4] = HexToLowerChar(a >> 12); + ptr[5] = HexToLowerChar(a >> 8); + ptr[6] = HexToLowerChar(a >> 4); + ptr[7] = HexToLowerChar(a); + return ptr + 8; + } + + private static char HexToUpperChar(int a) + { + a &= 0xF; + return a > 9 ? (char)(a - 10 + 'A') : (char)(a + '0'); + } + + private static unsafe char* HexsToUpperChars(char* ptr, int a) { Contract.Requires(ptr != null); - ptr[0] = HexToChar(a >> 28); - ptr[1] = HexToChar(a >> 24); - ptr[2] = HexToChar(a >> 20); - ptr[3] = HexToChar(a >> 16); - ptr[4] = HexToChar(a >> 12); - ptr[5] = HexToChar(a >> 8); - ptr[6] = HexToChar(a >> 4); - ptr[7] = HexToChar(a); + ptr[0] = HexToUpperChar(a >> 28); + ptr[1] = HexToUpperChar(a >> 24); + ptr[2] = HexToUpperChar(a >> 20); + ptr[3] = HexToUpperChar(a >> 16); + ptr[4] = HexToUpperChar(a >> 12); + ptr[5] = HexToUpperChar(a >> 8); + ptr[6] = HexToUpperChar(a >> 4); + ptr[7] = HexToUpperChar(a); return ptr + 8; } - private unsafe static string Encode16(ulong value, bool separator, bool quotes) + private unsafe static string Encode16(ulong value, bool separator, bool quotes, bool upper) { int size = 16 + (separator ? 1 : 0) + (quotes ? 2 : 0); char* buffer = stackalloc char[24]; // max 19 mais on arrondi a 24 char* ptr = buffer; if (quotes) *ptr++ = '{'; - ptr = HexsToChars(ptr, (int)(value >> 32)); + ptr = upper + ? HexsToUpperChars(ptr, (int)(value >> 32)) + : HexsToLowerChars(ptr, (int)(value >> 32)); if (separator) *ptr++ = '-'; - ptr = HexsToChars(ptr, (int)(value & 0xFFFFFFFF)); + ptr = upper + ? HexsToUpperChars(ptr, (int)(value & 0xFFFFFFFF)) + : HexsToLowerChars(ptr, (int)(value & 0xFFFFFFFF)); if (quotes) *ptr++ = '}'; Contract.Assert(ptr == buffer + size); return new string(buffer, 0, size); } + private const int INVALID_CHAR = -1; + private static int CharToHex(char c) { if (c <= '9') { - return c >= '0' ? (c - 48) : -1; + return c >= '0' ? (c - 48) : INVALID_CHAR; } if (c <= 'F') { - return c >= 'A' ? (c - 55) : -1; + return c >= 'A' ? (c - 55) : INVALID_CHAR; } if (c <= 'f') { - return c >= 'a' ? (c - 87) : -1; + return c >= 'a' ? (c - 87) : INVALID_CHAR; } - return -1; + return INVALID_CHAR; } private static bool TryCharsToHexs(char[] chars, int offset, out uint result) @@ -384,7 +421,7 @@ private static bool TryCharsToHexs(char[] chars, int offset, out uint result) for (int i = 0; i < 8; i++) { int a = CharToHex(chars[offset++]); - if (a == -1) + if (a == INVALID_CHAR) { result = 0; return false; From 91dd3a7f5b93eb3be3cc9739d9697a34687ef3e9 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 10 Jul 2015 16:51:38 +0200 Subject: [PATCH 026/153] Eat the exception when resizing the console screen fails --- FdbShell/Program.cs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/FdbShell/Program.cs b/FdbShell/Program.cs index 3367d15df..bbc08976a 100644 --- a/FdbShell/Program.cs +++ b/FdbShell/Program.cs @@ -89,10 +89,17 @@ public static async Task> RunAsyncCommand(Func 0 && Console.LargestWindowHeight > 0) + try + { + if (Console.LargestWindowWidth > 0 && Console.LargestWindowHeight > 0) + { + Console.WindowWidth = 160; + Console.WindowHeight = 60; + } + } + catch (Exception e) { - Console.WindowWidth = 160; - Console.WindowHeight = 60; + // this sometimes fail on small screen sizes } // Initialize FDB From 79dd0efe0a31ab3041a5ffadb05714efe03b4c30 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 23 Jul 2015 23:30:30 +0200 Subject: [PATCH 027/153] [ItemNotNull] all the things! - Equivalent of [NotNull] but for async methods that return Task starting from R# 9.2 EAP 4 - Also added [PublicAPI] were applicable --- .../Core/IFdbClusterHandler.cs | 5 ++++- .../Core/IFdbDatabaseHandler.cs | 3 ++- .../Core/IFdbTransactionHandler.cs | 6 ++++- FoundationDB.Client/Fdb.Bulk.cs | 3 ++- FoundationDB.Client/Fdb.Options.cs | 4 +++- FoundationDB.Client/Fdb.System.cs | 12 +++++++++- FoundationDB.Client/Fdb.cs | 13 +++++++++-- FoundationDB.Client/FdbCluster.cs | 4 +++- FoundationDB.Client/FdbDatabaseExtensions.cs | 6 ++++- FoundationDB.Client/FdbRangeQuery.cs | 4 +++- .../FdbTransactionExtensions.cs | 14 +++++++++++- .../Logging/FdbTransactionLog.Commands.cs | 4 +++- .../FoundationDB.Client.csproj | 4 +--- FoundationDB.Client/IFdbCluster.cs | 5 +++-- FoundationDB.Client/IFdbDatabase.cs | 3 ++- FoundationDB.Client/IFdbKey.cs | 2 +- FoundationDB.Client/IFdbReadOnlyRetryable.cs | 3 ++- .../IFdbReadOnlyTransaction.cs | 6 ++++- FoundationDB.Client/IFdbRetryable.cs | 3 ++- FoundationDB.Client/IFdbTransaction.cs | 4 +++- .../Layers/Directories/FdbDirectoryLayer.cs | 22 +++++++++++++++++-- .../Directories/FdbDirectorySubspace.cs | 1 + FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 3 ++- .../Linq/FdbAsyncEnumerable.cs | 6 +++++ FoundationDB.Client/Native/FdbFuture.cs | 1 + FoundationDB.Client/Status/FdbSystemStatus.cs | 1 + .../Subspaces/Fdb.Directory.cs | 4 ++++ FoundationDB.Client/Subspaces/FdbSubspace.cs | 1 + .../Subspaces/IFdbDynamicSubspace.cs | 1 + .../Subspaces/IFdbEncoderSubspace.cs | 1 + FoundationDB.Client/Subspaces/IFdbSubspace.cs | 2 +- .../TypeSystem/Encoders/KeyValueEncoders.cs | 1 + FoundationDB.Client/Utils/Slice.cs | 2 +- FoundationDB.Client/Utils/Uuid128.cs | 2 +- FoundationDB.Client/Utils/Uuid64.cs | 2 +- 35 files changed, 127 insertions(+), 31 deletions(-) diff --git a/FoundationDB.Client/Core/IFdbClusterHandler.cs b/FoundationDB.Client/Core/IFdbClusterHandler.cs index 4585138e9..e5bba6094 100644 --- a/FoundationDB.Client/Core/IFdbClusterHandler.cs +++ b/FoundationDB.Client/Core/IFdbClusterHandler.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 @@ -28,11 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Core { + using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; /// Basic API for FoundationDB clusters + [PublicAPI] public interface IFdbClusterHandler : IDisposable { bool IsInvalid { get; } @@ -40,6 +42,7 @@ public interface IFdbClusterHandler : IDisposable void SetOption(FdbClusterOption option, Slice data); + [ItemNotNull] Task OpenDatabaseAsync(string databaseName, CancellationToken cancellationToken); } diff --git a/FoundationDB.Client/Core/IFdbDatabaseHandler.cs b/FoundationDB.Client/Core/IFdbDatabaseHandler.cs index 1bb0a477e..3859bd262 100644 --- a/FoundationDB.Client/Core/IFdbDatabaseHandler.cs +++ b/FoundationDB.Client/Core/IFdbDatabaseHandler.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 @@ -32,6 +32,7 @@ namespace FoundationDB.Client.Core using System; /// Basic API for FoundationDB databases + [PublicAPI] public interface IFdbDatabaseHandler : IDisposable { bool IsInvalid { get; } diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 49280b240..4dada99aa 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.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 @@ -34,6 +34,7 @@ namespace FoundationDB.Client.Core using System.Threading.Tasks; /// Basic API for FoundationDB transactions + [PublicAPI] public interface IFdbTransactionHandler : IDisposable { /// Returns the estimated payload size of the transaction (including keys and values) @@ -80,6 +81,7 @@ public interface IFdbTransactionHandler : IDisposable /// Set to true for snapshot reads /// Token used to cancel the operation from the outside /// 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. + [ItemNotNull] Task GetValuesAsync([NotNull] Slice[] keys, bool snapshot, CancellationToken cancellationToken); /// Resolves a key selector against the keys in the database snapshot represented by the current transaction. @@ -94,6 +96,7 @@ public interface IFdbTransactionHandler : IDisposable /// Set to true for snapshot reads /// Token used to cancel the operation from the outside /// Task that will return an array of keys matching the selectors, or an exception + [ItemNotNull] Task GetKeysAsync([NotNull] FdbKeySelector[] selectors, bool snapshot, CancellationToken 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. @@ -110,6 +113,7 @@ public interface IFdbTransactionHandler : IDisposable /// Name of the key whose location is to be queried. /// Token used to cancel the operation from the outside /// Task that will return an array of strings, or an exception + [ItemNotNull] Task GetAddressesForKeyAsync(Slice key, CancellationToken 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. diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index da4a0fc3f..f5da73655 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.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 @@ -40,6 +40,7 @@ namespace FoundationDB.Client public static partial class Fdb { /// Helper class for bulk operations + [PublicAPI] public static class Bulk { diff --git a/FoundationDB.Client/Fdb.Options.cs b/FoundationDB.Client/Fdb.Options.cs index 545860788..738b3c1ee 100644 --- a/FoundationDB.Client/Fdb.Options.cs +++ b/FoundationDB.Client/Fdb.Options.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 @@ -28,6 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { + using JetBrains.Annotations; using System; public static partial class Fdb @@ -36,6 +37,7 @@ public static partial class Fdb //REVIEW: consider changing this to an instance class so that we could do a Fluent API ? ex: Fdb.Options.WithFoo(...).WithBar(...).WithBaz(...) /// Global settings for the FoundationDB binding + [PublicAPI] public static class Options { diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 51eb9fbf0..8152640d9 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.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 @@ -43,6 +43,7 @@ public static partial class Fdb { /// Helper class for reading from the reserved System subspace + [PublicAPI] public static class System { //REVIEW: what happens if someone mutates (by mitake or not) the underlying buffer of the defaults keys ? @@ -85,6 +86,7 @@ public static class System private static readonly Slice StatusJsonKey = Slice.FromAscii("\xFF\xFF/status/json"); + [ItemCanBeNull] public static async Task GetStatusAsync([NotNull] IFdbReadOnlyTransaction trans) { if (trans == null) throw new ArgumentNullException("trans"); @@ -107,6 +109,7 @@ public static async Task GetStatusAsync([NotNull] IFdbReadOnlyT return new FdbSystemStatus(doc, rv, jsonText); } + [ItemCanBeNull] public static async Task GetStatusAsync([NotNull] IFdbDatabase db, CancellationToken ct) { if (db == null) throw new ArgumentNullException("db"); @@ -128,6 +131,7 @@ public static async Task GetStatusAsync([NotNull] IFdbDatabase /// Database to use for the operation /// Token used to cancel the operation /// Since the list of coordinators may change at anytime, the results may already be obsolete once this method completes! + [ItemNotNull] public static async Task GetCoordinatorsAsync([NotNull] IFdbDatabase db, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); @@ -200,6 +204,7 @@ public static Slice WorkersKey([NotNull] string id, [NotNull] string name) /// Token used to cancel the operation /// Returns either "memory" or "ssd" /// Will return a string starting with "unknown" if the storage engine mode is not recognized + [ItemNotNull] public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabase db, CancellationToken cancellationToken) { // The '\xFF/conf/storage_engine' keys has value "0" (ASCII) for ssd engine, and "1" (ASCII) for memory engine @@ -226,6 +231,7 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas /// End key (exclusive) of the range to inspect /// List of keys that mark the start of a new chunk /// This method is not transactional. It will return an answer no older than the Transaction object it is passed, but the returned boundaries are an estimate and may not represent the exact boundary locations at any database version. + [ItemNotNull] public static async Task> GetBoundaryKeysAsync([NotNull] IFdbReadOnlyTransaction trans, Slice beginInclusive, Slice endExclusive) { if (trans == null) throw new ArgumentNullException("trans"); @@ -250,6 +256,7 @@ public static async Task> GetBoundaryKeysAsync([NotNull] IFdbReadOnl /// Token used to cancel the operation /// List of keys that mark the start of a new chunk /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned boundaries are an estimate and may not represent the exact boundary locations at any database version. + [ItemNotNull] public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); @@ -265,6 +272,7 @@ public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, /// Token used to cancel the operation /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. + [ItemNotNull] public static Task> GetChunksAsync([NotNull] IFdbDatabase db, FdbKeyRange range, CancellationToken cancellationToken) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? @@ -278,6 +286,7 @@ public static Task> GetChunksAsync([NotNull] IFdbDatabase db, /// Token used to cancel the operation /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. + [ItemNotNull] public static async Task> GetChunksAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? @@ -310,6 +319,7 @@ public static async Task> GetChunksAsync([NotNull] IFdbDatabas return chunks; } + [ItemNotNull] private static async Task> GetBoundaryKeysInternalAsync([NotNull] IFdbReadOnlyTransaction trans, Slice begin, Slice end) { Contract.Requires(trans != null && end >= begin); diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index a78622e9a..3005f7dfc 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.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 @@ -42,6 +42,7 @@ namespace FoundationDB.Client using SystemIO = System.IO; /// FoundationDB binding + [PublicAPI] public static partial class Fdb { @@ -441,6 +442,7 @@ private static void FailCannotExecuteOnNetworkThread() /// Opens a connection to an existing FoundationDB cluster using the default cluster file /// Token used to abort the operation /// Task that will return an FdbCluster, or an exception + [ItemNotNull] public static Task CreateClusterAsync(CancellationToken cancellationToken) { return CreateClusterAsync(null, cancellationToken); @@ -450,12 +452,14 @@ public static Task CreateClusterAsync(CancellationToken cancellatio /// Path to the 'fdb.cluster' file to use, or null for the default cluster file /// Token used to abort the operation /// Task that will return an FdbCluster, or an exception + [ItemNotNull] public static async Task CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) { return await CreateClusterInternalAsync(clusterFile, cancellationToken).ConfigureAwait(false); } - internal static async Task CreateClusterInternalAsync(string clusterFile, CancellationToken cancellationToken) + [ItemNotNull] + private static async Task CreateClusterInternalAsync(string clusterFile, CancellationToken cancellationToken) { EnsureIsStarted(); @@ -482,6 +486,7 @@ internal static async Task CreateClusterInternalAsync(string cluster /// Task that will return an FdbDatabase, or an exception /// 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. + [ItemNotNull] public static Task OpenAsync(CancellationToken cancellationToken = default(CancellationToken)) { return OpenAsync(clusterFile: null, dbName: null, globalSpace: FdbSubspace.Empty, cancellationToken: cancellationToken); @@ -493,6 +498,7 @@ internal static async Task CreateClusterInternalAsync(string cluster /// Task that will return an FdbDatabase, or an exception /// 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. + [ItemNotNull] public static Task OpenAsync(IFdbSubspace globalSpace, CancellationToken cancellationToken = default(CancellationToken)) { return OpenAsync(clusterFile: null, dbName: null, globalSpace: globalSpace, cancellationToken: cancellationToken); @@ -507,6 +513,7 @@ 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. + [ItemNotNull] public static Task OpenAsync(string clusterFile, string dbName, CancellationToken cancellationToken = default(CancellationToken)) { return OpenAsync(clusterFile, dbName, FdbSubspace.Empty, readOnly: false, cancellationToken: cancellationToken); @@ -523,12 +530,14 @@ 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. + [ItemNotNull] public static async Task OpenAsync(string clusterFile, string dbName, IFdbSubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) { return await 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 + [ItemNotNull] internal static async Task OpenInternalAsync(string clusterFile, string dbName, IFdbSubspace globalSpace, bool readOnly, CancellationToken cancellationToken) { cancellationToken.ThrowIfCancellationRequested(); diff --git a/FoundationDB.Client/FdbCluster.cs b/FoundationDB.Client/FdbCluster.cs index c4a633c44..c5da090ae 100644 --- a/FoundationDB.Client/FdbCluster.cs +++ b/FoundationDB.Client/FdbCluster.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 @@ -111,6 +111,7 @@ protected virtual void Dispose(bool disposing) /// If is anything other than 'DB' /// If the token is cancelled /// Any attempt to use a key outside the specified subspace will throw an exception + [ItemNotNull] public async Task OpenDatabaseAsync(string databaseName, IFdbSubspace subspace, bool readOnly, CancellationToken cancellationToken) { if (subspace == null) throw new ArgumentNullException("subspace"); @@ -127,6 +128,7 @@ public async Task OpenDatabaseAsync(string databaseName, IFdbSubsp /// If is anything other than 'DB' /// If the token is cancelled /// As of Beta2, the only supported database name is 'DB' + [ItemNotNull] internal async Task OpenDatabaseInternalAsync(string databaseName, IFdbSubspace subspace, bool readOnly, bool ownsCluster, CancellationToken cancellationToken) { ThrowIfDisposed(); diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index d07b96cdf..86da37643 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.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 @@ -229,6 +229,7 @@ public static Task GetAsync(this IFdbReadOnlyRetryable db, Slice key, Can /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// + [ItemNotNull] public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNull] Slice[] keys, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); @@ -241,6 +242,7 @@ public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNu /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// + [ItemNotNull] public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keys, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); @@ -265,6 +267,7 @@ public static Task GetKeyAsync(this IFdbReadOnlyRetryable db, FdbKeySelec /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// + [ItemNotNull] public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull] FdbKeySelector[] keySelectors, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); @@ -278,6 +281,7 @@ public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// + [ItemNotNull] public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index 5d2765fdd..a2877ce61 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.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 @@ -278,12 +278,14 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) } /// Return a list of all the elements of the range results + [ItemNotNull] public Task> ToListAsync() { return FdbAsyncEnumerable.ToListAsync(this, this.Transaction.Cancellation); } /// Return an array with all the elements of the range results + [ItemNotNull] public Task ToArrayAsync() { return FdbAsyncEnumerable.ToArrayAsync(this, this.Transaction.Cancellation); diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index ecbf5986e..90534829c 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.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 @@ -946,6 +946,7 @@ public static FdbWatch SetAndWatch(this IFdbTransaction trans, TKe /// /// Sequence of keys to be looked up in the database /// Task that will return an array of values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value will be Slice.Nil. + [ItemNotNull] public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) { if (trans == null) throw new ArgumentNullException("trans"); @@ -962,6 +963,7 @@ public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [ /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . + [ItemNotNull] public static async Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) { if (decoder == null) throw new ArgumentNullException("decoder"); @@ -974,6 +976,7 @@ public static async Task GetValuesAsync(this IFdbReadOnlyTrans /// /// Sequence of keys to be looked up in the database /// Task that will return an array of values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value will be Slice.Nil. + [ItemNotNull] public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) where TKey : IFdbKey { @@ -988,6 +991,7 @@ public static Task GetValuesAsync(this IFdbReadOnlyTransaction tr /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . + [ItemNotNull] public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) where TKey : IFdbKey { @@ -1001,6 +1005,7 @@ public static Task GetValuesAsync(this IFdbReadOnlyTrans /// /// Sequence of key selectors to resolve /// Task that will return an array of keys matching the selectors, or an exception + [ItemNotNull] public static Task GetKeysAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable selectors) { if (trans == null) throw new ArgumentNullException("trans"); @@ -1017,6 +1022,7 @@ public static Task GetKeysAsync(this IFdbReadOnlyTransaction trans, [No /// Sequence of keys to be looked up in the database /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. /// This method is equivalent to calling , except that it will return the keys in addition to the values. + [ItemNotNull] public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) { if (keys == null) throw new ArgumentNullException("keys"); @@ -1032,6 +1038,7 @@ public static Task[]> GetBatchAsync(this IFdbReadOnly /// Array of keys to be looked up in the database /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. /// This method is equivalent to calling , except that it will return the keys in addition to the values. + [ItemNotNull] public static async Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys) { if (trans == null) throw new ArgumentNullException("trans"); @@ -1054,6 +1061,7 @@ public static async Task[]> GetBatchAsync(this IFdbRe /// Array of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . + [ItemNotNull] public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) { if (keys == null) throw new ArgumentNullException("keys"); @@ -1069,6 +1077,7 @@ public static Task[]> GetBatchAsync(this IFd /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . + [ItemNotNull] public static async Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys, [NotNull] IValueEncoder decoder) { if (trans == null) throw new ArgumentNullException("trans"); @@ -1092,6 +1101,7 @@ public static async Task[]> GetBatchAsync(th /// Sequence of keys to be looked up in the database /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. /// This method is equivalent to calling , except that it will return the keys in addition to the values. + [ItemNotNull] public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) where TKey : IFdbKey { @@ -1105,6 +1115,7 @@ public static Task[]> GetBatchAsync(this IFdbRe /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . + [ItemNotNull] public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) where TKey : IFdbKey { @@ -1121,6 +1132,7 @@ public static Task[]> GetBatchAsync(th /// Lambda function that returns an async enumerable. The function may be called multiple times if the transaction conflicts. /// Token used to cancel the operation /// Task returning the list of all the elements of the async enumerable returned by the last successfull call to . + [ItemNotNull] public static Task> QueryAsync(this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index 0f33ea038..1c39075fc 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -30,13 +30,14 @@ namespace FoundationDB.Filters.Logging { using FoundationDB.Async; using FoundationDB.Client; + using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.Text; using System.Threading.Tasks; - using Layers.Directories; + using FoundationDB.Layers.Directories; public partial class FdbTransactionLog { @@ -296,6 +297,7 @@ public DirectoryKeyResolver(Dictionary knownSubspaces) /// Create a key resolver using the content of a DirectoryLayer as the map /// Resolver that replace each directory prefix by its name + [ItemNotNull] public static async Task BuildFromDirectoryLayer(IFdbReadOnlyTransaction tr, FdbDirectoryLayer directory) { var location = directory.NodeSubspace.Keys; diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index cea93251c..737f34d38 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -269,9 +269,7 @@ - - - + - + \ No newline at end of file From 34376465c67c5f9aa55faf237cdd18808f706a08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gr=C3=A9goire=20Castre?= Date: Thu, 18 Aug 2016 16:46:52 +0200 Subject: [PATCH 045/153] Add ToHashsetAsync method --- .../Linq/FdbAsyncEnumerable.Iterators.cs | 81 ++++++++++++++----- .../Linq/FdbAsyncEnumerable.cs | 15 ++++ 2 files changed, 75 insertions(+), 21 deletions(-) diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs index ffe927d0c..2aaa9b0f3 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs @@ -35,6 +35,7 @@ namespace FoundationDB.Linq using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -148,7 +149,7 @@ internal static FdbWhereAsyncIterator Filter( [NotNull] internal static FdbWhereSelectAsyncIterator Offset( - [NotNull] IFdbAsyncEnumerable source, + [NotNull] IFdbAsyncEnumerable source, int offset) { return new FdbWhereSelectAsyncIterator(source, filter: null, transform: new AsyncTransformExpression(TaskHelpers.Cache.Identity), limit: null, offset: offset); @@ -184,7 +185,7 @@ internal class Buffer /// Default intial capacity, if not specified const int DefaultCapacity = 16; - //REVIEW: should we use a power of 2 or of 10 for initial capacity? + //REVIEW: should we use a power of 2 or of 10 for initial capacity? // Since humans prefer the decimal system, it is more likely that query limit count be set to something like 10, 50, 100 or 1000 // but most "human friendly" limits are close to the next power of 2, like 10 ~= 16, 50 ~= 64, 100 ~= 128, 500 ~= 512, 1000 ~= 1024, so we don't waste that much space... @@ -306,33 +307,71 @@ public List ToList() } var list = new List(count); - if (count > 0) + var chunks = this.Chunks; + for (int i = 0; i < chunks.Length - 1; i++) { - var chunks = this.Chunks; - for (int i = 0; i < chunks.Length - 1; i++) - { - list.AddRange(chunks[i]); - count -= chunks[i].Length; - } + list.AddRange(chunks[i]); + count -= chunks[i].Length; + } - var current = this.Current; - if (count == current.Length) - { // the last chunk fits perfectly - list.AddRange(current); - } - else - { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer - // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum - for (int i = 0; i < count; i++) - { - list.Add(current[i]); - } + var current = this.Current; + if (count == current.Length) + { // the last chunk fits perfectly + list.AddRange(current); + } + else + { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer + // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum + for (int i = 0; i < count; i++) + { + list.Add(current[i]); } } return list; } + /// Return the content of the buffer + /// List of size containing all the items in this buffer + [NotNull] + public HashSet ToHashSet(IEqualityComparer comparer = null) + { + int count = this.Count; + var hashset = new HashSet(comparer); + if (count == 0) + { + return hashset; + } + + var chunks = this.Chunks; + + for (int i = 0; i < chunks.Length - 1; i++) + { + foreach (var item in chunks[i]) + { + hashset.Add(item); + } + count -= chunks[i].Length; + } + + var current = this.Current; + if (count == current.Length) + { // the last chunk fits perfectly + foreach (var item in current) + { + hashset.Add(item); + } + } + else + { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer + // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum + for (int i = 0; i < count; i++) + { + hashset.Add(current[i]); + } + } + return hashset; + } } /// Immediately execute an action on each element of an async sequence diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs index c15be783b..1a9faa67a 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs @@ -598,6 +598,21 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingCreate an Hashset from an async sequence. + [ItemNotNull] + public static Task> ToHashsetAsync([NotNull] this IFdbAsyncEnumerable source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + if (source == null) throw new ArgumentNullException("source"); + + return AggregateAsync( + source, + new Buffer(), + (buffer, x) => buffer.Add(x), + (buffer) => buffer.ToHashSet(comparer), + ct + ); + } + /// Create a list from an async sequence. [ItemNotNull] public static Task> ToListAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) From b1a10d562359a88f41ac76f32ca37a75d0f0e65e Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 19 Aug 2016 14:50:34 +0200 Subject: [PATCH 046/153] FailFast if network already set up in reused process --- FoundationDB.Client/Fdb.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 26a1d74ee..6c4ae7fe5 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -614,8 +614,8 @@ public static void Start() #if DEBUG case FdbError.ApiVersionAlreadySet: { // Temporary hack to allow multiple debugging using the cached host process in VS - Console.WriteLine("REUSING EXISTING PROCESS! IF THINGS BREAK IN WEIRD WAYS, PLEASE RESTART THE PROCESS!"); - err = FdbError.OperationFailed; + Console.Error.WriteLine("FATAL: CANNOT REUSE EXISTING PROCESS! FoundationDB client cannot be restarted once stopped. Current process will be terminated."); + Environment.FailFast("FATAL: CANNOT REUSE EXISTING PROCESS! FoundationDB client cannot be restarted once stopped. Current process will be terminated."); break; } #endif From 380cafdde9e97e3ea9626d4d18f2f372cb731a46 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 22 Aug 2016 14:12:25 +0200 Subject: [PATCH 047/153] Fail Fast if network already set up --- FoundationDB.Client/Fdb.cs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 26a1d74ee..6c4ae7fe5 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -614,8 +614,8 @@ public static void Start() #if DEBUG case FdbError.ApiVersionAlreadySet: { // Temporary hack to allow multiple debugging using the cached host process in VS - Console.WriteLine("REUSING EXISTING PROCESS! IF THINGS BREAK IN WEIRD WAYS, PLEASE RESTART THE PROCESS!"); - err = FdbError.OperationFailed; + Console.Error.WriteLine("FATAL: CANNOT REUSE EXISTING PROCESS! FoundationDB client cannot be restarted once stopped. Current process will be terminated."); + Environment.FailFast("FATAL: CANNOT REUSE EXISTING PROCESS! FoundationDB client cannot be restarted once stopped. Current process will be terminated."); break; } #endif From 8f1e844a0e7bdba5822d877a5965ed9069ff6d05 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 26 Aug 2016 14:51:04 +0200 Subject: [PATCH 048/153] Add a bunch of test cases --- FoundationDB.Tests/TransactionFacts.cs | 728 ++++++++++++++++++++++++- 1 file changed, 723 insertions(+), 5 deletions(-) diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index de4d4e3cb..cd6fbc903 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -26,9 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +// ReSharper disable ConsiderUsingConfigureAwait namespace FoundationDB.Client.Tests { - using FoundationDB.Layers.Tuples; using NUnit.Framework; using System; using System.Collections.Generic; @@ -43,6 +43,7 @@ public class TransactionFacts : FdbTest { [Test] + public async Task Test_Can_Create_And_Dispose_Transactions() { using (var db = await OpenTestDatabaseAsync()) @@ -1443,7 +1444,7 @@ public async Task Test_Has_Access_To_System_Keys() await TestHelpers.AssertThrowsFdbErrorAsync( () => tr.GetRange(Slice.FromAscii("\xFF"), Slice.FromAscii("\xFF\xFF"), new FdbRangeOptions { Limit = 10 }).ToListAsync(), - FdbError.KeyOutsideLegalRange, + FdbError.KeyOutsideLegalRange, "Should not have access to system keys by default" ); @@ -1843,7 +1844,7 @@ public async Task Test_Can_Get_Boundary_Keys() string[] ids = null; foreach (var key in shards) { - // - the first 12 bytes are some sort of header: + // - the first 12 bytes are some sort of header: // - bytes 0-5 usually are 01 00 01 10 A2 00 // - bytes 6-7 contains 0x0FDB which is the product's signature // - bytes 8-9 contains the version (02 00 for "2.0"?) @@ -1859,7 +1860,7 @@ public async Task Test_Can_Get_Boundary_Keys() distinctNodes.Add(ids[i]); } replicationFactor = Math.Max(replicationFactor, ids.Length); - + // the node id seems to be at offset 12 //Console.WriteLine("- " + key.Value.Substring(0, 12).ToAsciiOrHexaString() + " : " + String.Join(", ", ids) + " = " + key.Key); @@ -1901,7 +1902,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); @@ -2047,6 +2048,723 @@ await db.WriteAsync((tr) => } } + + [Test] + public async void Test_Case_1() + { + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); + tr.Set(Slice.FromString("BBB"), Slice.FromString("222")); + tr.Set(Slice.FromString("CCC"), Slice.FromString("333")); + tr.Set(Slice.FromString("DDD"), Slice.FromString("444")); + tr.Set(Slice.FromString("EEE"), Slice.FromString("555")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_2() + { + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + tr.ClearRange(Slice.FromString("AAA"), Slice.FromString("ZZZ")); + tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); + tr.Set(Slice.FromString("BBB"), Slice.FromString("222")); + tr.Set(Slice.FromString("CCC"), Slice.FromString("333")); + tr.Set(Slice.FromString("DDD"), Slice.FromString("444")); + tr.Set(Slice.FromString("EEE"), Slice.FromString("555")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_3() + { + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + tr.ClearRange(Slice.FromString("AAA"), Slice.FromString("BBB")); + tr.ClearRange(Slice.FromString("BBB"), Slice.FromString("CCC")); + tr.ClearRange(Slice.FromString("CCC"), Slice.FromString("DDD")); + // should be merged into a single AAA..DDD + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_4() + { + Fdb.Start(300); + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + //initial setup: + // A: none + // B: 0 + // C: 255 + // D: none + // E: none + tr.Set(Slice.FromString("BBB"), Slice.FromFixed32(0)); + tr.Set(Slice.FromString("CCC"), Slice.FromFixed32(255)); + + // add 1 to everybody + tr.AtomicAdd(Slice.FromString("AAA"), Slice.FromFixed32(1)); + tr.AtomicAdd(Slice.FromString("BBB"), Slice.FromFixed32(1)); + tr.AtomicAdd(Slice.FromString("CCC"), Slice.FromFixed32(1)); + tr.AtomicAdd(Slice.FromString("DDD"), Slice.FromFixed32(1)); + tr.AtomicAdd(Slice.FromString("EEE"), Slice.FromFixed32(1)); + + // overwrite DDD with a fixed value + tr.Set(Slice.FromString("DDD"), Slice.FromFixed32(5)); + // double add on EEE + tr.AtomicAdd(Slice.FromString("EEE"), Slice.FromFixed32(1)); + + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_5() + { + Fdb.Start(300); + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); + tr.AtomicAdd(Slice.FromString("BBB"), Slice.FromString("222")); + tr.AtomicAnd(Slice.FromString("CCC"), Slice.FromString("333")); + tr.AtomicOr(Slice.FromString("DDD"), Slice.FromString("444")); + tr.AtomicXor(Slice.FromString("EEE"), Slice.FromString("555")); + tr.AtomicMax(Slice.FromString("FFF"), Slice.FromString("666")); + tr.AtomicMin(Slice.FromString("GGG"), Slice.FromString("777")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_6() + { + Fdb.Start(300); + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + { + var tr = db.BeginTransaction(this.Cancellation); + + tr.AtomicMax(Slice.FromString("MAXMAX1"), Slice.FromString("EEE")); + tr.AtomicMax(Slice.FromString("MAXMAX1"), Slice.FromString("FFF")); + + tr.AtomicMax(Slice.FromString("MAXMAX2"), Slice.FromString("FFF")); + tr.AtomicMax(Slice.FromString("MAXMAX2"), Slice.FromString("EEE")); + + tr.AtomicMin(Slice.FromString("MINMIN1"), Slice.FromString("111")); + tr.AtomicMin(Slice.FromString("MINMIN1"), Slice.FromString("222")); + + tr.AtomicMin(Slice.FromString("MINMIN2"), Slice.FromString("222")); + tr.AtomicMin(Slice.FromString("MINMIN2"), Slice.FromString("111")); + + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_6b() + { + Fdb.Start(300); + using (var db = await Fdb.OpenAsync(this.Cancellation)) + { + Slice init = Slice.Repeat(0xCC, 9); + Slice mask = Slice.Repeat(0xAA, 9); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(Slice.FromString("AAA"), init); + tr.Set(Slice.FromString("BBB"), init); + tr.Set(Slice.FromString("CCC"), init); + tr.Set(Slice.FromString("DDD"), init); + tr.Set(Slice.FromString("EEE"), init); + tr.Set(Slice.FromString("FFF"), init); + tr.Set(Slice.FromString("GGG"), init); + + await tr.CommitAsync(); + } + + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(Slice.FromString("AAA"), mask); + tr.AtomicAdd(Slice.FromString("BBB"), mask); + tr.AtomicAnd(Slice.FromString("CCC"), mask); + tr.AtomicOr(Slice.FromString("DDD"), mask); + tr.AtomicXor(Slice.FromString("EEE"), mask); + tr.AtomicMin(Slice.FromString("FFF"), mask); + tr.AtomicMax(Slice.FromString("GGG"), mask); + + await tr.CommitAsync(); + } + + await DumpSubspace(db, db.GlobalSpace); + + } + } + + [Test] + public async void Test_Case_7() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + using (var tr = db.BeginTransaction(this.Cancellation)) + { + + var vX = Slice.FromFixedU32BE(0x55555555); // X + var vY = Slice.FromFixedU32BE(0x66666666); // Y + var vL1 = Slice.FromFixedU32BE(0x11111111); // Low + var vL2 = Slice.FromFixedU32BE(0x22222222); // Low + var vH2 = Slice.FromFixedU32BE(0xFFFFFFFF); // Hi + var vH1 = Slice.FromFixedU32BE(0xEEEEEEEE); // Hi + var vA = Slice.FromFixedU32BE(0xAAAAAAAA); // 10101010 + var vC = Slice.FromFixedU32BE(0xCCCCCCCC); // 11001100 + + var cmds = new[] + { + new { Op = "SET", Left = vX, Right = vY }, + new { Op = "ADD", Left = vX, Right = vY }, + new { Op = "AND", Left = vA, Right = vC }, + new { Op = "OR", Left = vA, Right = vC }, + new { Op = "XOR", Left = vA, Right = vC }, + new { Op = "MIN", Left = vL1, Right = vL2 }, + new { Op = "MAX", Left = vH1, Right = vH2 }, + }; + + Action apply = (t, op, k, v) => + { + switch (op) + { + case "SET": + t.Set(k, v); + break; + case "ADD": + t.AtomicAdd(k, v); + break; + case "AND": + t.AtomicAnd(k, v); + break; + case "OR": + t.AtomicOr(k, v); + break; + case "XOR": + t.AtomicXor(k, v); + break; + case "MIN": + t.AtomicMin(k, v); + break; + case "MAX": + t.AtomicMax(k, v); + break; + default: + Assert.Fail(); + break; + } + }; + + for (int i = 0; i < cmds.Length; i++) + { + for (int j = 0; j < cmds.Length; j++) + { + Slice key = Slice.FromString(cmds[i].Op + "_" + cmds[j].Op); + Log("{0};{1} = {2}", i, j, key); + apply(tr, cmds[i].Op, key, cmds[i].Left); + apply(tr, cmds[j].Op, key, cmds[j].Right); + } + } + + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_8() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + +#if false // RUNONCE + + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999\x00")); + for (int i = 0; i < 1000; i++) + { + tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromFixedU32BE((uint)i)); + } + }, this.Cancellation); +#endif + + + for (int i = 0; i < 100; i++) + { + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + var res = await tr.GetAsync(Slice.FromString("K" + i.ToString("D4"))); + Console.WriteLine(res); + } + } + } + } + } + + [Test] + public async void Test_Case_9() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything + await db.WriteAsync((tr) => tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")), this.Cancellation); + + await db.WriteAsync(tr => tr.Set(Slice.FromString("K0123"), Slice.FromString("V0123")), this.Cancellation); + await db.WriteAsync(tr => tr.Set(Slice.FromString("K0789"), Slice.FromString("V0789")), this.Cancellation); + + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetValuesAsync(new[] { + Slice.FromString("K0123"), + Slice.FromString("K0234"), + Slice.FromString("K0456"), + Slice.FromString("K0567"), + Slice.FromString("K0789") + }); + } + + // once more with feelings + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetValuesAsync(new[] { + Slice.FromString("K0123"), + Slice.FromString("K0234"), + Slice.FromString("K0456"), + Slice.FromString("K0567"), + Slice.FromString("K0789") + }); + } + } + } + } + + [Test] + public async void Test_Case_10() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")); + for (int i = 0; i < 100; i++) + { + tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); + } + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(Slice.FromString("K0010"), Slice.FromString("K0020")); + tr.ClearRange(Slice.FromString("K0050"), Slice.FromString("K0060")); + + var chunk = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0000")), + FdbKeySelector.LastLessOrEqual(Slice.FromString("K9999")), + new FdbRangeOptions { Mode = FdbStreamingMode.WantAll, Reverse = true } + ); + + //no commit + } + + } + } + } + + [Test] + public async void Test_Case_11() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")); + for (int i = 0; i < 100; i++) + { + tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); + } + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(Slice.FromString("K0010"), Slice.FromString("K0020")); + tr.ClearRange(Slice.FromString("K0050"), Slice.FromString("K0060")); + tr.Set(Slice.FromString("K0021"), Slice.Empty); + tr.Set(Slice.FromString("K0042"), Slice.Empty); + + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0005"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0010"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0015"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0022"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0049"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0050"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0055"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0061"))); + + //no commit + } + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + //tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0000"))); // equal=false, offset=1 + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(Slice.FromString("K0011"))); // equal=true, offset=1 + await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(Slice.FromString("K0022"))); // equal=true, offset=0 + await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromString("K0033"))); // equal=false, offset=0 + + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0040")) + 1000); // equal=false, offset=7 ? + await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromString("K0050")) + 1000); // equal=false, offset=6 ? + } + + } + } + } + + [Test] + public async void Test_Case_12() + { + + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + await tr.GetAsync(Slice.FromString("KGET")); + tr.AddReadConflictRange(Slice.FromString("KRC0"), Slice.FromString("KRC0")); + tr.AddWriteConflictRange(Slice.FromString("KWRITECONFLICT0"), Slice.FromString("KWRITECONFLICT1")); + tr.Set(Slice.FromString("KWRITE"), Slice.Empty); + await tr.CommitAsync(); + } + + // once more with feelings + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("KGETKEY"))); + } + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.AddReadConflictRange(Slice.FromString("KRC0"), Slice.FromString("KRC1")); + tr.Set(Slice.FromString("KWRITE"), Slice.Empty); + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_13() + { + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); + tr.Set(Slice.FromString("K000"), Slice.FromString("BEGIN")); + for (int i = 0; i < 5; i++) + { + tr.Set(Slice.FromString("K" + i + "A"), Slice.FromString("V111")); + tr.Set(Slice.FromString("K" + i + "B"), Slice.FromString("V222")); + tr.Set(Slice.FromString("K" + i + "C"), Slice.FromString("V333")); + tr.Set(Slice.FromString("K" + i + "D"), Slice.FromString("V444")); + tr.Set(Slice.FromString("K" + i + "E"), Slice.FromString("V555")); + tr.Set(Slice.FromString("K" + i + "F"), Slice.FromString("V666")); + tr.Set(Slice.FromString("K" + i + "G"), Slice.FromString("V777")); + tr.Set(Slice.FromString("K" + i + "H"), Slice.FromString("V888")); + } + tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(Slice.FromString("KZZZ"), Slice.FromString("V999")); + + var r = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0B")), + FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0G")) + ); + + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_14() + { + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); + tr.SetValues(Enumerable.Range(0, 100).Select(i => new KeyValuePair(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))))); + tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(Slice.FromString("K0042"), Slice.FromString("K0069")); + + var r = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0040")), + FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0080")), + new FdbRangeOptions { Mode = FdbStreamingMode.WantAll } + ); + // T 1 + // => GETRANGE( (< 'KAAA<00>' +1) .. (< LAST +1) + Log($"Count={r.Count}, HasMore={r.HasMore}"); + foreach (var kvp in r.Chunk) + { + Log($"{kvp.Key} = {kvp.Value}"); + } + } + + } + } + } + + [Test] + public async void Test_Case_15() + { + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); + tr.Set(Slice.FromString("KAAA"), Slice.FromString("V111")); + tr.Set(Slice.FromString("KBBB"), Slice.FromString("V222")); + tr.Set(Slice.FromString("KCCC"), Slice.FromString("V333")); + tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + // set a key, then read it, and check if it could conflict on it (it should not!) + tr.Set(Slice.FromString("KBBB"), Slice.FromString("V222b")); + await tr.GetAsync(Slice.FromString("KBBB")); + + // read a key, then set it, and check if it could conflict on it (it should!) + await tr.GetAsync(Slice.FromString("KCCC")); + tr.Set(Slice.FromString("KCCC"), Slice.FromString("V333b")); + + await tr.CommitAsync(); + } + + } + } + } + + [Test] + public async void Test_Case_16() + { + Fdb.Start(300); + + Slice aaa = Slice.FromString("KAAA"); + Slice bbb = Slice.FromString("KBBB"); + Slice ccc = Slice.FromString("KCCC"); + Slice hugeValue = Slice.FromString("BIGVALUE_" + new string('Z', 100)); + + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + + //using (var tr = db.BeginTransaction(this.Cancellation)) + //{ + // tr.ClearRange(Slice.FromString("K"), Slice.FromString("KZZZZZZZZZ")); + // await tr.CommitAsync(); + //} + //return; + + // set the key + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(aaa, Slice.FromString("VALUE_AAA")); + await tr.CommitAsync(); + } + // set the key + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetAsync(aaa); + } + + await Task.Delay(500); + + // first: concurrent trans, set only, no conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetReadVersionAsync(), tr2.GetReadVersionAsync()); + + tr1.Set(bbb, Slice.FromString("VALUE_BBB_111")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_111")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + await Task.WhenAll(task1, task2); + } + + await Task.Delay(500); + + // first: concurrent trans, read + set, no conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetAsync(aaa), tr2.GetAsync(aaa)); + + tr1.Set(bbb, Slice.FromString("VALUE_BBB_222")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_222")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + await Task.WhenAll(task1, task2); + } + + await Task.Delay(500); + + // first: concurrent trans, read + set, conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetAsync(ccc), tr2.GetAsync(bbb)); + tr1.Set(bbb, Slice.FromString("VALUE_BBB_333")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_333")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + try + { + await Task.WhenAll(task1, task2); + } + catch (Exception e) + { + Log(e.Message); + } + } + + Log("DONE!!!"); + } + } + } + + + [Test] + public async void Test_Case_17() + { + Fdb.Start(300); + using (var zedb = await Fdb.OpenAsync(this.Cancellation)) + { + + //THIS TEST MUST BE PERFORMED WITH THE CLUSTER DOWN! (net stop fdbmonitor) + + // measured latencies: + // "past_version": ALWAYS ~10 ms + // "future_version": ALWAYS ~10 ms + // "not_committed": start with 5, 10, 15, etc... but after 4 or 5, then transition into a random number between 0 and 1 sec + + using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.OnErrorAsync(FdbError.PastVersion).ConfigureAwait(false); + await tr.OnErrorAsync(FdbError.NotCommitted).ConfigureAwait(false); + } + + + using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) + { + for (int i = 0; i < 20; i++) + { + //tr.Timeout = 500; + //try + //{ + // await tr.GetAsync(Slice.FromAscii("SomeRandomKey")); + // Assert.Fail("The database must be offline !"); + //} + //catch(FdbException e) + { + var code = i > 1 && i < 10 ? FdbError.PastVersion : FdbError.CommitUnknownResult; + var sw = Stopwatch.StartNew(); + await tr.OnErrorAsync(code).ConfigureAwait(false); + sw.Stop(); + Log($"{sw.Elapsed.TotalMilliseconds:N3}"); + } + } + } + + + } + } + } } From 7cb43db53f49532e19131151e55db14b2d674dc3 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 10:44:02 +0200 Subject: [PATCH 049/153] Fix old test cases that required API 300 --- FoundationDB.Tests/TransactionFacts.cs | 27 ++------------------------ 1 file changed, 2 insertions(+), 25 deletions(-) diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index cd6fbc903..d22caccb2 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -2103,7 +2103,6 @@ public async void Test_Case_3() [Test] public async void Test_Case_4() { - Fdb.Start(300); using (var db = await Fdb.OpenAsync(this.Cancellation)) { { @@ -2137,7 +2136,6 @@ public async void Test_Case_4() [Test] public async void Test_Case_5() { - Fdb.Start(300); using (var db = await Fdb.OpenAsync(this.Cancellation)) { { @@ -2157,7 +2155,6 @@ public async void Test_Case_5() [Test] public async void Test_Case_6() { - Fdb.Start(300); using (var db = await Fdb.OpenAsync(this.Cancellation)) { { @@ -2183,7 +2180,6 @@ public async void Test_Case_6() [Test] public async void Test_Case_6b() { - Fdb.Start(300); using (var db = await Fdb.OpenAsync(this.Cancellation)) { Slice init = Slice.Repeat(0xCC, 9); @@ -2224,8 +2220,6 @@ public async void Test_Case_6b() [Test] public async void Test_Case_7() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2304,8 +2298,6 @@ public async void Test_Case_7() [Test] public async void Test_Case_8() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2339,8 +2331,6 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_9() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2381,8 +2371,6 @@ await tr.GetValuesAsync(new[] { [Test] public async void Test_Case_10() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2419,8 +2407,6 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_11() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2474,8 +2460,6 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_12() { - - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2510,7 +2494,6 @@ public async void Test_Case_12() [Test] public async void Test_Case_13() { - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2553,7 +2536,6 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_14() { - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2592,7 +2574,6 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_15() { - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); @@ -2628,12 +2609,10 @@ await db.WriteAsync((tr) => [Test] public async void Test_Case_16() { - Fdb.Start(300); - Slice aaa = Slice.FromString("KAAA"); Slice bbb = Slice.FromString("KBBB"); Slice ccc = Slice.FromString("KCCC"); - Slice hugeValue = Slice.FromString("BIGVALUE_" + new string('Z', 100)); + //Slice hugeValue = Slice.FromString("BIGVALUE_" + new string('Z', 100)); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { @@ -2719,13 +2698,11 @@ public async void Test_Case_16() } - [Test] + [Test][Ignore("This test requires the database to be stopped!")] public async void Test_Case_17() { - Fdb.Start(300); using (var zedb = await Fdb.OpenAsync(this.Cancellation)) { - //THIS TEST MUST BE PERFORMED WITH THE CLUSTER DOWN! (net stop fdbmonitor) // measured latencies: From 11f333bd0c0c7cf8a07a329936410f8e4d2db253 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 11:14:52 +0200 Subject: [PATCH 050/153] Target API v300 by default - The DefaultApiVersion is set to 300 (v3.x) to enable support for Atomic operations - Fdb.Start() with no arguments will target this version. --- FoundationDB.Client/Fdb.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 6c4ae7fe5..2cd256481 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -65,7 +65,7 @@ public static partial class Fdb internal const int MaxSafeApiVersion = FdbNative.FDB_API_MAX_VERSION; /// Default API version that will be selected, if the application does not specify otherwise. - internal const int DefaultApiVersion = 200; // v2.0.x + internal const int DefaultApiVersion = 300; // v3.0.x //INVARIANT: MinSafeApiVersion <= DefaultApiVersion <= MaxSafeApiVersion #endregion From c28c805faafea70374c2ddb8392880d4d65e6e75 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 11:16:34 +0200 Subject: [PATCH 051/153] Moved exotic test cases to their own file and disable them by default - These tests would clobber the content of the database, and were not meant to be executed all at once. --- FoundationDB.Tests/ExoticTestCases.cs | 754 +++++++++++++++++++ FoundationDB.Tests/FoundationDB.Tests.csproj | 1 + FoundationDB.Tests/TransactionFacts.cs | 695 ----------------- 3 files changed, 755 insertions(+), 695 deletions(-) create mode 100644 FoundationDB.Tests/ExoticTestCases.cs diff --git a/FoundationDB.Tests/ExoticTestCases.cs b/FoundationDB.Tests/ExoticTestCases.cs new file mode 100644 index 000000000..73d54be21 --- /dev/null +++ b/FoundationDB.Tests/ExoticTestCases.cs @@ -0,0 +1,754 @@ +#region BSD Licence +/* Copyright (c) 2013, Doxense SARL +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 + + +namespace FoundationDB.Client.Tests +{ + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Linq; + using System.Threading.Tasks; + using NUnit.Framework; + + [TestFixture][Ignore("These tests are not meant to be run as part of a CI build")] + public class ExoticTestCases : FdbTest + { + // This is a collection of specific test cases, used to trigger specific behaviors from the client + // => THEY ARE NOT TESTING THE DATABASE ITSELF, ONLY USED AS TOOLS TO OBSERVE THE CHANGES TO THE DATABASE! + + [Test] + public async void Test_Case_1() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + tr.Set(subspace.Keys.Encode("AAA"), Slice.FromString("111")); + tr.Set(subspace.Keys.Encode("BBB"), Slice.FromString("222")); + tr.Set(subspace.Keys.Encode("CCC"), Slice.FromString("333")); + tr.Set(subspace.Keys.Encode("DDD"), Slice.FromString("444")); + tr.Set(subspace.Keys.Encode("EEE"), Slice.FromString("555")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_2() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + tr.ClearRange(subspace.Keys.Encode("AAA"), Slice.FromString("ZZZ")); + tr.Set(subspace.Keys.Encode("AAA"), Slice.FromString("111")); + tr.Set(subspace.Keys.Encode("BBB"), Slice.FromString("222")); + tr.Set(subspace.Keys.Encode("CCC"), Slice.FromString("333")); + tr.Set(subspace.Keys.Encode("DDD"), Slice.FromString("444")); + tr.Set(subspace.Keys.Encode("EEE"), Slice.FromString("555")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_3() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + tr.ClearRange(subspace.Keys.Encode("AAA"), Slice.FromString("BBB")); + tr.ClearRange(subspace.Keys.Encode("BBB"), Slice.FromString("CCC")); + tr.ClearRange(subspace.Keys.Encode("CCC"), Slice.FromString("DDD")); + // should be merged into a single AAA..DDD + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_4() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + //initial setup: + // A: none + // B: 0 + // C: 255 + // D: none + // E: none + tr.Set(subspace.Keys.Encode("BBB"), Slice.FromFixed32(0)); + tr.Set(subspace.Keys.Encode("CCC"), Slice.FromFixed32(255)); + + // add 1 to everybody + tr.AtomicAdd(subspace.Keys.Encode("AAA"), Slice.FromFixed32(1)); + tr.AtomicAdd(subspace.Keys.Encode("BBB"), Slice.FromFixed32(1)); + tr.AtomicAdd(subspace.Keys.Encode("CCC"), Slice.FromFixed32(1)); + tr.AtomicAdd(subspace.Keys.Encode("DDD"), Slice.FromFixed32(1)); + tr.AtomicAdd(subspace.Keys.Encode("EEE"), Slice.FromFixed32(1)); + + // overwrite DDD with a fixed value + tr.Set(subspace.Keys.Encode("DDD"), Slice.FromFixed32(5)); + // double add on EEE + tr.AtomicAdd(subspace.Keys.Encode("EEE"), Slice.FromFixed32(1)); + + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_5() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + tr.Set(subspace.Keys.Encode("AAA"), Slice.FromString("111")); + tr.AtomicAdd(subspace.Keys.Encode("BBB"), Slice.FromString("222")); + tr.AtomicAnd(subspace.Keys.Encode("CCC"), Slice.FromString("333")); + tr.AtomicOr(subspace.Keys.Encode("DDD"), Slice.FromString("444")); + tr.AtomicXor(subspace.Keys.Encode("EEE"), Slice.FromString("555")); + tr.AtomicMax(subspace.Keys.Encode("FFF"), Slice.FromString("666")); + tr.AtomicMin(subspace.Keys.Encode("GGG"), Slice.FromString("777")); + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_6() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + { + var tr = db.BeginTransaction(this.Cancellation); + + tr.AtomicMax(subspace.Keys.Encode("MAXMAX1"), Slice.FromString("EEE")); + tr.AtomicMax(subspace.Keys.Encode("MAXMAX1"), Slice.FromString("FFF")); + + tr.AtomicMax(subspace.Keys.Encode("MAXMAX2"), Slice.FromString("FFF")); + tr.AtomicMax(subspace.Keys.Encode("MAXMAX2"), Slice.FromString("EEE")); + + tr.AtomicMin(subspace.Keys.Encode("MINMIN1"), Slice.FromString("111")); + tr.AtomicMin(subspace.Keys.Encode("MINMIN1"), Slice.FromString("222")); + + tr.AtomicMin(subspace.Keys.Encode("MINMIN2"), Slice.FromString("222")); + tr.AtomicMin(subspace.Keys.Encode("MINMIN2"), Slice.FromString("111")); + + await tr.CommitAsync(); + } + } + } + + [Test] + public async void Test_Case_6b() + { + using (var db = await OpenTestDatabaseAsync()) + { + var subspace = db.GlobalSpace; + + Slice init = Slice.Repeat(0xCC, 9); + Slice mask = Slice.Repeat(0xAA, 9); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(subspace.Keys.Encode("AAA"), init); + tr.Set(subspace.Keys.Encode("BBB"), init); + tr.Set(subspace.Keys.Encode("CCC"), init); + tr.Set(subspace.Keys.Encode("DDD"), init); + tr.Set(subspace.Keys.Encode("EEE"), init); + tr.Set(subspace.Keys.Encode("FFF"), init); + tr.Set(subspace.Keys.Encode("GGG"), init); + + await tr.CommitAsync(); + } + + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(subspace.Keys.Encode("AAA"), mask); + tr.AtomicAdd(subspace.Keys.Encode("BBB"), mask); + tr.AtomicAnd(subspace.Keys.Encode("CCC"), mask); + tr.AtomicOr(subspace.Keys.Encode("DDD"), mask); + tr.AtomicXor(subspace.Keys.Encode("EEE"), mask); + tr.AtomicMin(subspace.Keys.Encode("FFF"), mask); + tr.AtomicMax(subspace.Keys.Encode("GGG"), mask); + + await tr.CommitAsync(); + } + + await DumpSubspace(db, db.GlobalSpace); + + } + } + + [Test] + public async void Test_Case_7() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + + var vX = Slice.FromFixedU32BE(0x55555555); // X + var vY = Slice.FromFixedU32BE(0x66666666); // Y + var vL1 = Slice.FromFixedU32BE(0x11111111); // Low + var vL2 = Slice.FromFixedU32BE(0x22222222); // Low + var vH2 = Slice.FromFixedU32BE(0xFFFFFFFF); // Hi + var vH1 = Slice.FromFixedU32BE(0xEEEEEEEE); // Hi + var vA = Slice.FromFixedU32BE(0xAAAAAAAA); // 10101010 + var vC = Slice.FromFixedU32BE(0xCCCCCCCC); // 11001100 + + var cmds = new[] + { + new { Op = "SET", Left = vX, Right = vY }, + new { Op = "ADD", Left = vX, Right = vY }, + new { Op = "AND", Left = vA, Right = vC }, + new { Op = "OR", Left = vA, Right = vC }, + new { Op = "XOR", Left = vA, Right = vC }, + new { Op = "MIN", Left = vL1, Right = vL2 }, + new { Op = "MAX", Left = vH1, Right = vH2 }, + }; + + Action apply = (t, op, k, v) => + { + switch (op) + { + case "SET": + t.Set(k, v); + break; + case "ADD": + t.AtomicAdd(k, v); + break; + case "AND": + t.AtomicAnd(k, v); + break; + case "OR": + t.AtomicOr(k, v); + break; + case "XOR": + t.AtomicXor(k, v); + break; + case "MIN": + t.AtomicMin(k, v); + break; + case "MAX": + t.AtomicMax(k, v); + break; + default: + Assert.Fail(); + break; + } + }; + + for (int i = 0; i < cmds.Length; i++) + { + for (int j = 0; j < cmds.Length; j++) + { + Slice key = subspace.Keys.Encode(cmds[i].Op + "_" + cmds[j].Op); + Log("{0};{1} = {2}", i, j, key); + apply(tr, cmds[i].Op, key, cmds[i].Left); + apply(tr, cmds[j].Op, key, cmds[j].Right); + } + } + + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_8() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K9999\x00")); + for (int i = 0; i < 1000; i++) + { + tr.Set(subspace.Keys.Encode("K" + i.ToString("D4")), Slice.FromFixedU32BE((uint)i)); + } + }, this.Cancellation); + + for (int i = 0; i < 100; i++) + { + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + var res = await tr.GetAsync(subspace.Keys.Encode("K" + i.ToString("D4"))); + Console.WriteLine(res); + } + } + } + } + } + + [Test] + public async void Test_Case_9() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything + await db.WriteAsync((tr) => tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K9999Z")), this.Cancellation); + + await db.WriteAsync(tr => tr.Set(subspace.Keys.Encode("K0123"), Slice.FromString("V0123")), this.Cancellation); + await db.WriteAsync(tr => tr.Set(subspace.Keys.Encode("K0789"), Slice.FromString("V0789")), this.Cancellation); + + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetValuesAsync(new[] { + subspace.Keys.Encode("K0123"), + subspace.Keys.Encode("K0234"), + subspace.Keys.Encode("K0456"), + subspace.Keys.Encode("K0567"), + subspace.Keys.Encode("K0789") + }); + } + + // once more with feelings + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetValuesAsync(new[] { + subspace.Keys.Encode("K0123"), + subspace.Keys.Encode("K0234"), + subspace.Keys.Encode("K0456"), + subspace.Keys.Encode("K0567"), + subspace.Keys.Encode("K0789") + }); + } + } + } + } + + [Test] + public async void Test_Case_10() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K9999Z")); + for (int i = 0; i < 100; i++) + { + tr.Set(subspace.Keys.Encode("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); + } + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(subspace.Keys.Encode("K0010"), subspace.Keys.Encode("K0020")); + tr.ClearRange(subspace.Keys.Encode("K0050"), subspace.Keys.Encode("K0060")); + + var chunk = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000")), + FdbKeySelector.LastLessOrEqual(subspace.Keys.Encode("K9999")), + new FdbRangeOptions { Mode = FdbStreamingMode.WantAll, Reverse = true } + ); + + //no commit + } + + } + } + } + + [Test] + public async void Test_Case_11() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K9999Z")); + for (int i = 0; i < 100; i++) + { + tr.Set(subspace.Keys.Encode("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); + } + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(subspace.Keys.Encode("K0010"), subspace.Keys.Encode("K0020")); + tr.ClearRange(subspace.Keys.Encode("K0050"), subspace.Keys.Encode("K0060")); + tr.Set(subspace.Keys.Encode("K0021"), Slice.Empty); + tr.Set(subspace.Keys.Encode("K0042"), Slice.Empty); + + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0005"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0010"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0015"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0022"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0049"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0050"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0055"))); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0061"))); + + //no commit + } + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + //tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000"))); // equal=false, offset=1 + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(subspace.Keys.Encode("K0011"))); // equal=true, offset=1 + await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(subspace.Keys.Encode("K0022"))); // equal=true, offset=0 + await tr.GetKeyAsync(FdbKeySelector.LastLessThan(subspace.Keys.Encode("K0033"))); // equal=false, offset=0 + + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")) + 1000); // equal=false, offset=7 ? + await tr.GetKeyAsync(FdbKeySelector.LastLessThan(subspace.Keys.Encode("K0050")) + 1000); // equal=false, offset=6 ? + } + + } + } + } + + [Test] + public async void Test_Case_12() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + await tr.GetAsync(subspace.Keys.Encode("KGET")); + tr.AddReadConflictRange(subspace.Keys.Encode("KRC0"), subspace.Keys.Encode("KRC0")); + tr.AddWriteConflictRange(subspace.Keys.Encode("KWRITECONFLICT0"), subspace.Keys.Encode("KWRITECONFLICT1")); + tr.Set(subspace.Keys.Encode("KWRITE"), Slice.Empty); + await tr.CommitAsync(); + } + + // once more with feelings + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); + await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("KGETKEY"))); + } + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.AddReadConflictRange(subspace.Keys.Encode("KRC0"), subspace.Keys.Encode("KRC1")); + tr.Set(subspace.Keys.Encode("KWRITE"), Slice.Empty); + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_13() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K~~~~")); + tr.Set(subspace.Keys.Encode("K000"), Slice.FromString("BEGIN")); + for (int i = 0; i < 5; i++) + { + tr.Set(subspace.Keys.Encode("K" + i + "A"), Slice.FromString("V111")); + tr.Set(subspace.Keys.Encode("K" + i + "B"), Slice.FromString("V222")); + tr.Set(subspace.Keys.Encode("K" + i + "C"), Slice.FromString("V333")); + tr.Set(subspace.Keys.Encode("K" + i + "D"), Slice.FromString("V444")); + tr.Set(subspace.Keys.Encode("K" + i + "E"), Slice.FromString("V555")); + tr.Set(subspace.Keys.Encode("K" + i + "F"), Slice.FromString("V666")); + tr.Set(subspace.Keys.Encode("K" + i + "G"), Slice.FromString("V777")); + tr.Set(subspace.Keys.Encode("K" + i + "H"), Slice.FromString("V888")); + } + tr.Set(subspace.Keys.Encode("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(subspace.Keys.Encode("KZZZ"), Slice.FromString("V999")); + + var r = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0B")), + FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0G")) + ); + + await tr.CommitAsync(); + } + } + } + } + + [Test] + public async void Test_Case_14() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K~~~~")); + tr.SetValues(Enumerable.Range(0, 100).Select(i => new KeyValuePair(subspace.Keys.Encode("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))))); + tr.Set(subspace.Keys.Encode("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.ClearRange(subspace.Keys.Encode("K0042"), Slice.FromString("K0069")); + + var r = await tr.GetRangeAsync( + FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")), + FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0080")), + new FdbRangeOptions { Mode = FdbStreamingMode.WantAll } + ); + // T 1 + // => GETRANGE( (< 'KAAA<00>' +1) .. (< LAST +1) + Log($"Count={r.Count}, HasMore={r.HasMore}"); + foreach (var kvp in r.Chunk) + { + Log($"{kvp.Key} = {kvp.Value}"); + } + } + + } + } + } + + [Test] + public async void Test_Case_15() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + // clear everything and write some values + await db.WriteAsync((tr) => + { + tr.ClearRange(subspace.Keys.Encode("K0000"), subspace.Keys.Encode("K~~~~")); + tr.Set(subspace.Keys.Encode("KAAA"), Slice.FromString("V111")); + tr.Set(subspace.Keys.Encode("KBBB"), Slice.FromString("V222")); + tr.Set(subspace.Keys.Encode("KCCC"), Slice.FromString("V333")); + tr.Set(subspace.Keys.Encode("K~~~"), Slice.FromString("END")); + }, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + // set a key, then read it, and check if it could conflict on it (it should not!) + tr.Set(subspace.Keys.Encode("KBBB"), Slice.FromString("V222b")); + await tr.GetAsync(subspace.Keys.Encode("KBBB")); + + // read a key, then set it, and check if it could conflict on it (it should!) + await tr.GetAsync(subspace.Keys.Encode("KCCC")); + tr.Set(subspace.Keys.Encode("KCCC"), Slice.FromString("V333b")); + + await tr.CommitAsync(); + } + + } + } + } + + [Test] + public async void Test_Case_16() + { + + using (var zedb = await OpenTestDatabaseAsync()) + { + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + { + var subspace = db.GlobalSpace; + + Slice aaa = subspace.Keys.Encode("KAAA"); + Slice bbb = subspace.Keys.Encode("KBBB"); + Slice ccc = subspace.Keys.Encode("KCCC"); + + //using (var tr = db.BeginTransaction(this.Cancellation)) + //{ + // tr.ClearRange(subspace.Keys.Encode("K"), subspace.Keys.Encode("KZZZZZZZZZ")); + // await tr.CommitAsync(); + //} + //return; + + // set the key + using (var tr = db.BeginTransaction(this.Cancellation)) + { + tr.Set(aaa, Slice.FromString("VALUE_AAA")); + await tr.CommitAsync(); + } + // set the key + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.GetAsync(aaa); + } + + await Task.Delay(500); + + // first: concurrent trans, set only, no conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetReadVersionAsync(), tr2.GetReadVersionAsync()); + + tr1.Set(bbb, Slice.FromString("VALUE_BBB_111")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_111")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + await Task.WhenAll(task1, task2); + } + + await Task.Delay(500); + + // first: concurrent trans, read + set, no conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetAsync(aaa), tr2.GetAsync(aaa)); + + tr1.Set(bbb, Slice.FromString("VALUE_BBB_222")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_222")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + await Task.WhenAll(task1, task2); + } + + await Task.Delay(500); + + // first: concurrent trans, read + set, conflict + using (var tr1 = db.BeginTransaction(this.Cancellation)) + using (var tr2 = db.BeginTransaction(this.Cancellation)) + { + await Task.WhenAll(tr1.GetAsync(ccc), tr2.GetAsync(bbb)); + tr1.Set(bbb, Slice.FromString("VALUE_BBB_333")); + tr2.Set(ccc, Slice.FromString("VALUE_CCC_333")); + var task1 = tr1.CommitAsync(); + var task2 = tr2.CommitAsync(); + + try + { + await Task.WhenAll(task1, task2); + } + catch (Exception e) + { + Log(e.Message); + } + } + + Log("DONE!!!"); + } + } + } + + + [Test][Ignore("This test requires the database to be stopped!")] + public async void Test_Case_17() + { + using (var zedb = await OpenTestDatabaseAsync()) + { + //THIS TEST MUST BE PERFORMED WITH THE CLUSTER DOWN! (net stop fdbmonitor) + + // measured latencies: + // "past_version": ALWAYS ~10 ms + // "future_version": ALWAYS ~10 ms + // "not_committed": start with 5, 10, 15, etc... but after 4 or 5, then transition into a random number between 0 and 1 sec + + using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) + { + await tr.OnErrorAsync(FdbError.PastVersion).ConfigureAwait(false); + await tr.OnErrorAsync(FdbError.NotCommitted).ConfigureAwait(false); + } + + + using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) + { + for (int i = 0; i < 20; i++) + { + //tr.Timeout = 500; + //try + //{ + // await tr.GetAsync(Slice.FromAscii("SomeRandomKey")); + // Assert.Fail("The database must be offline !"); + //} + //catch(FdbException e) + { + var code = i > 1 && i < 10 ? FdbError.PastVersion : FdbError.CommitUnknownResult; + var sw = Stopwatch.StartNew(); + await tr.OnErrorAsync(code).ConfigureAwait(false); + sw.Stop(); + Log($"{sw.Elapsed.TotalMilliseconds:N3}"); + } + } + } + + + } + } + + } +} \ No newline at end of file diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index 5ed05da6a..de2513e18 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -75,6 +75,7 @@ Properties\VersionInfo.cs + diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index d22caccb2..8a7281c10 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -2048,700 +2048,5 @@ await db.WriteAsync((tr) => } } - - [Test] - public async void Test_Case_1() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); - tr.Set(Slice.FromString("BBB"), Slice.FromString("222")); - tr.Set(Slice.FromString("CCC"), Slice.FromString("333")); - tr.Set(Slice.FromString("DDD"), Slice.FromString("444")); - tr.Set(Slice.FromString("EEE"), Slice.FromString("555")); - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_2() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - tr.ClearRange(Slice.FromString("AAA"), Slice.FromString("ZZZ")); - tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); - tr.Set(Slice.FromString("BBB"), Slice.FromString("222")); - tr.Set(Slice.FromString("CCC"), Slice.FromString("333")); - tr.Set(Slice.FromString("DDD"), Slice.FromString("444")); - tr.Set(Slice.FromString("EEE"), Slice.FromString("555")); - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_3() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - tr.ClearRange(Slice.FromString("AAA"), Slice.FromString("BBB")); - tr.ClearRange(Slice.FromString("BBB"), Slice.FromString("CCC")); - tr.ClearRange(Slice.FromString("CCC"), Slice.FromString("DDD")); - // should be merged into a single AAA..DDD - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_4() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - //initial setup: - // A: none - // B: 0 - // C: 255 - // D: none - // E: none - tr.Set(Slice.FromString("BBB"), Slice.FromFixed32(0)); - tr.Set(Slice.FromString("CCC"), Slice.FromFixed32(255)); - - // add 1 to everybody - tr.AtomicAdd(Slice.FromString("AAA"), Slice.FromFixed32(1)); - tr.AtomicAdd(Slice.FromString("BBB"), Slice.FromFixed32(1)); - tr.AtomicAdd(Slice.FromString("CCC"), Slice.FromFixed32(1)); - tr.AtomicAdd(Slice.FromString("DDD"), Slice.FromFixed32(1)); - tr.AtomicAdd(Slice.FromString("EEE"), Slice.FromFixed32(1)); - - // overwrite DDD with a fixed value - tr.Set(Slice.FromString("DDD"), Slice.FromFixed32(5)); - // double add on EEE - tr.AtomicAdd(Slice.FromString("EEE"), Slice.FromFixed32(1)); - - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_5() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - tr.Set(Slice.FromString("AAA"), Slice.FromString("111")); - tr.AtomicAdd(Slice.FromString("BBB"), Slice.FromString("222")); - tr.AtomicAnd(Slice.FromString("CCC"), Slice.FromString("333")); - tr.AtomicOr(Slice.FromString("DDD"), Slice.FromString("444")); - tr.AtomicXor(Slice.FromString("EEE"), Slice.FromString("555")); - tr.AtomicMax(Slice.FromString("FFF"), Slice.FromString("666")); - tr.AtomicMin(Slice.FromString("GGG"), Slice.FromString("777")); - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_6() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - { - var tr = db.BeginTransaction(this.Cancellation); - - tr.AtomicMax(Slice.FromString("MAXMAX1"), Slice.FromString("EEE")); - tr.AtomicMax(Slice.FromString("MAXMAX1"), Slice.FromString("FFF")); - - tr.AtomicMax(Slice.FromString("MAXMAX2"), Slice.FromString("FFF")); - tr.AtomicMax(Slice.FromString("MAXMAX2"), Slice.FromString("EEE")); - - tr.AtomicMin(Slice.FromString("MINMIN1"), Slice.FromString("111")); - tr.AtomicMin(Slice.FromString("MINMIN1"), Slice.FromString("222")); - - tr.AtomicMin(Slice.FromString("MINMIN2"), Slice.FromString("222")); - tr.AtomicMin(Slice.FromString("MINMIN2"), Slice.FromString("111")); - - await tr.CommitAsync(); - } - } - } - - [Test] - public async void Test_Case_6b() - { - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - Slice init = Slice.Repeat(0xCC, 9); - Slice mask = Slice.Repeat(0xAA, 9); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("AAA"), init); - tr.Set(Slice.FromString("BBB"), init); - tr.Set(Slice.FromString("CCC"), init); - tr.Set(Slice.FromString("DDD"), init); - tr.Set(Slice.FromString("EEE"), init); - tr.Set(Slice.FromString("FFF"), init); - tr.Set(Slice.FromString("GGG"), init); - - await tr.CommitAsync(); - } - - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("AAA"), mask); - tr.AtomicAdd(Slice.FromString("BBB"), mask); - tr.AtomicAnd(Slice.FromString("CCC"), mask); - tr.AtomicOr(Slice.FromString("DDD"), mask); - tr.AtomicXor(Slice.FromString("EEE"), mask); - tr.AtomicMin(Slice.FromString("FFF"), mask); - tr.AtomicMax(Slice.FromString("GGG"), mask); - - await tr.CommitAsync(); - } - - await DumpSubspace(db, db.GlobalSpace); - - } - } - - [Test] - public async void Test_Case_7() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var vX = Slice.FromFixedU32BE(0x55555555); // X - var vY = Slice.FromFixedU32BE(0x66666666); // Y - var vL1 = Slice.FromFixedU32BE(0x11111111); // Low - var vL2 = Slice.FromFixedU32BE(0x22222222); // Low - var vH2 = Slice.FromFixedU32BE(0xFFFFFFFF); // Hi - var vH1 = Slice.FromFixedU32BE(0xEEEEEEEE); // Hi - var vA = Slice.FromFixedU32BE(0xAAAAAAAA); // 10101010 - var vC = Slice.FromFixedU32BE(0xCCCCCCCC); // 11001100 - - var cmds = new[] - { - new { Op = "SET", Left = vX, Right = vY }, - new { Op = "ADD", Left = vX, Right = vY }, - new { Op = "AND", Left = vA, Right = vC }, - new { Op = "OR", Left = vA, Right = vC }, - new { Op = "XOR", Left = vA, Right = vC }, - new { Op = "MIN", Left = vL1, Right = vL2 }, - new { Op = "MAX", Left = vH1, Right = vH2 }, - }; - - Action apply = (t, op, k, v) => - { - switch (op) - { - case "SET": - t.Set(k, v); - break; - case "ADD": - t.AtomicAdd(k, v); - break; - case "AND": - t.AtomicAnd(k, v); - break; - case "OR": - t.AtomicOr(k, v); - break; - case "XOR": - t.AtomicXor(k, v); - break; - case "MIN": - t.AtomicMin(k, v); - break; - case "MAX": - t.AtomicMax(k, v); - break; - default: - Assert.Fail(); - break; - } - }; - - for (int i = 0; i < cmds.Length; i++) - { - for (int j = 0; j < cmds.Length; j++) - { - Slice key = Slice.FromString(cmds[i].Op + "_" + cmds[j].Op); - Log("{0};{1} = {2}", i, j, key); - apply(tr, cmds[i].Op, key, cmds[i].Left); - apply(tr, cmds[j].Op, key, cmds[j].Right); - } - } - - await tr.CommitAsync(); - } - } - } - } - - [Test] - public async void Test_Case_8() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - -#if false // RUNONCE - - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999\x00")); - for (int i = 0; i < 1000; i++) - { - tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromFixedU32BE((uint)i)); - } - }, this.Cancellation); -#endif - - - for (int i = 0; i < 100; i++) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tr.GetAsync(Slice.FromString("K" + i.ToString("D4"))); - Console.WriteLine(res); - } - } - } - } - } - - [Test] - public async void Test_Case_9() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything - await db.WriteAsync((tr) => tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")), this.Cancellation); - - await db.WriteAsync(tr => tr.Set(Slice.FromString("K0123"), Slice.FromString("V0123")), this.Cancellation); - await db.WriteAsync(tr => tr.Set(Slice.FromString("K0789"), Slice.FromString("V0789")), this.Cancellation); - - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(new[] { - Slice.FromString("K0123"), - Slice.FromString("K0234"), - Slice.FromString("K0456"), - Slice.FromString("K0567"), - Slice.FromString("K0789") - }); - } - - // once more with feelings - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(new[] { - Slice.FromString("K0123"), - Slice.FromString("K0234"), - Slice.FromString("K0456"), - Slice.FromString("K0567"), - Slice.FromString("K0789") - }); - } - } - } - } - - [Test] - public async void Test_Case_10() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything and write some values - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")); - for (int i = 0; i < 100; i++) - { - tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); - } - }, this.Cancellation); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.ClearRange(Slice.FromString("K0010"), Slice.FromString("K0020")); - tr.ClearRange(Slice.FromString("K0050"), Slice.FromString("K0060")); - - var chunk = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0000")), - FdbKeySelector.LastLessOrEqual(Slice.FromString("K9999")), - new FdbRangeOptions { Mode = FdbStreamingMode.WantAll, Reverse = true } - ); - - //no commit - } - - } - } - } - - [Test] - public async void Test_Case_11() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything and write some values - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K9999Z")); - for (int i = 0; i < 100; i++) - { - tr.Set(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))); - } - }, this.Cancellation); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.ClearRange(Slice.FromString("K0010"), Slice.FromString("K0020")); - tr.ClearRange(Slice.FromString("K0050"), Slice.FromString("K0060")); - tr.Set(Slice.FromString("K0021"), Slice.Empty); - tr.Set(Slice.FromString("K0042"), Slice.Empty); - - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0005"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0010"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0015"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0022"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0049"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0050"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0055"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0061"))); - - //no commit - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - //tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0000"))); // equal=false, offset=1 - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(Slice.FromString("K0011"))); // equal=true, offset=1 - await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(Slice.FromString("K0022"))); // equal=true, offset=0 - await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromString("K0033"))); // equal=false, offset=0 - - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0040")) + 1000); // equal=false, offset=7 ? - await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromString("K0050")) + 1000); // equal=false, offset=6 ? - } - - } - } - } - - [Test] - public async void Test_Case_12() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - await tr.GetAsync(Slice.FromString("KGET")); - tr.AddReadConflictRange(Slice.FromString("KRC0"), Slice.FromString("KRC0")); - tr.AddWriteConflictRange(Slice.FromString("KWRITECONFLICT0"), Slice.FromString("KWRITECONFLICT1")); - tr.Set(Slice.FromString("KWRITE"), Slice.Empty); - await tr.CommitAsync(); - } - - // once more with feelings - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("KGETKEY"))); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.AddReadConflictRange(Slice.FromString("KRC0"), Slice.FromString("KRC1")); - tr.Set(Slice.FromString("KWRITE"), Slice.Empty); - await tr.CommitAsync(); - } - } - } - } - - [Test] - public async void Test_Case_13() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything and write some values - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); - tr.Set(Slice.FromString("K000"), Slice.FromString("BEGIN")); - for (int i = 0; i < 5; i++) - { - tr.Set(Slice.FromString("K" + i + "A"), Slice.FromString("V111")); - tr.Set(Slice.FromString("K" + i + "B"), Slice.FromString("V222")); - tr.Set(Slice.FromString("K" + i + "C"), Slice.FromString("V333")); - tr.Set(Slice.FromString("K" + i + "D"), Slice.FromString("V444")); - tr.Set(Slice.FromString("K" + i + "E"), Slice.FromString("V555")); - tr.Set(Slice.FromString("K" + i + "F"), Slice.FromString("V666")); - tr.Set(Slice.FromString("K" + i + "G"), Slice.FromString("V777")); - tr.Set(Slice.FromString("K" + i + "H"), Slice.FromString("V888")); - } - tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); - }, this.Cancellation); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("KZZZ"), Slice.FromString("V999")); - - var r = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0B")), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0G")) - ); - - await tr.CommitAsync(); - } - } - } - } - - [Test] - public async void Test_Case_14() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything and write some values - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); - tr.SetValues(Enumerable.Range(0, 100).Select(i => new KeyValuePair(Slice.FromString("K" + i.ToString("D4")), Slice.FromString("V" + i.ToString("D4"))))); - tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); - }, this.Cancellation); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.ClearRange(Slice.FromString("K0042"), Slice.FromString("K0069")); - - var r = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0040")), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString("K0080")), - new FdbRangeOptions { Mode = FdbStreamingMode.WantAll } - ); - // T 1 - // => GETRANGE( (< 'KAAA<00>' +1) .. (< LAST +1) - Log($"Count={r.Count}, HasMore={r.HasMore}"); - foreach (var kvp in r.Chunk) - { - Log($"{kvp.Key} = {kvp.Value}"); - } - } - - } - } - } - - [Test] - public async void Test_Case_15() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - // clear everything and write some values - await db.WriteAsync((tr) => - { - tr.ClearRange(Slice.FromString("K0000"), Slice.FromString("K~~~~")); - tr.Set(Slice.FromString("KAAA"), Slice.FromString("V111")); - tr.Set(Slice.FromString("KBBB"), Slice.FromString("V222")); - tr.Set(Slice.FromString("KCCC"), Slice.FromString("V333")); - tr.Set(Slice.FromString("K~~~"), Slice.FromString("END")); - }, this.Cancellation); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - // set a key, then read it, and check if it could conflict on it (it should not!) - tr.Set(Slice.FromString("KBBB"), Slice.FromString("V222b")); - await tr.GetAsync(Slice.FromString("KBBB")); - - // read a key, then set it, and check if it could conflict on it (it should!) - await tr.GetAsync(Slice.FromString("KCCC")); - tr.Set(Slice.FromString("KCCC"), Slice.FromString("V333b")); - - await tr.CommitAsync(); - } - - } - } - } - - [Test] - public async void Test_Case_16() - { - Slice aaa = Slice.FromString("KAAA"); - Slice bbb = Slice.FromString("KBBB"); - Slice ccc = Slice.FromString("KCCC"); - //Slice hugeValue = Slice.FromString("BIGVALUE_" + new string('Z', 100)); - - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); - { - - //using (var tr = db.BeginTransaction(this.Cancellation)) - //{ - // tr.ClearRange(Slice.FromString("K"), Slice.FromString("KZZZZZZZZZ")); - // await tr.CommitAsync(); - //} - //return; - - // set the key - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(aaa, Slice.FromString("VALUE_AAA")); - await tr.CommitAsync(); - } - // set the key - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetAsync(aaa); - } - - await Task.Delay(500); - - // first: concurrent trans, set only, no conflict - using (var tr1 = db.BeginTransaction(this.Cancellation)) - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - await Task.WhenAll(tr1.GetReadVersionAsync(), tr2.GetReadVersionAsync()); - - tr1.Set(bbb, Slice.FromString("VALUE_BBB_111")); - tr2.Set(ccc, Slice.FromString("VALUE_CCC_111")); - var task1 = tr1.CommitAsync(); - var task2 = tr2.CommitAsync(); - - await Task.WhenAll(task1, task2); - } - - await Task.Delay(500); - - // first: concurrent trans, read + set, no conflict - using (var tr1 = db.BeginTransaction(this.Cancellation)) - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - await Task.WhenAll(tr1.GetAsync(aaa), tr2.GetAsync(aaa)); - - tr1.Set(bbb, Slice.FromString("VALUE_BBB_222")); - tr2.Set(ccc, Slice.FromString("VALUE_CCC_222")); - var task1 = tr1.CommitAsync(); - var task2 = tr2.CommitAsync(); - - await Task.WhenAll(task1, task2); - } - - await Task.Delay(500); - - // first: concurrent trans, read + set, conflict - using (var tr1 = db.BeginTransaction(this.Cancellation)) - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - await Task.WhenAll(tr1.GetAsync(ccc), tr2.GetAsync(bbb)); - tr1.Set(bbb, Slice.FromString("VALUE_BBB_333")); - tr2.Set(ccc, Slice.FromString("VALUE_CCC_333")); - var task1 = tr1.CommitAsync(); - var task2 = tr2.CommitAsync(); - - try - { - await Task.WhenAll(task1, task2); - } - catch (Exception e) - { - Log(e.Message); - } - } - - Log("DONE!!!"); - } - } - } - - - [Test][Ignore("This test requires the database to be stopped!")] - public async void Test_Case_17() - { - using (var zedb = await Fdb.OpenAsync(this.Cancellation)) - { - //THIS TEST MUST BE PERFORMED WITH THE CLUSTER DOWN! (net stop fdbmonitor) - - // measured latencies: - // "past_version": ALWAYS ~10 ms - // "future_version": ALWAYS ~10 ms - // "not_committed": start with 5, 10, 15, etc... but after 4 or 5, then transition into a random number between 0 and 1 sec - - using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.OnErrorAsync(FdbError.PastVersion).ConfigureAwait(false); - await tr.OnErrorAsync(FdbError.NotCommitted).ConfigureAwait(false); - } - - - using (var tr = zedb.BeginReadOnlyTransaction(this.Cancellation)) - { - for (int i = 0; i < 20; i++) - { - //tr.Timeout = 500; - //try - //{ - // await tr.GetAsync(Slice.FromAscii("SomeRandomKey")); - // Assert.Fail("The database must be offline !"); - //} - //catch(FdbException e) - { - var code = i > 1 && i < 10 ? FdbError.PastVersion : FdbError.CommitUnknownResult; - var sw = Stopwatch.StartNew(); - await tr.OnErrorAsync(code).ConfigureAwait(false); - sw.Stop(); - Log($"{sw.Elapsed.TotalMilliseconds:N3}"); - } - } - } - - - } - } - } - } From 1961c3ccb22a46efa37b8420dc2ef944e13a884b Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 11:16:49 +0200 Subject: [PATCH 052/153] Update R# settings --- .../FoundationDB.Storage.Memory.Test.csproj | 3 +++ FoundationDb.Client.sln.DotSettings | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj b/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj index 7a277a40e..700832ec1 100644 --- a/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj +++ b/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj @@ -85,6 +85,9 @@ FoundationDB.Storage.Memory + + + - - - - - - diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedDictionaryFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedDictionaryFacts.cs deleted file mode 100644 index be9e4989e..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedDictionaryFacts.cs +++ /dev/null @@ -1,260 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core.Test -{ - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Threading; - - [TestFixture] - public class ColaOrderedDictionaryFacts - { - - [Test] - public void Test_Empty_Dictionary() - { - var cola = new ColaOrderedDictionary(42, Comparer.Default, StringComparer.Ordinal); - Assert.That(cola.Count, Is.EqualTo(0)); - Assert.That(cola.KeyComparer, Is.SameAs(Comparer.Default)); - Assert.That(cola.ValueComparer, Is.SameAs(StringComparer.Ordinal)); - Assert.That(cola.Capacity, Is.EqualTo(63), "Capacity should be the next power of 2, minus 1"); - } - - [Test] - public void Test_ColaOrderedDictionary_Add() - { - var cmp = new CountingComparer(); - - var cola = new ColaOrderedDictionary(cmp); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Add(42, "42"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.ContainsKey(42), Is.True); - - cola.Add(1, "1"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - Assert.That(cola.ContainsKey(1), Is.True); - - cola.Add(66, "66"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(3)); - Assert.That(cola.ContainsKey(66), Is.True); - - cola.Add(123, "123"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(4)); - Assert.That(cola.ContainsKey(123), Is.True); - - for(int i = 1; i < 100; i++) - { - cola.Add(-i, (-i).ToString()); - } - cola.Debug_Dump(); - - - cmp.Reset(); - cola.ContainsKey(-99); - Console.WriteLine("Lookup last inserted: " + cmp.Count); - - cmp.Reset(); - cola.ContainsKey(42); - Console.WriteLine("Lookup first inserted: " + cmp.Count); - - cmp.Reset(); - cola.ContainsKey(77); - Console.WriteLine("Lookup not found: " + cmp.Count); - - var keys = new List(); - - foreach(var kvp in cola) - { - Assert.That(kvp.Value, Is.EqualTo(kvp.Key.ToString())); - keys.Add(kvp.Key); - } - - Assert.That(keys.Count, Is.EqualTo(cola.Count)); - Assert.That(keys, Is.Ordered); - Console.WriteLine(String.Join(", ", keys)); - - } - - [Test] - public void Test_ColaOrderedDictionary_Remove() - { - const int N = 100; - - // add a bunch of random values - var rnd = new Random(); - int seed = 1333019583;// rnd.Next(); - Console.WriteLine("Seed " + seed); - rnd = new Random(seed); - - var cola = new ColaOrderedDictionary(); - var list = new List(); - - int x = 0; - for (int i = 0; i < N; i++) - { - x += (1 + rnd.Next(10)); - string s = "value of " + x; - - cola.Add(x, s); - list.Add(x); - } - Assert.That(cola.Count, Is.EqualTo(N)); - - foreach(var item in list) - { - Assert.That(cola.ContainsKey(item), "{0} is missing", item); - } - - cola.Debug_Dump(); - - // now start removing items one by one - while(list.Count > 0) - { - int p = rnd.Next(list.Count); - x = list[p]; - list.RemoveAt(p); - - bool res = cola.Remove(x); - if (!res) cola.Debug_Dump(); - Assert.That(res, Is.True, "Remove({0}) failed", x); - - Assert.That(cola.Count, Is.EqualTo(list.Count), "After removing {0}", x); - } - - cola.Debug_Dump(); - - } - - [Test] - [Category("LongRunning")] - public void Test_MiniBench() - { - const int N = 10 * 1000 * 1000; - - var rnd = new Random(); - long x; - - - //WARMUP - var store = new ColaOrderedDictionary(); - store.Add(1, 1); - store.Add(42, 42); - store.Add(1234, 1234); - store.TryGetValue(42, out x); - store.TryGetValue(404, out x); - - #region Sequentially inserted.... - - Console.WriteLine("Inserting " + N.ToString("N0") + " sequential key/value pairs into a COLA ordered dictionary"); - GC.Collect(); - store = new ColaOrderedDictionary(); - long total = 0; - var sw = Stopwatch.StartNew(); - for (int i = 0; i < N; i++) - { - store.SetItem(i, i); - Interlocked.Increment(ref total); - if (i % (N / 10) == 0) Console.Write("."); - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS: " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(i, out x); - if (!_ || x != i) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("SeqReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(rnd.Next(N), out x); - if (!_) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("RndReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - #region Randomly inserted.... - - Console.WriteLine("(preparing random insert list)"); - - var tmp = new long[N]; - var values = new long[N]; - for (int i = 0; i < N; i++) - { - tmp[i] = rnd.Next(N); - values[i] = i; - } - Array.Sort(tmp, values); - - Console.WriteLine("Inserting " + N.ToString("N0") + " sequential keys into a COLA store"); - GC.Collect(); - store = new ColaOrderedDictionary(); - total = 0; - sw.Restart(); - for (int i = 0; i < N; i++) - { - store.Add(values[i], i); - Interlocked.Increment(ref total); - if (i % (N / 10) == 0) Console.Write("."); - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS : " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(i, out x); - if (!_) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("SeqReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(rnd.Next(N), out x); - if (!_) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("RndReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedSetFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedSetFacts.cs deleted file mode 100644 index dd9208a97..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaOrderedSetFacts.cs +++ /dev/null @@ -1,345 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core.Test -{ - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Globalization; - using System.Linq; - using System.Threading; - - [TestFixture] - public class ColaOrderedSetFacts - { - - [Test] - public void Test_Empty_ColaSet() - { - var cola = new ColaOrderedSet(42, StringComparer.Ordinal); - Assert.That(cola.Count, Is.EqualTo(0)); - Assert.That(cola.Comparer, Is.SameAs(StringComparer.Ordinal)); - Assert.That(cola.Capacity, Is.EqualTo(63), "Capacity should be the next power of 2, minus 1"); - } - - [Test] - public void Test_Capacity_Is_Rounded_Up() - { - // default capacity is 4 levels, for 31 items max - var cola = new ColaOrderedSet(); - Assert.That(cola.Capacity, Is.EqualTo(31)); - - // 63 items completely fill 5 levels - cola = new ColaOrderedSet(63); - Assert.That(cola.Capacity, Is.EqualTo(63)); - - // 64 items need 6 levels, which can hold up to 127 items - cola = new ColaOrderedSet(64); - Assert.That(cola.Capacity, Is.EqualTo(127)); - } - - [Test] - public void Test_ColaOrderedSet_Add() - { - var cola = new ColaOrderedSet(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Add(42); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.Contains(42), Is.True); - - cola.Add(1); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - Assert.That(cola.Contains(1), Is.True); - - cola.Add(66); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(3)); - Assert.That(cola.Contains(66), Is.True); - - cola.Add(123); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(4)); - Assert.That(cola.Contains(123), Is.True); - - cola.Add(-77); - cola.Add(-76); - cola.Add(-75); - cola.Add(-74); - cola.Add(-73); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(9)); - } - - [Test] - public void Test_ColaOrderedSet_Remove() - { - const int N = 1000; - - var cola = new ColaOrderedSet(); - var list = new List(); - - for (int i = 0; i < N;i++) - { - cola.Add(i); - list.Add(i); - } - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(N)); - - for (int i = 0; i < N; i++) - { - Assert.That(cola.Contains(list[i]), Is.True, "{0} is missing (offset {1})", list[i], i); - } - - var rnd = new Random(); - int seed = 1073704892; // rnd.Next(); - Console.WriteLine("Seed: " + seed); - rnd = new Random(seed); - int old = -1; - while (list.Count > 0) - { - int p = rnd.Next(list.Count); - int x = list[p]; - if (!cola.Contains(x)) - { - Assert.Fail("{0} disapeared after removing {1} ?", x, old); - } - - bool res = cola.Remove(x); - Assert.That(res, Is.True, "Removing {0} did nothing", x); - //Assert.That(cola.Contains(191), "blah {0}", x); - - list.RemoveAt(p); - Assert.That(cola.Count, Is.EqualTo(list.Count)); - old = x; - } - cola.Debug_Dump(); - - } - - [Test] - public void Test_CopyTo_Return_Ordered_Values() - { - const int N = 1000; - var rnd = new Random(); - - var cola = new ColaOrderedSet(); - - // create a list of random values - var numbers = new int[N]; - for (int i = 0, x = 0; i < N; i++) numbers[i] = (x += 1 + rnd.Next(10)); - - // insert the list in a random order - var list = new List(numbers); - while(list.Count > 0) - { - int p = rnd.Next(list.Count); - cola.Add(list[p]); - list.RemoveAt(p); - } - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(N)); - - // we can now sort the numbers to get a reference sequence - Array.Sort(numbers); - - // foreach()ing should return the value in natural order - list.Clear(); - foreach (var x in cola) list.Add(x); - Assert.That(list.Count, Is.EqualTo(N)); - Assert.That(list, Is.EqualTo(numbers)); - - // CopyTo() should produce the item in the expected order - var tmp = new int[N]; - cola.CopyTo(tmp); - Assert.That(tmp, Is.EqualTo(numbers)); - - // ToArray() should do the same thing - tmp = cola.ToArray(); - Assert.That(tmp, Is.EqualTo(numbers)); - - } - - [Test] - public void Test_Check_Costs() - { - const int N = 100; - var cmp = new CountingComparer(Comparer.Default); - var cola = new ColaOrderedSet(cmp); - - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, "Parameters: N = {0}, Log(N) = {1}, Log2(N) = {2}, N.Log2(N) = {3}", N, Math.Log(N), Math.Log(N, 2), N * Math.Log(N, 2))); - - Console.WriteLine("Inserting (" + N + " items)"); - for (int i = 0; i < N; i++) - { - cola.Add(FdbTuple.EncodeKey(i << 1)); - } - - Console.WriteLine("> " + cmp.Count + " cmps (" + ((double)cmp.Count / N) + " / insert)"); - cola.Debug_Dump(); - - Console.WriteLine("Full scan (" + (N << 1) + " lookups)"); - cmp.Reset(); - int n = 0; - for (int i = 0; i < (N << 1); i++) - { - if (cola.Contains(FdbTuple.EncodeKey(i))) ++n; - } - Assert.That(n, Is.EqualTo(N)); - Console.WriteLine("> " + cmp.Count + " cmps (" + ((double)cmp.Count / (N << 1)) + " / lookup)"); - - cmp.Reset(); - n = 0; - int tail = Math.Min(16, N >> 1); - int offset = N - tail; - Console.WriteLine("Tail scan (" + tail + " lookups)"); - for (int i = 0; i < tail; i++) - { - if (cola.Contains(FdbTuple.EncodeKey(offset + i))) ++n; - } - Console.WriteLine("> " + cmp.Count + " cmps (" + ((double)cmp.Count / tail) + " / lookup)"); - - Console.WriteLine("ForEach"); - cmp.Reset(); - int p = 0; - foreach(var x in cola) - { - Assert.That(FdbTuple.DecodeKey(x), Is.EqualTo(p << 1)); - ++p; - } - Assert.That(p, Is.EqualTo(N)); - Console.WriteLine("> " + cmp.Count + " cmps (" + ((double)cmp.Count / N) + " / item)"); - } - - [Test] - [Category("LongRunning")] - public void Test_MiniBench() - { - const int N = 10 * 1000 * 1000; - - var rnd = new Random(); - long x; - - - //WARMUP - var store = new ColaOrderedSet(); - store.Add(1); - store.Add(42); - store.Add(1234); - store.TryGetValue(42, out x); - store.TryGetValue(404, out x); - - #region Sequentially inserted.... - - Console.WriteLine("Inserting " + N.ToString("N0") + " sequential key/value pairs into a COLA ordered set"); - GC.Collect(); - store = new ColaOrderedSet(); - long total = 0; - var sw = Stopwatch.StartNew(); - for (int i = 0; i < N; i++) - { - store.Add(i); - Interlocked.Increment(ref total); - if (i % (N / 10) == 0) Console.Write("."); - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS: " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(i, out x); - if (!_ || x != i) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("SeqReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(rnd.Next(N), out x); - if (!_) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("RndReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - #region Randomly inserted.... - - Console.WriteLine("(preparing random insert list)"); - - var tmp = new long[N]; - var values = new long[N]; - for (int i = 0; i < N; i++) - { - tmp[i] = rnd.Next(N); - values[i] = i; - } - Array.Sort(tmp, values); - - Console.WriteLine("Inserting " + N.ToString("N0") + " sequential keys into a COLA store"); - GC.Collect(); - store = new ColaOrderedSet(); - total = 0; - sw.Restart(); - for (int i = 0; i < N; i++) - { - store.Add(values[i]); - Interlocked.Increment(ref total); - if (i % (N / 10) == 0) Console.Write("."); - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS : " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - - - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(i, out x); - if (!_ || x != i) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("SeqReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - var _ = store.TryGetValue(rnd.Next(N), out x); - if (!_) Assert.Fail(); - } - sw.Stop(); - Console.WriteLine("RndReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs deleted file mode 100644 index 0783a58d1..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeDictionaryFacts.cs +++ /dev/null @@ -1,737 +0,0 @@ -using FoundationDB.Client; -using FoundationDB.Layers.Tuples; -using NUnit.Framework; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - -namespace FoundationDB.Storage.Memory.Core.Test -{ - [TestFixture] - public class ColaRangeDictionaryFacts - { - - [Test] - public void Test_Empty_RangeDictionary() - { - var cola = new ColaRangeDictionary(0, Comparer.Default, StringComparer.Ordinal); - Assert.That(cola.Count, Is.EqualTo(0)); - Assert.That(cola.KeyComparer, Is.SameAs(Comparer.Default)); - Assert.That(cola.ValueComparer, Is.SameAs(StringComparer.Ordinal)); - Assert.That(cola.Capacity, Is.EqualTo(15), "Capacity should be the next power of 2, minus 1"); - Assert.That(cola.Bounds, Is.Not.Null); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(0)); - } - - [Test] - public void Test_RangeDictionary_Insert_Single() - { - var cola = new ColaRangeDictionary(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1, "A"); - Assert.That(cola.Count, Is.EqualTo(1)); - - var items = cola.ToArray(); - Assert.That(items.Length, Is.EqualTo(1)); - Assert.That(items[0].Begin, Is.EqualTo(0)); - Assert.That(items[0].End, Is.EqualTo(1)); - Assert.That(items[0].Value, Is.EqualTo("A")); - } - - [Test] - public void Test_RangeDictionary_Insert_In_Order_Non_Overlapping() - { - var cola = new ColaRangeDictionary(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1, "A"); - Console.WriteLine("FIRST = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(2, 3, "B"); - Console.WriteLine("SECOND = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(4, 5, "C"); - Console.WriteLine("THIRD = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - - Assert.That(cola.Count, Is.EqualTo(3)); - var runs = cola.ToArray(); - Assert.That(runs.Length, Is.EqualTo(3)); - - Assert.That(runs[0].Begin, Is.EqualTo(0)); - Assert.That(runs[0].End, Is.EqualTo(1)); - Assert.That(runs[0].Value, Is.EqualTo("A")); - - Assert.That(runs[1].Begin, Is.EqualTo(2)); - Assert.That(runs[1].End, Is.EqualTo(3)); - Assert.That(runs[1].Value, Is.EqualTo("B")); - - Assert.That(runs[2].Begin, Is.EqualTo(4)); - Assert.That(runs[2].End, Is.EqualTo(5)); - Assert.That(runs[2].Value, Is.EqualTo("C")); - - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - } - - [Test] - public void Test_RangeDictionary_Insert_Out_Of_Order_Non_Overlapping() - { - var cola = new ColaRangeDictionary(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1, "A"); - Console.WriteLine("FIRST = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(4, 5, "B"); - Console.WriteLine("SECOND = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(2, 3, "C"); - Console.WriteLine("THIRD = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - - Assert.That(cola.Count, Is.EqualTo(3)); - var runs = cola.ToArray(); - Assert.That(runs.Length, Is.EqualTo(3)); - - Assert.That(runs[0].Begin, Is.EqualTo(0)); - Assert.That(runs[0].End, Is.EqualTo(1)); - Assert.That(runs[0].Value, Is.EqualTo("A")); - - Assert.That(runs[1].Begin, Is.EqualTo(2)); - Assert.That(runs[1].End, Is.EqualTo(3)); - Assert.That(runs[1].Value, Is.EqualTo("C")); - - Assert.That(runs[2].Begin, Is.EqualTo(4)); - Assert.That(runs[2].End, Is.EqualTo(5)); - Assert.That(runs[2].Value, Is.EqualTo("B")); - - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - - } - [Test] - public void Test_RangeDictionary_Insert_Partially_Overlapping() - { - var cola = new ColaRangeDictionary(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1, "A"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(0, 2, "B"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(1, 3, "C"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(-1, 2, "D"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeDictionary_Insert_Completly_Overlapping() - { - var cola = new ColaRangeDictionary(); - cola.Mark(4, 5, "A"); - Console.WriteLine("BEFORE = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(4)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - - // overlaps all the ranges at once - // 0123456789 0123456789 0123456789 - // ____A_____ + BBBBBBBBBB = BBBBBBBBBB - cola.Mark(0, 10, "B"); - Console.WriteLine("AFTER = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - - Assert.That(cola.Count, Is.EqualTo(1)); - var runs = cola.ToArray(); - Assert.That(runs.Length, Is.EqualTo(1)); - Assert.That(runs[0].Begin, Is.EqualTo(0)); - Assert.That(runs[0].End, Is.EqualTo(10)); - Assert.That(runs[0].Value, Is.EqualTo("B")); - - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(10)); - } - - [Test] - public void Test_RangeDictionary_Insert_Contained() - { - var cola = new ColaRangeDictionary(); - cola.Mark(0, 10, "A"); - Console.WriteLine("BEFORE = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(10)); - - // overlaps all the ranges at once - - // 0123456789 0123456789 0123456789 - // AAAAAAAAAA + ____B_____ = AAAABAAAAA - cola.Mark(4, 5, "B"); - Console.WriteLine("AFTER = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(3)); - var items = cola.ToArray(); - Assert.That(items.Length, Is.EqualTo(3)); - - Assert.That(items[0].Begin, Is.EqualTo(0)); - Assert.That(items[0].End, Is.EqualTo(4)); - Assert.That(items[0].Value, Is.EqualTo("A")); - - Assert.That(items[1].Begin, Is.EqualTo(4)); - Assert.That(items[1].End, Is.EqualTo(5)); - Assert.That(items[1].Value, Is.EqualTo("B")); - - Assert.That(items[2].Begin, Is.EqualTo(5)); - Assert.That(items[2].End, Is.EqualTo(10)); - Assert.That(items[2].Value, Is.EqualTo("A")); - - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(10)); - } - - [Test] - public void Test_RangeDictionary_Insert_That_Fits_Between_Two_Ranges() - { - var cola = new ColaRangeDictionary(); - cola.Mark(0, 1, "A"); - cola.Mark(2, 3, "B"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(1, 2, "C"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(3)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - - var items = cola.ToArray(); - Assert.That(items.Length, Is.EqualTo(3)); - - Assert.That(items[0].Begin, Is.EqualTo(0)); - Assert.That(items[0].End, Is.EqualTo(1)); - Assert.That(items[0].Value, Is.EqualTo("A")); - - Assert.That(items[1].Begin, Is.EqualTo(1)); - Assert.That(items[1].End, Is.EqualTo(2)); - Assert.That(items[1].Value, Is.EqualTo("C")); - - Assert.That(items[2].Begin, Is.EqualTo(2)); - Assert.That(items[2].End, Is.EqualTo(3)); - Assert.That(items[2].Value, Is.EqualTo("B")); - - } - - [Test] - public void Test_RangeDictionary_Insert_That_Join_Two_Ranges() - { - var cola = new ColaRangeDictionary(); - cola.Mark(0, 1, "A"); - cola.Mark(2, 3, "A"); - Console.WriteLine("BEFORE = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - // A_A_ + _A__ = AAA_ - cola.Mark(1, 2, "A"); - Console.WriteLine("AFTER = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - cola.Debug_Dump(); - - Assert.That(cola.Count, Is.EqualTo(1)); - var runs = cola.ToArray(); - Assert.That(runs[0].Begin, Is.EqualTo(0)); - Assert.That(runs[0].End, Is.EqualTo(3)); - Assert.That(runs[0].Value, Is.EqualTo("A")); - - } - - [Test] - public void Test_RangeDictionary_Insert_That_Replace_All_Ranges() - { - var cola = new ColaRangeDictionary(); - cola.Mark(0, 1, "A"); - cola.Mark(2, 3, "A"); - cola.Mark(4, 5, "A"); - cola.Mark(6, 7, "A"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(4)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(7)); - - cola.Mark(-1, 10, "B"); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(-1)); - Assert.That(cola.Bounds.End, Is.EqualTo(10)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeDictionary_Insert_Backwards() - { - const int N = 100; - - var cola = new ColaRangeDictionary(); - - for(int i = N; i > 0; i--) - { - int x = i << 1; - cola.Mark(x - 1, x, i.ToString()); - } - - Assert.That(cola.Count, Is.EqualTo(N)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_Can_Remove() - { - var dico = GetFilledRange(); - //on supprime tout - dico.Remove(0, 100, -100, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(0)); - - dico = GetFilledRange(); - //on ampute le premier range - dico.Remove(0, 12, -12, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(5)); - int i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 0, 3, true); - else if (i == 1) CompareEntries(entry, 8, 38, false); - else if (i == 2) CompareEntries(entry, 39, 50, true); - else if (i == 3) CompareEntries(entry, 51, 53, true); - else if (i == 4) CompareEntries(entry, 56, 63, false); - i++; - } - - //on supprime un truc a cheval sur plusieurs ranges - dico = GetFilledRange(); - dico.Remove(12, 55, -43, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(4)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 12, true); - if (i == 1) CompareEntries(entry, 12, 19, true); - else if (i == 2) CompareEntries(entry, 20, 22, true); - else if (i == 3) CompareEntries(entry, 25, 32, false); - i++; - } - - //on supprime avant le début - dico = GetFilledRange(); - dico.Remove(0, 8, -8, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(5)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 2, 7, true); - else if (i == 1) CompareEntries(entry, 12, 42, false); - else if (i == 2) CompareEntries(entry, 43, 54, true); - else if (i == 3) CompareEntries(entry, 55, 57, true); - else if (i == 4) CompareEntries(entry, 60, 67, false); - i++; - } - - //on supprimme exactement 2 ranges - dico = GetFilledRange(); - dico.Remove(20, 62, -42, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(3)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 15, true); - else if (i == 1) CompareEntries(entry, 21, 23, true); - else if (i == 2) CompareEntries(entry, 26, 33, false); - i++; - } - - //on supprime de maniere a ce que ca termine sur la fin d'un range - dico = GetFilledRange(); - dico.Remove(0, 50, -50, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(3)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 1, 12, true); - else if (i == 1) CompareEntries(entry, 13, 15, true); - else if (i == 2) CompareEntries(entry, 18, 25, false); - i++; - } - - //on supprimme jusqu'a la fin du premier - dico = GetFilledRange(); - dico.Remove(0, 15, -15, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(4)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 5, 35, false); - else if (i == 1) CompareEntries(entry, 36, 47, true); - else if (i == 2) CompareEntries(entry, 48, 50, true); - else if (i == 3) CompareEntries(entry, 53, 60, false); - i++; - } - - //on supprime jusqu'au milieu du 3e - dico = GetFilledRange(); - dico.Remove(0, 60, -60, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(3)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 0, 2, true); - else if (i == 1) CompareEntries(entry, 3, 5, true); - else if (i == 2) CompareEntries(entry, 8, 15, false); - i++; - } - - //on supprime jusqu'au debut du 3e - dico = GetFilledRange(); - dico.Remove(0, 51, -51, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(3)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 0, 11, true); - else if (i == 1) CompareEntries(entry, 12, 14, true); - else if (i == 2) CompareEntries(entry, 17, 24, false); - i++; - } - - //on supprime le début du 2e - dico = GetFilledRange(); - dico.Remove(20, 30, -10, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(5)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 15, true); - else if (i == 1) CompareEntries(entry, 20, 40, false); - else if (i == 2) CompareEntries(entry, 41, 52, true); - else if (i == 3) CompareEntries(entry, 53, 55, true); - else if (i == 4) CompareEntries(entry, 58, 65, false); - i++; - } - - //on supprime le 2e - dico = GetFilledRange(); - dico.Remove(20, 50, -30, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(4)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 15, true); - else if (i == 1) CompareEntries(entry, 21, 32, true); - else if (i == 2) CompareEntries(entry, 33, 35, true); - else if (i == 3) CompareEntries(entry, 38, 45, false); - i++; - } - - //on supprime le 1er et un bout du 2e - dico = GetFilledRange(); - dico.Remove(10, 30, -20, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(4)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 30, false); - else if (i == 1) CompareEntries(entry, 31, 42, true); - else if (i == 2) CompareEntries(entry, 43, 45, true); - else if (i == 3) CompareEntries(entry, 48, 55, false); - i++; - } - - //on supprime un morceau du second - dico = GetFilledRange(); - dico.Remove(30, 40, -10, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(6)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 15, true); - else if (i == 1) CompareEntries(entry, 20, 30, false); - else if (i == 2) CompareEntries(entry, 30, 40, false); - else if (i == 3) CompareEntries(entry, 41, 52, true); - else if (i == 4) CompareEntries(entry, 53, 55, true); - else if (i == 5) CompareEntries(entry, 58, 65, false); - i++; - } - - //on supprime la fin du second - dico = GetFilledRange(); - dico.Remove(30, 50, -20, (x, y) => x + y); - Assert.That(dico.Count, Is.EqualTo(5)); - i = 0; - foreach (var entry in dico) - { - if (i == 0) CompareEntries(entry, 10, 15, true); - else if (i == 1) CompareEntries(entry, 20, 30, false); - else if (i == 2) CompareEntries(entry, 31, 42, true); - else if (i == 3) CompareEntries(entry, 43, 45, true); - else if (i == 4) CompareEntries(entry, 48, 55, false); - i++; - } - } - - public void CompareEntries(ColaRangeDictionary.Entry entry, TKey begin, TKey end, TValue value) - { - Assert.That(entry.Begin, Is.EqualTo(begin)); - Assert.That(entry.End, Is.EqualTo(end)); - Assert.That(entry.Value, Is.EqualTo(value)); - } - - public ColaRangeDictionary GetFilledRange() - { - //returns a colaRange prefilled with this : - // {10-15, true} - // {20-50, false} - // {51,62, true} - // {63,65, true} - // {68,75, false} - - var cola = new ColaRangeDictionary(); - cola.Mark(10, 15, true); - cola.Mark(20, 50, false); - cola.Mark(51, 62, true); - cola.Mark(63, 65, true); - cola.Mark(68, 75, false); - - return cola; - } - - enum RangeColor - { - Black, - White - } - - [Test] - public void Test_RangeDictionary_Black_And_White() - { - // we have a space from 0 <= x < 100 that is empty - // we insert a random serie of ranges that are either Black or White - // after each run, we check that all ranges are correctly ordered, merged, and so on. - - const int S = 100; // [0, 100) - const int N = 1000; // number of repetitions - const int K = 25; // max number of ranges inserted per run - - var rnd = new Random(); - int seed = rnd.Next(); - Console.WriteLine("Using random seed " + seed); - rnd = new Random(seed); - - for(int i = 0; i< N; i++) - { - var cola = new ColaRangeDictionary(); - - var witnessColors = new RangeColor?[S]; - var witnessIndexes = new int?[S]; - - // choose a random number of ranges - int k = rnd.Next(3, K); - - Trace.WriteLine(""); - Trace.WriteLine(String.Format("# Starting run {0} with {1} insertions", i, k)); - - int p = 0; - for(int j = 0; j |{0}|", String.Join("", witnessIndexes.Select(x => x.HasValue ? (char)('A' + x.Value) : ' '))); - Debug.WriteLine(" |{0}| + [{1,2}, {2,2}) = {3} > #{4,2} [ {5} ]", String.Join("", witnessColors.Select(w => !w.HasValue ? ' ' : w.Value == RangeColor.Black ? '#' : '°')), begin, end, color, cola.Count, String.Join(", ", cola)); - - ++p; - } - - // pack the witness list into ranges - var witnessRanges = new List>(); - RangeColor? prev = null; - p = 0; - for (int z = 1; z < S;z++) - { - if (witnessColors[z] != prev) - { // switch - - if (prev.HasValue) - { - witnessRanges.Add(FdbTuple.Create(p, z, prev.Value)); - } - p = z; - prev = witnessColors[z]; - } - } - - Trace.WriteLine(String.Format("> RANGES: #{0,2} [ {1} ]", cola.Count, String.Join(", ", cola))); - Trace.WriteLine(String.Format(" #{0,2} [ {1} ]", witnessRanges.Count, String.Join(", ", witnessRanges))); - - var counter = new int[S]; - var observedIndexes = new int?[S]; - var observedColors = new RangeColor?[S]; - p = 0; - foreach(var range in cola) - { - Assert.That(range.Begin < range.End, "Begin < End {0}", range); - for (int z = range.Begin; z < range.End; z++) - { - observedIndexes[z] = p; - counter[z]++; - observedColors[z] = range.Value; - } - ++p; - } - - Trace.WriteLine(String.Format("> INDEXS: |{0}|", String.Join("", observedIndexes.Select(x => x.HasValue ? (char)('A' + x.Value) : ' ')))); - Trace.WriteLine(String.Format(" |{0}|", String.Join("", witnessIndexes.Select(x => x.HasValue ? (char)('A' + x.Value) : ' ')))); - - Trace.WriteLine(String.Format("> COLORS: |{0}|", String.Join("", observedColors.Select(w => !w.HasValue ? ' ' : w.Value == RangeColor.Black ? '#' : '°')))); - Trace.WriteLine(String.Format(" |{0}|", String.Join("", witnessColors.Select(w => !w.HasValue ? ' ' : w.Value == RangeColor.Black ? '#' : '°')))); - - // verify the colors - foreach(var range in cola) - { - for (int z = range.Begin; z < range.End; z++) - { - Assert.That(range.Value, Is.EqualTo(witnessColors[z]), "#{0} color mismatch for {1}", z, range); - Assert.That(counter[z], Is.EqualTo(1), "Duplicate at offset #{0} for {1}", z, range); - } - } - - // verify that nothing was missed - for(int z = 0; z < S; z++) - { - if (witnessColors[z] == null) - { - if (counter[z] != 0) Trace.WriteLine("@ FAIL!!! |" + new string('-', z) + "^"); - Assert.That(counter[z], Is.EqualTo(0), "Should be void at offset {0}", z); - } - else - { - if (counter[z] != 1) Trace.WriteLine("@ FAIL!!! |" + new string('-', z) + "^"); - Assert.That(counter[z], Is.EqualTo(1), "Should be filled with {1} at offset {0}", z, witnessColors[z]); - } - } - } - } - - [Test] - public void Test_RangeDictionary_Insert_Random_Ranges() - { - const int N = 1000; - const int K = 1000 * 1000; - - var cola = new ColaRangeDictionary(); - - var rnd = new Random(); - int seed = 2040305906; // rnd.Next(); - Console.WriteLine("seed " + seed); - rnd = new Random(seed); - - int[] expected = new int[N]; - - var sw = Stopwatch.StartNew(); - for (int i = 0; i < K; i++) - { - if (rnd.Next(10000) < 42) - { - //Console.WriteLine("Clear"); - cola.Clear(); - } - else - { - - int x = rnd.Next(N); - int y = rnd.Next(2) == 0 ? x + 1 : rnd.Next(N); - if (y == x) ++y; - if (x <= y) - { - //Console.WriteLine(); - //Console.WriteLine("Add " + x + " ~ " + y + " = " + i); - cola.Mark(x, y, i); - } - else - { - //Console.WriteLine(); - //Console.WriteLine("ddA " + y + " ~ " + x + " = " + i); - cola.Mark(y, x, i); - } - } - //Console.WriteLine(" = " + cola + " -- <> = " + cola.Bounds); - //cola.Debug_Dump(); - - } - sw.Stop(); - - Console.WriteLine("Inserted " + K.ToString("N0") + " random ranges in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - cola.Debug_Dump(); - - Console.WriteLine("Result = " + cola); - Console.WriteLine("Bounds = " + cola.Bounds); - - } - } -} diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeSetFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaRangeSetFacts.cs deleted file mode 100644 index 4d0f2cdb8..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaRangeSetFacts.cs +++ /dev/null @@ -1,163 +0,0 @@ -using FoundationDB.Client; -using FoundationDB.Layers.Tuples; -using NUnit.Framework; -using System; -using System.Collections.Generic; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - -namespace FoundationDB.Storage.Memory.Core.Test -{ - [TestFixture] - public class ColaRangeSetFacts - { - - [Test] - public void Test_Empty_RangeSet() - { - var cola = new ColaRangeSet(0, Comparer.Default); - Assert.That(cola.Count, Is.EqualTo(0)); - Assert.That(cola.Comparer, Is.SameAs(Comparer.Default)); - Assert.That(cola.Capacity, Is.EqualTo(15), "Capacity should be the next power of 2, minus 1"); - Assert.That(cola.Bounds, Is.Not.Null); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(0)); - } - - [Test] - public void Test_RangeSet_Insert_Non_Overlapping() - { - var cola = new ColaRangeSet(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(2, 3); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(4, 5); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(3)); - - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeSet_Insert_Partially_Overlapping() - { - var cola = new ColaRangeSet(); - Assert.That(cola.Count, Is.EqualTo(0)); - - cola.Mark(0, 1); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(0, 2); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(1, 3); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - cola.Mark(-1, 2); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeSet_Insert_Completly_Overlapping() - { - var cola = new ColaRangeSet(); - cola.Mark(1, 2); - cola.Mark(4, 5); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(1)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - - // overlaps the first range completely - cola.Mark(0, 3); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(5)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - - } - - [Test] - public void Test_RangeSet_Insert_That_Join_Two_Ranges() - { - var cola = new ColaRangeSet(); - cola.Mark(0, 1); - cola.Mark(2, 3); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(2)); - - cola.Mark(1, 2); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeSet_Insert_That_Replace_All_Ranges() - { - var cola = new ColaRangeSet(); - cola.Mark(0, 1); - cola.Mark(2, 3); - cola.Mark(4, 5); - cola.Mark(6, 7); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(4)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(0)); - Assert.That(cola.Bounds.End, Is.EqualTo(7)); - - cola.Mark(-1, 10); - cola.Debug_Dump(); - Assert.That(cola.Count, Is.EqualTo(1)); - Assert.That(cola.Bounds.Begin, Is.EqualTo(-1)); - Assert.That(cola.Bounds.End, Is.EqualTo(10)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - - [Test] - public void Test_RangeSet_Insert_Backwards() - { - const int N = 100; - - var cola = new ColaRangeSet(); - - for(int i = N; i > 0; i--) - { - int x = i << 1; - cola.Mark(x - 1, x); - } - - Assert.That(cola.Count, Is.EqualTo(N)); - - Console.WriteLine("Result = { " + String.Join(", ", cola) + " }"); - Console.WriteLine("Bounds = " + cola.Bounds); - } - } -} diff --git a/FoundationDB.Storage.Memory.Test/Collections/ColaStoreFacts.cs b/FoundationDB.Storage.Memory.Test/Collections/ColaStoreFacts.cs deleted file mode 100644 index cf7b89763..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/ColaStoreFacts.cs +++ /dev/null @@ -1,462 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core.Test -{ - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; - using System.Text; - using System.Threading; - using System.Threading.Tasks; - - [TestFixture] - public class ColaStoreFacts - { - [Test] - public void Test_Bit_Twiddling() - { - Assert.That(ColaStore.LowestBit(0), Is.EqualTo(0)); - Assert.That(ColaStore.HighestBit(0), Is.EqualTo(0)); - - Assert.That(ColaStore.LowestBit(42), Is.EqualTo(1)); - Assert.That(ColaStore.HighestBit(42), Is.EqualTo(5)); - - for (int i = 1; i < 30; i++) - { - int x = 1 << i; - Assert.That(ColaStore.LowestBit(x), Is.EqualTo(i)); - Assert.That(ColaStore.HighestBit(x), Is.EqualTo(i)); - - Assert.That(ColaStore.HighestBit(x - 1), Is.EqualTo(i - 1)); - Assert.That(ColaStore.LowestBit(x - 1), Is.EqualTo(0)); - } - - Assert.That(ColaStore.LowestBit(0x60000000), Is.EqualTo(29)); - for (int i = 1; i < 30; i++) - { - int x = int.MaxValue - ((1 << i) - 1); - Assert.That(ColaStore.LowestBit(x), Is.EqualTo(i), "i={0}, x={1} : {2}", i, x.ToString("X8"), Convert.ToString(x, 2)); - } - } - - [Test] - public void Test_Map_Index_To_Address() - { - // index => (level, offset) - - int level, offset; - for (int i = 0; i < 1024; i++) - { - level = ColaStore.FromIndex(i, out offset); - Assert.That(((1 << level) - 1) + offset, Is.EqualTo(i), "{0} => ({1}, {2})", i, level, offset); - } - } - - [Test] - public void Test_Map_Address_Index() - { - // index => (level, offset) - - for (int level = 0; level <= 10; level++) - { - int n = 1 << level; - for (int offset = 0; offset < n; offset++) - { - int index = ColaStore.ToIndex(level, offset); - Assert.That(index, Is.EqualTo(n - 1 + offset), "({0}, {1}) => {2}", level, offset, index); - } - } - } - - [Test] - public void Test_Map_Offset_To_Index() - { - //N = 1 - // > 0 [0] - Assert.That(ColaStore.MapOffsetToIndex(1, 0), Is.EqualTo(0)); - Assert.That(() => ColaStore.MapOffsetToIndex(1, 1), Throws.InstanceOf()); - - //N = 2 - // > 0 [_] - // > 1 [0, 1] - Assert.That(ColaStore.MapOffsetToIndex(2, 0), Is.EqualTo(1)); - Assert.That(ColaStore.MapOffsetToIndex(2, 1), Is.EqualTo(2)); - Assert.That(() => ColaStore.MapOffsetToIndex(2, 2), Throws.InstanceOf()); - - //N = 3 - // > 0 [2] - // > 1 [0, 1] - Assert.That(ColaStore.MapOffsetToIndex(3, 0), Is.EqualTo(1)); - Assert.That(ColaStore.MapOffsetToIndex(3, 1), Is.EqualTo(2)); - Assert.That(ColaStore.MapOffsetToIndex(3, 2), Is.EqualTo(0)); - Assert.That(() => ColaStore.MapOffsetToIndex(3, 3), Throws.InstanceOf()); - - //N = 5 - // > 0 [4] - // > 1 [_, _] - // > 2 [0, 1, 2, 3] - Assert.That(ColaStore.MapOffsetToIndex(5, 0), Is.EqualTo(3)); - Assert.That(ColaStore.MapOffsetToIndex(5, 1), Is.EqualTo(4)); - Assert.That(ColaStore.MapOffsetToIndex(5, 2), Is.EqualTo(5)); - Assert.That(ColaStore.MapOffsetToIndex(5, 3), Is.EqualTo(6)); - Assert.That(ColaStore.MapOffsetToIndex(5, 4), Is.EqualTo(0)); - Assert.That(() => ColaStore.MapOffsetToIndex(5, 5), Throws.InstanceOf()); - - // N = 10 - // > 0 [_] - // > 1 [8,9] - // > 2 [_,_,_,_] - // > 3 [0,1,2,3,4,5,6,7] - for (int i = 0; i < 8; i++) Assert.That(ColaStore.MapOffsetToIndex(10, i), Is.EqualTo(7 + i), "MapOffset(10, {0})", i); - for (int i = 8; i < 10; i++) Assert.That(ColaStore.MapOffsetToIndex(10, i), Is.EqualTo(1 + (i - 8)), "MapOffset(10, {0})", i); - Assert.That(() => ColaStore.MapOffsetToIndex(10, 123), Throws.InstanceOf()); - } - - [Test] - public void Test_ColaStore_Iterator_Seek() - { - - var store = new ColaStore(0, Comparer.Default); - - for (int i = 0; i < 10; i++) - { - store.Insert(i); - } - store.Debug_Dump(); - - var iterator = store.GetIterator(); - - Assert.That(iterator.Seek(5, true), Is.True); - Assert.That(iterator.Current, Is.EqualTo(5)); - - Assert.That(iterator.Seek(5, false), Is.True); - Assert.That(iterator.Current, Is.EqualTo(4)); - - Assert.That(iterator.Seek(9, true), Is.True); - Assert.That(iterator.Current, Is.EqualTo(9)); - - Assert.That(iterator.Seek(9, false), Is.True); - Assert.That(iterator.Current, Is.EqualTo(8)); - - Assert.That(iterator.Seek(0, true), Is.True); - Assert.That(iterator.Current, Is.EqualTo(0)); - - Assert.That(iterator.Seek(0, false), Is.False); - Assert.That(iterator.Current, Is.Null); - - Assert.That(iterator.Seek(10, true), Is.True); - Assert.That(iterator.Current, Is.EqualTo(9)); - - Assert.That(iterator.Seek(10, false), Is.True); - Assert.That(iterator.Current, Is.EqualTo(9)); - - } - - [Test] - public void Test_ColaStore_Iterator_Seek_Randomized() - { - const int N = 1000; - - var store = new ColaStore(0, Comparer.Default); - - var rnd = new Random(); - int seed = rnd.Next(); - Console.WriteLine("seed = " + seed); - rnd = new Random(seed); - - var list = Enumerable.Range(0, N).ToList(); - while(list.Count > 0) - { - int p = rnd.Next(list.Count); - store.Insert(list[p]); - list.RemoveAt(p); - } - store.Debug_Dump(); - - for (int i = 0; i < N; i++) - { - var iterator = store.GetIterator(); - - int p = rnd.Next(N); - bool orEqual = rnd.Next(2) == 0; - - bool res = iterator.Seek(p, orEqual); - - if (orEqual) - { // the key should exist - Assert.That(res, Is.True, "Seek({0}, '<=')", p); - Assert.That(iterator.Current, Is.EqualTo(p), "Seek({0}, '<=')", p); - Assert.That(iterator.Valid, Is.True, "Seek({0}, '<=')", p); - } - else if (p == 0) - { // there is no key before the first - Assert.That(res, Is.False, "Seek(0, '<')"); - Assert.That(iterator.Current, Is.Null, "Seek(0, '<')"); - Assert.That(iterator.Valid, Is.False, "Seek(0, '<')"); - } - else - { // the key should exist - Assert.That(res, Is.True, "Seek({0}, '<')", p); - Assert.That(iterator.Current, Is.EqualTo(p - 1), "Seek({0}, '<')", p); - Assert.That(iterator.Valid, Is.True, "Seek({0}, '<')", p); - } - } - - } - - [Test] - public void Test_ColaStore_Iterator_Seek_Then_Next_Randomized() - { - const int N = 1000; - const int K = 10; - - var store = new ColaStore(0, Comparer.Default); - - var rnd = new Random(); - int seed = rnd.Next(); - Console.WriteLine("seed = " + seed); - rnd = new Random(seed); - - var list = Enumerable.Range(0, N).ToList(); - while (list.Count > 0) - { - int p = rnd.Next(list.Count); - store.Insert(list[p]); - list.RemoveAt(p); - } - store.Debug_Dump(); - - for (int i = 0; i < N; i++) - { - var iterator = store.GetIterator(); - - int p = rnd.Next(N); - bool orEqual = rnd.Next(2) == 0; - - if (p == 0 && !orEqual) continue; //TODO: what to do for this case ? - - Assert.That(iterator.Seek(p, orEqual), Is.True); - int? x = iterator.Current; - Assert.That(x, Is.EqualTo(orEqual ? p : p - 1)); - - // all the next should be ordered (starting from p) - while (x < N - 1) - { - Assert.That(iterator.Next(), Is.True, "Seek({0}).Current({1}).Next()", p, x); - Assert.That(iterator.Current, Is.EqualTo(x + 1), "Seek({0}).Current({1}).Next()", p, x); - ++x; - } - // the following Next() should go past the end - Assert.That(iterator.Next(), Is.False); - Assert.That(iterator.Current, Is.Null); - Assert.That(iterator.Valid, Is.False); - - // re-seek to the original location - Assert.That(iterator.Seek(p, orEqual), Is.True); - x = iterator.Current; - Assert.That(x, Is.EqualTo(orEqual ? p : p - 1)); - - // now go backwards - while (x > 0) - { - Assert.That(iterator.Previous(), Is.True, "Seek({0}).Current({1}).Previous()", p, x); - Assert.That(iterator.Current, Is.EqualTo(x - 1), "Seek({0}).Current({1}).Previous()", p, x); - --x; - } - // the following Previous() should go past the beginning - Assert.That(iterator.Previous(), Is.False); - Assert.That(iterator.Current, Is.Null); - Assert.That(iterator.Valid, Is.False); - - if (p >= K && p < N - K) - { // jitter dance - - // start to original location - Assert.That(iterator.Seek(p, true), Is.True); - Assert.That(iterator.Current, Is.EqualTo(p)); - - var sb = new StringBuilder(); - sb.Append("Seek -> "); - for(int j = 0; j < K; j++) - { - x = iterator.Current; - sb.Append(iterator.Current); - if (rnd.Next(2) == 0) - { // next - sb.Append(" -> "); - Assert.That(iterator.Next(), Is.True, "{0}", sb); - Assert.That(iterator.Current, Is.EqualTo(x + 1), "{0} = ?", sb); - } - else - { // prev - sb.Append(" <- "); - Assert.That(iterator.Previous(), Is.True, "{0}", sb); - Assert.That(iterator.Current, Is.EqualTo(x - 1), "{0} = ?", sb); - } - } - } - - } - - } - - [Test] - [Category("LongRunning")] - public void Test_MiniBench() - { - const int N = (1 << 23) - 1; // 10 * 1000 * 1000; - - var rnd = new Random(); - int offset, level; - long x; - - - //WARMUP - var store = new ColaStore(0, Comparer.Default); - store.Insert(1); - store.Insert(42); - store.Insert(1234); - level = store.Find(1, out offset, out x); - - const int BS = (N + 1) / 128; - var timings = new List(BS); - timings.Add(TimeSpan.Zero); - timings.Clear(); - - #region Sequentially inserted.... - - Console.WriteLine("Inserting {0} sequential keys into a COLA store", N); - GC.Collect(); - store = new ColaStore(0, Comparer.Default); - long total = 0; - var sw = Stopwatch.StartNew(); - for (int i = 0; i < N; i++) - { - - int y = rnd.Next(100); - - level = store.Find(y, out offset, out x); - if (level < 0) store.Insert(i); - else store.SetAt(level, offset, x); - - Interlocked.Increment(ref total); - if ((i % BS) == BS - 1) - { - sw.Stop(); - timings.Add(sw.Elapsed); - Console.Write("."); - sw.Start(); - } - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS: " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - for (int i = 0; i < timings.Count; i++) - { - Console.WriteLine("" + ((i + 1) * BS).ToString() + "\t" + timings[i].TotalSeconds); - } - return; - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - level = store.Find(i, out offset, out x); - } - sw.Stop(); - Console.WriteLine("SeqReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - level = store.Find(rnd.Next(N), out offset, out x); - } - sw.Stop(); - Console.WriteLine("RndReadOrdered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - #region Randomly inserted.... - - Console.WriteLine("(preparing random insert list)"); - - var tmp = new long[N]; - var values = new long[N]; - for (int i = 0; i < N; i++) - { - tmp[i] = rnd.Next(N); - values[i] = i; - } - Array.Sort(tmp, values); - - Console.WriteLine("Inserting " + N.ToString("N0") + " sequential keys into a COLA store"); - GC.Collect(); - store = new ColaStore(0, Comparer.Default); - total = 0; - - timings.Clear(); - - sw.Restart(); - for (int i = 0; i < N; i++) - { - level = store.Find(i, out offset, out x); - store.Insert(values[i]); - Interlocked.Increment(ref total); - if ((i % BS) == BS - 1) - { - sw.Stop(); - timings.Add(sw.Elapsed); - Console.Write("."); - sw.Start(); - } - } - sw.Stop(); - - Console.WriteLine("done"); - Console.WriteLine("* Inserted: " + total.ToString("N0") + " keys"); - Console.WriteLine("* Elapsed : " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec"); - Console.WriteLine("* KPS : " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " key/sec"); - Console.WriteLine("* Latency : " + (sw.Elapsed.TotalMilliseconds * 1000000 / total).ToString("N1") + " nanos / insert"); - - for (int i = 0; i < timings.Count;i++) - { - Console.WriteLine("" + ((i + 1) * BS).ToString() + "\t" + timings[i].TotalSeconds); - } - - // sequential reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - level = store.Find(i, out offset, out x); - } - sw.Stop(); - Console.WriteLine("SeqReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - // random reads - - sw.Restart(); - for (int i = 0; i < total; i++) - { - level = store.Find(rnd.Next(N), out offset, out x); - } - sw.Stop(); - Console.WriteLine("RndReadUnordered: " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - #endregion - - } - - } - -} diff --git a/FoundationDB.Storage.Memory.Test/Collections/CountingComparer`1.cs b/FoundationDB.Storage.Memory.Test/Collections/CountingComparer`1.cs deleted file mode 100644 index 31e178267..000000000 --- a/FoundationDB.Storage.Memory.Test/Collections/CountingComparer`1.cs +++ /dev/null @@ -1,42 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core.Test -{ - using System; - using System.Collections.Generic; - using System.Threading; - - /// Wrapper for an that counts the number of calls to the method - public class CountingComparer : IComparer - { - - private int m_count; - private IComparer m_comparer; - - - public CountingComparer() - : this(Comparer.Default) - { } - - public CountingComparer(IComparer comparer) - { - m_comparer = comparer; - } - - public int Count { get { return Volatile.Read(ref m_count); } } - - public void Reset() - { - Volatile.Write(ref m_count, 0); - } - - public int Compare(T x, T y) - { - Interlocked.Increment(ref m_count); - return m_comparer.Compare(x, y); - } - } - -} diff --git a/FoundationDB.Storage.Memory.Test/FdbTest.cs b/FoundationDB.Storage.Memory.Test/FdbTest.cs deleted file mode 100644 index 48b38ddcc..000000000 --- a/FoundationDB.Storage.Memory.Test/FdbTest.cs +++ /dev/null @@ -1,141 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013, Doxense SARL -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 - -namespace FoundationDB.Storage.Memory.Tests -{ - using FoundationDB.Client; - using FoundationDB.Layers.Directories; - using NUnit.Framework; - using System; - using System.Diagnostics; - using System.Globalization; - using System.Threading; - using System.Threading.Tasks; - - public abstract class FdbTest - { - - private CancellationTokenSource m_cts; - private CancellationToken m_ct; - - [TestFixtureSetUp] - protected void BeforeAllTests() - { - Trace.WriteLine("### " + this.GetType().FullName + " starting"); - //TODO? - } - - [SetUp] - protected void BeforeEachTest() - { - lock (this) - { - m_cts = null; - m_ct = CancellationToken.None; - } - Trace.WriteLine("=== " + TestContext.CurrentContext.Test.FullName + " === " + DateTime.Now.TimeOfDay); - } - - [TearDown] - protected void AfterEachTest() - { - if (m_cts != null) - { - try { m_cts.Cancel(); } catch { } - m_cts.Dispose(); - } - } - - [TestFixtureTearDown] - protected void AfterAllTests() - { - //TODO? - Trace.WriteLine("### " + this.GetType().FullName + " completed"); - } - - /// Cancellation token usable by any test - protected CancellationToken Cancellation - { - [DebuggerStepThrough] - get - { - if (m_cts == null) SetupCancellation(); - return m_ct; - } - } - - private void SetupCancellation() - { - lock (this) - { - if (m_cts == null) - { - m_cts = new CancellationTokenSource(); - m_ct = m_cts.Token; - } - } - } - - #region Logging... - - // These methods are just there to help with the problem of culture-aware string formatting - - [DebuggerStepThrough] - protected static void Log() - { - Console.WriteLine(); - } - - [DebuggerStepThrough] - protected static void Log(string text) - { - Console.WriteLine(text); - } - - [DebuggerStepThrough] - protected static void Log(string format, object arg0) - { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, arg0)); - } - - [DebuggerStepThrough] - protected static void Log(string format, object arg0, object arg1) - { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, arg0, arg1)); - } - - [DebuggerStepThrough] - protected static void Log(string format, params object[] args) - { - Console.WriteLine(String.Format(CultureInfo.InvariantCulture, format, args)); - } - - #endregion - - } -} diff --git a/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj b/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj deleted file mode 100644 index 700832ec1..000000000 --- a/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.csproj +++ /dev/null @@ -1,99 +0,0 @@ - - - - - Debug - AnyCPU - {AF76A8D4-E682-4E72-B656-BE3D935712DB} - Exe - Properties - FoundationDB.Storage.Memory.Test - FoundationDB.Storage.Memory.Test - v4.6.1 - 512 - ..\ - true - - - AnyCPU - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - false - false - - - AnyCPU - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - false - - - true - - - FoundationDB.Storage.Memory.Test.snk - - - - False - ..\packages\NUnit.2.6.4\lib\nunit.framework.dll - - - - - - - - - - - - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} - FoundationDB.Layers.Common - - - {cc98db39-31a1-4642-b4fc-9cb0ab26bf2e} - FoundationDB.Storage.Memory - - - - - - - - \ No newline at end of file diff --git a/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.snk b/FoundationDB.Storage.Memory.Test/FoundationDB.Storage.Memory.Test.snk deleted file mode 100644 index 5b29927c38aee32e67ec2db3d5d68f043df22760..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 596 zcmV-a0;~N80ssI2Bme+XQ$aES1ONa50096a(jMgH;w8W3yi~$XKc;UN2!MO&h(*l4 z{;F~12H#SrtsZ-ofrw$WX1Sy~8c^LzkA=D~w>vx5^ zJpgJ^P4_H^l&Q8snw}BgzpBWZEltAt^ovl7DGAwRJU6sqAQb(#moW_Kqz+&H(i#8< zgj3dJVQXZ7Ns^D)dP=F2bb`|**`m%xJC!J}?E-rAf@v@z8)TGdkrb(f0Ohv{Q!`h6g9L@rWxK*We~t~@wKc33XA+1L1mfA?(52l6zD zj~_9~p+s${F>CsWhh*jqM5FR`yiVXJRD7G8Mx8AFPQ2cLXOt>qipY#Cn-cY>Ms6&P zXvJ|S1e-;n;}@RaeltzK{N$pobKw6y-Q^Wn3@Sdgx iaZbWS6hqdsl?h{)~F^Wfr {3,9:N0} kps, {4,7:N0} tps", - label, - total, - elapsed.TotalSeconds, - total / elapsed.TotalSeconds, - trans / elapsed.TotalSeconds - ); - } - - private static void DumpMemory(bool collect = false) - { - if (collect) - { - GC.Collect(); - GC.WaitForPendingFinalizers(); - GC.Collect(); - } - Log("Total memory: Managed={0:N1} KiB, WorkingSet={1:N1} KiB", GC.GetTotalMemory(false) / 1024.0, Environment.WorkingSet / 1024.0); - } - - [Test] - public async Task MiniBench() - { - const int M = 1 * 1000 * 1000; - const int B = 100; - const int ENTROPY = 10 * 1000; - - const int T = M / B; - const int KEYSIZE = 10; - const int VALUESIZE = 100; - const bool RANDOM = false; - - var rnd = new Random(); - - //WARMUP - using (var db = MemoryDatabase.CreateNew("FOO")) - { - await db.WriteAsync((tr) => tr.Set(db.Keys.Encode("hello"), Slice.FromString("world")), this.Cancellation); - Slice.Random(rnd, KEYSIZE); - Slice.Random(rnd, VALUESIZE); - } - - Log("Inserting {0}-bytes {1} keys / {2}-bytes values, in {3:N0} transactions", KEYSIZE, RANDOM ? "random" : "ordered", VALUESIZE, T); - - bool random = RANDOM; - string fmt = "D" + KEYSIZE; - using (var db = MemoryDatabase.CreateNew("FOO")) - { - DumpMemory(collect: true); - - long total = 0; - - var payload = new byte[ENTROPY + VALUESIZE]; - rnd.NextBytes(payload); - // help with compression by doubling every byte - for (int i = 0; i < payload.Length; i += 2) payload[i + 1] = payload[i]; - - var sw = Stopwatch.StartNew(); - sw.Stop(); - - sw.Restart(); - for (int i = 0; i < T; i++) - { - using (var tr = db.BeginTransaction(this.Cancellation)) - { - for (int j = 0; j < B; j++) - { - Slice key; - if (random) - { - do - { - key = Slice.Random(rnd, KEYSIZE); - } - while (key[0] == 255); - } - else - { - int x = i * B + j; - //x = x % 1000; - key = Slice.FromString(x.ToString(fmt)); - } - - tr.Set(key, Slice.Create(payload, rnd.Next(ENTROPY), VALUESIZE)); - Interlocked.Increment(ref total); - } - await tr.CommitAsync().ConfigureAwait(false); - } - if (i % 1000 == 0) Console.Write(".");// + (i * B).ToString("D10")); - } - - sw.Stop(); - Log("done"); - Log("* Inserted: {0:N0} keys", total); - Log("* Elapsed : {0:N3} sec", sw.Elapsed.TotalSeconds); - Log("* TPS: {0:N0} transactions/sec", T / sw.Elapsed.TotalSeconds); - Log("* KPS: {0:N0} keys/sec", total / sw.Elapsed.TotalSeconds); - Log("* BPS: {0:N0} bytes/sec", (total * (KEYSIZE + VALUESIZE)) / sw.Elapsed.TotalSeconds); - - DumpMemory(collect: true); - - db.Debug_Dump(false); - - DumpResult("WriteSeq" + B, total, total / B, sw.Elapsed); - - string path = @".\\minibench.pndb"; - Log("Saving {0} ...", path); - sw.Restart(); - await db.SaveSnapshotAsync(path); - sw.Stop(); - Log("* Saved {0:N0} bytes in {1:N3} sec", new System.IO.FileInfo(path).Length, sw.Elapsed.TotalSeconds); - - Log("Warming up reads..."); - var data = await db.GetValuesAsync(Enumerable.Range(0, 100).Select(i => Slice.FromString(i.ToString(fmt))), this.Cancellation); - - Log("Starting read tests..."); - - #region sequential reads - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 10).Select(x => Slice.FromString(x.ToString(fmt)))).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead10", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.Snapshot.GetValuesAsync(Enumerable.Range(i, 10).Select(x => Slice.FromString(x.ToString(fmt)))).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead10S", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - int x = i; - int y = i + 10; - await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(x.ToString(fmt))), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(y.ToString(fmt))) - ).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead10R", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 100) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 100).Select(x => Slice.FromString(x.ToString(fmt)))).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead100", total, total / 100, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 100) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.Snapshot.GetValuesAsync(Enumerable.Range(i, 100).Select(x => Slice.FromString(x.ToString(fmt)))).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead100S", total, total / 100, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 100) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - int x = i; - int y = i + 100; - await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(x.ToString(fmt))), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(y.ToString(fmt))) - ).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead100R", total, total / 100, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 100) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - int x = i; - int y = i + 100; - await tr.Snapshot.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(x.ToString(fmt))), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(y.ToString(fmt))) - ).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead100RS", total, total / 100, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 1000) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 1000).Select(x => Slice.FromString(x.ToString(fmt)))).ConfigureAwait(false); - } - } - sw.Stop(); - DumpResult("SeqRead1k", total, total / 1000, sw.Elapsed); - - #endregion - - DumpMemory(); - - #region random reads - - //sw.Restart(); - //for (int i = 0; i < total; i++) - //{ - // using (var tr = db.BeginReadOnlyTransaction()) - // { - // int x = rnd.Next((int)total); - // await tr.GetAsync(Slice.FromString(x.ToString(fmt))); - // } - //} - //sw.Stop(); - //Log("RndRead1 : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 10).Select(x => Slice.FromString(rnd.Next((int)total).ToString(fmt)))).ConfigureAwait(false); - } - - } - sw.Stop(); - //Log("RndRead10 : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps, " + (total / (10 * sw.Elapsed.TotalSeconds)).ToString("N0") + " tps"); - DumpResult("RndRead10", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.Snapshot.GetValuesAsync(Enumerable.Range(i, 10).Select(x => Slice.FromString(rnd.Next((int)total).ToString(fmt)))).ConfigureAwait(false); - } - - } - sw.Stop(); - //Log("RndRead10S : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps, " + (total / (10 * sw.Elapsed.TotalSeconds)).ToString("N0") + " tps"); - DumpResult("RndRead10S", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 10) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - int x = rnd.Next((int)total - 10); - int y = x + 10; - await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(x.ToString(fmt))), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(y.ToString(fmt))) - ).ConfigureAwait(false); - } - - } - sw.Stop(); - //Log("RndRead10R : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps, " + (total / (10 * sw.Elapsed.TotalSeconds)).ToString("N0") + " tps"); - DumpResult("RndRead10R", total, total / 10, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 100) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 100).Select(x => Slice.FromString(rnd.Next((int)total).ToString(fmt)))).ConfigureAwait(false); - } - - } - sw.Stop(); - //Log("RndRead100 : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps, " + (total / (100 * sw.Elapsed.TotalSeconds)).ToString("N0") + " tps"); - DumpResult("RndRead100", total, total / 100, sw.Elapsed); - - sw.Restart(); - for (int i = 0; i < total; i += 1000) - { - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - await tr.GetValuesAsync(Enumerable.Range(i, 1000).Select(x => Slice.FromString(rnd.Next((int)total).ToString(fmt)))).ConfigureAwait(false); - } - - } - sw.Stop(); - //Log("RndRead1k : " + total.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (total / sw.Elapsed.TotalSeconds).ToString("N0") + " kps, " + (total / (1000 * sw.Elapsed.TotalSeconds)).ToString("N0") + " tps"); - DumpResult("RndRead1k", total, total / 1000, sw.Elapsed); - - #endregion - - DumpMemory(); - - #region Parallel Reads... - - int CPUS = Environment.ProcessorCount; - - long read = 0; - var mre = new ManualResetEvent(false); - var tasks = Enumerable - .Range(0, CPUS) - .Select(k => Task.Run(async () => - { - var rndz = new Random(k); - mre.WaitOne(); - - int keys = 0; - for (int j = 0; j < 20; j++) - { - for (int i = 0; i < total / CPUS; i += 100) - { - int pp = i;// rndz.Next((int)total - 10); - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tr.GetValuesAsync(Enumerable.Range(i, 100).Select(x => Slice.FromString((pp + x).ToString(fmt)))).ConfigureAwait(false); - keys += res.Length; - } - } - } - Interlocked.Add(ref read, keys); - return keys; - })).ToArray(); - - sw.Restart(); - mre.Set(); - await Task.WhenAll(tasks); - sw.Stop(); - mre.Dispose(); - //Log("ParaSeqRead: " + read.ToString("N0") + " keys in " + sw.Elapsed.TotalSeconds.ToString("N3") + " sec => " + (read / sw.Elapsed.TotalSeconds).ToString("N0") + " kps"); - DumpResult("ParaSeqRead", read, read / 100, sw.Elapsed); - - read = 0; - mre = new ManualResetEvent(false); - tasks = Enumerable - .Range(0, CPUS) - .Select(k => Task.Run(async () => - { - var rndz = new Random(k); - mre.WaitOne(); - - int keys = 0; - for (int j = 0; j < 20; j++) - { - for (int i = 0; i < total / CPUS; i += 100) - { - int pp = i;// rndz.Next((int)total - 100); - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString(pp.ToString(fmt))), - FdbKeySelector.FirstGreaterOrEqual(Slice.FromString((pp + 100).ToString(fmt))) - ).ConfigureAwait(false); - - keys += res.Count; - } - } - } - Interlocked.Add(ref read, keys); - return keys; - })).ToArray(); - - sw.Restart(); - mre.Set(); - await Task.WhenAll(tasks); - sw.Stop(); - mre.Dispose(); - DumpResult("ParaSeqRange", read, read / 100, sw.Elapsed); - #endregion - - DumpMemory(); - - } - - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/Transactions/Comparisons.cs b/FoundationDB.Storage.Memory.Test/Transactions/Comparisons.cs deleted file mode 100644 index 733213ca9..000000000 --- a/FoundationDB.Storage.Memory.Test/Transactions/Comparisons.cs +++ /dev/null @@ -1,138 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API.Tests -{ - using FoundationDB.Client; - using FoundationDB.Storage.Memory.Tests; - using NUnit.Framework; - using System; - using System.Threading.Tasks; - - [TestFixture] - public class Comparisons : FdbTest - { - // Compare the behavior of the MemoryDB against a FoundationDB database - - private async Task Scenario1(IFdbTransaction tr) - { - tr.Set(Slice.FromAscii("hello"), Slice.FromAscii("world!")); - tr.Clear(Slice.FromAscii("removed")); - var result = await tr.GetAsync(Slice.FromAscii("narf")); - } - - private Task Scenario2(IFdbTransaction tr) - { - var location = FdbSubspace.CreateDynamic(Slice.FromAscii("TEST")); - tr.ClearRange(FdbKeyRange.StartsWith(location.Key)); - for (int i = 0; i < 10; i++) - { - tr.Set(location.Keys.Encode(i), Slice.FromString("value of " + i)); - } - return Task.FromResult(null); - } - - private Task Scenario3(IFdbTransaction tr) - { - var location = FdbSubspace.Create(Slice.FromAscii("TEST")); - - tr.Set(location.Key + (byte)'a', Slice.FromAscii("A")); - tr.AtomicAdd(location.Key + (byte)'k', Slice.FromFixed32(1)); - tr.Set(location.Key + (byte)'z', Slice.FromAscii("C")); - tr.ClearRange(location.Key + (byte)'a', location.Key + (byte)'k'); - tr.ClearRange(location.Key + (byte)'k', location.Key + (byte)'z'); - return Task.FromResult(null); - } - - private Task Scenario4(IFdbTransaction tr) - { - var location = FdbSubspace.Create(Slice.FromAscii("TEST")); - - //tr.Set(location.Key, Slice.FromString("NARF")); - //tr.AtomicAdd(location.Key, Slice.FromFixedU32(1)); - tr.AtomicAnd(location.Key, Slice.FromFixedU32(7)); - tr.AtomicXor(location.Key, Slice.FromFixedU32(3)); - tr.AtomicXor(location.Key, Slice.FromFixedU32(15)); - return Task.FromResult(null); - } - - private async Task Scenario5(IFdbTransaction tr) - { - var location = FdbSubspace.CreateDynamic(Slice.FromAscii("TEST")); - - //tr.Set(location.Pack(42), Slice.FromString("42")); - //tr.Set(location.Pack(50), Slice.FromString("50")); - //tr.Set(location.Pack(60), Slice.FromString("60")); - - var x = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(location.Keys.Encode(49))); - Console.WriteLine(x); - - tr.Set(location.Keys.Encode("FOO"), Slice.FromString("BAR")); - - } - - private async Task Scenario6(IFdbTransaction tr) - { - var location = FdbSubspace.CreateDynamic(Slice.FromAscii("TEST")); - - tr.AtomicAdd(location.Keys.Encode("ATOMIC"), Slice.FromFixed32(0x55555555)); - - var x = await tr.GetAsync(location.Keys.Encode("ATOMIC")); - Console.WriteLine(x.ToInt32().ToString("x")); - } - - [Test][Ignore] - public async Task Test_Compare_Implementations() - { - for (int mode = 1; mode <= 6; mode++) - { - - Console.WriteLine("#### SCENARIO " + mode + " ####"); - - using (var db = await Fdb.OpenAsync(this.Cancellation)) - { - using (var tr = db.BeginTransaction(this.Cancellation)) - { - await tr.GetReadVersionAsync(); - - switch (mode) - { - case 1: await Scenario1(tr); break; - case 2: await Scenario2(tr); break; - case 3: await Scenario3(tr); break; - case 4: await Scenario4(tr); break; - case 5: await Scenario5(tr); break; - case 6: await Scenario6(tr); break; - } - - await tr.CommitAsync(); - } - } - - using (var db = MemoryDatabase.CreateNew("DB")) - { - using (var tr = db.BeginTransaction(FdbTransactionMode.Default, this.Cancellation)) - { - await tr.GetReadVersionAsync(); - - switch (mode) - { - case 1: await Scenario1(tr); break; - case 2: await Scenario2(tr); break; - case 3: await Scenario3(tr); break; - case 4: await Scenario4(tr); break; - case 5: await Scenario5(tr); break; - case 6: await Scenario6(tr); break; - } - - await tr.CommitAsync(); - } - - db.Debug_Dump(); - } - } - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/Transactions/MemoryTransactionFacts.cs b/FoundationDB.Storage.Memory.Test/Transactions/MemoryTransactionFacts.cs deleted file mode 100644 index 43de8d3a2..000000000 --- a/FoundationDB.Storage.Memory.Test/Transactions/MemoryTransactionFacts.cs +++ /dev/null @@ -1,1008 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API.Tests -{ - using FoundationDB.Client; - using FoundationDB.Layers.Collections; - using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Indexing; - using FoundationDB.Linq; - using FoundationDB.Storage.Memory.Tests; - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; - using System.Threading.Tasks; - - [TestFixture] - public class MemoryTransactionFacts : FdbTest - { - - [Test] - public async Task Test_Hello_World() - { - using (var db = MemoryDatabase.CreateNew("DB", FdbSubspace.Empty, false)) - { - var key = db.Keys.Encode("hello"); - - // v1 - await db.WriteAsync((tr) => tr.Set(key, Slice.FromString("World!")), this.Cancellation); - db.Debug_Dump(); - var data = await db.ReadAsync((tr) => tr.GetAsync(key), this.Cancellation); - Assert.That(data.ToUnicode(), Is.EqualTo("World!")); - - // v2 - await db.WriteAsync((tr) => tr.Set(key, Slice.FromString("Le Monde!")), this.Cancellation); - db.Debug_Dump(); - data = await db.ReadAsync((tr) => tr.GetAsync(key), this.Cancellation); - Assert.That(data.ToUnicode(), Is.EqualTo("Le Monde!")); - - using (var tr1 = db.BeginTransaction(this.Cancellation)) - { - await tr1.GetReadVersionAsync(); - - await db.WriteAsync((tr2) => tr2.Set(key, Slice.FromString("Sekai!")), this.Cancellation); - db.Debug_Dump(); - - data = await tr1.GetAsync(key); - Assert.That(data.ToUnicode(), Is.EqualTo("Le Monde!")); - } - - data = await db.ReadAsync((tr) => tr.GetAsync(key), this.Cancellation); - Assert.That(data.ToUnicode(), Is.EqualTo("Sekai!")); - - // Collect memory - Trace.WriteLine("### GARBAGE COLLECT! ###"); - db.Collect(); - db.Debug_Dump(); - - } - } - - [Test] - public async Task Test_GetKey() - { - Slice key; - Slice value; - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(location.Encode(0), Slice.FromString("first")); - tr.Set(location.Encode(10), Slice.FromString("ten")); - tr.Set(location.Encode(20), Slice.FromString("ten ten")); - tr.Set(location.Encode(42), Slice.FromString("narf!")); - tr.Set(location.Encode(100), Slice.FromString("a hundred missipis")); - await tr.CommitAsync(); - } - - db.Debug_Dump(); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - value = await tr.GetAsync(location.Encode(42)); - Console.WriteLine(value); - Assert.That(value.ToString(), Is.EqualTo("narf!")); - - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(location.Encode(42))); - Assert.That(key, Is.EqualTo(location.Encode(42))); - - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(location.Encode(42))); - Assert.That(key, Is.EqualTo(location.Encode(100))); - - key = await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(location.Encode(42))); - Assert.That(key, Is.EqualTo(location.Encode(42))); - - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(location.Encode(42))); - Assert.That(key, Is.EqualTo(location.Encode(20))); - - var keys = await tr.GetKeysAsync(new[] - { - FdbKeySelector.FirstGreaterOrEqual(location.Encode(42)), - FdbKeySelector.FirstGreaterThan(location.Encode(42)), - FdbKeySelector.LastLessOrEqual(location.Encode(42)), - FdbKeySelector.LastLessThan(location.Encode(42)) - }); - - Assert.That(keys.Length, Is.EqualTo(4)); - Assert.That(keys[0], Is.EqualTo(location.Encode(42))); - Assert.That(keys[1], Is.EqualTo(location.Encode(100))); - Assert.That(keys[2], Is.EqualTo(location.Encode(42))); - Assert.That(keys[3], Is.EqualTo(location.Encode(20))); - - await tr.CommitAsync(); - } - - } - - } - - [Test] - public async Task Test_GetKey_ReadConflicts() - { - Slice key; - - using (var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(location.Encode(42), Slice.FromString("42")); - tr.Set(location.Encode(50), Slice.FromString("50")); - tr.Set(location.Encode(60), Slice.FromString("60")); - await tr.CommitAsync(); - } - db.Debug_Dump(); - - Func check = async (selector, expected) => - { - using (var tr = db.BeginTransaction(this.Cancellation)) - { - key = await tr.GetKeyAsync(selector); - await tr.CommitAsync(); - Assert.That(key, Is.EqualTo(expected), selector.ToString() + " => " + FdbKey.Dump(expected)); - } - }; - - await check( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(50)), - location.Encode(50) - ); - await check( - FdbKeySelector.FirstGreaterThan(location.Encode(50)), - location.Encode(60) - ); - - await check( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(49)), - location.Encode(50) - ); - await check( - FdbKeySelector.FirstGreaterThan(location.Encode(49)), - location.Encode(50) - ); - - await check( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(49)) + 1, - location.Encode(60) - ); - await check( - FdbKeySelector.FirstGreaterThan(location.Encode(49)) + 1, - location.Encode(60) - ); - - await check( - FdbKeySelector.LastLessOrEqual(location.Encode(49)), - location.Encode(42) - ); - await check( - FdbKeySelector.LastLessThan(location.Encode(49)), - location.Encode(42) - ); - } - } - - [Test] - public async Task Test_GetRangeAsync() - { - Slice key; - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - for (int i = 0; i <= 100; i++) - { - tr.Set(location.Encode(i), Slice.FromString("value of " + i)); - } - await tr.CommitAsync(); - } - - db.Debug_Dump(); - - // verify that key selectors work find - using (var tr = db.BeginTransaction(this.Cancellation)) - { - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(FdbKey.MaxValue)); - if (key != FdbKey.MaxValue) Assert.Inconclusive("Key selectors are buggy: fGE(max)"); - key = await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(FdbKey.MaxValue)); - if (key != FdbKey.MaxValue) Assert.Inconclusive("Key selectors are buggy: lLE(max)"); - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(FdbKey.MaxValue)); - if (key != location.Encode(100)) Assert.Inconclusive("Key selectors are buggy: lLT(max)"); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var chunk = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(0)), - FdbKeySelector.FirstGreaterOrEqual(location.Encode(50)) - ); -#if DEBUG - for (int i = 0; i < chunk.Count; i++) - { - Console.WriteLine(i.ToString() + " : " + chunk.Chunk[i].Key + " = " + chunk.Chunk[i].Value); - } -#endif - - Assert.That(chunk.Count, Is.EqualTo(50), "chunk.Count"); - Assert.That(chunk.HasMore, Is.False, "chunk.HasMore"); - Assert.That(chunk.Reversed, Is.False, "chunk.Reversed"); - Assert.That(chunk.Iteration, Is.EqualTo(1), "chunk.Iteration"); - - for (int i = 0; i < 50; i++) - { - Assert.That(chunk.Chunk[i].Key, Is.EqualTo(location.Encode(i)), "[{0}].Key", i); - Assert.That(chunk.Chunk[i].Value.ToString(), Is.EqualTo("value of " + i), "[{0}].Value", i); - } - - await tr.CommitAsync(); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var chunk = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(0)), - FdbKeySelector.FirstGreaterOrEqual(location.Encode(50)), - new FdbRangeOptions { Reverse = true } - ); -#if DEBUG - for (int i = 0; i < chunk.Count; i++) - { - Console.WriteLine(i.ToString() + " : " + chunk.Chunk[i].Key + " = " + chunk.Chunk[i].Value); - } -#endif - - Assert.That(chunk.Count, Is.EqualTo(50), "chunk.Count"); - Assert.That(chunk.HasMore, Is.False, "chunk.HasMore"); - Assert.That(chunk.Reversed, Is.True, "chunk.Reversed"); - Assert.That(chunk.Iteration, Is.EqualTo(1), "chunk.Iteration"); - - for (int i = 0; i < 50; i++) - { - Assert.That(chunk.Chunk[i].Key, Is.EqualTo(location.Encode(49 - i)), "[{0}].Key", i); - Assert.That(chunk.Chunk[i].Value.ToString(), Is.EqualTo("value of " + (49 - i)), "[{0}].Value", i); - } - - await tr.CommitAsync(); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var chunk = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(location.Encode(0)), - FdbKeySelector.FirstGreaterOrEqual(FdbKey.MaxValue), - new FdbRangeOptions { Reverse = true, Limit = 1 } - ); -#if DEBUG - for (int i = 0; i < chunk.Count; i++) - { - Console.WriteLine(i.ToString() + " : " + chunk.Chunk[i].Key + " = " + chunk.Chunk[i].Value); - } -#endif - - Assert.That(chunk.Count, Is.EqualTo(1), "chunk.Count"); - Assert.That(chunk.HasMore, Is.True, "chunk.HasMore"); - Assert.That(chunk.Reversed, Is.True, "chunk.Reversed"); - Assert.That(chunk.Iteration, Is.EqualTo(1), "chunk.Iteration"); - - await tr.CommitAsync(); - } - - } - - } - - [Test] - public async Task Test_GetRange() - { - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - for (int i = 0; i <= 100; i++) - { - tr.Set(location.Encode(i), Slice.FromString("value of " + i)); - } - await tr.CommitAsync(); - } - - db.Debug_Dump(); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var results = await tr - .GetRange(location.Encode(0), location.Encode(50)) - .ToListAsync(); - - Assert.That(results, Is.Not.Null); -#if DEBUG - for (int i = 0; i < results.Count; i++) - { - Console.WriteLine(i.ToString() + " : " + results[i].Key + " = " + results[i].Value); - } -#endif - - Assert.That(results.Count, Is.EqualTo(50)); - for (int i = 0; i < 50; i++) - { - Assert.That(results[i].Key, Is.EqualTo(location.Encode(i)), "[{0}].Key", i); - Assert.That(results[i].Value.ToString(), Is.EqualTo("value of " + i), "[{0}].Value", i); - } - - await tr.CommitAsync(); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - - var results = await tr - .GetRange(location.Encode(0), location.Encode(50), new FdbRangeOptions { Reverse = true }) - .ToListAsync(); - Assert.That(results, Is.Not.Null); -#if DEBUG - for (int i = 0; i < results.Count; i++) - { - Console.WriteLine(i.ToString() + " : " + results[i].Key + " = " + results[i].Value); - } -#endif - - Assert.That(results.Count, Is.EqualTo(50)); - for (int i = 0; i < 50; i++) - { - Assert.That(results[i].Key, Is.EqualTo(location.Encode(49 - i)), "[{0}].Key", i); - Assert.That(results[i].Value.ToString(), Is.EqualTo("value of " + (49 - i)), "[{0}].Value", i); - } - - await tr.CommitAsync(); - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - var result = await tr - .GetRange(location.Encode(0), FdbKey.MaxValue, new FdbRangeOptions { Reverse = true }) - .FirstOrDefaultAsync(); - -#if DEBUG - Console.WriteLine(result.Key + " = " + result.Value); -#endif - Assert.That(result.Key, Is.EqualTo(location.Encode(100))); - Assert.That(result.Value.ToString(), Is.EqualTo("value of 100")); - - await tr.CommitAsync(); - } - - } - - } - - [Test] - public async Task Test_CommittedVersion_On_ReadOnly_Transactions() - { - //note: until CommitAsync() is called, the value of the committed version is unspecified, but current implementation returns -1 - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - long ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.EqualTo(-1), "Initial committed version"); - - var _ = await tr.GetAsync(location.Encode("foo")); - - // until the transction commits, the committed version will stay -1 - ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.EqualTo(-1), "Committed version after a single read"); - - // committing a read only transaction - - await tr.CommitAsync(); - - ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.EqualTo(-1), "Read-only comiitted transaction have a committed version of -1"); - } - - db.Debug_Dump(); - } - } - - [Test] - public async Task Test_CommittedVersion_On_Write_Transactions() - { - //note: until CommitAsync() is called, the value of the committed version is unspecified, but current implementation returns -1 - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - // take the read version (to compare with the committed version below) - long readVersion = await tr.GetReadVersionAsync(); - - long ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.EqualTo(-1), "Initial committed version"); - - tr.Set(location.Encode("foo"), Slice.FromString("bar")); - - // until the transction commits, the committed version should still be -1 - ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.EqualTo(-1), "Committed version after a single write"); - - // committing a read only transaction - - await tr.CommitAsync(); - - ver = tr.GetCommittedVersion(); - Assert.That(ver, Is.GreaterThanOrEqualTo(readVersion), "Committed version of write transaction should be >= the read version"); - } - - db.Debug_Dump(); - } - } - - [Test] - public async Task Test_CommittedVersion_After_Reset() - { - //note: until CommitAsync() is called, the value of the committed version is unspecified, but current implementation returns -1 - - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - // take the read version (to compare with the committed version below) - long rv1 = await tr.GetReadVersionAsync(); - // do something and commit - tr.Set(location.Encode("foo"), Slice.FromString("bar")); - await tr.CommitAsync(); - long cv1 = tr.GetCommittedVersion(); - Console.WriteLine("COMMIT: " + rv1 + " / " + cv1); - Assert.That(cv1, Is.GreaterThanOrEqualTo(rv1), "Committed version of write transaction should be >= the read version"); - - // reset the transaction - tr.Reset(); - - long rv2 = await tr.GetReadVersionAsync(); - long cv2 = tr.GetCommittedVersion(); - Console.WriteLine("RESET: " + rv2 + " / " + cv2); - //Note: the current fdb_c client does not revert the commited version to -1 ... ? - //Assert.That(cv2, Is.EqualTo(-1), "Committed version should go back to -1 after reset"); - - // read-only + commit - await tr.GetAsync(location.Encode("foo")); - await tr.CommitAsync(); - cv2 = tr.GetCommittedVersion(); - Console.WriteLine("COMMIT2: " + rv2 + " / " + cv2); - Assert.That(cv2, Is.EqualTo(-1), "Committed version of read-only transaction should be -1 even the transaction was previously used to write something"); - - } - } - } - - [Test] - public async Task Test_Conflicts() - { - - // this SHOULD NOT conflict - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr1 = db.BeginTransaction(this.Cancellation)) - { - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - tr2.Set(location.Encode("foo"), Slice.FromString("changed")); - await tr2.CommitAsync(); - } - - var x = await tr1.GetAsync(location.Encode("foo")); - tr1.Set(location.Encode("bar"), Slice.FromString("other")); - - await tr1.CommitAsync(); - } - - } - - // this SHOULD conflict - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr1 = db.BeginTransaction(this.Cancellation)) - { - var x = await tr1.GetAsync(location.Encode("foo")); - - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - tr2.Set(location.Encode("foo"), Slice.FromString("changed")); - await tr2.CommitAsync(); - } - - tr1.Set(location.Encode("bar"), Slice.FromString("other")); - - Assert.That(async () => await tr1.CommitAsync(), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.NotCommitted)); - } - - } - - // this SHOULD conflict - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr1 = db.BeginTransaction(this.Cancellation)) - { - await tr1.GetReadVersionAsync(); - - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - tr2.Set(location.Encode("foo"), Slice.FromString("changed")); - await tr2.CommitAsync(); - } - - var x = await tr1.GetAsync(location.Encode("foo")); - tr1.Set(location.Encode("bar"), Slice.FromString("other")); - - Assert.That(async () => await tr1.CommitAsync(), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.NotCommitted)); - } - - } - - // this SHOULD NOT conflict - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - using (var tr1 = db.BeginTransaction(this.Cancellation)) - { - var x = await tr1.Snapshot.GetAsync(location.Encode("foo")); - - using (var tr2 = db.BeginTransaction(this.Cancellation)) - { - tr2.Set(location.Encode("foo"), Slice.FromString("changed")); - await tr2.CommitAsync(); - } - - tr1.Set(location.Encode("bar"), Slice.FromString("other")); - - await tr1.CommitAsync(); - } - - } - } - - [Test] - public async Task Test_Write_Then_Read() - { - using (var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.Keys; - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("hello"), Slice.FromString("World!")); - tr.AtomicAdd(Slice.FromString("counter"), Slice.FromFixed32(1)); - tr.Set(Slice.FromString("foo"), Slice.FromString("bar")); - await tr.CommitAsync(); - } - - db.Debug_Dump(); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - var result = await tr.GetAsync(Slice.FromString("hello")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToString(), Is.EqualTo("World!")); - - result = await tr.GetAsync(Slice.FromString("counter")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToInt32(), Is.EqualTo(1)); - - result = await tr.GetAsync(Slice.FromString("foo")); - Assert.That(result.ToString(), Is.EqualTo("bar")); - - } - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("hello"), Slice.FromString("Le Monde!")); - tr.AtomicAdd(Slice.FromString("counter"), Slice.FromFixed32(1)); - tr.Set(Slice.FromString("narf"), Slice.FromString("zort")); - await tr.CommitAsync(); - } - - db.Debug_Dump(); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - var result = await tr.GetAsync(Slice.FromString("hello")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToString(), Is.EqualTo("Le Monde!")); - - result = await tr.GetAsync(Slice.FromString("counter")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToInt32(), Is.EqualTo(2)); - - result = await tr.GetAsync(Slice.FromString("foo")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToString(), Is.EqualTo("bar")); - - result = await tr.GetAsync(Slice.FromString("narf")); - Assert.That(result, Is.Not.Null); - Assert.That(result.ToString(), Is.EqualTo("zort")); - } - - // Collect memory - Trace.WriteLine("### GARBAGE COLLECT! ###"); - db.Collect(); - db.Debug_Dump(); - } - } - - [Test] - public async Task Test_Atomic() - { - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.Keys; - - var key1 = location.Encode(1); - var key2 = location.Encode(2); - var key16 = location.Encode(16); - - for (int i = 0; i < 10; i++) - { - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.AtomicAdd(key1, Slice.FromFixed64(1)); - tr.AtomicAdd(key2, Slice.FromFixed64(2)); - tr.AtomicAdd(key16, Slice.FromFixed64(16)); - - await tr.CommitAsync(); - } - } - - db.Debug_Dump(); - - // Collect memory - Trace.WriteLine("### GARBAGE COLLECT! ###"); - db.Collect(); - db.Debug_Dump(); - } - } - - [Test] - public async Task Test_Use_Simple_Layer() - { - using (var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.GlobalSpace; - - var map = new FdbMap("Foos", db.GlobalSpace.Partition.ByKey("Foos"), KeyValueEncoders.Values.StringEncoder); - var index = new FdbIndex("Foos.ByColor", db.GlobalSpace.Partition.ByKey("Foos", "Color")); - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - map.Set(tr, 3, @"{ ""name"": ""Juliet"", ""color"": ""red"" }"); - map.Set(tr, 2, @"{ ""name"": ""Joey"", ""color"": ""blue"" }"); - map.Set(tr, 1, @"{ ""name"": ""Bob"", ""color"": ""red"" }"); - - index.Add(tr, 3, "red"); - index.Add(tr, 2, "blue"); - index.Add(tr, 1, "red"); - - await tr.CommitAsync(); - } - - db.Debug_Dump(true); - - //// Collect memory - //Trace.WriteLine("### GARBAGE COLLECT! ###"); - //db.Collect(); - //db.Debug_Dump(); - } - } - - [Test] - public async Task Test_Use_Directory_Layer() - { - using (var db = MemoryDatabase.CreateNew("DB")) - { - var location = db.GlobalSpace; - - var foos = await db.Directory.CreateOrOpenAsync("Foos", this.Cancellation); - var bars = await db.Directory.CreateOrOpenAsync("Bars", this.Cancellation); - - var foo123 = await db.Directory.CreateOrOpenAsync(new[] { "Foos", "123" }, this.Cancellation); - var bar456 = await bars.CreateOrOpenAsync(db, new[] { "123" }, this.Cancellation); - - db.Debug_Dump(true); - - //// Collect memory - //Trace.WriteLine("### GARBAGE COLLECT! ###"); - //db.Collect(); - //db.Debug_Dump(); - } - } - - [Test] - public async Task Test_Can_Resolve_Key_Selector_Outside_Boundaries() - { - // test various corner cases: - - // - k < first_key or k <= <00> resolves to: - // - '' always - - // - k > last_key or k >= resolve to: - // - '' when access to system keys is off - // - '/backupRange' (usually) when access to system keys is ON - - // - k >= <00> resolves to: - // - key_outside_legal_range when access to system keys is off - // - '/backupRange' (usually) when access to system keys is ON - - // - k >= resolved to: - // - key_outside_legal_range when access to system keys is off - // - '' when access to system keys is ON - - Slice key; - - using (var db = MemoryDatabase.CreateNew("FOO")) - { - - using (var tr = db.BeginTransaction(this.Cancellation)) - { - tr.Set(Slice.FromString("A"), Slice.FromString("min")); - tr.Set(Slice.FromString("Z"), Slice.FromString("max")); - await tr.CommitAsync(); - } - - using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) - { - // before <00> - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(FdbKey.MinValue)); - Assert.That(key, Is.EqualTo(Slice.Empty), "lLT(<00>) => ''"); - - // before the first key in the db - var minKey = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(FdbKey.MinValue)); - Assert.That(minKey, Is.Not.Null); - Console.WriteLine("minKey = " + minKey); - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(minKey)); - Assert.That(key, Is.EqualTo(Slice.Empty), "lLT(min_key) => ''"); - - // after the last key in the db - - var maxKey = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(FdbKey.MaxValue)); - Assert.That(maxKey, Is.Not.Null); - Console.WriteLine("maxKey = " + maxKey); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(maxKey)); - Assert.That(key, Is.EqualTo(FdbKey.MaxValue), "fGT(maxKey) => "); - - // after - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(FdbKey.MaxValue)); - Assert.That(key, Is.EqualTo(FdbKey.MaxValue), "fGT() => "); - Assert.That(async () => await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(Slice.FromAscii("\xFF\xFF"))), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); - Assert.That(async () => await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromAscii("\xFF\x00"))), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); - - tr.WithReadAccessToSystemKeys(); - - var firstSystemKey = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(FdbKey.MaxValue)); - // usually the first key in the system space is /backupDataFormat, but that may change in the future version. - Assert.That(firstSystemKey, Is.Not.Null); - Assert.That(firstSystemKey, Is.GreaterThan(FdbKey.MaxValue), "key should be between and "); - Assert.That(firstSystemKey, Is.LessThan(Slice.FromAscii("\xFF\xFF")), "key should be between and "); - - // with access to system keys, the maximum possible key becomes - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Slice.FromAscii("\xFF\xFF"))); - Assert.That(key, Is.EqualTo(Slice.FromAscii("\xFF\xFF")), "fGE() => (with access to system keys)"); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(Slice.FromAscii("\xFF\xFF"))); - Assert.That(key, Is.EqualTo(Slice.FromAscii("\xFF\xFF")), "fGT() => (with access to system keys)"); - - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Slice.FromAscii("\xFF\x00"))); - Assert.That(key, Is.EqualTo(maxKey), "lLT(<00>) => max_key (with access to system keys)"); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(maxKey)); - Assert.That(key, Is.EqualTo(firstSystemKey), "fGT(max_key) => first_system_key (with access to system keys)"); - - } - } - - } - - [Test] - public async Task Test_Can_BulkLoad_Data_Ordered() - { - const int N = 1 * 1000 * 1000; - - // insert N sequential items and bulk load with "ordered = true" to skip the sorting of levels - - Console.WriteLine("Warmup..."); - using (var db = MemoryDatabase.CreateNew("WARMUP")) - { - await db.BulkLoadAsync(Enumerable.Range(0, 100).Select(i => new KeyValuePair(db.Keys.Encode(i), Slice.FromFixed32(i))).ToList(), ordered: true); - } - - using(var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.Keys; - - Console.WriteLine("Generating " + N.ToString("N0") + " keys..."); - var data = new KeyValuePair[N]; - for (int i = 0; i < N; i++) - { - data[i] = new KeyValuePair( - location.Encode(i), - Slice.FromFixed32(i) - ); - } - Console.WriteLine("Inserting ..."); - - var sw = Stopwatch.StartNew(); - await db.BulkLoadAsync(data, ordered: true); - sw.Stop(); - DumpResult("BulkLoadSeq", N, 1, sw.Elapsed); - - db.Debug_Dump(); - - var rnd = new Random(); - for (int i = 0; i < 100 * 1000; i++) - { - int x = rnd.Next(N); - using (var tx = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tx.GetAsync(location.Encode(x)).ConfigureAwait(false); - Assert.That(res.ToInt32(), Is.EqualTo(x)); - } - } - - } - } - - [Test] - public async Task Test_Can_BulkLoad_Data_Sequential_Unordered() - { - const int N = 1 * 1000 * 1000; - - // insert N sequential items, but without specifying "ordered = true" to force a sort of all levels - - Console.WriteLine("Warmup..."); - using(var db = MemoryDatabase.CreateNew("WARMUP")) - { - await db.BulkLoadAsync(Enumerable.Range(0, 100).Select(i => new KeyValuePair(db.Keys.Encode(i), Slice.FromFixed32(i))).ToList(), ordered: false); - } - - using (var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.Keys; - - Console.WriteLine("Generating " + N.ToString("N0") + " keys..."); - var data = new KeyValuePair[N]; - var rnd = new Random(); - for (int i = 0; i < N; i++) - { - data[i] = new KeyValuePair( - location.Encode(i), - Slice.FromFixed32(i) - ); - } - - Console.WriteLine("Inserting ..."); - var sw = Stopwatch.StartNew(); - await db.BulkLoadAsync(data, ordered: false); - sw.Stop(); - DumpResult("BulkLoadSeqSort", N, 1, sw.Elapsed); - - db.Debug_Dump(); - - for (int i = 0; i < 100 * 1000; i++) - { - int x = rnd.Next(N); - using (var tx = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tx.GetAsync(location.Encode(x)).ConfigureAwait(false); - Assert.That(res.ToInt32(), Is.EqualTo(x)); - } - } - - } - } - - [Test] - public async Task Test_Can_BulkLoad_Data_Random_Unordered() - { - const int N = 1 * 1000 * 1000; - - // insert N randomized items - - Console.WriteLine("Warmup..."); - using (var db = MemoryDatabase.CreateNew("WARMUP")) - { - await db.BulkLoadAsync(Enumerable.Range(0, 100).Select(i => new KeyValuePair(db.Keys.Encode(i), Slice.FromFixed32(i))).ToList(), ordered: false); - } - - using (var db = MemoryDatabase.CreateNew("FOO")) - { - var location = db.Keys; - - Console.WriteLine("Generating " + N.ToString("N0") + " keys..."); - var data = new KeyValuePair[N]; - var ints = new int[N]; - var rnd = new Random(); - for (int i = 0; i < N; i++) - { - data[i] = new KeyValuePair( - location.Encode(i), - Slice.FromFixed32(i) - ); - ints[i] = rnd.Next(int.MaxValue); - } - Console.WriteLine("Shuffling..."); - Array.Sort(ints, data); - - Console.WriteLine("Inserting ..."); - - var sw = Stopwatch.StartNew(); - await db.BulkLoadAsync(data, ordered: false); - sw.Stop(); - DumpResult("BulkLoadRndSort", N, 1, sw.Elapsed); - - db.Debug_Dump(); - - for (int i = 0; i < 100 * 1000; i++) - { - int x = rnd.Next(N); - using (var tx = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tx.GetAsync(location.Encode(x)).ConfigureAwait(false); - Assert.That(res.ToInt32(), Is.EqualTo(x)); - } - } - - } - } - - private static void DumpResult(string label, long total, long trans, TimeSpan elapsed) - { - Console.WriteLine( - "{0,-12}: {1, 10} keys in {2,4} sec => {3,9} kps, {4,7} tps", - label, - total.ToString("N0"), - elapsed.TotalSeconds.ToString("N3"), - (total / elapsed.TotalSeconds).ToString("N0"), - (trans / elapsed.TotalSeconds).ToString("N0") - ); - } - - private static void DumpMemory(bool collect = false) - { - if (collect) - { - GC.Collect(); - GC.WaitForPendingFinalizers(); - GC.Collect(); - } - Console.WriteLine("Total memory: Managed=" + (GC.GetTotalMemory(false) / 1024.0).ToString("N1") + " kB, WorkingSet=" + (Environment.WorkingSet / 1024.0).ToString("N1") + " kB"); - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/Transactions/SnapshotFacts.cs b/FoundationDB.Storage.Memory.Test/Transactions/SnapshotFacts.cs deleted file mode 100644 index a5b23b6b2..000000000 --- a/FoundationDB.Storage.Memory.Test/Transactions/SnapshotFacts.cs +++ /dev/null @@ -1,110 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API.Tests -{ - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using FoundationDB.Storage.Memory.Tests; - using FoundationDB.Linq; - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Threading.Tasks; - using System.Linq; - using System.IO; - - [TestFixture] - public class SnapshotFacts : FdbTest - { - - [Test] - public async Task Test_Can_Save_And_Reload_Snapshot() - { - const string FILE_PATH = ".\\test.pndb"; - const int N = 1 * 1000 * 1000; - - if (File.Exists(FILE_PATH)) File.Delete(FILE_PATH); - - // insert N sequential items and bulk load with "ordered = true" to skip the sorting of levels - - Console.WriteLine("Generating " + N.ToString("N0") + " keys..."); - var data = new KeyValuePair[N]; - var rnd = new Random(); - for (int i = 0; i < N; i++) - { - data[i] = new KeyValuePair( - Slice.FromAscii(i.ToString("D16")), - Slice.Random(rnd, 50) - ); - } - - var sw = new Stopwatch(); - - using (var db = MemoryDatabase.CreateNew()) - { - Console.Write("Inserting ..."); - sw.Restart(); - await db.BulkLoadAsync(data, ordered: true); - sw.Stop(); - Console.WriteLine(" done in " + sw.Elapsed.TotalSeconds.ToString("N1") + " secs"); - - db.Debug_Dump(); - - Console.Write("Saving..."); - sw.Restart(); - await db.SaveSnapshotAsync(FILE_PATH, null, this.Cancellation); - sw.Stop(); - Console.WriteLine(" done in " + sw.Elapsed.TotalSeconds.ToString("N1") + " secs"); - } - - var fi = new FileInfo(FILE_PATH); - Assert.That(fi.Exists, Is.True, "Snapshot file not found"); - Console.WriteLine("File size is " + fi.Length.ToString("N0") + " bytes (" + (fi.Length * 1.0d / N).ToString("N2") + " bytes/item, " + (fi.Length / (1048576.0 * sw.Elapsed.TotalSeconds)).ToString("N3") + " MB/sec)"); - - Console.Write("Loading..."); - sw.Restart(); - using (var db = await MemoryDatabase.LoadFromAsync(FILE_PATH, this.Cancellation)) - { - sw.Stop(); - Console.WriteLine(" done in " + sw.Elapsed.TotalSeconds.ToString("N1") + " secs (" + (fi.Length / (1048576.0 * sw.Elapsed.TotalSeconds)).ToString("N0") + " MB/sec)"); - db.Debug_Dump(); - - Console.WriteLine("Checking data integrity..."); - sw.Restart(); - long n = 0; - foreach (var batch in data.Buffered(50 * 1000)) - { - using (var tx = db.BeginReadOnlyTransaction(this.Cancellation)) - { - var res = await tx - .Snapshot - .GetRange( - FdbKeySelector.FirstGreaterOrEqual(batch[0].Key), - FdbKeySelector.FirstGreaterThan(batch[batch.Count - 1].Key)) - .ToListAsync() - .ConfigureAwait(false); - - Assert.That(res.Count, Is.EqualTo(batch.Count), "Some keys are missing from {0} to {1} :(", batch[0], batch[batch.Count - 1]); - - for (int i = 0; i < res.Count; i++) - { - // note: Is.EqualTo(...) is slow on Slices so we speed things a bit - if (res[i].Key != batch[i].Key) Assert.That(res[i].Key, Is.EqualTo(batch[i].Key), "Key is different :("); - if (res[i].Value != batch[i].Value) Assert.That(res[i].Value, Is.EqualTo(batch[i].Value), "Value is different for key {0} :(", batch[i].Key); - } - } - n += batch.Count; - Console.Write("\r" + n.ToString("N0")); - } - sw.Stop(); - Console.WriteLine(" done in " + sw.Elapsed.TotalSeconds.ToString("N1") + " secs"); - } - - Console.WriteLine("Content of database are identical ^_^"); - } - - } -} diff --git a/FoundationDB.Storage.Memory.Test/packages.config b/FoundationDB.Storage.Memory.Test/packages.config deleted file mode 100644 index c714ef3a2..000000000 --- a/FoundationDB.Storage.Memory.Test/packages.config +++ /dev/null @@ -1,4 +0,0 @@ - - - - \ No newline at end of file diff --git a/FoundationDB.Storage.Memory/API/LevelWriter.cs b/FoundationDB.Storage.Memory/API/LevelWriter.cs deleted file mode 100644 index 6155d3c02..000000000 --- a/FoundationDB.Storage.Memory/API/LevelWriter.cs +++ /dev/null @@ -1,82 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API -{ - using FoundationDB.Client; - using FoundationDB.Storage.Memory.Core; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - - /// Helper class to add key/value pairs to a level - /// This class is not thread-safe - internal sealed class LevelWriter : IDisposable - { - - private readonly UnmanagedSliceBuilder m_scratch = new UnmanagedSliceBuilder(128 * 1024); // > 80KB will go to the LOH - private readonly List m_list; - private readonly KeyHeap m_keys; - private readonly ValueHeap m_values; - - public LevelWriter(int count, KeyHeap keyHeap, ValueHeap valueHeap) - { - Contract.Requires(count > 0 && keyHeap != null && valueHeap != null); - m_keys = keyHeap; - m_values = valueHeap; - m_list = new List(count); - } - - public List Data { get { return m_list; } } - - public unsafe void Add(ulong sequence, KeyValuePair current) - { - // allocate the key - var tmp = MemoryDatabaseHandler.PackUserKey(m_scratch, current.Key); - Key* key = m_keys.Append(tmp); - Contract.Assert(key != null, "key == null"); - - // allocate the value - Slice userValue = current.Value; - uint size = checked((uint)userValue.Count); - Value* value = m_values.Allocate(size, sequence, null, key); - Contract.Assert(value != null, "value == null"); - UnmanagedHelpers.CopyUnsafe(&(value->Data), userValue); - - key->Values = value; - - m_list.Add(new IntPtr(key)); - } - - public unsafe void Add(ulong sequence, USlice userKey, USlice userValue) - { - // allocate the key - var tmp = MemoryDatabaseHandler.PackUserKey(m_scratch, userKey); - Key* key = m_keys.Append(tmp); - Contract.Assert(key != null, "key == null"); - - // allocate the value - uint size = userValue.Count; - Value* value = m_values.Allocate(size, sequence, null, key); - Contract.Assert(value != null, "value == null"); - UnmanagedHelpers.CopyUnsafe(&(value->Data), userValue); - - key->Values = value; - - m_list.Add(new IntPtr(key)); - } - - public void Reset() - { - m_list.Clear(); - } - - public void Dispose() - { - m_scratch.Dispose(); - } - } - -} diff --git a/FoundationDB.Storage.Memory/API/MemoryClusterHandler.cs b/FoundationDB.Storage.Memory/API/MemoryClusterHandler.cs deleted file mode 100644 index ff33c9828..000000000 --- a/FoundationDB.Storage.Memory/API/MemoryClusterHandler.cs +++ /dev/null @@ -1,64 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API -{ - using FoundationDB.Client; - using FoundationDB.Client.Core; - using System; - using System.Threading; - using System.Threading.Tasks; - - internal class MemoryClusterHandler : IFdbClusterHandler, IDisposable - { - - private bool m_disposed; - - public MemoryClusterHandler() - { - //TODO ? - } - - public bool IsInvalid - { - get { return false; } - } - - public bool IsClosed - { - get { return m_disposed; } - } - - public void SetOption(FdbClusterOption option, Slice data) - { - throw new NotImplementedException(); - } - - internal MemoryDatabaseHandler OpenDatabase(Guid uid) - { - return new MemoryDatabaseHandler(uid); - } - - public Task OpenDatabaseAsync(string databaseName, CancellationToken cancellationToken) - { - // fdb currently disallow anthing other than "DB" - if (databaseName != null && databaseName != "DB") throw new FdbException(FdbError.InvalidDatabaseName); - - var uid = Guid.NewGuid(); - return Task.FromResult(OpenDatabase(uid)); - } - - public void Dispose() - { - if (!m_disposed) - { - m_disposed = true; - //TODO - } - - GC.SuppressFinalize(this); - } - } - -} diff --git a/FoundationDB.Storage.Memory/API/MemoryDatabase.cs b/FoundationDB.Storage.Memory/API/MemoryDatabase.cs deleted file mode 100644 index 8f47adcdf..000000000 --- a/FoundationDB.Storage.Memory/API/MemoryDatabase.cs +++ /dev/null @@ -1,136 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API -{ - using FoundationDB.Client; - using FoundationDB.Layers.Directories; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; - using System.Threading; - using System.Threading.Tasks; - - /// In-memory database instance - public class MemoryDatabase : FdbDatabase - { - - #region Static Helpers... - - public static MemoryDatabase CreateNew() - { - return CreateNew("DB", FdbSubspace.Empty, false); - } - - public static MemoryDatabase CreateNew(string name) - { - return CreateNew(name, FdbSubspace.Empty, false); - } - - public static MemoryDatabase CreateNew(string name, IFdbSubspace globalSpace, bool readOnly) - { - globalSpace = globalSpace ?? FdbSubspace.Empty; - var uid = Guid.NewGuid(); - - MemoryClusterHandler cluster = null; - MemoryDatabaseHandler db = null; - try - { - cluster = new MemoryClusterHandler(); - db = cluster.OpenDatabase(uid); - - // initialize the system keys for this new db - db.PopulateSystemKeys(); - - return new MemoryDatabase(new FdbCluster(cluster, ":memory:"), db, name, globalSpace, null, readOnly, true); - } - catch - { - if (db != null) db.Dispose(); - if (cluster != null) cluster.Dispose(); - throw; - } - } - - public static async Task LoadFromAsync(string path, CancellationToken cancellationToken) - { - cancellationToken.ThrowIfCancellationRequested(); - - MemoryClusterHandler cluster = null; - MemoryDatabaseHandler db = null; - try - { - cluster = new MemoryClusterHandler(); - db = cluster.OpenDatabase(Guid.Empty); - - // load the snapshot from the disk - var options = new MemorySnapshotOptions(); //TODO! - await db.LoadSnapshotAsync(path, options, cancellationToken); - - return new MemoryDatabase(new FdbCluster(cluster, ":memory:"), db, "DB", FdbSubspace.Empty, null, false, true); - } - catch(Exception) - { - if (db != null) db.Dispose(); - if (cluster != null) cluster.Dispose(); - throw; - } - } - - #endregion - - private readonly MemoryDatabaseHandler m_handler; - - private MemoryDatabase(IFdbCluster cluster, MemoryDatabaseHandler handler, string name, IFdbSubspace globalSpace, IFdbDirectory directory, bool readOnly, bool ownsCluster) - : base(cluster, handler, name, globalSpace, directory, readOnly, ownsCluster) - { - m_handler = handler; - } - - [Conditional("DEBUG")] - public void Debug_Dump(bool detailed = false) - { - m_handler.Debug_Dump(detailed); - } - - /// Trigger a garbage collection of the memory database - /// If the amount of memory that can be collected is too small, this operation will do nothing. - public void Collect() - { - m_handler.Collect(); - } - - /// Replace the content of the database with existing data. - /// Data that will replace the content of the database. The elements do not need to be sorted, but best performance is achieved if all the keys are lexicographically ordered (smallest to largest) - /// Optionnal cancellation token - /// Task that completes then the data has been loaded into the database - /// Any pre-existing data will be removed! - public Task BulkLoadAsync(IEnumerable> data, bool ordered = false, CancellationToken cancellationToken = default(CancellationToken)) - { - if (data == null) throw new ArgumentNullException("data"); - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - - var coll = data as ICollection> ?? data.ToList(); - - return m_handler.BulkLoadAsync(coll, ordered, false, cancellationToken); - } - - public Task SaveSnapshotAsync(string path, MemorySnapshotOptions options = null, CancellationToken cancellationToken = default(CancellationToken)) - { - if (path == null) throw new ArgumentNullException("path"); - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - - options = options ?? new MemorySnapshotOptions() - { - Mode = MemorySnapshotMode.Full - }; - - return m_handler.SaveSnapshotAsync(path, options, cancellationToken); - } - - } - -} diff --git a/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs b/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs deleted file mode 100644 index d70c9aac5..000000000 --- a/FoundationDB.Storage.Memory/API/MemoryDatabaseHandler.cs +++ /dev/null @@ -1,1726 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -//#define FULL_DEBUG - -namespace FoundationDB.Storage.Memory.API -{ - using FoundationDB.Client; - using FoundationDB.Client.Core; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Storage.Memory.Core; - using FoundationDB.Storage.Memory.IO; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Concurrent; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Text; - using System.Threading; - using System.Threading.Tasks; - - internal class MemoryDatabaseHandler : IFdbDatabaseHandler, IDisposable - { - internal const uint MAX_KEY_SIZE = 10 * 1000; - internal const uint MAX_VALUE_SIZE = 100 * 1000; - - internal const uint KEYHEAP_MIN_PAGESIZE = 64 * 1024; - internal const uint KEYHEAP_MAX_PAGESIZE = 4 * 1024 * 1024; - internal const uint VALUEHEAP_MIN_PAGESIZE = 256 * 1024; - internal const uint VALUEHEAP_MAX_PAGESIZE = 16 * 1024 * 1024; - - internal void PopulateSystemKeys() - { - // we need to create the System keyspace, under \xFF - - // cheap way to generate machine & datacenter ids - var databaseId = new Uuid128(m_uid).ToSlice(); - var machineId = Slice.FromFixed64(Environment.MachineName.GetHashCode()) + databaseId[0, 8]; - var datacenterId = Slice.FromFixed64(Environment.MachineName.GetHashCode()) + databaseId[8, 16]; - var keyServerBlob = Slice.FromFixed16(1) + Slice.FromFixed32(0xA22000) + Slice.FromFixed16(0xFDB) + Slice.FromFixed32(1) + databaseId + Slice.FromFixed32(0); - var one = Slice.FromAscii("1"); - - var systemKeys = new Dictionary() - { - { Fdb.System.BackupDataFormat, one }, - { Fdb.System.ConfigKey("initialized"), one }, - { Fdb.System.ConfigKey("storage_engine"), one }, // ~= memory - { Fdb.System.ConfigKey("storage_replicas"), one }, // single replica - { Fdb.System.Coordinators, Slice.FromString("local:" + m_uid.ToString("N") + "@memory") }, - { Fdb.System.GlobalsKey("lastEpochEnd"), Slice.FromFixed64(0) }, - { Fdb.System.InitId, Slice.FromAscii(Guid.NewGuid().ToString("N")) }, - - { Fdb.System.KeyServers, keyServerBlob }, - { Fdb.System.KeyServers + Fdb.System.KeyServers, keyServerBlob }, - { Fdb.System.KeyServers + Fdb.System.MaxValue, Slice.Empty }, - - { Fdb.System.ServerKeys + databaseId + Slice.FromAscii("/"), one }, - { Fdb.System.ServerKeys + databaseId + Slice.FromAscii("/\xFF\xFF"), Slice.Empty }, - - //TODO: serverList ? - - { Fdb.System.WorkersKey("memory", "datacenter"), datacenterId }, - { Fdb.System.WorkersKey("memory", "machine"), machineId }, - { Fdb.System.WorkersKey("memory", "mclass"), Slice.FromAscii("unset") }, - }; - - BulkLoadAsync(systemKeys, false, false, CancellationToken.None).GetAwaiter().GetResult(); - } - - #region Private Members... - - /// Set to true when the current db instance gets disposed. - private volatile bool m_disposed; - - /// Current version of the database - private long m_currentVersion; - /// Oldest legal read version of the database - private long m_oldestVersion; - - /// Unique number for this database - private Guid m_uid; - - //TODO: replace this with an Async lock ? - private readonly ReaderWriterLockSlim m_dataLock = new ReaderWriterLockSlim(); - private readonly object m_heapLock = new object(); - - private readonly KeyHeap m_keys = new KeyHeap(); - private readonly ValueHeap m_values = new ValueHeap(); - - private ColaStore m_data = new ColaStore(0, new NativeKeyComparer()); - private long m_estimatedSize; - - /// List of all active transaction windows - private LinkedList m_transactionWindows = new LinkedList(); - /// Last transaction window - private TransactionWindow m_currentWindow; - - // note: all scratch buffers should have a size larger than 80KB, so that they to the LOH - /// Pool of builders uses by read operations from transactions (concurrent) - private UnmanagedSliceBuilderPool m_scratchPool = new UnmanagedSliceBuilderPool(128 * 1024, 64); - /// Scratch use to format keys when committing (single writer) - private UnmanagedSliceBuilder m_scratchKey = new UnmanagedSliceBuilder(128 * 1024); - /// Scratch use to hold values when committing (single writer) - private UnmanagedSliceBuilder m_scratchValue = new UnmanagedSliceBuilder(128 * 1024); - - #endregion - - public MemoryDatabaseHandler(Guid uid) - { - m_uid = uid; - } - - public Guid Id { get { return m_uid; } } - - public bool IsInvalid { get { return false; } } - - public bool IsClosed { get { return m_disposed; } } - - public void SetOption(FdbDatabaseOption option, Slice data) - { - throw new NotImplementedException(); - } - - internal long GetCurrentVersion() - { - m_dataLock.EnterReadLock(); - try - { - return Volatile.Read(ref m_currentVersion); - } - finally - { - m_dataLock.ExitReadLock(); - } - } - - /// Format a user key using a slice buffer for temporary storage - /// The buffer is cleared prior to usage! - internal unsafe static USlice PackUserKey(UnmanagedSliceBuilder buffer, Slice userKey) - { - Contract.Requires(buffer != null && userKey.Array != null && userKey.Count >= 0 && userKey.Offset >= 0); - Contract.Requires(userKey.Count <= MemoryDatabaseHandler.MAX_KEY_SIZE); - - buffer.Clear(); - uint keySize = (uint)userKey.Count; - uint size = Key.SizeOf + keySize; - var tmp = buffer.Allocate(size); - var key = (Key*)tmp.Data; - key->Size = (ushort)keySize; - key->HashCode = UnmanagedHelpers.ComputeHashCode(ref userKey); - key->Header = ((ushort)EntryType.Key) << Entry.TYPE_SHIFT; - key->Values = null; - - if (keySize > 0) UnmanagedHelpers.CopyUnsafe(&(key->Data), userKey); - return tmp; - } - - /// Format a user key - internal unsafe static USlice PackUserKey(UnmanagedSliceBuilder buffer, USlice userKey) - { - Contract.Requires(buffer != null && userKey.Data != null); - Contract.Requires(userKey.Count <= MemoryDatabaseHandler.MAX_KEY_SIZE); - - buffer.Clear(); - uint keySize = userKey.Count; - var size = Key.SizeOf + keySize; - var tmp = buffer.Allocate(size); - var key = (Key*)tmp.Data; - key->Size = (ushort)keySize; - key->HashCode = UnmanagedHelpers.ComputeHashCode(ref userKey); - key->Header = ((ushort)EntryType.Key) << Entry.TYPE_SHIFT; - key->Values = null; - - if (keySize > 0) UnmanagedHelpers.CopyUnsafe(&(key->Data), userKey); - return tmp; - } - - private TimeSpan m_transactionHalfLife = TimeSpan.FromSeconds(2.5); - private TimeSpan m_windowMaxDuration = TimeSpan.FromSeconds(5); - private int m_windowMaxWrites = 1000; - - private TransactionWindow GetActiveTransactionWindow_NeedsLocking(ulong sequence) - { - var window = m_currentWindow; - var now = DateTime.UtcNow; - - // open a new window if the previous one is already closed, or is too old - if (window != null) - { // is it still active ? - if (window.Closed || now.Subtract(window.StartedUtc) >= m_transactionHalfLife || window.CommitCount >= m_windowMaxWrites) - { - Log("Recycling previous window " + window); - window = null; - } - } - - if (window == null) - { // need to start a new window - window = new TransactionWindow(now, sequence); - m_currentWindow = window; - m_transactionWindows.AddFirst(window); - } - - // check the oldest transaction window - PurgeOldTransactionWindows(now); - - return window; - } - - private void PurgeOldTransactionWindows(DateTime utcNow) - { - var stop = m_currentWindow; - var node = m_transactionWindows.Last; - TransactionWindow window; - - while ((node != null && (window = node.Value) != null && window != stop)) - { - if (!window.Closed && utcNow.Subtract(window.StartedUtc) <= m_windowMaxDuration) - { - break; - } - Log("Purging old transaction window " + window.ToString()); - - window.Close(); - var tmp = node.Previous; - m_transactionWindows.RemoveLast(); - node = tmp; - } - } - - /// Commits the changes made by a transaction to the database. - /// - /// - /// - /// - /// - /// - /// - /// This method is not thread safe and must be called from the writer thread. - internal unsafe long CommitTransaction(MemoryTransactionHandler trans, long readVersion, ColaRangeSet readConflicts, ColaRangeSet writeConflicts, ColaRangeSet clearRanges, ColaOrderedDictionary writes) - { - if (trans == null) throw new ArgumentNullException("trans"); - if (m_disposed) ThrowDisposed(); - - // version at which the transaction was created (and all reads performed) - ulong readSequence = (ulong)readVersion; - // commit version created by this transaction (if it writes something) - ulong committedSequence = 0; - - Log("Comitting transaction created at readVersion " + readVersion + " ..."); - - bool hasReadConflictRanges = readConflicts != null && readConflicts.Count > 0; - bool hasWriteConflictRanges = writeConflicts != null && writeConflicts.Count > 0; - bool hasClears = clearRanges != null && clearRanges.Count > 0; - bool hasWrites = writes != null && writes.Count > 0; - - bool isReadOnlyTransaction = !hasClears && !hasWrites && !hasWriteConflictRanges; - - m_dataLock.EnterUpgradeableReadLock(); - try - { - TransactionWindow window; - - if (!isReadOnlyTransaction) - { - committedSequence = (ulong)Interlocked.Increment(ref m_currentVersion); - window = GetActiveTransactionWindow_NeedsLocking(committedSequence); - Contract.Assert(window != null); - Log("... will create version " + committedSequence + " in window " + window.ToString()); - } - else - { - Log("... which is read-only"); - window = null; - } - - #region Read Conflict Check - - if (hasReadConflictRanges) - { - - var current = m_transactionWindows.First; - while (current != null && current.Value.LastVersion >= readSequence) - { - if (current.Value.Conflicts(readConflicts, readSequence)) - { - // the transaction has conflicting reads - throw new FdbException(FdbError.NotCommitted); - } - current = current.Next; - } - } - - #endregion - - if (!isReadOnlyTransaction) - { - #region Clear Ranges... - - if (hasClears) - { - foreach (var clear in clearRanges) - { - //TODO! - throw new NotImplementedException("ClearRange not yet implemented. Sorry!"); - } - } - - #endregion - - #region Writes... - - if (hasWrites) - { - IntPtr singleInsert = IntPtr.Zero; - List pendingInserts = null; - - foreach (var write in writes) - { - Key* key; - Value* value; - - // apply all the transformations at once on the key, add a new version if required - - // Only two allowed cases: - // - a single SET operation that create or update the value - // - one or more ATOMIC operations that create or mutate the value - - // For both case, we will do a lookup in the db to get the previous value and location - - // create the lookup key - USlice lookupKey = PackUserKey(m_scratchKey, write.Key); - - IntPtr previous; - int offset, level = m_data.Find(lookupKey.GetPointer(), out offset, out previous); - key = level >= 0 ? (Key*)previous : null; - Contract.Assert((level < 0 && key == null) || (level >= 0 && offset >= 0 && key != null)); - - bool valueMutated = false; - bool currentIsDeleted = false; - bool hasTmpData = false; - - foreach (var op in write.Value) - { - if (op.Type == MemoryTransactionHandler.Operation.Nop) continue; - - if (op.Type == MemoryTransactionHandler.Operation.Set) - { - m_scratchValue.Set(op.Value); - hasTmpData = true; - valueMutated = true; - continue; - } - - // apply the atomic operation to the previous value - if (!hasTmpData) - { - m_scratchValue.Clear(); - if (key != null) - { // grab the current value of this key - - Value* p = key->Values; - if ((p->Header & Value.FLAGS_DELETION) == 0) - { - m_scratchValue.Append(&(p->Data), p->Size); - } - else - { - m_scratchValue.Clear(); - currentIsDeleted = true; - } - } - hasTmpData = true; - } - - switch (op.Type) - { - case MemoryTransactionHandler.Operation.AtomicAdd: - { - op.ApplyAddTo(m_scratchValue); - valueMutated = true; - break; - } - case MemoryTransactionHandler.Operation.AtomicBitAnd: - { - op.ApplyBitAndTo(m_scratchValue); - valueMutated = true; - break; - } - case MemoryTransactionHandler.Operation.AtomicBitOr: - { - op.ApplyBitOrTo(m_scratchValue); - valueMutated = true; - break; - } - case MemoryTransactionHandler.Operation.AtomicBitXor: - { - op.ApplyBitXorTo(m_scratchValue); - valueMutated = true; - break; - } - default: - { - throw new InvalidOperationException(); - } - } - } - - if (valueMutated) - { // we have a new version for this key - - lock (m_heapLock) - { - value = m_values.Allocate(m_scratchValue.Count, committedSequence, key != null ? key->Values : null, null); - } - Contract.Assert(value != null); - m_scratchValue.CopyTo(&(value->Data)); - Interlocked.Add(ref m_estimatedSize, value->Size); - - if (key != null) - { // mutate the previous version for this key - var prev = key->Values; - value->Parent = key; - key->Values = value; - prev->Header |= Value.FLAGS_MUTATED; - prev->Parent = value; - - // make sure no thread seees an inconsitent view of the key - Interlocked.MemoryBarrier(); - } - else - { // add this key to the data store - - // we can reuse the lookup key (which is only missing the correct flags and pointers to the values) - lock (m_heapLock) - { - key = m_keys.Append(lookupKey); - } - key->Values = value; - value->Parent = key; - Contract.Assert(key->Size == write.Key.Count); - Interlocked.Add(ref m_estimatedSize, key->Size); - - // make sure no thread seees an inconsitent view of the key - Interlocked.MemoryBarrier(); - - if (pendingInserts != null) - { - pendingInserts.Add(new IntPtr(key)); - } - else if (singleInsert != IntPtr.Zero) - { - pendingInserts = new List(); - pendingInserts.Add(singleInsert); - pendingInserts.Add(new IntPtr(key)); - singleInsert = IntPtr.Zero; - } - else - { - singleInsert = new IntPtr(key); - } - } - - } - } - - if (singleInsert != IntPtr.Zero || pendingInserts != null) - { - // insert the new key into the data store - m_dataLock.EnterWriteLock(); - try - { - if (singleInsert != IntPtr.Zero) - { - m_data.Insert(singleInsert); - } - else - { - m_data.InsertItems(pendingInserts, ordered: true); - } - } - finally - { - m_dataLock.ExitWriteLock(); - } - } - } - - #endregion - - #region Merge Write Conflicts... - - if (hasWriteConflictRanges) - { - window.MergeWrites(writeConflicts, committedSequence); - } - - #endregion - } - } - finally - { - m_dataLock.ExitUpgradeableReadLock(); - } - - var version = isReadOnlyTransaction ? -1L : (long)committedSequence; - - return version; - } - - internal unsafe Task BulkLoadAsync(ICollection> data, bool ordered, bool append, CancellationToken cancellationToken) - { - Contract.Requires(data != null); - - int count = data.Count; - - // Since we can "only" create a maximum of 28 levels, there is a maximum limit or 2^28 - 1 items that can be loaded in the database (about 268 millions) - if (count >= 1 << 28) throw new InvalidOperationException("Data set is too large. Cannot insert more than 2^28 - 1 items in the memory database"); - - // clear everything, and import the specified data - - m_dataLock.EnterWriteLock(); - try - { - - // the fastest way to insert data, is to insert vectors that are a power of 2 - int min = ColaStore.LowestBit(count); - int max = ColaStore.HighestBit(count); - Contract.Assert(min <= max && max <= 28); - if (append) - { // the appended layers have to be currently free - for (int level = min; level <= max; level++) - { - if (!m_data.IsFree(level)) throw new InvalidOperationException(String.Format("Cannot bulk load level {0} because it is already in use", level)); - } - } - else - { // start from scratch - m_data.Clear(); - m_estimatedSize = 0; - //TODO: clear the key and value heaps ! - //TODO: clear the transaction windows ! - //TODO: kill all pending transactions ! - } - - m_data.EnsureCapacity(count); - - ulong sequence = (ulong)Interlocked.Increment(ref m_currentVersion); - - using (var iter = data.GetEnumerator()) - using (var writer = new LevelWriter(1 << max, m_keys, m_values)) - { - for (int level = max; level >= min && !cancellationToken.IsCancellationRequested; level--) - { - if (ColaStore.IsFree(level, count)) continue; - - //TODO: consider pre-sorting the items before inserting them in the heap using m_comparer (maybe faster than doing the same with the key comparer?) - - // take of batch of values - writer.Reset(); - int batch = 1 << level; - while(batch-- > 0) - { - if (!iter.MoveNext()) - { - throw new InvalidOperationException("Iterator stopped before reaching the expected number of items"); - } - writer.Add(sequence, iter.Current); - } - - // and insert it (should fit nicely in a level without cascading) - m_data.InsertItems(writer.Data, ordered); - } - } - } - finally - { - m_dataLock.ExitWriteLock(); - } - - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - return TaskHelpers.CompletedTask; - } - - private static readonly Task NilResult = Task.FromResult(Slice.Nil); - private static readonly Task EmptyResult = Task.FromResult(Slice.Empty); - private static readonly Task MaxResult = Task.FromResult(Slice.FromByte(255)); - - private void EnsureReadVersionNotInTheFuture_NeedsLocking(ulong readVersion) - { - if ((ulong)Volatile.Read(ref m_currentVersion) < readVersion) - { // a read for a future version? This is most probably a bug ! -#if DEBUG - if (Debugger.IsAttached) Debugger.Break(); -#endif - throw new FdbException(FdbError.FutureVersion); - } - } - - [Conditional("FULL_DEBUG")] - private unsafe static void DumpKey(string label, IntPtr userKey) - { -#if FULL_DEBUG - var sb = new StringBuilder("(*) " + (label ?? "key") + " = "); - if (userKey == IntPtr.Zero) - { - sb.Append(""); - } - else - { - sb.Append(userKey).Append(" => "); - - Key* key = (Key*)userKey; - Contract.Assert(key != null); - - sb.Append('\'').Append(FdbKey.Dump(Key.GetData(key).ToSlice())).Append('\''); - - Value* value = key->Values; - if (value != null) - { - sb.Append(" => [").Append(value->Sequence).Append("] "); - if ((value->Header & Value.FLAGS_DELETION) != 0) - { - sb.Append("DELETED"); - } - else if (value->Size == 0) - { - sb.Append(""); - } - else - { - sb.Append(Value.GetData(value).ToSlice().ToAsciiOrHexaString()); - } - } - } - Trace.WriteLine(sb.ToString()); -#endif - } - - private unsafe bool TryGetValueAtVersion(USlice lookupKey, ulong sequence, out USlice result) - { - result = default(USlice); - - IntPtr existing; - int _, level = m_data.Find(lookupKey.GetPointer(), out _, out existing); - if (level < 0) - { - return false; - } - - Key* key = (Key*)existing; - //TODO: aserts! - - // walk the chain of version until we find one that existed at the request version - Value* current = key->Values; - while (current != null) - { - if (current->Sequence <= sequence) - { // found it - break; - } - current = current->Previous; - } - - if (current == null || (current->Header & Value.FLAGS_DELETION) != 0) - { // this key was created after our read version, or this version is a deletion marker - return false; - } - - if (current->Size > 0) - { // the value is not empty - result = Value.GetData(current); - } - return true; - - } - - /// Read the value of one or more keys, at a specific database version - /// List of keys to read (MUST be ordered) - /// Version of the read - /// Array of results - internal unsafe Slice[] GetValuesAtVersion(Slice[] userKeys, long readVersion) - { - if (m_disposed) ThrowDisposed(); - if (userKeys == null) throw new ArgumentNullException("userKeys"); - - var results = new Slice[userKeys.Length]; - - if (userKeys.Length > 0) - { - m_dataLock.EnterReadLock(); - try - { - ulong sequence = (ulong)readVersion; - EnsureReadVersionNotInTheFuture_NeedsLocking(sequence); - - var buffer = new SliceBuffer(); - - using (var scratch = m_scratchPool.Use()) - { - var builder = scratch.Builder; - - for (int i = 0; i < userKeys.Length; i++) - { - // create a lookup key - var lookupKey = PackUserKey(builder, userKeys[i]); - - USlice value; - if (!TryGetValueAtVersion(lookupKey, sequence, out value)) - { // this key does not exist, or was deleted at that time - results[i] = default(Slice); - } - else if (value.Count == 0) - { // the value is the empty slice - results[i] = Slice.Empty; - } - else - { // move this value to the slice buffer - var data = buffer.Allocate(checked((int)value.Count)); - Contract.Assert(data.Array != null && data.Offset >= 0 && data.Count == (int)value.Count); - UnmanagedHelpers.CopyUnsafe(data, value.Data, value.Count); - results[i] = data; - } - } - } - } - finally - { - m_dataLock.ExitReadLock(); - } - } - return results; - } - - /// Walk the value chain, to return the value of a key that was the latest at a specific read version - /// User key to resolve - /// Sequence number - /// Value of the key at that time, or null if the key was either deleted or not yet created. - internal static unsafe Value* ResolveValueAtVersion(IntPtr userKey, ulong sequence) - { - if (userKey == IntPtr.Zero) return null; - - Key* key = (Key*)userKey; - Contract.Assert((key->Header & Entry.FLAGS_DISPOSED) == 0, "Attempted to read value from a disposed key"); - Contract.Assert(key->Size <= MemoryDatabaseHandler.MAX_KEY_SIZE, "Attempted to read value from a key that is too large"); - - Value* current = key->Values; - while(current != null && current->Sequence > sequence) - { - current = current->Previous; - } - - if (current == null || (current->Header & Value.FLAGS_DELETION) != 0) - { - return null; - } - - Contract.Ensures((current->Header & Entry.FLAGS_DISPOSED) == 0 && current->Sequence <= sequence); - return current; - } - - private unsafe ColaStore.Iterator ResolveCursor(USlice lookupKey, bool orEqual, int offset, ulong sequence) - { - var iterator = m_data.GetIterator(); - - DumpKey(orEqual ? "seek(<=)" : "seek(<)", lookupKey.GetPointer()); - - // seek to the closest key - if (!iterator.Seek(lookupKey.GetPointer(), orEqual)) - { // we are before the first key in the database! - if (offset <= 0) - { - iterator.SeekBeforeFirst(); - return iterator; - } - else - { - iterator.SeekFirst(); - --offset; - } - } - - bool forward = offset >= 0; - - while (iterator.Current != IntPtr.Zero) - { - DumpKey("offset " + offset, iterator.Current); - Value* value = ResolveValueAtVersion(iterator.Current, sequence); - //Trace.WriteLine("[*] " + (long)value); - if (value != null) - { - if (offset == 0) - { // we found a key that was alive, and at the correct offset - break; - } - if (forward) - { - --offset; - } - else - { - ++offset; - } - } - - if (forward) - { // move forward - - //Trace.WriteLine("> next!"); - if (!iterator.Next()) - { - //Trace.WriteLine(" > EOF"); - break; - } - } - else - { // move backward - //Trace.WriteLine("> prev!"); - if (!iterator.Previous()) - { - //Trace.WriteLine(" > EOF"); - break; - } - } - } - - return iterator; - } - - internal unsafe Task GetKeysAtVersion(FdbKeySelector[] selectors, long readVersion) - { - if (m_disposed) ThrowDisposed(); - if (selectors == null) throw new ArgumentNullException("selectors"); - - var results = new Slice[selectors.Length]; - - m_dataLock.EnterReadLock(); - try - { - ulong sequence = (ulong)readVersion; - EnsureReadVersionNotInTheFuture_NeedsLocking(sequence); - - // TODO: convert all selectors to a FirstGreaterThan ? - var buffer = new SliceBuffer(); - - using (var scratch = m_scratchPool.Use()) - { - var builder = scratch.Builder; - - for (int i = 0; i < selectors.Length; i++) - { - var selector = selectors[i]; - - var lookupKey = PackUserKey(builder, selector.Key); - - var iterator = ResolveCursor(lookupKey, selector.OrEqual, selector.Offset, sequence); - Contract.Assert(iterator != null); - - if (iterator.Current == IntPtr.Zero) - { - //Trace.WriteLine("> NOTHING :("); - results[i] = default(Slice); - continue; - } - - // we want the key! - Key* key = (Key*)iterator.Current; - Contract.Assert(key != null && key->Size <= MemoryDatabaseHandler.MAX_KEY_SIZE); - - var data = buffer.Allocate(checked((int)key->Size)); - Contract.Assert(data.Array != null && data.Offset >= 0 && data.Count == (int)key->Size); - UnmanagedHelpers.CopyUnsafe(data, &(key->Data), key->Size); - results[i] = data; - } - } - } - finally - { - m_dataLock.ExitReadLock(); - } - - return Task.FromResult(results); - } - - private static unsafe KeyValuePair CopyResultToManagedMemory(SliceBuffer buffer, Key* key, Value* value) - { - Contract.Requires(buffer != null && key != null && value != null); - - var keyData = buffer.Allocate(checked((int)key->Size)); - UnmanagedHelpers.CopyUnsafe(keyData, &(key->Data), key->Size); - - var valueData = buffer.Allocate(checked((int)value->Size)); - UnmanagedHelpers.CopyUnsafe(valueData, &(value->Data), value->Size); - - return new KeyValuePair(keyData, valueData); - } - - /// Range iterator that will return the keys and values at a specific sequence - internal sealed unsafe class RangeIterator : IDisposable - { - private readonly MemoryDatabaseHandler m_handler; - private readonly ulong m_sequence; - private readonly ColaStore.Iterator m_iterator; - private readonly IntPtr m_stopKey; - private readonly IComparer m_comparer; - private readonly long m_limit; - private readonly long m_targetBytes; - private readonly bool m_reverse; - private bool m_done; - private long m_readKeys; - private long m_readBytes; - private Key* m_currentKey; - private Value* m_currentValue; - private bool m_disposed; - - internal RangeIterator(MemoryDatabaseHandler handler, ulong sequence, ColaStore.Iterator iterator, IntPtr stopKey, IComparer comparer, bool reverse) - { - Contract.Requires(handler != null && iterator != null && comparer != null); - m_handler = handler; - m_sequence = sequence; - m_iterator = iterator; - m_stopKey = stopKey; - m_comparer = comparer; - m_reverse = reverse; - } - - public long Sequence { get { return (long)m_sequence; } } - - public long Count { get { return m_readKeys; } } - - public long Bytes { get { return m_readBytes; } } - - public long TargetBytes { get { return m_targetBytes; } } - - public bool Reverse { get { return m_reverse; } } - - public Key* Key { get { return m_currentKey; } } - - public Value* Value { get { return m_currentValue; } } - - public bool Done { get { return m_done; } } - - public bool MoveNext() - { - if (m_done || m_disposed) return false; - - bool gotOne = false; - - while (!gotOne) - { - var current = m_iterator.Current; - DumpKey("current", current); - - Value* value = MemoryDatabaseHandler.ResolveValueAtVersion(current, m_sequence); - if (value != null) - { - if (m_stopKey != IntPtr.Zero) - { - int c = m_comparer.Compare(current, m_stopKey); - if (m_reverse ? (c < 0 /* BEGIN KEY IS INCLUDED! */) : (c >= 0 /* END KEY IS EXCLUDED! */)) - { // we reached the end, stop there ! - DumpKey("stopped at ", current); - MarkAsDone(); - break; - } - } - Key* key = (Key*)current; - ++m_readKeys; - m_readBytes += checked(key->Size + value->Size); - m_currentKey = key; - m_currentValue = value; - gotOne = true; - } - - // prepare for the next value - if (!(m_reverse ? m_iterator.Previous() : m_iterator.Next())) - { - // out of data to read ? - MarkAsDone(); - break; - } - } - - if (gotOne) - { // we have found a value - return true; - } - - m_currentKey = null; - m_currentValue = null; - return false; - } - - private void MarkAsDone() - { - m_done = true; - } - - public void Dispose() - { - if (!m_disposed) - { - m_disposed = true; - m_currentKey = null; - m_currentValue = null; - //TODO: release any locks taken - } - } - } - - internal unsafe Task GetRangeAtVersion(FdbKeySelector begin, FdbKeySelector end, int limit, int targetBytes, FdbStreamingMode mode, int iteration, bool reverse, long readVersion) - { - if (m_disposed) ThrowDisposed(); - - //HACKHACK - var results = new List>(limit); - - if (limit == 0) limit = 10000; - if (targetBytes == 0) targetBytes = int.MaxValue; - - //bool done = false; - - m_dataLock.EnterReadLock(); - try - { - ulong sequence = (ulong)readVersion; - EnsureReadVersionNotInTheFuture_NeedsLocking(sequence); - - // TODO: convert all selectors to a FirstGreaterThan ? - var buffer = new SliceBuffer(); - - ColaStore.Iterator iterator; - IntPtr stopKey; - - if (!reverse) - { // forward range read: we read from beginKey, and stop once we reach a key >= endKey - - using (var scratch = m_scratchPool.Use()) - { - // first resolve the end to get the stop point - iterator = ResolveCursor(PackUserKey(scratch.Builder, end.Key), end.OrEqual, end.Offset, sequence); - stopKey = iterator.Current; // note: can be ZERO ! - - // now, set the cursor to the begin of the range - iterator = ResolveCursor(PackUserKey(scratch.Builder, begin.Key), begin.OrEqual, begin.Offset, sequence); - if (iterator.Current == IntPtr.Zero) iterator.SeekFirst(); - } - -#if REFACTORED - while (limit > 0 && targetBytes > 0) - { - DumpKey("current", iterator.Current); - - Value* value = ResolveValueAtVersion(iterator.Current, sequence); - if (value != null) - { - if (stopKey != IntPtr.Zero && m_data.Comparer.Compare(iterator.Current, stopKey) >= 0) /* END KEY IS EXCLUDED! */ - { // we reached the end, stop there ! - done = true; - break; - } - - var item = CopyResultToManagedMemory(buffer, (Key*)iterator.Current.ToPointer(), value); - results.Add(item); - --limit; - targetBytes -= item.Key.Count + item.Value.Count; - if (targetBytes < 0) targetBytes = 0; - } - - if (!iterator.Next()) - { // out of data to read ? - done = true; - break; - } - } -#endif - } - else - { // reverse range read: we start from the key before endKey, and stop once we read a key < beginKey - - using (var scratch = m_scratchPool.Use()) - { - // first resolve the begin to get the stop point - iterator = ResolveCursor(PackUserKey(scratch.Builder, begin.Key), begin.OrEqual, begin.Offset, sequence); - DumpKey("resolved(" + begin + ")", iterator.Current); - if (iterator.Current == IntPtr.Zero) iterator.SeekFirst(); - stopKey = iterator.Current; // note: can be ZERO ! - - DumpKey("stopKey", stopKey); - - // now, set the cursor to the end of the range - iterator = ResolveCursor(PackUserKey(scratch.Builder, end.Key), end.OrEqual, end.Offset, sequence); - DumpKey("resolved(" + end + ")", iterator.Current); - if (iterator.Current == IntPtr.Zero) - { - iterator.SeekLast(); - DumpKey("endKey", iterator.Current); - } - else - { - // note: since the end is NOT included in the result, we need to already move the cursor once - iterator.Previous(); - } - } - -#if REFACTORED - while (limit > 0 && targetBytes > 0) - { - DumpKey("current", iterator.Current); - - Value* value = ResolveValueAtVersion(iterator.Current, sequence); - if (value != null) - { - if (stopKey != IntPtr.Zero && m_data.Comparer.Compare(iterator.Current, stopKey) < 0) /* BEGIN KEY IS INCLUDED! */ - { // we reached past the beginning, stop there ! - DumpKey("stopped at ", iterator.Current); - done = true; - break; - } - - var item = CopyResultToManagedMemory(buffer, (Key*)iterator.Current.ToPointer(), value); - results.Add(item); - --limit; - targetBytes -= item.Key.Count + item.Value.Count; - if (targetBytes < 0) targetBytes = 0; - } - - if (!iterator.Previous()) - { // out of data to read ? - done = true; - break; - } - } -#endif - } - - // run the iterator until we reach the end of the range, the end of the database, or any count or size limit - using (var rangeIterator = new RangeIterator(this, sequence, iterator, stopKey, m_data.Comparer, reverse)) - { - while (rangeIterator.MoveNext()) - { - var item = CopyResultToManagedMemory(buffer, rangeIterator.Key, rangeIterator.Value); - results.Add(item); - - if (limit > 0 && rangeIterator.Count >= limit) break; - if (targetBytes > 0 && rangeIterator.Bytes >= targetBytes) break; - } - - bool hasMore = !rangeIterator.Done; - - var chunk = new FdbRangeChunk( - hasMore, - results.ToArray(), - iteration, - reverse - ); - return Task.FromResult(chunk); - } - } - finally - { - m_dataLock.ExitReadLock(); - } - } - - public IFdbTransactionHandler CreateTransaction(FdbOperationContext context) - { - if (m_disposed) ThrowDisposed(); - Contract.Assert(context != null); - - MemoryTransactionHandler transaction = null; - try - { - transaction = new MemoryTransactionHandler(this); - //m_pendingTransactions.Add(transaction); - return transaction; - } - catch(Exception) - { - if (transaction != null) - { - transaction.Dispose(); - //m_pendingTransactions.Remove(transaction); - } - throw; - } - } - - /// Return the read version of the oldest pending transaction - /// Sequence number of the oldest active transaction, or the current read version if there are no pending transactions - private ulong GetOldestReadVersion() - { - //HACKHACK: TODO! - return (ulong)Volatile.Read(ref m_currentVersion); - } - - #region Loading & Saving... - - internal async Task SaveSnapshotAsync(string path, MemorySnapshotOptions options, CancellationToken cancellationToken) - { - Contract.Requires(path != null && options != null); - - if (string.IsNullOrWhiteSpace(path)) throw new ArgumentNullException("path"); - cancellationToken.ThrowIfCancellationRequested(); - - // while we are generating the snapshot on the disk: - // * readers can read without any problems - // * writers can mutate values of existing keys, but cannot INSERT new keys - - var attributes = new Dictionary(StringComparer.Ordinal); - - // Flags bits: - // 0-3: FileType (4 bits) - // 0: Versionned Snapshot - // 1: Compact Snapshot - // 2-15: reserved - - SnapshotFormat.Flags headerFlags = SnapshotFormat.Flags.None; - switch (options.Mode) - { - case MemorySnapshotMode.Full: - case MemorySnapshotMode.Last: - { - headerFlags |= SnapshotFormat.Flags.TYPE_SNAPSHOT_VERSIONNED; - break; - } - case MemorySnapshotMode.Compact: - { - headerFlags |= SnapshotFormat.Flags.TYPE_SNAPSHOT_COMPACT; - break; - } - default: - { - throw new InvalidOperationException("Invalid snapshot mode"); - } - } - - attributes["version"] = FdbTuple.Create(1, 0); - attributes["host"] = FdbTuple.Create(Environment.MachineName); - attributes["timestamp"] = FdbTuple.Create(DateTimeOffset.Now.ToString("O")); - - if (options.Compressed) - { // file is compressed - - headerFlags |= SnapshotFormat.Flags.COMPRESSED; - //TODO: specify compression algorithm... - attributes["compression"] = FdbTuple.Create(true); - attributes["compression.algorithm"] = FdbTuple.Create("lz4"); - } - - if (options.Signed) - { // file will have a cryptographic signature - //TODO: specifiy digital signing algorithm - headerFlags |= SnapshotFormat.Flags.SIGNED; - attributes["signature"] = FdbTuple.Create(true); - attributes["signature.algorithm"] = FdbTuple.Create("pkcs1"); - } - - if (options.Encrypted) - { // file will be encrypted - //TODO: specify crypto algo, key sizes, initialization vectors, ... - headerFlags |= SnapshotFormat.Flags.ENCRYPTED; - attributes["encryption"] = FdbTuple.Create(true); - attributes["encryption.algorithm"] = FdbTuple.Create("pkcs1"); - attributes["encryption.keysize"] = FdbTuple.Create(4096); //ex: RSA 4096 ? - } - - //m_dataLock.EnterReadLock(); - try - { - - // take the current version of the db (that will be used for the snapshot) - ulong sequence = (ulong)Volatile.Read(ref m_currentVersion); - long timestamp = DateTime.UtcNow.Ticks; - int levels = m_data.Depth; - int count = m_data.Count; - - using (var output = new Win32SnapshotFile(path)) - { - var snapshot = new SnapshotWriter(output, levels, SnapshotFormat.PAGE_SIZE, SnapshotFormat.FLUSH_SIZE); - - //Console.WriteLine("> Writing header...."); - await snapshot.WriteHeaderAsync( - headerFlags, - new Uuid128(m_uid), - sequence, - count, - timestamp, - attributes - ).ConfigureAwait(false); - - //Console.WriteLine("> Writing level data..."); - for (int level = levels - 1; level >= 0; level--) - { - if (ColaStore.IsFree(level, count)) - { // this level is not allocated - //Console.WriteLine(" > Skipping empty level " + level); - continue; - } - - //Console.WriteLine(" > Dumping " + levels + " levels..."); - await snapshot.WriteLevelAsync(level, m_data.GetLevel(level), cancellationToken); - } - - // Write the JumpTable to the end of the file - //Console.WriteLine("> Writing Jump Table..."); - await snapshot.WriteJumpTableAsync(cancellationToken); - - // flush any remaining data to the disc - //Console.WriteLine("> Flushing..."); - await snapshot.FlushAsync(cancellationToken); - - //Console.WriteLine("> Final file size if " + output.Length.ToString("N0") + " bytes"); - } - //Console.WriteLine("> Done!"); - - return (long)sequence; - } - finally - { - //m_dataLock.ExitReadLock(); - } - } - - internal Task LoadSnapshotAsync(string path, MemorySnapshotOptions options, CancellationToken cancellationToken) - { - if (string.IsNullOrWhiteSpace(path)) throw new ArgumentNullException("path"); - - //TODO: should this run on the writer thread ? - return Task.Run(() => LoadSnapshotInternal(path, options, cancellationToken), cancellationToken); - } - - private void LoadSnapshotInternal(string path, MemorySnapshotOptions options, CancellationToken cancellationToken) - { - Contract.Requires(path != null && options != null); - - var attributes = new Dictionary(StringComparer.Ordinal); - - //m_dataLock.EnterWriteLock(); - try - { - using (var source = Win32MemoryMappedFile.OpenRead(path)) - { - var snapshot = new SnapshotReader(source); - - // Read the header - //Console.WriteLine("> Reading Header"); - snapshot.ReadHeader(cancellationToken); - - // Read the jump table (at the end) - //Console.WriteLine("> Reading Jump Table"); - snapshot.ReadJumpTable(cancellationToken); - - // we should have enough information to allocate memory - m_data.Clear(); - m_estimatedSize = 0; - - using (var writer = new LevelWriter(1 << snapshot.Depth, m_keys, m_values)) - { - // Read the levels - for (int level = snapshot.Depth - 1; level >= 0; level--) - { - if (!snapshot.HasLevel(level)) - { - continue; - } - - //Console.WriteLine("> Reading Level " + level); - //TODO: right we read the complete level before bulkloading it - // we need to be able to bulk load directly from the stream! - snapshot.ReadLevel(level, writer, cancellationToken); - - m_data.InsertItems(writer.Data, ordered: true); - writer.Reset(); - } - } - - m_uid = snapshot.Id.ToGuid(); - m_currentVersion = (long)snapshot.Sequence; - - //Console.WriteLine("> done!"); - } - } - finally - { - //m_dataLock.ExitWriteLock(); - } - } - - #endregion - - #region Writer Thread... - - private sealed class CommitState : TaskCompletionSource - { - public CommitState(MemoryTransactionHandler trans) - : base() - { - Contract.Requires(trans != null); - this.Transaction = trans; - } - - public void MarkAsCompleted() - { - if (!this.Task.IsCompleted) - { - ThreadPool.UnsafeQueueUserWorkItem((state) => { ((CommitState)state).TrySetResult(null); }, this); - } - } - - public void MarkAsFailed(Exception e) - { - if (!this.Task.IsCompleted) - { - ThreadPool.UnsafeQueueUserWorkItem( - (state) => - { - var items = (Tuple)state; - items.Item1.TrySetException(items.Item2); - }, - Tuple.Create(this, e) - ); - } - } - - public void MarkAsCancelled() - { - if (!this.Task.IsCompleted) - { - ThreadPool.UnsafeQueueUserWorkItem((state) => { ((CommitState)state).TrySetResult(null); }, this); - } - } - - public MemoryTransactionHandler Transaction { get; private set; } - - } - - [Conditional("FULL_DEBUG")] - private static void Log(string msg) - { -#if FULL_DEBUG - Trace.WriteLine("MemoryDatabaseHandler[#" + Thread.CurrentThread.ManagedThreadId + "]: " + msg); -#endif - } - - private const int STATE_IDLE = 0; - private const int STATE_RUNNNING = 1; - private const int STATE_SHUTDOWN = 2; - - private int m_eventLoopState = STATE_IDLE; - private AutoResetEvent m_writerEvent = new AutoResetEvent(false); - private ConcurrentQueue m_writerQueue = new ConcurrentQueue(); - private ManualResetEvent m_shutdownEvent = new ManualResetEvent(false); - - internal Task EnqueueCommit(MemoryTransactionHandler trans) - { - if (trans == null) throw new ArgumentNullException("trans"); - - if (Volatile.Read(ref m_eventLoopState) == STATE_SHUTDOWN) - { - throw new FdbException(FdbError.OperationFailed, "The database has already been disposed"); - } - - var entry = new CommitState(trans); - try - { - m_writerQueue.Enqueue(entry); - - // wake up the writer thread if needed - // note: we need to set the event BEFORE changing the eventloop state, because the writer thread may be in the process of shutting down - m_writerEvent.Set(); - Log("Enqueued new commit"); - - if (Interlocked.CompareExchange(ref m_eventLoopState, STATE_RUNNNING, STATE_IDLE) == STATE_IDLE) - { // we have to start the event loop - Log("Starting new Writer EventLoop..."); - var _ = Task.Factory.StartNew(() => WriteEventLoop(), CancellationToken.None, TaskCreationOptions.LongRunning, TaskScheduler.Default); - } - } - catch (Exception e) - { - entry.SetException(e); - } - return entry.Task; - } - - /// Event loop that is called to process all the writes to the database - private void WriteEventLoop() - { - TimeSpan quanta = TimeSpan.FromSeconds(30); - - // confirm that we can still run - if (Interlocked.CompareExchange(ref m_eventLoopState, STATE_RUNNNING, STATE_RUNNNING) != STATE_RUNNNING) - { // a shutdown was retquested, exit immediately - Log("WriteEventLoop fast abort"); - return; - } - - Log("WriteEventLoop started"); - - try - { - bool keepGoing = true; - while (keepGoing) - { - // Wait() will: - // - return true if we have a new entry to process - // - return false if the quanta timeout has expired - // - throw an OperationCanceledException if the cancellation token was triggered - if (m_writerEvent.WaitOne(quanta)) - { - Log("WriteEventLoop wake up"); - CommitState entry; - - // process all the pending writes - while (Volatile.Read(ref m_eventLoopState) != STATE_SHUTDOWN && m_writerQueue.TryDequeue(out entry)) - { - if (entry.Task.IsCompleted) - { // the task has already been completed/cancelled? - continue; - } - - try - { - Log("WriteEventLoop process transaction"); - //TODO: work ! - entry.Transaction.CommitInternal(); - entry.MarkAsCompleted(); - } - catch (Exception e) - { - Log("WriteEventLoop transaction failed: " + e.Message); - entry.MarkAsFailed(new FdbException(FdbError.InternalError, "The transaction failed to commit", e)); - } - } - - if (Volatile.Read(ref m_eventLoopState) == STATE_SHUTDOWN) - { // we have been asked to shutdown - Log("WriteEventLoop shutdown requested"); - // drain the commit queue, and mark all of them as failed - while (m_writerQueue.TryDequeue(out entry)) - { - if (entry != null) entry.MarkAsCancelled(); - } - keepGoing = false; - } - } - else - { // try to step down - - Log("WriteEventLoop no activity"); - Interlocked.CompareExchange(ref m_eventLoopState, STATE_IDLE, STATE_RUNNNING); - // check again if nobody was trying to queue a write at the same time - if (!m_writerEvent.WaitOne(TimeSpan.Zero, false) || Interlocked.CompareExchange(ref m_eventLoopState, STATE_RUNNNING, STATE_IDLE) == STATE_IDLE) - { // either there were no pending writes, or we lost the race and will be replaced by another thread - Log("WriteEventLoop will step down"); - keepGoing = false; // stop - } -#if DEBUG - else - { - Log("WriteEventLoop will resume"); - } -#endif - } - } - Log("WriteEventLoop exit"); - } - catch(Exception) - { - //TODO: fail all pending commits ? - // reset the state to IDLE so that another write can restart us - Interlocked.CompareExchange(ref m_eventLoopState, STATE_IDLE, STATE_RUNNNING); - throw; - } - finally - { - if (Volatile.Read(ref m_eventLoopState) == STATE_SHUTDOWN) - { - m_shutdownEvent.Set(); - } - } - } - - private void StopWriterEventLoop() - { - // signal a shutdown - Log("WriterEventLoop requesting stop..."); - int oldState; - if ((oldState = Interlocked.Exchange(ref m_eventLoopState, STATE_SHUTDOWN)) != STATE_SHUTDOWN) - { - switch (oldState) - { - case STATE_RUNNNING: - { - // need to wake up the thread, if it was waiting for new writes - m_writerEvent.Set(); - // and wait for it to finish... - if (!m_shutdownEvent.WaitOne(TimeSpan.FromSeconds(5))) - { - // what should we do ? - } - Log("WriterEventLoop stopped"); - break; - } - default: - { // not running, or already shutdown ? - m_shutdownEvent.Set(); - break; - } - } - } - } - - #endregion - - /// Perform a complete garbage collection - public void Collect() - { - // - determine the old read version that is in use - // - look for all the windows that are older than that - // - collect all keys that were modified in these windows (value changed, or deleted) - // - for all heap pages that are above a freespace threshold, merge them into fewer full pages - - m_dataLock.EnterUpgradeableReadLock(); - try - { - - // collect everything that is oldest than the oldest active read version. - ulong sequence = GetOldestReadVersion(); - - lock (m_heapLock) - { - // purge the dead values - m_values.Collect(sequence); - - // pack the keys - //m_keys.Collect(sequence); - //BUGBUG: need to purge the colastore also ! - } - - m_oldestVersion = (long)sequence; - } - finally - { - m_dataLock.ExitUpgradeableReadLock(); - } - - - } - - public void Dispose() - { - if (!m_disposed) - { - m_disposed = true; - - StopWriterEventLoop(); - //TODO: need to lock and ensure that all pending transactions are done - - m_writerEvent.Dispose(); - m_shutdownEvent.Dispose(); - - m_keys.Dispose(); - m_values.Dispose(); - if (m_transactionWindows != null) - { - foreach (var window in m_transactionWindows) - { - if (window != null) window.Dispose(); - } - } - if (m_scratchPool != null) m_scratchPool.Dispose(); - m_scratchKey.Dispose(); - m_scratchValue.Dispose(); - } - } - - private void ThrowDisposed() - { - throw new ObjectDisposedException("The database has already been disposed"); - } - - [Conditional("DEBUG")] - public void Debug_Dump(bool detailed = false) - { -#if DEBUG - Debug.WriteLine("Dumping content of Database"); - m_dataLock.EnterReadLock(); - try - { - Debug.WriteLine("> Version: {0}", m_currentVersion); - Debug.WriteLine("> Items: {0:N0}", m_data.Count); - Debug.WriteLine("> Estimated size: {0:N0} bytes", m_estimatedSize); - Debug.WriteLine("> Transaction windows: {0}", m_transactionWindows.Count); - foreach(var window in m_transactionWindows) - { - Debug.WriteLine(" > {0} : {1:N0} commits{2}", window.ToString(), window.CommitCount, window.Closed ? " [CLOSED]" : ""); - } - long cmps, eqs, ghcs; - NativeKeyComparer.GetCounters(out cmps, out eqs, out ghcs); - Debug.WriteLine("> Comparisons: {0:N0} compares, {1:N0} equals, {2:N0} hashcodes", cmps, eqs, ghcs); - NativeKeyComparer.ResetCounters(); - lock (m_heapLock) - { - unsafe - { - m_keys.Debug_Dump(detailed); - m_values.Debug_Dump(detailed); - } - } - Debug.WriteLine(""); - } - finally - { - m_dataLock.ExitReadLock(); - } -#endif - } - - } - -} diff --git a/FoundationDB.Storage.Memory/API/MemorySnapshotOptions.cs b/FoundationDB.Storage.Memory/API/MemorySnapshotOptions.cs deleted file mode 100644 index 65101f15b..000000000 --- a/FoundationDB.Storage.Memory/API/MemorySnapshotOptions.cs +++ /dev/null @@ -1,36 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.API -{ - using System; - - public enum MemorySnapshotMode - { - /// Include all keys (included the deletions), as well as all their mutations, timestamped with their sequence number - Full = 0, - /// Include all keys (inlcuded the deletions), but with only their latest value. - Last, - /// Include only the live keys, with their latest value. - Compact, - - } - - public sealed class MemorySnapshotOptions - { - - public MemorySnapshotOptions() - { } - - public MemorySnapshotMode Mode { get; set; } - - public bool Compressed { get; set; } - - public bool Signed { get; set; } - - public bool Encrypted { get; set; } - - } - -} diff --git a/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs b/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs deleted file mode 100644 index f58df674e..000000000 --- a/FoundationDB.Storage.Memory/API/MemoryTransactionHandler.cs +++ /dev/null @@ -1,1271 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -//#define DUMP_TRANSACTION_STATE -//#define FULL_DEBUG - -namespace FoundationDB.Storage.Memory.API -{ - using FoundationDB.Client; - using FoundationDB.Client.Core; - using FoundationDB.Client.Utils; - using FoundationDB.Storage.Memory.Core; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Globalization; - using System.Linq; - using System.Runtime.InteropServices; - using System.Threading; - using System.Threading.Tasks; - - public sealed class MemoryTransactionHandler : IFdbTransactionHandler, IDisposable - { - internal const int MaxKeySize = 10 * 1000; //note: this should be the same as FoundationDB ! - internal const int MaxValueSize = 100 * 1000; //note: this should be the same as FoundationDB ! - private const int InitialBufferSize = 32 * 1024; //note: this should be at least 2x larger than the max key size, and if possible a power of 2 - - #region Private Fields... - - private readonly MemoryDatabaseHandler m_db; - - private volatile bool m_disposed; - - /// Buffer used to store the keys and values of this transaction - private SliceBuffer m_buffer; - - /// Lock that protects the state of the transaction - private readonly object m_lock = new object(); - /// List of all conflicts due to read operations - private ColaRangeSet m_readConflicts; - /// List of all conflicts due to write operations - private ColaRangeSet m_writeConflicts; - /// List of all ClearRange - private ColaRangeSet m_clears; - /// List of all Set operations (Set, Atomic, ..) - private ColaOrderedDictionary m_writes; - - /// Read version of the transaction - private long? m_readVersion; - /// Committed version of the transaction - private long m_committedVersion; - - private int m_retryCount; - - #endregion - - internal enum Operation - { - Nop = 0, - - Set = 1, - //note: the AtomicXXX should match the value of FdbMutationType - AtomicAdd = 2, - AtomicBitAnd = 6, - AtomicBitOr = 7, - AtomicBitXor = 8, - } - - [StructLayout(LayoutKind.Sequential)] - internal struct WriteCommand - { - public Slice Key; //PERF: readonly struct - public Slice Value; //PERF: readonly struct - public readonly Operation Type; - - public WriteCommand(Operation type, Slice key, Slice value) - { - this.Type = type; - this.Key = key; - this.Value = value; - } - - public override string ToString() - { - return String.Format(CultureInfo.InvariantCulture, "{0}({1}, {2}))", this.Type.ToString(), this.Key.ToAsciiOrHexaString(), this.Value.ToAsciiOrHexaString()); - } - - internal static byte[] PrepareValueForAtomicOperation(Slice value, int size) - { - if (value.Count >= size) - { // truncate if needed - return value.GetBytes(0, size); - } - - // pad with zeroes - var tmp = new byte[size]; - value.CopyTo(tmp, 0); - return tmp; - } - - public Slice ApplyTo(Slice value) - { - switch(this.Type) - { - case Operation.Set: - { - return this.Value; - } - case Operation.Nop: - { - return value; - } - case Operation.AtomicAdd: - { - return ApplyAdd(value); - } - case Operation.AtomicBitAnd: - { - return ApplyBitAnd(value); - } - case Operation.AtomicBitOr: - { - return ApplyBitOr(value); - } - case Operation.AtomicBitXor: - { - return ApplyBitXor(value); - } - default: - { - throw new NotSupportedException("Invalid write command type"); - } - } - } - - public Slice ApplyAdd(Slice value) - { - var tmp = PrepareValueForAtomicOperation(value, this.Value.Count); - BufferAdd(tmp, 0, this.Value.Array, this.Value.Offset, this.Value.Count); - return Slice.Create(tmp); - } - - public void ApplyAddTo(UnmanagedSliceBuilder value) - { - uint size = checked((uint)this.Value.Count); - - // if the value is empty, then this is the same thing as adding to 0 - if (value.Count == 0) - { - value.Append(this.Value); - return; - } - - // truncate the value if larger, or pad it with zeroes if shorter - value.Resize(size, 0); - - if (size > 0) - { - unsafe - { - fixed (byte* ptr = this.Value.Array) - { - byte* left = value.Data; - byte* right = ptr + this.Value.Offset; - - //TODO: find a way to optimize this for common sizes like 4 or 8 bytes! - int carry = 0; - while (size-- > 0) - { - carry += *left + *right++; - *left++ = (byte)carry; - carry >>= 8; - } - } - } - } - } - - public Slice ApplyBitAnd(Slice value) - { - var tmp = PrepareValueForAtomicOperation(value, this.Value.Count); - BufferBitAnd(tmp, 0, this.Value.Array, this.Value.Offset, this.Value.Count); - return Slice.Create(tmp); - } - - public void ApplyBitAndTo(UnmanagedSliceBuilder value) - { - uint size = checked((uint)this.Value.Count); - - // if the value is empty, then 0 AND * will always be zero - if (value.Count == 0) - { - value.Resize(size, 0); - return; - } - - // truncate the value if larger, or pad it with zeroes if shorter - value.Resize(size, 0); - - if (size > 0) - { - unsafe - { - fixed (byte* ptr = this.Value.Array) - { - byte* left = value.Data; - byte* right = ptr + this.Value.Offset; - - //TODO: find a way to optimize this for common sizes like 4 or 8 bytes! - while (size-- > 0) - { - *left++ &= *right++; - } - } - } - } - } - - public Slice ApplyBitOr(Slice value) - { - var tmp = PrepareValueForAtomicOperation(value, this.Value.Count); - BufferBitOr(tmp, 0, this.Value.Array, this.Value.Offset, this.Value.Count); - return Slice.Create(tmp); - } - - public void ApplyBitOrTo(UnmanagedSliceBuilder value) - { - uint size = checked((uint)this.Value.Count); - - // truncate the value if larger, or pad it with zeroes if shorter - value.Resize(size, 0); - - if (size > 0) - { - unsafe - { - fixed (byte* ptr = this.Value.Array) - { - byte* left = value.Data; - byte* right = ptr + this.Value.Offset; - - //TODO: find a way to optimize this for common sizes like 4 or 8 bytes! - while (size-- > 0) - { - *left++ |= *right++; - } - } - } - } - } - - public Slice ApplyBitXor(Slice value) - { - var tmp = PrepareValueForAtomicOperation(value, this.Value.Count); - BufferBitXor(tmp, 0, this.Value.Array, this.Value.Offset, this.Value.Count); - return Slice.Create(tmp); - } - - public void ApplyBitXorTo(UnmanagedSliceBuilder value) - { - uint size = checked((uint)this.Value.Count); - - // truncate the value if larger, or pad it with zeroes if shorter - value.Resize(size, 0); - - if (size > 0) - { - unsafe - { - fixed (byte* ptr = this.Value.Array) - { - byte* left = value.Data; - byte* right = ptr + this.Value.Offset; - - //TODO: find a way to optimize this for common sizes like 4 or 8 bytes! - while (size-- > 0) - { - *left++ ^= *right++; - } - } - } - } - } - - internal static int BufferAdd(byte[] buffer, int offset, byte[] arg, int argOffset, int count) - { - // TODO: optimize this! - int carry = 0; - while (count-- > 0) - { - carry += buffer[offset] + arg[argOffset++]; - buffer[offset++] = (byte)carry; - carry >>= 8; - } - return carry; - } - - internal static void BufferBitAnd(byte[] buffer, int offset, byte[] arg, int argOffset, int count) - { - while (count-- > 0) - { - buffer[offset++] &= arg[argOffset++]; - } - } - - internal static void BufferBitOr(byte[] buffer, int offset, byte[] arg, int argOffset, int count) - { - while (count-- > 0) - { - buffer[offset++] |= arg[argOffset++]; - } - } - - internal static void BufferBitXor(byte[] buffer, int offset, byte[] arg, int argOffset, int count) - { - while (count-- > 0) - { - buffer[offset++] ^= arg[argOffset++]; - } - } - - internal static WriteCommand MergeSetAndAtomicOperation(WriteCommand command, Operation op, Slice argument) - { - // truncate/resize the previous value to the size of the add - int size = argument.Count; - var tmp = PrepareValueForAtomicOperation(command.Value, size); - - switch (op) - { - case Operation.AtomicAdd: - { // do a littlee-endian ADD between the two buffers - BufferAdd(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitAnd: - { // do an AND between the two buffers - BufferBitAnd(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitOr: - { // do a OR between the two buffers - BufferBitOr(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitXor: - { // do a XOR between the two buffers - BufferBitXor(tmp, 0, argument.Array, argument.Offset, size); - break; - } - default: - { // not supposed to happen - throw new InvalidOperationException(); - } - } - - return new WriteCommand(Operation.Set, command.Key, Slice.Create(tmp)); - } - - internal static WriteCommand MergeTwoAtomicOperations(WriteCommand command, Slice argument) - { - // truncate/resize the previous value to the size of the add - int size = argument.Count; - var tmp = PrepareValueForAtomicOperation(command.Value, size); - - switch (command.Type) - { - case Operation.AtomicAdd: - { // do a littlee-endian ADD between the two buffers - BufferAdd(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitAnd: - { // do an AND between the two buffers - BufferBitAnd(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitOr: - { // do a OR between the two buffers - BufferBitOr(tmp, 0, argument.Array, argument.Offset, size); - break; - } - case Operation.AtomicBitXor: - { // do a XOR between the two buffers - BufferBitXor(tmp, 0, argument.Array, argument.Offset, size); - break; - } - default: - { // not supposed to happen - throw new InvalidOperationException(); - } - } - - return new WriteCommand(command.Type, command.Key, Slice.Create(tmp)); - } - - internal static Slice ApplyTo(WriteCommand[] commands, Slice value) - { - var result = value; - for(int i=0;i(SliceComparer.Default); - m_writes = new ColaOrderedDictionary(SliceComparer.Default); - m_readConflicts = new ColaRangeSet(SliceComparer.Default); - m_writeConflicts = new ColaRangeSet(SliceComparer.Default); - } - else - { - m_clears.Clear(); - m_writes.Clear(); - m_readConflicts.Clear(); - m_writeConflicts.Clear(); - } - - m_retryCount = 0; - this.AccessSystemKeys = NO_ACCESS; - this.NextWriteNoWriteConflictRange = false; - this.ReadYourWritesDisable = false; - } - } - - public int Size - { - get { return m_buffer.Size; } - } - - public FdbIsolationLevel IsolationLevel - { - get - { - //BUGBUG: this is currently a lie! until we support "ReadYourWrite", we actually only support Snapshot isolation level! - return FdbIsolationLevel.Serializable; - } - } - - /// Adds a range to teh clear list of this transaction - /// Must be called with m_lock taken - private void AddClearCommand_NeedsLocking(FdbKeyRange range) - { - // merge the cleared range with the others - m_clears.Mark(range.Begin, range.End); - - // remove all writes that where in this range - var keys = m_writes.FindBetween(range.Begin, true, range.End, false).ToList(); - if (keys.Count > 0) - { - foreach(var key in keys) - { - m_writes.Remove(key); - } - } - } - - /// Adds a command to the write list of this transaction - /// Must be called with m_lock taken - private void AddWriteCommand_NeedsLocking(WriteCommand command) - { - var commands = new WriteCommand[1]; - commands[0] = command; - - if (!m_writes.GetOrAdd(command.Key, commands, out commands)) - { // there is already a command for that key - - if (command.Type == Operation.Set) - { // Set always overwrites everything - if (commands.Length == 1) - { // reuse the command array - commands[0] = command; - return; - } - // overwrite - m_writes.SetItem(command.Key, new[] { command }); - return; - } - - var last = commands[commands.Length - 1]; - if (last.Type == Operation.Set) - { // "SET(X) x ATOMIC(op, P)" are merged into "SET(X')" with X' = atomic(op, X, P) - Contract.Assert(commands.Length == 1); - - command = WriteCommand.MergeSetAndAtomicOperation(last, command.Type, command.Value); - // update in place - commands[commands.Length - 1] = command; - return; - - } - - if (last.Type == command.Type) - { // atomics of the same kind can be merged - - command = WriteCommand.MergeTwoAtomicOperations(last, command.Value); - // update in place - commands[commands.Length - 1] = command; - return; - } - - // just queue the command at the end - Array.Resize(ref commands, commands.Length + 1); - commands[commands.Length - 1] = command; - - m_writes.SetItem(command.Key, commands); - - } - } - - /// Read and clear the NextWriteNoConflict flags. - /// Value of the flag, which is cleared for the following write. - /// Must be called with m_lock taken - private bool ConsumeNextWriteNoConflict_NeedsLocking() - { - if (this.NextWriteNoWriteConflictRange) - { - this.NextWriteNoWriteConflictRange = false; - return true; - } - return false; - } - - /// Adds a range to the write conflict list - /// Must be called with m_lock taken - private void AddWriteConflict_NeedsLocking(FdbKeyRange range) - { - m_writeConflicts.Mark(range.Begin, range.End); - } - - /// Adds a range to the read conflict list - /// Must be called with m_lock taken - private void AddReadConflict_NeedsLocking(FdbKeyRange range) - { - m_readConflicts.Mark(range.Begin, range.End); - } - - private void CheckAccessToSystemKeys(Slice key, bool end = false) - { - if (this.AccessSystemKeys == 0 && key[0] == 0xFF) - { // access to system keys is not allowed - if (!end || key.Count > 1) - { - throw new FdbException(FdbError.KeyOutsideLegalRange); - } - } - } - - private Slice MergeResultWithLocalState(Slice key, Slice value) - { - WriteCommand[] commands; - if (m_writes.TryGetValue(key, out commands)) - { // the key will be mutated by this transaction - return WriteCommand.ApplyTo(commands, value); - } - - if (m_clears.ContainsKey(key)) - { // the key will be deleted by this transaction - return Slice.Nil; - } - - return value; - } - - public Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken) - { - Contract.Requires(key.HasValue); - cancellationToken.ThrowIfCancellationRequested(); - - CheckAccessToSystemKeys(key); - - FdbKeyRange range; - lock (m_buffer) - { - range = m_buffer.InternRangeFromKey(key); - } - - // we need the read version - EnsureHasReadVersion(); - - // read the value in the db - //TODO: how to lock ? - var results = m_db.GetValuesAtVersion(new Slice[] { range.Begin }, m_readVersion.Value); - Contract.Assert(results != null && results.Length == 1); - var result = results[0]; - - // snapshot read always see the db, regular read must merge with local mutation, unless option ReadYourWrites is set - if (!snapshot && !this.ReadYourWritesDisable) - { // we need to merge the db state with the local mutations - result = MergeResultWithLocalState(range.Begin, result); - } - - if (!snapshot) - { - lock (m_lock) - { - AddReadConflict_NeedsLocking(range); - } - } - - return Task.FromResult(result); - } - - public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationToken cancellationToken) - { - Contract.Requires(keys != null); - cancellationToken.ThrowIfCancellationRequested(); - - // order and check the keys - var ordered = new Slice[keys.Length]; - for (int i = 0; i < keys.Length;i++) - { - var key = keys[i]; - if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty"); - CheckAccessToSystemKeys(key); - ordered[i] = key; - } - if (ordered.Length > 1) - { // the db expect the keys to be sorted - Array.Sort(ordered, SliceComparer.Default); - } - - // we need the read version - EnsureHasReadVersion(); - - var ranges = new FdbKeyRange[ordered.Length]; - lock (m_buffer) - { - for (int i = 0; i < ordered.Length; i++) - { - ranges[i] = m_buffer.InternRangeFromKey(ordered[i]); - ordered[i] = ranges[i].Begin; - } - } - - // read the values in the db - //TODO: how to lock ? - var results = m_db.GetValuesAtVersion(ordered, m_readVersion.Value); - - // snapshot read always see the db, regular read must merge with local mutation, unless option ReadYourWrites is set - if (!snapshot && !this.ReadYourWritesDisable) - { // we need to merge the db state with the local mutations - for (int i = 0; i < ordered.Length; i++) - { - results[i] = MergeResultWithLocalState(ordered[i], results[i]); - } - } - - if (!snapshot) - { - lock (m_lock) - { - for (int i = 0; i < ranges.Length; i++) - { - AddReadConflict_NeedsLocking(ranges[i]); - } - } - } - - return Task.FromResult(results); - } - - private sealed class SelectorKeyComparer : IComparer - { - - public static readonly SelectorKeyComparer Default = new SelectorKeyComparer(); - - private static readonly SliceComparer s_comparer = SliceComparer.Default; - - private SelectorKeyComparer() - { } - - public int Compare(FdbKeySelector x, FdbKeySelector y) - { - return s_comparer.Compare(x.Key, y.Key); - } - } - - public async Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken) - { - Contract.Requires(selector.Key.HasValue); - cancellationToken.ThrowIfCancellationRequested(); - - CheckAccessToSystemKeys(selector.Key, end: true); - - //Trace.WriteLine("## GetKey " + selector + ", snapshot=" + snapshot); - - FdbKeyRange keyRange; - lock (m_buffer) - { - keyRange = m_buffer.InternRangeFromKey(selector.Key); - selector = new FdbKeySelector(keyRange.Begin, selector.OrEqual, selector.Offset); - } - - // we need the read version - EnsureHasReadVersion(); - - var results = await m_db.GetKeysAtVersion(new [] { selector }, m_readVersion.Value).ConfigureAwait(false); - Contract.Assert(results != null && results.Length == 1); - var result = results[0]; - - FdbKeyRange resultRange; - int c = result.CompareTo(selector.Key); - if (c == 0) - { // the result is identical to the key - resultRange = keyRange; - result = keyRange.Begin; - } - else - { // intern the result - lock(m_buffer) - { - resultRange = m_buffer.InternRangeFromKey(result); - result = resultRange.Begin; - } - } - - //TODO: how to merge the results with the local state mutations ? - // => add values that were inserted - // => remove values that were cleared - // => change the value of keys that were mutated locally - - if (!snapshot) - { - lock (m_lock) - { - //TODO: use the result to create the conflict range (between the resolver key and the returned key) - if (c == 0) - { // the key itself was selected, so it can only conflict if it gets deleted by another transaction - // [ result, result+\0 ) - AddReadConflict_NeedsLocking(resultRange); - } - else if (c < 0) - { // the result is before the selected key, so any change between them (including deletion of the result) will conflict - // orEqual == true => [ result, key + \0 ) - // orEqual == false => [ result, key ) - AddReadConflict_NeedsLocking(FdbKeyRange.Create(resultRange.Begin, selector.OrEqual ? keyRange.End : keyRange.Begin)); - } - else - { // the result is after the selected key, so any change between it and the result will conflict - // orEqual == true => [ key + \0, result + \0 ) - // orEqual == false => [ key , result + \0 ) - AddReadConflict_NeedsLocking(FdbKeyRange.Create(selector.OrEqual ? keyRange.End : keyRange.Begin, resultRange.End)); - } - } - } - - return result; - } - - public async Task GetKeysAsync(FdbKeySelector[] selectors, bool snapshot, CancellationToken cancellationToken) - { - Contract.Requires(selectors != null); - - cancellationToken.ThrowIfCancellationRequested(); - - // order and check the keys - var ordered = new FdbKeySelector[selectors.Length]; - for (int i = 0; i < selectors.Length; i++) - { - if (selectors[i].Key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty"); - //CheckAccessToSystemKeys(key); - ordered[i] = selectors[i]; - } - if (ordered.Length > 1) - { // the db expects the keys to be sorted - Array.Sort(ordered, SelectorKeyComparer.Default); - } - - // we need the read version - EnsureHasReadVersion(); - - lock (m_buffer) - { - for (int i = 0; i < ordered.Length; i++) - { - ordered[i] = m_buffer.InternSelector(ordered[i]); - } - } - - var results = await m_db.GetKeysAtVersion(ordered, m_readVersion.Value).ConfigureAwait(false); - - if (!snapshot) - { - lock (m_lock) - { -#if !DEBUGz - throw new NotImplementedException("TODO: track read ranges in GetKeysAsync() !"); -#endif - } - } - - return results; - } - - public async Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken) - { - Contract.Requires(beginInclusive.Key.HasValue && endExclusive.Key.HasValue && options != null); - - cancellationToken.ThrowIfCancellationRequested(); - - //TODO: check system keys - - //Trace.WriteLine("## GetRange " + beginInclusive + " <= k < " + endExclusive + ", limit=" + options.Limit + ", reverse=" + options.Reverse + ", snapshot=" + snapshot); - - lock (m_buffer) - { - beginInclusive = m_buffer.InternSelector(beginInclusive); - endExclusive = m_buffer.InternSelector(endExclusive); - } - - // we need the read version - EnsureHasReadVersion(); - - options = FdbRangeOptions.EnsureDefaults(options, null, null, FdbStreamingMode.Iterator, false); - options.EnsureLegalValues(); - - var result = await m_db.GetRangeAtVersion(beginInclusive, endExclusive, options.Limit ?? 0, options.TargetBytes ?? 0, options.Mode.Value, iteration, options.Reverse.Value, m_readVersion.Value).ConfigureAwait(false); - - if (!snapshot) - { - lock (m_lock) - { - //TODO: use the result to create the conflict range (between the resolver key and the returned key) - //AddReadConflict_NeedsLocking(range); - } - } - return result; - } - - public void Set(Slice key, Slice value) - { - // check - if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty"); - if (value.IsNull) throw new ArgumentNullException(nameof(value), "Value cannot be null"); - CheckAccessToSystemKeys(key); - - - // first thing is copy the data in our own buffer, and only use those for the rest - FdbKeyRange range; - lock (m_buffer) - { - range = m_buffer.InternRangeFromKey(key); - value = m_buffer.Intern(value); - } - - lock (m_lock) - { - if (!ConsumeNextWriteNoConflict_NeedsLocking()) - { - AddWriteConflict_NeedsLocking(range); - } - AddWriteCommand_NeedsLocking(new WriteCommand(Operation.Set, range.Begin, value)); - } - } - - public void Atomic(Slice key, Slice param, FdbMutationType mutation) - { - // check - if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty"); - if (param.IsNull) throw new ArgumentNullException(nameof(param), "Parameter cannot be null"); - CheckAccessToSystemKeys(key); - - if (mutation != FdbMutationType.Add && mutation != FdbMutationType.BitAnd && mutation != FdbMutationType.BitOr && mutation != FdbMutationType.BitXor && mutation != FdbMutationType.Max && mutation != FdbMutationType.Min) - { - //TODO: throw an FdbException instead? - throw new ArgumentException("Invalid mutation type", nameof(mutation)); - } - - FdbKeyRange range; - lock (m_buffer) - { - range = m_buffer.InternRangeFromKey(key); - param = m_buffer.Intern(param); - } - - lock (m_lock) - { - if (!ConsumeNextWriteNoConflict_NeedsLocking()) - { - AddWriteConflict_NeedsLocking(range); - } - AddWriteCommand_NeedsLocking(new WriteCommand((Operation)mutation, range.Begin, param)); - } - } - - public void Clear(Slice key) - { - // check - if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty"); - CheckAccessToSystemKeys(key); - - FdbKeyRange range; - lock (m_buffer) - { - range = m_buffer.InternRangeFromKey(key); - } - - lock (m_lock) - { - if (!ConsumeNextWriteNoConflict_NeedsLocking()) - { - AddWriteConflict_NeedsLocking(range); - } - AddClearCommand_NeedsLocking(range); - } - } - - public void ClearRange(Slice beginKeyInclusive, Slice endKeyExclusive) - { - // check - if (beginKeyInclusive.IsNullOrEmpty) throw new ArgumentException("Begin key cannot be null or empty"); - if (endKeyExclusive.IsNullOrEmpty) throw new ArgumentException("End key cannot be null or empty"); - CheckAccessToSystemKeys(beginKeyInclusive); - CheckAccessToSystemKeys(endKeyExclusive, end: true); - - FdbKeyRange range; - lock (m_buffer) - { - range = m_buffer.InternRange(beginKeyInclusive, endKeyExclusive); - } - - lock (m_lock) - { - if (!ConsumeNextWriteNoConflict_NeedsLocking()) - { - AddWriteConflict_NeedsLocking(range); - } - AddClearCommand_NeedsLocking(range); - } - } - - public void AddConflictRange(Slice beginKeyInclusive, Slice endKeyExclusive, FdbConflictRangeType type) - { - // check - if (beginKeyInclusive.IsNullOrEmpty) throw new ArgumentException("Begin key cannot be null or empty"); - if (endKeyExclusive.IsNullOrEmpty) throw new ArgumentException("End key cannot be null or empty"); - if (type != FdbConflictRangeType.Read && type != FdbConflictRangeType.Write) throw new ArgumentOutOfRangeException(nameof(type), "Invalid range conflict type"); - - CheckAccessToSystemKeys(beginKeyInclusive); - CheckAccessToSystemKeys(endKeyExclusive, end: true); - - FdbKeyRange range; - lock(m_buffer) - { - range = m_buffer.InternRange(beginKeyInclusive, endKeyExclusive); - } - - lock (m_lock) - { - if (type == FdbConflictRangeType.Read) - { - AddReadConflict_NeedsLocking(range); - } - else - { - AddWriteConflict_NeedsLocking(range); - } - } - } - - public void Reset() - { - //TODO: kill any pending "async" reads - //TODO: release the current read version and/or transaction window ? - Initialize(true); - } - - public Task CommitAsync(CancellationToken cancellationToken) - { - Log("CommitAsync() called"); - cancellationToken.ThrowIfCancellationRequested(); - - if (!m_readVersion.HasValue) - { - EnsureHasReadVersion(); - } - -#if DUMP_TRANSACTION_STATE - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "=== COMMITING TRANSACTION {0} ===", this.Id)); - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# ReadVersion: {0}", m_readVersion ?? -1)); - - if (m_readConflicts.Count == 0) - { - Trace.WriteLine("# Read Conflicts: none"); - } - else - { - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# Read Conflicts: ({0}) => {1}", m_readConflicts.Count, m_readConflicts.ToString())); - } - - if (m_writeConflicts.Count == 0) - { - Trace.WriteLine("# Write Conflicts: none"); - } - else - { - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# Write Conflicts: ({0}) => {1}", m_writeConflicts.Count, m_writeConflicts.ToString())); - } - - if (m_clears.Count == 0) - { - Trace.WriteLine("# Clears: none"); - } - else - { - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# Clears: ({0})", m_clears.Count)); - foreach (var op in m_clears) - { - Trace.WriteLine(" > " + new FdbKeyRange(op.Begin, op.End)); - } - } - - if (m_writes.Count == 0) - { - Trace.WriteLine("# Writes: none"); - } - else - { - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# Writes: ({0})", m_writes.Count)); - foreach (var op in m_writes) - { - Trace.WriteLine(" > " + String.Join("; ", op.Value)); - } - } - - var pages = m_buffer.GetPages(); - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "# Slice buffer: {0} bytes in {1} pages ({2} allocated, {3:##0.00}% wasted)", m_buffer.Size, pages.Length, m_buffer.Allocated, 100.0 - (m_buffer.Size * 100.0 / m_buffer.Allocated))); - foreach(var page in pages) - { - Trace.WriteLine(" > " + page.ToString()); - } -#endif - - //m_committedVersion = await m_db.CommitTransactionAsync(this, m_readVersion.Value, m_readConflicts, m_writeConflicts, m_clears, m_writes).ConfigureAwait(false); - - return m_db.EnqueueCommit(this); - -#if DUMP_TRANSACTION_STATE - Trace.WriteLine("=== DONE with commit version " + m_committedVersion); -#endif - } - - internal void CommitInternal() - { - Log("CommitInternalAsync() called"); - m_committedVersion = m_db.CommitTransaction(this, m_readVersion.Value, m_readConflicts, m_writeConflicts, m_clears, m_writes); - Log("committed at " + m_committedVersion); - } - - public long GetCommittedVersion() - { - return m_committedVersion; - } - - public void SetReadVersion(long version) - { - throw new NotImplementedException(); - } - - public async Task OnErrorAsync(FdbError code, CancellationToken cancellationToken) - { - cancellationToken.ThrowIfCancellationRequested(); - - switch (code) - { - case FdbError.TimedOut: - case FdbError.PastVersion: - { // wait a bit - - ++m_retryCount; - if (m_retryCount > this.RetryLimit) - { // max rety limit reached - throw new FdbException(code); - } - - //HACKHACK: implement a real back-off delay logic - await Task.Delay(15, cancellationToken).ConfigureAwait(false); - - this.Reset(); - - return; - } - default: - { - throw new FdbException(code); - } - } - } - - public FdbWatch Watch(Slice key, System.Threading.CancellationToken cancellationToken) - { - Contract.Requires(key.HasValue); - cancellationToken.ThrowIfCancellationRequested(); - - throw new NotSupportedException(); - } - - public Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken) - { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - - throw new NotImplementedException(); - } - - private long EnsureHasReadVersion() - { - if (!m_readVersion.HasValue) - { - m_readVersion = m_db.GetCurrentVersion(); - } - return m_readVersion.Value; - } - - public Task GetReadVersionAsync(CancellationToken cancellationToken) - { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); - return Task.FromResult(EnsureHasReadVersion()); - } - - public void Cancel() - { - if (m_disposed) ThrowDisposed(); - throw new NotImplementedException(); - } - - public int RetryLimit { get; internal set; } - - public int Timeout { get; internal set; } - - /// The transaction has access to the system keyspace - public int AccessSystemKeys { get; internal set; } - const int NO_ACCESS = 0; - const int READ_ACCESS = 1; - const int READ_WRITE_ACCESS = 2; - - /// The next write will not cause a write conflict - public bool NextWriteNoWriteConflictRange { get; internal set; } - - /// If true, the transaction always read the value from the database, and does not see the local mutations - public bool ReadYourWritesDisable { get; internal set; } - - /// Number of retries already done by this transaction - public int RetryCount { get { return m_retryCount; } } - - /// Décode the value of a transaction option into a boolean - private static bool DecodeBooleanOption(Slice data) - { - if (data.Count == 8) - { // spec says that ints should be passed as 8 bytes integers, so we need to accept all zeroes as "false" - return data.ToInt64() != 0; - } - else - { - return data.ToBool(); - } - } - - public void SetOption(FdbTransactionOption option, Slice data) - { - switch(option) - { - case FdbTransactionOption.AccessSystemKeys: - { - this.AccessSystemKeys = (data.IsNullOrEmpty || DecodeBooleanOption(data)) ? READ_WRITE_ACCESS : NO_ACCESS; - break; - } - case FdbTransactionOption.ReadSystemKeys: - { - this.AccessSystemKeys = (data.IsNullOrEmpty || DecodeBooleanOption(data)) ? READ_ACCESS : NO_ACCESS; - break; - } - case FdbTransactionOption.RetryLimit: - { - if (data.Count != 8) throw new FdbException(FdbError.InvalidOptionValue); - long value = data.ToInt64(); - if (value < 0 || value >= int.MaxValue) throw new FdbException(FdbError.InvalidOptionValue); - this.RetryLimit = (int)value; - break; - } - - case FdbTransactionOption.Timeout: - { - if (data.Count != 8) throw new FdbException(FdbError.InvalidOptionValue); - long value = data.ToInt64(); - if (value < 0 || value >= int.MaxValue) throw new FdbException(FdbError.InvalidOptionValue); - this.Timeout = (int)value; - break; - } - - case FdbTransactionOption.NextWriteNoWriteConflictRange: - { - this.NextWriteNoWriteConflictRange = data.IsNullOrEmpty || DecodeBooleanOption(data); - break; - } - case FdbTransactionOption.ReadYourWritesDisable: - { - this.ReadYourWritesDisable = data.IsNullOrEmpty || DecodeBooleanOption(data); - break; - } - - default: - { - throw new FdbException(FdbError.InvalidOption); - } - } - } - - private static void ThrowDisposed() - { - throw new ObjectDisposedException("This transaction has already been disposed."); - } - - public void Dispose() - { - if (m_disposed) - { - //TODO: locking ? - m_disposed = true; - - //TODO! - m_buffer = null; - m_readConflicts = null; - m_writeConflicts = null; - m_clears = null; - m_writes = null; - } - - GC.SuppressFinalize(this); - } - - [Conditional("FULL_DEBUG")] - private static void Log(string msg) - { -#if FULL_DEBUG - Trace.WriteLine("MemoryTransactionHandler[#" + Thread.CurrentThread.ManagedThreadId + "]: " + msg); -#endif - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs deleted file mode 100644 index ff467031d..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaOrderedDictionary.cs +++ /dev/null @@ -1,444 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; - using System.Runtime.InteropServices; - - /// Represent an ordered set of key/value pairs, stored in a Cache Oblivious Lookahead Array - /// Type of ordered keys stored in the dictionary. - /// Type of values stored in the dictionary. - [DebuggerDisplay("Count={m_items.Count}"), DebuggerTypeProxy(typeof(ColaOrderedDictionary<,>.DebugView))] - public class ColaOrderedDictionary : IEnumerable> - { - - /// Debug view helper - private sealed class DebugView - { - private readonly ColaOrderedDictionary m_dictionary; - - public DebugView(ColaOrderedDictionary dictionary) - { - m_dictionary = dictionary; - } - - [DebuggerBrowsable(DebuggerBrowsableState.RootHidden)] - public KeyValuePair[] Items - { - get - { - var tmp = new KeyValuePair[m_dictionary.Count]; - m_dictionary.CopyTo(tmp, 0); - return tmp; - } - } - } - - /// Wrapper for a comparer on the keys of a key/value pair - private sealed class KeyOnlyComparer : IComparer> - { - private readonly IComparer m_comparer; - - public KeyOnlyComparer(IComparer comparer) - { - m_comparer = comparer; - } - - public int Compare(KeyValuePair x, KeyValuePair y) - { - return m_comparer.Compare(x.Key, y.Key); - } - } - - /// COLA array used to store the entries in the dictionary - private readonly ColaStore> m_items; - - /// Comparer for the keys of the dictionary - private readonly IComparer m_keyComparer; - - /// Comparer for the values of the dictionary - private readonly IEqualityComparer m_valueComparer; - - private volatile int m_version; - - #region Constructors... - - public ColaOrderedDictionary(IComparer keyComparer = null, IEqualityComparer valueComparer = null) - : this(0, keyComparer, valueComparer) - { } - - public ColaOrderedDictionary(int capacity) - : this(capacity, null, null) - { } - - public ColaOrderedDictionary(int capacity, IComparer keyComparer, IEqualityComparer valueComparer) - { - m_keyComparer = keyComparer ?? Comparer.Default; - m_valueComparer = valueComparer ?? EqualityComparer.Default; - m_items = new ColaStore>(capacity, new KeyOnlyComparer(m_keyComparer)); - } - - #endregion - - #region Public Properties... - - public int Count - { - get { return m_items.Count; } - } - - public int Capacity - { - get { return m_items.Capacity; } - } - - public TValue this[TKey key] - { - get { return GetValue(key); } - set { SetItem(key, value); } - } - - #endregion - - public void Clear() - { - ++m_version; - m_items.Clear(); - } - - public IComparer KeyComparer - { - get { return m_keyComparer; } - } - - public IEqualityComparer ValueComparer - { - get { return m_valueComparer; } - } - - internal ColaStore> Items - { - get { return m_items; } - } - - /// Adds an entry with the specified key and value to the sorted dictionary. - /// The key of the entry to add. - /// The value of the entry to add. - /// If an entry with the same key already exist in the dictionary. - public void Add(TKey key, TValue value) - { - if (key == null) ThrowKeyCannotBeNull(); - - ++m_version; - if (!m_items.SetOrAdd(new KeyValuePair(key, value), overwriteExistingValue: false)) - { - --m_version; - ThrowKeyAlreadyExists(); - } - } - - /// Sets the specified key and value in the immutable sorted dictionary, possibly overwriting an existing value for the given key. - /// The key of the entry to add. - /// The key value to set. - public void SetItem(TKey key, TValue value) - { - if (key == null) ThrowKeyCannotBeNull(); - ++m_version; - m_items.SetOrAdd(new KeyValuePair(key, value), overwriteExistingValue: true); - } - - /// Try to add an entry with the specified key and value to the sorted dictionary, or update its value if it already exists. - /// The key of the entry to add. - /// The value of the entry to add. - /// true if the key did not previously exist and was inserted; otherwise, false. - public bool AddOrUpdate(TKey key, TValue value) - { - if (key == null) ThrowKeyCannotBeNull(); - - KeyValuePair entry; - int offset, level = m_items.Find(new KeyValuePair(key, default(TValue)), out offset, out entry); - if (level >= 0) - { // already exists - // keep the old key, and update the value - ++m_version; - m_items.SetAt(level, offset, new KeyValuePair(entry.Key, value)); - return false; - } - - ++m_version; - m_items.Insert(new KeyValuePair(key, value)); - return true; - } - - /// Try to add an entry with the specified key and value to the sorted dictionary, if it does not already exists. - /// The key of the entry to add. - /// The value of the entry to add. - /// Receives the previous value if already exists, or if it was inserted - /// true if the key did not previously exist and was inserted; otherwise, false. - public bool GetOrAdd(TKey key, TValue value, out TValue actualValue) - { - if (key == null) ThrowKeyCannotBeNull(); - - KeyValuePair entry; - int _, level = m_items.Find(new KeyValuePair(key, default(TValue)), out _, out entry); - if (level >= 0) - { // already exists - actualValue = entry.Value; - return false; - } - - ++m_version; - m_items.Insert(new KeyValuePair(key, value)); - actualValue = value; - return true; - } - - public bool ContainsKey(TKey key) - { - if (key == null) ThrowKeyCannotBeNull(); - - int _; - KeyValuePair __; - return m_items.Find(new KeyValuePair(key, default(TValue)), out _, out __) >= 0; - } - - public bool ContainsValue(TValue value) - { - foreach(var kvp in m_items.IterateUnordered()) - { - if (m_valueComparer.Equals(kvp.Value)) return true; - } - return false; - } - - /// Determines whether this dictionary contains a specified key. - /// The key to search for. - /// The matching key located in the dictionary if found, or equalkey if no match is found. - /// true if a match for is found; otherwise, false. - public bool TryGetKey(TKey equalKey, out TKey actualKey) - { - if (equalKey == null) ThrowKeyCannotBeNull(); - - KeyValuePair entry; - int _, level = m_items.Find(new KeyValuePair(equalKey, default(TValue)), out _, out entry); - if (level < 0) - { - actualKey = equalKey; - return false; - } - actualKey = entry.Key; - return true; - } - - /// Gets the value associated with the specified key. - /// The key to search for. - /// - /// true if a match for is found; otherwise, false. - public bool TryGetValue(TKey key, out TValue value) - { - if (key == null) ThrowKeyCannotBeNull(); - - KeyValuePair entry; - int _, level = m_items.Find(new KeyValuePair(key, default(TValue)), out _, out entry); - if (level < 0) - { - value = default(TValue); - return false; - } - value = entry.Value; - return true; - } - - public TValue GetValue(TKey key) - { - if (key == null) ThrowKeyCannotBeNull(); - - KeyValuePair entry; - int _, level = m_items.Find(new KeyValuePair(key, default(TValue)), out _, out entry); - if (level < 0) - { - ThrowKeyNotFound(); - } - return entry.Value; - } - - /// Gets the existing key and value associated with the specified key. - /// The key to search for. - /// The matching key and value pair located in the dictionary if found. - /// true if a match for is found; otherwise, false. - public bool TryGetKeyValue(TKey key, out KeyValuePair entry) - { - if (key == null) ThrowKeyCannotBeNull(); - - int _, level = m_items.Find(new KeyValuePair(key, default(TValue)), out _, out entry); - return level >= 0; - } - - /// Removes the entry with the specified key from the dictionary. - /// The key of the entry to remove. - /// true if the value was found and removed from the dictionary; otherwise, false. - /// It is NOT allowed to remove keys while iterating on the dictionary at the same time! - public bool Remove(TKey key) - { - if (key == null) ThrowKeyCannotBeNull(); - - KeyValuePair _; - int offset, level = m_items.Find(new KeyValuePair(key, default(TValue)), out offset, out _); - - if (level >= 0) - { - ++m_version; - m_items.RemoveAt(level, offset); - return true; - } - return false; - } - - /// Remove the entries with the specified keys from the dictionary. - /// The keys of the entries to remove. - /// Number of entries that were found and removed. - /// It is NOT allowed to remove keys while iterating on the dictionary at the same time! - public int RemoveRange(IEnumerable keys) - { - if (keys == null) throw new ArgumentNullException("keys"); - - // we need to protect against people passing in the result of calling FindBetween, - // because we can't remove while iterating at the same time ! - - int count = 0; - foreach (var key in keys) - { - if (Remove(key)) ++count; - } - return count; - } - - /// Enumerate all the keys in the dictionary that are in the specified range - /// Start of the range - /// If true, the key is included in the range - /// End of the range - /// If true, the key is included in the range - /// Unordered list of the all the keys in the dictionary that are in the range. - /// There is no guarantee in the actual order of the keys returned. It is also not allowed to remove keys while iterating over the sequence. - public IEnumerable FindBetween(TKey begin, bool beginOrEqual, TKey end, bool endOrEqual) - { - // return the unordered list of all the keys that are between the begin/end pair. - // each bound is included in the list if its corresponding 'orEqual' is set to true - - if (m_items.Count > 0) - { - var start = new KeyValuePair(begin, default(TValue)); - var stop = new KeyValuePair(end, default(TValue)); - - foreach (var kvp in m_items.FindBetween(start, beginOrEqual, stop, endOrEqual, int.MaxValue)) - { - yield return kvp.Key; - } - } - } - - /// Returns an enumerator that iterates through the ordered dictionary - public ColaStore.Enumerator> GetEnumerator() - { - return new ColaStore.Enumerator>(m_items, reverse: false); - } - - IEnumerator> IEnumerable>.GetEnumerator() - { - return this.GetEnumerator(); - } - - System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - internal void CopyTo(KeyValuePair[] array, int index) - { - m_items.CopyTo(array, index, m_items.Count); - } - - private static void ThrowKeyCannotBeNull() - { - throw new ArgumentNullException("key"); - } - - private static void ThrowKeyNotFound() - { - throw new KeyNotFoundException(); - } - - private static void ThrowKeyAlreadyExists() - { - throw new InvalidOperationException("An entry with the same key but a different value already exists."); - } - - //TODO: remove or set to internal ! - [Conditional("DEBUG")] - public void Debug_Dump() - { -#if DEBUG - Trace.WriteLine("Dumping ColaOrderedDictionary<" + typeof(TKey).Name + ", " + typeof(TValue).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); - m_items.Debug_Dump(); -#endif - } - - [StructLayout(LayoutKind.Sequential)] - public struct Enumerator : IEnumerator>, IDisposable - { - private const int NOT_FOUND = -1; - - private readonly int m_version; - private readonly ColaOrderedDictionary m_parent; - private ColaStore.Enumerator> m_iterator; - - internal Enumerator(ColaOrderedDictionary parent, bool reverse) - { - m_version = parent.m_version; - m_parent = parent; - m_iterator = new ColaStore.Enumerator>(parent.m_items, reverse); - } - - public bool MoveNext() - { - if (m_version != m_parent.m_version) - { - ColaStore.ThrowStoreVersionChanged(); - } - - return m_iterator.MoveNext(); - } - - public KeyValuePair Current - { - get { return m_iterator.Current; } - } - - public void Dispose() - { - // we are a struct that can be copied by value, so there is no guarantee that Dispose() will accomplish anything anyway... - } - - object System.Collections.IEnumerator.Current - { - get { return m_iterator.Current; } - } - - void System.Collections.IEnumerator.Reset() - { - if (m_version != m_parent.m_version) - { - ColaStore.ThrowStoreVersionChanged(); - } - m_iterator = new ColaStore.Enumerator>(m_parent.m_items, m_iterator.Reverse); - } - - } - - } -} diff --git a/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs b/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs deleted file mode 100644 index 9582ce49c..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaOrderedSet.cs +++ /dev/null @@ -1,287 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - /// Represent an ordered set of elements, stored in a Cache Oblivous Lookup Array - /// Type of elements stored in the set - /// Inserts are in O(LogN) amortized. Lookups are in O(Log(N)) - public class ColaOrderedSet : IEnumerable - { - private const int NOT_FOUND = -1; - - /// COLA array used to store the elements in the set - private readonly ColaStore m_items; - - private volatile int m_version; - - #region Constructors... - - public ColaOrderedSet() - : this(0, Comparer.Default) - { } - - public ColaOrderedSet(int capacity) - : this(capacity, Comparer.Default) - { } - - public ColaOrderedSet(IComparer comparer) - : this(0, comparer) - { } - - public ColaOrderedSet(int capacity, IComparer comparer) - { - if (capacity < 0) throw new ArgumentOutOfRangeException("capacity", "Capacity cannot be less than zero."); - Contract.EndContractBlock(); - - m_items = new ColaStore(capacity, comparer ?? Comparer.Default); - } - - #endregion - - #region Public Properties... - - /// Gets the number of elements in the immutable sorted set. - public int Count - { - get { return m_items.Count; } - } - - /// Current capacity of the set - public int Capacity - { - get { return m_items.Capacity; } - } - - public IComparer Comparer - { - get { return m_items.Comparer; } - } - - public T this[int index] - { - get - { - if (index < 0 || index >= m_items.Count) ThrowIndexOutOfRangeException(); - int offset; - int level = ColaStore.MapOffsetToLocation(m_items.Count, index, out offset); - Contract.Assert(level >= 0); - return m_items.GetAt(level, offset); - } - } - - private static void ThrowIndexOutOfRangeException() - { - throw new IndexOutOfRangeException("Index is out of range"); - } - - #endregion - - #region Public Methods... - - public void Clear() - { - ++m_version; - m_items.Clear(); - } - - /// Adds the specified value to this ordered set. - /// The value to add. - /// If the value already exists in the set, it will not be overwritten - public bool Add(T value) - { - ++m_version; - if (!m_items.SetOrAdd(value, overwriteExistingValue: false)) - { - --m_version; - return false; - } - return true; - } - - /// Adds or overwrite the specified value to this ordered set. - /// The value to add. - /// If the value already exists in the set, it will be overwritten by - public bool Set(T value) - { - ++m_version; - return m_items.SetOrAdd(value, overwriteExistingValue: true); - } - - public bool TryRemove(T value, out T actualValue) - { - int offset; - int level = m_items.Find(value, out offset, out actualValue); - if (level != NOT_FOUND) - { - ++m_version; - m_items.RemoveAt(level, offset); - return true; - } - return false; - } - - public bool Remove(T value) - { - T _; - return TryRemove(value, out _); - } - - public T RemoveAt(int arrayIndex) - { - if (arrayIndex < 0 || arrayIndex >= m_items.Count) throw new ArgumentOutOfRangeException("arrayIndex", "Index is outside the array"); - - int offset; - int level = ColaStore.MapOffsetToLocation(m_items.Count, arrayIndex, out offset); - Contract.Assert(level >= 0 && offset >= 0 && offset < 1 << level); - - ++m_version; - return m_items.RemoveAt(level, offset); - } - - /// Determines whether this immutable sorted set contains the specified value. - /// The value to check for. - /// true if the set contains the specified value; otherwise, false. - public bool Contains(T value) - { - int _; - T __; - return m_items.Find(value, out _, out __) >= 0; - } - - /// Find an element - /// - /// The zero-based index of the first occurrence of within the entire list, if found; otherwise, –1. - public int IndexOf(T value) - { - T _; - int offset, level = m_items.Find(value, out offset, out _); - if (level >= 0) - { - return ColaStore.MapLocationToOffset(m_items.Count, level, offset); - } - return NOT_FOUND; - } - - /// Searches the set for a given value and returns the equal value it finds, if any. - /// The value to search for. - /// The value from the set that the search found, or the original value if the search yielded no match. - /// A value indicating whether the search was successful. - public bool TryGetValue(T value, out T actualValue) - { - int _; - return m_items.Find(value, out _, out actualValue) >= 0; - } - - /// Copy the ordered elements of the set to an array - /// The one-dimensional array that is the destination of the elements copied from collection. The array must have zero-based indexing. - public void CopyTo(T[] array) - { - Contract.Requires(array != null); - m_items.CopyTo(array, 0, array.Length); - } - - /// Copies the ordered elements of the set to an array, starting at a particular array index. - /// The one-dimensional array that is the destination of the elements copied from collection. The array must have zero-based indexing. - /// The zero-based index in array at which copying begins. - public void CopyTo(T[] array, int arrayIndex) - { - Contract.Requires(array != null && arrayIndex >= 0); - m_items.CopyTo(array, arrayIndex, m_items.Count); - } - - public void CopyTo(T[] array, int arrayIndex, int count) - { - Contract.Requires(array != null && arrayIndex >= 0 && count >= 0); - m_items.CopyTo(array, arrayIndex, count); - } - - public ColaStore.Enumerator GetEnumerator() - { - return new ColaStore.Enumerator(m_items, reverse: false); - } - - IEnumerator IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - #endregion - - //TODO: remove or set to internal ! - [Conditional("DEBUG")] - public void Debug_Dump() - { -#if DEBUG - Trace.WriteLine("Dumping ColaOrderedSet<" + typeof(T).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); - m_items.Debug_Dump(); -#endif - } - - [StructLayout(LayoutKind.Sequential)] - public struct Enumerator : IEnumerator, IDisposable - { - private const int NOT_FOUND = -1; - - private readonly int m_version; - private readonly ColaOrderedSet m_parent; - private ColaStore.Enumerator m_iterator; - - internal Enumerator(ColaOrderedSet parent, bool reverse) - { - m_version = parent.m_version; - m_parent = parent; - m_iterator = new ColaStore.Enumerator(parent.m_items, reverse); - } - - public bool MoveNext() - { - if (m_version != m_parent.m_version) - { - ColaStore.ThrowStoreVersionChanged(); - } - - return m_iterator.MoveNext(); - } - - public T Current - { - get { return m_iterator.Current; } - } - - public void Dispose() - { - // we are a struct that can be copied by value, so there is no guarantee that Dispose() will accomplish anything anyway... - } - - object System.Collections.IEnumerator.Current - { - get { return m_iterator.Current; } - } - - void System.Collections.IEnumerator.Reset() - { - if (m_version != m_parent.m_version) - { - ColaStore.ThrowStoreVersionChanged(); - } - m_iterator = new ColaStore.Enumerator(m_parent.m_items, m_iterator.Reverse); - } - - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs deleted file mode 100644 index 10ddf94d5..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeDictionary.cs +++ /dev/null @@ -1,1020 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -// enables consitency checks after each operation to the set -//#define ENFORCE_INVARIANTS - -using FoundationDB.Layers.Tuples; - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Globalization; - - /// Represent an ordered list of ranges, each associated with a specific value, stored in a Cache Oblivous Lookup Array - /// Type of the keys stored in the set - /// Type of the values associated with each range - [DebuggerDisplay("Count={m_items.Count}, Bounds={m_bounds.Begin}..{m_bounds.End}")] - public sealed class ColaRangeDictionary : IEnumerable.Entry> - { - // This class is equivalent to ColaRangeSet, except that we have an extra value stored in each range. - // That means that we only merge ranges with the same value, and split/truncate/overwrite ranges with different values - - // INVARIANTS - // * If there is at least on range, the set is not empty (ie: Begin <= End) - // * The Begin key is INCLUDED in range, but the End key is EXCLUDED from the range (ie: Begin <= K < End) - // * The End key of a range MUST be GREATER THAN or EQUAL TO the Begin key of a range (ie: ranges are not backwards) - // * The End key of a range CANNOT be GREATER THAN the Begin key of the next range (ie: ranges do not overlap) - // * If the End key of a range is EQUAL TO the Begin key of the next range, then they MUST have a DIFFERENT value - - /// Mutable range - public sealed class Entry - { - public TKey Begin { get; internal set; } - public TKey End { get; internal set; } - public TValue Value { get; internal set; } - - public Entry(TKey begin, TKey end, TValue value) - { - this.Begin = begin; - this.End = end; - this.Value = value; - } - - /// Overwrite this range with another one - /// New range that will overwrite the current instance - internal void Set(Entry other) - { - this.Begin = other.Begin; - this.End = other.End; - this.Value = other.Value; - } - - public override string ToString() - { - return String.Format(CultureInfo.InvariantCulture, "({0} ~ {1}, {2})", this.Begin, this.End, this.Value); - } - } - - /// Range comparer that only test the Begin key - private sealed class BeginKeyComparer : IComparer - { - private readonly IComparer m_comparer; - - public BeginKeyComparer(IComparer comparer) - { - m_comparer = comparer; - } - - public int Compare(Entry x, Entry y) - { -#if DEBUG - if (x == null || y == null) Debugger.Break(); -#endif - return m_comparer.Compare(x.Begin, y.Begin); - } - } - - private sealed class EndKeyComparer : IComparer - { - private readonly IComparer m_comparer; - - public EndKeyComparer(IComparer comparer) - { - m_comparer = comparer; - } - - public int Compare(Entry x, Entry y) - { -#if DEBUG - if (x == null || y == null) Debugger.Break(); -#endif - return m_comparer.Compare(x.End, y.End); - } - } - - private readonly ColaStore m_items; - private readonly IComparer m_keyComparer; - private readonly IComparer m_valueComparer; - private readonly Entry m_bounds; - - public ColaRangeDictionary() - : this(0, null, null) - { } - - public ColaRangeDictionary(int capacity) - : this(capacity, null, null) - { } - - public ColaRangeDictionary(IComparer keyComparer, IComparer valueComparer) - : this(0, keyComparer, valueComparer) - { } - - public ColaRangeDictionary(int capacity, IComparer keyComparer, IComparer valueComparer) - { - m_keyComparer = keyComparer ?? Comparer.Default; - m_valueComparer = valueComparer ?? Comparer.Default; - if (capacity == 0) capacity = 15; - m_items = new ColaStore(capacity, new BeginKeyComparer(m_keyComparer)); - m_bounds = new Entry(default(TKey), default(TKey), default(TValue)); - } - - [Conditional("ENFORCE_INVARIANTS")] - private void CheckInvariants() - { -#if ENFORCE_INVARIANTS - Contract.Assert(m_bounds != null); - Debug.WriteLine("INVARIANTS:" + this.ToString() + " <> " + m_bounds.ToString()); - - if (m_items.Count == 0) - { - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.Begin, default(TKey))); - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.End, default(TKey))); - } - else if (m_items.Count == 1) - { - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.Begin, m_items[0].Begin)); - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.End, m_items[0].End)); - } - else - { - Entry previous = null; - Entry first = null; - foreach (var item in this) - { - Contract.Assert(m_keyComparer.Compare(item.Begin, item.End) < 0, "End key should be after begin"); - - if (previous == null) - { - first = item; - } - else - { - int c = m_keyComparer.Compare(previous.End, item.Begin); - if (c > 0) Contract.Assert(false, String.Format("Range overlapping: {0} and {1}", previous, item)); - if (c == 0 && m_valueComparer.Compare(previous.Value, item.Value) == 0) Contract.Assert(false, String.Format("Unmerged ranges: {0} and {1}", previous, item)); - } - previous = item; - } - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.Begin, first.Begin), String.Format("Min bound {0} does not match with {1}", m_bounds.Begin, first.Begin)); - Contract.Assert(EqualityComparer.Default.Equals(m_bounds.End, previous.End), String.Format("Max bound {0} does not match with {1}", m_bounds.End, previous.End)); - } -#endif - } - - public int Count { get { return m_items.Count; } } - - public int Capacity { get { return m_items.Capacity; } } - - public IComparer KeyComparer { get { return m_keyComparer; } } - - public IComparer ValueComparer { get { return m_valueComparer; } } - - public Entry Bounds { get { return m_bounds; } } - - private Entry GetBeginRangeIntersecting(Entry range) - { - // look for the first existing range that is intersected by the start of the new range - - Entry cursor; - int offset, level = m_items.FindPrevious(range, true, out offset, out cursor); - if (level < 0) - { - return null; - } - return cursor; - } - - private Entry GetEndRangeIntersecting(Entry range) - { - // look for the last existing range that is intersected by the end of the new range - - Entry cursor; - int offset, level = m_items.FindPrevious(range, true, out offset, out cursor); - if (level < 0) - { - return null; - } - return cursor; - } - - private TKey Min(TKey a, TKey b) - { - return m_keyComparer.Compare(a, b) <= 0 ? a : b; - } - - private TKey Max(TKey a, TKey b) - { - return m_keyComparer.Compare(a, b) >= 0 ? a : b; - } - - public void Clear() - { - m_items.Clear(); - m_bounds.Begin = default(TKey); - m_bounds.End = default(TKey); - - CheckInvariants(); - } - - /// - /// Removes everything between begin and end then translates everything - /// - /// begin key - /// end key - /// offset to apply - /// func to apply offset to a key - public void Remove(TKey begin, TKey end, TKey offset, Func applyOffset) - { - if (m_keyComparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); - - try - { - var entry = new Entry(begin, end, default(TValue)); - var iterator = m_items.GetIterator(); - var comparer = m_keyComparer; - if (!iterator.Seek(entry, true)) - { - //on ne trouve pas l'item exacte, on prends le premier. - iterator.SeekFirst(); - } - var cursor = iterator.Current; - var c1 = comparer.Compare(begin, cursor.Begin); - var c2 = comparer.Compare(end, cursor.End); - List toRemove = null; - //begin < cursor.Begin - if (c1 < 0) - { - var c3 = comparer.Compare(end, cursor.Begin); - //end <= cursor.Begin - // [++++++++++ - // ------[ - //ou - // [++++++ - // ------[ - if (c3 <= 0) - { - TranslateAfter(null, offset, applyOffset); - return; - } - //end < cursor.End - // [+++++++++++[ - //-----------[ - if (c2 < 0) - { - cursor.Begin = applyOffset(end, offset); - cursor.End = applyOffset(cursor.End, offset); - TranslateAfter(cursor, offset, applyOffset); - return; - } - //end == cursor.End - // [+++++++++[ - //---------------[ - if (c2 == 0) - { - m_items.RemoveItem(cursor); - TranslateAfter(cursor, offset, applyOffset); - return; - } - //end > cursor.End - // [+++++++++[ - //-------------------... - if (c2 > 0) - { - toRemove = new List(); - toRemove.Add(cursor); - while (iterator.Next()) - { - cursor = iterator.Current; - c2 = comparer.Compare(end, cursor.End); - c3 = comparer.Compare(end, cursor.Begin); - //end <= cursor.Begin - // [+++++ - // ----[ - //ou - // [+++++ - // ------[ - if (c3 <= 0) - { - //on set cursor pour que la translation soit faite correctement - cursor = entry; - break; - } - //end > cursor.Begin - if (c3 > 0) - { - //end < cursor.End - // [+++++++++++ - // ----------[ - if (c2 < 0) - { - cursor.Begin = begin; - cursor.End = applyOffset(cursor.End, offset); - break; - } - // end >= cursor.End - // [+++++++++[ - // ---------------[ - //ou - // [+++++++[ - // ----------------... - if (c2 >= 0) - { - toRemove.Add(cursor); - if (c2 == 0) break; - } - } - } - m_items.RemoveItems(toRemove); - TranslateAfter(cursor, offset, applyOffset); - return; - } - } - //begin == cursor.Begin - else if (c1 == 0) - { - //end < cursor.End - // [+++++++++[ - // [-----[ - if (c2 < 0) - { - cursor.Begin = begin; - cursor.End = applyOffset(cursor.End, offset); - TranslateAfter(cursor, offset, applyOffset); - return; - } - //end == cursor.End - // [++++++++[ - // [--------[ - else if (c2 == 0) - { - toRemove = new List(); - toRemove.Add(cursor); - } - // end > cursor.End - // [+++++++[ - // [-----------.... - else - { - toRemove = new List(); - toRemove.Add(cursor); - while (iterator.Next()) - { - cursor = iterator.Current; - var c3 = comparer.Compare(end, cursor.Begin); - c2 = comparer.Compare(end, cursor.End); - //end < cursor.Begin - // [++++++++[ - //---------[ - //ou - // [+++++++[ - //---------[ - if (c3 <= 0) - { - break; - } - else - { - //end < cursor.End - // [++++++++++++[ - //-----[ - if (c2 < 0) - { - cursor.Begin = begin; - cursor.End = applyOffset(cursor.End, offset); - break; - } - //end >= cursor.End - // [+++++++++[ - //---------------... - //ou - // [+++++++++[ - //-----------[ - if (c2 >= 0) - { - toRemove.Add(cursor); - if (c2 == 0) break; - } - } - } - } - m_items.RemoveItems(toRemove); - TranslateAfter(cursor, offset, applyOffset); - return; - } - //begin > cursor.Begin - else - { - //end < cursor.End - // [++++++++++++[ - // [----[ - // = [++[[++++[ - if (c2 < 0) - { - var oldEnd = cursor.End; - cursor.End = begin; - TranslateAfter(cursor, offset, applyOffset); - m_items.Insert(new Entry(begin, applyOffset(oldEnd, offset), cursor.Value)); - return; - } - //end == cursor.End - // [+++++++++++++[ - // [-------[ - if (c2 == 0) - { - cursor.End = begin; - TranslateAfter(cursor, offset, applyOffset); - return; - } - //end > cursor.End - // [+++++++++++++[ - // [------------- - else - { - cursor.End = begin; - while (iterator.Next()) - { - cursor = iterator.Current; - var c3 = comparer.Compare(end, cursor.Begin); - c2 = comparer.Compare(end, cursor.End); - //end <= cursor.Begin - // [++++++++++++[ - // --[ - //ou - // [++++++++++++[ - // -----[ - if (c3 <= 0) - { - break; - } - else - { - //end < cursor.End - // [+++++++++++++[ - // ------------[ - if (c2 < 0) - { - cursor.Begin = begin; - cursor.End = applyOffset(cursor.End, offset); - break; - } - //end >= cursor.End - // [+++++++++++[ - //--------------------... - //ou - // [+++++++++++[ - //---------------[ - else - { - if(toRemove == null) toRemove = new List(); - toRemove.Add(cursor); - if (c2 == 0) break; - } - } - } - - if (toRemove != null) m_items.RemoveItems(toRemove); - TranslateAfter(cursor, offset, applyOffset); - return; - } - } - } - finally - { - CheckInvariants(); - } - } - - private void TranslateAfter(Entry lastOk, TKey offset, Func applyKeyOffset) - { - var iterator = m_items.GetIterator(); - //null il faut tout décaller - if (lastOk == null) - { - if (!iterator.SeekFirst()) return; - } - else - { - if (!iterator.Seek(lastOk, true)) - { - //l'element passé en parametre à été supprimé - //on cherche l'élément suivant - //si tout à été supprimé on sort. - if (!iterator.SeekFirst()) return; - var c = m_keyComparer.Compare(lastOk.End, iterator.Current.Begin); - while (c > 0 && iterator.Next()) - { - c = m_keyComparer.Compare(lastOk.End, iterator.Current.Begin); - } - } - //on veut décaller les suivants de celui passé en parametre - else iterator.Next(); - } - do - { - var cursor = iterator.Current; - //dans le cas ou tout à été supprimé après le lastOK l'iterator est déjà au bout quand on arrive ici... - if (cursor == null) break; - cursor.Begin = applyKeyOffset(cursor.Begin, offset); - cursor.End = applyKeyOffset(cursor.End, offset); - } - while (iterator.Next()); - //on décalle les bounds correctement - if (iterator.SeekFirst()) m_bounds.Begin = iterator.Current.Begin; - if (iterator.SeekLast()) m_bounds.End = iterator.Current.End; - } - - public void Mark(TKey begin, TKey end, TValue value) - { - if (m_keyComparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); - - // adds a new interval to the dictionary by overwriting or splitting any previous interval - // * if there are no interval, or the interval is disjoint from all other intervals, it is inserted as-is - // * if the new interval completly overwrites one or more intervals, they will be replaced by the new interval - // * if the new interval partially overlaps with one or more intervals, they will be split into chunks, and the new interval will be inserted between them - - // Examples: - // { } + [0..1,A] => { [0..1,A] } - // { [0..1,A] } + [2..3,B] => { [0..1,A], [2..3,B] } - // { [4..5,A] } + [0..10,B] => { [0..10,B] } - // { [0..10,A] } + [4..5,B] => { [0..4,A], [4..5,B], [5..10,A] } - // { [2..4,A], [6..8,B] } + [3..7,C] => { [2..3,A], [3..7,C], [7..8,B] } - // { [1..2,A], [2..3,B], ..., [9..10,Y] } + [0..10,Z] => { [0..10,Z] } - - var entry = new Entry(begin, end, value); - Entry cursor; - var cmp = m_keyComparer; - int c1, c2; - - try - { - - switch (m_items.Count) - { - case 0: - { // the list empty - - // no checks required - m_items.Insert(entry); - m_bounds.Begin = entry.Begin; - m_bounds.End = entry.End; - break; - } - - case 1: - { // there is only one value - - cursor = m_items[0]; - - c1 = cmp.Compare(begin, cursor.End); - if (c1 >= 0) - { - // [--------) [========) - // or [--------|========) - if (c1 == 0 && m_valueComparer.Compare(cursor.Value, value) == 0) - { - cursor.End = end; - } - else - { - m_items.Insert(entry); - } - m_bounds.End = end; - return; - } - c1 = cmp.Compare(end, cursor.Begin); - if (c1 <= 0) - { - // [========) [--------) - // or [========|--------) - if (c1 == 0 && m_valueComparer.Compare(cursor.Value, value) == 0) - { - cursor.Begin = begin; - } - else - { - m_items.Insert(entry); - } - m_bounds.Begin = begin; - return; - } - - c1 = cmp.Compare(begin, cursor.Begin); - c2 = cmp.Compare(end, cursor.End); - if (c1 == 0) - { // same start - if (c2 == 0) - { // same end - // [--------) - // + [========) - // = [========) - cursor.Value = value; - } - else if (c2 < 0) - { - // [----------) - // + [======) - // = [======|---) - if (m_valueComparer.Compare(cursor.Value, value) != 0) - { - cursor.Begin = end; - m_items.Insert(entry); - } - } - else - { - // [------) - // + [==========) - // = [==========) - cursor.Set(entry); - m_bounds.End = end; - } - } - else if (c1 > 0) - { // entry is to the right - if (c2 >= 0) - { - // [------) - // + [=======) - // = [---|=======) - - cursor.End = begin; - m_items.Insert(entry); - if (c2 > 0) m_bounds.End = end; - } - else - { - // [-----------) - // + [====) - // = [---|====|--) - var tmp = new Entry(end, cursor.End, cursor.Value); - cursor.End = begin; - m_items.InsertItems(entry, tmp); - } - } - else - { // entry is to the left - if (c2 >= 0) - { - cursor.Set(entry); - m_bounds.End = end; - } - else - { - cursor.Begin = end; - m_items.Insert(entry); - } - m_bounds.Begin = begin; - } - break; - } - - default: - { - // check with the bounds first - - if (cmp.Compare(begin, m_bounds.End) > 0) - { // completely to the right - m_items.Insert(entry); - m_bounds.End = end; - break; - } - if (cmp.Compare(end, m_bounds.Begin) < 0) - { // completely to the left - m_items.Insert(entry); - m_bounds.Begin = begin; - break; - } - if (cmp.Compare(begin, m_bounds.Begin) <= 0 && cmp.Compare(end, m_bounds.End) >= 0) - { // overlaps with all the ranges - // note :if we are here, there was at least 2 items, so just clear everything - m_items.Clear(); - m_items.Insert(entry); - m_bounds.Begin = entry.Begin; - m_bounds.End = entry.End; - break; - } - - // note: we have already bound checked, so we know that there is at least one overlap ! - - bool inserted = false; - - // => we will try to find the first range and last range in the dictionary that would be impacted, mutate them and delete all ranges in between - - var iterator = m_items.GetIterator(); - // seek to the range that starts before (or at) the new range's begin point - if (!iterator.Seek(entry, true)) - { // the new range will go into first position - // => still need to check if we are overlapping with the next ranges - iterator.SeekFirst(); - //Console.WriteLine(" . new lower bound, but intersects with first range..."); - m_bounds.Begin = begin; - } - - m_bounds.End = Max(m_bounds.End, end); - - cursor = iterator.Current; - - c1 = cmp.Compare(cursor.Begin, begin); - c2 = cmp.Compare(cursor.End, end); - if (c1 >= 0) - { - if (c2 == 0) - { // same end - // [-------).. [-------).. - // + [=======) + [==========) - // = [=======).. = [==========).. - cursor.Set(entry); - return; - } - - if (c2 > 0) - { // truncate begin - // [----------).. [----------).. - // + [=======) + [=======) - // = [=======|--).. = [=======|-----).. - cursor.Begin = end; - m_items.Insert(entry); - return; - } - - // replace + propagate - // [-------)???.. [-----)????.. - // + [==========) + [============) - // = [==========).. = [============).. - - cursor.Set(entry); - //we keep the reference to cursor to be able to modify it later - entry = cursor; - inserted = true; - //TODO: need to propagate ! - } - else - { - if (c2 == 0) - { // same end - // [------------) - // [========) - // = [---|========) - - cursor.End = begin; - m_items.Insert(entry); - return; - } - - if (c2 > 0) - { - // [------------) - // [=====) - // = [---|=====|--) - - var tmp = new Entry(end, cursor.End, cursor.Value); - cursor.End = begin; - m_items.InsertItems(entry, tmp); - return; - } - - int c3 = cmp.Compare(begin, cursor.End); - if (c3 >= 0) - { - if (c3 == 0 && m_valueComparer.Compare(value, cursor.Value) == 0) - { // touching same value => merge - cursor.End = end; - entry = cursor; - inserted = true; - } - else - { - // [---) - // [=====???? - // = [---) [=====???? - } - } - else - { - // [--------???? - // [====???? - // = [---|====???? - cursor.End = begin; - } - } - - // if we end up here, it means that we may be overlapping with following items - // => we need to delete them until we reach the last one, which we need to either delete or mutate - // => also, if we haven't inserted the entry yet, we will reuse the first deleted range to insert the entry, and only insert at the end if we haven't found a spot - - List deleted = null; - - while (true) - { - if (!iterator.Next()) - { // we reached past the end of the db - break; - } - - // cursor: existing range that we need to either delete or mutate - cursor = iterator.Current; - - c1 = cmp.Compare(cursor.Begin, end); - if (c1 == 0) - { // touching the next range - if (m_valueComparer.Compare(value, cursor.Value) == 0) - { // contiguous block with same value => merge - // [===========) - // [=====) - // = [=================) - if (inserted) - { - if (cmp.Compare(cursor.End, entry.End) > 0) - { - entry.End = cursor.End; - } - //note: we can't really delete while iterating with a cursor, so just mark it for deletion - if (deleted == null) deleted = new List(); - deleted.Add(cursor); - } - else - { - cursor.Begin = begin; - entry = cursor; - inserted = true; - } - break; - } - else - { - // [-----------) - // [=====) - // = [=====|-----------) - } - break; - } - else if (c1 > 0) - { // we are past the inserted range, nothing to do any more - // [------------) - // [=====) - // = [=====) [------------) - //Console.WriteLine(" . no overlap => break"); - break; - } - - c1 = cmp.Compare(cursor.End, end); - if (c1 <= 0) - { // we are completely covered => delete - - // [-------) [-------) - // + [...=======) + [...=======...) - // = [...=======) = [...=======...) - if (!inserted) - { // use that slot to insert ourselves - cursor.Set(entry); - //get the reference to be able to eventually merge it afterwards - entry = cursor; - inserted = true; - } - else - { - //note: we can't really delete while iterating with a cursor, so just mark it for deletion - if (deleted == null) deleted = new List(); - deleted.Add(cursor); - - } - } - else - { // we are only partially overlapped - - // [------------) - // [....========) - // = [....========|---) - - cursor.Begin = end; - break; - } - } - - if (deleted != null && deleted.Count > 0) - { - m_items.RemoveItems(deleted); - } - - if (!inserted) - { // we did not find an existing spot to re-use, so we need to insert the new range - m_items.Insert(entry); - } - break; - } - } - } - finally - { - CheckInvariants(); - } - } - - /// Checks if there is at least one range in the dictionary that intersects with the specified range, and matches the predicate - /// Lower bound of the intersection - /// Higher bound (excluded) of the intersection - /// Value that is passed as the second argument to - /// Predicate called for each intersected range. - /// True if there was at least one intersecting range, and returned true for that range. - public bool Intersect(TKey begin, TKey end, TValue arg, Func predicate) - { - if (m_items.Count == 0) return false; - - var cmp = m_keyComparer; - if (cmp.Compare(m_bounds.Begin, end) >= 0) return false; - if (cmp.Compare(m_bounds.End, begin) <= 0) return false; - - var entry = new Entry(begin, end, default(TValue)); - - var iterator = m_items.GetIterator(); - if (!iterator.Seek(entry, true)) - { // starts before - iterator.SeekFirst(); - } - - do - { - var cursor = iterator.Current; - - // A and B intersects if: CMP(B.end, A.begin) <= 0 .OR. CMP(A.end, B.begin) <= 0 - - if (cmp.Compare(end, cursor.Begin) <= 0) - { - return false; - } - - if (cmp.Compare(cursor.End, begin) > 0 && predicate(cursor.Value, arg)) - { - return true; - } - } - while(iterator.Next()); - - return false; - } - - public ColaStore.Enumerator GetEnumerator() - { - return new ColaStore.Enumerator(m_items, reverse: false); - } - - IEnumerator IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - [Conditional("DEBUG")] - //TODO: remove or set to internal ! - public void Debug_Dump() - { -#if DEBUG - Debug.WriteLine("Dumping ColaRangeDictionary<" + typeof(TKey).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); - m_items.Debug_Dump(); -#endif - } - - public override string ToString() - { - if (m_items.Count == 0) return "{ }"; - - var sb = new System.Text.StringBuilder(); - Entry previous = null; - foreach(var item in this) - { - if (previous == null) - { - sb.Append('['); - } - else if (m_keyComparer.Compare(previous.End, item.Begin) < 0) - { - sb.Append(previous.End).Append(") ["); - } - else - { - sb.Append(previous.End).Append('|'); - } - - sb.Append(item.Begin).Append("..(").Append(item.Value).Append(").."); - previous = item; - } - if (previous != null) - { - sb.Append(previous.End).Append(")"); - } - - return sb.ToString(); - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs b/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs deleted file mode 100644 index b1109b041..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaRangeSet.cs +++ /dev/null @@ -1,345 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -// enables consitency checks after each operation to the set -#define ENFORCE_INVARIANTS - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Globalization; - - /// Represent an ordered list of ranges, stored in a Cache Oblivous Lookup Array - /// Type of keys stored in the set - [DebuggerDisplay("Count={m_items.Count}, Bounds={m_bounds.Begin}..{m_bounds.End}")] - public sealed class ColaRangeSet : IEnumerable.Entry> - { - // We store the ranges in a COLA array that is sorted by the Begin keys - // The range are mutable, which allows for efficient merging - - // INVARIANTS - // * If there is at least on range, the set is not empty - // * The Begin key is INCLUDED in range, but the End key is EXCLUDED from the range (ie: Begin <= K < End) - // * The End key of a range is always GREATER than or EQUAL to the Begin key of a range (ie: ranges are not backwards) - // * The End key of a range is always strictly LESS than the Begin key of the next range (ie: there are gaps between ranges) - - // This should give us a sorted set of disjoint ranges - - /// Mutable range - public sealed class Entry - { - public TKey Begin { get; internal set; } - public TKey End { get; internal set; } - - public Entry(TKey begin, TKey end) - { - this.Begin = begin; - this.End = end; - } - - internal void ReplaceWith(Entry other) - { - this.Begin = other.Begin; - this.End = other.End; - } - - internal void Update(TKey begin, TKey end) - { - this.Begin = begin; - this.End = end; - } - - internal bool Contains(TKey key, IComparer comparer) - { - return comparer.Compare(key, this.Begin) >= 0 && comparer.Compare(key, this.End) < 0; - } - - public override string ToString() - { - return String.Format(CultureInfo.InvariantCulture, "[{0}, {1})", this.Begin, this.End); - } - } - - /// Range comparer that only test the Begin key - private sealed class BeginKeyComparer : IComparer - { - private readonly IComparer m_comparer; - - public BeginKeyComparer(IComparer comparer) - { - m_comparer = comparer; - } - - public int Compare(Entry x, Entry y) - { - return m_comparer.Compare(x.Begin, y.Begin); - } - - } - - private readonly ColaStore m_items; - private readonly IComparer m_comparer; - private readonly Entry m_bounds; - - public ColaRangeSet() - : this(0, null) - { } - - public ColaRangeSet(int capacity) - : this(capacity, null) - { } - - public ColaRangeSet(IComparer keyComparer) - : this(0, keyComparer) - { } - - public ColaRangeSet(int capacity, IComparer keyComparer) - { - m_comparer = keyComparer ?? Comparer.Default; - if (capacity == 0) capacity = 15; - m_items = new ColaStore(capacity, new BeginKeyComparer(m_comparer)); - m_bounds = new Entry(default(TKey), default(TKey)); - } - - [Conditional("ENFORCE_INVARIANTS")] - private void CheckInvariants() - { - } - - public int Count { get { return m_items.Count; } } - - public int Capacity { get { return m_items.Capacity; } } - - public IComparer Comparer { get { return m_comparer; } } - - public Entry Bounds { get { return m_bounds; } } - - private bool Resolve(Entry previous, Entry candidate) - { - - int c = m_comparer.Compare(previous.Begin, candidate.Begin); - if (c == 0) - { // they share the same begin key ! - - if (m_comparer.Compare(previous.End, candidate.End) < 0) - { // candidate replaces the previous ony - previous.ReplaceWith(candidate); - } - return true; - } - - if (c < 0) - { // b is to the right - if (m_comparer.Compare(previous.End, candidate.Begin) < 0) - { // there is a gap in between - return false; - } - // they touch or overlap - previous.Update(previous.Begin, Max(previous.End, candidate.End)); - return true; - } - else - { // b is to the left - if (m_comparer.Compare(candidate.End, previous.Begin) < 0) - { // there is a gap in between - return false; - } - // they touch or overlap - previous.Update(candidate.Begin, Max(previous.End, candidate.End)); - return true; - } - } - - private TKey Min(TKey a, TKey b) - { - return m_comparer.Compare(a, b) <= 0 ? a : b; - } - - private TKey Max(TKey a, TKey b) - { - return m_comparer.Compare(a, b) >= 0 ? a : b; - } - - public void Clear() - { - m_items.Clear(); - m_bounds.Update(default(TKey), default(TKey)); - } - - public void Mark(TKey begin, TKey end) - { - if (m_comparer.Compare(begin, end) >= 0) throw new InvalidOperationException("End key must be greater than the Begin key."); - - var entry = new Entry(begin, end); - Entry cursor; - - switch (m_items.Count) - { - case 0: - { // the list empty - - // no checks required - m_items.Insert(entry); - m_bounds.ReplaceWith(entry); - break; - } - - case 1: - { // there is only one value - - cursor = m_items[0]; - if (!Resolve(cursor, entry)) - { // no conflict - m_items.Insert(entry); - m_bounds.Update( - Min(entry.Begin, cursor.Begin), - Max(entry.End, cursor.End) - ); - } - else - { // merged with the previous range - m_bounds.ReplaceWith(cursor); - } - break; - } - default: - { - // check with the bounds first - - if (m_comparer.Compare(begin, m_bounds.End) > 0) - { // completely to the right - m_items.Insert(entry); - m_bounds.Update(m_bounds.Begin, end); - break; - } - if (m_comparer.Compare(end, m_bounds.Begin) < 0) - { // completely to the left - m_items.Insert(entry); - m_bounds.Update(begin, m_bounds.End); - break; - } - if (m_comparer.Compare(begin, m_bounds.Begin) <= 0 && m_comparer.Compare(end, m_bounds.End) >= 0) - { // overlaps with all the ranges - // note :if we are here, there was at least 2 items, so just clear everything - m_items.Clear(); - m_items.Insert(entry); - m_bounds.ReplaceWith(entry); - break; - } - - - // overlaps with existing ranges, we may need to resolve conflicts - int offset, level; - bool inserted = false; - - // once inserted, will it conflict with the previous entry ? - if ((level = m_items.FindPrevious(entry, true, out offset, out cursor)) >= 0) - { - if (Resolve(cursor, entry)) - { - entry = cursor; - inserted = true; - } - } - - // also check for potential conflicts with the next entries - while (true) - { - level = m_items.FindNext(entry, false, out offset, out cursor); - if (level < 0) break; - - if (inserted) - { // we already have inserted the key so conflicts will remove the next segment - if (Resolve(entry, cursor)) - { // next segment has been removed - //Console.WriteLine(" > folded with previous: " + entry); - m_items.RemoveAt(level, offset); - } - else - { - break; - } - } - else - { // we havent inserted the key yet, so in case of conflict, we will use the next segment's slot - if (Resolve(cursor, entry)) - { - //Console.WriteLine(" > merged in place: " + cursor); - inserted = true; - } - else - { - break; - } - } - } - - if (!inserted) - { // no conflict, we have to insert the new range - m_items.Insert(entry); - } - - m_bounds.Update( - Min(m_bounds.Begin, entry.Begin), - Max(m_bounds.End, entry.End) - ); - - break; - } - } - - //TODO: check constraints ! - } - - /// Checks if there is at least one range that contains the specified key - /// Key to test - /// True if the key is contained by one range; otherwise, false. - public bool ContainsKey(TKey key) - { - if (m_bounds.Contains(key, m_comparer)) - { - var entry = new Entry(key, key); - int offset, level = m_items.FindPrevious(entry, true, out offset, out entry); - return level >= 0 && entry.Contains(key, m_comparer); - } - return false; - } - - public ColaStore.Enumerator GetEnumerator() - { - return new ColaStore.Enumerator(m_items, reverse: false); - } - - IEnumerator IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } - - [Conditional("DEBUG")] - //TODO: remove or set to internal ! - public void Debug_Dump() - { -#if DEBUG - Console.WriteLine("Dumping ColaRangeSet<" + typeof(TKey).Name + "> filled at " + (100.0d * this.Count / this.Capacity).ToString("N2") + "%"); - m_items.Debug_Dump(); -#endif - } - - public override string ToString() - { - if (m_items.Count == 0) return "{ }"; - return "{ " + String.Join(", ", this) + " }"; - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Collections/ColaStore.cs b/FoundationDB.Storage.Memory/Collections/ColaStore.cs deleted file mode 100644 index 7f37f5090..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaStore.cs +++ /dev/null @@ -1,1197 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - public static class ColaStore - { - private const int NOT_FOUND = -1; - - private static readonly int[] MultiplyDeBruijnLowestBitPosition = new int[32] - { - 0, 1, 28, 2, 29, 14, 24, 3, 30, 22, 20, 15, 25, 17, 4, 8, - 31, 27, 13, 23, 21, 19, 16, 7, 26, 12, 18, 6, 11, 5, 10, 9 - }; - - private static readonly int[] MultiplyDeBruijnHighestBitPosition = new int[32] - { - 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, - 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 - }; - - internal static bool IsFree(int level, int count) - { - Contract.Requires(level >= 0 && count >= 0); - return (count & (1 << level)) == 0; - } - - internal static bool IsAllocated(int level, int count) - { - Contract.Requires(level >= 0 && count >= 0); - return (count & (1 << level)) != 0; - } - - /// Finds the level that holds an absolute index - /// Absolute index in a COLA array where 0 is the root, 1 is the first item of level 1, and so on - /// Receive the offset in the level that contains is located - /// Level that contains the specified location. - public static int FromIndex(int index, out int offset) - { - Contract.Requires(index >= 0); - - int level = HighestBit(index); - offset = index - (1 << level) + 1; - Contract.Ensures(level >= 0 && level < 31 && offset >= 0 && offset < (1 << level)); - return level; - } - - /// Convert a (level, offset) pair into the corresponding absolute index - /// Level of the location (0 for the root) - /// Offset within the level of the location - /// Absolute index where 0 is the root, 1 is the first item of level 1, and so on - public static int ToIndex(int level, int offset) - { - Contract.Requires(level >= 0 && level < 31 && offset >= 0 && offset < (1 << level)); - int index = (1 << level) - 1 + offset; - Contract.Ensures(index >= 0 && index < 1 << level); - return index; - } - - public static int LowestBit(int value) - { - uint v = (uint)value; - v = (uint)((v & -v) * 0x077CB531U); - - return MultiplyDeBruijnLowestBitPosition[v >> 27]; - } - - public static int HighestBit(int value) - { - // first round down to one less than a power of 2 - uint v = (uint)value; - v |= v >> 1; - v |= v >> 2; - v |= v >> 4; - v |= v >> 8; - v |= v >> 16; - - return MultiplyDeBruijnHighestBitPosition[(int)((v * 0x07C4ACDDU) >> 27)]; - } - - /// Computes the absolute index from a value offset (in the allocated levels) - /// Number of items in the COLA array - /// Offset of the value in the allocated levels of the COLA array, with 0 being the oldest (first item of the last allocated level) - /// Absolute index of the location where that value would be stored in the COLA array (from the top) - public static int MapOffsetToIndex(int count, int arrayIndex) - { - Contract.Requires(count >= 0 && arrayIndex >= 0 && arrayIndex < count); - - int offset; - int level = MapOffsetToLocation(count, arrayIndex, out offset); - return (1 << level) - 1 + offset; - } - - /// Computes the (level, offset) pair from a value offset (in the allocated levels) - /// Number of items in the COLA array - /// Offset of the value in the allocated levels of the COLA array, with 0 being the oldest (first item of the last allocated level) - /// Absolute index of the location where that value would be stored in the COLA array (from the top) - public static int MapOffsetToLocation(int count, int arrayIndex, out int offset) - { - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - if (arrayIndex < 0 || arrayIndex >= count) throw new ArgumentOutOfRangeException("arrayIndex", "Index is outside the array"); - - if (count == 0) - { // special case for the empty array - offset = 0; - return 0; - } - - // find the highest allocated level (note: 50% of values will be in this segment!) - int level = HighestBit(count); - int k = 1 << level; - int p = k - 1; - do - { - if ((count & k) != 0) - { // this level is allocated - if (arrayIndex < k) - { - offset = arrayIndex; - return level; - } - arrayIndex -= k; - } - k >>= 1; - --level; - p -= k; - } - while (k > 0); - - // should not happen ! - throw new InvalidOperationException(); - } - - public static int MapLocationToOffset(int count, int level, int offset) - { - Contract.Assert(count >= 0 && level >= 0 && offset >= 0 && offset < 1 << level); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - - if (count == 0) - { // special case for the empty array - return 0; - } - - // compute the base location of the selected level - int p = 0; - int k = 1; - for (int i = 0; i < level; i++) - { - if ((count & k) != 0) - { - p += k; - } - k <<= 1; - } - - return p + offset; - } - - internal static void ThrowDuplicateKey(T value) - { - throw new InvalidOperationException(String.Format("Cannot insert '{0}' because the key already exists in the set", value)); - } - - internal static int BinarySearch(T[] array, int offset, int count, T value, IComparer comparer) - { - Contract.Assert(array != null && offset >= 0 && count >= 0 && comparer != null); - - // Instead of starting from the midle we will exploit the fact that, since items are usually inserted in order, the value is probably either to the left or the right of the segment. - // Also, since most activity happens in the top levels, the search array is probably very small (size 1, 2 or 4) - - if (count == 0) - { - // note: there should be no array of size 0, this is probably a bug ! - return ~offset; - } - - int end = offset - 1 + count; - int c; - - // compare with the last item - c = comparer.Compare(array[end], value); - if (c == 0) return end; - if (count == 1) - { - return c < 0 ? ~(offset + 1) : ~offset; - } - if (c < 0) return ~(end + 1); - --end; - - // compare with the first - c = comparer.Compare(array[offset], value); - if (c == 0) return offset; - if (c > 0) return ~offset; - - int cursor = offset + 1; - while (cursor <= end) - { - int center = cursor + ((end - cursor) >> 1); - c = comparer.Compare(array[center], value); - if (c == 0) - { // the value is the center point - return center; - } - if (c < 0) - { // the value is after the center point - cursor = center + 1; - } - else - { // the value is before the center point - end = center - 1; - } - } - return ~cursor; - } - - /// Merge two values into level 1 - /// Segment for level 1 (should be of size 2) - /// Left value - /// Right value - /// Comparer to use - internal static void MergeSimple(T[] segment, T left, T right, IComparer comparer) - { - Contract.Requires(segment != null && segment.Length == 2); - - int c = comparer.Compare(left, right); - if (c == 0) ThrowDuplicateKey(right); - else if (c < 0) - { - segment[0] = left; - segment[1] = right; - } - else - { - segment[0] = right; - segment[1] = left; - } - } - - /// Replace a value in a segment with another value, while keeping it sorted - /// Segment that will received the new value - /// Offset of replaced value in the segment - /// New value to insert into the segment - /// Comparer to use - internal static void MergeInPlace(T[] segment, int offset, T value, IComparer comparer) - { - Contract.Requires(segment != null && offset >= 0 && comparer != null); - - // Find the spot where the new value should be inserted - int p = BinarySearch(segment, 0, segment.Length, value, comparer); - if (p >= 0) - { // this is not supposed to happen! - ThrowDuplicateKey(value); - } - - int index = (~p); - Contract.Assert(index >= 0 && index <= segment.Length); - if (index == offset) - { // merge in place - - // _______ offset == index - // V - // before: [...] X [...] - // after: [...] O [...] - - segment[index] = value; - return; - } - if (index < offset) - { // shift right - - // ____________ index - // / _______ offset - // V V - // before: [...] # # # X [...] - // after: [...] O # # # [...] - - Array.Copy(segment, index, segment, index + 1, offset - index); - segment[index] = value; - } - else - { // shift left - - --index; - - // ____________ offset - // / _______ index - // V V - // before: [...] X # # # [...] - // after: [...] # # # O [...] - - Array.Copy(segment, offset + 1, segment, offset, index - offset); - segment[index] = value; - } - } - - /// Spread the content of a level to all the previous levels into pieces, except the first item that is returned - /// Level that should be broken into chunks - /// List of all the levels - /// The last element of the broken level - /// The broken segment will be cleared - internal static T SpreadLevel(int level, T[][] inputs) - { - Contract.Requires(level >= 0 && inputs != null && inputs.Length > level); - - // Spread all items in the target level - except the first - to the previous level (which should ALL be EMPTY) - - var source = inputs[level]; - - int p = 1; - for (int i = level - 1; i >= 0; i--) - { - var segment = inputs[i]; - Contract.Assert(segment != null); - int n = segment.Length; - Array.Copy(source, p, segment, 0, n); - p += n; - } - Contract.Assert(p == source.Length); - T res = source[0]; - Array.Clear(source, 0, source.Length); - return res; - } - - /// Merge two ordered segments of level N into an ordered segment of level N + 1 - /// Destination, level N + 1 (size 2^(N+1)) - /// First level N segment (size 2^N) - /// Second level N segment (taille 2^N) - /// Comparer used for the merge - internal static void MergeSort(T[] output, T[] left, T[] right, IComparer comparer) - { - Contract.Requires(output != null && left != null && right != null && comparer != null); - Contract.Requires(left.Length > 0 && output.Length == left.Length * 2 && right.Length == left.Length); - - int c, n = left.Length; - // note: The probality to merge an array of size N is rougly 1/N (with N being a power of 2), - // which means that we will spend roughly half the time merging arrays of size 1 into an array of size 2.. - - if (n == 1) - { // Most frequent case (p=0.5) - var l = left[0]; - var r = right[0]; - if ((c = comparer.Compare(l, r)) < 0) - { - output[0] = l; - output[1] = r; - } - else - { - Contract.Assert(c != 0); - output[0] = r; - output[1] = l; - } - return; - } - - if (n == 2) - { // second most frequent case (p=0.25) - - // We are merging 2 pairs of ordered values into an array of size 4 - if (comparer.Compare(left[1], right[0]) < 0) - { // left << right - output[0] = left[0]; - output[1] = left[1]; - output[2] = right[0]; - output[3] = right[1]; - return; - } - - if (comparer.Compare(right[1], left[0]) < 0) - { // right << left - output[0] = right[0]; - output[1] = right[1]; - output[2] = left[0]; - output[3] = left[1]; - return; - } - - // left and right intersects - // => just use the regular merge sort below - } - - int pLeft = 0; - int pRight = 0; - int pOutput = 0; - - while (true) - { - if ((c = comparer.Compare(left[pLeft], right[pRight])) < 0) - { // left is smaller than right => advance - - output[pOutput++] = left[pLeft++]; - - if (pLeft >= n) - { // the left array is done, copy the remainder of the right array - if (pRight < n) Array.Copy(right, pRight, output, pOutput, n - pRight); - return; - } - } - else - { // right is smaller or equal => advance - Contract.Assert(c != 0); - - output[pOutput++] = right[pRight++]; - - if (pRight >= n) - { // the right array is done, copy the remainder of the left array - if (pLeft < n) Array.Copy(left, pLeft, output, pOutput, n - pLeft); - return; - } - } - } - - } - - internal static int[] CreateCursors(int count, out int min) - { - min = LowestBit(count); - var cursors = new int[HighestBit(count) + 1]; - int k = 1; - for (int i = 0; i < cursors.Length; i++) - { - if (i < min || (count & k) == 0) cursors[i] = NOT_FOUND; - k <<= 1; - } - return cursors; - } - - /// Search for the smallest element that is larger than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the next element, or 0 if not found - /// Receive the value of the next element, or default(T) if not found - /// Level of the next element, or -1 if was already the largest - public static int FindNext(T[][] levels, int count, T value, bool orEqual, IComparer comparer, out int offset, out T result) - { - int level = NOT_FOUND; - T min = default(T); - int minOffset = 0; - - // scan each segment for a value that would be larger, keep track of the smallest found - for (int i = 0; i < levels.Length; i++) - { - if (ColaStore.IsFree(i, count)) continue; - - var segment = levels[i]; - int pos = ColaStore.BinarySearch(segment, 0, segment.Length, value, comparer); - if (pos >= 0) - { // we found an exact match in this segment - if (orEqual) - { - offset = pos; - result = segment[pos]; - return i; - } - - // the next item in this segment should be larger - ++pos; - } - else - { // we found where it would be stored in this segment - pos = ~pos; - } - - if (pos < segment.Length) - { - if (level == NOT_FOUND || comparer.Compare(segment[pos], min) < 0) - { // we found a better candidate - min = segment[pos]; - level = i; - minOffset = pos; - } - } - } - - offset = minOffset; - result = min; - return level; - } - - /// Search for the largest element that is smaller than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the previous element, or 0 if not found - /// Receive the value of the previous element, or default(T) if not found - /// Level of the previous element, or -1 if was already the smallest - public static int FindPrevious(T[][] levels, int count, T value, bool orEqual, IComparer comparer, out int offset, out T result) - { - int level = NOT_FOUND; - T max = default(T); - int maxOffset = 0; - - // scan each segment for a value that would be smaller, keep track of the smallest found - for (int i = 0; i < levels.Length; i++) - { - if (ColaStore.IsFree(i, count)) continue; - - var segment = levels[i]; - int pos = ColaStore.BinarySearch(segment, 0, segment.Length, value, comparer); - // the previous item in this segment should be smaller - if (pos < 0) - { // it is not - pos = ~pos; - } - else if (orEqual) - { // we found an exact match in this segment - offset = pos; - result = segment[pos]; - return i; - } - - --pos; - - if (pos >= 0) - { - if (level == NOT_FOUND || comparer.Compare(segment[pos], max) > 0) - { // we found a better candidate - max = segment[pos]; - level = i; - maxOffset = pos; - } - } - } - - offset = maxOffset; - result = max; - return level; - } - - public static IEnumerable FindBetween(T[][] levels, int count, T begin, bool beginOrEqual, T end, bool endOrEqual, int limit, IComparer comparer) - { - if (limit > 0) - { - for (int i = 0; i < levels.Length; i++) - { - if (ColaStore.IsFree(i, count)) continue; - - var segment = levels[i]; - - int to = ColaStore.BinarySearch(segment, 0, segment.Length, end, comparer); - if (to >= 0) - { - if (!endOrEqual) - { - to--; - } - } - else - { - to = ~to; - } - if (to < 0 || to >= segment.Length) continue; - - int from = ColaStore.BinarySearch(segment, 0, segment.Length, begin, comparer); - if (from >= 0) - { - if (!beginOrEqual) - { - ++from; - } - } - else - { - from = ~from; - } - if (from >= segment.Length) continue; - - if (from > to) continue; - - for (int j = from; j <= to && limit > 0; j++) - { - yield return segment[j]; - --limit; - } - if (limit <= 0) break; - } - } - } - - /// Find the next smallest key pointed by a list of cursors - /// List of source arrays - /// Lit of cursors in source arrays - /// Key comparer - /// Received the next smallest element if the method returns true; otherwise set to default(T) - /// The index of the level that returned the value, or -1 if all levels are done - internal static int IterateFindNext(T[][] inputs, int[] cursors, int min, int max, IComparer comparer, out T result) - { - Contract.Requires(inputs != null && cursors != null && min >= 0 && max >= min && comparer != null); - - int index = NOT_FOUND; - int pos = NOT_FOUND; - var next = default(T); - - // look for the smallest element - // note: we scan from the bottom up, because older items are usually in the lower levels - for (int i = max; i >= min; i--) - { - int cursor = cursors[i]; - if (cursor < 0) continue; - var segment = inputs[i]; - if (cursor >= segment.Length) continue; - var x = segment[cursor]; - if (index == NOT_FOUND || comparer.Compare(x, next) < 0) - { // found a candidate - index = i; - pos = cursor; - next = x; - } - } - - if (index != NOT_FOUND) - { - ++pos; - if (pos >= (1 << index)) - { // this array is done - pos = NOT_FOUND; - } - cursors[index] = pos; - result = next; - return index; - } - - result = default(T); - return NOT_FOUND; - } - - /// Find the next largest key pointed by a list of cursors - /// List of source arrays - /// Lit of cursors in source arrays - /// Key comparer - /// Received the next largest element if the method returns true; otherwise set to default(T) - /// The index of the level that returned the value, or -1 if all levels are done - internal static int IterateFindPrevious(T[][] inputs, int[] cursors, int min, int max, IComparer comparer, out T result) - { - Contract.Requires(inputs != null && cursors != null && min >= 0 && max >= min && comparer != null); - // NOT TESTED !!!!! - // NOT TESTED !!!!! - // NOT TESTED !!!!! - - //Trace.WriteLine("IterateFindPrevious(" + min + ".." + max + ")"); - - int index = NOT_FOUND; - int pos = NOT_FOUND; - var next = default(T); - - // look for the largest element - // note: we scan from the top down, because more recent items are usually in the upper levels - for (int i = min; i >= max; i--) - { - int cursor = cursors[i]; - if (cursor < 0) continue; - var segment = inputs[i]; - if (cursor >= segment.Length) continue; - var x = segment[cursor]; - if (index == NOT_FOUND || comparer.Compare(x, next) < 0) - { // found a candidate - index = i; - pos = cursor; - next = x; - } - } - - if (index != NOT_FOUND) - { - --pos; - if (pos < 0) - { // this array is done - pos = NOT_FOUND; - } - cursors[index] = pos; - result = next; - return index; - } - - result = default(T); - return NOT_FOUND; - } - - /// Iterate over all the values in the set, using their natural order - internal static IEnumerable IterateOrdered(int count, T[][] inputs, IComparer comparer, bool reverse) - { - Contract.Requires(count >= 0 && inputs != null && comparer != null && count < (1 << inputs.Length)); - // NOT TESTED !!!!! - // NOT TESTED !!!!! - // NOT TESTED !!!!! - - Contract.Requires(count >= 0 && inputs != null && comparer != null); - - // We will use a list of N cursors, set to the start of their respective levels. - // A each turn, look for the smallest key referenced by the cursors, return that one, and advance its cursor. - // Once a cursor is past the end of its level, it is set to -1 and is ignored for the rest of the operation - - if (count > 0) - { - // setup the cursors, with the empty levels already marked as completed - var cursors = new int[inputs.Length]; - for (int i = 0; i < cursors.Length; i++) - { - if (ColaStore.IsFree(i, count)) - { - cursors[i] = NOT_FOUND; - } - } - - // pre compute the first/last active level - int min = ColaStore.LowestBit(count); - int max = ColaStore.HighestBit(count); - - while (count-- > 0) - { - T item; - int pos; - if (reverse) - { - pos = IterateFindPrevious(inputs, cursors, min, max, comparer, out item); - } - else - { - pos = IterateFindNext(inputs, cursors, min, max, comparer, out item); - } - - if (pos == NOT_FOUND) - { // we unexpectedly ran out of stuff before the end ? - //TODO: should we fail or stop here ? - throw new InvalidOperationException("Not enough data in the source arrays to fill the output array"); - } - yield return item; - - // update the bounds if needed - if (pos == max) - { - if (cursors[max] == NOT_FOUND) --max; - } - else if (pos == min) - { - if (cursors[min] == NOT_FOUND) ++min; - } - } - } - } - - /// Iterate over all the values in the set, without any order guarantee - internal static IEnumerable IterateUnordered(int count, T[][] inputs) - { - Contract.Requires(count >= 0 && inputs != null && count < (1 << inputs.Length)); - - for (int i = 0; i < inputs.Length; i++) - { - if (ColaStore.IsFree(i, count)) continue; - var segment = inputs[i]; - Contract.Assert(segment != null && segment.Length == 1 << i); - for (int j = 0; j < segment.Length; j++) - { - yield return segment[j]; - } - } - } - - internal static void ThrowStoreVersionChanged() - { - throw new InvalidOperationException("The version of the store has changed. This usually means that the collection has been modified while it was being enumerated"); - } - - [StructLayout(LayoutKind.Sequential)] - public struct Enumerator : IEnumerator, IDisposable - { - private readonly ColaStore m_items; - private readonly bool m_reverse; - private int[] m_cursors; - private T m_current; - private int m_min; - private int m_max; - - internal Enumerator(ColaStore items, bool reverse) - { - m_items = items; - m_reverse = reverse; - m_cursors = ColaStore.CreateCursors(m_items.Count, out m_min); - m_max = m_cursors.Length - 1; - m_current = default(T); - } - - public bool MoveNext() - { - int pos; - if (m_reverse) - { - pos = ColaStore.IterateFindPrevious(m_items.Levels, m_cursors, m_min, m_max, m_items.Comparer, out m_current); - } - else - { - pos = ColaStore.IterateFindNext(m_items.Levels, m_cursors, m_min, m_max, m_items.Comparer, out m_current); - } - - if (pos == NOT_FOUND) - { // that was the last item! - return false; - } - - // update the bounds if necessary - if (pos == m_max) - { - if (m_cursors[m_max] == NOT_FOUND) --m_max; - } - else if (pos == m_min) - { - if (m_cursors[m_min] == NOT_FOUND) ++m_min; - } - - return true; - } - - public T Current - { - get { return m_current; } - } - - public bool Reverse - { - get { return m_reverse; } - } - - public void Dispose() - { - // we are a struct that can be copied by value, so there is no guarantee that Dispose() will accomplish anything anyway... - } - - object System.Collections.IEnumerator.Current - { - get { return m_current; } - } - - void System.Collections.IEnumerator.Reset() - { - m_cursors = ColaStore.CreateCursors(m_items.Count, out m_min); - m_max = m_cursors.Length - 1; - m_current = default(T); - } - - } - - public sealed class Iterator - { - private const int DIRECTION_PREVIOUS = -1; - private const int DIRECTION_SEEK = 0; - private const int DIRECTION_NEXT = +1; - - private readonly T[][] m_levels; - private readonly int m_count; - private readonly IComparer m_comparer; - private readonly int[] m_cursors; - private readonly int m_min; - private T m_current; - private int m_currentLevel; - private int m_direction; - - internal Iterator(T[][] levels, int count, IComparer comparer) - { - Contract.Requires(levels != null && count >= 0 && comparer != null); - m_levels = levels; - m_count = count; - m_comparer = comparer; - - m_cursors = ColaStore.CreateCursors(m_count, out m_min); - } - - [Conditional("FULL_DEBUG")] - private void Debug_Dump(string label = null) - { -#if FULL_DEBUG - Trace.WriteLine("* Cursor State: " + label); - for (int i = m_min; i < m_cursors.Length; i++) - { - if (ColaStore.IsFree(i, m_count)) - { - Trace.WriteLine(" - L" + i + ": unallocated"); - continue; - } - - int p = m_cursors[i]; - Trace.WriteLine(" - L" + i + ": " + p + " [" + (1 << i) + "] = " + (p < 0 ? "" : (p >= (1 << i)) ? "" : ("" + m_levels[i][p]))); - } - Trace.WriteLine(" > Current at " + m_currentLevel + " : " + m_current); -#endif - } - - /// Set the cursor just before the first key in the store - public void SeekBeforeFirst() - { - var cursors = m_cursors; - for (int i = m_min; i < cursors.Length; i++) - { - cursors[i] = -1; - } - m_currentLevel = NOT_FOUND; - m_current = default(T); - m_direction = DIRECTION_SEEK; - } - - /// Set the cursor just before the first key in the store - public void SeekAfterLast() - { - var cursors = m_cursors; - for (int i = m_min; i < cursors.Length; i++) - { - cursors[i] = 1 << i; - } - m_currentLevel = NOT_FOUND; - m_current = default(T); - m_direction = DIRECTION_SEEK; - } - - /// Seek the cursor to the smallest key in the store - public bool SeekFirst() - { - T min = default(T); - int minLevel = NOT_FOUND; - - var cursors = m_cursors; - - for (int i = m_min; i < cursors.Length; i++) - { - if (IsFree(i, m_count)) continue; - - cursors[i] = 0; - var segment = m_levels[i]; - Contract.Assert(segment != null && segment.Length == 1 << i); - if (minLevel < 0 || m_comparer.Compare(segment[0], min) < 0) - { - min = segment[0]; - minLevel = i; - } - } - - m_current = min; - m_currentLevel = minLevel; - m_direction = DIRECTION_SEEK; - - Debug_Dump("SeekFirst"); - - return minLevel >= 0; - } - - /// Seek the cursor to the largest key in the store - public bool SeekLast() - { - T max = default(T); - int maxLevel = NOT_FOUND; - - var cursors = m_cursors; - - for (int i = m_min; i < cursors.Length; i++) - { - if (IsFree(i, m_count)) continue; - var segment = m_levels[i]; - Contract.Assert(segment != null && segment.Length == 1 << i); - int pos = segment.Length - 1; - cursors[i] = pos; - if (maxLevel < 0 || m_comparer.Compare(segment[pos], max) > 0) - { - max = segment[segment.Length - 1]; - maxLevel = i; - } - } - - m_current = max; - m_currentLevel = maxLevel; - m_direction = DIRECTION_SEEK; - - Debug_Dump("SeekLast"); - - return maxLevel >= 0; - } - - - - /// Seek the iterator at the smallest value that is closest to the desired item - /// Item to seek to - /// If true, then seek to this item is found. If false, seek to the previous value - /// If true, the cursors are setup for moving backward (by calling Previous). Is false, the cursors are set up for moving forward (by calling Next) - public bool Seek(T item, bool orEqual) - { - // Goal: we want to find the item key itself (if it exists and orEqual==true), or the max key that is stricly less than item - // We can use BinarySearch to look in each segment for where that key would be, but we have to compensate for the fact that BinarySearch looks for the smallest key that is greater than or equal to the search key. - - // Also, the iterator can be used to move: - // - forward: from the current location, find the smallest key that is greater than the current cursor position - // - backward: from the current location, find the largest key that is smaller than the current cursor position - - T max = default(T); - int maxLevel = NOT_FOUND; - bool exact = false; - - var cursors = m_cursors; - var count = m_count; - - for (int i = m_min; i < cursors.Length; i++) - { - if (IsFree(i, count)) continue; - - var segment = m_levels[i]; - - int pos = BinarySearch(segment, 0, segment.Length, item, m_comparer); - - if (pos >= 0) - { // we found a match in this segment - - if (orEqual) - { // the item exist and is allowed - max = segment[pos]; - maxLevel = i; - exact = true; // stop checking for the max in other levels - } - else - { // the previous value is by definition less than 'item' - --pos; - } - } - else - { // not in this segment - - pos = ~pos; // <- position of where item would be place in this segment == position of the first item that is larger than item - // since segment[pos] > item, and item is not in segment, then segment[pos - 1] < item - --pos; - } - - // bound check - - if (pos < 0) - { // the value would be before this segment - cursors[i] = 0; - } - else if (pos >= segment.Length) - { // the value would be after this segment - cursors[i] = segment.Length; - } - else - { - cursors[i] = pos; - if (!exact && (maxLevel < 0 || m_comparer.Compare(segment[pos], max) > 0)) - { - max = segment[pos]; - maxLevel = i; - } - } - } - - m_currentLevel = maxLevel; - m_current = max; - m_direction = DIRECTION_SEEK; - Debug_Dump("Seek"); - return maxLevel >= 0; - } - - /// Move the cursor the the smallest value that is greater than the current value - public bool Next() - { - // invalid position, or no more values - if (m_currentLevel < 0) return false; - - var cursors = m_cursors; - var count = m_count; - - T prev = m_current; - T min = default(T); - int minLevel = NOT_FOUND; - int pos; - - if (m_direction >= DIRECTION_SEEK) - { // we know that the current position CANNOT be the next value, so increment that cursor - cursors[m_currentLevel]++; - Debug_Dump("Next:continue"); - } - else - { // previous call was a Previous() - // we know that the current is the largest value of all the current cursors. Since we want even larger than that, we have to increment ALL the cursors - for (int i = m_min; i < cursors.Length; i++) - { - if (!IsFree(i, count) && ((pos = cursors[i]) < m_levels[i].Length)) cursors[i] = pos + 1; - } - Debug_Dump("Next:reverse"); - } - - for (int i = m_min; i < cursors.Length; i++) - { - if (IsFree(i, count)) continue; - - pos = cursors[i]; - if (pos < 0) continue; //?? - - var segment = m_levels[i]; - - T x = default(T); - while(pos < segment.Length && m_comparer.Compare((x = segment[pos]), prev) < 0) - { // cannot be less than the previous value - cursors[i] = ++pos; - } - if (pos >= segment.Length) continue; - - if (minLevel < 0 || m_comparer.Compare(x, min) < 0) - { // new minimum - min = x; - minLevel = i; - } - } - - m_current = min; - m_currentLevel = minLevel; - m_direction = DIRECTION_NEXT; - return minLevel >= 0; - } - - /// Move the cursor the the largest value that is smaller than the current value - public bool Previous() - { - // invalid position, or no more values - if (m_currentLevel < 0) return false; - - var cursors = m_cursors; - var count = m_count; - - T prev = m_current; - T max = default(T); - int pos; - int maxLevel = NOT_FOUND; - - if (m_direction <= DIRECTION_SEEK) - { // we know that the current position CANNOT be the next value, so decrement that cursor - cursors[m_currentLevel]--; - Debug_Dump("Previous:continue"); - } - else - { // previous call was a call to Seek(), or Next() - // we know that the current is the smallest value of all the current cursors. Since we want even smaller than that, we have to decrement ALL the cursors - for (int i = m_min; i < cursors.Length; i++) - { - if (!IsFree(i, count) && ((pos = cursors[i]) >= 0)) cursors[i] = pos - 1; - } - Debug_Dump("Previous:reverse"); - } - - for (int i = m_min; i < cursors.Length; i++) - { - if (IsFree(i, count)) continue; - - pos = cursors[i]; - var segment = m_levels[i]; - if (pos >= segment.Length) continue; //?? - - T x = default(T); - while (pos >= 0 && m_comparer.Compare((x = segment[pos]), prev) > 0) - { // cannot be more than the previous value - cursors[i] = --pos; - } - if (pos < 0) continue; - - if (maxLevel < 0 || m_comparer.Compare(x, max) > 0) - { // new maximum - max = x; - maxLevel = i; - } - } - - m_current = max; - m_currentLevel = maxLevel; - m_direction = DIRECTION_PREVIOUS; - return maxLevel >= 0; - } - - /// Value of the current entry - public T Current - { - get { return m_current; } - } - - /// Checks if the current position of the iterator is valid - public bool Valid - { - get { return m_currentLevel >= 0; } - } - - /// Direction of the last operation - public int Direction - { - get { return m_direction; } - } - - } - - } -} \ No newline at end of file diff --git a/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs b/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs deleted file mode 100644 index 82576c64a..000000000 --- a/FoundationDB.Storage.Memory/Collections/ColaStore`1.cs +++ /dev/null @@ -1,1117 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -// enables consitency checks after each operation to the set -//#define ENFORCE_INVARIANTS - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Globalization; - using System.Linq; - using System.Runtime.CompilerServices; - - /// Store elements in a list of ordered levels - /// Type of elements stored in the set - public sealed class ColaStore - { - - #region Documentation - - // Based on http://supertech.csail.mit.edu/papers/sbtree.pdf (COLA) - - /* - The cache-oblivious lookahead array (COLA) is similar to the binomial list structure [9] of Bentley and Saxe. It consists of ⌈log2 N⌉ arrays, - or levels, each of which is either completely full or completely empty. The kth array is of size 2^k and the arrays are stored contiguously in memory. - - The COLA maintains the following invariants: - 1. The kth array contains items if and only if the kth least significant bit of the binary representation of N is a 1. - 2. Each array contains its items in ascending order by key - */ - - // DEFINITIONS - // - // "Level" is the index in the list of segments with level 0 being the top (or root) - // "Segment" is an array whose length is equal to 2^i (where i is the "level" of the segment). - // "Doubling Array" means that each segment has double the length of its predecessor - // "Cache Oblivious" means that the algorithm is not tuned for a specific CPU cache size (L1, L2, ou block size on disk), and amortize the cost of insertion over the lifespan of the set. - // - // INVARIANTS: - // - // * Each segment is twice the size of the previous segment, i.e.: m_levels[i].Length == 1 << i - // 0 [ ] 1 - // 1 [ , ] 2 - // 2 [ , , , ] 4 - // 3 [ , , , , , , , ] 8 - // 4 [ , , , , , , , , , , , , , , , ] 16 - // ... - // * A segment is either EMPTY, or completely FULL - // legal: [ , , , ] or [1,2,3,4] - // illegal: [1,2,3, ] - // * A segment has all its elements sorted - // legal: [3,12,42,66] - // illegal: [12,66,42,3] - // - // NOTES: - // - // - 50% of all inserts will always be done on the root (level 0), so will be O(1) - // - 87.5% of all inserts will only touch levels 0, 1 and 2, which should be contiguous in memory - // - For random insertions, it is difficult to predict in which level a specific value will be found, except that older values are towards the bottom, and younger values are towards the top. - // - A range of values (ex: "from 10 to 20") can have its elements scattered in multiple segments - // - If all inserts are ordered, then all items of level N will be sorted after all the items of level N + 1 - // - Most inserts are usually pretty fast, but every times the count goes to the next power of 2, the duration will be more and more noticeable (ie : the (2^N)th INSERT will have to merge (2^N) values) - // - // COST - // - // The cost for inserting N values is about N.Log2(N) comparisons - // - This is amortized to Log2(N) per insert, which means that insertion is O(log(N)) - // - This means that N should stay relatively low (ideally under 2^10 items) - - #endregion - - private const int INITIAL_LEVELS = 5; // 5 initial levels will pre-allocate space for 31 items - private const int MAX_SPARE_ORDER = 6; // 6 levels of spares will satisfy ~98.4% of all insertions, while only allocating the space for 63 items (~500 bytes for reference types) - private const int NOT_FOUND = -1; - - /// Number of elements in the store - private volatile int m_count; - - /// Array of all the segments making up the levels - private T[][] m_levels; - - /// Shortcut to level 0 (of size 1) - private T[] m_root; - - /// List of spare temporary buffers, used during merging - private T[][] m_spares; -#if ENFORCE_INVARIANTS - private bool[] m_spareUsed; -#endif - - /// Key comparer - private readonly IComparer m_comparer; - - #region Constructors... - - /// Allocates a new store - /// Initial capacity, or 0 for the default capacity - /// Comparer used to order the elements - public ColaStore(int capacity, IComparer comparer) - { - if (capacity < 0) throw new ArgumentOutOfRangeException(nameof(capacity), "Capacity cannot be less than zero."); - if (comparer == null) throw new ArgumentNullException(nameof(comparer)); - Contract.EndContractBlock(); - - int levels; - if (capacity == 0) - { // use the default capacity - levels = INITIAL_LEVELS; - } - else - { // L levels will only store (2^L - 1) - // note: there is no real penalty if the capacity was not correctly estimated, appart from the fact that all levels will not be contiguous in memory - // 1 => 1 - // 2..3 => 2 - // 4..7 => 3 - levels = ColaStore.HighestBit(capacity) + 1; - } - // allocating more than 31 levels would mean having an array of length 2^31, which is not possible - if (levels >= 31) throw new ArgumentOutOfRangeException(nameof(capacity), "Cannot allocate more than 30 levels"); - - // pre-allocate the segments and spares at the same time, so that they are always at the same memory location - var segments = new T[levels][]; - var spares = new T[MAX_SPARE_ORDER][]; - for (int i = 0; i < segments.Length; i++) - { - segments[i] = new T[1 << i]; - if (i < spares.Length) spares[i] = new T[1 << i]; - } - - m_levels = segments; - m_root = segments[0]; - m_spares = spares; -#if ENFORCE_INVARIANTS - m_spareUsed = new bool[spares.Length]; -#endif - m_comparer = comparer; - } - - [Conditional("ENFORCE_INVARIANTS")] - private void CheckInvariants() - { -#if ENFORCE_INVARIANTS - Contract.Assert(m_count >= 0, "Count cannot be less than zero"); - Contract.Assert(m_levels != null, "Storage array should not be null"); - Contract.Assert(m_levels.Length > 0, "Storage array should always at least contain one level"); - Contract.Assert(object.ReferenceEquals(m_root, m_levels[0]), "The root should always be the first level"); - Contract.Assert(m_count < 1 << m_levels.Length, "Count should not exceed the current capacity"); - - for (int i = 0; i < m_levels.Length; i++) - { - var segment = m_levels[i]; - Contract.Assert(segment != null, "All segments should be allocated in memory"); - Contract.Assert(segment.Length == 1 << i, "The size of a segment should be 2^LEVEL"); - - if (IsFree(i)) - { // All unallocated segments SHOULD be filled with default(T) - for (int j = 0; j < segment.Length; j++) - { - if (!EqualityComparer.Default.Equals(segment[j], default(T))) - { - if (Debugger.IsAttached) { Debug_Dump(); Debugger.Break(); } - Contract.Assert(false, String.Format("Non-zero value at offset {0} of unused level {1} : {2}", j, i, String.Join(", ", segment))); - } - } - } - else - { // All allocated segments SHOULD be sorted - T previous = segment[0]; - for (int j = 1; j < segment.Length; j++) - { - T x = segment[j]; - if (m_comparer.Compare(previous, x) >= 0) - { - if (Debugger.IsAttached) { Debug_Dump(); Debugger.Break(); } - Contract.Assert(false, String.Format("Unsorted value {3} at offset {0} of allocated level {1} : {2}", j, i, String.Join(", ", segment), segment[j])); - } - previous = segment[j]; - } - } - - if (i < m_spares.Length) - { - Contract.Assert(!m_spareUsed[i], "A spare level wasn't returned after being used!"); - var spare = m_spares[i]; - if (spare == null) continue; - // All spare segments SHOULD be filled with default(T) - for (int j = 0; j < spare.Length; j++) - { - if (!EqualityComparer.Default.Equals(spare[j], default(T))) - { - if (Debugger.IsAttached) { Debug_Dump(); Debugger.Break(); } - Contract.Assert(false, String.Format("Non-zero value at offset {0} of spare level {1} : {2}", j, i, String.Join(", ", spare))); - } - } - - } - } -#endif - } - - #endregion - - #region Public Properties... - - /// Gets the number of elements in the store. - public int Count - { - get { return m_count; } - } - - /// Gets the current capacity of the store. - public int Capacity - { - // note: the capacity is always 2^L - 1 where L is the number of levels - get { return m_levels == null ? 0 : (1 << m_levels.Length) - 1; } - } - - /// Gets the comparer used to sort the elements in the store - public IComparer Comparer - { - get { return m_comparer; } - } - - /// Gets the current number of levels - /// Note that the last level may not be currently used! - public int Depth - { - get { return m_levels.Length; } - } - - /// Gets the index of the first currently allocated level - public int MinLevel - { - get { return ColaStore.HighestBit(m_count); } - } - - /// Gets the index of the last currently allocated level - public int MaxLevel - { - get { return ColaStore.HighestBit(m_count); } - } - - /// Gets the list of all levels - public T[][] Levels - { - get { return m_levels; } - } - - /// Returns the content of a level - /// Index of the level (0-based) - /// Segment that contains all the elements of that level - public T[] GetLevel(int level) - { - Contract.Assert(level >= 0 && level < m_levels.Length); - return m_levels[level]; - } - - /// Gets of sets the value store at the specified index - /// Absolute index in the vector-array - /// Value stored at that location, or default(T) if the location is in an unallocated level - public T this[int arrayIndex] - { - get - { - if (m_count == 1 && arrayIndex == 0) return m_root[0]; - return GetAt(arrayIndex); - } - set - { - SetAt(arrayIndex, value); - } - } - - #endregion - - #region Public Methods... - - /// Finds the location of an element in the array - /// Value of the element to search for. - /// Receives the offset of the element inside the level if found; otherwise, 0. - /// Level that contains the element if found; otherwise, -1. - public int Find(T value, out int offset, out T actualValue) - { - if ((m_count & 1) != 0) - { - // If someone gets the last inserted key, there is a 50% change that it is in the root - // (if not, it will the the last one of the first non-empty level) - if (m_comparer.Compare(value, m_root[0]) == 0) - { - offset = 0; - actualValue = m_root[0]; - return 0; - } - } - - var levels = m_levels; - for (int i = 1; i < levels.Length; i++) - { - if (IsFree(i)) - { // this segment is not allocated - continue; - } - - int p = ColaStore.BinarySearch(levels[i], 0, 1 << i, value, m_comparer); - if (p >= 0) - { - offset = p; - actualValue = levels[i][p]; - return i; - } - } - offset = 0; - actualValue = default(T); - return NOT_FOUND; - } - - /// Search for the smallest element that is larger than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the next element, or 0 if not found - /// Receive the value of the next element, or default(T) if not found - /// Level of the next element, or -1 if was already the largest - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int FindNext(T value, bool orEqual, out int offset, out T result) - { - return ColaStore.FindNext(m_levels, m_count, value, orEqual, m_comparer, out offset, out result); - } - - /// Search for the smallest element that is larger than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the next element, or 0 if not found - /// Receive the value of the next element, or default(T) if not found - /// Level of the next element, or -1 if was already the largest - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int FindNext(T value, bool orEqual, IComparer comparer, out int offset, out T result) - { - return ColaStore.FindNext(m_levels, m_count, value, orEqual, comparer ?? m_comparer, out offset, out result); - } - - /// Search for the largest element that is smaller than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the previous element, or 0 if not found - /// Receive the value of the previous element, or default(T) if not found - /// Level of the previous element, or -1 if was already the smallest - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int FindPrevious(T value, bool orEqual, out int offset, out T result) - { - return ColaStore.FindPrevious(m_levels, m_count, value, orEqual, m_comparer, out offset, out result); - } - - /// Search for the largest element that is smaller than a reference element - /// Reference element - /// If true, return the position of the value itself if it is found. If false, return the position of the closest value that is smaller. - /// Receive the offset within the level of the previous element, or 0 if not found - /// Receive the value of the previous element, or default(T) if not found - /// Level of the previous element, or -1 if was already the smallest - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public int FindPrevious(T value, bool orEqual, IComparer comparer, out int offset, out T result) - { - return ColaStore.FindPrevious(m_levels, m_count, value, orEqual, comparer ?? m_comparer, out offset, out result); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public IEnumerable FindBetween(T begin, bool beginOrEqual, T end, bool endOrEqual, int limit) - { - return ColaStore.FindBetween(m_levels, m_count, begin, beginOrEqual, end, endOrEqual, limit, m_comparer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public IEnumerable FindBetween(T begin, bool beginOrEqual, T end, bool endOrEqual, int limit, IComparer comparer) - { - return ColaStore.FindBetween(m_levels, m_count, begin, beginOrEqual, end, endOrEqual, limit, comparer ?? m_comparer); - } - - /// Return the value stored at a specific location in the array - /// Absolute index in the vector-array - /// Value stored at this location, or default(T) if the level is not allocated - public T GetAt(int arrayIndex) - { - Contract.Assert(arrayIndex >= 0 && arrayIndex <= this.Capacity); - - int offset; - int level = ColaStore.FromIndex(arrayIndex, out offset); - - return GetAt(level, offset); - } - - /// Returns the value at a specific location in the array - /// Index of the level (0-based) - /// Offset in the level (0-based) - /// Returns the value at this location, or default(T) if the level is not allocated - public T GetAt(int level, int offset) - { - Contract.Assert(level >= 0 && level < m_levels.Length && offset >= 0 && offset < 1 << level); - //TODO: check if level is allocated ? - - var segment = m_levels[level]; - Contract.Assert(segment != null && segment.Length == 1 << level); - return segment[offset]; - } - - /// Store a value at a specific location in the arrayh - /// Absolute index in the vector-array - /// Value to store - /// Previous value at that location - public T SetAt(int arrayIndex, T value) - { - Contract.Assert(arrayIndex >= 0 && arrayIndex <= this.Capacity); - - int offset; - int level = ColaStore.FromIndex(arrayIndex, out offset); - - return SetAt(level, offset, value); - } - - /// Overwrites a specific location in the array with a new value, and returns its previous value - /// Index of the level (0-based) - /// Offset in the level (0-based) - /// New value for this location - /// Previous value at this location - public T SetAt(int level, int offset, T value) - { - Contract.Assert(level >= 0 && level < m_levels.Length && offset >= 0 && offset < 1 << level); - //TODO: check if level is allocated ? - - var segment = m_levels[level]; - Contract.Assert(segment != null && segment.Length == 1 << level); - T previous = segment[offset]; - segment[offset] = value; - return previous; - } - - /// Clear the array - public void Clear() - { - for (int i = 0; i < m_levels.Length; i++) - { - if (i < MAX_SPARE_ORDER) - { - Array.Clear(m_levels[i], 0, 1 << i); - } - else - { - m_levels[i] = null; - } - } - m_count = 0; - if (m_levels.Length > MAX_SPARE_ORDER) - { - Array.Resize(ref m_levels, MAX_SPARE_ORDER); - } - - CheckInvariants(); - } - - /// Add a value to the array - /// Value to add to the array - /// If already exists in the array and is true, it will be overwritten with - /// If the value did not if the value was been added to the array, or false if it was already there. - public bool SetOrAdd(T value, bool overwriteExistingValue) - { - T _; - int offset, level = Find(value, out offset, out _); - if (level >= 0) - { - if (overwriteExistingValue) - { - m_levels[level][offset] = value; - } - return false; - } - - Insert(value); - return true; - } - - /// Insert a new element in the set, and returns its index. - /// Value to insert. Warning: if the value already exists, the store will be corrupted ! - /// The index is the absolute index, as if all the levels where a single, contiguous, array (0 = root, 7 = first element of level 3) - public void Insert(T value) - { - if (IsFree(0)) - { // half the inserts (when the count is even) can be done in the root - m_root[0] = value; - } - else if (IsFree(1)) - { // a quarter of the inserts only need to move the root and the value to level 1 - ColaStore.MergeSimple(m_levels[1], m_root[0], value, m_comparer); - m_root[0] = default(T); - } - else - { // we need to merge one or more levels - - var spare = GetSpare(0); - if (object.ReferenceEquals(spare, m_root)) Debugger.Break(); - Contract.Assert(spare != null && spare.Length == 1); - spare[0] = value; - MergeCascade(1, m_root, spare); - PutSpare(0, spare); - m_root[0] = default(T); - } - ++m_count; - - CheckInvariants(); - } - - /// Insert two elements in the set. - public void InsertItems(T first, T second) - { - Contract.Requires(m_comparer.Compare(first, second) != 0, "Cannot insert the same value twice"); - - if (IsFree(1)) - { - ColaStore.MergeSimple(m_levels[1], first, second, m_comparer); - } - else - { - //Console.WriteLine("InsertItems([2]) Cascade"); - var spare = GetSpare(1); - spare[0] = first; - spare[1] = second; - var segment = m_levels[1]; - MergeCascade(2, segment, spare); - segment[0] = default(T); - segment[1] = default(T); - PutSpare(1, spare); - } - m_count += 2; - - CheckInvariants(); - } - - /// Insert one or more new elements in the set. - /// Array of elements to insert. Warning: if a value already exist, the store will be corrupted ! - /// If true, the entries in are guaranteed to already be sorted (using the store default comparer). - /// The best performances are achieved when inserting a number of items that is a power of 2. The worst performances are when doubling the size of a store that is full. - /// Warning: if is true but is not sorted, or is sorted using a different comparer, then the store will become corrupted ! - /// - public void InsertItems(List values, bool ordered = false) - { - if (values == null) throw new ArgumentNullException(nameof(values)); - - int count = values.Count; - T[] segment, spare; - - if (count < 2) - { - if (count == 1) - { - Insert(values[0]); - } - return; - } - - if (count == 2) - { - if (IsFree(1)) - { - segment = m_levels[1]; - if (ordered) - { - segment[0] = values[0]; - segment[1] = values[1]; - } - else - { - ColaStore.MergeSimple(segment, values[0], values[1], m_comparer); - } - } - else - { - spare = GetSpare(1); - spare[0] = values[0]; - spare[1] = values[1]; - segment = m_levels[1]; - MergeCascade(2, segment, spare); - segment[0] = default(T); - segment[1] = default(T); - PutSpare(1, spare); - } - } - else - { - // Inserting a size that is a power of 2 is very simple: - // * either the corresponding level is empty, in that case we just copy the items and do a quicksort - // * or it is full, then we just need to do a cascade merge - // For non-power of 2s, we can split decompose them into a suite of power of 2s and insert them one by one - - int min = ColaStore.LowestBit(count); - int max = ColaStore.HighestBit(count); - - if (max >= m_levels.Length) - { // we need to allocate new levels - Grow(max); - } - - int p = 0; - for (int i = min; i <= max; i++) - { - if (ColaStore.IsFree(i, count)) continue; - - segment = m_levels[i]; - if (IsFree(i)) - { // the target level is free, we can copy and sort in place - values.CopyTo(p, segment, 0, segment.Length); - if (!ordered) Array.Sort(segment, 0, segment.Length, m_comparer); - p += segment.Length; - m_count += segment.Length; - } - else - { // the target level is used, we will have to do a cascade merge, using a spare - spare = GetSpare(i); - values.CopyTo(p, spare, 0, spare.Length); - if (!ordered) Array.Sort(spare, 0, spare.Length, m_comparer); - p += segment.Length; - MergeCascade(i + 1, segment, spare); - Array.Clear(segment, 0, segment.Length); - PutSpare(i, spare); - m_count += segment.Length; - } - } - Contract.Assert(p == count); - } - - CheckInvariants(); - } - - /// Remove the value at the specified location - /// Absolute index in the vector-array - /// Value that was removed - public T RemoveAt(int arrayIndex) - { - Contract.Requires(arrayIndex >= 0 && arrayIndex <= this.Capacity); - int offset, level = ColaStore.FromIndex(arrayIndex, out offset); - return RemoveAt(level, offset); - } - - /// Remove the value at the specified location - /// Index of the level (0-based) - /// Offset in the level (0-based) - /// Value that was removed - public T RemoveAt(int level, int offset) - { - Contract.Assert(level >= 0 && offset >= 0 && offset < 1 << level); - //TODO: check if level is allocated ? - - var segment = m_levels[level]; - Contract.Assert(segment != null && segment.Length == 1 << level); - T removed = segment[offset]; - - if (level == 0) - { // removing the last inserted value - segment[0] = default(T); - } - else if (level == 1) - { // split the first level in two - if (IsFree(0)) - { // move up to root - - // ex: remove 'b' at (1,1) and move the 'a' back to the root - // 0 [_] => [a] - // 1 [a,b] => [_,_] - - m_root[0] = segment[1 - offset]; - segment[0] = default(T); - segment[1] = default(T); - } - else - { // merge the root in missing spot - - // ex: remove 'b' at (1,1) and move the 'c' down a level - // N = 3 N = 2 - // 0 [c] => 0 [_] - // 1 [a,b] => 1 [a,c] - - ColaStore.MergeSimple(segment, m_root[0], segment[1 - offset], m_comparer); - m_root[0] = default(T); - } - } - else if ((m_count & 1) == 1) - { // Remove an item from an odd-numbered set - - // Since the new count will be even, we only need to merge the root in place with the level that is missing a spot - - // ex: replace the 'b' at (2,1) with the 'e' in the root - // N = 5 N = 4 - // 0 [e] => 0 [_] - // 1 [_,_] 1 [_,_] - // 2 [a,b,c,d] => 2 [a,c,d,e] - - ColaStore.MergeInPlace(segment, offset, m_root[0], m_comparer); - m_root[0] = default(T); - } - else - { - // we are missing a spot in out modified segment, that need to fill - // > we will take the first non empty segment, and break it in pieces - // > its last item will be used to fill the empty spot - // > the rest of its items will be spread to all the previous empty segments - - // find the first non empty segment that can be broken - int firstNonEmptyLevel = ColaStore.LowestBit(m_count); - - if (firstNonEmptyLevel == level) - { // we are the first level, this is easy ! - - // move the empty spot at the start - if (offset > 0) Array.Copy(segment, 0, segment, 1, offset); - - // and spread the rest to all the previous levels - ColaStore.SpreadLevel(level, m_levels); - //TODO: modify SpreadLevel(..) to take the offset of the value to skip ? - } - else - { // break that level, and merge its last item with the level that is missing one spot - - // break down this level - T tmp = ColaStore.SpreadLevel(firstNonEmptyLevel, m_levels); - - // merge its last item with the empty spot in the modified level - ColaStore.MergeInPlace(m_levels[level], offset, tmp, m_comparer); - } - } - - --m_count; - - if (m_levels.Length > MAX_SPARE_ORDER) - { // maybe release the last level if it is empty - ShrinkIfRequired(); - } - - CheckInvariants(); - - return removed; - } - - public bool RemoveItem(T item) - { - T _; - int offset, level = Find(item, out offset, out _); - if (level < 0) return false; - _ = RemoveAt(level, offset); - CheckInvariants(); - return true; - } - - public int RemoveItems(IEnumerable items) - { - if (items == null) throw new ArgumentNullException(nameof(items)); - - T _; - int count = 0; - - //TODO: optimize this !!!! - foreach(var item in items) - { - int offset, level = Find(item, out offset, out _); - if (level >= 0) - { - RemoveAt(level, offset); - ++count; - } - - } - CheckInvariants(); - return count; - } - - public void CopyTo(T[] array, int arrayIndex, int count) - { - if (array == null) throw new ArgumentNullException(nameof(array)); - if (arrayIndex < 0) throw new ArgumentOutOfRangeException(nameof(arrayIndex), "Index cannot be less than zero."); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero."); - if (arrayIndex > array.Length || count > (array.Length - arrayIndex)) throw new ArgumentException("Destination array is too small"); - Contract.EndContractBlock(); - - int p = arrayIndex; - count = Math.Min(count, m_count); - foreach (var item in ColaStore.IterateOrdered(count, m_levels, m_comparer, false)) - { - array[p++] = item; - } - Contract.Assert(p == arrayIndex + count); - } - - /// Checks if a level is currently not allocated - /// Index of the level (0-based) - /// True is the level is unallocated and does not store any elements; otherwise, false. - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public bool IsFree(int level) - { - Contract.Requires(level >= 0); - return (m_count & (1 << level)) == 0; - } - - /// Gets a temporary buffer with the length corresponding to the specified level - /// Level of this spare buffer - /// Temporary buffer whose size is 2^level - /// The buffer should be returned after use by calling - public T[] GetSpare(int level) - { - Contract.Requires(level >= 0 && m_spares != null); - - if (level < m_spares.Length) - { // this level is kept in the spare list - -#if ENFORCE_INVARIANTS - Contract.Assert(!m_spareUsed[level], "this spare is already in use!"); -#endif - - var t = m_spares[level]; - if (t == null) - { // allocate a new one - t = new T[1 << level]; - m_spares[level] = t; - } -#if ENFORCE_INVARIANTS - m_spareUsed[level] = true; -#endif - return t; - } - else - { // this level is always allocated - return new T[1 << level]; - } - } - - /// Return a temporary buffer after use - /// Level of the temporary buffer - /// True if the buffer has been cleared and returned to the spare list, false if it was discarded - /// Kept buffers are cleared to prevent values from being kept alive and not garbage collected. - public bool PutSpare(int level, T[] spare) - { - Contract.Assert(level >= 0 && spare != null); - -#if ENFORCE_INVARIANTS - // make sure that we do not mix levels and spares - for (int i = 0; i < m_levels.Length; i++) - { - if (object.ReferenceEquals(m_levels[i], spare)) Debugger.Break(); - } -#endif - - // only clear spares that are kept alive - if (level < m_spares.Length) - { -#if ENFORCE_INVARIANTS - Contract.Assert(m_spareUsed[level], "this spare wasn't used"); -#endif - - // clear it in case it holds onto dead values that could be garbage collected - spare[0] = default(T); - if (level > 0) - { - spare[1] = default(T); - if (level > 1) Array.Clear(spare, 2, spare.Length - 2); - } -#if ENFORCE_INVARIANTS - m_spareUsed[level] = false; -#endif - return true; - } - return false; - } - - /// Find the smallest element in the store - /// Smallest element found, or default(T) if the store is empty - public T Min() - { - switch (m_count) - { - case 0: return default(T); - case 1: return m_root[0]; - case 2: return m_levels[1][0]; - default: - { - - int level = ColaStore.LowestBit(m_count); - int end = ColaStore.HighestBit(m_count); - T min = m_levels[level][0]; - while (level <= end) - { - if (!IsFree(level) && m_comparer.Compare(min, m_levels[level][0]) > 0) - { - min = m_levels[level][0]; - } - ++level; - } - return min; - } - } - } - - /// Find the largest element in the store - /// Largest element found, or default(T) if the store is empty - public T Max() - { - switch (m_count) - { - case 0: return default(T); - case 1: return m_root[0]; - case 2: return m_levels[1][1]; - default: - { - int level = ColaStore.LowestBit(m_count); - int end = ColaStore.HighestBit(m_count); - T max = m_levels[level][0]; - while (level <= end) - { - if (!IsFree(level) && m_comparer.Compare(max, m_levels[level][0]) < 0) - { - max = m_levels[level][0]; - } - ++level; - } - return max; - } - } - - } - - /// Returns the smallest and largest element in the store - /// Receives the value of the smallest element (or default(T) is the store is Empty) - /// Receives the value of the largest element (or default(T) is the store is Empty) - /// If the store contains only one element, than min and max will be equal - public void GetBounds(out T min, out T max) - { - switch (m_count) - { - case 0: - { - min = default(T); - max = default(T); - break; - } - case 1: - { - min = m_root[0]; - max = min; - break; - } - case 2: - { - min = m_levels[1][0]; - max = m_levels[1][1]; - break; - } - default: - { - - int level = ColaStore.LowestBit(m_count); - int end = ColaStore.HighestBit(m_count); - var segment = m_levels[level]; - min = segment[0]; - max = segment[segment.Length - 1]; - while (level <= end) - { - if (IsFree(level)) continue; - segment = m_levels[level]; - if (m_comparer.Compare(min, segment[0]) > 0) min = segment[0]; - if (m_comparer.Compare(max, segment[segment.Length - 1]) < 0) min = segment[segment.Length - 1]; - ++level; - } - break; - } - } - } - - public ColaStore.Iterator GetIterator() - { - return new ColaStore.Iterator(m_levels, m_count, m_comparer); - } - - /// Pre-allocate memory in the store so that it can store a specified amount of items - /// Number of items that will be inserted in the store - public void EnsureCapacity(int minimumRequired) - { - int level = ColaStore.HighestBit(minimumRequired); - if ((1 << level) < minimumRequired) ++level; - - if (level >= m_levels.Length) - { - Grow(level); - } - } - - #endregion - - private void MergeCascade(int level, T[] left, T[] right) - { - Contract.Requires(level > 0, "level"); - Contract.Requires(left != null && left.Length == (1 << (level - 1)), "left"); - Contract.Requires(right != null && right.Length == (1 << (level - 1)), "right"); - - if (IsFree(level)) - { // target level is empty - - if (level >= m_levels.Length) Grow(level); - Contract.Assert(level < m_levels.Length); - - ColaStore.MergeSort(m_levels[level], left, right, m_comparer); - } - else if (IsFree(level + 1)) - { // the next level is empty - - if (level + 1 >= m_levels.Length) Grow(level + 1); - Contract.Assert(level + 1 < m_levels.Length); - - var spare = GetSpare(level); - ColaStore.MergeSort(spare, left, right, m_comparer); - var next = m_levels[level]; - ColaStore.MergeSort(m_levels[level + 1], next, spare, m_comparer); - Array.Clear(next, 0, next.Length); - PutSpare(level, spare); - } - else - { // both are full, need to do a cascade merge - - Contract.Assert(level < m_levels.Length); - - // merge N and N +1 - var spare = GetSpare(level); - ColaStore.MergeSort(spare, left, right, m_comparer); - - // and cascade to N + 2 ... - var next = m_levels[level]; - MergeCascade(level + 1, next, spare); - Array.Clear(next, 0, next.Length); - PutSpare(level, spare); - } - } - - /// Grow the capacity of the level array - /// Minimum level required - private void Grow(int level) - { - Contract.Requires(level >= 0); - - // note: we want m_segments[level] to not be empty, which means there must be at least (level + 1) entries in the level array - int current = m_levels.Length; - int required = level + 1; - Contract.Assert(current < required); - - var tmpSegments = m_levels; - Array.Resize(ref tmpSegments, required); - for (int i = current; i < required; i++) - { - tmpSegments[i] = new T[1 << i]; - } - m_levels = tmpSegments; - - Contract.Ensures(m_levels != null && m_levels.Length > level); - } - - private void ShrinkIfRequired() - { - int n = m_levels.Length - 1; - if (n <= MAX_SPARE_ORDER) return; - if (IsFree(n)) - { // less than 50% full - - // to avoid the degenerate case of constantly Adding/Removing when at the threshold of a new level, - // we will only remove the last level if the previous level is also empty - - if (IsFree(n - 1)) - { // less than 25% full - - // remove the last level - var tmpSegments = new T[n][]; - Array.Copy(m_levels, tmpSegments, n); - m_levels = tmpSegments; - } - } - } - - internal IEnumerable IterateOrdered(bool reverse = false) - { - return ColaStore.IterateOrdered(m_count, m_levels, m_comparer, reverse); - } - - internal IEnumerable IterateUnordered() - { - return ColaStore.IterateUnordered(m_count, m_levels); - } - - //TODO: remove or set to internal ! - [Conditional("DEBUG")] - public void Debug_Dump(Func dump = null) - { -#if DEBUG - Trace.WriteLine("> " + m_levels.Length + " levels:"); - for(int i = 0; i < m_levels.Length; i++) - { - string s = dump == null ? String.Join(", ", m_levels[i]) : String.Join(", ", m_levels[i].Select(dump)); - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, " - {0,2}|{1}: {2}", i, IsFree(i) ? "_" : "#", s)); - } -#if false - Trace.WriteLine("> " + m_spares.Length + " spares:"); - for (int i = 0; i < m_spares.Length; i++) - { - var spare = m_spares[i]; - Trace.WriteLine(String.Format(CultureInfo.InvariantCulture, "> {0,2}: {1}", i, spare == null ? "" : String.Join(", ", spare))); - } -#endif - Trace.WriteLine("> " + m_count + " items"); -#endif - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Entry.cs b/FoundationDB.Storage.Memory/Core/Entry.cs deleted file mode 100644 index 041e44c21..000000000 --- a/FoundationDB.Storage.Memory/Core/Entry.cs +++ /dev/null @@ -1,107 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Diagnostics; - using System.Runtime.InteropServices; - - public enum EntryType : ushort - { - Free = 0, - Key = 1, - Value = 2, - Search = 3 - } - - [DebuggerDisplay("Header={Header}, Size={Size}")] - [StructLayout(LayoutKind.Sequential, Pack = 1)] - internal unsafe struct Entry - { - /// Default alignement for objects (8 by default) - public const int ALIGNMENT = 8; // MUST BE A POWER OF 2 ! - public const int ALIGNMENT_MASK = ~(ALIGNMENT - 1); - - /// A read lock has been taken on this entry - public const ushort FLAGS_READ_LOCK = 1 << 8; - - /// A write lock has been taken on this entry - public const ushort FLAGS_WRITE_LOCK = 1 << 9; - - /// A GC lock has been taken on this entry - public const ushort FLAGS_GC_LOCK = 1 << 10; - - /// This entry has been moved to another page by the last GC - public const ushort FLAGS_MOVED = 1 << 11; - - /// This key has been flaged as being unreachable by current of future transaction (won't survive the next GC) - public const ushort FLAGS_UNREACHABLE = 1 << 12; - - /// The entry has been disposed and should be access anymore - public const ushort FLAGS_DISPOSED = 1 << 15; - - public const int TYPE_SHIFT = 13; - public const ushort TYPE_MASK_AFTER_SHIFT = 0x3; - - // Object Layout - // ============== - - // Offset Field Type Desc - // - // 0 HEADER ushort type, Flags, ... - // 2 HASH ushort 16-bit hashcode - // 4 SIZE uint Size of the data - // ... object fields ... - // x DATA byte[] Value of the object, size in the SIZE field - // y (pad) 0..7 padding bytes (set to 00 or FF ?) - // - // HEADER: bit flags - // - bit 8: READ LOCK - // - bit 9: WRITE LOCK - // - bit 10: GC LOCK - // - bit 11: MOVED - // - bit 12: UNREACHABLE - // - bit 13-14: TYPE - // - bit 15: DISPOSED, set if object is disposed - - /// Various flags - public ushort Header; - - public ushort Hash; - - /// Size of the key (in bytes) - public uint Size; - - /// Return the type of the object - public static unsafe EntryType GetObjectType(void* item) - { - return item == null ? EntryType.Free : (EntryType)((((Entry*)item)->Header >> TYPE_SHIFT) & TYPE_MASK_AFTER_SHIFT); - } - - /// Checks if the object is disposed - public static unsafe bool IsDisposed(void* item) - { - return item == null || (((Entry*)item)->Header & FLAGS_DISPOSED) != 0; - } - - internal static byte* Align(byte* ptr) - { - long r = ((long)ptr) & (ALIGNMENT - 1); - if (r > 0) ptr += ALIGNMENT - r; - return ptr; - } - - internal static bool IsAligned(void* ptr) - { - return (((long)ptr) & (ALIGNMENT - 1)) == 0; - } - - internal static int Padding(void* ptr) - { - return (int)(((long)ptr) & (ALIGNMENT - 1)); - } - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Key.cs b/FoundationDB.Storage.Memory/Core/Key.cs deleted file mode 100644 index ce75d2dd2..000000000 --- a/FoundationDB.Storage.Memory/Core/Key.cs +++ /dev/null @@ -1,93 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - [StructLayout(LayoutKind.Sequential, Pack = 1)] - internal unsafe struct Key - { - // A Key contains the key's bytes, an hashcode, and a pointer to the most current Value for this key, or null if the key is currently deleted - - // Field Offset Bits Type Desc - // HEADER 0 16 flags Type, status flags, deletion or mutation flags, .... - // SIZE 2 16 uint16 Size of the DATA field (from 0 to 10,000). Note: bit 14 and 15 are usually 0 and could be used for something?) - // HASHCODE 4 32 uint32 Hashcode (note: size only need 2 bytes, so maybe we could extand this to 24 bits?) - // VALUEPTR 8 64 Value* Pointer to the most current value of this key (or null if the DELETION bit is set in the header) - // DATA 16 .. byte[] First byte of the key - - // The HEADER flags are as follow: - // - bit 0: NEW If set, this key has been inserted after the last GC - // - bit 1: MUTATED If set, this key has changed aster the last GC - // - bit 2-5: unused - // - bit 7: HAS_WATCH If set, this key is currently being watched - // - bit 8-15: ENTRY_FLAGS (inherited from Entry) - - public static readonly uint SizeOf = (uint)Marshal.OffsetOf(typeof(Key), "Data").ToInt32(); - - /// The key has been inserted after the last GC - public const ushort FLAGS_NEW = 1 << 0; - /// The key has been created/mutated since the last GC - public const ushort FLAGS_MUTATED = 1 << 1; - /// There is a watch listening on this key - public const ushort FLAGS_HAS_WATCH = 1 << 7; - - /// Various flags (TODO: enum?) - public ushort Header; - /// Size of the key (in bytes) - public ushort Size; - /// Hashcode of the key - public int HashCode; - /// Pointer to the head of the value chain for this key (should not be null) - public Value* Values; - /// Offset to the first byte of the key - public byte Data; - - public static USlice GetData(Key* self) - { - if (self == null) return default(USlice); - Contract.Assert((self->Header & Entry.FLAGS_DISPOSED) == 0, "Attempt to read a key that was disposed"); - return new USlice(&(self->Data), self->Size); - } - - public static bool StillAlive(Key* self, ulong sequence) - { - if (self == null) return false; - - if ((self->Header & Entry.FLAGS_UNREACHABLE) != 0) - { // we have been marked as dead - - var value = self->Values; - if (value == null) return false; - - // check if the last value is a deletion? - if (value->Sequence <= sequence && (value->Header & Value.FLAGS_DELETION) != 0) - { // it is deleted - return false; - } - } - - return true; - } - - public static bool IsDisposed(Key* self) - { - return (self->Header & Entry.FLAGS_DISPOSED) != 0; - } - - /// Return the address of the following value in the heap - internal static Key* WalkNext(Key* self) - { - Contract.Requires(self != null && Entry.GetObjectType(self) == EntryType.Key); - - return (Key*)Entry.Align((byte*)self + Key.SizeOf + self->Size); - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs b/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs deleted file mode 100644 index 5a62577bf..000000000 --- a/FoundationDB.Storage.Memory/Core/Memory/ElasticHeap`1.cs +++ /dev/null @@ -1,171 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Linq; - using System.Runtime.InteropServices; - - /// Generic implementation of an elastic heap that uses one or more page to store objects of the same type, using multiple buckets for different page sizes - /// Type of the pages in the elastic heap - internal abstract class ElasticHeap : IDisposable - where TPage : EntryPage - { - private const uint MinAllowedPageSize = 4096; // ~= memory mapped page - private const uint MaxAllowedPageSize = 1 << 30; // 1GB - - protected readonly TPage[] m_currents; - protected readonly PageBucket[] m_buckets; - protected Func m_allocator; - private volatile bool m_disposed; - - protected struct PageBucket - { - public readonly uint PageSize; - public readonly List Pages; - public readonly List FreeList; - - public PageBucket(uint size) - { - this.PageSize = size; - this.Pages = new List(); - this.FreeList = new List(); - } - } - - protected ElasticHeap(uint[] sizes, Func allocator) - { - if (sizes == null) throw new ArgumentNullException("sizes"); - if (allocator == null) throw new ArgumentNullException("allocator"); - if (sizes.Length == 0) throw new ArgumentException("There must be at least one allocation size"); - - var buckets = new PageBucket[sizes.Length]; - for (int i = 0; i < buckets.Length; i++) - { - if (sizes[i] < MinAllowedPageSize || sizes[i] > MaxAllowedPageSize) throw new ArgumentException(String.Format("Page size {0} too small or not a power of two", sizes[i]), "sizes"); - if (sizes[i] % Entry.ALIGNMENT != 0) throw new ArgumentException(String.Format("Page size {0} must be aligned to {1} bytes", sizes[i], Entry.ALIGNMENT)); - buckets[i] = new PageBucket(sizes[i]); - } - m_buckets = buckets; - m_currents = new TPage[sizes.Length]; - m_allocator = allocator; - } - - /// Allocate a new page for a specific bucket - /// Bucet index - protected TPage CreateNewPage(int bucket) - { - uint size = m_buckets[bucket].PageSize; - - UnmanagedHelpers.SafeLocalAllocHandle handle = null; - try - { - handle = UnmanagedHelpers.AllocMemory(size); - return m_allocator(handle, size); - } - catch (Exception e) - { - if (handle != null) - { - if (!handle.IsClosed) handle.Dispose(); - handle = null; - } - if (e is OutOfMemoryException) - { - throw new OutOfMemoryException(String.Format("Failed to allocate new memory for new page of size {0}", size), e); - } - throw; - } - finally - { - if (handle != null) GC.AddMemoryPressure(size); - } - } - - /// Returns the estimated allocated size in all the buckets - public ulong GetAllocatedSize() - { - ulong sum = 0; - foreach (var bucket in m_buckets) - { - if (bucket.PageSize > 0 && bucket.Pages != null) - { - sum += (ulong)bucket.PageSize * (uint)bucket.Pages.Count; - } - } - return sum; - } - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - protected virtual void Dispose(bool disposing) - { - if (!m_disposed) - { - m_disposed = true; - if (disposing) - { - for (int i = 0; i < m_buckets.Length; i++) - { - foreach (var page in m_buckets[i].Pages) - { - if (page != null) page.Dispose(); - } - foreach (var page in m_buckets[i].FreeList) - { - if (page != null) page.Dispose(); - } - } - Array.Clear(m_buckets, 0, m_buckets.Length); - Array.Clear(m_currents, 0, m_currents.Length); - } - m_allocator = null; - } - } - - [Conditional("DEBUG")] - public void Debug_Dump(bool detailed) - { -#if DEBUG - Debug.WriteLine("# Dumping {0} heap ({1:N0} pages in {2:N0} buckets)", this.GetType().Name, m_buckets.Sum(b => (long)b.Pages.Count), m_buckets.Length); - //TODO: needs locking but should only be called from unit tests anyway... - ulong entries = 0; - ulong allocated = 0; - ulong used = 0; - for (int i = 0; i < m_buckets.Length; i++) - { - var bucket = m_buckets[i]; - if (bucket.Pages == null) continue; - if (bucket.Pages.Count == 0) - { - Debug.WriteLine(" # Bucket #{0}: {1:N0} bytes is empty", i, bucket.PageSize); - } - else - { - Debug.WriteLine(" # Bucket #{0}: {1:N0} bytes (allocated: {2:N0} pages, free: {3:N0} pages)", i, bucket.PageSize, bucket.Pages.Count, bucket.FreeList.Count); - foreach (var page in bucket.Pages) - { - if (page == null) continue; - page.Debug_Dump(detailed); - allocated += bucket.PageSize; - entries += (uint)page.Count; - used += page.MemoryUsage; - } - } - } - Debug.WriteLine("# Found a total of {0:N0} entries using {1:N0} bytes out of {2:N0} bytes allocated", entries, used, allocated); -#endif - } - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Memory/EntryPage.cs b/FoundationDB.Storage.Memory/Core/Memory/EntryPage.cs deleted file mode 100644 index b68edc69e..000000000 --- a/FoundationDB.Storage.Memory/Core/Memory/EntryPage.cs +++ /dev/null @@ -1,172 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - using System.Threading; - - /// Base implementation of a page of memory that can store items of the same type - [DebuggerDisplay("Start={m_start}, Current={m_current}, Entries={m_count}, Usage={(m_current-m_start)} / {m_capacity}")] - internal unsafe abstract class EntryPage : IDisposable - { - /// Pointer to the next free slot in the page - protected byte* m_current; - /// Pointer to the first byte of the page - protected byte* m_start; - /// Pointer to the next byte after the last byte of the page - protected byte* m_end; - /// Size of the page - protected uint m_capacity; - /// Number of entries stored in this page - protected int m_count; - /// Handle to the allocated memory - protected SafeHandle m_handle; - - protected EntryPage(SafeHandle handle, uint capacity) - { - Contract.Requires(handle != null && !handle.IsInvalid && !handle.IsClosed); - - m_handle = handle; - m_capacity = capacity; - m_start = (byte*) handle.DangerousGetHandle(); - m_end = m_start + capacity; - m_current = m_start; - CheckInvariants(); - } - - ~EntryPage() - { - Dispose(false); - } - - [Conditional("DEBUG")] - protected void CheckInvariants() - { - Contract.Assert(!m_handle.IsInvalid, "Memory handle should not be invalid"); - Contract.Assert(!m_handle.IsClosed, "Memory handle should not be closed"); - Contract.Ensures(Entry.IsAligned(m_current), "Current pointer should always be aligned"); - Contract.Assert(m_current <= m_start + m_capacity, "Current pointer should never be outside the page"); - } - - /// Number of entries store in this page - public int Count { get { return m_count; } } - - /// Number of bytes allocated inside this page - public ulong MemoryUsage { get { return (ulong)(m_current - m_start); } } - - /// Type of the entries stored in this page - public abstract EntryType Type { get; } - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - protected virtual void Dispose(bool disposing) - { - try - { - if (disposing) - { - var handle = m_handle; - if (handle != null && !handle.IsClosed) - { - m_handle.Close(); - GC.RemoveMemoryPressure(m_capacity); - } - } - } - finally - { - m_handle = null; - m_start = null; - m_current = null; - } - } - - private void ThrowDisposed() - { - throw new ObjectDisposedException(this.GetType().Name); - } - - /// Align a pointer in this page - /// Unaligned location in the page - /// Aligned pointer cannot be greater than or equal to this address - /// New pointer that is aligned, and is guaranteed to be less than the - internal static byte* Align(byte* ptr, byte* end) - { - long r = ((long)ptr) & (Entry.ALIGNMENT - 1); - if (r > 0) ptr += Entry.ALIGNMENT - r; - if (ptr > end) return end; - return ptr; - } - - /// Try to allocate a segment in this page - /// Minimum size of the segment - /// Pointer to the start of the allocated segment, or null if this page cannot satisfy the allocation - /// The pointer will be aligned before being returned. The method may return null even if there was enough space remaining, if the aligment padding causes the segment to overshoot the end of the page. - protected byte* TryAllocate(uint size) - { - // try to allocate an amount of memory - // - returns null if the page is full, or too small - // - returns a pointer to the allocated space - - byte* ptr = m_current; - if (ptr == null) ThrowDisposed(); - byte* end = m_end; - byte* next = ptr + size; - if (next > m_end) - { // does not fit in this page - return null; - } - - // update the cursor for the next value - next = (byte*) (((long)next + Entry.ALIGNMENT - 1) & Entry.ALIGNMENT_MASK); - if (next > end) next = end; - m_current = next; - ++m_count; - - CheckInvariants(); - return ptr; - } - - /// Update this instance to use another memory location, and release the previously allocated memory - /// Page that will be absorbed - /// The content of the current page will be deleted, and will be disposed - public void Swap(EntryPage target) - { - if (target == null) throw new ArgumentNullException("target"); - Contract.Requires(target.m_handle != null); - - if (m_current == null) ThrowDisposed(); - if (target.m_current == null) target.ThrowDisposed(); - - try - { } - finally - { - var old = m_handle; - m_handle = Interlocked.Exchange(ref target.m_handle, null); - m_count = target.m_count; - m_capacity = target.m_capacity; - m_start = target.m_start; - m_end = target.m_end; - m_current = target.m_current; - - old.Dispose(); - target.Dispose(); - } - CheckInvariants(); - } - - [Conditional("DEBUG")] - public abstract void Debug_Dump(bool detailed); - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Memory/KeyHeap.cs b/FoundationDB.Storage.Memory/Core/Memory/KeyHeap.cs deleted file mode 100644 index beff5aabe..000000000 --- a/FoundationDB.Storage.Memory/Core/Memory/KeyHeap.cs +++ /dev/null @@ -1,235 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Client; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - internal unsafe class KeyHeap : ElasticHeap - { - - // Some facts about keys: - // - The overhead per key is 16 bytes on x64 - // - The maximum allowed size for a key is 10,000 bytes. - // - Well designed layers will tend to use small keys. - // - Text-based indexes may need longer keys. - // - Very large keys should be rare and will already be slow due to longer memcmps anyway. - // - The smallest possible entry will be 16 bytes (empty key) which can occur only once per database - // - A typical small key "(42, small_int)" will be ~24 bytes - // - Page size of 4 KB can fit 170 keys with waste ~ 0.4% - // - Page size of 16 KB can fit 682 keys with waste ~ 0.1% - // - Page size of 64 KB can fit 2730 keys with waste negligible - // - A typical index composite key "(42, GUID, TimeStamp, int16)" will be ~48 bytes - // - Page size of 4 KB can fit 85 keys with waste ~ 0.4% - // - Page size of 16 KB can fit 341 keys with waste ~ 0.1% - // - Page size of 64 KB can fit 1365 keys with waste negligible - // - A somewhat longer key "(42, 1, GUID, GUID, TimeStamp, int16)" will be ~64 bytes - // - Page size of 4 KB can fit 64 keys with no waste - // - Page size of 16 KB can fit 256 keys with no waste - // - Page size of 64 KB can fit 1024 keys with no waste - // - A "big" key will be ~1000 bytes and should be pretty rare (either very specific scenario, or badly designed Layer) - // - Page size of 4 KB can fit 4 keys with waste ~ 2.3% - // - Page size of 16 KB can fit 16 keys with waste ~ 2.3% - // - Page size of 64 KB can fit 64 keys with waste ~ 0.8% - // - Page size of 128 KB can fit 128 keys with waste negligible - // - The largest possible entry size is 10,016 bytes and should never happen in well designed Layers - // - Page size smaller than 16KB are not possible (too small) - // - Page size of 16 KB can fit 1 key with a waste of 6368 bytes (38.8%) - // - Page size of 32 KB can fit 3 keys with a waste of 2720 bytes ( 8.3%) - // - Page size of 64 KB can fit 6 keys with a waste of 5440 bytes ( 8.3%) - // - Page size of 128 KB can fit 13 keys with a waste of 864 bytes ( 0.6%) - // - Page size of 256 KB can fit 26 keys with a waste of 1728 bytes ( 0.6%) - // - Page size of 1 MB can fit 104 keys with a waste of 6912 bytes ( 0.6%) - - // We should probably optimize for keys up to ~100 bytes, and try our best for longer keys. - // => We will use 4 buckets for the pages, and try to have at least 256 entries per page - // - SMALL : keys up to 64 bytes, with page size of 16 KB - // - MEDIUM: keys up to 256 bytes, with page size of 64 KB - // - LARGE : keys up to 1,024 bytes, with page size of 256 KB - // - HUGE : keys up to 10,016 bytes, with page size of 1 MB (fit up to 104 entries) - - /// Page of memory used to store Keys - public sealed unsafe class Page : EntryPage - { - - public Page(SafeHandle handle, uint capacity) - : base(handle, capacity) - { } - - public override EntryType Type - { - get { return EntryType.Key; } - } - - /// Copy an existing value to this page, and return the pointer to the copy - /// Value that must be copied to this page - /// Pointer to the copy in this page - public Key* TryAppend(Key* value) - { - Contract.Requires(value != null && Entry.GetObjectType(value) == EntryType.Value); - - uint rawSize = Key.SizeOf + value->Size; - var entry = (Key*)TryAllocate(rawSize); - if (entry == null) return null; // this page is full - - UnmanagedHelpers.CopyUnsafe((byte*)entry, (byte*)value, rawSize); - - return entry; - } - - public Key* TryAppend(USlice buffer) - { - Contract.Requires(buffer.Data != null - && buffer.Count >= Key.SizeOf - && ((Key*)buffer.Data)->Size == buffer.Count - Key.SizeOf); - - var entry = (Key*)TryAllocate(buffer.Count); - if (entry == null) return null; // this page is full - - UnmanagedHelpers.CopyUnsafe((byte*)entry, buffer.Data, buffer.Count); - entry->Header = ((ushort)EntryType.Key) << Entry.TYPE_SHIFT; - - return entry; - } - - public void Collect(KeyHeap.Page target, ulong sequence) - { - var current = (Key*)m_start; - var end = (Key*)m_current; - - while (current < end) - { - bool keep = Key.StillAlive(current, sequence); - - if (keep) - { // copy to the target page - - var moved = target.TryAppend(current); - if (moved == null) throw new InvalidOperationException("The target page was too small"); - - var values = current->Values; - if (values != null) - { - values->Parent = moved; - } - - current->Header |= Entry.FLAGS_MOVED | Entry.FLAGS_DISPOSED; - } - else - { - current->Header |= Entry.FLAGS_DISPOSED; - } - - current = Key.WalkNext(current); - } - - - } - - public override void Debug_Dump(bool detailed) - { - Contract.Requires(m_start != null && m_current != null); - Key* current = (Key*)m_start; - Key* end = (Key*)m_current; - - Trace.WriteLine(" # KeyPage: count=" + m_count.ToString("N0") + ", used=" + this.MemoryUsage.ToString("N0") + ", capacity=" + m_capacity.ToString("N0") + ", start=0x" + new IntPtr(m_start).ToString("X8") + ", end=0x" + new IntPtr(m_current).ToString("X8")); - if (detailed) - { - while (current < end) - { - Trace.WriteLine(" - [" + Entry.GetObjectType(current).ToString() + "] 0x" + new IntPtr(current).ToString("X8") + " : " + current->Header.ToString("X8") + ", size=" + current->Size + ", h=0x" + current->HashCode.ToString("X4") + " : " + FdbKey.Dump(Key.GetData(current).ToSlice())); - var value = current->Values; - while (value != null) - { - Trace.WriteLine(" -> [" + Entry.GetObjectType(value) + "] 0x" + new IntPtr(value).ToString("X8") + " @ " + value->Sequence + " : " + Value.GetData(value).ToSlice().ToAsciiOrHexaString()); - value = value->Previous; - } - current = Key.WalkNext(current); - } - } - } - } - - private const int NUM_BUCKETS = 4; - private const uint SMALL_KEYS = 64; - private const uint MEDIUM_KEYS = 256; - private const uint LARGE_KEYS = 1024; - private const uint HUGE_KEYS = uint.MaxValue; // should nether be larger than 10,016 bytes - - private static readonly uint[] KeySizes = new uint[NUM_BUCKETS] { - SMALL_KEYS, - MEDIUM_KEYS, - LARGE_KEYS, - HUGE_KEYS - }; - - private static readonly uint[] PageSizes = new uint[NUM_BUCKETS] - { - /* SMALL */ 16 * 1024, - /* MEDIUM */ 64 * 1024, - /* LARGE */ 256 * 1024, - /* HUGE */ 1024 * 1024 - }; - - public KeyHeap() - : base(PageSizes, (handle, size) => new KeyHeap.Page(handle, size)) - { } - - private static int GetBucket(uint size) - { - if (size <= SMALL_KEYS) return 0; - if (size <= MEDIUM_KEYS) return 1; - if (size <= LARGE_KEYS) return 2; - return 3; - } - - public Key* Append(USlice buffer) - { - int bucket = GetBucket(buffer.Count + Key.SizeOf); - - var page = m_currents[bucket]; - var entry = page != null ? page.TryAppend(buffer) : null; - if (entry == null) - { // allocate a new page and try again - entry = AppendSlow(bucket, buffer); - } - return entry; - } - - private Key* AppendSlow(int bucket, USlice buffer) - { - var page = CreateNewPage(bucket); - Contract.Assert(page != null); - m_currents[bucket] = page; - m_buckets[bucket].Pages.Add(page); - - var entry = page.TryAppend(buffer); - if (entry == null) throw new OutOfMemoryException(String.Format("Failed to allocate memory from the key heap ({0})", m_buckets[bucket].PageSize)); - return entry; - } - - public void Collect(ulong sequence) - { - for (int bucket = 0; bucket < m_buckets.Length; bucket++) - { - if (m_buckets[bucket].Pages.Count > 0) - { - //TODO:!!! - //- allocate a scratch page - //- for all pages in bucket that have more than x% of free space - // - copy as many surviving keys into scratch page - // - if scratch page is too small, add it to the list, allocate new scratch page (note: from the free list?) - // - put page into "free list" - } - } - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Memory/ValueHeap.cs b/FoundationDB.Storage.Memory/Core/Memory/ValueHeap.cs deleted file mode 100644 index f16eb971a..000000000 --- a/FoundationDB.Storage.Memory/Core/Memory/ValueHeap.cs +++ /dev/null @@ -1,310 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - internal unsafe class ValueHeap : ElasticHeap - { - - // Some facts about values: - // - The overhead per value is 32 bytes on x64 - // - The largest possible value size is 100,000 bytes - // - A lot of layers (indexes, ...) use empty keys which could be optimized away and not take any space. - // - Document layers that split documents into a field per key will use values from 1 or 2 bytes (bool, ints) to ~64 bytes (strings, labels, text GUIDs, ...) - // - Some layers may pack complete documents in keys, or pack arrays, which will occupy a couple of KB - // - Blob-type layers will need to split very large documents (files, pictures, logs, ...) into as few chunks as possible, and be pegged at 10,000 or 100,000 bytes - // - A typical small value is an 32-bit or 64-bit integer or counter, which will be padded to 40 bytes - // - Page size of 4 KB can fit 170 keys with waste ~ 0.4% - // - Page size of 16 KB can fit 682 keys with waste ~ 0.1% - // - Page size of 64 KB can fit 2730 keys with waste negligible - // - A GUID will be 48 bytes - // - Page size of 4 KB can fit 85 keys with waste ~ 0.4% - // - Page size of 16 KB can fit 341 keys with waste ~ 0.1% - // - Page size of 64 KB can fit 1365 keys with waste negligible - // - A very small JSON doc {Id:"..",Value:"...",Tag:".."} will be less than ~128 bytes - // - Page size of 4 KB can fit 85 keys with waste ~ 0.4% - // - An array of 60 doubles will be 512 bytes - // - Page size of 4 KB can fit 85 keys with waste ~ 0.4% - // - A "small" chunk of a blob layer will be ~16K - // - Page size of 16 KB can fit 1 key with no waste - // - Page size of 64 KB can fit 64 keys with waste ~ 0.8% - // - Page size of 128 KB can fit 128 keys with waste negligible - // - The largest possible key is 10,032 bytes (header + pointer + 10,000 bytes) and should never happen in well designed Layers - // - Page size smaller than 16KB are not possible (too small) - // - Page size of 16 KB can fit 1 key with a waste of 6368 bytes (38.8%) - // - Page size of 32 KB can fit 3 keys with a waste of 2720 bytes ( 8.3%) - // - Page size of 64 KB can fit 6 keys with a waste of 5440 bytes ( 8.3%) - // - Page size of 128 KB can fit 13 keys with a waste of 864 bytes ( 0.6%) - // - Page size of 256 KB can fit 26 keys with a waste of 1728 bytes ( 0.6%) - // - Page size of 1 MB can fit 104 keys with a waste of 6912 bytes ( 0.6%) - - // pb: layers wanting to target a size that is a power of two (1K, 2K, 16K, ...) will always be misaligned due to the 32 bytes overhead and may create waste in pages (especially small pages) - - // We should probably optimize for keys up to ~100 bytes, and try our best for longer keys. - // => We will use 4 buckets for the pages, and try to have at least 256 entries per page - // - SMALL : keys up to 64 bytes, with page size of 16 KB - // - MEDIUM: keys up to 256 bytes, with page size of 64 KB - // - LARGE : keys up to 1,024 bytes, with page size of 256 KB - // - HUGE : keys up to 10,016 bytes, with page size of 1 MB (fit up to 104 entries) - - /// Page of memory used to store Values - public sealed class Page : EntryPage - { - - public Page(SafeHandle handle, uint capacity) - : base(handle, capacity) - { } - - public override EntryType Type - { - get { return EntryType.Value; } - } - - /// Copy an existing value to this page, and return the pointer to the copy - /// Value that must be copied to this page - /// Pointer to the copy in this page - public Value* TryAppend(Value* value) - { - Contract.Requires(value != null && Entry.GetObjectType(value) == EntryType.Value); - - uint rawSize = Value.SizeOf + value->Size; - Value* entry = (Value*)TryAllocate(rawSize); - if (entry == null) return null; // the page is full - - UnmanagedHelpers.CopyUnsafe((byte*)entry, (byte*)value, rawSize); - - return entry; - } - - public Value* TryAppend(USlice buffer) - { - Contract.Requires(buffer.Data != null - && buffer.Count >= Value.SizeOf - && ((Key*)buffer.Data)->Size == buffer.Count - Value.SizeOf); - - var entry = (Value*)TryAllocate(buffer.Count); - if (entry == null) return null; // the page is full - UnmanagedHelpers.CopyUnsafe((byte*)entry, buffer.Data, buffer.Count); - - return entry; - } - - public Value* TryAllocate(uint dataSize, ulong sequence, Value* previous, void* parent) - { - Value* entry = (Value*)TryAllocate(Value.SizeOf + dataSize); - if (entry == null) return null; // the page is full - - entry->Header = ((ushort)EntryType.Value) << Entry.TYPE_SHIFT; - entry->Size = dataSize; - entry->Sequence = sequence; - entry->Previous = previous; - entry->Parent = parent; - - return entry; - } - - public void Collect(Page target, ulong sequence) - { - var current = (Value*)m_start; - var end = (Value*)m_current; - - while (current < end) - { - bool keep = Value.StillAlive(current, sequence); - - void* parent = current->Parent; - - if (keep) - { // copy to the target page - - var moved = target.TryAppend(current); - if (moved == null) throw new InvalidOperationException(); // ?? - - // update the parent - switch (Entry.GetObjectType(parent)) - { - case EntryType.Key: - { - ((Key*)parent)->Values = moved; - break; - } - case EntryType.Value: - { - ((Value*)parent)->Previous = moved; - break; - } - case EntryType.Free: - { - //NO-OP - break; - } - default: - { - throw new InvalidOperationException("Unexpected parent while moving value"); - } - } - current->Header |= Entry.FLAGS_MOVED | Entry.FLAGS_DISPOSED; - } - else - { - // we need to kill the link from the parent - switch (Entry.GetObjectType(parent)) - { - case EntryType.Key: - { - ((Key*)parent)->Values = null; - break; - } - case EntryType.Value: - { - ((Value*)parent)->Previous = null; - break; - } - case EntryType.Free: - { - //NO-OP - break; - } - default: - { - throw new InvalidOperationException("Unexpected parent while destroying value"); - } - } - - current->Header |= Entry.FLAGS_DISPOSED; - } - - current = Value.WalkNext(current); - } - } - - public override void Debug_Dump(bool detailed) - { - Contract.Requires(m_start != null && m_current != null); - Value* current = (Value*)m_start; - Value* end = (Value*)m_current; - - Trace.WriteLine(" # ValuePage: count=" + m_count.ToString("N0") + ", used=" + this.MemoryUsage.ToString("N0") + ", capacity=" + m_capacity.ToString("N0") + ", start=0x" + new IntPtr(m_start).ToString("X8") + ", end=0x" + new IntPtr(m_current).ToString("X8")); - if (detailed) - { - while (current < end) - { - Trace.WriteLine(" - [" + Entry.GetObjectType(current).ToString() + "] 0x" + new IntPtr(current).ToString("X8") + " : " + current->Header.ToString("X8") + ", seq=" + current->Sequence + ", size=" + current->Size + " : " + Value.GetData(current).ToSlice().ToAsciiOrHexaString()); - if (current->Previous != null) Trace.WriteLine(" -> Previous: [" + Entry.GetObjectType(current->Previous) + "] 0x" + new IntPtr(current->Previous).ToString("X8")); - if (current->Parent != null) Trace.WriteLine(" <- Parent: [" + Entry.GetObjectType(current->Parent) + "] 0x" + new IntPtr(current->Parent).ToString("X8")); - - current = Value.WalkNext(current); - } - } - } - - } - - private const int NUM_BUCKETS = 5; - - //note: we try to target more than 100 entries per page to reduce overhead and possible waste - - private const int TINY_VALUES = 16 + 32; // note (GUIDs or smaller) - private const uint SMALL_VALUES = 128 + 32; // a tiny JSON doc should fit without problem - private const uint MEDIUM_VALUES = 60 * 8 + 32; // an array of 60 doubles - private const uint LARGE_VALUES = 4096 + 32; // a small size JSON doc (possibly compressed) - private const uint HUGE_VALUES = uint.MaxValue; // > 2KB would be "large documents", chunks of very large documents, or binary blobs - - private static readonly uint[] KeySizes = new uint[NUM_BUCKETS] { - TINY_VALUES, - SMALL_VALUES, - MEDIUM_VALUES, - LARGE_VALUES, - HUGE_VALUES - }; - - private static readonly uint[] PageSizes = new uint[NUM_BUCKETS] - { - /* TINY */ 16 * 1024, // from 341 to 512 per page - /* SMALL */ 64 * 1024, // from 409 to 1337 per page - /* MEDIUM */ 128 * 1024, // from 256 to 814 per page - /* LARGE */ 256 * 1024, // from 63 to 511 per page - /* HUGE */ 1024 * 1024, // from 10 to 253 per page - }; - - public ValueHeap() - : base(PageSizes, (handle, size) => new ValueHeap.Page(handle, size)) - { } - - private static int GetBucket(uint size) - { - if (size <= TINY_VALUES) return 0; - if (size <= SMALL_VALUES) return 1; - if (size <= MEDIUM_VALUES) return 2; - if (size <= LARGE_VALUES) return 3; - return 4; - } - - public Value* Allocate(uint dataSize, ulong sequence, Value* previous, void* parent) - { - int bucket = GetBucket(dataSize + Value.SizeOf); - - var page = m_currents[bucket]; - var entry = page != null ? page.TryAllocate(dataSize, sequence, previous, parent) : null; - if (entry == null) - { - entry = AllocateSlow(bucket, dataSize, sequence, previous, parent); - } - return entry; - } - - private Value* AllocateSlow(int bucket, uint dataSize, ulong sequence, Value* previous, void* parent) - { - var page = CreateNewPage(bucket); - Contract.Assert(page != null); - m_currents[bucket] = page; - m_buckets[bucket].Pages.Add(page); - - var entry = page.TryAllocate(dataSize, sequence, previous, parent); - if (entry == null) throw new OutOfMemoryException(String.Format("Failed to allocate memory from the the value heap ({0})", m_buckets[bucket].PageSize)); - return entry; - } - - public void Collect(ulong sequence) - { - for (int bucket = 0; bucket < m_buckets.Length; bucket++) - { - if (m_buckets[bucket].Pages.Count > 0) - { - //TODO:!!! - //- allocate a scratch page - //- for all pages in bucket that have more than x% of free space - // - copy as many surviving keys into scratch page - // - if scratch page is too small, add it to the list, allocate new scratch page (note: from the free list?) - // - put page into "free list" - } - } - -#if REFACTORED - foreach (var page in m_pages) - { - var target = CreateNewPage(m_pageSize, Entry.ALIGNMENT); - if (page.Count == 1) - { // this is a standalone page - page.Collect(target, sequence); - page.Swap(target); - } - else - { - page.Collect(target, sequence); - page.Swap(target); - } - } -#endif - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Core/NativeKeyComparer.cs b/FoundationDB.Storage.Memory/Core/NativeKeyComparer.cs deleted file mode 100644 index 941a78cc4..000000000 --- a/FoundationDB.Storage.Memory/Core/NativeKeyComparer.cs +++ /dev/null @@ -1,109 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -//#define INSTRUMENT - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - - internal unsafe sealed class NativeKeyComparer : IComparer, IEqualityComparer - { - - public int Compare(IntPtr left, IntPtr right) - { -#if INSTRUMENT - System.Threading.Interlocked.Increment(ref s_compareCalls); -#endif - // this method will be called A LOT, so it should be as fast as possible... - // We know that: - // - caller should never compare nulls (it's a bug) - // - empty keys can exist - // - number of calls with left == right will be very small so may not be worth it to optimize (will slow down everything else) - // - for db using the DirectoryLayer, almost all keys will start with 0x15 (prefix for an int in a tuple) so checking the first couple of bytes will not help much (long runs of keys starting with the same 2 or 3 bytes) - Contract.Assert(left != IntPtr.Zero && right != IntPtr.Zero); - - // unwrap as pointers to the Key struct - var leftKey = (Key*)left; - var rightKey = (Key*)right; - - // these will probably cause a cache miss - uint leftCount = leftKey->Size; - uint rightCount = rightKey->Size; - - // but then memcmp will probably have the data in the cpu cache... - int c = UnmanagedHelpers.NativeMethods.memcmp( - &(leftKey->Data), - &(rightKey->Data), - new UIntPtr(leftCount < rightCount ? leftCount : rightCount) - ); - return c != 0 ? c : (int)leftCount - (int)rightCount; - } - - public bool Equals(IntPtr left, IntPtr right) - { -#if INSTRUMENT - System.Threading.Interlocked.Increment(ref s_equalsCalls); -#endif - // unwrap as pointers to the Key struct - var leftKey = (Key*)left; - var rightKey = (Key*)right; - - if (leftKey->HashCode != rightKey->HashCode) - { - return false; - } - - uint leftCount, rightCount; - - if (leftKey == null || (leftCount = leftKey->Size) == 0) return rightKey == null || rightKey->Size == 0; - if (rightKey == null || (rightCount = rightKey->Size) == 0) return false; - - return leftCount == rightCount && 0 == UnmanagedHelpers.NativeMethods.memcmp(&(leftKey->Data), &(rightKey->Data), new UIntPtr(leftCount)); - } - - public int GetHashCode(IntPtr value) - { -#if INSTRUMENT - System.Threading.Interlocked.Increment(ref s_getHashCodeCalls); -#endif - var key = (Key*)value; - if (key == null) return -1; - return key->HashCode; - } - -#if INSTRUMENT - private static long s_compareCalls; - private static long s_equalsCalls; - private static long s_getHashCodeCalls; -#endif - - public static void GetCounters(out long compare, out long equals, out long getHashCode) - { -#if INSTRUMENT - compare = System.Threading.Interlocked.Read(ref s_compareCalls); - equals = System.Threading.Interlocked.Read(ref s_equalsCalls); - getHashCode = System.Threading.Interlocked.Read(ref s_getHashCodeCalls); -#else - compare = 0; - equals = 0; - getHashCode = 0; -#endif - } - - public static void ResetCounters() - { -#if INSTRUMENT - System.Threading.Interlocked.Exchange(ref s_compareCalls, 0); - System.Threading.Interlocked.Exchange(ref s_equalsCalls, 0); - System.Threading.Interlocked.Exchange(ref s_getHashCodeCalls, 0); -#endif - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Core/SequenceComparer.cs b/FoundationDB.Storage.Memory/Core/SequenceComparer.cs deleted file mode 100644 index a1a931407..000000000 --- a/FoundationDB.Storage.Memory/Core/SequenceComparer.cs +++ /dev/null @@ -1,35 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using System; - using System.Collections.Generic; - - internal sealed class SequenceComparer : IComparer, IEqualityComparer - { - public static readonly SequenceComparer Default = new SequenceComparer(); - - private SequenceComparer() - { } - - public int Compare(ulong x, ulong y) - { - if (x < y) return -1; - if (x > y) return +1; - return 0; - } - - public bool Equals(ulong x, ulong y) - { - return x == y; - } - - public int GetHashCode(ulong x) - { - return (((int)x) ^ ((int)(x >> 32))); - } - } - -} diff --git a/FoundationDB.Storage.Memory/Core/TransactionWindow.cs b/FoundationDB.Storage.Memory/Core/TransactionWindow.cs deleted file mode 100644 index 237e69998..000000000 --- a/FoundationDB.Storage.Memory/Core/TransactionWindow.cs +++ /dev/null @@ -1,169 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Client; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Diagnostics; - using System.Diagnostics.Contracts; - - [DebuggerDisplay("Sarted={m_startedUtc}, Min={m_minVersion}, Max={m_maxVersion}, Closed={m_closed}, Disposed={m_disposed}")] - internal sealed class TransactionWindow : IDisposable - { - /// Creation date of this transaction window - private readonly DateTime m_startedUtc; - /// First commit version for this transaction window - private readonly ulong m_minVersion; - /// Sequence of the last commited transaction from this window - private ulong m_maxVersion; - /// Counter for committed write transactions - private int m_commitCount; - /// If true, the transaction is closed (no more transaction can write to it) - private bool m_closed; - /// If true, the transaction has been disposed - private volatile bool m_disposed; - - /// Heap used to store the write conflict keys - private readonly UnmanagedMemoryHeap m_keys = new UnmanagedMemoryHeap(65536); - - /// List of all the writes made by transactions committed in this window - private readonly ColaRangeDictionary m_writeConflicts = new ColaRangeDictionary(USliceComparer.Default, SequenceComparer.Default); - - public TransactionWindow(DateTime startedUtc, ulong version) - { - m_startedUtc = startedUtc; - m_minVersion = version; - } - - public bool Closed { get { return m_closed; } } - - public ulong FirstVersion { get { return m_minVersion; } } - - public ulong LastVersion { get { return m_maxVersion; } } - - public DateTime StartedUtc { get { return m_startedUtc; } } - - /// Number of write transaction that committed during this window - public int CommitCount { get { return m_commitCount; } } - - public ColaRangeDictionary Writes { get { return m_writeConflicts; } } - - public void Close() - { - Contract.Requires(!m_closed && !m_disposed); - - if (m_disposed) ThrowDisposed(); - - m_closed = true; - } - - private unsafe USlice Store(Slice data) - { - uint size = checked((uint)data.Count); - var buffer = m_keys.AllocateAligned(size); - UnmanagedHelpers.CopyUnsafe(buffer, data); - return new USlice(buffer, size); - } - - public void MergeWrites(ColaRangeSet writes, ulong version) - { - Contract.Requires(!m_closed && writes != null && version >= m_minVersion && (!m_closed || version <= m_maxVersion)); - - if (m_disposed) ThrowDisposed(); - if (m_closed) throw new InvalidOperationException("This transaction has already been closed"); - - //Debug.WriteLine("* Merging writes conflicts for version " + version + ": " + String.Join(", ", writes)); - - foreach (var range in writes) - { - var begin = range.Begin; - var end = range.End; - - USlice beginKey, endKey; - if (begin.Offset == end.Offset && object.ReferenceEquals(begin.Array, end.Array) && end.Count >= begin.Count) - { // overlapping keys - endKey = Store(end); - beginKey = endKey.Substring(0, (uint)begin.Count); - } - else - { - beginKey = Store(begin); - endKey = Store(end); - } - - m_writeConflicts.Mark(beginKey, endKey, version); - } - - ++m_commitCount; - if (version > m_maxVersion) - { - m_maxVersion = version; - } - } - - /// Checks if a list of reads conflicts with at least one write performed in this transaction window - /// List of reads to check for conflicts - /// Sequence number of the transaction that performed the reads - /// True if at least one read is conflicting with a write with a higher sequence number; otherwise, false. - public bool Conflicts(ColaRangeSet reads, ulong version) - { - Contract.Requires(reads != null); - - //Debug.WriteLine("* Testing for conflicts for: " + String.Join(", ", reads)); - - if (version > m_maxVersion) - { // all the writes are before the reads, so no possible conflict! - //Debug.WriteLine(" > cannot conflict"); - return false; - } - - using (var scratch = new UnmanagedSliceBuilder()) - { - //TODO: do a single-pass version of intersection checking ! - foreach (var read in reads) - { - scratch.Clear(); - scratch.Append(read.Begin); - var p = scratch.Count; - scratch.Append(read.End); - var begin = scratch.ToUSlice(p); - var end = scratch.ToUSlice(p, scratch.Count - p); - - if (m_writeConflicts.Intersect(begin, end, version, (v, min) => v > min)) - { - Debug.WriteLine(" > Conflicting read: " + read); - return true; - } - } - } - - //Debug.WriteLine(" > No conflicts found"); - return false; - } - - private void ThrowDisposed() - { - throw new ObjectDisposedException(this.GetType().Name); - } - - public void Dispose() - { - if (!m_disposed) - { - m_disposed = true; - m_keys.Dispose(); - - } - GC.SuppressFinalize(this); - } - - public override string ToString() - { - return String.Format(System.Globalization.CultureInfo.InvariantCulture, "#{0} [{1}~{2}]", m_startedUtc.Ticks / TimeSpan.TicksPerMillisecond, m_minVersion, m_maxVersion); - } - } - -} diff --git a/FoundationDB.Storage.Memory/Core/Value.cs b/FoundationDB.Storage.Memory/Core/Value.cs deleted file mode 100644 index c24b38dfa..000000000 --- a/FoundationDB.Storage.Memory/Core/Value.cs +++ /dev/null @@ -1,91 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Core -{ - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - [StructLayout(LayoutKind.Sequential, Pack = 1)] - internal unsafe struct Value - { - - // A Value contains the pointer to the key's bytes, and a pointer to the most current Value for this key, or null if the key is currently deleted - - // Field Offset Bits Type Desc - // HEADER 0 16 flags Type, status flags, deletion or mutation flags, .... - // reserved 2 16 uint16 unused - // SIZE 4 32 uint Size of the DATA field (can be 0, should only use 24 bits at most) - // SEQUENCE 8 64 ulong Sequence version of this value - // PREVIOUS 16 64 Value* Pointer to the previous value that was supersed by this entry (or null if we are the oldest one in the chain) - // PARENT 24 64 void* Pointer to the parent of this value - // DATA 32 .. byte[] First byte of the key - - // The HEADER flags are as follow: - // - bit 0: DELETION If set, this value is a deletion marker (and its size must be zero) - // - bit 1: MUTATED If set, this value is not the last one for this key - // - bit 2-5: unused - // - bit 7: HAS_WATCH If set, this key is currently being watched - // - bit 8-15: ENTRY_FLAGS (inherited from Entry) - // - bit 8-15: ENTRY_FLAGS (inherited from Entry) - - public static readonly uint SizeOf = (uint)Marshal.OffsetOf(typeof(Value), "Data").ToInt32(); - - /// This value is a deletion marker - public const ushort FLAGS_DELETION = 1 << 0; - - /// This value has been mutated and is not up to date - public const ushort FLAGS_MUTATED = 1 << 1; - - /// Various flags (TDB) - public ushort Header; - /// Not used - public uint Reseved; - /// Size of the value - public uint Size; - /// Version where this version of the key first appeared - public ulong Sequence; - /// Pointer to the previous version of this key, or NULL if this is the earliest known - public Value* Previous; - /// Pointer to the parent node (can be a Key or a Value) - public void* Parent; - /// Offset to the first byte of the value - public byte Data; - - public static USlice GetData(Value* value) - { - if (value == null) return default(USlice); - - Contract.Assert((value->Header & Entry.FLAGS_DISPOSED) == 0, "Attempt to read a value that was disposed"); - return new USlice(&(value->Data), value->Size); - } - - public static bool StillAlive(Value* value, ulong sequence) - { - if (value == null) return false; - if ((value->Header & Value.FLAGS_MUTATED) != 0) - { - return value->Sequence >= sequence; - } - return true; - } - - public static bool IsDisposed(Value* value) - { - return (value->Header & Entry.FLAGS_DISPOSED) != 0; - } - - /// Return the address of the following value in the heap - internal static Value* WalkNext(Value* self) - { - Contract.Requires(self != null && Entry.GetObjectType(self) == EntryType.Value); - - return (Value*)Entry.Align((byte*)self + Value.SizeOf + self->Size); - } - - } - -} diff --git a/FoundationDB.Storage.Memory/FoundationDB.Storage.Memory.csproj b/FoundationDB.Storage.Memory/FoundationDB.Storage.Memory.csproj deleted file mode 100644 index e0ff07c0c..000000000 --- a/FoundationDB.Storage.Memory/FoundationDB.Storage.Memory.csproj +++ /dev/null @@ -1,109 +0,0 @@ - - - - - Debug - AnyCPU - {CC98DB39-31A1-4642-B4FC-9CB0AB26BF2E} - Library - Properties - FoundationDB.Storage.Memory - FoundationDB.Storage.Memory - v4.6.1 - 512 - ..\ - true - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - true - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - true - - - true - - - FoundationDB.Storage.Memory.snk - - - - - ..\..\W2M\packages\System.Collections.Immutable.1.1.36\lib\portable-net45+win8+wp8+wpa81\System.Collections.Immutable.dll - True - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - - - \ No newline at end of file diff --git a/FoundationDB.Storage.Memory/FoundationDB.Storage.Memory.snk b/FoundationDB.Storage.Memory/FoundationDB.Storage.Memory.snk deleted file mode 100644 index 5b29927c38aee32e67ec2db3d5d68f043df22760..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 596 zcmV-a0;~N80ssI2Bme+XQ$aES1ONa50096a(jMgH;w8W3yi~$XKc;UN2!MO&h(*l4 z{;F~12H#SrtsZ-ofrw$WX1Sy~8c^LzkA=D~w>vx5^ zJpgJ^P4_H^l&Q8snw}BgzpBWZEltAt^ovl7DGAwRJU6sqAQb(#moW_Kqz+&H(i#8< zgj3dJVQXZ7Ns^D)dP=F2bb`|**`m%xJC!J}?E-rAf@v@z8)TGdkrb(f0Ohv{Q!`h6g9L@rWxK*We~t~@wKc33XA+1L1mfA?(52l6zD zj~_9~p+s${F>CsWhh*jqM5FR`yiVXJRD7G8Mx8AFPQ2cLXOt>qipY#Cn-cY>Ms6&P zXvJ|S1e-;n;}@RaeltzK{N$pobKw6y-Q^Wn3@Sdgx iaZbWS6hqdsl?h{)~F^Wfr note: the Jump Table must fit in a single page so could probably not be smaller than 512 ... - public const int PAGE_SIZE_BITS = 10; // 1KB - public const int PAGE_SIZE = 1 << PAGE_SIZE_BITS; - - public const uint HEADER_MAGIC_NUMBER = 0x42444E50; // "PNDB" - public const uint JUMP_TABLE_MAGIC_NUMBER = 0x54504D4A; // "JMPT" - public const uint LEVEL_MAGIC_NUMBER = 0x204C564C; // "LVL "; - - // Size of the header CRC (in bytes) - public const int HEADER_METADATA_BYTES = 64; - public const int HEADER_CRC_SIZE = 4; - public const int LEVEL_HEADER_BYTES = 16; - - // The maximum size for key + value is 10,000 + 100,000 with 2 + 3 additional bytes to encode the variable-length size - // The buffer size should be multiple of the pageSize value AND a power of two for convenience. - // Also, it would help if the buffer is x2 that to simplify buffering - // The worst case scenario would be where the first byte of the key starts on the last byte of a page, and last byte of the value cross into a new page, added 2 pages to the total - // Minimum size will be 2 + 10,000 + 3 + 100,000 + 2 * 1,024 = 112,053 and the next power of two is 2 ^ 17, so use 2 ^ 18 for double buffering - public const int MAX_KEYVALUE_BITS = 18; - public const int BUFFER_SIZE = 1 << MAX_KEYVALUE_BITS; - - public static uint ComputeChecksum(Slice data) - { - if (data.Offset < 0 || data.Count < 0 || (data.Array == null && data.Count > 0)) throw new ArgumentException("Data is invalid"); - - unsafe - { - fixed (byte* ptr = data.Array) - { - return ComputeChecksum(ptr + data.Offset, (ulong)data.Count); - } - } - } - - public static unsafe uint ComputeChecksum(byte* start, ulong count) - { - if (start == null && count != 0) throw new ArgumentException("Invalid address"); - - byte* ptr = start; - byte* end = checked(ptr + count); - - // : unoptimized 32 bits FNV-1a implementation - uint h = 2166136261; // FNV1 32 bits offset basis - while (ptr < end) - { - h = (h ^ *ptr) * 16777619; // FNV1 32 prime - ++ptr; - } - return h; - // - } - - } - -} diff --git a/FoundationDB.Storage.Memory/IO/SnapshotReader.cs b/FoundationDB.Storage.Memory/IO/SnapshotReader.cs deleted file mode 100644 index be450ed3c..000000000 --- a/FoundationDB.Storage.Memory/IO/SnapshotReader.cs +++ /dev/null @@ -1,353 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.IO -{ - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Storage.Memory.API; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - using System.Threading; - using System.Threading.Tasks; - - internal unsafe sealed class SnapshotReader - { - private struct LevelAddress - { - public ulong Offset; - public ulong Size; - public ulong PaddedSize; - } - - private readonly Win32MemoryMappedFile m_file; - - private bool m_hasHeader; - private bool m_hasJumpTable; - private LevelAddress[] m_jumpTable; - - private Version m_version; - private SnapshotFormat.Flags m_dbFlags; - private Uuid128 m_uid; - private ulong m_sequence; - private long m_itemCount; - private ulong m_timestamp; - private uint m_headerChecksum; - private Dictionary m_attributes; - - private uint m_pageSize; - private uint m_headerSize; - - private ulong m_dataStart; - private ulong m_dataEnd; - - private int m_levels; - - public SnapshotReader(Win32MemoryMappedFile file) - { - Contract.Requires(file != null); //TODO: && file.CanRead ? - m_file = file; - } - - public int Depth - { - get { return m_levels; } - } - - public ulong Sequence { get { return m_sequence; } } - public ulong TimeStamp { get { return m_timestamp; } } - public Version Version { get { return m_version; } } - public Uuid128 Id { get { return m_uid; } } - - private Exception ParseError(string message) - { - message = "Database snapshot is invalid or corrupted: " + message; -#if DEBUG - if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); -#endif - return new InvalidOperationException(message); - } - - private Exception ParseError(string message, params object[] args) - { - return ParseError(String.Format(message, args)); - } - - private static uint RoundDown(uint size, uint pageSize) - { - return size & ~(pageSize - 1U); - } - - private static uint RoundUp(uint size, uint pageSize) - { - return checked(size + pageSize - 1U) & ~(pageSize - 1U); - } - - private static ulong RoundDown(ulong size, uint pageSize) - { - return size & ~((ulong)pageSize - 1UL); - } - - private static ulong RoundUp(ulong size, uint pageSize) - { - return checked(size + pageSize - 1UL) & ~((ulong)pageSize - 1UL); - } - - public void ReadHeader(CancellationToken ct) - { - ct.ThrowIfCancellationRequested(); - - // minimum header prolog size is 64 but most will only a single page - // we can preallocate a full page, and we will resize it later if needed - - var reader = m_file.CreateReader(0, SnapshotFormat.HEADER_METADATA_BYTES); - - // "PNDB" - var signature = reader.ReadFixed32(); - // v1.0 - uint major = reader.ReadFixed16(); - uint minor = reader.ReadFixed16(); - m_version = new Version((int)major, (int)minor); - // FLAGS - m_dbFlags = (SnapshotFormat.Flags) reader.ReadFixed64(); - // Database ID - m_uid = new Uuid128(reader.ReadBytes(16).GetBytes()); - // Database Version - m_sequence = reader.ReadFixed64(); - // Number of items in the database - m_itemCount = checked((long)reader.ReadFixed64()); - // Database Timestamp - m_timestamp = reader.ReadFixed64(); - // Page Size - m_pageSize = reader.ReadFixed32(); - // Header Size - m_headerSize = reader.ReadFixed32(); - - Contract.Assert(!reader.HasMore); - - #region Sanity checks - - // Signature - if (signature != SnapshotFormat.HEADER_MAGIC_NUMBER) throw ParseError("Invalid magic number"); - - // Version - if (m_version.Major != 1) throw ParseError("Unsupported file version (major)"); - if (m_version.Minor > 0) throw ParseError("Unsupported file version (minor)"); - - // Flags - - // Page Size - if (m_pageSize != UnmanagedHelpers.NextPowerOfTwo(m_pageSize)) throw ParseError("Page size ({0}) is not a power of two", m_pageSize); - if (m_pageSize < SnapshotFormat.HEADER_METADATA_BYTES) throw ParseError("Page size ({0}) is too small", m_pageSize); - if (m_pageSize > 1 << 20) throw ParseError("Page size ({0}) is too big", m_pageSize); - - // Header Size - if (m_headerSize < 64 + 4 + 4) throw ParseError("Header size ({0}) is too small", m_headerSize); - if (m_headerSize > m_file.Length) throw ParseError("Header size is bigger than the file itself ({0} < {1})", m_headerSize, m_file.Length); - if (m_headerSize > 1 << 10) throw ParseError("Header size ({0}) exceeds the maximum allowed size", m_headerSize); - - #endregion - - // we know the page size and header size, read the rest... - - // read the rest - reader = m_file.CreateReader(0, m_headerSize); - reader.Skip(SnapshotFormat.HEADER_METADATA_BYTES); - - // parse the attributes - Contract.Assert(reader.Offset == SnapshotFormat.HEADER_METADATA_BYTES); - var attributeCount = checked((int)reader.ReadFixed32()); - if (attributeCount < 0 || attributeCount > 1024) throw ParseError("Attributes count is invalid"); - - var attributes = new Dictionary(attributeCount); - for (int i = 0; i < attributeCount; i++) - { - var name = reader.ReadVarbytes().ToSlice(); //TODO: max size ? - if (name.IsNullOrEmpty) throw ParseError("Header attribute name is empty"); - - var data = reader.ReadVarbytes().ToSlice(); //TODO: max size + have a small scratch pad buffer for these ? - var value = FdbTuple.Unpack(data); - attributes.Add(name.ToUnicode(), value); - } - m_attributes = attributes; - - // read the header en marker - var marker = reader.ReadFixed32(); - if (marker != uint.MaxValue) throw ParseError("Header end marker is invalid"); - - // verify the header checksum - uint actualHeaderChecksum = SnapshotFormat.ComputeChecksum(reader.Base, reader.Offset); - uint headerChecksum = reader.ReadFixed32(); - m_headerChecksum = headerChecksum; - - if (headerChecksum != actualHeaderChecksum) - { - throw ParseError("The header checksum does not match ({0} != {1}). This may be an indication of data corruption", headerChecksum, actualHeaderChecksum); - } - - m_dataStart = RoundUp(m_headerSize, m_pageSize); - m_hasHeader = true; - } - - public bool HasLevel(int level) - { - return m_hasJumpTable && level >= 0 && level < m_jumpTable.Length && m_jumpTable[level].Size != 0; - } - - public void ReadJumpTable(CancellationToken ct) - { - ct.ThrowIfCancellationRequested(); - - if (!m_hasHeader) - { - throw new InvalidOperationException("Cannot read the Jump Table without reading the Header first!"); - } - - // an empty database will have at least 2 pages: the header and the JT - if (m_file.Length < checked(m_pageSize << 1)) - { - throw ParseError("File size ({0}) is too small to be a valid snapshot", m_file.Length); - } - - // the jumptable is always in the last page of the file and is expected to fit nicely - // > file size MUST be evenly divible by page size - // > then JT offset will be file.Length - pageSize - if (m_file.Length % m_pageSize != 0) - { - throw ParseError("The file size ({0}) is not a multiple of the page size ({1}), which may be a symptom of truncation", m_file.Length, m_pageSize); - } - - var jumpTableStart = m_file.Length - m_pageSize; - Contract.Assert(jumpTableStart % m_pageSize == 0); - m_dataEnd = jumpTableStart; - - var reader = m_file.CreateReader(jumpTableStart, m_pageSize); - - // "JMPT" - var signature = reader.ReadFixed32(); - // Page Size (repeated) - var pageSizeRepeated = (int)reader.ReadFixed32(); - // Sequence Number (repeated) - var sequenceRepeated = reader.ReadFixed64(); - // Database ID (repeated) - var uidRepeated = new Uuid128(reader.ReadBytes(16).GetBytes()); - // Header CRC (repeated) - var headerChecksumRepeated = reader.ReadFixed32(); - - // Sanity checks - - if (signature != SnapshotFormat.JUMP_TABLE_MAGIC_NUMBER) throw ParseError("Last page does not appear to be the Jump Table"); - if (pageSizeRepeated != m_pageSize) throw ParseError("Page size in Jump Table does not match the header value"); - if (sequenceRepeated != m_sequence) throw ParseError("Sequence in Jump Table does not match the header value"); - if (uidRepeated != m_uid) throw ParseError("Database ID in Jump Table does not match the header value"); - if (headerChecksumRepeated != m_headerChecksum) throw ParseError("Database ID in Jump Table does not match the header value"); - - // read the table itself - int levels = (int)reader.ReadFixed32(); - if (levels < 0 || levels > 32) throw ParseError("The number of levels in the snapshot does not appear to be valid"); - - var table = new LevelAddress[levels]; - for (int level = 0; level < levels; level++) - { - ulong offset = reader.ReadFixed64(); - ulong size = reader.ReadFixed64(); - - // Offset and Size cannot be negative - // Empty levels (size == 0) must have a zero offset - // Non empty levels (size > 0) must have a non zero offset that is greater than the headerSize - if ((size == 0 && offset != 0) || (size > 0 && offset < m_dataStart)) throw ParseError("Level in Jump Table has invalid size ({0}) or offset ({1})", size, offset); - if (checked(offset + size) > m_dataEnd) throw ParseError("Level in Jump Table would end after the end of the file"); - - table[level].Offset = offset; - table[level].Size = size; - table[level].PaddedSize = RoundUp(size, m_pageSize); - } - - // end attributes - uint attributeCount = reader.ReadFixed32(); - if (attributeCount != 0) throw new NotImplementedException("Footer attributes not yet implemented!"); - - // end marker - if (reader.ReadFixed32() != uint.MaxValue) throw ParseError("Jump Table end marker not found"); - - // checksum - uint actualChecksum = SnapshotFormat.ComputeChecksum(reader.Base, reader.Offset); - uint checksum = reader.ReadFixed32(); - if (actualChecksum != checksum) throw ParseError("Jump Table checksum does not match ({0} != {1}). This may be an indication of data corruption", checksum, actualChecksum); - - m_jumpTable = table; - m_levels = levels; - m_hasJumpTable = true; - } - - public void ReadLevel(int level, LevelWriter writer, CancellationToken ct) - { - Contract.Requires(level >= 0 && writer != null); - ct.ThrowIfCancellationRequested(); - - if (!m_hasJumpTable) - { - throw new InvalidOperationException("Cannot read a level without reading the Jump Table first!"); - } - - int itemCount = checked(1 << level); - - var address = m_jumpTable[level]; - - if (address.Offset < m_dataStart || address.Offset > m_dataEnd) - { - throw ParseError("Level {0} offset ({1}) is invalid", level, address.Offset); - } - if (checked(address.Offset + address.PaddedSize) > m_dataEnd) - { - throw ParseError("Level {0} size ({1}) is invalid", level, address.PaddedSize); - } - - var reader = m_file.CreateReader(address.Offset, address.PaddedSize); - - // "LVL_" - var signature = reader.ReadFixed32(); - // Level Flags - var flags = reader.ReadFixed32(); - // Level ID - int levelId = (int)reader.ReadFixed32(); - // Item count (always 2^level) - int levelCount = (int)reader.ReadFixed32(); - - if (signature != SnapshotFormat.LEVEL_MAGIC_NUMBER) throw ParseError("Page does not appear to be a valid Level header"); - //TODO: check flags - if (levelId != level) throw ParseError("Page contains the header of a different Level ({0} != {1})", levelId, level); - if (levelCount != itemCount) throw ParseError("Item count ({0}) in level {1} header is not valid", levelCount, level); - - for (int i = 0; i < levelCount;i++) - { - // read the key - uint keySize = reader.ReadVarint32(); - if (keySize > MemoryDatabaseHandler.MAX_KEY_SIZE) throw ParseError("Key size ({0}) is too big", keySize); - USlice key = keySize == 0 ? USlice.Nil : reader.ReadBytes(keySize); - - // read the sequence - ulong sequence = reader.ReadVarint64(); - - // read the value - uint valueSize = reader.ReadVarint32(); - if (valueSize > MemoryDatabaseHandler.MAX_VALUE_SIZE) throw ParseError("Value size ({0) is too big", valueSize); - USlice value = valueSize == 0 ? USlice.Nil : reader.ReadBytes(valueSize); - - writer.Add(sequence, key, value); - } - - if (reader.ReadFixed32() != uint.MaxValue) throw ParseError("Invalid end marker in level"); - //TODO: check end marker, CRC, ... ? - uint checksum = reader.ReadFixed32(); - //TODO: verify checksum! - } - - } - - -} diff --git a/FoundationDB.Storage.Memory/IO/SnapshotWriter.cs b/FoundationDB.Storage.Memory/IO/SnapshotWriter.cs deleted file mode 100644 index ae5ba8c61..000000000 --- a/FoundationDB.Storage.Memory/IO/SnapshotWriter.cs +++ /dev/null @@ -1,324 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.IO -{ - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using FoundationDB.Storage.Memory.API; - using FoundationDB.Storage.Memory.Core; - using FoundationDB.Storage.Memory.Utils; - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - using System.Threading; - using System.Threading.Tasks; - using System.Runtime.InteropServices; - - internal class SnapshotWriter - { - private readonly Win32SnapshotFile m_file; - private SliceWriter m_writer; - - private readonly int m_levels; - private readonly int m_pageSize; - private readonly int m_bufferSize; - - private Uuid128 m_uid; - private ulong m_sequence; - private long m_itemCount; - private long m_timestamp; - private uint m_headerChecksum; - - private readonly KeyValuePair[] m_jumpTable; - - public SnapshotWriter(Win32SnapshotFile file, int levels, int pageSize, int bufferSize) - { - Contract.Requires(file != null && levels >= 0 && pageSize >= 0 && bufferSize >= pageSize); //TODO: && file.CanRead ? - m_file = file; - m_pageSize = pageSize; - m_bufferSize = bufferSize; - //TODO: verify pageSize is a power of two, and bufferSize is a multiple of pageSize! - Contract.Assert(bufferSize % pageSize == 0); - - m_writer = new SliceWriter(bufferSize); - m_levels = levels; - - m_jumpTable = new KeyValuePair[levels]; - for (int i = 0; i < levels; i++) - { - m_jumpTable[i] = new KeyValuePair(0, 0); - } - } - - /// Write the header to the file - /// - /// - /// - /// - /// - /// - /// This needs to be called before writing any level to the file - public Task WriteHeaderAsync(SnapshotFormat.Flags headerFlags, Uuid128 uid, ulong sequence, long count, long timestamp, IDictionary attributes) - { - // The header will be use on ore more "pages", to simplify the job of loading / peeking at a stream content (no need for fancy buffering, just need to read 4K pages) - // > The last page is padded with 0xAAs to detect corruption. - - m_uid = uid; - m_sequence = sequence; - m_itemCount = count; - m_timestamp = timestamp; - - // HEADER - // - DB_HEADER (64 bytes) - // - DB ATTRIBUTES (variable size list of k/v) - // - END_MARKER + HEADER_CRC - // - PADDING (to fill last page) - - // DB Header - - // "PNDB" - m_writer.WriteFixed32(SnapshotFormat.HEADER_MAGIC_NUMBER); - // v1.0 - m_writer.WriteFixed16(1); // major - m_writer.WriteFixed16(0); // minor - // FLAGS - m_writer.WriteFixed64((ulong)headerFlags); - // Database ID - m_writer.WriteBytes(uid.ToSlice()); - // Database Version - m_writer.WriteFixed64(sequence); - // Number of items in the database - m_writer.WriteFixed64((ulong)count); - // Database Timestamp - m_writer.WriteFixed64((ulong)timestamp); - // Page Size - m_writer.WriteFixed32(SnapshotFormat.PAGE_SIZE); - // Header Size (not known yet and will be filled in later) - int offsetToHeaderSize = m_writer.Skip(4); - - // we should be at the 64 byte mark - Contract.Assert(m_writer.Position == SnapshotFormat.HEADER_METADATA_BYTES); - - // DB Attributes - m_writer.WriteFixed32((uint)attributes.Count); - foreach (var kvp in attributes) - { - // Name - m_writer.WriteVarbytes(Slice.FromString(kvp.Key)); - - // Value - m_writer.WriteVarbytes(kvp.Value.ToSlice()); - } - - // Mark the end of the header - m_writer.WriteFixed32(uint.MaxValue); - - // we now have the size of the header, and can fill in the blank - var headerEnd = m_writer.Position; - m_writer.Position = offsetToHeaderSize; - // write the header size (includes the CRC) - m_writer.WriteFixed32((uint)checked(headerEnd + SnapshotFormat.HEADER_CRC_SIZE)); - m_writer.Position = headerEnd; - - // now we can compute the actual CRC - uint headerChecksum = SnapshotFormat.ComputeChecksum(m_writer.ToSlice()); - m_writer.WriteFixed32(headerChecksum); - m_headerChecksum = headerChecksum; - - // optional padding to fill the rest of the page - PadPageIfNeeded(SnapshotFormat.PAGE_SIZE, 0xFD); - - return TaskHelpers.CompletedTask; - } - - public async Task WriteLevelAsync(int level, IntPtr[] segment, CancellationToken ct) - { - ct.ThrowIfCancellationRequested(); - - if (m_jumpTable[level].Value > 0) - { - throw new InvalidOperationException("The level has already be written to this snapshot"); - } - - var levelStart = checked(m_file.Length + (uint)m_writer.Position); - //Console.WriteLine("## level " + level + " starts at " + levelStart); - - //TODO: ensure that we start on a PAGE? - - //Console.WriteLine("> Writing level " + level); - - // "LVL_" - m_writer.WriteFixed32(SnapshotFormat.LEVEL_MAGIC_NUMBER); - // Level Flags - m_writer.WriteFixed32(0); //TODO: flags! - // Level ID - m_writer.WriteFixed32((uint)level); - // Item count (always 2^level) - m_writer.WriteFixed32((uint)segment.Length); - - for (int i = 0; i < segment.Length; i++) - { - unsafe - { -#if __MonoCS__ - var valuePointer =new IntPtr((void*) MemoryDatabaseHandler.ResolveValueAtVersion(segment[i], m_sequence)); - - if (valuePointer == IntPtr.Zero) - continue; - - Value value = new Value(); - Marshal.PtrToStructure(valuePointer, value); - - var keyPointer = new IntPtr((void*)segment[i]); - - Key key = new Key(); - Marshal.PtrToStructure(keyPointer, key); - - Contract.Assert(key.Size <= MemoryDatabaseHandler.MAX_KEY_SIZE); - - // Key Size - uint size = key.Size; - m_writer.WriteVarint32(size); - m_writer.WriteBytesUnsafe(&(key.Data), (int)size); - - // Value - m_writer.WriteVarint64(value.Sequence); // sequence - size = value.Size; - if (size == 0) - { // empty key - m_writer.WriteByte(0); - } - else - { - m_writer.WriteVarint32(size); // value size - m_writer.WriteBytesUnsafe(&(value.Data), (int)size); // value data - } -#else - - Value* value = MemoryDatabaseHandler.ResolveValueAtVersion(segment[i], m_sequence); - if (value == null) - { - continue; - } - Key* key = (Key*)segment[i]; //.ToPointer(); - - Contract.Assert(key != null && key->Size <= MemoryDatabaseHandler.MAX_KEY_SIZE); - - // Key Size - uint size = key->Size; - m_writer.WriteVarint32(size); - m_writer.WriteBytesUnsafe(&(key->Data), (int)size); - - // Value - - m_writer.WriteVarint64(value->Sequence); // sequence - size = value->Size; - if (size == 0) - { // empty key - m_writer.WriteByte(0); - } - else - { - m_writer.WriteVarint32(size); // value size - m_writer.WriteBytesUnsafe(&(value->Data), (int)size); // value data - } -#endif - } - - if (m_writer.Position >= SnapshotFormat.FLUSH_SIZE) - { - //Console.WriteLine("> partial flush (" + writer.Position + ")"); - int written = await m_file.WriteCompletePagesAsync(m_writer.Buffer, m_writer.Position, ct).ConfigureAwait(false); - if (written > 0) m_writer.Flush(written); - } - } - - m_writer.WriteFixed32(uint.MaxValue); - - //TODO: CRC? (would need to be computed on the fly, because we don't have the full slice in memory probably) - m_writer.WriteFixed32(0); - - var levelEnd = checked(m_file.Length + (uint)m_writer.Position); - m_jumpTable[level] = new KeyValuePair(levelStart, levelEnd - levelStart); - //Console.WriteLine("## level " + level + " ends at " + levelEnd); - - // optional padding to fill the rest of the page - PadPageIfNeeded(SnapshotFormat.PAGE_SIZE, (byte)(0xFC - level)); - - } - - public Task WriteJumpTableAsync(CancellationToken ct) - { - ct.ThrowIfCancellationRequested(); - - // The jump table is the last page of the file - // - it contains the list of (offset, size) of all the levels that are in the file - // - it contains any additional attributes (that were only known after writing all the data) - // - it repeats a few important values (sequence, header crc, ...) - // - it would contain any optional signature or data that is only know after writing the data to disk, and are needed to decode the rest - - // marks the start of the JT because we will need to compute the checksum later on - int startOffset = m_writer.Position; - - // "JMPT" - m_writer.WriteFixed32(SnapshotFormat.JUMP_TABLE_MAGIC_NUMBER); - // Page Size (repeated) - m_writer.WriteFixed32((uint)m_pageSize); - // Sequence Number (repeated) - m_writer.WriteFixed64(m_sequence); - // Database ID (repeated) - m_writer.WriteBytes(m_uid.ToSlice()); - // Header CRC (repeated) - m_writer.WriteFixed32(m_headerChecksum); - - int levels = m_levels; - m_writer.WriteFixed32((uint)levels); // Level Count - for (int level = 0; level < levels; level++) - { - // Level Offset (from start of file) - m_writer.WriteFixed64((ulong)m_jumpTable[level].Key); - // Level Size (in bytes) - m_writer.WriteFixed64((ulong)m_jumpTable[level].Value); - } - - //TODO: additional attributes! - m_writer.WriteFixed32(0); // 0 for now - - // End Marker - m_writer.WriteFixed32(uint.MaxValue); - - // Checksum - int endOffset = m_writer.Position; - uint jumpTableChecksum = SnapshotFormat.ComputeChecksum(m_writer[startOffset, endOffset]); - m_writer.WriteFixed32(jumpTableChecksum); - - // optional padding to fill the rest of the page - PadPageIfNeeded(SnapshotFormat.PAGE_SIZE, 0xFE); - - // we are done ! - return TaskHelpers.CompletedTask; - } - - public Task FlushAsync(CancellationToken ct) - { - //Console.WriteLine("> final flush (" + writer.Position + ")"); - return m_file.FlushAsync(m_writer.Buffer, m_writer.Position, ct); - } - - private void PadPageIfNeeded(int pageSize, byte padByte) - { - // Ensure the page is full - int pageOffset = m_writer.Position & (SnapshotFormat.PAGE_SIZE - 1); - if (pageOffset != 0) - { // Pad the remainder of the page - int pad = SnapshotFormat.PAGE_SIZE - pageOffset; - m_writer.Skip(pad, padByte); - //Console.WriteLine("@@@ added " + pad + " pad bytes => " + m_writer.Position); - } - } - - } - -} diff --git a/FoundationDB.Storage.Memory/IO/Win32MemoryMappedFile.cs b/FoundationDB.Storage.Memory/IO/Win32MemoryMappedFile.cs deleted file mode 100644 index 077d669fc..000000000 --- a/FoundationDB.Storage.Memory/IO/Win32MemoryMappedFile.cs +++ /dev/null @@ -1,254 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.IO -{ - using FoundationDB.Storage.Memory.Utils; - using Microsoft.Win32.SafeHandles; - using System; - using System.Diagnostics.Contracts; - using System.IO; - using System.Runtime.InteropServices; - using System.Security; - using System.Security.AccessControl; - - [SuppressUnmanagedCodeSecurity] - internal static class UnsafeNativeMethods - { - - [StructLayout(LayoutKind.Sequential)] - public sealed class SECURITY_ATTRIBUTES - { - public int nLength; - public IntPtr lpSecurityDescriptor; - public int bInheritHandle; - } - - [StructLayout(LayoutKind.Sequential)] - public struct SYSTEM_INFO - { - internal int dwOemId; - internal int dwPageSize; - internal IntPtr lpMinimumApplicationAddress; - internal IntPtr lpMaximumApplicationAddress; - internal IntPtr dwActiveProcessorMask; - internal int dwNumberOfProcessors; - internal int dwProcessorType; - internal uint dwAllocationGranularity; - internal short wProcessorLevel; - internal short wProcessorRevision; - } - - [StructLayout(LayoutKind.Sequential, CharSet = CharSet.Auto)] - internal class MEMORYSTATUSEX - { - internal uint dwLength = ((uint)Marshal.SizeOf(typeof(UnsafeNativeMethods.MEMORYSTATUSEX))); - internal uint dwMemoryLoad; - internal ulong ullTotalPhys; - internal ulong ullAvailPhys; - internal ulong ullTotalPageFile; - internal ulong ullAvailPageFile; - internal ulong ullTotalVirtual; - internal ulong ullAvailVirtual; - internal ulong ullAvailExtendedVirtual; - } - - [Flags] - public enum FileMapProtection : uint - { - PageReadonly = 0x02, - PageReadWrite = 0x04, - PageWriteCopy = 0x08, - PageExecuteRead = 0x20, - PageExecuteReadWrite = 0x40, - SectionCommit = 0x8000000, - SectionImage = 0x1000000, - SectionNoCache = 0x10000000, - SectionReserve = 0x4000000, - } - - [Flags] - public enum FileMapAccess : uint - { - FileMapCopy = 0x0001, - FileMapWrite = 0x0002, - FileMapRead = 0x0004, - FileMapAllAccess = 0x001f, - FileMapExecute = 0x0020, - } - - [SecurityCritical, DllImport("kernel32.dll", SetLastError = true)] - private static extern void GetSystemInfo(ref SYSTEM_INFO lpSystemInfo); - - [SecurityCritical, DllImport("kernel32.dll", SetLastError = true)] - [return: MarshalAs(UnmanagedType.Bool)] - public static extern bool GlobalMemoryStatusEx([In, Out] MEMORYSTATUSEX lpBuffer); - - [SecurityCritical, DllImport("kernel32.dll", CharSet = CharSet.Unicode, SetLastError = true)] - public static extern SafeMemoryMappedFileHandle CreateFileMapping(SafeFileHandle hFile, SECURITY_ATTRIBUTES lpAttributes, FileMapProtection fProtect, uint dwMaximumSizeHigh, uint dwMaximumSizeLow, string lpName); - - [SecurityCritical, DllImport("kernel32.dll", SetLastError = true, ExactSpelling = true)] - public static extern SafeMemoryMappedViewHandle MapViewOfFile(SafeMemoryMappedFileHandle handle, FileMapAccess dwDesiredAccess, uint dwFileOffsetHigh, uint dwFileOffsetLow, UIntPtr dwNumberOfBytesToMap); - - /// Gets the granularity for the starting address at which virtual memory can be allocated. - [SecurityCritical] - public static uint GetSystemPageAllocationGranularity() - { - var sysInfo = new SYSTEM_INFO(); - GetSystemInfo(ref sysInfo); - return sysInfo.dwAllocationGranularity; - } - - /// Gets the total size of the user mode portion of the virtual address space of the calling process, in bytes. - [SecurityCritical] - public static ulong GetTotalVirtualAddressSpaceSize() - { - var memStatusEx = new MEMORYSTATUSEX(); - GlobalMemoryStatusEx(memStatusEx); - return memStatusEx.ullTotalVirtual; - } - } - - internal unsafe sealed class Win32MemoryMappedFile : IDisposable - { - private readonly SafeMemoryMappedFileHandle m_mapHandle; - private readonly SafeMemoryMappedViewHandle m_viewHandle; - private readonly FileStream m_file; - private readonly ulong m_size; - private readonly byte* m_baseAddress; - private bool m_disposed; - - private Win32MemoryMappedFile(FileStream fs, SafeMemoryMappedFileHandle handle, ulong size) - { - Contract.Requires(fs != null && handle != null && !handle.IsInvalid && !handle.IsClosed); - m_mapHandle = handle; - m_file = fs; - m_size = size; - - // verify that it fits on 32 bit OS... - if (IntPtr.Size == 4 && size > uint.MaxValue) - { // won't work with 32-bit pointers - throw new InvalidOperationException("Memory mapped file size is too big to be opened on a 32-bit system."); - } - - // verifiy that it will fit in the virtual address space of the process - var totalVirtual = UnsafeNativeMethods.GetTotalVirtualAddressSpaceSize(); - if (size > totalVirtual) - { - throw new InvalidOperationException("Memory mapped file size is too big to fit in the current process virtual address space"); - } - - SafeMemoryMappedViewHandle view = null; - byte* baseAddress = null; - try - { - view = UnsafeNativeMethods.MapViewOfFile(m_mapHandle, UnsafeNativeMethods.FileMapAccess.FileMapRead, 0, 0, new UIntPtr(size)); - if (view.IsInvalid) throw Marshal.GetExceptionForHR(Marshal.GetHRForLastWin32Error()); - view.Initialize(size); - m_viewHandle = view; - - view.AcquirePointer(ref baseAddress); - m_baseAddress = baseAddress; - } - catch - { - if (baseAddress != null) view.ReleasePointer(); - if (view != null) view.Dispose(); - m_file = null; - m_viewHandle = null; - m_mapHandle = null; - m_baseAddress = null; - throw; - } - } - - [SecurityCritical] - public static Win32MemoryMappedFile OpenRead(string path) - { - Contract.Requires(!string.IsNullOrEmpty(path)); - - - if (!File.Exists(path)) - { - throw new FileNotFoundException("Memory mapped file not found", path); - } - - FileStream fs = null; - SafeMemoryMappedFileHandle handle = null; - try - { - // Open the file - fs = new FileStream(path, FileMode.Open, FileSystemRights.ListDirectory, FileShare.None, 0x1000, FileOptions.SequentialScan); - Contract.Assert(fs != null); - ulong capacity = checked((ulong)fs.Length); - if (capacity == 0) throw new ArgumentException("Cannot memory map an empty file"); - - // Create the memory mapping - uint dwMaximumSizeLow = (uint)(capacity & 0xffffffffL); - uint dwMaximumSizeHigh = (uint)(capacity >> 32); - handle = UnsafeNativeMethods.CreateFileMapping(fs.SafeFileHandle, null /*TODO?*/, UnsafeNativeMethods.FileMapProtection.PageReadonly, dwMaximumSizeHigh, dwMaximumSizeLow, null); - int errorCode = Marshal.GetLastWin32Error(); - if (handle.IsInvalid || errorCode == 183) - { - throw Marshal.GetExceptionForHR(errorCode); - } - - return new Win32MemoryMappedFile(fs, handle, capacity); - } - catch - { - if (handle != null) handle.Dispose(); - if (fs != null) fs.Dispose(); - throw; - } - } - - public string Name - { - get { return m_file.Name; } - } - - public ulong Length - { - get { return m_size; } - } - - private void EnsureReadable(ulong offset, ulong size) - { - if (m_disposed) throw new ObjectDisposedException(this.GetType().Name, "Memory mapped file has already been closed"); - if (offset > m_size) throw new ArgumentException("Offset is outside the bounds of the memory mapped file"); - if (checked(offset + size) > m_size) throw new ArgumentException("Size is outside the bounds of the memory mapped file"); - } - - public unsafe UnmanagedSliceReader CreateReader(ulong offset, ulong size) - { - EnsureReadable(offset, size); - - byte* start = m_baseAddress + offset; - - return UnmanagedSliceReader.FromAddress(start, size); - } - - #region IDisposable... - - public void Dispose() - { - if (!m_disposed) - { - m_disposed = true; - - if (m_viewHandle != null) - { - if (m_baseAddress != null) m_viewHandle.ReleasePointer(); - m_viewHandle.Dispose(); - } - if (m_mapHandle != null) m_mapHandle.Dispose(); - if (m_file != null) m_file.Dispose(); - } - } - - #endregion - } - -} diff --git a/FoundationDB.Storage.Memory/IO/Win32SnapshotFile.cs b/FoundationDB.Storage.Memory/IO/Win32SnapshotFile.cs deleted file mode 100644 index f26eb8be9..000000000 --- a/FoundationDB.Storage.Memory/IO/Win32SnapshotFile.cs +++ /dev/null @@ -1,187 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.IO -{ - using System; - using System.Diagnostics.Contracts; - using System.IO; - using System.Threading; - using System.Threading.Tasks; - - internal sealed class Win32SnapshotFile : IDisposable - { - private readonly string m_path; - private readonly int m_pageSize; - private FileStream m_fs; - - public const int SECTOR_SIZE = 4096; - - public Win32SnapshotFile(string path, bool read = false) - : this(path, SECTOR_SIZE, read) - { } - - public Win32SnapshotFile(string path, int pageSize, bool read = false) - { - if (string.IsNullOrEmpty(path)) throw new ArgumentNullException("path"); - if (pageSize < 512) throw new ArgumentException("Page size must be at least 512.", "pageSize"); - if (pageSize == 0) pageSize = SECTOR_SIZE; - //TODO: check that pageSize is a power of two ?? - - path = Path.GetFullPath(path); - m_path = path; - m_pageSize = pageSize; - - FileStream fs = null; - try - { - if (read) - { - fs = new FileStream(path, FileMode.Open, FileAccess.Read, FileShare.Read, pageSize, FileOptions.Asynchronous | FileOptions.SequentialScan | (FileOptions)0x20000000/* NO_BUFFERING */); - } - else - { - fs = new FileStream(path, FileMode.Create, FileAccess.Write, FileShare.Read, pageSize, FileOptions.Asynchronous | FileOptions.WriteThrough | (FileOptions)0x20000000/* NO_BUFFERING */); - } - } - catch(Exception) - { - if (fs != null) - { - fs.Dispose(); - fs = null; - } - throw; - } - finally - { - m_fs = fs; - } - - Contract.Ensures(m_fs != null && m_fs.IsAsync); - } - - public ulong Length - { - get - { - var fs = m_fs; - return fs != null ? (ulong)fs.Length : 0UL; - } - } - - public void Seek(ulong position) - { - Contract.Requires(position <= this.Length); - - long fpos = checked((long)position); - long apos = m_fs.Seek(fpos, SeekOrigin.Begin); - if (apos != fpos) throw new IOException("Failed to seek to the desired position"); - } - - /// Read a certain number of bytes into a buffer - /// Buffer where to store the data - /// Offset in the buffer where the data will be written - /// Number of bytes to read - /// - /// Number of bytes read. If it is less than , it means the file was truncated. - /// May execute more than one read operation if the first one did not return enough data (reading from a network stream or NFS share??) - public async Task ReadExactlyAsync(byte[] buffer, uint offset, ulong count, CancellationToken cancellationToken) - { - if (m_fs == null) throw new ObjectDisposedException(this.GetType().Name); - - if (count > int.MaxValue) throw new OverflowException("Count is too big"); - - int remaining = (int)count; - uint read = 0; - if (remaining > 0) - { - int p = (int)offset; - while (remaining > 0) - { - Contract.Assert(p >= 0 && p < buffer.Length && remaining > 0 && p + remaining <= buffer.Length, "Read buffer overflow"); - try - { - int n = await m_fs.ReadAsync(buffer, p, remaining, cancellationToken).ConfigureAwait(false); - if (n <= 0) break; - p += n; - remaining -= n; - read += (uint)n; - } - catch(IOException) - { - throw; - } - } - } - return read; - } - - /// Write as many full pages to the file - /// Buffer that contains the data to write - /// Number of bytes in the buffer (that may or may not be aligned to a page size) - /// Optional cancellation token - /// Number of bytes written to the disk (always a multiple of 4K), or 0 if the buffer did not contain enough data. - public async Task WriteCompletePagesAsync(byte[] buffer, int count, CancellationToken cancellationToken) - { - if (m_fs == null) throw new ObjectDisposedException(this.GetType().Name); - - int complete = (count / m_pageSize) * m_pageSize; - if (complete > 0) - { - await m_fs.WriteAsync(buffer, 0, complete, cancellationToken).ConfigureAwait(false); - } - - return complete; - } - - /// Flush the remaining of the buffer to the disk, and ensures that the content has been fsync'ed - /// Buffer that may contains data (can be null if is equal to 0) - /// Number of bytes remaining in the buffer (or 0 if there is no more data to written) - /// - /// - public async Task FlushAsync(byte[] buffer, int count, CancellationToken cancellationToken) - { - Contract.Assert(count == 0 || buffer != null); - - if (count > 0) - { - int complete = (count / m_pageSize) * m_pageSize; - if (complete > 0) - { - await m_fs.WriteAsync(buffer, 0, complete, cancellationToken).ConfigureAwait(false); - count -= complete; - } - if (count > 0) - { // we have to write full 4K sectors, so we'll need to copy the rest to a temp 4K buffer (padded with 0s) - var tmp = new byte[m_pageSize]; - Buffer.BlockCopy(buffer, complete, tmp, 0, count); - await m_fs.WriteAsync(tmp, 0, count, cancellationToken).ConfigureAwait(false); - } - } - //REVIEW: since we are using WRITE_THROUGH + NO_BUFFERING, the OS is *supposed* to write directly to the disk ... - // need to verify that this is actually the case! - await m_fs.FlushAsync(cancellationToken); - } - - public override string ToString() - { - return "Snapshot:" + m_path; - } - - public void Dispose() - { - try - { - var fs = m_fs; - if (fs != null) fs.Close(); - } - finally - { - m_fs = null; - } - } - } - -} diff --git a/FoundationDB.Storage.Memory/Properties/AssemblyInfo.cs b/FoundationDB.Storage.Memory/Properties/AssemblyInfo.cs deleted file mode 100644 index 9ddda036a..000000000 --- a/FoundationDB.Storage.Memory/Properties/AssemblyInfo.cs +++ /dev/null @@ -1,17 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -/* -TODO: CHOOSE A LICENSE! -*/ -#endregion - -using System.Reflection; -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; - -[assembly: AssemblyTitle("FoundationDB.Storage.Memory")] -[assembly: AssemblyDescription("In-Memory Storage Engine for FoundationDB")] -[assembly: AssemblyConfiguration("")] - -[assembly: ComVisible(false)] - -[assembly: Guid("6eaef97f-bc30-4dcf-b8d4-c22f749907e3")] \ No newline at end of file diff --git a/FoundationDB.Storage.Memory/Properties/VersionInfo.cs b/FoundationDB.Storage.Memory/Properties/VersionInfo.cs deleted file mode 100644 index c479cd88f..000000000 --- a/FoundationDB.Storage.Memory/Properties/VersionInfo.cs +++ /dev/null @@ -1,19 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -/* -TODO: CHOOSE A LICENSE! -*/ -#endregion - -using System.Reflection; -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; - -[assembly: AssemblyCompany("Doxense")] -[assembly: AssemblyProduct("FoundationDB.Storage.Memory")] -[assembly: AssemblyCopyright("Copyright © Doxense 2013-2014")] -[assembly: AssemblyTrademark("")] -[assembly: AssemblyCulture("")] - -[assembly: AssemblyVersion("0.1.0.0")] //note: Only change this when doing breaking API changes! -[assembly: AssemblyFileVersion("0.1.0.0")] //note: change this to "x.y.z.build_number" in a build step of your favorite C.I. build server -[assembly: AssemblyInformationalVersion("0.1.0-alpha")] //note: this is the version seen by NuGet, add "-alpha"/"-beta"/"-rc1" at the end to create pre-release packages diff --git a/FoundationDB.Storage.Memory/README.md b/FoundationDB.Storage.Memory/README.md deleted file mode 100644 index c477ea5c3..000000000 --- a/FoundationDB.Storage.Memory/README.md +++ /dev/null @@ -1,3 +0,0 @@ -This project contains an experimental In-Memory database that is compatible with the FoundationDB API - -**WARNING**: This doesn't work yet! It is only a very early prototype that is not intended for production use! \ No newline at end of file diff --git a/FoundationDB.Storage.Memory/Utils/TaskHelpers.cs b/FoundationDB.Storage.Memory/Utils/TaskHelpers.cs deleted file mode 100644 index e9b47433c..000000000 --- a/FoundationDB.Storage.Memory/Utils/TaskHelpers.cs +++ /dev/null @@ -1,60 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Threading; - using System.Threading.Tasks; - - /// Helper methods to work on tasks - internal static class TaskHelpers - { - - /// Return a task that is already completed - // README: There is a Task.CompletedTask object in the BCL that is internal, and one 'easy' way to get access to it is via Task.Delay(0) that returns it if param is equal to 0... - public static readonly Task CompletedTask = Task.Delay(0); - - /// Returns a failed Task that wraps an exception - /// Type of the result of the task - /// Exception that will be wrapped in the task - /// Task that is already completed, and that will rethrow the exception once observed - public static Task FromException(Exception e) - { - // There is a Task.FromException() method in the BCL, but unfortunately it is internal :( - // We can only emulate it by calling TrySetException on a dummy TaskCompletionSource - // Also, we should flattent AggregateException so as not to create huge chain of aggEx - - var tcs = new TaskCompletionSource(); - - var aggEx = e as AggregateException; - if (aggEx == null) - tcs.TrySetException(e); - else - tcs.TrySetException(aggEx.InnerExceptions); - - //note: also, to avoid blowing up the process if nobody observes the task, we observe it once - var _ = tcs.Task.Exception; - - return tcs.Task; - } - - /// Returns a cancelled Task that is linked with a specific token - /// Type of the result of the task - /// Cancellation token that should already be cancelled - /// Task in the cancelled state that is linked with this cancellation token - public static Task FromCancellation(CancellationToken cancellationToken) - { - // There is a Task.FromCancellation() method in the BCL, but unfortunately it is internal :( - // The "best" way I've seen to emulate the same behavior, is creating a fake task (with a dummy action) with the same alread-cancelled CancellationToken - // This should throw the correct TaskCanceledException that is linked with this token - - // ensure that it is actually cancelled, so that we don't deadlock - if (!cancellationToken.IsCancellationRequested) throw new InvalidOperationException(); - - return new Task(() => default(T), cancellationToken); - } - - } -} diff --git a/FoundationDB.Storage.Memory/Utils/USlice.cs b/FoundationDB.Storage.Memory/Utils/USlice.cs deleted file mode 100644 index 988d4bacd..000000000 --- a/FoundationDB.Storage.Memory/Utils/USlice.cs +++ /dev/null @@ -1,198 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.ConstrainedExecution; - using System.Runtime.InteropServices; - using System.Security; - - /// Slice of unmanaged memory - [DebuggerDisplay("({Data}, {Count})"), DebuggerTypeProxy(typeof(USliceDebugView))] - public unsafe struct USlice : IEquatable, IComparable - { - public readonly byte* Data; - public readonly uint Count; - - /// Gets an empty slice (equivalent to the NULL pointer) - public static USlice Nil - { - get { return default(USlice); } - } - - public USlice(byte* data, uint count) - { - Contract.Requires(data != null || count == 0); - this.Data = data; - this.Count = count; - } - - /// Checks if this is the empty slice (NULL pointer) - public bool IsNull - { - get { return this.Data == null; } - } - - public byte* AtOffset(uint offset) - { - if (this.Data == null || offset >= this.Count) ThrowInvalidAccess(this.Data); - return this.Data + offset; - } - - public byte this[uint offset] - { - get { return *(AtOffset(offset)); } - } - - public USlice Substring(uint startIndex, uint count) - { - if (count == 0) return default(USlice); - Contract.Requires(this.Data != null && startIndex <= this.Count && count <= this.Count && startIndex + count <= this.Count); - - if (this.Data == null) ThrowNullReference(); - if (startIndex > this.Count) ThrowIndexOutsideTheSlice(); - if (count > this.Count || startIndex + count > this.Count) ThrowSliceTooSmall(); - - return new USlice(this.Data + startIndex, count); - } - - private static void ThrowIndexOutsideTheSlice() - { - throw new ArgumentOutOfRangeException("startIndex", "Start index must be inside the slice"); - } - - private static void ThrowSliceTooSmall() - { - throw new ArgumentOutOfRangeException("count", "Slice is too small"); - } - - public IntPtr GetPointer() - { - return new IntPtr(this.Data); - } - - public IntPtr GetPointer(uint offset) - { - return new IntPtr(AtOffset(offset)); - } - - public byte* Successor - { - get - { - if (this.Data == null) ThrowNullReference(); - return this.Data + this.Count; - } - } - - public byte[] GetBytes() - { - Contract.Requires(this.Count >= 0); - var tmp = new byte[this.Count]; - if (this.Count > 0) - { - Contract.Assert(this.Data != null); - fixed (byte* ptr = tmp) - { - UnmanagedHelpers.CopyUnsafe(ptr, this.Data, this.Count); - } - } - return tmp; - } - - public byte[] GetBytes(uint offset, uint count) - { - Contract.Requires(this.Count >= 0); - - if (offset > this.Count) throw new ArgumentOutOfRangeException("offset"); - if (offset + count >= this.Count) throw new ArgumentOutOfRangeException("count"); - - var tmp = new byte[count]; - if (count > 0) - { - Contract.Assert(this.Data != null); - fixed (byte* ptr = tmp) - { - UnmanagedHelpers.CopyUnsafe(ptr, this.Data + offset, count); - } - } - return tmp; - } - - public FoundationDB.Client.Slice ToSlice() - { - return FoundationDB.Client.Slice.Create(GetBytes()); - } - - public bool Equals(USlice other) - { - if (this.Count != other.Count) return false; - if (this.Data == other.Data) return true; - if (this.Data == null || other.Data == null) return false; - - //TODO: optimize! - return 0 == UnmanagedHelpers.CompareUnsafe(this.Data, this.Count, other.Data, other.Count); - } - - public int CompareTo(USlice other) - { - return UnmanagedHelpers.CompareUnsafe(this.Data, this.Count, other.Data, other.Count); - } - - public override bool Equals(object obj) - { - if (obj == null) return this.Data == null && this.Count == 0; - return obj is USlice && Equals((USlice)obj); - } - - public override int GetHashCode() - { - return UnmanagedHelpers.ComputeHashCode(ref this); - } - - public override string ToString() - { - return "{" + (long)this.Data + ", " + this.Count + "}"; - } - - private static void ThrowNullReference() - { - throw new InvalidOperationException("Cannot access NULL pointer"); - } - - private static void ThrowInvalidAccess(byte* ptr) - { - if (ptr == null) ThrowNullReference(); - throw new IndexOutOfRangeException(); - - } - - private sealed class USliceDebugView - { - private readonly USlice m_slice; - - public USliceDebugView(USlice slice) - { - m_slice = slice; - } - - public uint Size - { - get { return m_slice.Count; } - } - - public byte[] Data - { - get { return m_slice.GetBytes(); } - } - - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/USliceComparer.cs b/FoundationDB.Storage.Memory/Utils/USliceComparer.cs deleted file mode 100644 index 6761d5e29..000000000 --- a/FoundationDB.Storage.Memory/Utils/USliceComparer.cs +++ /dev/null @@ -1,54 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Collections.Generic; - - /// Performs optimized equality and comparison checks on Slices - public unsafe sealed class USliceComparer : IComparer, IEqualityComparer, IComparer> - { - /// Default instance of the slice comparator - public static readonly USliceComparer Default = new USliceComparer(); - - private USliceComparer() - { } - - /// Lexicographically compare two slices and returns an indication of their relative sort order - /// Slice compared with - /// Slice compared with - /// Returns a NEGATIVE value if is LESS THAN , ZERO if is EQUAL TO , and a POSITIVE value if is GREATER THAN . - /// If both and are nil or empty, the comparison will return ZERO. If only is nil or empty, it will return a NEGATIVE value. If only is nil or empty, it will return a POSITIVE value. - public int Compare(USlice x, USlice y) - { - return UnmanagedHelpers.CompareUnsafe(x.Data, x.Count, y.Data, y.Count); - } - - /// Checks if two slices are equal. - /// Slice compared with - /// Slice compared with - /// true if and have the same size and contain the same sequence of bytes; otherwise, false. - public bool Equals(USlice x, USlice y) - { - return x.Count == y.Count && 0 == UnmanagedHelpers.CompareUnsafe(x.Data, x.Count, y.Data, y.Count); - } - - /// Computes the hash code of a slice - /// A slice - /// A 32-bit signed hash coded calculated from all the bytes in the slice - public int GetHashCode(USlice obj) - { - if (obj.Data == null) return 0; - //return ComputeHashCode(obj.Array, obj.Offset, obj.Count); - return 123; //TODO! - } - - int IComparer>.Compare(KeyValuePair x, KeyValuePair y) - { - return UnmanagedHelpers.CompareUnsafe(x.Key.Data, x.Key.Count, y.Key.Data, y.Key.Count); - } - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedHelpers.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedHelpers.cs deleted file mode 100644 index 7e90d4656..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedHelpers.cs +++ /dev/null @@ -1,382 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -#define USE_NATIVE_MEMORY_OPERATORS - -namespace FoundationDB.Storage.Memory.Utils -{ - using FoundationDB.Client; - using System; - using System.Diagnostics.Contracts; - using System.Runtime.CompilerServices; - using System.Runtime.ConstrainedExecution; - using System.Runtime.InteropServices; - using System.Security; - - internal static unsafe class UnmanagedHelpers - { - - /// Round a number to the next power of 2 - /// Positive integer that will be rounded up (if not already a power of 2) - /// Smallest power of 2 that is greater than or equal to - /// Will return 1 for = 0 (because 0 is not a power of 2 !), and will throw for < 0 - /// If is a negative number - public static uint NextPowerOfTwo(uint x) - { - // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two - - // special case - if (x == 0) return 1; - - --x; - x |= (x >> 1); - x |= (x >> 2); - x |= (x >> 4); - x |= (x >> 8); - x |= (x >> 16); - return x + 1; - } - - public static SafeLocalAllocHandle AllocMemory(uint size) - { - var handle = NativeMethods.LocalAlloc(0, new UIntPtr(size)); - if (handle.IsInvalid) throw new OutOfMemoryException(String.Format("Failed to allocate from unmanaged memory ({0} bytes)", size)); - return handle; - } - - /// Copy a managed slice to the specified memory location - /// Where to copy the bytes - /// Slice of managed memory that will be copied to the destination - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] - public static void CopyUnsafe(byte* dest, Slice src) - { - if (src.Count > 0) - { - Contract.Requires(dest != null && src.Array != null && src.Offset >= 0 && src.Count >= 0); - fixed (byte* ptr = src.Array) - { - CopyUnsafe(dest, ptr + src.Offset, (uint)src.Count); - } - } - } - - public static void CopyUnsafe(Slice dest, byte* src, uint count) - { - if (count > 0) - { - Contract.Requires(dest.Array != null && dest.Offset >= 0 && dest.Count >= 0 && src != null); - fixed (byte* ptr = dest.Array) - { - NativeMethods.memmove(ptr + dest.Offset, src, new UIntPtr(count)); - } - } - } - - /// Copy an unmanaged slice to the specified memory location - /// Where to copy the bytes - /// Slice un unmanaged memory that will be copied to the destination - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] - public static void CopyUnsafe(byte* dest, USlice src) - { - if (src.Count > 0) - { - Contract.Requires(dest != null && src.Data != null); - CopyUnsafe(dest, src.Data, src.Count); - } - } - - public static void CopyUnsafe(USlice dest, byte* src, uint count) - { - if (count > 0) - { - Contract.Requires(dest.Data != null && src != null); - CopyUnsafe(dest.Data, src, 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)] - public static void CopyUnsafe(byte* dest, byte* src, uint count) - { - Contract.Requires(dest != null && src != null); - -#if USE_NATIVE_MEMORY_OPERATORS - NativeMethods.memmove(dest, src, new UIntPtr(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 - } - - /// Retourne l'offset de la première différence trouvée entre deux buffers de même taille - /// Pointeur sur le premier buffer (de taille égale à 'count') - /// Taille du premier buffer (en octets) - /// Pointeur sur le deuxième buffer (de taille égale à 'count') - /// Taille du deuxième buffer (en octets) - /// Offset vers le premier élément qui diffère, ou -1 si les deux buffers sont identiques - [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static int CompareUnsafe(byte* left, uint leftCount, byte* right, uint rightCount) - { - if (leftCount == 0) return rightCount == 0 ? 0 : -1; - if (rightCount == 0) return +1; - - Contract.Requires(left != null && right != null); - -#if USE_NATIVE_MEMORY_OPERATORS - int c = NativeMethods.memcmp(left, right, new UIntPtr(leftCount < rightCount ? leftCount : rightCount)); - if (c != 0) return c; - return (int)leftCount - (int)rightCount; -#else - - // On va scanner par segments de 8, en continuant tant qu'ils sont identiques. - // Dés qu'on tombe sur un segment de 8 différent, on backtrack au début du segment, et on poursuit en mode octet par octet - // Recherche la première position où les octets diffèrent, et retourne left[POS] - right[POS]. - // Si tous les octets sont identiques, retourne 0 - - byte* start = left; - - - // OPTIMISATION DE LA MORT QUI TUE - // Si on calcul le XOR entre les blocs de 8 bytes, chaque byte identique deviendra 0. - // Si le XOR total n'est pas 0, on regarde a quel endroit se trouve le premier byte non-0, et cela nous donne l'offset de la différence - - // Données identiques: - // 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 - - // Différence - // 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 - // le premier byte différent de 0 est le byte 5 (note: on part de la fin, offset 0 !) qui est 0xCC - - // d'abord, on compare 8 bytes par 8 bytes - while (count >= 8) - { - // XOR les deux segments - // => s'il sont identiques, on obtient 0 - // => sinon, le premier byte non 0 (big-endian) indiquera l'offset de la différence - ulong k = *((ulong*)left) ^ *((ulong*)right); - - if (k != 0) - { // on a trouvé une différence, mais cela pourrait être n'importe quel byte - //System.Diagnostics.Trace.WriteLine("Found mistmatch\n\t\t0x" + k.ToString("x16") + " between\n\t\t0x" + ((ulong*)left)[0].ToString("x16") + " and\n\t\t0x" + ((ulong*)right)[0].ToString("x16")); - int p = 0; - while ((k & 0xFF) == 0) - { - ++p; - k >>= 8; - } - //System.Diagnostics.Trace.WriteLine("First differing byte at +" + p + " => " + left[p] + " != " + right[p]); - return left[p] - right[p]; - } - left += 8; - right += 8; - count -= 8; - } - - // la taille restante est forcément entre 0 et 7 - if (count >= 4) - { - if (*((uint*)left) != *((uint*)right)) - { // on a trouvé une différence, mais cela pourrait être n'importe quel byte - goto compare_tail; - } - left += 4; - right += 4; - count -= 4; - } - - // la taille restante est forcément entre 0 et 3 - - compare_tail: - while (count-- > 0) - { - int n = *(left++) - *(right++); - if (n != 0) return n; - } - return 0; -#endif - } - - public static void FillUnsafe(byte* ptr, uint count, byte filler) - { - if (count == 0) return; - if (ptr == null) throw new ArgumentNullException("ptr"); - -#if USE_NATIVE_MEMORY_OPERATORS - NativeMethods.memset(ptr, filler, new UIntPtr(count)); - -#else - if (filler == 0) - { - while (count-- > 0) *ptr++ = 0; - } - else - { - while (count-- > 0) *ptr++ = filler; - } -#endif - } - - public static int ComputeHashCode(ref Slice slice) - { - if (slice.Array == null) return 0; - fixed (byte* ptr = slice.Array) - { - return ComputeHashCodeUnsafe(checked(ptr + slice.Offset), checked((uint)slice.Count)); - } - } - - public static int ComputeHashCode(ref USlice slice) - { - if (slice.Data == null) return 0; - return ComputeHashCodeUnsafe(slice.Data, slice.Count); - } - - /// Compute the hash code of a byte buffer - /// Buffer - /// Number of bytes in the buffer - /// A 32-bit signed hash code calculated from all the bytes in the segment. - public static int ComputeHashCodeUnsafe(byte* bytes, uint count) - { - //note: bytes is allowed to be null only if count == 0 - Contract.Requires(count == 0 || bytes != null); - - //TODO: use a better hash algorithm? (xxHash, CityHash, SipHash, ...?) - // => will be called a lot when Slices are used as keys in an hash-based dictionary (like Dictionary) - // => won't matter much for *ordered* dictionary that will probably use IComparer.Compare(..) instead of the IEqalityComparer.GetHashCode()/Equals() combo - // => we don't need a cryptographic hash, just something fast and suitable for use with hashtables... - // => probably best to select an algorithm that works on 32-bit or 64-bit chunks - - // : unoptimized 32 bits FNV-1a implementation - uint h = 2166136261; // FNV1 32 bits offset basis - while (count-- > 0) - { - h = (h ^ *bytes++) * 16777619; // FNV1 32 prime - } - return (int)h; - // - } - -#if USE_NATIVE_MEMORY_OPERATORS - - internal class SafeLocalAllocHandle : SafeBuffer - { - public static SafeLocalAllocHandle InvalidHandle - { - get {return new SafeLocalAllocHandle();} - } - - - private SafeLocalAllocHandle() - : base(true) - { } - - public SafeLocalAllocHandle(IntPtr handle) - : base(true) - { } - - [SecurityCritical] - protected override bool ReleaseHandle() - { - return NativeMethods.LocalFree(base.handle) == IntPtr.Zero; - } - - } - - [SuppressUnmanagedCodeSecurity] - internal static unsafe class NativeMethods - { - // C/C++ .NET - // --------------------------------- - // void* byte* (or IntPtr) - // size_t UIntPtr (or IntPtr) - // int int - // char byte - - /// Compare characters in two buffers. - /// First buffer. - /// Second buffer. - /// Number of bytes to compare. - /// The return value indicates the relationship between the buffers. - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern int memcmp(byte* buf1, byte* buf2, UIntPtr count); - - /// Moves one buffer to another. - /// Destination object. - /// Source object. - /// Number of bytes to copy. - /// The value of dest. - /// Copies count bytes from src to dest. If some regions of the source area and the destination overlap, both functions ensure that the original source bytes in the overlapping region are copied before being overwritten. - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern byte* memmove(byte* dest, byte* src, UIntPtr count); - - /// Sets buffers to a specified character. - /// Pointer to destination - /// Character to set - /// Number of characters - /// memset returns the value of dest. - /// The memset function sets the first count bytes of dest to the character c. - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern byte* memset(byte* dest, int ch, UIntPtr count); - - [DllImport("kernel32.dll", CharSet = CharSet.Auto, SetLastError = true)] - public static extern SafeLocalAllocHandle LocalAlloc(uint uFlags, UIntPtr uBytes); - - [DllImport("kernel32.dll", SetLastError = true), ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] - public static extern IntPtr LocalFree(IntPtr hMem); - - [DllImport("kernel32.dll")] - public static extern IntPtr LocalReAlloc(IntPtr hMem, UIntPtr uBytes, uint uFlags); - - - } - -#endif - - } -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedMemoryHeap.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedMemoryHeap.cs deleted file mode 100644 index 17140839b..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedMemoryHeap.cs +++ /dev/null @@ -1,382 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using FoundationDB.Client; - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - using System.Text; - - /// Allocate of unmanage memory pages - [DebuggerDisplay("Used={m_memoryUsage}, PageSize={m_pageSize}, Pages={m_pages.Count}")] - public unsafe sealed class UnmanagedMemoryHeap : IDisposable - { - // Allocator strategy: - // To keep it simple, we have several pages that get filled one by one - // If a page is too small to fit the next allocation, a new one is allocated - // Large objects (more than half the size of the memory page) are allocated seperately on their own - - /// Default size for new pages - private const uint DefaultPageSize = 1024 * 1024; - - /// Default alignment for pointers (note: 8 minimum) - private static readonly uint DefaultAlignment = (uint) Math.Max(IntPtr.Size, 8); - - [DebuggerDisplay("Id={m_id}, Usage={this.Used} / {m_size}, Free={m_size-m_nextFree}, Ptr={m_handle}"), DebuggerTypeProxy(typeof(Page.DebugView))] - internal sealed unsafe class Page : IDisposable - { - - private readonly int m_id; - private IntPtr m_handle; - private uint m_size; - - private byte* m_begin; - private uint m_nextFree; - - public Page(int id, IntPtr handle, uint size) - { - Contract.Requires(handle != IntPtr.Zero && size > 0); - - m_id = id; - m_handle = handle; - m_size = size; - - m_begin = (byte*)handle; - - // fill with zeroes ! - UnmanagedHelpers.FillUnsafe(m_begin, size, 0); - - GC.AddMemoryPressure(size); - - Contract.Ensures(m_handle != IntPtr.Zero && m_size > 0 && m_nextFree == 0); - } - - ~Page() - { - Dispose(false); - } - - public byte* Start { get { return m_begin; } } - - public int Id { get { return m_id; } } - - public uint Size { get { return m_size; } } - - public uint Used { get { return m_nextFree; } } - - public uint Remaining { get { return m_size - m_nextFree; } } - - public bool Alive { get { return m_handle != IntPtr.Zero; } } - - private uint GetAlignmentOffset(uint alignment) - { - if (alignment <= 1) return 0; - uint r = m_nextFree & (alignment - 1); - return r == 0 ? 0 : (alignment - r); - } - - public bool CanFit(uint size, uint alignment) - { - Contract.Requires(size > 0); - - return m_nextFree + size + GetAlignmentOffset(alignment) <= m_size; - } - - public byte* Allocate(uint size, uint alignment) - { - Contract.Requires(size > 0); - - uint offset = GetAlignmentOffset(alignment); - - uint pos = m_nextFree + offset; - byte* ptr = m_begin + pos; - m_nextFree = pos + size; - - Contract.Ensures(ptr != null && ptr >= m_begin && ptr <= m_begin + m_size && m_nextFree <= m_size); - return ptr; - } - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - private void Dispose(bool disposing) - { - try - { - } - finally - { - GC.RemoveMemoryPressure(m_size); - m_size = 0; - m_begin = null; - m_nextFree = 0; - - var handle = m_handle; - if (handle != IntPtr.Zero) Marshal.FreeHGlobal(handle); - m_handle = IntPtr.Zero; - } - } - - internal byte[] GetBytes() - { - if (m_handle == IntPtr.Zero) throw new ObjectDisposedException(this.GetType().Name); - - var tmp = new byte[this.Used]; - Marshal.Copy(m_handle, tmp, 0, tmp.Length); - return tmp; - } - - private sealed class DebugView - { - private readonly Page m_page; - - public DebugView(Page page) - { - m_page = page; - } - - public uint Size { get { return m_page.Size; } } - - public byte[] Data - { - get { return m_page.GetBytes(); } - } - } - - } - - // HACKHACKHACK - private readonly List m_pages = new List(); - - /// Current page used by the heap - private Page m_current = null; - - /// Default size for each memory page - private readonly uint m_pageSize; - - /// Default pointer alignment - private readonly uint m_alignment = DefaultAlignment; - - /// Total size of memory allocated from this heap - private long m_memoryUsage; - - #region Constructors... - - public UnmanagedMemoryHeap() - : this(0, 0) - { } - - public UnmanagedMemoryHeap(uint pageSize) - : this(pageSize, 0) - { } - - public UnmanagedMemoryHeap(uint pageSize, uint alignment) - { - if (pageSize > (1 << 30)) throw new ArgumentOutOfRangeException("pageSize", "Page size cannot be larger than 1 GB"); - if (pageSize == 0) pageSize = DefaultPageSize; - if (m_alignment == 0) m_alignment = DefaultAlignment; - - m_pageSize = pageSize; - m_alignment = alignment; - } - - #endregion - - #region Public Properties... - - public uint PageSize { get { return m_pageSize; } } - - public int PageCount { get { return m_pages.Count; } } - - internal IReadOnlyList Pages { get { return m_pages; } } - - public uint Alignment { get { return m_alignment; } } - - public long MemoryUsage { get { return m_memoryUsage; } } - - #endregion - - private Page AllocateNewPage(uint pageSize) - { - Page page; - try - { } - finally - { - var handle = IntPtr.Zero; - try - { - Contract.Assert(pageSize <= 1 << 30); - handle = Marshal.AllocHGlobal((int)pageSize); - page = new Page(m_pages.Count, handle, pageSize); - } - catch (Exception) - { - if (handle != IntPtr.Zero) Marshal.FreeHGlobal(handle); - throw; - } - - m_memoryUsage += pageSize; - m_pages.Add(page); - } - return page; - } - - /// Allocate a new slice of unmanaged memory - /// Size (in bytes) of the slice. Must be greater than zero. - /// Slice pointing to the newly allocated memory. - public byte* Allocate(uint size) - { - // even though the caller don't require alignemnt, we still want to align to a multiple of 2 so that at least memory moves/cmps are aligned on a WORD boundary. - return Allocate(size, 2); - } - - public byte* AllocateAligned(uint size) - { - // align using the platform's pointer size (4 on x86, 8 on x64) - return Allocate(size, m_alignment); - } - - private byte* Allocate(uint size, uint align) - { - Contract.Requires(align == 1 || (align & (align - 1)) == 0); // only accept alignemnts that are a power of 2 ! - - if (size == 0) throw new ArgumentOutOfRangeException("size", "Cannot allocate zero bytes"); - - Page page; - if (size > (m_pageSize >> 2)) - { // big data go into its own page - page = AllocateNewPage(size); - } - else - { // use the current page - page = m_current; - if (page == null || !page.CanFit(size, align)) - { // need to allocate a new page - page = AllocateNewPage(m_pageSize); - m_current = page; - } - } - Contract.Assert(page != null && page.Remaining >= size); - - byte* ptr = page.Allocate(size, align); - if (ptr == null) throw new OutOfMemoryException(); - return ptr; - } - - /// Copy the content of an unmanaged slice of memory - /// Slice of unmanaged memory to copy - /// New slice pointing to the copied bytes in the allocator memory - public USlice Memoize(USlice data) - { - return Memoize(data, 1); - } - - /// Copy the content of an unmanaged slice of memory, starting at an aligned address - /// Slice of unmanaged memory to copy - /// New slice pointing to the copied bytes in the allocator memory. The start address should be aligned to either 4 or 8 bytes, depending on the platform architecture. - public USlice MemoizeAligned(USlice data) - { - return Memoize(data, m_alignment); - } - - /// Copy the content of an unmanaged slice of memory, using a specific alignment - /// Slice of unmanaged memory to copy - /// Required memory alignment. MUST BE A POWER OF 2 ! - /// New slice pointing to the copied bytes in the allocator memory. The start address should be aligned to either 4 or 8 bytes, depending on the platform architecture. - private USlice Memoize(USlice data, uint align) - { - if (data.Count == 0) return default(USlice); - byte* ptr = Allocate(data.Count, align); - if (ptr == null) throw new OutOfMemoryException(); - UnmanagedHelpers.CopyUnsafe(ptr, data); - return new USlice(ptr, data.Count); - } - - public USlice Memoize(Slice data) - { - return Memoize(data, 1); - } - - public USlice MemoizeAligned(Slice data) - { - return Memoize(data, m_alignment); - } - - private USlice Memoize(Slice data, uint align) - { - if (data.Count < 0 || data.Offset < 0) throw new InvalidOperationException("Cannot allocate less than zero bytes"); - if (data.Count == 0) return default(USlice); - byte* ptr = Allocate((uint)data.Count, align); - if (ptr == null) throw new OutOfMemoryException(); - Marshal.Copy(data.Array, data.Offset, new IntPtr(ptr), data.Count); - return new USlice(ptr, (uint)data.Count); - } - - public void Dispose() - { - foreach (var page in m_pages) - { - if (page.Alive) page.Dispose(); - } - m_pages.Clear(); - - GC.SuppressFinalize(this); - } - - public void Dump(bool detailed = false) - { - Console.WriteLine("Dumping arena state:"); - long used = 0; - foreach (var page in m_pages) - { - Console.WriteLine("- Page #" + page.Id + " (Used=" + page.Used + " / " + page.Size + ", " + (page.Remaining * 100.0 / page.Size).ToString("N1") + "% free)"); - used += page.Used; - var data = page.GetBytes(); - if (detailed) - { - var sb = new StringBuilder(">"); - var txt = detailed ? new StringBuilder(32) : null; - for (int i = 0; i < data.Length; i++) - { - byte b = data[i]; - sb.Append(' ').Append(b.ToString("X2")); - if (detailed) txt.Append(b < 32 || b >= 254 ? '.' : (char)b); - - if (i % 32 == 31) - { - if (detailed) sb.Append("\t").Append(txt.ToString()); - txt.Clear(); - sb.Append("\r\n>"); - } - } - Console.WriteLine(sb.ToString()); - } - } - Console.WriteLine("> Memory usage: " + m_memoryUsage.ToString("N0") + " total, " + used.ToString("N0") + " used"); - } - - public void DumpToDisk(string path) - { - path = System.IO.Path.GetFullPath(path); - Console.WriteLine("> Dumping heap content on disk ({0} bytes): {1}", m_memoryUsage, path); - using (var fs = new System.IO.FileStream(path, System.IO.FileMode.Create, System.IO.FileAccess.Write, System.IO.FileShare.ReadWrite, 4096, System.IO.FileOptions.None)) - { - foreach (var page in m_pages) - { - var data = page.GetBytes(); - fs.Write(data, 0, data.Length); - } - } - } - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilder.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilder.cs deleted file mode 100644 index 66cc22d79..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilder.cs +++ /dev/null @@ -1,476 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using FoundationDB.Client; - using System; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Runtime.InteropServices; - - /// Unmanaged slice builder backed by a pinned managed buffer - /// This class is not thread-safe. - [DebuggerDisplay("Count={m_count}, Capacity={m_buffer.Length}"), DebuggerTypeProxy(typeof(UnmanagedSliceBuilder.DebugView))] - public unsafe sealed class UnmanagedSliceBuilder : IDisposable - { - private static readonly byte[] s_empty = new byte[0]; - - //TODO: define a good default value for this. - public const uint DEFAULT_CAPACITY = 1024; - - /// Managed buffer used to store the values - private byte[] m_buffer; - /// Pinned address of the buffer - private byte* m_data; - /// Number of bytes currently written to the buffer - private uint m_count; - /// GC handle used to pin the managed buffer - private GCHandle m_handle; - - #region Constuctors... - - public UnmanagedSliceBuilder() - : this(0) - { } - - public UnmanagedSliceBuilder(uint capacity) - { - if (capacity == 0) - { - m_buffer = s_empty; - } - else - { - GrowBuffer(capacity); - } - } - - public UnmanagedSliceBuilder(USlice slice) - : this(slice.Data, slice.Count) - { } - - public UnmanagedSliceBuilder(Slice slice) - { - if (slice.Count < 0 || slice.Offset < 0) ThrowMalformedManagedSlice(); - - uint size = (uint)slice.Count; - if (size > 0) - { - if (slice.Array == null || slice.Array.Length < slice.Offset + slice.Count) ThrowMalformedManagedSlice(); - GrowBuffer(size); - - fixed (byte* ptr = slice.Array) - { - UnmanagedHelpers.CopyUnsafe(this.Data, ptr + slice.Offset, size); - } - m_count = size; - } - } - - private static void ThrowMalformedManagedSlice() - { - throw new ArgumentException("Malformed slice", "slice"); - } - - public UnmanagedSliceBuilder(byte* data, uint size) - { - if (data == null && size != 0) throw new ArgumentNullException(nameof(data)); - if (size == 0) - { - m_buffer = s_empty; - } - else - { - GrowBuffer(size); - UnmanagedHelpers.CopyUnsafe(m_data, data, size); - m_count = size; - } - } - - ~UnmanagedSliceBuilder() - { - Dispose(false); - } - - #endregion - - #region Public Properties... - - /// Gets the managed buffer - public byte[] Buffer - { - get { return m_buffer; } - } - - /// Gets a pointer to the first byte in the buffer - public byte* Data - { - get { return m_data; } - } - - /// Gets the number of bytes written to the buffer - public uint Count - { - get { return m_count; } - } - - /// Checks if the builder is empty. - public bool Empty - { - get { return m_count == 0; } - } - - /// Gets the current capacity of the buffer - public uint Capacity - { - get { return m_buffer == null ? 0U : (uint)m_buffer.Length; } - } - - /// Gets or sets the byte at the specified offset - /// Offset from the start of the buffer (0-based) - /// Value of the byte at this offset - /// if is outside the current size of the buffer - public byte this[uint offset] - { - get - { - if (offset >= m_count) ThrowIndexOutOfRange(); - return this.Data[offset]; - } - set - { - if (offset >= m_count) ThrowIndexOutOfRange(); - this.Data[offset] = value; - } - } - - #endregion - - /// Grow the buffer to be able to hold the specified number of bytes - /// Minimum capacity required - /// The buffer may be resize to more than - private void GrowBuffer(uint required) - { - try - { } - finally - { - if (!m_handle.IsAllocated) - { // initial allocation of the buffer - uint newsize = UnmanagedHelpers.NextPowerOfTwo(Math.Max(required, DEFAULT_CAPACITY)); - var buffer = new byte[newsize]; - m_buffer = buffer; - m_count = 0; - } - else - { // resize an existing buffer - uint newsize = (uint)m_buffer.Length; - newsize = UnmanagedHelpers.NextPowerOfTwo(Math.Max(required, newsize << 1)); - if (newsize > int.MaxValue) - { // cannot alloc more than 2GB in managed code! - newsize = int.MaxValue; - if (newsize < required) throw new OutOfMemoryException("Cannot grow slice builder above 2GB"); - } - // temporary release the handle - m_data = null; - m_handle.Free(); - // resize to the new capacity, and re-pin - Array.Resize(ref m_buffer, (int)newsize); - } - m_handle = GCHandle.Alloc(m_buffer, GCHandleType.Pinned); - m_data = (byte*)m_handle.AddrOfPinnedObject(); - } - Contract.Ensures(m_buffer != null && m_handle.IsAllocated && m_data != null && m_count >= 0 && m_count <= m_buffer.Length, "GrowBuffer corruption"); - } - - public void Clear() - { - if (m_buffer == null) ThrowAlreadyDisposed(); - m_count = 0; - } - - private byte* AllocateInternal(uint size, bool zeroed) - { - if (m_buffer == null) ThrowAlreadyDisposed(); - Contract.Requires(size != 0, "size == 0"); - - Contract.Assert(m_buffer != null && m_count <= m_buffer.Length, "Builder is corrupted"); - uint remaining = checked(((uint)m_buffer.Length) - m_count); - if (remaining < size) - { - GrowBuffer(m_count + size); - } - - uint pos = m_count; - m_count = pos + size; - byte* ptr = this.Data + pos; - if (zeroed) UnmanagedHelpers.FillUnsafe(ptr, size, 0); - return ptr; - } - - public USlice Allocate(uint size, bool zeroed = false) - { - if (size == 0) return default(USlice); - return new USlice(AllocateInternal(size, zeroed), size); - } - - public void Append(byte* source, uint size) - { - if (size == 0) return; - if (source == null) ThrowInvalidSource(); - - byte* ptr = AllocateInternal(size, zeroed: false); - Contract.Assert(ptr != null, "AllocateInternal() => null"); - UnmanagedHelpers.CopyUnsafe(ptr, source, size); - } - - public void Append(USlice source) - { - if (source.Count == 0) return; - if (source.Data == null) ThrowInvalidSource(); - - byte* ptr = AllocateInternal(source.Count, zeroed: false); - Contract.Assert(ptr != null); - UnmanagedHelpers.CopyUnsafe(ptr, source); - } - - public void Append(Slice source) - { - if (source.Count > 0) - { - if (source.Array == null || source.Offset < 0) ThrowInvalidSource(); - - var ptr = AllocateInternal((uint)source.Count, zeroed: false); - Contract.Assert(ptr != null, "AllocateInternal() => null"); - UnmanagedHelpers.CopyUnsafe(ptr, source); - } - } - - public void Set(USlice source) - { - m_count = 0; - if (source.Count > 0) - { - if (source.Data == null) ThrowInvalidSource(); - - var ptr = AllocateInternal(source.Count, zeroed: false); - Contract.Assert(ptr != null); - UnmanagedHelpers.CopyUnsafe(ptr, source); - } - } - - public void Set(Slice source) - { - m_count = 0; - if (source.Count > 0) - { - if (source.Array == null || source.Offset < 0) ThrowInvalidSource(); - - var ptr = AllocateInternal((uint)source.Count, zeroed: false); - Contract.Assert(ptr != null); - UnmanagedHelpers.CopyUnsafe(ptr, source); - } - } - - public void Resize(uint newSize, byte filler) - { - if (m_buffer == null) ThrowAlreadyDisposed(); - if (newSize <= m_count) - { - m_count = newSize; - } - else - { - if (newSize > m_buffer.Length) GrowBuffer(newSize); - - // fill the extra space with zeroes - uint pos = m_count; - uint r = checked((uint)m_buffer.Length - newSize); - if (r > 0) - { - UnmanagedHelpers.FillUnsafe(this.Data + pos, r, 0); - } - m_count = newSize; - } - } - - public void Swap(UnmanagedSliceBuilder other) - { - if (other == null) throw new ArgumentNullException("other"); - if (m_buffer == null || other.m_buffer == null) ThrowAlreadyDisposed(); - - try - { } - finally - { - var handle = other.m_handle; - var buffer = other.m_buffer; - var data = other.m_data; - var sz = other.m_count; - - other.m_handle = m_handle; - other.m_buffer = buffer; - other.m_data = m_data; - other.m_count = m_count; - - m_handle = handle; - m_buffer = buffer; - m_data = data; - m_count = sz; - } - } - - /// Gets the current content of the buffer as a managed slice - /// Slice that points to the content of the buffer. - /// Caution: do NOT use the returned slice after the buffer has been changed (it can get relocated during a resize) - public Slice ToSlice() - { - if (m_buffer == null) ThrowAlreadyDisposed(); - return m_count > 0 ? Slice.Create(m_buffer, 0, (int)m_count) : default(Slice); - } - - /// Gets the current content of the buffer as an unmanaged slice - /// Slice that points to the content of the buffer. - /// Caution: do NOT use the returned slice after the buffer has been changed (it can get relocated during a resize) - public USlice ToUSlice() - { - if (m_buffer == null) ThrowAlreadyDisposed(); - return m_count > 0 ? new USlice(m_data, m_count) : default(USlice); - } - - /// Gets the a segment of the buffer as an unmanaged slice - /// Number of bytes (from the start) to return - /// Slice that points to the specified segment of the buffer. - /// Caution: do NOT use the returned slice after the buffer has been changed (it can get relocated during a resize) - public USlice ToUSlice(uint count) - { - return ToUSlice(0, count); - } - - /// Gets the a segment of the buffer as an unmanaged slice - /// Offset from the start of the buffer - /// Number of bytes to return - /// Slice that points to the specified segment of the buffer. - /// Caution: do NOT use the returned slice after the buffer has been changed (it can get relocated during a resize) - public USlice ToUSlice(uint offset, uint count) - { - if (m_buffer == null) ThrowAlreadyDisposed(); - if (offset > m_count) throw new ArgumentOutOfRangeException("offset"); - if (count == 0) return default(USlice); - if (offset + count > m_count) throw new ArgumentOutOfRangeException("count"); - - return new USlice(m_data + offset, count); - } - - /// Copy the content of the buffer to an unmanaged pointer, and return the corresponding slice - /// Destination pointer where the buffer will be copied. Caution: the destination buffer must be large enough! - /// Slice that points to the copied segment in the destination buffer - internal USlice CopyTo(byte* dest) - { - return CopyTo(dest, m_count); - } - - /// Copy a segment of the buffer to an unmanaged pointer, and return the corresponding slice - /// Number of bytes to copy - /// Destination pointer where the buffer will be copied. Caution: the destination buffer must be large enough! - /// Slice that points to the copied segment in the destination buffer - internal USlice CopyTo(byte* dest, uint count) - { - if (m_buffer == null) ThrowAlreadyDisposed(); - if (count == 0) return default(USlice); - if (count > m_count) throw new ArgumentOutOfRangeException("count"); - - UnmanagedHelpers.CopyUnsafe(dest, m_data, count); - return new USlice(dest, count); - } - - public byte[] GetBytes() - { - if (m_buffer == null) ThrowAlreadyDisposed(); - - var tmp = new byte[m_count]; - if (m_count >= 0) - { - fixed (byte* ptr = tmp) - { - UnmanagedHelpers.CopyUnsafe(ptr, m_data, m_count); - } - } - return tmp; - } - - private static void ThrowIndexOutOfRange() - { - throw new IndexOutOfRangeException(); - } - - private void ThrowAlreadyDisposed() - { - throw new ObjectDisposedException(this.GetType().Name); - } - - private void ThrowInvalidSource() - { - throw new ArgumentException("The source memory location is invalid"); - } - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - private void Dispose(bool disposing) - { - if (disposing) - { - if (m_handle.IsAllocated) - { - m_handle.Free(); - } - - } - m_data = null; - m_buffer = null; - m_count = 0; - } - - private sealed class DebugView - { - private readonly UnmanagedSliceBuilder m_builder; - - public DebugView(UnmanagedSliceBuilder builder) - { - m_builder = builder; - } - - public byte[] Data - { - get - { - if (m_builder.m_count == 0) return s_empty; - var buffer = m_builder.m_buffer; - if (buffer == null) return null; - var tmp = new byte[m_builder.Count]; - System.Buffer.BlockCopy(m_builder.m_buffer, 0, tmp, 0, tmp.Length); - return tmp; - } - } - - public uint Count - { - get { return m_builder.m_count; } - } - - public uint Capacity - { - get { return m_builder.m_buffer == null ? 0U : (uint)m_builder.m_buffer.Length; } - } - - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilderPool.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilderPool.cs deleted file mode 100644 index 4341ad4c8..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceBuilderPool.cs +++ /dev/null @@ -1,160 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Diagnostics.Contracts; - using System.Threading; - - [DebuggerDisplay("Count={m_buckets.Count}, Used={m_memoryUsed}, Loaned={m_memoryLoaned}")] - public class UnmanagedSliceBuilderPool : IDisposable - { - private readonly Stack m_buckets; - private uint m_initialCapacity; - private int m_maxCount; - private long m_memoryUsed; - private long m_memoryLoaned; - private bool m_disposed; - - public UnmanagedSliceBuilderPool(uint initialCapacity, int maxCount) - { - m_initialCapacity = UnmanagedHelpers.NextPowerOfTwo(Math.Min(initialCapacity, 64)); - m_maxCount = Math.Max(1, maxCount); - m_buckets = new Stack(Math.Max(m_maxCount, 100)); - } - - /// Subscription to a scratch buffer from the pool. DO NOT COPY BY VALUE! - /// Copying this struct by value will break the pool. Only use it as a local variable in a method or in a class ! - public struct Subscription : IDisposable - { - private readonly UnmanagedSliceBuilderPool m_pool; - private UnmanagedSliceBuilder m_builder; - - internal Subscription(UnmanagedSliceBuilderPool pool, UnmanagedSliceBuilder builder) - { - Contract.Requires(pool != null && builder != null); - m_pool = pool; - m_builder = builder; - } - - public UnmanagedSliceBuilder Builder - { - get - { - Contract.Assert(m_builder != null, "Builder already returned to the pool"); - return m_builder; - } - } - - public bool Allocated - { - get { return m_builder != null; } - } - - public void Dispose() - { -#pragma warning disable 420 - var builder = Interlocked.Exchange(ref m_builder, null); -#pragma warning restore 420 - if (builder != null && builder.Buffer != null) - { - m_pool.Return(builder); - } - } - } - - /// Borrow a builder from this pool - /// Builder subscription that should be disposed as soon as the buffer is not needed anymore - /// ALWAYS wrap the subscription in a using(...) statement! Do NOT pass the subscription by value, always pass the Builder by reference ! Do NOT keep a reference on the Builder or reuse it after it has been disposed! Do NOT return or store slices that point to this buffer! - public Subscription Use() - { - UnmanagedSliceBuilder builder = null; - lock (m_buckets) - { - if (m_disposed) ThrowDisposed(); - - while(m_buckets.Count > 0) - { - builder = m_buckets.Pop(); - if (builder != null && builder.Buffer != null) - { - Interlocked.Add(ref m_memoryUsed, -((long)builder.Capacity)); - Contract.Assert(m_memoryUsed >= 0, "m_memoryUsed desync"); - break; - } - builder = null; - } - } - if (builder == null) - { - builder = new UnmanagedSliceBuilder(m_initialCapacity); - } - Interlocked.Add(ref m_memoryLoaned, builder.Capacity); - Contract.Assert(builder != null && builder.Buffer != null); - return new Subscription(this, builder); - } - - /// Return a builder into the pool - /// Builder that is no longer in use - internal void Return(UnmanagedSliceBuilder builder) - { - if (m_disposed || builder == null) return; - - lock (m_buckets) - { - if (m_disposed) return; - - var size = builder.Capacity; - Contract.Assert(size == UnmanagedHelpers.NextPowerOfTwo(size), "builder size should always be a power of two"); - - Interlocked.Add(ref m_memoryLoaned, -((long)builder.Capacity)); - Contract.Assert(m_memoryUsed >= 0, "m_memoryLoaned desync"); - - if (m_buckets.Count < m_maxCount) - { - m_buckets.Push(builder); - Interlocked.Add(ref m_memoryUsed, builder.Capacity); - } - else - { - builder.Dispose(); - } - } - } - - private static void ThrowDisposed() - { - throw new InvalidOperationException("The buffer pool as already been disposed"); - } - - public void Dispose() - { - Dispose(true); - GC.SuppressFinalize(this); - } - - protected virtual void Dispose(bool disposing) - { - if (!m_disposed) - { - m_disposed = true; - if (disposing) - { - lock (m_buckets) - { - foreach(var builder in m_buckets) - { - if (builder != null) builder.Dispose(); - } - m_buckets.Clear(); - } - } - } - } - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceReader.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedSliceReader.cs deleted file mode 100644 index d08557912..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceReader.cs +++ /dev/null @@ -1,305 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013, Doxense SARL -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 - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Diagnostics.Contracts; - - /// Helper class that holds the internal state used to parse tuples from slices - public unsafe class UnmanagedSliceReader - { - - /// Creates a reader on a byte array - public static UnmanagedSliceReader FromSlice(USlice slice) - { - return new UnmanagedSliceReader(slice.Data, slice.Count); - } - - /// Creates a reader on a segment of a byte array - public static UnmanagedSliceReader FromAddress(byte* address, ulong count) - { - if (address == null && count != 0) throw new ArgumentException("Address cannot be null"); - return new UnmanagedSliceReader(address, count); - } - - /// Buffer containing the tuple being parsed - public readonly byte* Base; - - /// Current position inside the buffer - public byte* Position; - - /// Memory address just after the end of the buffer - public readonly byte* End; - - private UnmanagedSliceReader(byte* address, ulong count) - { - Contract.Requires(address != null || count == 0); - - this.Base = address; - this.Position = address; - this.End = address + count; - - Contract.Ensures(this.End >= this.Base && this.Position >= this.Base && this.Position <= this.End); - } - - public ulong Offset { get { return this.Position > this.Base ? (ulong)(this.Position - this.Base) : 0UL; } } - - public ulong Length { get { return (ulong)(this.End - this.Base); } } - - /// Returns true if there are more bytes to parse - public bool HasMore { get { return this.Position < this.End; } } - - /// Returns the number of bytes remaining - public ulong Remaining { get { return this.Position < this.End ? (ulong)(this.End - this.Position) : 0UL; } } - - /// Ensure that there are at least bytes remaining in the buffer - public void EnsureBytes(uint count) - { - if (checked(this.Position + count) > this.End) throw new ArgumentOutOfRangeException("count"); - } - - /// Return the value of the next byte in the buffer, or -1 if we reached the end - public int PeekByte() - { - byte* p = this.Position; - return p < this.End ? (*p) : -1; - } - - /// Skip the next bytes of the buffer - public void Skip(uint count) - { - EnsureBytes(count); - - this.Position += count; - } - - /// Read the next byte from the buffer - public byte ReadByte() - { - EnsureBytes(1); - - byte* p = this.Position; - byte b = *p; - this.Position = checked(p + 1); - return b; - } - - /// Read the next bytes from the buffer - public USlice ReadBytes(uint count) - { - EnsureBytes(count); - - byte* p = this.Position; - this.Position = checked(p + count); - return new USlice(p, count); - } - - /// Read the next 2 bytes as an unsigned 16-bit integer, encoded in little-endian - public ushort ReadFixed16() - { - EnsureBytes(2); - byte* p = this.Position; - this.Position = checked(p + 2); - return (ushort)(p[0] | p[1] << 8); - } - - /// Read the next 4 bytes as an unsigned 32-bit integer, encoded in little-endian - public uint ReadFixed32() - { - EnsureBytes(4); - byte* p = this.Position; - this.Position = checked(p + 4); - return p[0] | (uint)p[1] << 8 | (uint)p[2] << 16 | (uint)p[3] << 24; - } - - /// Read the next 8 bytes as an unsigned 64-bit integer, encoded in little-endian - public ulong ReadFixed64() - { - EnsureBytes(8); - byte* p = this.Position; - this.Position = checked(p + 8); - return p[0] | (ulong)p[1] << 8 | (ulong)p[2] << 16 | (ulong)p[3] << 24 | (ulong)p[4] << 32 | (ulong)p[5] << 40 | (ulong)p[6] << 48 | (ulong)p[7] << 56; - } - - /// Reads a 7-bit encoded unsigned int (aka 'Varint16') from the buffer, and advances the cursor - /// Can read up to 3 bytes from the input - public ushort ReadVarint16() - { - byte* p = this.Position; - byte* end = this.End; - uint n = 1; - - if (p >= end) goto overflow; - uint b = p[0]; - uint res = b & 0x7F; - if (res < 0x80) { goto done; } - - if (p >= end) goto overflow; - b = p[1]; - res |= (b & 0x7F) << 7; - if (b < 0x80) { n = 2; goto done; } - - // the third byte should only have 2 bits worth of data - if (p >= end) goto overflow; - b = p[2]; - if (b >= 0x4) throw new FormatException("Varint is bigger than 16 bits"); - res |= (b & 0x2) << 14; - n = 3; - - done: - this.Position = checked(p + n); - return (ushort)res; - - overflow: - throw new FormatException("Truncated Varint"); - } - - /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor - /// Can read up to 5 bytes from the input - public uint ReadVarint32() - { - byte* p = this.Position; - byte* end = this.End; - uint n = 1; - - if (p >= end) goto overflow; - uint b = p[0]; - uint res = b & 0x7F; - if (res < 0x80) { goto done; } - - if (p >= end) goto overflow; - b = p[1]; - res |= (b & 0x7F) << 7; - if (b < 0x80) { n = 2; goto done; } - - if (p >= end) goto overflow; - b = p[2]; - res |= (b & 0x7F) << 14; - if (b < 0x80) { n = 3; goto done; } - - if (p >= end) goto overflow; - b = p[3]; - res |= (b & 0x7F) << 21; - if (b < 0x80) { n = 4; goto done; } - - // the fifth byte should only have 4 bits worth of data - if (p >= end) goto overflow; - b = p[4]; - if (b >= 0x20) throw new FormatException("Varint is bigger than 32 bits"); - res |= (b & 0x1F) << 28; - n = 5; - - done: - this.Position = checked(p + n); - return res; - - overflow: - throw new FormatException("Truncated Varint"); - } - - /// Reads a 7-bit encoded unsigned long (aka 'Varint32') from the buffer, and advances the cursor - /// Can read up to 10 bytes from the input - public ulong ReadVarint64() - { - byte* p = this.Position; - byte* end = this.End; - uint n = 1; - - if (p >= end) goto overflow; - uint b = p[0]; - ulong res = b & 0x7F; - if (res < 0x80) { goto done; } - - if (p >= end) goto overflow; - b = p[1]; - res |= (b & 0x7F) << 7; - if (b < 0x80) { n = 2; goto done; } - - if (p >= end) goto overflow; - b = p[2]; - res |= (b & 0x7F) << 14; - if (b < 0x80) { n = 3; goto done; } - - if (p >= end) goto overflow; - b = p[3]; - res |= (b & 0x7F) << 21; - if (b < 0x80) { n = 4; goto done; } - - if (p >= end) goto overflow; - b = p[4]; - res |= (b & 0x7F) << 28; - if (b < 0x80) { n = 5; goto done; } - - if (p >= end) goto overflow; - b = p[5]; - res |= (b & 0x7F) << 35; - if (b < 0x80) { n = 6; goto done; } - - if (p >= end) goto overflow; - b = p[6]; - res |= (b & 0x7F) << 42; - if (b < 0x80) { n = 7; goto done; } - - if (p >= end) goto overflow; - b = p[7]; - res |= (b & 0x7F) << 49; - if (b < 0x80) { n = 8; goto done; } - - if (p >= end) goto overflow; - b = p[8]; - res |= (b & 0x7F) << 56; - if (b < 0x80) { n = 9; goto done; } - - // the tenth byte should only have 1 bit worth of data - if (p >= end) goto overflow; - b = p[4]; - if (b > 1) throw new FormatException("Varint is bigger than 64 bits"); - res |= (b & 0x1) << 63; - n = 10; - - done: - this.Position = checked(p + n); - return res; - - overflow: - throw new FormatException("Truncated Varint"); - } - - /// Reads a variable sized slice, by first reading its size (stored as a Varint32) and then the data - public USlice ReadVarbytes() - { - uint size = ReadVarint32(); - if (size > uint.MaxValue) throw new FormatException("Malformed variable size"); - if (size == 0) return USlice.Nil; - return ReadBytes(size); - } - - } - -} diff --git a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceStream.cs b/FoundationDB.Storage.Memory/Utils/UnmanagedSliceStream.cs deleted file mode 100644 index 03bac62b9..000000000 --- a/FoundationDB.Storage.Memory/Utils/UnmanagedSliceStream.cs +++ /dev/null @@ -1,228 +0,0 @@ -#region Copyright (c) 2013-2014, Doxense SAS. All rights reserved. -// See License.MD for license information -#endregion - -namespace FoundationDB.Storage.Memory.Utils -{ - using System; - using System.Diagnostics.Contracts; - using System.IO; - using System.Runtime.InteropServices; - using System.Text; - using System.Threading; - using System.Threading.Tasks; - - /// Stream that can read from a slice of unmanaged memory - public unsafe sealed class UnmanagedSliceStream : Stream - { - private byte* m_begin; - private uint m_pos; - private readonly uint m_size; - private Task m_lastReadTask; - - internal UnmanagedSliceStream(USlice slice) - { - Contract.Requires(slice.Count == 0 || slice.Data != null); - - m_begin = slice.Data; - m_size = slice.Count; - } - - internal UnmanagedSliceStream(byte* data, uint size) - { - Contract.Requires(size == 0 || data != null); - - m_begin = data; - m_size = size; - } - - public override bool CanRead - { - get { return m_begin != null; } - } - - public override bool CanSeek - { - get { return true; } - } - - public override bool CanWrite - { - get { return false; } - } - - public override void Flush() - { - //NO OP - } - - public override Task FlushAsync(CancellationToken cancellationToken) - { - return TaskHelpers.CompletedTask; - } - - public override long Length - { - get { return m_size; } - } - - public override long Position - { - get - { - return m_pos; - } - set - { - Seek(value, SeekOrigin.Begin); - } - } - - public override int ReadByte() - { - if (m_begin == null) ThrowDisposed(); - uint pos = m_pos; - if (pos < m_size) - { - int res = (int)m_begin[pos]; - m_pos = pos + 1; - return res; - } - return -1; - } - - public override int Read(byte[] buffer, int offset, int count) - { - if (m_begin == null) ThrowDisposed(); - - if (buffer == null) throw new ArgumentNullException("buffer"); - if (offset < 0 || offset > buffer.Length) throw new ArgumentOutOfRangeException("offset"); - if (count < 0 || offset + count >= buffer.Length) throw new ArgumentOutOfRangeException("count"); - - uint pos = m_pos; - if (pos >= m_size) return 0; // EOF - - uint chunk; - checked { chunk = (uint)Math.Max(m_size - pos, count); } - - if (chunk > 0) - { - fixed (byte* ptr = buffer) - { - UnmanagedHelpers.CopyUnsafe(ptr + offset, m_begin + pos, chunk); - } - m_pos = pos + chunk; - } - return (int)chunk; - } - - public override Task ReadAsync(byte[] buffer, int offset, int count, CancellationToken cancellationToken) - { - if (cancellationToken.IsCancellationRequested) - { - return TaskHelpers.FromCancellation(cancellationToken); - } - try - { - int result = Read(buffer, offset, count); - var t = m_lastReadTask; - return t != null && t.Result == result ? t : (t = Task.FromResult(result)); - } - catch (Exception e) - { - return TaskHelpers.FromException(e); - } - } - - public override long Seek(long offset, SeekOrigin origin) - { - if (m_begin == null) ThrowDisposed(); - - switch (origin) - { - case SeekOrigin.Begin: - { - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); - offset = offset >= m_size ? m_size : offset; - Contract.Assert(offset >= 0); - m_pos = (uint)offset; - return m_pos; - } - case SeekOrigin.End: - { - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); - offset += m_size; - offset = offset < 0 ? 0 : offset; - Contract.Assert(offset >= 0); - m_pos = (uint)offset; - return m_pos; - } - case SeekOrigin.Current: - { - offset += m_pos; - offset = offset < 0 ? 0 : offset >= m_size ? m_size : offset; - Contract.Assert(offset >= 0); - m_pos = (uint)offset; - return m_pos; - } - default: - { - throw new ArgumentOutOfRangeException("origin"); - } - } - } - - public override void SetLength(long value) - { - throw new NotSupportedException("Cannot set the length of a read-only stream"); - } - - public override void Write(byte[] buffer, int offset, int count) - { - throw new NotSupportedException("Cannot write to a read-only stream"); - } - - public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken cancellationToken) - { - return TaskHelpers.FromException(new NotSupportedException("Cannot write to a read-only stream")); - } - - public byte[] ToArray() - { - if (m_begin == null) ThrowDisposed(); - var tmp = new byte[m_size]; - if (tmp.Length > 0) - { - fixed (byte* ptr = tmp) - { - UnmanagedHelpers.CopyUnsafe(ptr, m_begin, (uint)m_size); - } - } - return tmp; - } - - public FoundationDB.Client.Slice ToSlice() - { - return FoundationDB.Client.Slice.Create(this.ToArray()); - } - - public USlice ToUSlice() - { - if (m_begin == null) ThrowDisposed(); - return new USlice(m_begin, m_size); - } - - private void ThrowDisposed() - { - throw new ObjectDisposedException(this.GetType().Name); - } - - protected override void Dispose(bool disposing) - { - m_begin = null; - m_pos = m_size; - m_lastReadTask = null; - } - } - -} diff --git a/FoundationDB.Storage.Memory/packages.config b/FoundationDB.Storage.Memory/packages.config deleted file mode 100644 index 835f33a70..000000000 --- a/FoundationDB.Storage.Memory/packages.config +++ /dev/null @@ -1,4 +0,0 @@ - - - - \ No newline at end of file From 833b45878e531cd2886b910aeebdcb68dc0d7a4f Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 12:36:29 +0200 Subject: [PATCH 055/153] Renamed IAsyncEnumerator.MoveNext into MoveNextAsync (cf Issue #64) --- FoundationDB.Client/Async/IAsyncEnumerator.cs | 2 +- FoundationDB.Client/FdbQueryMergeIterator.cs | 4 ++-- .../FdbRangeQuery.ResultIterator.cs | 2 +- .../Linq/FdbAsyncEnumerable.EmptySequence.cs | 4 ++-- .../FdbAsyncEnumerable.EnumerableIterator.cs | 2 +- .../Linq/FdbAsyncEnumerable.Iterators.cs | 12 +++++----- .../FdbAsyncEnumerable.OrderedSequence.cs | 4 ++-- .../Linq/FdbAsyncEnumerable.cs | 12 +++++----- .../Linq/Iterators/FdbAsyncIterator.cs | 2 +- .../Linq/Iterators/FdbAsyncIteratorPump.cs | 2 +- .../Iterators/FdbBatchingAsyncIterator.cs | 4 ++-- .../Iterators/FdbDistinctAsyncIterator.cs | 6 ++--- .../Linq/Iterators/FdbObserverIterator.cs | 2 +- .../Iterators/FdbPrefetchingAsyncIterator.cs | 4 ++-- .../Iterators/FdbSelectManyAsyncIterator.cs | 4 ++-- .../Iterators/FdbTakeWhileAsyncIterator.cs | 2 +- .../Linq/Iterators/FdbWhereAsyncIterator.cs | 10 ++++----- .../Iterators/FdbWhereSelectAsyncIterator.cs | 6 ++--- .../Iterators/FdbWindowingAsyncIterator.cs | 4 ++-- .../Providers/FdbAsyncQuery.cs | 4 ++-- .../Linq/FdbAsyncEnumerableFacts.cs | 22 +++++++++---------- 21 files changed, 57 insertions(+), 57 deletions(-) diff --git a/FoundationDB.Client/Async/IAsyncEnumerator.cs b/FoundationDB.Client/Async/IAsyncEnumerator.cs index 0db0e3564..f7ae14e81 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerator.cs +++ b/FoundationDB.Client/Async/IAsyncEnumerator.cs @@ -45,7 +45,7 @@ public interface IAsyncEnumerator : IDisposable /// Task containing the result of the operation: true if the enumerator was successfully advanced /// to the next element; false if the enumerator has passed the end of the sequence. /// - Task MoveNext(CancellationToken cancellationToken); + Task MoveNextAsync(CancellationToken cancellationToken); /// Gets the current element in the iteration. T Current { get; } diff --git a/FoundationDB.Client/FdbQueryMergeIterator.cs b/FoundationDB.Client/FdbQueryMergeIterator.cs index 1205aed78..979f15edf 100644 --- a/FoundationDB.Client/FdbQueryMergeIterator.cs +++ b/FoundationDB.Client/FdbQueryMergeIterator.cs @@ -99,7 +99,7 @@ protected override Task OnFirstAsync(CancellationToken cancellationToken) var state = new IteratorState(); state.Active = true; state.Iterator = sources[i].GetEnumerator(mode); - state.Next = state.Iterator.MoveNext(cancellationToken); + state.Next = state.Iterator.MoveNextAsync(cancellationToken); iterators[i] = state; } @@ -185,7 +185,7 @@ protected void AdvanceIterator(int index, CancellationToken cancellationToken) { m_iterators[index].HasCurrent = false; m_iterators[index].Current = default(TKey); - m_iterators[index].Next = m_iterators[index].Iterator.MoveNext(cancellationToken); + m_iterators[index].Next = m_iterators[index].Iterator.MoveNextAsync(cancellationToken); } private static void Cleanup(IteratorState[] iterators) diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 618c8d05b..0270bd5fd 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -126,7 +126,7 @@ private async Task ReadAnotherBatchAsync(CancellationToken cancellationTok var iterator = m_chunkIterator; // start reading the next batch - if (await iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (await iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // we got a new chunk ! //note: Dispose() or Cleanup() maybe have been called concurrently! diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs index be338d299..62aca1e76 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs @@ -45,7 +45,7 @@ private sealed class EmptySequence : IFdbAsyncEnumerable, IFdb private EmptySequence() { } - Task IAsyncEnumerator.MoveNext(CancellationToken cancellationToken) + Task IAsyncEnumerator.MoveNextAsync(CancellationToken cancellationToken) { cancellationToken.ThrowIfCancellationRequested(); return TaskHelpers.FalseTask; @@ -107,7 +107,7 @@ IAsyncEnumerator IAsyncEnumerable.GetEnumerator() return this.GetEnumerator(); } - async Task IAsyncEnumerator.MoveNext(CancellationToken cancellationToken) + async Task IAsyncEnumerator.MoveNextAsync(CancellationToken cancellationToken) { cancellationToken.ThrowIfCancellationRequested(); if (m_called) return false; diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs index 2085bbb61..497928878 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs @@ -56,7 +56,7 @@ public EnumerableIterator(IEnumerator iterator, Func MoveNext(CancellationToken cancellationToken) + public async Task MoveNextAsync(CancellationToken cancellationToken) { if (m_disposed) { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs index 2aaa9b0f3..5af99cd7f 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs @@ -399,7 +399,7 @@ internal static async Task Run( { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNext(ct)) + while (await iterator.MoveNextAsync(ct)) { action(iterator.Current); ++count; @@ -433,7 +433,7 @@ internal static async Task Run( { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNext(ct)) + while (await iterator.MoveNextAsync(ct)) { if (!action(iterator.Current)) { @@ -467,7 +467,7 @@ internal static async Task Run( { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNext(ct)) + while (await iterator.MoveNextAsync(ct)) { await action(iterator.Current, ct); ++count; @@ -498,7 +498,7 @@ internal static async Task Run( { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNext(ct)) + while (await iterator.MoveNextAsync(ct)) { ct.ThrowIfCancellationRequested(); await action(iterator.Current); @@ -529,12 +529,12 @@ internal static async Task Head( { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - if (await iterator.MoveNext(ct)) + if (await iterator.MoveNextAsync(ct)) { TSource first = iterator.Current; if (single) { - if (await iterator.MoveNext(ct)) throw new InvalidOperationException("The sequence contained more than one element"); + if (await iterator.MoveNextAsync(ct)) throw new InvalidOperationException("The sequence contained more than one element"); } return first; } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs index 21b0216a7..d0cfbe70a 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs @@ -171,7 +171,7 @@ private async Task ReadAllThenSort(CancellationToken ct) } else { - while (await inner.MoveNext(ct).ConfigureAwait(false)) + while (await inner.MoveNextAsync(ct).ConfigureAwait(false)) { buffer.Add(inner.Current); } @@ -193,7 +193,7 @@ private async Task ReadAllThenSort(CancellationToken ct) return true; } - public Task MoveNext(CancellationToken cancellationToken) + public Task MoveNextAsync(CancellationToken cancellationToken) { // Firt call will be slow (and async), but the rest of the calls will use the results already sorted in memory, and should be as fast as possible! diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs index 1a9faa67a..b64c3be34 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs @@ -715,13 +715,13 @@ public static IFdbAsyncOrderedEnumerable ThenByDescending( using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) { - return await iterator.MoveNext(ct).ConfigureAwait(false); + return await iterator.MoveNextAsync(ct).ConfigureAwait(false); } } @@ -1067,7 +1067,7 @@ await ForEachAsync( using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) { - while (await iterator.MoveNext(ct).ConfigureAwait(false)) + while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { if (predicate(iterator.Current)) return true; } @@ -1084,7 +1084,7 @@ await ForEachAsync( using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) { - return !(await iterator.MoveNext(ct).ConfigureAwait(false)); + return !(await iterator.MoveNextAsync(ct).ConfigureAwait(false)); } } @@ -1097,7 +1097,7 @@ await ForEachAsync( using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) { - while (await iterator.MoveNext(ct).ConfigureAwait(false)) + while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { if (predicate(iterator.Current)) return false; } diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs index 016d67af5..46aa7dccc 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs @@ -89,7 +89,7 @@ public TResult Current } } - public async Task MoveNext(CancellationToken ct) + public async Task MoveNextAsync(CancellationToken ct) { var state = Volatile.Read(ref m_state); diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs index c377af54a..7a9ea51ad 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs @@ -101,7 +101,7 @@ public async Task PumpAsync(CancellationToken ct) { LogDebug("waiting for next"); m_state = STATE_WAITING_FOR_NEXT; - if (!(await m_iterator.MoveNext(ct).ConfigureAwait(false))) + if (!(await m_iterator.MoveNextAsync(ct).ConfigureAwait(false))) { LogDebug("completed"); m_state = STATE_DONE; diff --git a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs index 4fa05b116..7384ac696 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs @@ -86,14 +86,14 @@ protected override async Task OnNextAsync(CancellationToken ct) var iterator = m_iterator; var buffer = m_buffer; - bool hasMore = await iterator.MoveNext(ct).ConfigureAwait(false); + bool hasMore = await iterator.MoveNextAsync(ct).ConfigureAwait(false); while(hasMore && !ct.IsCancellationRequested) { buffer.Add(iterator.Current); if (buffer.Count >= m_batchSize) break; - hasMore = await iterator.MoveNext(ct).ConfigureAwait(false); + hasMore = await iterator.MoveNextAsync(ct).ConfigureAwait(false); } ct.ThrowIfCancellationRequested(); diff --git a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs index 57678dba7..2f4b74042 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs @@ -68,7 +68,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo { while (!cancellationToken.IsCancellationRequested) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // completed m_set = null; return Completed(); @@ -102,7 +102,7 @@ public override async Task ExecuteAsync(Action handler, CancellationTok { var set = new HashSet(m_comparer); - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (set.Add(current)) @@ -129,7 +129,7 @@ public override async Task ExecuteAsync(Func a { var set = new HashSet(m_comparer); - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (set.Add(current)) diff --git a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs index a19101a92..d32c851dd 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs @@ -56,7 +56,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo { while (!cancellationToken.IsCancellationRequested) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // completed return Completed(); } diff --git a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs index bea0581f4..704844077 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs @@ -99,7 +99,7 @@ protected virtual async Task PrefetchNextItemsAsync(CancellationToken ct) if (t == null) { // read the next item from the inner iterator if (m_innerHasCompleted) return Completed(); - t = m_iterator.MoveNext(ct); + t = m_iterator.MoveNextAsync(ct); } // always wait for the first item (so that we have at least something in the batch) @@ -113,7 +113,7 @@ protected virtual async Task PrefetchNextItemsAsync(CancellationToken ct) if (m_buffer == null) m_buffer = new Queue(m_prefetchCount); m_buffer.Enqueue(m_iterator.Current); - t = m_iterator.MoveNext(ct); + t = m_iterator.MoveNextAsync(ct); if (m_buffer.Count >= m_prefetchCount || !t.IsCompleted) { // save it for next time m_nextTask = t; diff --git a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs index fbf8ef6ce..4d26e9f67 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs @@ -68,7 +68,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo if (m_batch == null) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // inner completed return Completed(); } @@ -160,7 +160,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo if (m_batch == null) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // inner completed return Completed(); } diff --git a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs index efb9dd840..08f674efe 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs @@ -58,7 +58,7 @@ protected override async Task OnNextAsync(CancellationToken ct) { while (!ct.IsCancellationRequested) { - if (!await m_iterator.MoveNext(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed return Completed(); } diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs index acafcb712..a4529d08e 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs @@ -58,7 +58,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo { while (!cancellationToken.IsCancellationRequested) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // completed return Completed(); } @@ -148,7 +148,7 @@ public override async Task ExecuteAsync(Action handler, CancellationTok { if (!m_filter.Async) { - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (m_filter.Invoke(current)) @@ -159,7 +159,7 @@ public override async Task ExecuteAsync(Action handler, CancellationTok } else { - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) @@ -183,7 +183,7 @@ public override async Task ExecuteAsync(Func a { if (!m_filter.Async) { - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (m_filter.Invoke(current)) @@ -194,7 +194,7 @@ public override async Task ExecuteAsync(Func a } else { - while (!ct.IsCancellationRequested && (await iter.MoveNext(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) { var current = iter.Current; if (await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs index 5ea8a7dcc..8d5418c22 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs @@ -89,7 +89,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo while (!cancellationToken.IsCancellationRequested) { - if (!await m_iterator.MoveNext(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) { // completed return Completed(); } @@ -319,7 +319,7 @@ public override async Task ExecuteAsync(Action action, CancellationToke { while (remaining == null || remaining.Value > 0) { - if (!await iterator.MoveNext(ct).ConfigureAwait(false)) + if (!await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed break; } @@ -388,7 +388,7 @@ public override async Task ExecuteAsync(Func a { while (remaining == null || remaining.Value > 0) { - if (!await iterator.MoveNext(ct).ConfigureAwait(false)) + if (!await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed break; } diff --git a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs index 69dd2f4ad..78f0edaef 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs @@ -128,7 +128,7 @@ protected override async Task OnNextAsync(CancellationToken ct) if (t == null) { // read the next item from the inner iterator if (m_innerHasCompleted) return Completed(); - t = iterator.MoveNext(ct); + t = iterator.MoveNextAsync(ct); } // always wait for the first item (so that we have at least something in the batch) @@ -141,7 +141,7 @@ protected override async Task OnNextAsync(CancellationToken ct) { buffer.Add(iterator.Current); - t = iterator.MoveNext(ct); + t = iterator.MoveNextAsync(ct); if (buffer.Count >= m_maxWindowSize || !t.IsCompleted) { // save it for next time diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs index 31c7094bf..b6a3751f1 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs @@ -245,9 +245,9 @@ public TransactionIterator(IFdbTransaction transaction, IFdbAsyncEnumerator i m_iterator = iterator; } - public Task MoveNext(CancellationToken cancellationToken) + public Task MoveNextAsync(CancellationToken cancellationToken) { - return m_iterator.MoveNext(cancellationToken); + return m_iterator.MoveNextAsync(cancellationToken); } public T Current diff --git a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs index 825a9e9e0..babd12852 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs @@ -55,7 +55,7 @@ public async Task Test_Can_Convert_Enumerable_To_AsyncEnumerable() var results = new List(); using (var iterator = source.GetEnumerator()) { - while (await iterator.MoveNext(CancellationToken.None)) + while (await iterator.MoveNextAsync(CancellationToken.None)) { Assert.That(results.Count, Is.LessThan(10)); results.Add(iterator.Current); @@ -80,7 +80,7 @@ public async Task Test_Can_Convert_Enumerable_To_AsyncEnumerable_With_Async_Tran var results = new List(); using (var iterator = source.GetEnumerator()) { - while (await iterator.MoveNext(CancellationToken.None)) + while (await iterator.MoveNextAsync(CancellationToken.None)) { Assert.That(results.Count, Is.LessThan(10)); results.Add(iterator.Current); @@ -175,10 +175,10 @@ public async Task Test_Producer_Single() using(var iterator = singleton.GetEnumerator()) { - var res = await iterator.MoveNext(this.Cancellation); + var res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNext(this.Cancellation); + res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.False); } @@ -201,10 +201,10 @@ public async Task Test_Producer_Single() using (var iterator = singleton.GetEnumerator()) { - var res = await iterator.MoveNext(this.Cancellation); + var res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNext(this.Cancellation); + res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.False); } @@ -227,10 +227,10 @@ public async Task Test_Producer_Single() using (var iterator = singleton.GetEnumerator()) { - var res = await iterator.MoveNext(this.Cancellation); + var res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNext(this.Cancellation); + res = await iterator.MoveNextAsync(this.Cancellation); Assert.That(res, Is.False); } @@ -1119,18 +1119,18 @@ public async Task Test_Exceptions_Are_Propagated_To_Caller() using (var iterator = query.GetEnumerator()) { // first move next should succeed - bool res = await iterator.MoveNext(CancellationToken.None); + bool res = await iterator.MoveNextAsync(CancellationToken.None); Assert.That(res, Is.True); // second move next should fail - var x = Assert.Throws(async () => await iterator.MoveNext(CancellationToken.None), "Should have failed"); + var x = Assert.Throws(async () => await iterator.MoveNextAsync(CancellationToken.None), "Should have failed"); Assert.That(x.Message, Is.EqualTo("KABOOM")); // accessing current should rethrow the exception Assert.That(() => iterator.Current, Throws.InstanceOf()); // another attempt at MoveNext should fail immediately but with a different error - Assert.Throws(async () => await iterator.MoveNext(CancellationToken.None)); + Assert.Throws(async () => await iterator.MoveNextAsync(CancellationToken.None)); } } From a5eba140a28ba118a16c72fad5e9e2a4cf134e77 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 14:44:39 +0200 Subject: [PATCH 056/153] Updated .editorconfig --- .editorconfig | 37 +++++++++++++++++++++++++++++++++++-- 1 file changed, 35 insertions(+), 2 deletions(-) diff --git a/.editorconfig b/.editorconfig index 49f34bd03..4a9719c4b 100644 --- a/.editorconfig +++ b/.editorconfig @@ -1,5 +1,38 @@ -; 4-column tab indentation -[*.cs] +# Editor Config for foundationdb-dotnet-client + +# root +root = true + + +# Global defaults +[*] +end_of_line = crlf +insert_final_newline = true indent_style = tab indent_size = 4 +[*.cs] +csharp_indent_block_contents = true +csharp_indent_braces = false +csharp_indent_case_contents = false +csharp_indent_labels = one_less_than_current +csharp_indent_switch_labels = true +csharp_new_line_before_members_in_anonymous_types = true +csharp_new_line_before_members_in_object_initializers = true +csharp_prefer_braces = true +csharp_preserve_single_line_statements = true +csharp_space_after_cast = true +csharp_space_after_comma = true +csharp_space_after_keywords_in_control_flow_statements = true +csharp_space_between_method_call_name_and_opening_parenthesis = false +csharp_style_conditional_delegate_call = true:suggestion +csharp_style_pattern_matching_over_as_with_null_check = true:suggestion +csharp_style_pattern_matching_over_is_with_cast_check = true:suggestion +csharp_style_throw_expression = true:suggestion +dotnet_sort_system_directives_first = true:suggestion +dotnet_style_coalesce_expression = true:suggestion +dotnet_style_explicit_tuple_names = true:warning +dotnet_style_null_propagation = true:suggestion +dotnet_style_qualification_for_method = false:suggestion + +csharp_style_inlined_variable_declaration = true:suggestion From dcd36b3bc300069f62cea706a184af3dee63d928 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 14:46:04 +0200 Subject: [PATCH 057/153] First pass at merging back the changes that happened outside the repository - Transplanted new Contract and R# annotations - Updated and sorted usings - Added missing contract annotations - Fix misc minor bugs --- FoundationDB.Client/Async/AsyncBuffer.cs | 4 +- FoundationDB.Client/Async/AsyncHelpers.cs | 2 +- FoundationDB.Client/Async/AsyncTaskBuffer.cs | 4 +- .../Async/AsyncTransformQueue.cs | 4 +- FoundationDB.Client/Async/Maybe.cs | 4 +- FoundationDB.Client/Async/TaskHelpers.cs | 4 +- .../Converters/ConversionHelper.cs | 4 +- .../Converters/FdbConverters.cs | 6 +- FoundationDB.Client/Fdb.Bulk.cs | 10 +- FoundationDB.Client/Fdb.Errors.cs | 2 +- FoundationDB.Client/Fdb.System.cs | 13 +- FoundationDB.Client/FdbDatabase.cs | 35 +- FoundationDB.Client/FdbDatabaseExtensions.cs | 138 +- FoundationDB.Client/FdbKey.cs | 12 +- FoundationDB.Client/FdbKeyRange.cs | 2 +- FoundationDB.Client/FdbKeyRangeComparer.cs | 2 +- .../FdbMergeQueryExtensions.cs | 95 +- FoundationDB.Client/FdbOperationContext.cs | 7 +- FoundationDB.Client/FdbQueryMergeIterator.cs | 6 +- FoundationDB.Client/FdbRangeOptions.cs | 2 +- .../FdbRangeQuery.PagingIterator.cs | 9 +- .../FdbRangeQuery.ResultIterator.cs | 13 +- FoundationDB.Client/FdbRangeQuery.cs | 69 +- FoundationDB.Client/FdbTransaction.cs | 13 +- .../FdbTransactionExtensions.cs | 278 ++-- FoundationDB.Client/FdbWatch.cs | 4 +- .../Filters/FdbDatabaseFilter.cs | 10 +- .../Filters/Logging/FdbLoggedTransaction.cs | 4 +- .../Filters/Logging/FdbTransactionLog.cs | 6 +- .../FoundationDB.Client.csproj | 2 + .../Layers/Directories/FdbDirectoryLayer.cs | 23 +- .../Directories/FdbDirectorySubspace.cs | 8 +- .../Directories/FdbHighContentionAllocator.cs | 6 +- .../Layers/Tuples/FdbLinkedTuple.cs | 8 +- .../Layers/Tuples/FdbListTuple.cs | 6 +- .../Layers/Tuples/FdbMemoizedTuple.cs | 6 +- .../Layers/Tuples/FdbPrefixedTuple.cs | 8 +- .../Layers/Tuples/FdbSlicedTuple.cs | 6 +- FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 6 +- .../Layers/Tuples/FdbTuplePackers.cs | 8 +- .../Layers/Tuples/FdbTupleParser.cs | 6 +- .../Linq/FdbAsyncEnumerable.EmptySequence.cs | 4 +- .../FdbAsyncEnumerable.EnumerableIterator.cs | 2 +- .../FdbAsyncEnumerable.EnumerableSequence.cs | 4 +- .../Linq/FdbAsyncEnumerable.Iterators.cs | 8 +- .../FdbAsyncEnumerable.OrderedSequence.cs | 6 +- .../Linq/FdbAsyncEnumerable.Sorters.cs | 4 +- .../Linq/FdbAsyncEnumerable.cs | 8 +- FoundationDB.Client/Linq/FdbAsyncSequence.cs | 4 +- .../Iterators/FdbAnonymousAsyncGenerator.cs | 4 +- .../Linq/Iterators/FdbAsyncFilterIterator.cs | 6 +- .../Linq/Iterators/FdbAsyncIteratorPump.cs | 4 +- .../Iterators/FdbBatchingAsyncIterator.cs | 4 +- .../Iterators/FdbDistinctAsyncIterator.cs | 4 +- .../Linq/Iterators/FdbObserverIterator.cs | 2 +- .../FdbParallelSelectAsyncIterator.cs | 6 +- .../Iterators/FdbPrefetchingAsyncIterator.cs | 6 +- .../Iterators/FdbSelectManyAsyncIterator.cs | 4 +- .../Iterators/FdbTakeWhileAsyncIterator.cs | 4 +- .../Linq/Iterators/FdbWhereAsyncIterator.cs | 6 +- .../Iterators/FdbWhereSelectAsyncIterator.cs | 4 +- .../Iterators/FdbWindowingAsyncIterator.cs | 6 +- FoundationDB.Client/Native/FdbFuture.cs | 5 +- FoundationDB.Client/Native/FdbNative.cs | 17 +- .../Native/FdbNativeCluster.cs | 9 +- .../Native/FdbNativeTransaction.cs | 8 +- .../Subspaces/FdbDatabasePartition.cs | 4 +- .../Subspaces/FdbDynamicSubspace.cs | 9 +- .../Subspaces/FdbDynamicSubspaceKeys.cs | 29 +- .../Subspaces/FdbDynamicSubspacePartition.cs | 23 +- .../Subspaces/FdbEncoderSubspaceKeys`1.cs | 14 +- .../Subspaces/FdbEncoderSubspaceKeys`2.cs | 16 +- .../Subspaces/FdbEncoderSubspaceKeys`3.cs | 12 +- .../Subspaces/FdbEncoderSubspaceKeys`4.cs | 14 +- .../FdbEncoderSubspacePartition`2.cs | 8 +- .../FdbEncoderSubspacePartition`3.cs | 10 +- .../FdbEncoderSubspacePartition`4.cs | 11 +- .../Subspaces/FdbEncoderSubspace`1.cs | 7 +- .../Subspaces/FdbEncoderSubspace`2.cs | 10 +- .../Subspaces/FdbEncoderSubspace`3.cs | 10 +- .../Subspaces/FdbEncoderSubspace`4.cs | 11 +- FoundationDB.Client/Subspaces/FdbSubspace.cs | 86 +- .../Subspaces/FdbSubspaceExtensions.cs | 60 +- .../Subspaces/IFdbDynamicSubspace.cs | 7 +- .../Subspaces/IFdbEncoderSubspace.cs | 8 +- FoundationDB.Client/Subspaces/IFdbSubspace.cs | 21 +- .../TypeSystem/Encoders/KeyValueEncoders.cs | 6 +- FoundationDB.Client/Utils/CodeAnnotations.cs | 214 ++- FoundationDB.Client/Utils/Contract.cs | 1452 +++++++++++++++-- .../Utils/ContractException.cs | 66 + FoundationDB.Client/Utils/Slice.cs | 5 +- FoundationDB.Client/Utils/SliceBuffer.cs | 3 +- FoundationDB.Client/Utils/SliceHelpers.cs | 4 +- FoundationDB.Client/Utils/SliceListStream.cs | 6 +- .../Utils/SlicePairComparer.cs | 2 +- FoundationDB.Client/Utils/SliceStream.cs | 4 +- FoundationDB.Client/Utils/SliceWriter.cs | 4 +- FoundationDB.Client/Utils/ThrowHelper.cs | 563 +++++++ FoundationDB.Client/Utils/TinyJsonParser.cs | 14 +- FoundationDB.Client/Utils/Uuid64.cs | 4 +- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 6 +- .../Collections/FdbMap`2.cs | 10 +- .../Collections/FdbRankedSet.cs | 8 +- .../Collections/FdbVector`1.cs | 8 +- .../Documents/FdbHashSetCollection.cs | 13 +- .../Bitmaps/CompressedBitmapBuilder.cs | 6 +- .../Bitmaps/CompressedBitmapWordIterator.cs | 4 +- .../Indexes/Bitmaps/CompressedBitmapWriter.cs | 4 +- .../Indexes/Bitmaps/CompressedWord.cs | 2 +- .../Indexes/Bitmaps/WordAlignHybridCoding.cs | 10 +- .../FdbQueryAsyncEnumerableExpression.cs | 6 +- .../Expressions/FdbQueryExpression.cs | 11 +- .../Expressions/FdbQueryFilterExpression.cs | 7 +- .../FdbQueryIndexLookupExpression.cs | 9 +- .../FdbQueryIntersectExpression.cs | 9 +- .../FdbQueryTransformExpression.cs | 8 +- .../Experimental/JsonNetCodec.cs | 6 +- .../Experimental/ProtobufCodec.cs | 4 +- 118 files changed, 2916 insertions(+), 926 deletions(-) create mode 100644 FoundationDB.Client/Utils/ContractException.cs create mode 100644 FoundationDB.Client/Utils/ThrowHelper.cs diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Async/AsyncBuffer.cs index bbcb29564..421ac78da 100644 --- a/FoundationDB.Client/Async/AsyncBuffer.cs +++ b/FoundationDB.Client/Async/AsyncBuffer.cs @@ -30,13 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Buffer that holds a fixed number of items and can rate-limit the producer /// diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Async/AsyncHelpers.cs index 9bc4dd899..4ea5fec01 100644 --- a/FoundationDB.Client/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Async/AsyncHelpers.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; /// Helper methods for creating and manipulating async sequences. public static class AsyncHelpers diff --git a/FoundationDB.Client/Async/AsyncTaskBuffer.cs b/FoundationDB.Client/Async/AsyncTaskBuffer.cs index 7750b8ed8..1c04ab714 100644 --- a/FoundationDB.Client/Async/AsyncTaskBuffer.cs +++ b/FoundationDB.Client/Async/AsyncTaskBuffer.cs @@ -30,13 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Buffer that holds a fixed number of Tasks, output them in arrival or completion order, and can rate-limit the producer /// diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Async/AsyncTransformQueue.cs index 0784d4363..76e527c3b 100644 --- a/FoundationDB.Client/Async/AsyncTransformQueue.cs +++ b/FoundationDB.Client/Async/AsyncTransformQueue.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Implements an async queue that asynchronously transform items, outputing them in arrival order, while throttling the producer /// Type of the input elements (from the inner async iterator) diff --git a/FoundationDB.Client/Async/Maybe.cs b/FoundationDB.Client/Async/Maybe.cs index 34379e10b..55bb6df31 100644 --- a/FoundationDB.Client/Async/Maybe.cs +++ b/FoundationDB.Client/Async/Maybe.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Runtime.ExceptionServices; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Either has a value, nothing, or an exception /// Type of the value diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Async/TaskHelpers.cs index 9334e5e1b..ce5ad2cd0 100644 --- a/FoundationDB.Client/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Async/TaskHelpers.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Async { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Helper methods to work on tasks internal static class TaskHelpers diff --git a/FoundationDB.Client/Converters/ConversionHelper.cs b/FoundationDB.Client/Converters/ConversionHelper.cs index dd26ca9b7..d7f28d736 100644 --- a/FoundationDB.Client/Converters/ConversionHelper.cs +++ b/FoundationDB.Client/Converters/ConversionHelper.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Converters { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Concurrent; using System.Collections.Generic; using System.Globalization; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Helper classe used to compare object of "compatible" types internal static class ComparisonHelper diff --git a/FoundationDB.Client/Converters/FdbConverters.cs b/FoundationDB.Client/Converters/FdbConverters.cs index 20938c363..dee9c62c2 100644 --- a/FoundationDB.Client/Converters/FdbConverters.cs +++ b/FoundationDB.Client/Converters/FdbConverters.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Converters { - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Globalization; using System.Linq.Expressions; using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// Helper class to convert object from one type to another public static class FdbConverters diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index f5da73655..b4a2d8790 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using FoundationDB.Filters.Logging; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Filters.Logging; + using JetBrains.Annotations; public static partial class Fdb { @@ -978,7 +978,7 @@ public static Task ForEachAsync( //REVIEW: what is the point if the body is not async ? // > either is can read and generate past_version errors then it needs to be async // > either it's not async, then it could only Write/Clear, and in which case we need a writeable transaction ... ? (and who will commit and when ??) - // It could maybe make sense if the source was an IFdbAsyncEnumerable because you could not use Parallel.ForEach(...) for that + // It could maybe make sense if the source was an IAsyncEnumerable because you could not use Parallel.ForEach(...) for that return RunBatchedReadOperationAsync(db, source, localInit, body, localFinally, DefaultInitialBatchSize, cancellationToken); } @@ -1024,7 +1024,7 @@ public static Task ForEachAsync( //REVIEW: what is the point if the body is not async ? // > either is can read and generate past_version errors then it needs to be async // > either it's not async, then it could only Write/Clear, and in which case we need a writeable transaction ... ? (and who will commit and when ??) - // It could maybe make sense if the source was an IFdbAsyncEnumerable because you could not use Parallel.ForEach(...) for that + // It could maybe make sense if the source was an IAsyncEnumerable because you could not use Parallel.ForEach(...) for that return RunBatchedReadOperationAsync(db, source, null, body, null, DefaultInitialBatchSize, cancellationToken); } diff --git a/FoundationDB.Client/Fdb.Errors.cs b/FoundationDB.Client/Fdb.Errors.cs index aa6c6082c..9bac05602 100644 --- a/FoundationDB.Client/Fdb.Errors.cs +++ b/FoundationDB.Client/Fdb.Errors.cs @@ -28,8 +28,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; + using Doxense.Diagnostics.Contracts; public static partial class Fdb { diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index f2b330cd0..9330108da 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -30,14 +30,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Status; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client.Status; + using FoundationDB.Client.Utils; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; public static partial class Fdb { @@ -218,7 +219,7 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas case "0": return "ssd"; case "1": return "memory"; default: - { + { // welcome to the future! return "unknown(" + value.ToAsciiOrHexaString() + ")"; } @@ -397,6 +398,7 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRange range, CancellationToken cancellationToken) { return EstimateCountAsync(db, range.Begin, range.End, null, cancellationToken); + //REVIEW: BUGBUG: REFACTORING: deal with null value for End! } /// Estimate the number of keys in the specified range. @@ -409,6 +411,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRan public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRange range, IProgress> onProgress, CancellationToken cancellationToken) { return EstimateCountAsync(db, range.Begin, range.End, onProgress, cancellationToken); + //REVIEW: BUGBUG: REFACTORING: deal with null value for End! } /// Estimate the number of keys in the specified range. diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index d2d001755..49b1ef004 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -26,15 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Filters.Logging; - namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Core; - using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Directories; using JetBrains.Annotations; using System; using System.Collections.Concurrent; @@ -42,6 +35,12 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client.Core; + using FoundationDB.Client.Native; + using FoundationDB.Client.Utils; + using FoundationDB.Layers.Directories; /// FoundationDB database session handle /// An instance of this class can be used to create any number of concurrent transactions that will read and/or write to this particular database. @@ -129,9 +128,9 @@ protected FdbDatabase(IFdbCluster cluster, IFdbDatabaseHandler handler, string n /// If true, the cluster instance lifetime is linked with the database instance public static FdbDatabase Create(IFdbCluster cluster, IFdbDatabaseHandler handler, string name, IFdbSubspace contentSubspace, IFdbDirectory directory, bool readOnly, bool ownsCluster) { - if (cluster == null) throw new ArgumentNullException("cluster"); - if (handler == null) throw new ArgumentNullException("handler"); - if (contentSubspace == null) throw new ArgumentNullException("contentSubspace"); + if (cluster == null) throw new ArgumentNullException(nameof(cluster)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); + if (contentSubspace == null) throw new ArgumentNullException(nameof(contentSubspace)); return new FdbDatabase(cluster, handler, name, contentSubspace, directory, readOnly, ownsCluster); } @@ -411,7 +410,7 @@ public void SetOption(FdbDatabaseOption option) { ThrowIfDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting database option {0}", option.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting database option {option}"); m_handler.SetOption(option, Slice.Nil); } @@ -423,7 +422,7 @@ public void SetOption(FdbDatabaseOption option, string value) { ThrowIfDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting database option {0} to '{1}'", option.ToString(), value ?? "")); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting database option {option} to '{value ?? ""}'"); var data = FdbNative.ToNativeString(value, nullTerminated: true); m_handler.SetOption(option, data); @@ -436,7 +435,7 @@ public void SetOption(FdbDatabaseOption option, long value) { ThrowIfDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting database option {0} to {1}", option.ToString(), value)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting database option {option} to {value}"); // Spec says: "If the option is documented as taking an Int parameter, value must point to a signed 64-bit integer (little-endian), and value_length must be 8." var data = Slice.FromFixed64(value); @@ -675,7 +674,7 @@ public int DefaultTimeout get { return m_defaultTimeout; } set { - if (value < 0) throw new ArgumentOutOfRangeException("value", value, "Timeout value cannot be negative"); + if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "Timeout value cannot be negative"); m_defaultTimeout = value; } } @@ -687,7 +686,7 @@ public int DefaultRetryLimit get { return m_defaultRetryLimit; } set { - if (value < 0) throw new ArgumentOutOfRangeException("value", value, "RetryLimit value cannot be negative"); + if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "RetryLimit value cannot be negative"); m_defaultRetryLimit = value; } } @@ -699,7 +698,7 @@ public int DefaultMaxRetryDelay get { return m_defaultMaxRetryDelay; } set { - if (value < 0) throw new ArgumentOutOfRangeException("value", value, "MaxRetryDelay value cannot be negative"); + if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "MaxRetryDelay value cannot be negative"); m_defaultMaxRetryDelay = value; } } @@ -732,7 +731,7 @@ protected virtual void Dispose(bool disposing) { // mark this db has dead, but keep the handle alive until after all the callbacks have fired - //TODO: kill all pending transactions on this db? + //TODO: kill all pending transactions on this db? foreach (var trans in m_transactions.Values) { if (trans != null && trans.StillAlive) @@ -749,7 +748,7 @@ protected virtual void Dispose(bool disposing) { if (m_handler != null) { - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Dispose", String.Format("Disposing database {0} handler", m_name)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Dispose", $"Disposing database {m_name} handler"); try { m_handler.Dispose(); } catch (Exception e) { diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index 86da37643..2d808a32f 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Provides a set of extensions methods shared by all FoundationDB database implementations. public static class FdbDatabaseExtensions @@ -52,10 +52,10 @@ public static class FdbDatabaseExtensions /// var result = await tr.Get(Slice.FromString("Hello")); /// var items = await tr.GetRange(FdbKeyRange.StartsWith(Slice.FromString("ABC"))).ToListAsync(); /// } - [NotNull] - public static IFdbReadOnlyTransaction BeginReadOnlyTransaction(this IFdbDatabase db, CancellationToken cancellationToken) + [Pure, NotNull] + public static IFdbReadOnlyTransaction BeginReadOnlyTransaction([NotNull] this IFdbDatabase db, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.BeginTransaction(FdbTransactionMode.ReadOnly, cancellationToken, default(FdbOperationContext)); } @@ -71,10 +71,10 @@ public static IFdbReadOnlyTransaction BeginReadOnlyTransaction(this IFdbDatabase /// tr.Clear(Slice.FromString("OldValue")); /// await tr.CommitAsync(); /// } - [NotNull] - public static IFdbTransaction BeginTransaction(this IFdbDatabase db, CancellationToken cancellationToken) + [Pure, NotNull] + public static IFdbTransaction BeginTransaction([NotNull] this IFdbDatabase db, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.BeginTransaction(FdbTransactionMode.Default, cancellationToken, default(FdbOperationContext)); } @@ -85,9 +85,9 @@ public static IFdbTransaction BeginTransaction(this IFdbDatabase db, Cancellatio /// Set the size of the client location cache. Raising this value can boost performance in very large databases where clients access data in a near-random pattern. Defaults to 100000. /// Database instance /// Max location cache entries - public static void SetLocationCacheSize(this IFdbDatabase db, int size) + public static void SetLocationCacheSize([NotNull] this IFdbDatabase db, int size) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); if (size < 0) throw new FdbException(FdbError.InvalidOptionValue, "Location cache size must be a positive integer"); //REVIEW: we can't really change this to a Property, because we don't have a way to get the current value for the getter, and set only properties are weird... @@ -98,9 +98,9 @@ public static void SetLocationCacheSize(this IFdbDatabase db, int size) /// Set the maximum number of watches allowed to be outstanding on a database connection. Increasing this number could result in increased resource usage. Reducing this number will not cancel any outstanding watches. Defaults to 10000 and cannot be larger than 1000000. /// Database instance /// Max outstanding watches - public static void SetMaxWatches(this IFdbDatabase db, int count) + public static void SetMaxWatches([NotNull] this IFdbDatabase db, int count) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); if (count < 0) throw new FdbException(FdbError.InvalidOptionValue, "Maximum outstanding watches count must be a positive integer"); //REVIEW: we can't really change this to a Property, because we don't have a way to get the current value for the getter, and set only properties are weird... @@ -111,9 +111,9 @@ public static void SetMaxWatches(this IFdbDatabase db, int count) /// Specify the machine ID that was passed to fdbserver processes running on the same machine as this client, for better location-aware load balancing. /// Database instance /// Hexadecimal ID - public static void SetMachineId(this IFdbDatabase db, string hexId) + public static void SetMachineId([NotNull] this IFdbDatabase db, string hexId) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); //REVIEW: we can't really change this to a Property, because we don't have a way to get the current value for the getter, and set only properties are weird... //TODO: cache this into a local variable ? db.SetOption(FdbDatabaseOption.MachineId, hexId); @@ -122,9 +122,9 @@ public static void SetMachineId(this IFdbDatabase db, string hexId) /// Specify the datacenter ID that was passed to fdbserver processes running in the same datacenter as this client, for better location-aware load balancing. /// Database instance /// Hexadecimal ID - public static void SetDataCenterId(this IFdbDatabase db, string hexId) + public static void SetDataCenterId([NotNull] this IFdbDatabase db, string hexId) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); //REVIEW: we can't really change this to a Property, because we don't have a way to get the current value for the getter, and set only properties are weird... //TODO: cache this into a local variable ? db.SetOption(FdbDatabaseOption.DataCenterId, hexId); @@ -138,7 +138,8 @@ public static void SetDataCenterId(this IFdbDatabase db, string hexId) /// Database instance /// Key to test /// Returns true if the key is not null or empty, does not exceed the maximum key size, and is contained in the global key space of this database instance. Otherwise, returns false. - public static bool IsKeyValid(this IFdbDatabase db, Slice key) + [Pure] + public static bool IsKeyValid([NotNull] this IFdbDatabase db, Slice key) { Exception _; return FdbDatabase.ValidateKey(db, ref key, false, true, out _); @@ -149,7 +150,7 @@ public static bool IsKeyValid(this IFdbDatabase db, Slice key) /// Key to verify /// If true, the key is allowed to be one past the maximum key allowed by the global namespace /// If the key is outside of the allowed keyspace, throws an FdbException with code FdbError.KeyOutsideLegalRange - internal static void EnsureKeyIsValid(this IFdbDatabase db, Slice key, bool endExclusive = false) + internal static void EnsureKeyIsValid([NotNull] this IFdbDatabase db, Slice key, bool endExclusive = false) { Exception ex; if (!FdbDatabase.ValidateKey(db, ref key, endExclusive, false, out ex)) throw ex; @@ -160,7 +161,7 @@ internal static void EnsureKeyIsValid(this IFdbDatabase db, Slice key, bool endE /// Key to verify /// If true, the key is allowed to be one past the maximum key allowed by the global namespace /// If the key is outside of the allowed keyspace, throws an FdbException with code FdbError.KeyOutsideLegalRange - internal static void EnsureKeyIsValid(this IFdbDatabase db, ref Slice key, bool endExclusive = false) + internal static void EnsureKeyIsValid([NotNull] this IFdbDatabase db, ref Slice key, bool endExclusive = false) { Exception ex; if (!FdbDatabase.ValidateKey(db, ref key, endExclusive, false, out ex)) throw ex; @@ -171,9 +172,9 @@ internal static void EnsureKeyIsValid(this IFdbDatabase db, ref Slice key, bool /// Array of keys to verify /// If true, the keys are allowed to be one past the maximum key allowed by the global namespace /// If at least on key is outside of the allowed keyspace, throws an FdbException with code FdbError.KeyOutsideLegalRange - internal static void EnsureKeysAreValid(this IFdbDatabase db, Slice[] keys, bool endExclusive = false) + internal static void EnsureKeysAreValid([NotNull] this IFdbDatabase db, Slice[] keys, bool endExclusive = false) { - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(keys, nameof(keys)); for (int i = 0; i < keys.Length; i++) { Exception ex; @@ -192,7 +193,8 @@ internal static void EnsureKeysAreValid(this IFdbDatabase db, Slice[] keys, bool /// db.Extract('<02>TopSecret<00><02>Password<00>') => Slice.Nil /// db.Extract(Slice.Nil) => Slice.Nil /// - public static Slice Extract(this IFdbDatabase db, Slice keyAbsolute) + [Pure] + public static Slice Extract([NotNull] this IFdbDatabase db, Slice keyAbsolute) { return db.GlobalSpace.ExtractKey(keyAbsolute); } @@ -214,9 +216,9 @@ public static Slice Extract(this IFdbDatabase db, Slice keyAbsolute) /// If you need to read several keys at once, use a version of . /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetAsync(this IFdbReadOnlyRetryable db, Slice key, CancellationToken cancellationToken) + public static Task GetAsync([NotNull] this IFdbReadOnlyRetryable db, Slice key, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetAsync(key), cancellationToken); } @@ -230,9 +232,9 @@ public static Task GetAsync(this IFdbReadOnlyRetryable db, Slice key, Can /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNull] Slice[] keys, CancellationToken cancellationToken) + public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] Slice[] keys, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetValuesAsync(keys), cancellationToken); } @@ -243,9 +245,9 @@ public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNu /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keys, CancellationToken cancellationToken) + public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keys, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetValuesAsync(keys), cancellationToken); } @@ -255,9 +257,9 @@ public static Task GetValuesAsync(this IFdbReadOnlyRetryable db, [NotNu /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetKeyAsync(this IFdbReadOnlyRetryable db, FdbKeySelector keySelector, CancellationToken cancellationToken) + public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, FdbKeySelector keySelector, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetKeyAsync(keySelector), cancellationToken); } @@ -268,10 +270,10 @@ public static Task GetKeyAsync(this IFdbReadOnlyRetryable db, FdbKeySelec /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull] FdbKeySelector[] keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] FdbKeySelector[] keySelectors, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); - if (keySelectors == null) throw new ArgumentNullException("keySelectors"); + Contract.NotNull(db, nameof(db)); + Contract.NotNull(keySelectors, nameof(keySelectors)); return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), cancellationToken); } @@ -282,10 +284,10 @@ public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); - if (keySelectors == null) throw new ArgumentNullException("keySelectors"); + Contract.NotNull(db, nameof(db)); + Contract.NotNull(keySelectors, nameof(keySelectors)); return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), cancellationToken); } @@ -295,9 +297,9 @@ public static Task GetKeysAsync(this IFdbReadOnlyRetryable db, [NotNull /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetRangeAsync(this IFdbReadOnlyRetryable db, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken cancellationToken) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetryable db, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetRangeAsync(beginInclusive, endExclusive, options, iteration), cancellationToken); } @@ -307,9 +309,9 @@ public static Task GetRangeAsync(this IFdbReadOnlyRetryable db, F /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetAsync(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task SetAsync([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Set(key, value), cancellationToken); } @@ -319,9 +321,9 @@ public static Task SetAsync(this IFdbRetryable db, Slice key, Slice value, Cance /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetValuesAsync(this IFdbRetryable db, KeyValuePair[] keyValuePairs, CancellationToken cancellationToken) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair[] keyValuePairs, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => { foreach (var kv in keyValuePairs) @@ -337,9 +339,9 @@ public static Task SetValuesAsync(this IFdbRetryable db, KeyValuePair or overrides. /// - public static Task SetValuesAsync(this IFdbRetryable db, IEnumerable> keyValuePairs, CancellationToken cancellationToken) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable> keyValuePairs, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => { foreach (var kv in keyValuePairs) @@ -355,9 +357,9 @@ public static Task SetValuesAsync(this IFdbRetryable db, IEnumerable or overrides. /// - public static Task ClearAsync(this IFdbRetryable db, Slice key, CancellationToken cancellationToken) + public static Task ClearAsync([NotNull] this IFdbRetryable db, Slice key, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Clear(key), cancellationToken); } @@ -366,9 +368,9 @@ public static Task ClearAsync(this IFdbRetryable db, Slice key, CancellationToke /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task ClearRangeAsync(this IFdbRetryable db, Slice beginKeyInclusive, Slice endKeyExclusive, CancellationToken cancellationToken) + public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, Slice beginKeyInclusive, Slice endKeyExclusive, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.ClearRange(beginKeyInclusive, endKeyExclusive), cancellationToken); } @@ -377,9 +379,9 @@ public static Task ClearRangeAsync(this IFdbRetryable db, Slice beginKeyInclusiv /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task ClearRangeAsync(this IFdbRetryable db, FdbKeyRange range, CancellationToken cancellationToken) + public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, FdbKeyRange range, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.ClearRange(range), cancellationToken); } @@ -393,9 +395,9 @@ public static Task ClearRangeAsync(this IFdbRetryable db, FdbKeyRange range, Can /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicAdd(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicAdd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Add), cancellationToken); } @@ -404,9 +406,9 @@ public static Task AtomicAdd(this IFdbRetryable db, Slice key, Slice value, Canc /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitAnd(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitAnd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitAnd), cancellationToken); } @@ -415,9 +417,9 @@ public static Task AtomicBitAnd(this IFdbRetryable db, Slice key, Slice value, C /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitOr(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitOr([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitOr), cancellationToken); } @@ -426,9 +428,9 @@ public static Task AtomicBitOr(this IFdbRetryable db, Slice key, Slice value, Ca /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitXor(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitXor([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitXor), cancellationToken); } @@ -437,9 +439,9 @@ public static Task AtomicBitXor(this IFdbRetryable db, Slice key, Slice value, C /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicMax(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicMax([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Max), cancellationToken); } @@ -448,9 +450,9 @@ public static Task AtomicMax(this IFdbRetryable db, Slice key, Slice value, Canc /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicMin(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicMin([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Min), cancellationToken); } @@ -463,9 +465,9 @@ public static Task AtomicMin(this IFdbRetryable db, Slice key, Slice value, Canc /// Key to be looked up in the database /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property contains the current value of the key. - public static Task GetAndWatch(this IFdbRetryable db, Slice key, CancellationToken cancellationToken) + public static Task GetAndWatch([NotNull] this IFdbRetryable db, Slice key, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); return db.ReadWriteAsync(async (tr) => { @@ -476,7 +478,7 @@ public static Task GetAndWatch(this IFdbRetryable db, Slice key, Cance }, cancellationToken); } - public static Task GetAndWatch(this IFdbRetryable db, TKey key, CancellationToken cancellationToken) + public static Task GetAndWatch([NotNull] this IFdbRetryable db, TKey key, CancellationToken cancellationToken) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -489,11 +491,11 @@ public static Task GetAndWatch(this IFdbRetryable db, TKey key, /// Value to be inserted into the database. /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static async Task SetAndWatch(this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static async Task SetAndWatch([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); + Contract.NotNull(db, nameof(db)); - FdbWatch watch = default(FdbWatch); + var watch = default(FdbWatch); await db.WriteAsync((tr) => { diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 4478cd9f4..6ed2571ea 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Linq; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// Factory class for keys public static class FdbKey @@ -91,7 +91,7 @@ public static Slice Increment(Slice slice) [NotNull] public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) { - if (prefix == null) throw new ArgumentNullException("prefix"); + if (prefix.IsNull) throw new ArgumentNullException("prefix"); if (keys == null) throw new ArgumentNullException("keys"); //REVIEW: merge this code with Slice.ConcatRange! @@ -120,7 +120,7 @@ public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) [NotNull] public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) { - if (prefix == null) throw new ArgumentNullException("prefix"); + if (prefix.IsNull) throw new ArgumentNullException("prefix"); if (keys == null) throw new ArgumentNullException("keys"); //REVIEW: merge this code with Slice.ConcatRange! @@ -132,7 +132,7 @@ public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) // pre-allocate with a count if we can get one... var coll = keys as ICollection; var next = coll == null ? new List() : new List(coll.Count); - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); //TODO: use multiple buffers if item count is huge ? diff --git a/FoundationDB.Client/FdbKeyRange.cs b/FoundationDB.Client/FdbKeyRange.cs index 940704d30..5708188d1 100644 --- a/FoundationDB.Client/FdbKeyRange.cs +++ b/FoundationDB.Client/FdbKeyRange.cs @@ -28,9 +28,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] diff --git a/FoundationDB.Client/FdbKeyRangeComparer.cs b/FoundationDB.Client/FdbKeyRangeComparer.cs index 24d1f2c07..0fbe06ff7 100644 --- a/FoundationDB.Client/FdbKeyRangeComparer.cs +++ b/FoundationDB.Client/FdbKeyRangeComparer.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; [DebuggerDisplay("Mode={m_mode}")] public sealed class FdbKeyRangeComparer : IComparer, IEqualityComparer diff --git a/FoundationDB.Client/FdbMergeQueryExtensions.cs b/FoundationDB.Client/FdbMergeQueryExtensions.cs index 9a21a4062..1dd141e55 100644 --- a/FoundationDB.Client/FdbMergeQueryExtensions.cs +++ b/FoundationDB.Client/FdbMergeQueryExtensions.cs @@ -28,34 +28,45 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Linq; using System; using System.Collections.Generic; using System.Linq; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + using FoundationDB.Async; + using FoundationDB.Linq; public static class FdbMergeQueryExtensions { #region MergeSort (x OR y) + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable> MergeSort(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(ranges, nameof(ranges)); + Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); return new FdbMergeSortIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, - TaskHelpers.Cache>.Identity, + (kv) => kv, keyComparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable MergeSort(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(ranges, nameof(ranges)); + Contract.NotNull(keySelector, nameof(keySelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); trans.EnsureCanRead(); return new FdbMergeSortIterator, TKey, TResult>( @@ -67,24 +78,29 @@ public static IFdbAsyncEnumerable MergeSort(this IFdbRea ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Union(IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { + Contract.NotNull(sources, nameof(sources)); + Contract.NotNull(keySelector, nameof(keySelector)); return new FdbMergeSortIterator( sources, null, keySelector, - TaskHelpers.Cache.Identity, + (x) => x, keyComparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Union(IEnumerable> sources, IComparer keyComparer = null) { + Contract.NotNull(sources, nameof(sources)); return new FdbMergeSortIterator( sources, null, - TaskHelpers.Cache.Identity, - TaskHelpers.Cache.Identity, + (x) => x, + (x) => x, keyComparer ); } @@ -93,20 +109,25 @@ public static IFdbAsyncEnumerable Union(IEnumerable> Intersect(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(ranges, nameof(ranges)); + Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); return new FdbIntersectIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, - TaskHelpers.Cache>.Identity, + (kv) => kv, keyComparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Intersect(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? @@ -121,46 +142,57 @@ public static IFdbAsyncEnumerable Intersect(this IFdbRea ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Intersect(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) { + Contract.NotNull(first, nameof(first)); + Contract.NotNull(second, nameof(second)); return new FdbIntersectIterator( new[] { first, second }, null, keySelector, - TaskHelpers.Cache.Identity, + (x) => x, keyComparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Intersect(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) { + Contract.NotNull(first, nameof(first)); + Contract.NotNull(second, nameof(second)); return new FdbIntersectIterator( new [] { first, second }, null, - TaskHelpers.Cache.Identity, - TaskHelpers.Cache.Identity, + (x) => x, + (x) => x, comparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Intersect(IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { + Contract.NotNull(sources, nameof(sources)); + Contract.NotNull(keySelector, nameof(keySelector)); return new FdbIntersectIterator( sources, null, keySelector, - TaskHelpers.Cache.Identity, + (x) => x, keyComparer ); } + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Intersect(IEnumerable> sources, IComparer keyComparer = null) { + Contract.NotNull(sources, nameof(sources)); return new FdbIntersectIterator( sources, null, - TaskHelpers.Cache.Identity, - TaskHelpers.Cache.Identity, + (x) => x, + (x) => x, keyComparer ); } @@ -176,19 +208,20 @@ public static IFdbAsyncEnumerable Intersect(IEnumerableLambda called to extract the keys from the ranges /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable> Except(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? - if (trans == null) throw new ArgumentNullException("trans"); - if (ranges == null) throw new ArgumentNullException("ranges"); - if (keySelector == null) throw new ArgumentNullException("keySelector"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(ranges, nameof(ranges)); + Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); return new FdbExceptIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, - TaskHelpers.Cache>.Identity, + (kv) => kv, keyComparer ); } @@ -200,9 +233,10 @@ public static IFdbAsyncEnumerable> Except(this /// Lambda called to extract the keys from the ranges /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. - public static IFdbAsyncEnumerable> Except(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + [Pure, NotNull, LinqTunnel] + public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { - if (ranges == null) throw new ArgumentNullException("ranges"); + Contract.NotNull(ranges, nameof(ranges)); return Except(trans, ranges.Select(r => FdbKeySelectorPair.Create(r)), keySelector, keyComparer); } @@ -215,7 +249,8 @@ public static IFdbAsyncEnumerable> Except(this /// Lambda called to extract the values returned by the query /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. - public static IFdbAsyncEnumerable Except(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + [Pure, NotNull, LinqTunnel] + public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? @@ -238,9 +273,10 @@ public static IFdbAsyncEnumerable Except(this IFdbReadOn /// Lambda called to extract the values returned by the query /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. - public static IFdbAsyncEnumerable Except(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + [Pure, NotNull, LinqTunnel] + public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { - if (ranges == null) throw new ArgumentNullException("ranges"); + Contract.NotNull(ranges, nameof(ranges)); return Except(trans, ranges.Select(r => FdbKeySelectorPair.Create(r)), keySelector, resultSelector, keyComparer); } @@ -252,13 +288,17 @@ public static IFdbAsyncEnumerable Except(this IFdbReadOn /// Lambda used to extract keys from both queries. /// Instance used to compare keys /// Async query that returns only the elements that are in , and not in - public static IFdbAsyncEnumerable Except(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) + [Pure, NotNull, LinqTunnel] + public static IFdbAsyncEnumerable Except([NotNull] this IFdbAsyncEnumerable first, [NotNull] IFdbAsyncEnumerable second, [NotNull] Func keySelector, IComparer keyComparer = null) { + Contract.NotNull(first, nameof(first)); + Contract.NotNull(second, nameof(second)); + Contract.NotNull(keySelector, nameof(keySelector)); return new FdbExceptIterator( new[] { first, second }, null, keySelector, - TaskHelpers.Cache.Identity, + (x) => x, keyComparer ); } @@ -269,13 +309,16 @@ public static IFdbAsyncEnumerable Except(this IFdbAsyncE /// Second query that contains the elements that cannot be in the result /// Instance used to compare elements /// Async query that returns only the elements that are in , and not in + [Pure, NotNull, LinqTunnel] public static IFdbAsyncEnumerable Except(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) { + Contract.NotNull(first, nameof(first)); + Contract.NotNull(second, nameof(second)); return new FdbExceptIterator( new[] { first, second }, null, - TaskHelpers.Cache.Identity, - TaskHelpers.Cache.Identity, + (x) => x, + (x) => x, comparer ); } diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index c2dcec8e6..364a8a08e 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -28,14 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Diagnostics; using System.Globalization; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client.Utils; + using JetBrains.Annotations; /// /// Represents the context of a retryable transactional function which accepts a read-only or read-write transaction. diff --git a/FoundationDB.Client/FdbQueryMergeIterator.cs b/FoundationDB.Client/FdbQueryMergeIterator.cs index 979f15edf..5eaed639e 100644 --- a/FoundationDB.Client/FdbQueryMergeIterator.cs +++ b/FoundationDB.Client/FdbQueryMergeIterator.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; using System; using System.Collections.Generic; using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Linq; /// Performs a Merge Sort on several concurrent range queries /// Type of the elements in the source queries diff --git a/FoundationDB.Client/FdbRangeOptions.cs b/FoundationDB.Client/FdbRangeOptions.cs index 5c20d430e..c55d65541 100644 --- a/FoundationDB.Client/FdbRangeOptions.cs +++ b/FoundationDB.Client/FdbRangeOptions.cs @@ -28,9 +28,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; /// Container class for options in a Range query [DebuggerDisplay("Limit={Limit}, Reverse={Reverse}, TargetBytes={TargetBytes}, Mode={Mode}")] diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 9a70ffd18..74362966c 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -31,15 +31,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client.Utils; + using FoundationDB.Linq; + using JetBrains.Annotations; public partial class FdbRangeQuery { diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 0270bd5fd..2141c7250 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -31,15 +31,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Linq; + using JetBrains.Annotations; public partial class FdbRangeQuery { @@ -195,10 +195,7 @@ protected override void Cleanup() { try { - if (m_chunkIterator != null) - { - m_chunkIterator.Dispose(); - } + m_chunkIterator?.Dispose(); } finally { diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index a2877ce61..ffb446bcc 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -28,15 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Linq; + using JetBrains.Annotations; /// Query describing an ongoing GetRange operation [DebuggerDisplay("Begin={Begin}, End={End}, Limit={Limit}, Mode={Mode}, Reverse={Reverse}, Snapshot={Snapshot}")] @@ -83,11 +83,11 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption public FdbKeySelectorPair Range { get { return new FdbKeySelectorPair(this.Begin, this.End); } } /// Stores all the settings for this range query - internal FdbRangeOptions Options { get; private set; } + internal FdbRangeOptions Options { get; } /// Original key selector pair describing the bounds of the parent range. All the results returned by the query will be bounded by this original range. /// May differ from when combining certain operators. - internal FdbKeySelectorPair OriginalRange { get; private set; } + internal FdbKeySelectorPair OriginalRange { get; } /// Limit in number of rows to return public int? Limit { get { return this.Options.Limit; } } @@ -100,16 +100,16 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption public FdbStreamingMode Mode { get { return this.Options.Mode ?? FdbStreamingMode.Iterator; } } /// Should we perform the range using snapshot mode ? - public bool Snapshot { get; private set; } + public bool Snapshot { get; } /// Should the results be returned in reverse order (from last key to first key) public bool Reversed { get { return this.Options.Reverse ?? false; } } /// Parent transaction used to perform the GetRange operation - internal IFdbReadOnlyTransaction Transaction { [NotNull] get; private set; } + internal IFdbReadOnlyTransaction Transaction { [NotNull] get; } /// Transformation applied to the result - internal Func, T> Transform { [NotNull] get; private set; } + internal Func, T> Transform { [NotNull] get; } #endregion @@ -201,12 +201,11 @@ public FdbRangeQuery Reverse() { begin = this.End - limit.Value; } - limit = null; } return new FdbRangeQuery( this, - new FdbRangeOptions(this.Options) { Reverse = !this.Reversed, Limit = limit } + new FdbRangeOptions(this.Options) { Reverse = !this.Reversed } ) { Begin = begin, @@ -236,7 +235,7 @@ public FdbRangeQuery WithMode(FdbStreamingMode mode) { if (!Enum.IsDefined(typeof(FdbStreamingMode), mode)) { - throw new ArgumentOutOfRangeException("mode", "Unsupported streaming mode"); + throw new ArgumentOutOfRangeException(nameof(mode), "Unsupported streaming mode"); } return new FdbRangeQuery( @@ -251,7 +250,7 @@ public FdbRangeQuery WithMode(FdbStreamingMode mode) [NotNull] public FdbRangeQuery UseTransaction([NotNull] IFdbReadOnlyTransaction transaction) { - if (transaction == null) throw new ArgumentNullException("transaction"); + Contract.NotNull(transaction, nameof(transaction)); return new FdbRangeQuery( transaction, @@ -281,6 +280,7 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) [ItemNotNull] public Task> ToListAsync() { + // ReSharper disable once InvokeAsExtensionMethod return FdbAsyncEnumerable.ToListAsync(this, this.Transaction.Cancellation); } @@ -288,6 +288,7 @@ public Task> ToListAsync() [ItemNotNull] public Task ToArrayAsync() { + // ReSharper disable once InvokeAsExtensionMethod return FdbAsyncEnumerable.ToArrayAsync(this, this.Transaction.Cancellation); } @@ -295,14 +296,15 @@ public Task ToArrayAsync() /// This method has to read all the keys and values, which may exceed the lifetime of a transaction. Please consider using when reading potentially large ranges. public Task CountAsync() { + // ReSharper disable once InvokeAsExtensionMethod return FdbAsyncEnumerable.CountAsync(this, this.Transaction.Cancellation); } [NotNull] - internal FdbRangeQuery Map([NotNull] Func, R> transform) + internal FdbRangeQuery Map([NotNull] Func, TResult> transform) { Contract.Requires(transform != null); - return new FdbRangeQuery( + return new FdbRangeQuery( this.Transaction, this.Begin, this.End, @@ -314,13 +316,13 @@ internal FdbRangeQuery Map([NotNull] Func, R> t /// Projects each element of the range results into a new form. [NotNull] - public FdbRangeQuery Select([NotNull] Func lambda) + public FdbRangeQuery Select([NotNull] Func lambda) { Contract.Requires(lambda != null); // note: avoid storing the query in the scope by storing the transform locally so that only 'f' and 'lambda' are kept alive var f = this.Transform; Contract.Assert(f != null); - return Map((x) => lambda(f(x))); + return Map((x) => lambda(f(x))); } /// Filters the range results based on a predicate. @@ -389,6 +391,7 @@ public Task NoneAsync() /// Execute an action on each key/value pair of the range results public Task ForEachAsync([NotNull] Action action) { + // ReSharper disable once InvokeAsExtensionMethod return FdbAsyncEnumerable.ForEachAsync(this, action, this.Transaction.Cancellation); } @@ -404,7 +407,7 @@ internal async Task HeadAsync(bool single, bool orDefault) var options = new FdbRangeOptions() { - Limit = single ? 2 : 1, + Limit = Math.Min(single ? 2 : 1, this.Options.Limit ?? int.MaxValue), TargetBytes = 0, Mode = FdbStreamingMode.Exact, Reverse = this.Reversed @@ -437,10 +440,12 @@ internal async Task AnyOrNoneAsync(bool any) // we can use the EXACT streaming mode with Limit = 1, and it will work if TargetBytes is 0 if ((this.TargetBytes ?? 0) != 0 || (this.Mode != FdbStreamingMode.Iterator && this.Mode != FdbStreamingMode.Exact)) { // fallback to the default implementation + // ReSharper disable InvokeAsExtensionMethod if (any) return await FdbAsyncEnumerable.AnyAsync(this, this.Transaction.Cancellation); else return await FdbAsyncEnumerable.NoneAsync(this, this.Transaction.Cancellation); + // ReSharper restore InvokeAsExtensionMethod } //BUGBUG: do we need special handling if OriginalRange != Range ? (weird combinations of Take/Skip and Reverse) @@ -474,53 +479,53 @@ public static class FdbRangeQueryExtensions { [NotNull] - public static FdbRangeQuery Keys(this FdbRangeQuery> query) + public static FdbRangeQuery Keys([NotNull] this FdbRangeQuery> query) { - if (query == null) throw new ArgumentNullException("query"); + Contract.NotNull(query, nameof(query)); var f = query.Transform; //note: we only keep a reference on 'f' to allow the previous query instance to be collected. Contract.Assert(f != null); - return query.Map((x) => f(x).Key); + return query.Map((x) => f(x).Key); } [NotNull] - public static FdbRangeQuery Keys(this FdbRangeQuery> query, [NotNull] Func transform) + public static FdbRangeQuery Keys([NotNull] this FdbRangeQuery> query, [NotNull] Func transform) { - if (query == null) throw new ArgumentNullException("query"); - if (transform == null) throw new ArgumentNullException("transform"); + Contract.NotNull(query, nameof(query)); + Contract.NotNull(transform, nameof(transform)); var f = query.Transform; //note: we only keep a reference on 'f' to allow the previous query instance to be collected. Contract.Assert(f != null); - return query.Map((x) => transform(f(x).Key)); + return query.Map((x) => transform(f(x).Key)); } [NotNull] - public static FdbRangeQuery Values(this FdbRangeQuery> query) + public static FdbRangeQuery Values([NotNull] this FdbRangeQuery> query) { - if (query == null) throw new ArgumentNullException("query"); + Contract.NotNull(query, nameof(query)); var f = query.Transform; //note: we only keep a reference on 'f' to allow the previous query instance to be collected. Contract.Assert(f != null); - return query.Map((x) => f(x).Value); + return query.Map((x) => f(x).Value); } [NotNull] - public static FdbRangeQuery Values(this FdbRangeQuery> query, [NotNull] Func transform) + public static FdbRangeQuery Values([NotNull] this FdbRangeQuery> query, [NotNull] Func transform) { - if (query == null) throw new ArgumentNullException("query"); - if (transform == null) throw new ArgumentNullException("transform"); + Contract.NotNull(query, nameof(query)); + Contract.NotNull(transform, nameof(transform)); var f = query.Transform; //note: we only keep a reference on 'f' to allow the previous query instance to be collected. Contract.Assert(f != null); - return query.Map((x) => transform(f(x).Value)); + return query.Map((x) => transform(f(x).Value)); } } diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 1fd152845..e9952137e 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -31,16 +31,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Core; - using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client.Core; + using FoundationDB.Client.Native; + using FoundationDB.Client.Utils; + using JetBrains.Annotations; /// FounrationDB transaction handle. /// An instance of this class can be used to read from and/or write to a snapshot of a FoundationDB database. @@ -382,7 +383,7 @@ internal FdbRangeQuery> GetRangeCore(FdbKeySelector b if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetRangeCore", String.Format("Getting range '{0} <= x < {1}'", begin.ToString(), end.ToString())); #endif - return new FdbRangeQuery>(this, begin, end, TaskHelpers.Cache>.Identity, snapshot, options); + return new FdbRangeQuery>(this, begin, end, (kv) => kv, snapshot, options); } /// diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 90534829c..c61712f92 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -28,15 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.IO; using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Linq; + using JetBrains.Annotations; /// Provides a set of extensions methods shared by all FoundationDB transaction implementations. public static class FdbTransactionExtensions @@ -45,7 +45,7 @@ public static class FdbTransactionExtensions #region Fluent Options... /// Allows this transaction to read system keys (those that start with the byte 0xFF) - public static TTransaction WithReadAccessToSystemKeys(this TTransaction trans) + public static TTransaction WithReadAccessToSystemKeys([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(Fdb.ApiVersion >= 300 ? FdbTransactionOption.ReadSystemKeys : FdbTransactionOption.AccessSystemKeys); @@ -54,7 +54,7 @@ public static TTransaction WithReadAccessToSystemKeys(this TTransa } /// Allows this transaction to read and modify system keys (those that start with the byte 0xFF) - public static TTransaction WithWriteAccessToSystemKeys(this TTransaction trans) + public static TTransaction WithWriteAccessToSystemKeys([NotNull] this TTransaction trans) where TTransaction : IFdbTransaction { trans.SetOption(FdbTransactionOption.AccessSystemKeys); @@ -63,7 +63,7 @@ public static TTransaction WithWriteAccessToSystemKeys(this TTrans } /// Specifies that this transaction should be treated as highest priority and that lower priority transactions should block behind this one. Use is discouraged outside of low-level tools - public static TTransaction WithPrioritySystemImmediate(this TTransaction trans) + public static TTransaction WithPrioritySystemImmediate([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(FdbTransactionOption.PrioritySystemImmediate); @@ -72,7 +72,7 @@ public static TTransaction WithPrioritySystemImmediate(this TTrans } /// Specifies that this transaction should be treated as low priority and that default priority transactions should be processed first. Useful for doing batch work simultaneously with latency-sensitive work - public static TTransaction WithPriorityBatch(this TTransaction trans) + public static TTransaction WithPriorityBatch([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(FdbTransactionOption.PriorityBatch); @@ -81,7 +81,7 @@ public static TTransaction WithPriorityBatch(this TTransaction tra } /// Reads performed by a transaction will not see any prior mutations that occurred in that transaction, instead seeing the value which was in the database at the transaction's read version. This option may provide a small performance benefit for the client, but also disables a number of client-side optimizations which are beneficial for transactions which tend to read and write the same keys within a single transaction. Also note that with this option invoked any outstanding reads will return errors when transaction commit is called (rather than the normal behavior of commit waiting for outstanding reads to complete). - public static TTransaction WithReadYourWritesDisable(this TTransaction trans) + public static TTransaction WithReadYourWritesDisable([NotNull] this TTransaction trans) where TTransaction : IFdbTransaction { trans.SetOption(FdbTransactionOption.ReadYourWritesDisable); @@ -89,7 +89,7 @@ public static TTransaction WithReadYourWritesDisable(this TTransac } /// Snapshot reads performed by a transaction will see the results of writes done in the same transaction. - public static TTransaction WithSnapshotReadYourWritesEnable(this TTransaction trans) + public static TTransaction WithSnapshotReadYourWritesEnable([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(FdbTransactionOption.SnapshotReadYourWriteEnable); @@ -97,7 +97,7 @@ public static TTransaction WithSnapshotReadYourWritesEnable(this T } /// Reads performed by a transaction will not see the results of writes done in the same transaction. - public static TTransaction WithSnapshotReadYourWritesDisable(this TTransaction trans) + public static TTransaction WithSnapshotReadYourWritesDisable([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(FdbTransactionOption.SnapshotReadYourWriteDisable); @@ -105,7 +105,7 @@ public static TTransaction WithSnapshotReadYourWritesDisable(this } /// Disables read-ahead caching for range reads. Under normal operation, a transaction will read extra rows from the database into cache if range reads are used to page through a series of data one row at a time (i.e. if a range read with a one row limit is followed by another one row range read starting immediately after the result of the first). - public static TTransaction WithReadAheadDisable(this TTransaction trans) + public static TTransaction WithReadAheadDisable([NotNull] this TTransaction trans) where TTransaction : IFdbReadOnlyTransaction { trans.SetOption(FdbTransactionOption.ReadAheadDisable); @@ -113,7 +113,7 @@ public static TTransaction WithReadAheadDisable(this TTransaction } /// The next write performed on this transaction will not generate a write conflict range. As a result, other transactions which read the key(s) being modified by the next write will not conflict with this transaction. Care needs to be taken when using this option on a transaction that is shared between multiple threads. When setting this option, write conflict ranges will be disabled on the next write operation, regardless of what thread it is on. - public static TTransaction WithNextWriteNoWriteConflictRange(this TTransaction trans) + public static TTransaction WithNextWriteNoWriteConflictRange([NotNull] this TTransaction trans) where TTransaction : IFdbTransaction { trans.SetOption(FdbTransactionOption.NextWriteNoWriteConflictRange); @@ -126,8 +126,9 @@ public static TTransaction WithNextWriteNoWriteConflictRange(this /// All pending and any future uses of the transaction will throw an exception. /// The transaction can be used again after it is reset. /// + /// Transaction to use for the operation /// Timeout (with millisecond precision), or TimeSpan.Zero for infinite timeout - public static TTransaction WithTimeout(this TTransaction trans, TimeSpan timeout) + public static TTransaction WithTimeout([NotNull] this TTransaction trans, TimeSpan timeout) where TTransaction : IFdbReadOnlyTransaction { return WithTimeout(trans, timeout == TimeSpan.Zero ? 0 : (int)Math.Ceiling(timeout.TotalMilliseconds)); @@ -139,8 +140,9 @@ public static TTransaction WithTimeout(this TTransaction trans, Ti /// All pending and any future uses of the transaction will throw an exception. /// The transaction can be used again after it is reset. /// + /// Transaction to use for the operation /// Timeout in millisecond, or 0 for infinite timeout - public static TTransaction WithTimeout(this TTransaction trans, int milliseconds) + public static TTransaction WithTimeout([NotNull] this TTransaction trans, int milliseconds) where TTransaction : IFdbReadOnlyTransaction { trans.Timeout = milliseconds; @@ -150,7 +152,7 @@ public static TTransaction WithTimeout(this TTransaction trans, in /// Set a maximum number of retries after which additional calls to onError will throw the most recently seen error code. /// Valid parameter values are [-1, int.MaxValue]. /// If set to -1, will disable the retry limit. - public static TTransaction WithRetryLimit(this TTransaction trans, int retries) + public static TTransaction WithRetryLimit([NotNull] this TTransaction trans, int retries) where TTransaction : IFdbReadOnlyTransaction { trans.RetryLimit = retries; @@ -161,7 +163,7 @@ public static TTransaction WithRetryLimit(this TTransaction trans, /// Defaults to 1000 ms. Valid parameter values are [0, int.MaxValue]. /// If the maximum retry delay is less than the current retry delay of the transaction, then the current retry delay will be clamped to the maximum retry delay. /// - public static TTransaction WithMaxRetryDelay(this TTransaction trans, int milliseconds) + public static TTransaction WithMaxRetryDelay([NotNull] this TTransaction trans, int milliseconds) where TTransaction : IFdbReadOnlyTransaction { trans.MaxRetryDelay = milliseconds; @@ -172,7 +174,7 @@ public static TTransaction WithMaxRetryDelay(this TTransaction tra /// Defaults to 1000 ms. Valid parameter values are [TimeSpan.Zero, TimeSpan.MaxValue]. /// If the maximum retry delay is less than the current retry delay of the transaction, then the current retry delay will be clamped to the maximum retry delay. /// - public static TTransaction WithMaxRetryDelay(this TTransaction trans, TimeSpan delay) + public static TTransaction WithMaxRetryDelay([NotNull] this TTransaction trans, TimeSpan delay) where TTransaction : IFdbReadOnlyTransaction { return WithMaxRetryDelay(trans, delay == TimeSpan.Zero ? 0 : (int)Math.Ceiling(delay.TotalMilliseconds)); @@ -201,7 +203,7 @@ public static Task GetAsync(this IFdbReadOnlyTransaction trans, TKe /// Reads and decode a value from the database snapshot represented by by the current transaction. /// Type of the value. - /// Transaction instance + /// Transaction to use for the operation /// Key to be looked up in the database /// Encoder used to decode the value of the key. /// Task that will return the value of the key if it is found, Slice.Nil if the key does not exist, or an exception @@ -209,10 +211,10 @@ public static Task GetAsync(this IFdbReadOnlyTransaction trans, TKe /// If the cancellation token is already triggered /// If the transaction has already been completed /// If the operation method is called from the Network Thread - public static async Task GetAsync(this IFdbReadOnlyTransaction trans, Slice key, [NotNull] IValueEncoder encoder) + public static async Task GetAsync([NotNull] this IFdbReadOnlyTransaction trans, Slice key, [NotNull] IValueEncoder encoder) { - if (trans == null) throw new ArgumentNullException("trans"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(encoder, nameof(encoder)); return encoder.DecodeValue(await trans.GetAsync(key).ConfigureAwait(false)); } @@ -228,7 +230,7 @@ public static async Task GetAsync(this IFdbReadOnlyTransaction t /// If the cancellation token is already triggered /// If the transaction has already been completed /// If the operation method is called from the Network Thread - public static Task GetAsync(this IFdbReadOnlyTransaction trans, TKey key, [NotNull] IValueEncoder encoder) + public static Task GetAsync([NotNull] this IFdbReadOnlyTransaction trans, TKey key, [NotNull] IValueEncoder encoder) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -244,7 +246,7 @@ public static Task GetAsync(this IFdbReadOnlyTransaction t /// Transaction instance /// /// - public static void Set(this IFdbTransaction trans, TKey key, Slice value) + public static void Set([NotNull] this IFdbTransaction trans, TKey key, Slice value) where TKey : IFdbKey { if (trans == null) throw new ArgumentNullException("trans"); @@ -255,14 +257,14 @@ public static void Set(this IFdbTransaction trans, TKey key, Slice value) /// Set the value of a key in the database, using a custom value encoder. /// Type of the value - /// Transaction instance + /// Transaction to use for the operation /// Key to set /// Value of the key /// Encoder used to convert into a binary slice. - public static void Set(this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder) + public static void Set([NotNull] this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder) { - if (trans == null) throw new ArgumentNullException("trans"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(encoder, nameof(encoder)); trans.Set(key, encoder.EncodeValue(value)); } @@ -274,7 +276,7 @@ public static void Set(this IFdbTransaction trans, Slice key, TValue val /// Key to set /// Value of the key /// Encoder used to convert into a binary slice. - public static void Set(this IFdbTransaction trans, TKey key, TValue value, [NotNull] IValueEncoder encoder) + public static void Set([NotNull] this IFdbTransaction trans, TKey key, TValue value, [NotNull] IValueEncoder encoder) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -282,14 +284,14 @@ public static void Set(this IFdbTransaction trans, TKey key, TValu } /// Set the value of a key in the database, using the content of a Stream - /// Trasaction instance + /// Transaction to use for the operation /// Key to set /// Stream that holds the content of the key, whose length should not exceed the allowed maximum value size. /// This method works best with streams that do not block, like a . For streams that may block, consider using instead. - public static void Set(this IFdbTransaction trans, Slice key, [NotNull] Stream data) + public static void Set([NotNull] this IFdbTransaction trans, Slice key, [NotNull] Stream data) { - if (trans == null) throw new ArgumentNullException("trans"); - if (data == null) throw new ArgumentNullException("data"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(data, nameof(data)); trans.EnsureCanWrite(); @@ -299,14 +301,14 @@ public static void Set(this IFdbTransaction trans, Slice key, [NotNull] Stream d } /// Set the value of a key in the database, by reading the content of a Stream asynchronously - /// Trasaction instance + /// Transaction to use for the operation /// Key to set /// Stream that holds the content of the key, whose length should not exceed the allowed maximum value size. /// If reading from the stream takes more than 5 seconds, the transaction will not be able to commit. For streams that are stored in memory, like a MemoryStream, consider using instead. - public static async Task SetAsync(this IFdbTransaction trans, Slice key, [NotNull] Stream data) + public static async Task SetAsync([NotNull] this IFdbTransaction trans, Slice key, [NotNull] Stream data) { - if (trans == null) throw new ArgumentNullException("trans"); - if (data == null) throw new ArgumentNullException("data"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(data, nameof(data)); trans.EnsureCanWrite(); @@ -320,7 +322,7 @@ public static async Task SetAsync(this IFdbTransaction trans, Slice key, [NotNul #region SetValues /// Set the values of a list of keys in the database. - /// Transaction instance + /// Transaction to use for the operation /// Array of key and value pairs /// /// Only use this method if you know that the approximate size of count of keys and values will not exceed the maximum size allowed per transaction. @@ -328,10 +330,10 @@ public static async Task SetAsync(this IFdbTransaction trans, Slice key, [NotNul /// /// If either or is null. /// If this operation would exceed the maximum allowed size for a transaction. - public static void SetValues(this IFdbTransaction trans, KeyValuePair[] keyValuePairs) + public static void SetValues([NotNull] this IFdbTransaction trans, KeyValuePair[] keyValuePairs) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keyValuePairs == null) throw new ArgumentNullException("keyValuePairs"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keyValuePairs, nameof(keyValuePairs)); foreach (var kv in keyValuePairs) { @@ -340,7 +342,7 @@ public static void SetValues(this IFdbTransaction trans, KeyValuePairSet the values of a list of keys in the database. - /// Transaction instance + /// Transaction to use for the operation /// Array of keys to set /// Array of values for each key. Must be in the same order as and have the same length. /// @@ -350,12 +352,12 @@ public static void SetValues(this IFdbTransaction trans, KeyValuePairIf either , or is null. /// If the does not have the same length as . /// If this operation would exceed the maximum allowed size for a transaction. - public static void SetValues(this IFdbTransaction trans, Slice[] keys, Slice[] values) + public static void SetValues([NotNull] this IFdbTransaction trans, [NotNull] Slice[] keys, [NotNull] Slice[] values) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keys == null) throw new ArgumentNullException("keys"); - if (values == null) throw new ArgumentNullException("values"); - if (values.Length != keys.Length) throw new ArgumentException("Both key and value arrays must have the same size.", "values"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); + Contract.NotNull(values, nameof(values)); + if (values.Length != keys.Length) throw new ArgumentException("Both key and value arrays must have the same size.", nameof(values)); for (int i = 0; i < keys.Length;i++) { @@ -364,7 +366,7 @@ public static void SetValues(this IFdbTransaction trans, Slice[] keys, Slice[] v } /// Set the values of a sequence of keys in the database. - /// Transaction instance + /// Transaction to use for the operation /// Sequence of key and value pairs /// /// Only use this method if you know that the approximate size of count of keys and values will not exceed the maximum size allowed per transaction. @@ -372,10 +374,10 @@ public static void SetValues(this IFdbTransaction trans, Slice[] keys, Slice[] v /// /// If either or is null. /// If this operation would exceed the maximum allowed size for a transaction. - public static void SetValues(this IFdbTransaction trans, IEnumerable> keyValuePairs) + public static void SetValues([NotNull] this IFdbTransaction trans, [NotNull] IEnumerable> keyValuePairs) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keyValuePairs == null) throw new ArgumentNullException("keyValuePairs"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keyValuePairs, nameof(keyValuePairs)); foreach (var kv in keyValuePairs) { @@ -384,7 +386,7 @@ public static void SetValues(this IFdbTransaction trans, IEnumerableSet the values of a sequence of keys in the database. - /// Transaction instance + /// Transaction to use for the operation /// Sequence of keys to set /// Sequence of values for each key. Must be in the same order as and have the same number of elements. /// @@ -394,21 +396,21 @@ public static void SetValues(this IFdbTransaction trans, IEnumerableIf either , or is null. /// If the does not have the same number of elements as . /// If this operation would exceed the maximum allowed size for a transaction. - public static void SetValues(this IFdbTransaction trans, IEnumerable keys, IEnumerable values) + public static void SetValues([NotNull] this IFdbTransaction trans, [NotNull] IEnumerable keys, [NotNull] IEnumerable values) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keys == null) throw new ArgumentNullException("keys"); - if (values == null) throw new ArgumentNullException("values"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); + Contract.NotNull(values, nameof(values)); using(var keyIter = keys.GetEnumerator()) using(var valueIter = values.GetEnumerator()) { while(keyIter.MoveNext()) { - if (!valueIter.MoveNext()) throw new ArgumentException("Both key and value sequences must have the same size.", "values"); + if (!valueIter.MoveNext()) throw new ArgumentException("Both key and value sequences must have the same size.", nameof(values)); trans.Set(keyIter.Current, valueIter.Current); } - if (valueIter.MoveNext()) throw new ArgumentException("Both key and values sequences must have the same size.", "values"); + if (valueIter.MoveNext()) throw new ArgumentException("Both key and values sequences must have the same size.", nameof(values)); } } @@ -417,12 +419,12 @@ public static void SetValues(this IFdbTransaction trans, IEnumerable keys #region Atomic Ops... /// Modify the database snapshot represented by this transaction to add the value of to the value stored by the given . - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Value to add to existing value of key. - public static void AtomicAdd(this IFdbTransaction trans, Slice key, Slice value) + public static void AtomicAdd([NotNull] this IFdbTransaction trans, Slice key, Slice value) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, value, FdbMutationType.Add); } @@ -432,7 +434,7 @@ public static void AtomicAdd(this IFdbTransaction trans, Slice key, Slice value) /// Transaction instance /// Name of the key whose value is to be mutated. /// Value to add to existing value of key. - public static void AtomicAdd(this IFdbTransaction trans, TKey key, Slice value) + public static void AtomicAdd([NotNull] this IFdbTransaction trans, TKey key, Slice value) where TKey : IFdbKey { if (trans == null) throw new ArgumentNullException("trans"); @@ -442,13 +444,13 @@ public static void AtomicAdd(this IFdbTransaction trans, TKey key, Slice v } /// Modify the database snapshot represented by this transaction to perform a bitwise AND between and the value stored by the given . - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicAnd(this IFdbTransaction trans, Slice key, Slice mask) + public static void AtomicAnd([NotNull] this IFdbTransaction trans, Slice key, Slice mask) { //TODO: rename this to AtomicBitAnd(...) ? - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, mask, FdbMutationType.BitAnd); } @@ -458,7 +460,7 @@ public static void AtomicAnd(this IFdbTransaction trans, Slice key, Slice mask) /// Transaction instance /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicAnd(this IFdbTransaction trans, TKey key, Slice mask) + public static void AtomicAnd([NotNull] this IFdbTransaction trans, TKey key, Slice mask) where TKey : IFdbKey { //TODO: rename this to AtomicBitAnd(...) ? @@ -469,13 +471,13 @@ public static void AtomicAnd(this IFdbTransaction trans, TKey key, Slice m } /// Modify the database snapshot represented by this transaction to perform a bitwise OR between and the value stored by the given . - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicOr(this IFdbTransaction trans, Slice key, Slice mask) + public static void AtomicOr([NotNull] this IFdbTransaction trans, Slice key, Slice mask) { //TODO: rename this to AtomicBitOr(...) ? - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, mask, FdbMutationType.BitOr); } @@ -496,13 +498,13 @@ public static void AtomicOr(this IFdbTransaction trans, TKey key, Slice ma } /// Modify the database snapshot represented by this transaction to perform a bitwise XOR between and the value stored by the given . - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicXor(this IFdbTransaction trans, Slice key, Slice mask) + public static void AtomicXor([NotNull] this IFdbTransaction trans, Slice key, Slice mask) { //TODO: rename this to AtomicBitXOr(...) ? - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, mask, FdbMutationType.BitXor); } @@ -523,12 +525,12 @@ public static void AtomicXor(this IFdbTransaction trans, TKey key, Slice m } /// Modify the database snapshot represented by this transaction to update a value if it is larger than the value in the database. - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicMax(this IFdbTransaction trans, Slice key, Slice value) + public static void AtomicMax([NotNull] this IFdbTransaction trans, Slice key, Slice value) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, value, FdbMutationType.Max); } @@ -551,9 +553,9 @@ public static void AtomicMax(this IFdbTransaction trans, TKey key, Slice v /// Transaction instance /// Name of the key whose value is to be mutated. /// Bit mask. - public static void AtomicMin(this IFdbTransaction trans, Slice key, Slice value) + public static void AtomicMin([NotNull] this IFdbTransaction trans, Slice key, Slice value) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.Atomic(key, value, FdbMutationType.Min); } @@ -576,26 +578,26 @@ public static void AtomicMin(this IFdbTransaction trans, TKey key, Slice v #region GetRange... - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, int limit, bool reverse = false) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, int limit, bool reverse = false) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); return trans.GetRange(beginInclusive, endExclusive, new FdbRangeOptions(limit: limit, reverse: reverse)); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null) { return FdbTransactionExtensions.GetRange(trans, FdbKeySelectorPair.Create(range), options); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, FdbKeyRange range, int limit, bool reverse = false) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, int limit, bool reverse = false) { return FdbTransactionExtensions.GetRange(trans, range, new FdbRangeOptions(limit: limit, reverse: reverse)); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, FdbRangeOptions options = null) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); if (beginKeyInclusive.IsNullOrEmpty) beginKeyInclusive = FdbKey.MinValue; if (endKeyExclusive.IsNullOrEmpty) endKeyExclusive = FdbKey.MaxValue; @@ -616,7 +618,7 @@ public static FdbRangeQuery> GetRange(this IFdb return GetRange(trans, beginKeyInclusive.ToFoundationDbKey(), endKeyExclusive.ToFoundationDbKey(), options); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, int limit, bool reverse = false) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, int limit, bool reverse = false) { return GetRange(trans, beginKeyInclusive, endKeyExclusive, new FdbRangeOptions(limit: limit, reverse: reverse)); } @@ -633,12 +635,13 @@ public static FdbRangeQuery> GetRange(this IFdb /// /// Create a new range query that will read all key-value pairs in the database snapshot represented by the transaction /// + /// Transaction to use for the operation /// Pair of key selectors defining the beginning and the end of the range /// Optionnal query options (Limit, TargetBytes, Mode, Reverse, ...) /// Range query that, once executed, will return all the key-value pairs matching the providing selector pair - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); return trans.GetRange(range.Begin, range.End, options); } @@ -648,13 +651,14 @@ public static FdbRangeQuery> GetRange(this IFdbReadOn /// 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. /// + /// Transaction to use for the operation /// key selector pair defining the beginning and the end of the range /// 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. /// - public static Task GetRangeAsync(this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null, int iteration = 0) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null, int iteration = 0) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); return trans.GetRangeAsync(range.Begin, range.End, options, iteration); } @@ -664,13 +668,14 @@ public static Task GetRangeAsync(this IFdbReadOnlyTransaction tra /// 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. /// + /// Transaction to use for the operation /// Range of keys defining the beginning (inclusive) and the end (exclusive) of the range /// 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. /// - public static Task GetRangeAsync(this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null, int iteration = 0) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null, int iteration = 0) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); var sp = FdbKeySelectorPair.Create(range); return trans.GetRangeAsync(sp.Begin, sp.End, options, iteration); @@ -681,14 +686,15 @@ public static Task GetRangeAsync(this IFdbReadOnlyTransaction tra /// 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. /// + /// Transaction to use for the operation /// Key defining the beginning (inclusive) of the range /// Key defining the end (exclusive) of the range /// 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. /// - public static Task GetRangeAsync(this IFdbReadOnlyTransaction trans, Slice beginInclusive, Slice endExclusive, FdbRangeOptions options = null, int iteration = 0) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, Slice beginInclusive, Slice endExclusive, FdbRangeOptions options = null, int iteration = 0) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); var range = FdbKeySelectorPair.Create(beginInclusive, endExclusive); return trans.GetRangeAsync(range.Begin, range.End, options, iteration); @@ -715,10 +721,11 @@ public static void Clear(this IFdbTransaction trans, TKey key) /// Modify the database snapshot represented by this transaction to remove all keys (if any) which are lexicographically greater than or equal to the given begin key and lexicographically less than the given end_key. /// Sets and clears affect the actual database only if transaction is later committed with CommitAsync(). /// + /// Transaction to use for the operation /// Pair of keys defining the range to clear. - public static void ClearRange(this IFdbTransaction trans, FdbKeyRange range) + public static void ClearRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.ClearRange(range.Begin, range.End); } @@ -730,11 +737,12 @@ public static void ClearRange(this IFdbTransaction trans, FdbKeyRange range) /// /// Adds a conflict range to a transaction without performing the associated read or write. /// + /// Transaction to use for the operation /// Range of the keys specifying the conflict range. The end key is excluded /// One of the FDBConflictRangeType values indicating what type of conflict range is being set. - public static void AddConflictRange(this IFdbTransaction trans, FdbKeyRange range, FdbConflictRangeType type) + public static void AddConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range, FdbConflictRangeType type) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.AddConflictRange(range.Begin, range.End, type); } @@ -743,7 +751,7 @@ public static void AddConflictRange(this IFdbTransaction trans, FdbKeyRange rang /// /// Adds a range of keys to the transaction’s read conflict ranges as if you had read the range. As a result, other transactions that write a key in this range could cause the transaction to fail with a conflict. /// - public static void AddReadConflictRange(this IFdbTransaction trans, FdbKeyRange range) + public static void AddReadConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) { AddConflictRange(trans, range, FdbConflictRangeType.Read); } @@ -751,9 +759,9 @@ public static void AddReadConflictRange(this IFdbTransaction trans, FdbKeyRange /// /// Adds a range of keys to the transaction’s read conflict ranges as if you had read the range. As a result, other transactions that write a key in this range could cause the transaction to fail with a conflict. /// - public static void AddReadConflictRange(this IFdbTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive) + public static void AddReadConflictRange([NotNull] this IFdbTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.AddConflictRange(beginKeyInclusive, endKeyExclusive, FdbConflictRangeType.Read); } @@ -774,7 +782,7 @@ public static void AddReadConflictRange(this IFdbTransaction trans, TKey b /// /// Adds a key to the transaction’s read conflict ranges as if you had read the key. As a result, other transactions that write to this key could cause the transaction to fail with a conflict. /// - public static void AddReadConflictKey(this IFdbTransaction trans, Slice key) + public static void AddReadConflictKey([NotNull] this IFdbTransaction trans, Slice key) { AddConflictRange(trans, FdbKeyRange.FromKey(key), FdbConflictRangeType.Read); } @@ -792,7 +800,7 @@ public static void AddReadConflictKey(this IFdbTransaction trans, TKey key /// /// Adds a range of keys to the transaction’s write conflict ranges as if you had cleared the range. As a result, other transactions that concurrently read a key in this range could fail with a conflict. /// - public static void AddWriteConflictRange(this IFdbTransaction trans, FdbKeyRange range) + public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) { AddConflictRange(trans, range, FdbConflictRangeType.Write); } @@ -800,7 +808,7 @@ public static void AddWriteConflictRange(this IFdbTransaction trans, FdbKeyRange /// /// Adds a range of keys to the transaction’s write conflict ranges as if you had cleared the range. As a result, other transactions that concurrently read a key in this range could fail with a conflict. /// - public static void AddWriteConflictRange(this IFdbTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive) + public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive) { if (trans == null) throw new ArgumentNullException("trans"); @@ -823,7 +831,7 @@ public static void AddWriteConflictRange(this IFdbTransaction trans, TKey /// /// Adds a key to the transaction’s write conflict ranges as if you had cleared the key. As a result, other transactions that concurrently read this key could fail with a conflict. /// - public static void AddWriteConflictKey(this IFdbTransaction trans, Slice key) + public static void AddWriteConflictKey([NotNull] this IFdbTransaction trans, Slice key) { AddConflictRange(trans, FdbKeyRange.FromKey(key), FdbConflictRangeType.Write); } @@ -847,9 +855,9 @@ public static void AddWriteConflictKey(this IFdbTransaction trans, TKey ke /// Key to be looked up in the database /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property contains the current value of the key. - public static async Task GetAndWatchAsync(this IFdbTransaction trans, Slice key, CancellationToken cancellationToken) + public static async Task GetAndWatchAsync([NotNull] this IFdbTransaction trans, Slice key, CancellationToken cancellationToken) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); cancellationToken.ThrowIfCancellationRequested(); var value = await trans.GetAsync(key); @@ -878,9 +886,9 @@ public static Task GetAndWatchAsync(this IFdbTransaction trans, /// Value to be inserted into the database. /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch(this IFdbTransaction trans, Slice key, Slice value, CancellationToken cancellationToken) + public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, Slice value, CancellationToken cancellationToken) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); cancellationToken.ThrowIfCancellationRequested(); trans.Set(key, value); @@ -912,9 +920,9 @@ public static FdbWatch SetAndWatch(this IFdbTransaction trans, TKey key, S /// Encoder use to convert into a slice /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch(this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder, CancellationToken cancellationToken) + public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder, CancellationToken cancellationToken) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); cancellationToken.ThrowIfCancellationRequested(); return SetAndWatch(trans, key, encoder.EncodeValue(value), cancellationToken); } @@ -944,13 +952,14 @@ public static FdbWatch SetAndWatch(this IFdbTransaction trans, TKe /// /// Reads several values from the database snapshot represented by the current transaction /// + /// Transaction to use for the operation /// Sequence of keys to be looked up in the database /// Task that will return an array of values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value will be Slice.Nil. [ItemNotNull] - public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) + public static Task GetValuesAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); var array = keys as Slice[] ?? keys.ToArray(); @@ -960,13 +969,15 @@ public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [ /// /// Reads several values from the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . [ItemNotNull] - public static async Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) + public static async Task GetValuesAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) { - if (decoder == null) throw new ArgumentNullException("decoder"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(decoder, nameof(decoder)); return decoder.DecodeValues(await GetValuesAsync(trans, keys).ConfigureAwait(false)); } @@ -1003,13 +1014,14 @@ public static Task GetValuesAsync(this IFdbReadOnlyTrans /// /// Resolves several key selectors against the keys in the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Sequence of key selectors to resolve /// Task that will return an array of keys matching the selectors, or an exception [ItemNotNull] - public static Task GetKeysAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable selectors) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable selectors) { - if (trans == null) throw new ArgumentNullException("trans"); - if (selectors == null) throw new ArgumentNullException("selectors"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(selectors, nameof(selectors)); var array = selectors as FdbKeySelector[] ?? selectors.ToArray(); @@ -1019,13 +1031,15 @@ public static Task GetKeysAsync(this IFdbReadOnlyTransaction trans, [No /// /// Reads several values from the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Sequence of keys to be looked up in the database /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. /// This method is equivalent to calling , except that it will return the keys in addition to the values. [ItemNotNull] - public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) + public static Task[]> GetBatchAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) { - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); var array = keys as Slice[] ?? keys.ToArray(); @@ -1035,14 +1049,15 @@ public static Task[]> GetBatchAsync(this IFdbReadOnly /// /// Reads several values from the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Array of keys to be looked up in the database /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. /// This method is equivalent to calling , except that it will return the keys in addition to the values. [ItemNotNull] - public static async Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys) + public static async Task[]> GetBatchAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); var results = await trans.GetValuesAsync(keys).ConfigureAwait(false); Contract.Assert(results != null && results.Length == keys.Length); @@ -1058,13 +1073,15 @@ public static async Task[]> GetBatchAsync(this IFdbRe /// /// Reads several values from the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Array of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . [ItemNotNull] - public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) + public static Task[]> GetBatchAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) { - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); var array = keys as Slice[] ?? keys.ToArray(); @@ -1074,15 +1091,16 @@ public static Task[]> GetBatchAsync(this IFd /// /// Reads several values from the database snapshot represented by the current transaction. /// + /// Transaction to use for the operation /// Sequence of keys to be looked up in the database /// Decoder used to decoded the results into values of type /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . [ItemNotNull] - public static async Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys, [NotNull] IValueEncoder decoder) + public static async Task[]> GetBatchAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] Slice[] keys, [NotNull] IValueEncoder decoder) { - if (trans == null) throw new ArgumentNullException("trans"); - if (keys == null) throw new ArgumentNullException("keys"); - if (decoder == null) throw new ArgumentNullException("decoder"); + Contract.NotNull(trans, nameof(trans)); + Contract.NotNull(keys, nameof(keys)); + Contract.NotNull(decoder, nameof(decoder)); var results = await trans.GetValuesAsync(keys).ConfigureAwait(false); Contract.Assert(results != null && results.Length == keys.Length); @@ -1133,10 +1151,10 @@ public static Task[]> GetBatchAsync(th /// Token used to cancel the operation /// Task returning the list of all the elements of the async enumerable returned by the last successfull call to . [ItemNotNull] - public static Task> QueryAsync(this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken cancellationToken) + public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + Contract.NotNull(db, nameof(db)); + Contract.NotNull(handler, nameof(handler)); return db.ReadAsync( (tr) => diff --git a/FoundationDB.Client/FdbWatch.cs b/FoundationDB.Client/FdbWatch.cs index f447b5b36..e113cf397 100644 --- a/FoundationDB.Client/FdbWatch.cs +++ b/FoundationDB.Client/FdbWatch.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; using System; using System.Diagnostics; using System.Runtime.CompilerServices; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client.Native; /// Watch that triggers when the watched key is changed in the database [DebuggerDisplay("Status={m_future.Task.Status}, Key={m_key}")] diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index f6d501920..9a0c4782d 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -26,15 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Filters { - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Base class for simple database filters [DebuggerDisplay("Database={m_database.Name}")] @@ -63,7 +65,7 @@ public abstract class FdbDatabaseFilter : IFdbDatabase protected FdbDatabaseFilter([NotNull] IFdbDatabase database, bool forceReadOnly, bool ownsDatabase) { - if (database == null) throw new ArgumentNullException("database"); + Contract.NotNull(database, nameof(database)); m_database = database; m_readOnly = forceReadOnly || database.IsReadOnly; @@ -205,7 +207,7 @@ public virtual Slice[] ConcatKeys(IEnumerable keys) return m_database.ConcatKeys(keys); } - public virtual Slice ExtractKey(Slice key, bool boundCheck = false) + public virtual Slice ExtractKey(Slice key, bool boundCheck = false) { return m_database.ExtractKey(key, boundCheck); } diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index a5b02e8d0..8f920bf1d 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -41,7 +41,7 @@ public enum FdbLoggingOptions { /// Default logging options Default = 0, - + /// Capture the stacktrace of the caller method that created the transaction RecordCreationStackTrace = 0x100, @@ -277,7 +277,7 @@ public override void Reset() public override FdbWatch Watch(Slice key, CancellationToken cancellationToken) { var cmd = new FdbTransactionLog.WatchCommand(Grab(key)); - this.Log.AddOperation(cmd); + this.Log.AddOperation(cmd); return m_transaction.Watch(cmd.Key, cancellationToken); } diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs index 2da17d4a0..497c215a3 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs @@ -28,9 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Filters.Logging { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Concurrent; using System.Diagnostics; @@ -38,6 +35,9 @@ namespace FoundationDB.Filters.Logging using System.Reflection; using System.Text; using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Container that logs all operations performed by a transaction public sealed partial class FdbTransactionLog diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 55ce4f0b9..4bcb96fe0 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -228,9 +228,11 @@ + + diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 14afa4d63..5c0d29192 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -28,18 +28,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Directories { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; - using FoundationDB.Filters.Logging; using System; using System.Collections.Generic; using System.Diagnostics; using System.Linq; using System.Threading.Tasks; using JetBrains.Annotations; - + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using FoundationDB.Linq; + using FoundationDB.Filters.Logging; /// Provides a FdbDirectoryLayer class for managing directories in FoundationDB. /// Directories are a recommended approach for administering layers and applications. Directories work in conjunction with subspaces. Each layer or application should create or open at least one directory with which to manage its subspace(s). @@ -64,19 +63,19 @@ public class FdbDirectoryLayer : IFdbDirectory public static bool AnnotateTransactions { get; set; } /// Subspace where the content of each folder will be stored - public IFdbDynamicSubspace ContentSubspace { [NotNull] get; private set; } + public IFdbDynamicSubspace ContentSubspace { [NotNull] get; } /// Subspace where all the metadata nodes for each folder will be stored - public IFdbDynamicSubspace NodeSubspace { [NotNull] get; private set; } + public IFdbDynamicSubspace NodeSubspace { [NotNull] get; } /// Root node of the directory - internal IFdbDynamicSubspace RootNode { [NotNull] get; private set; } + internal IFdbDynamicSubspace RootNode { [NotNull] get; } /// Allocated used to generated prefix for new content - internal FdbHighContentionAllocator Allocator { [NotNull] get; private set; } + internal FdbHighContentionAllocator Allocator { [NotNull] get; } /// Gets the path for the root node of this FdbDirectoryLayer. - internal IFdbTuple Location { [NotNull] get; private set; } + internal IFdbTuple Location { [NotNull] get; } /// Name of root directory of this layer /// Returns String.Empty for the root Directory Layer, or the name of the partition @@ -636,7 +635,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly await CheckWriteVersionAsync(trans).ConfigureAwait(false); - if (prefix == null) + if (prefix.IsNull) { // automatically allocate a new prefix inside the ContentSubspace long id = await this.Allocator.AllocateAsync(trans).ConfigureAwait(false); prefix = this.ContentSubspace.Keys.Encode(id); diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index 4ab736c05..3a1cf408f 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Directories { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// A Directory Subspace represents the contents of a directory, but it also remembers the path with which it was opened and offers convenience methods to operate on the directory at that path. /// An instance of DirectorySubspace can be used for all the usual subspace operations. It can also be used to operate on the directory with which it was opened. diff --git a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs index cf9b268cc..32cc1be74 100644 --- a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs +++ b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs @@ -101,9 +101,9 @@ public async Task AllocateAsync([NotNull] IFdbTransaction trans) trans.AtomicAdd(this.Counters.Keys.Encode(start), Slice.FromFixed64(1)); // As of the snapshot being read from, the window is less than half - // full, so this should be expected to take 2 tries. Under high - // contention (and when the window advances), there is an additional - // subsequent risk of conflict for this transaction. + // full, so this should be expected to take 2 tries. Under high + // contention (and when the window advances), there is an additional + // subsequent risk of conflict for this transaction. while (true) { // Find a random free slot in the current window... diff --git a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs index e15b126d3..80abb61f6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that adds a value at the end of an already existing tuple /// Type of the last value of the tuple diff --git a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs index 738e183a9..260ad099e 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; using System; using System.Collections; using System.Collections.Generic; using System.Linq; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; /// Tuple that can hold any number of untyped items public sealed class FdbListTuple : IFdbTuple diff --git a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs index 2e03ac6d6..a906f3dad 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; using System; using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; /// Represents an immutable tuple where the packed bytes are cached [DebuggerDisplay("{ToString()}")] diff --git a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs index 2e007c03f..aa986c60b 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that has a fixed abitrary binary prefix [DebuggerDisplay("{ToString()}")] diff --git a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs index 871fc8354..90c4ff6dd 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; using System; using System.Collections; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; /// Lazily-evaluated tuple that was unpacked from a key internal sealed class FdbSlicedTuple : IFdbTuple diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs index d6fad48ac..b17d9cf5e 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs @@ -28,9 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; @@ -38,6 +35,9 @@ namespace FoundationDB.Layers.Tuples using System.Globalization; using System.Linq; using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Factory class for Tuples [PublicAPI] diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs index ab944cbf0..488616fce 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs @@ -28,15 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Globalization; using System.Linq.Expressions; using System.Reflection; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Helper methods used during serialization of values to the tuple binary format public static class FdbTuplePackers diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs index 15baae74c..dca0460f9 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Helper class that contains low-level encoders for the tuple binary format public static class FdbTupleParser diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs index 62aca1e76..3655ebe52 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs index 497928878..bc2b71b1b 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs index d55aa3c52..1a78ff6a1 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs index 5af99cd7f..9cd1318f5 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs @@ -26,18 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; - using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs index d0cfbe70a..488eef83e 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs index 9a0532dda..575fb0a5f 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; public static partial class FdbAsyncEnumerable { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs index b64c3be34..342368a4b 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client; + using JetBrains.Annotations; /// Provides a set of static methods for querying objects that implement . public static partial class FdbAsyncEnumerable diff --git a/FoundationDB.Client/Linq/FdbAsyncSequence.cs b/FoundationDB.Client/Linq/FdbAsyncSequence.cs index 0836d6013..0acedcd17 100644 --- a/FoundationDB.Client/Linq/FdbAsyncSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncSequence.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { + using System; + using Doxense.Diagnostics.Contracts; using FoundationDB.Async; - using FoundationDB.Client.Utils; using JetBrains.Annotations; - using System; /// Wraps an async sequence of items into another async sequence of items /// Type of elements of the inner async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs b/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs index ac5cc6964..3f7af64b5 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; /// Generate items asynchronously, using a user-provided lambda /// Type of the items produced by this generator diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs index 6c1eff46e..320a4b3f1 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; internal abstract class FdbAsyncFilterIterator : FdbAsyncIterator { diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs index 7a9ea51ad..2cdae90c0 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs @@ -30,13 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Diagnostics; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; /// Pump that repeatedly calls MoveNext on an iterator and tries to publish the values in a Producer/Consumer queue /// diff --git a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs index 7384ac696..c490eb2ad 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; /// Packs items from an inner sequence, into a sequence of fixed-size arrays. /// Type the the items from the source sequence @@ -49,7 +49,7 @@ internal class FdbBatchingAsyncIterator : FdbAsyncFilterIterator m_buffer; + private List m_buffer; /// Create a new batching iterator /// Source sequence of items that must be batched by waves diff --git a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs index 2f4b74042..544e6465c 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Filters duplicate items from an async sequence /// Type of elements of the async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs index d32c851dd..ffafd8aef 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; /// Observe the items of an async sequence /// Type of the observed elements diff --git a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs index 579d163d8..d44d9dc5b 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs @@ -30,13 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; /// [EXPERIMENTAL] Iterates over an async sequence of items, kick off an async task in parallel, and returning the results in order /// Type of elements of the inner async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs index 704844077..0d303db18 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; /// Prefetches items from the inner sequence, before outputing them down the line. /// Type the the items from the source sequence @@ -52,7 +52,7 @@ internal class FdbPrefetchingAsyncIterator : FdbAsyncFilterIterator m_buffer; // holds on to the last pending call to m_iterator.MoveNext() when our buffer is full - private Task m_nextTask; + private Task m_nextTask; /// Create a new batching iterator /// Source sequence of items that must be batched by waves diff --git a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs index 4d26e9f67..802f62260 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Iterates over an async sequence of items /// Type of elements of the inner async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs index 08f674efe..62e79a731 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Reads an async sequence of items until a condition becomes false /// Type of elements of the async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs index a4529d08e..ca2f3dadf 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; /// Filters an async sequence of items /// Type of elements of the async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs index 8d5418c22..84da430f5 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Iterates over an async sequence of items /// Type of elements of the inner async sequence diff --git a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs index 78f0edaef..7e357e396 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; /// Merges bursts of already-completed items from a source async sequence, into a sequence of batches. /// Type the the items from the source sequence @@ -94,7 +94,7 @@ internal class FdbWindowingAsyncIterator : FdbAsyncFilterIterator m_buffer; // holds on to the last pending call to m_iterator.MoveNext() when our buffer is full - private Task m_nextTask; + private Task m_nextTask; /// Create a new batching iterator /// Source sequence of items that must be batched by waves diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/FdbFuture.cs index 11211c2dd..e17fcd1bb 100644 --- a/FoundationDB.Client/Native/FdbFuture.cs +++ b/FoundationDB.Client/Native/FdbFuture.cs @@ -31,8 +31,6 @@ 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; @@ -40,6 +38,9 @@ namespace FoundationDB.Client.Native using System.Runtime.CompilerServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client.Utils; + using JetBrains.Annotations; /// Helper class to create FDBFutures internal static class FdbFuture diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 3c5d8dbbe..8aabe789e 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -29,18 +29,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY // enable this to help debug native calls to fdbc.dll //#define DEBUG_NATIVE_CALLS -using FoundationDB.Client.Utils; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Runtime.CompilerServices; -using System.Runtime.ExceptionServices; -using System.Runtime.InteropServices; -using System.Text; - namespace FoundationDB.Client.Native { + using System; + using System.Collections.Generic; + using System.IO; + using System.Runtime.ExceptionServices; + using System.Runtime.InteropServices; + using Doxense.Diagnostics.Contracts; + internal static unsafe class FdbNative { public const int FDB_API_MIN_VERSION = 200; diff --git a/FoundationDB.Client/Native/FdbNativeCluster.cs b/FoundationDB.Client/Native/FdbNativeCluster.cs index 1a8831418..1976acf77 100644 --- a/FoundationDB.Client/Native/FdbNativeCluster.cs +++ b/FoundationDB.Client/Native/FdbNativeCluster.cs @@ -28,13 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Native { - using FoundationDB.Async; - using FoundationDB.Client.Core; - using FoundationDB.Client.Utils; using System; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client.Core; /// Wraps a native FDBCluster* handle internal sealed class FdbNativeCluster : IFdbClusterHandler @@ -95,7 +94,7 @@ public void SetOption(FdbClusterOption option, Slice data) public Task OpenDatabaseAsync(string databaseName, CancellationToken cancellationToken) { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (cancellationToken.IsCancellationRequested) return Task.FromCanceled(cancellationToken); var future = FdbNative.ClusterCreateDatabase(m_handle, databaseName); return FdbFuture.CreateTaskFromHandle( @@ -110,7 +109,7 @@ public Task OpenDatabaseAsync(string databaseName, Cancella throw Fdb.MapToException(err); } var handler = new FdbNativeDatabase(database); - return (IFdbDatabaseHandler) handler; + return (IFdbDatabaseHandler) handler; }, cancellationToken ); diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index d3fc1037e..65db78649 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -33,14 +33,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Native { - using FoundationDB.Client.Core; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client.Core; + using JetBrains.Annotations; /// Wraps a native FDB_TRANSACTION handle [DebuggerDisplay("Handle={m_handle}, Size={m_payloadBytes}, Closed={m_handle.IsClosed}")] @@ -189,7 +189,7 @@ public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationTok { Contract.Requires(keys != null); - if (keys.Length == 0) return Task.FromResult(Slice.EmptySliceArray); + if (keys.Length == 0) return Task.FromResult(Array.Empty()); var futures = new FutureHandle[keys.Length]; try diff --git a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs index a53728466..c9f0ac156 100644 --- a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Layers.Directories; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Layers.Directories; + using JetBrains.Annotations; /// View of a database that is bound to a specific Partition [DebuggerDisplay("Database={Database.Name}, Partition={Directory.FullName}, Prefix={Database.GlobalSpace}")] diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs index 610e1a430..cee84e75f 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs @@ -26,12 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Diagnostics; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Diagnostics; + using JetBrains.Annotations; public class FdbDynamicSubspace : FdbSubspace, IFdbDynamicSubspace { /// Encoder for the keys of this subspace @@ -77,4 +76,4 @@ public FdbDynamicSubspacePartition Partition } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs index 271f98355..fbd3ef2c7 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs @@ -26,16 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Collections; -using System.Collections.Generic; -using System.Linq; -using FoundationDB.Client.Utils; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using Doxense.Diagnostics.Contracts; + using System.Collections.Generic; + using System.Linq; + using FoundationDB.Client.Utils; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; internal static class Batched { @@ -47,7 +46,7 @@ public static Slice[] Convert(SliceWriter writer, [NotNull, ItemNotNull] IEnumer { Contract.Requires(values != null && handler != null); - //Note on performance: + //Note on performance: // - we will reuse the same buffer for each temp key, and copy them into a slice buffer // - doing it this way adds a memory copy (writer => buffer) but reduce the number of byte[] allocations (and reduce the GC overhead) @@ -97,12 +96,10 @@ public struct FdbDynamicSubspaceKeys //NOTE: everytime an IFdbTuple is used here, it is as a container (vector of objects), and NOT as the Tuple Encoding scheme ! (separate concept) /// Parent subspace - [NotNull] - public readonly IFdbSubspace Subspace; + [NotNull] public readonly IFdbSubspace Subspace; /// Encoder used to format keys in this subspace - [NotNull] - public readonly IDynamicKeyEncoder Encoder; + [NotNull] public readonly IDynamicKeyEncoder Encoder; public FdbDynamicSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { @@ -151,7 +148,7 @@ public Slice this[[NotNull] ITupleFormattable item] /// Tuple that will be packed and appended to the subspace prefix public Slice Pack([NotNull] IFdbTuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + Contract.NotNull(tuple, nameof(tuple)); var writer = this.Subspace.GetWriter(); this.Encoder.PackKey(ref writer, tuple); @@ -211,7 +208,7 @@ public Slice[] EncodeMany(IEnumerable items) items, (ref SliceWriter writer, T item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, item), this.Encoder - ); + ); } /// Encode a batch of keys, each one composed of a single value extracted from each elements @@ -597,4 +594,4 @@ public IFdbTuple Append(T1 item1, T2 item2, T3 item3, T4 ite #endregion } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs index 03f7489e3..1008c4a7c 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs @@ -28,9 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { + using System; + using Doxense.Diagnostics.Contracts; using FoundationDB.Layers.Tuples; using JetBrains.Annotations; - using System; public struct FdbDynamicSubspacePartition { @@ -42,8 +43,8 @@ public struct FdbDynamicSubspacePartition public FdbDynamicSubspacePartition([NotNull] IFdbDynamicSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); this.Subspace = subspace; this.Encoder = encoder; } @@ -55,9 +56,10 @@ public FdbDynamicSubspacePartition([NotNull] IFdbDynamicSubspace subspace, [NotN /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. /// If you are calling this in a loop, consider creating a new subspace using that encoding. /// + [Pure] public FdbDynamicSubspacePartition Using([NotNull] IFdbKeyEncoding encoding) { - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(encoding, nameof(encoding)); var encoder = encoding.GetDynamicEncoder(); return UsingEncoder(encoder); } @@ -69,6 +71,7 @@ public FdbDynamicSubspacePartition Using([NotNull] IFdbKeyEncoding encoding) /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. /// If you are calling this in a loop, consider creating a new subspace using that encoder. /// + [Pure] public FdbDynamicSubspacePartition UsingEncoder([NotNull] IDynamicKeyEncoder encoder) { return new FdbDynamicSubspacePartition(this.Subspace, encoder); @@ -82,7 +85,7 @@ public IFdbDynamicSubspace this[Slice suffix] [NotNull] get { - if (suffix.IsNull) throw new ArgumentException("Partition suffix cannot be null", "suffix"); + if (suffix.IsNull) throw new ArgumentException("Partition suffix cannot be null", nameof(suffix)); //TODO: find a way to limit the number of copies of the key? return new FdbDynamicSubspace(this.Subspace.ConcatKey(suffix), false, this.Encoder); } @@ -118,7 +121,7 @@ public IFdbDynamicSubspace this[ITupleFormattable item] [ContractAnnotation("null => halt; notnull => notnull")] get { - if (item == null) throw new ArgumentNullException("item"); + Contract.NotNull(item, nameof(item)); var tuple = item.ToTuple(); if (tuple == null) throw new InvalidOperationException("Formattable item returned an empty tuple"); return this[tuple]; @@ -133,7 +136,7 @@ public IFdbDynamicSubspace this[ITupleFormattable item] /// /// new FdbSubspace(["Users", ]).Partition("Contacts") == new FdbSubspace(["Users", "Contacts", ]) /// - [NotNull] + [Pure, NotNull] public IFdbDynamicSubspace ByKey(T value) { return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value), false, this.Encoder); @@ -149,7 +152,7 @@ public IFdbDynamicSubspace ByKey(T value) /// /// new FdbSubspace(["Users", ]).Partition("Contacts", "Friends") == new FdbSubspace(["Users", "Contacts", "Friends", ]) /// - [NotNull] + [Pure, NotNull] public IFdbDynamicSubspace ByKey(T1 value1, T2 value2) { return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2), false, this.Encoder); @@ -166,7 +169,7 @@ public IFdbDynamicSubspace ByKey(T1 value1, T2 value2) /// /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", ]) /// - [NotNull] + [Pure, NotNull] public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3) { return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2, value3), false, this.Encoder); @@ -185,7 +188,7 @@ public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3) /// /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends", "Messages") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", "Messages", ]) /// - [NotNull] + [Pure, NotNull] public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) { return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2, value3, value4), false, this.Encoder); diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs index bc507e1b6..42a9b27a7 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs @@ -26,14 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Collections.Generic; -using System.Diagnostics.Contracts; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Collections.Generic; + using System.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + public struct FdbEncoderSubspaceKeys { @@ -83,4 +83,4 @@ public FdbKeyRange ToRange(T value) } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs index 7e35b28f6..04585dc81 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs @@ -26,14 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Collections.Generic; -using System.Diagnostics.Contracts; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + public struct FdbEncoderSubspaceKeys { @@ -62,7 +62,7 @@ public Slice Encode(T1 value1, T2 value2) public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2) { - if (values == null) throw new ArgumentNullException("values"); + Contract.NotNull(values, nameof(values)); return Batched>.Convert( this.Subspace.GetWriter(), values, @@ -83,4 +83,4 @@ public FdbKeyRange ToRange(T1 value1, T2 value2) } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs index d175c05ad..ccaf815f5 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs @@ -26,13 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Collections.Generic; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Collections.Generic; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + public struct FdbEncoderSubspaceKeys { @@ -78,4 +78,4 @@ public FdbKeyRange ToRange(T1 value1, T2 value2, T3 value3) } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs index 9983cbda0..b5ff8db2b 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs @@ -26,14 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Collections.Generic; -using System.Diagnostics.Contracts; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Collections.Generic; + using System.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + public struct FdbEncoderSubspaceKeys { @@ -84,4 +84,4 @@ public FdbKeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs index f2653bb06..0c99c2788 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs @@ -26,11 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using JetBrains.Annotations; + public struct FdbEncoderSubspacePartition { [NotNull] @@ -83,4 +83,4 @@ public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, IKeyEncoder } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs index f2d174088..a66e9d73a 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs @@ -26,12 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using System.Diagnostics.Contracts; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using System.Diagnostics.Contracts; + using JetBrains.Annotations; + public struct FdbEncoderSubspacePartition { [NotNull] @@ -84,4 +84,4 @@ public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs index 56191ebb4..5e60d3c36 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs @@ -26,12 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { + using System; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + using FoundationDB.Client.Utils; + public struct FdbEncoderSubspacePartition { [NotNull] @@ -84,4 +85,4 @@ public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs index 645c17fc1..1bcd94cd1 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs @@ -26,11 +26,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using JetBrains.Annotations; /// Subspace that knows how to encode and decode its key /// Type of the key handled by this subspace @@ -70,4 +69,4 @@ public FdbEncoderSubspacePartition Partition } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs index 64d522a43..9e4af1197 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs @@ -26,11 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Subspace that knows how to encode and decode its key /// Type of the first item of the keys handled by this subspace @@ -49,7 +49,7 @@ public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) : base(rawPrefix, copy) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); m_encoder = encoder; m_keys = new FdbEncoderSubspaceKeys(this, encoder); } @@ -78,4 +78,4 @@ public FdbEncoderSubspacePartition Partition } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs index 444f91b9a..213c8cea4 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs @@ -26,11 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Subspace that knows how to encode and decode its key /// Type of the first item of the keys handled by this subspace @@ -52,7 +52,7 @@ public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) : base(rawPrefix, copy) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); m_encoder = encoder; m_keys = new FdbEncoderSubspaceKeys(this, encoder); } @@ -84,4 +84,4 @@ public FdbEncoderSubspacePartition Partition } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs index 787cb5131..3c28c1ad2 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs @@ -26,16 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using JetBrains.Annotations; + using Doxense.Diagnostics.Contracts; /// Subspace that knows how to encode and decode its key /// Type of the first item of the keys handled by this subspace /// Type of the second item of the keys handled by this subspace /// Type of the third item of the keys handled by this subspace + /// Type of the fourth item of the keys handled by this subspace public class FdbEncoderSubspace : FdbSubspace, IFdbEncoderSubspace { private readonly ICompositeKeyEncoder m_encoder; @@ -52,7 +53,7 @@ public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) : base(rawPrefix, copy) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); m_encoder = encoder; m_keys = new FdbEncoderSubspaceKeys(this, encoder); } @@ -84,4 +85,4 @@ public FdbEncoderSubspacePartition Partition } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/FdbSubspace.cs index f780f0fca..47391d146 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspace.cs @@ -26,15 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System.Linq; - namespace FoundationDB.Client { - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// Adds a prefix on every keys, to group them inside a common subspace [PublicAPI] @@ -58,10 +58,10 @@ protected Slice InternalKey /// Wraps an existing subspace, without copying the prefix (if possible) protected FdbSubspace([NotNull] IFdbSubspace copy) { - if (copy == null) throw new ArgumentNullException("copy"); + Contract.NotNull(copy, nameof(copy)); var sub = copy as FdbSubspace; Slice key = sub != null ? sub.m_rawPrefix : copy.ToFoundationDbKey(); - if (key.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", "copy"); + if (key.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(copy)); m_rawPrefix = key; } @@ -70,7 +70,7 @@ protected FdbSubspace([NotNull] IFdbSubspace copy) /// If true, take a copy of the prefix internal FdbSubspace(Slice rawPrefix, bool copy) { - if (rawPrefix.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", "rawPrefix"); + if (rawPrefix.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(rawPrefix)); if (copy) rawPrefix = rawPrefix.Memoize(); m_rawPrefix = rawPrefix.Memoize(); } @@ -88,7 +88,7 @@ public FdbSubspace(Slice rawPrefix) /// Create a new Subspace using a binary key as the prefix /// Prefix of the new subspace /// New subspace that will use a copy of as its prefix - [NotNull] + [Pure, NotNull] public static IFdbSubspace Create(Slice slice) { return new FdbDynamicSubspace(slice, TypeSystem.Default.GetDynamicEncoder()); @@ -105,7 +105,7 @@ public static IFdbSubspace Create([NotNull] TKey key) /// Prefix of the new subspace /// Type System used to encode the keys of this subspace /// New subspace that will use a copy of as its prefix - [NotNull] + [Pure, NotNull] public static IFdbDynamicSubspace CreateDynamic(Slice slice, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); @@ -116,10 +116,10 @@ public static IFdbDynamicSubspace CreateDynamic(Slice slice, IFdbKeyEncoding enc /// Prefix of the new subspace /// Type System used to encode the keys of this subspace /// New subspace that will use a copy of as its prefix - [NotNull] + [Pure, NotNull] public static IFdbDynamicSubspace CreateDynamic(Slice slice, [NotNull] IDynamicKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbDynamicSubspace(slice, encoder); } @@ -143,74 +143,74 @@ public static IFdbDynamicSubspace CreateDynamic([NotNull] TKey key, IDynam /// Tuple that represents the prefix of the new subspace /// Optional type encoding used by this subspace. /// New subspace instance that will use the packed representation of as its prefix - [NotNull] + [Pure, NotNull] public static IFdbDynamicSubspace CreateDynamic([NotNull] IFdbTuple tuple, IFdbKeyEncoding encoding = null) { - if (tuple == null) throw new ArgumentNullException("tuple"); + Contract.NotNull(tuple, nameof(tuple)); var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new FdbDynamicSubspace(tuple.ToSlice(), true, encoder); + return new FdbDynamicSubspace(tuple.ToSlice(), true, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, IKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); return new FdbEncoderSubspace(slice, encoder); } - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(slice, encoder); } /// Clone this subspace /// New Subspace that uses the same prefix key /// Hint: Cloning a special Subspace like a or will not keep all the "special abilities" of the parent. - [NotNull] + [Pure, NotNull] public static IFdbSubspace Copy([NotNull] IFdbSubspace subspace) { var dyn = subspace as FdbDynamicSubspace; @@ -231,7 +231,7 @@ public static IFdbSubspace Copy([NotNull] IFdbSubspace subspace) /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbDynamicSubspace CopyDynamic([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); @@ -240,16 +240,16 @@ public static IFdbDynamicSubspace CopyDynamic([NotNull] IFdbSubspace subspace, I /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbDynamicSubspace CopyDynamic([NotNull] IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbDynamicSubspace(subspace.Key, true, encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); @@ -258,7 +258,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subsp /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) { if (encoder == null) throw new ArgumentNullException("encoder"); @@ -267,7 +267,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subsp /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); @@ -276,7 +276,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubs /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { if (encoder == null) throw new ArgumentNullException("encoder"); @@ -285,7 +285,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubs /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); @@ -294,7 +294,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { if (encoder == null) throw new ArgumentNullException("encoder"); @@ -303,7 +303,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); @@ -312,7 +312,7 @@ public static IFdbEncoderSubspace CopyEncoder([N /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System - [NotNull] + [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { if (encoder == null) throw new ArgumentNullException("encoder"); @@ -413,7 +413,7 @@ public Slice ConcatKey(TKey key) /// Array of slices (for all keys) that share the same underlying buffer public Slice[] ConcatKeys(IEnumerable keys) { - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(keys, nameof(keys)); //REVIEW: what to do with keys that are Slice.Nil ? return Slice.ConcatRange(GetKeyPrefix(), keys); } @@ -456,7 +456,7 @@ public Slice ExtractKey(Slice key, bool boundCheck = false) /// If is true and at least one key in is outside the current subspace. public Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) { - if (keys == null) throw new ArgumentNullException("keys"); + Contract.NotNull(keys, nameof(keys)); var prefix = GetKeyPrefix(); @@ -499,7 +499,7 @@ public Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) public SliceWriter GetWriter(int capacity = 0) { - if (capacity < 0) throw new ArgumentOutOfRangeException("capacity"); + if (capacity < 0) throw new ArgumentOutOfRangeException(nameof(capacity)); var prefix = GetKeyPrefix(); if (capacity > 0) @@ -586,11 +586,11 @@ protected void FailKeyOutOfBound(Slice key) { #if DEBUG // only in debug mode, because have the key and subspace in the exception message could leak sensitive information - string msg = String.Format("The key {0} does not belong to subspace {1}", FdbKey.Dump(key), this.ToString()); + string msg = $"The key {FdbKey.Dump(key)} does not belong to subspace {this}"; #else string msg = "The specifed key does not belong to this subspace"; #endif - throw new ArgumentException(msg, "key"); + throw new ArgumentException(msg, nameof(key)); } /// Return a user-friendly representation of a key from this subspace diff --git a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs index c6e2eb387..948b744a1 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Extensions methods to add FdbSubspace overrides to various types public static class FdbSubspaceExtensions @@ -43,10 +43,11 @@ public static class FdbSubspaceExtensions /// Instance of a generic subspace /// If non-null, uses this specific instance of the TypeSystem. If null, uses the default instance for this particular TypeSystem /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbDynamicSubspace Using([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); return FdbSubspace.CopyDynamic(subspace, encoding); } @@ -54,10 +55,11 @@ public static IFdbDynamicSubspace Using([NotNull] this IFdbSubspace subspace, [N /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbDynamicSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); return FdbSubspace.CopyDynamic(subspace, encoder); } @@ -65,10 +67,11 @@ public static IFdbDynamicSubspace UsingEncoder([NotNull] this IFdbSubspace subsp /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); return FdbSubspace.CopyEncoder(subspace, encoding); } @@ -76,10 +79,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); return FdbSubspace.CopyEncoder(subspace, encoder); } @@ -87,10 +91,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); return FdbSubspace.CopyEncoder(subspace, encoding); } @@ -98,10 +103,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IF /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); return FdbSubspace.CopyEncoder(subspace, encoder); } @@ -109,10 +115,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IF /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); return FdbSubspace.CopyEncoder(subspace, encoding); } @@ -120,10 +127,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); return FdbSubspace.CopyEncoder(subspace, encoder); } @@ -131,10 +139,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoding == null) throw new ArgumentNullException("encoding"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); return FdbSubspace.CopyEncoder(subspace, encoding); } @@ -142,10 +151,11 @@ public static IFdbEncoderSubspace UsingEncoder([ /// Instance of a generic subspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem + [Pure, NotNull] public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); return FdbSubspace.CopyEncoder(subspace, encoder); } @@ -161,14 +171,14 @@ public static void ClearRange(this IFdbTransaction trans, [NotNull] IFdbSubspace /// Clear the entire content of a subspace public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IFdbSubspace subspace, CancellationToken cancellationToken) { - if (db == null) throw new ArgumentNullException("db"); - if (subspace == null) throw new ArgumentNullException("subspace"); + Contract.NotNull(db, nameof(db)); + Contract.NotNull(subspace, nameof(subspace)); return db.WriteAsync((tr) => ClearRange(tr, subspace), cancellationToken); } /// Returns all the keys inside of a subspace - [NotNull] + [Pure, NotNull] public static FdbRangeQuery> GetRangeStartsWith(this IFdbReadOnlyTransaction trans, [NotNull] IFdbSubspace subspace, FdbRangeOptions options = null) { //REVIEW: should we remove this method? diff --git a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs index 1b9eea9c6..af1d657fd 100644 --- a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs @@ -26,11 +26,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using JetBrains.Annotations; [PublicAPI] public interface IFdbDynamicSubspace : IFdbSubspace @@ -47,4 +46,4 @@ public interface IFdbDynamicSubspace : IFdbSubspace } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs b/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs index 7b4f03a88..172af8455 100644 --- a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs @@ -26,11 +26,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using JetBrains.Annotations; [PublicAPI] public interface IFdbEncoderSubspace : IFdbSubspace @@ -92,7 +91,6 @@ public interface IFdbEncoderSubspace : IFdbSubspace IFdbEncoderSubspace Partial {[NotNull] get; } - //TODO: how to name ? } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Subspaces/IFdbSubspace.cs b/FoundationDB.Client/Subspaces/IFdbSubspace.cs index 9ab7f8097..05a0f3b2c 100644 --- a/FoundationDB.Client/Subspaces/IFdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbSubspace.cs @@ -38,35 +38,39 @@ public interface IFdbSubspace : IFdbKey // This interface helps solve some type resolution ambiguities at compile time between types that all implement IFdbKey but have different semantics for partitionning and concatenation /// Returns the prefix of this subspace - Slice Key { get; } + Slice Key { [Pure] get; } /// Return a key range that contains all the keys in this subspace, including the prefix itself /// Return the range: Key <= x <= Increment(Key) + [Pure] FdbKeyRange ToRange(); /// Return a key range that contains all the keys under a suffix in this subspace /// Binary suffix that will be appended to the current prefix, before computing the range /// Return the range: (this.Key + suffix) <= x <= Increment(this.Key + suffix) + [Pure] FdbKeyRange ToRange(Slice suffix); /// Return a key range that contains all the keys under a serializable key in this subspace /// Return the range: (this.Key + key.ToFoundationDbKey()) <= x <= Increment(this.Key + key.ToFoundationDbKey()) + [Pure] FdbKeyRange ToRange([NotNull] TKey key) where TKey : IFdbKey; /// Create a new subspace by adding a suffix to the key of the current subspace. /// Binary suffix that will be appended to the current prefix /// New subspace whose prefix is the concatenation of the parent prefix, and - IFdbSubspace this[Slice suffix] { [NotNull] get; } + IFdbSubspace this[Slice suffix] { [Pure, NotNull] get; } /// Create a new subspace by adding a suffix to the key of the current subspace. /// Item that can serialize itself into a binary suffix, that will be appended to the current subspace's prefix /// New subspace whose prefix is the concatenation of the parent prefix, and - IFdbSubspace this[[NotNull] IFdbKey key] { [NotNull] get; } + IFdbSubspace this[[NotNull] IFdbKey key] { [Pure, NotNull] get; } /// Test if a key is inside the range of keys logically contained by this subspace /// Key to test /// True if the key can exist inside the current subspace. /// Please note that this method does not test if the key *actually* exists in the database, only if the key is not ouside the range of keys defined by the subspace. + [Pure] bool Contains(Slice key); /// Check that a key fits inside this subspace, and return '' or '\xFF' if it is outside the bounds @@ -78,23 +82,25 @@ public interface IFdbSubspace : IFdbKey /// Return the key that is composed of the subspace's prefix and a binary suffix /// Binary suffix that will be appended to the current prefix /// Full binary key + [Pure] Slice ConcatKey(Slice suffix); /// Return the key that is composed of the subspace's prefix and a serializable key /// Item that can serialize itself into a binary suffix, that will be appended to the current prefix /// Full binary key + [Pure] Slice ConcatKey([NotNull] TKey key) where TKey : IFdbKey; /// Concatenate a batch of keys under this subspace /// List of suffixes to process /// Array of which is equivalent to calling on each entry in - [NotNull] + [Pure, NotNull] Slice[] ConcatKeys([NotNull] IEnumerable suffixes); /// Concatenate a batch of serializable keys under this subspace /// List of serializable keys to process /// Array of which is equivalent to calling on each entry in - [NotNull] + [Pure, NotNull] Slice[] ConcatKeys([NotNull, ItemNotNull] IEnumerable keys) where TKey : IFdbKey; /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace @@ -103,6 +109,7 @@ public interface IFdbSubspace : IFdbKey /// Binary suffix of the key (or Slice.Empty if the key is exactly equal to the subspace prefix). If the key is outside of the subspace, returns Slice.Nil /// This is the inverse operation of /// If is true and is outside the current subspace. + [Pure] Slice ExtractKey(Slice key, bool boundCheck = false); /// Remove the subspace prefix from a batch of binary keys, and only return the tail, or Slice.Nil if a key does not fit inside the namespace @@ -110,15 +117,15 @@ public interface IFdbSubspace : IFdbKey /// If true, verify that each key in is inside the bounds of the subspace /// Array of only the binary suffix of the keys, Slice.Empty for a key that is exactly equal to the subspace prefix, or Slice.Nil for a key that is outside of the subspace /// If is true and at least one key in is outside the current subspace. - [NotNull] + [Pure, NotNull] Slice[] ExtractKeys([NotNull] IEnumerable keys, bool boundCheck = false); /// Return a new slice buffer, initialized with the subspace prefix, that can be used for custom key serialization /// If non-zero, the expected buffer capacity. The size of the subspace prefix will be added to this value. /// Instance of a SliceWriter with the prefix of this subspace already copied. + [Pure] SliceWriter GetWriter(int capacity = 0); - } } diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index 4602f0677..ae20aa552 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Linq; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// Helper class for all key/value encoders [PublicAPI] diff --git a/FoundationDB.Client/Utils/CodeAnnotations.cs b/FoundationDB.Client/Utils/CodeAnnotations.cs index acbefd190..ad6581a39 100644 --- a/FoundationDB.Client/Utils/CodeAnnotations.cs +++ b/FoundationDB.Client/Utils/CodeAnnotations.cs @@ -15,37 +15,42 @@ namespace JetBrains.Annotations /// /// Indicates that the value of the marked element could be null sometimes, - /// so the check for null is necessary before its usage + /// so the check for null is necessary before its usage. /// /// - /// [CanBeNull] public object Test() { return null; } - /// public void UseTest() { + /// [CanBeNull] object Test() => null; + /// + /// void UseTest() { /// var p = Test(); /// var s = p.ToString(); // Warning: Possible 'System.NullReferenceException' /// } /// [AttributeUsage( AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | - AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event)] + AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event | + AttributeTargets.Class | AttributeTargets.Interface | AttributeTargets.GenericParameter)] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class CanBeNullAttribute : Attribute { } /// - /// Indicates that the value of the marked element could never be null + /// Indicates that the value of the marked element could never be null. /// /// - /// [NotNull] public object Foo() { + /// [NotNull] object Foo() { /// return null; // Warning: Possible 'null' assignment /// } /// [AttributeUsage( AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | - AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event)] + AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event | + AttributeTargets.Class | AttributeTargets.Interface | AttributeTargets.GenericParameter)] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class NotNullAttribute : Attribute { } /// - /// Indicates that collection or enumerable value does not contain null elements + /// Can be appplied to symbols of types derived from IEnumerable as well as to symbols of Task + /// and Lazy classes to indicate that the value of a collection item, of the Task.Result property + /// or of the Lazy.Value property can never be null. /// [AttributeUsage( AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | @@ -54,7 +59,9 @@ internal sealed class NotNullAttribute : Attribute { } internal sealed class ItemNotNullAttribute : Attribute { } /// - /// Indicates that collection or enumerable value can contain null elements + /// Can be appplied to symbols of types derived from IEnumerable as well as to symbols of Task + /// and Lazy classes to indicate that the value of a collection item, of the Task.Result property + /// or of the Lazy.Value property can be null. /// [AttributeUsage( AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | @@ -62,20 +69,30 @@ internal sealed class ItemNotNullAttribute : Attribute { } [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class ItemCanBeNullAttribute : Attribute { } + /// + /// Implicitly apply [NotNull]/[ItemNotNull] annotation to all the of type members and parameters + /// in particular scope where this annotation is used (type declaration or whole assembly). + /// + [AttributeUsage( + AttributeTargets.Class | AttributeTargets.Struct | AttributeTargets.Interface | AttributeTargets.Assembly)] + internal sealed class ImplicitNotNullAttribute : Attribute { } + /// /// Indicates that the marked method builds string by format pattern and (optional) arguments. /// Parameter, which contains format string, should be given in constructor. The format string - /// should be in -like form + /// should be in -like form. /// /// /// [StringFormatMethod("message")] - /// public void ShowError(string message, params object[] args) { /* do something */ } - /// public void Foo() { + /// void ShowError(string message, params object[] args) { /* do something */ } + /// + /// void Foo() { /// ShowError("Failed: {0}"); // Warning: Non-existing argument in format string /// } /// [AttributeUsage( - AttributeTargets.Constructor | AttributeTargets.Method | AttributeTargets.Delegate)] + AttributeTargets.Constructor | AttributeTargets.Method | + AttributeTargets.Property | AttributeTargets.Delegate)] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class StringFormatMethodAttribute : Attribute { @@ -87,16 +104,33 @@ public StringFormatMethodAttribute(string formatParameterName) FormatParameterName = formatParameterName; } - public string FormatParameterName { get; private set; } + public string FormatParameterName { get; } + } + + /// + /// For a parameter that is expected to be one of the limited set of values. + /// Specify fields of which type should be used as values for this parameter. + /// + [AttributeUsage(AttributeTargets.Parameter | AttributeTargets.Property | AttributeTargets.Field)] + [Conditional("JETBRAINS_ANNOTATIONS")] + internal sealed class ValueProviderAttribute : Attribute + { + public ValueProviderAttribute(string name) + { + Name = name; + } + + [NotNull] + public string Name { get; } } /// /// Indicates that the function argument should be string literal and match one /// of the parameters of the caller function. For example, ReSharper annotates - /// the parameter of + /// the parameter of . /// /// - /// public void Foo(string param) { + /// void Foo(string param) { /// if (param == null) /// throw new ArgumentNullException("par"); // Warning: Cannot resolve symbol /// } @@ -106,7 +140,7 @@ public StringFormatMethodAttribute(string formatParameterName) internal sealed class InvokerParameterNameAttribute : Attribute { } /// - /// Describes dependency between method input and output + /// Describes dependency between method input and output. /// /// ///

Function Definition Table syntax:

@@ -153,7 +187,7 @@ internal sealed class InvokerParameterNameAttribute : Attribute { } internal sealed class ContractAnnotationAttribute : Attribute { public ContractAnnotationAttribute([NotNull] string contract) - : this(contract, false) { } + : this(contract, false) { } public ContractAnnotationAttribute([NotNull] string contract, bool forceFullStates) { @@ -161,8 +195,8 @@ public ContractAnnotationAttribute([NotNull] string contract, bool forceFullStat ForceFullStates = forceFullStates; } - public string Contract { get; private set; } - public bool ForceFullStates { get; private set; } + public string Contract { get; } + public bool ForceFullStates { get; } } /// @@ -174,8 +208,9 @@ public ContractAnnotationAttribute([NotNull] string contract, bool forceFullStat /// /// [CannotApplyEqualityOperator] /// class NoEquality { } + /// /// class UsesNoEquality { - /// public void Test() { + /// void Test() { /// var ca1 = new NoEquality(); /// var ca2 = new NoEquality(); /// if (ca1 != null) { // OK @@ -184,8 +219,7 @@ public ContractAnnotationAttribute([NotNull] string contract, bool forceFullStat /// } /// } /// - [AttributeUsage( - AttributeTargets.Interface | AttributeTargets.Class | AttributeTargets.Struct)] + [AttributeUsage(AttributeTargets.Interface | AttributeTargets.Class | AttributeTargets.Struct)] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class CannotApplyEqualityOperatorAttribute : Attribute { } @@ -195,11 +229,12 @@ internal sealed class CannotApplyEqualityOperatorAttribute : Attribute { } /// /// /// [BaseTypeRequired(typeof(IComponent)] // Specify requirement - /// public class ComponentAttribute : Attribute { } + /// class ComponentAttribute : Attribute { } + /// /// [Component] // ComponentAttribute requires implementing IComponent interface - /// public class MyComponent : IComponent { } + /// class MyComponent : IComponent { } /// - [AttributeUsage(AttributeTargets.Class, AllowMultiple = true, Inherited = true)] + [AttributeUsage(AttributeTargets.Class, AllowMultiple = true)] [BaseTypeRequired(typeof(Attribute))] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class BaseTypeRequiredAttribute : Attribute @@ -210,13 +245,12 @@ public BaseTypeRequiredAttribute([NotNull] Type baseType) } [NotNull] - public Type BaseType { get; private set; } + public Type BaseType { get; set; } } /// - /// Indicates that the marked symbol is used implicitly - /// (e.g. via reflection, in external library), so this symbol - /// will not be marked as unused (as well as by other usage inspections) + /// Indicates that the marked symbol is used implicitly (e.g. via reflection, in external library), + /// so this symbol will not be marked as unused (as well as by other usage inspections). /// [AttributeUsage(AttributeTargets.All)] [Conditional("JETBRAINS_ANNOTATIONS")] @@ -231,21 +265,19 @@ public UsedImplicitlyAttribute(ImplicitUseKindFlags useKindFlags) public UsedImplicitlyAttribute(ImplicitUseTargetFlags targetFlags) : this(ImplicitUseKindFlags.Default, targetFlags) { } - public UsedImplicitlyAttribute( - ImplicitUseKindFlags useKindFlags, ImplicitUseTargetFlags targetFlags) + public UsedImplicitlyAttribute(ImplicitUseKindFlags useKindFlags, ImplicitUseTargetFlags targetFlags) { UseKindFlags = useKindFlags; TargetFlags = targetFlags; } - public ImplicitUseKindFlags UseKindFlags { get; private set; } - public ImplicitUseTargetFlags TargetFlags { get; private set; } + public ImplicitUseKindFlags UseKindFlags { get; } + public ImplicitUseTargetFlags TargetFlags { get; } } /// - /// Should be used on attributes and causes ReSharper - /// to not mark symbols marked with such attributes as unused - /// (as well as by other usage inspections) + /// Should be used on attributes and causes ReSharper to not mark symbols marked with such attributes + /// as unused (as well as by other usage inspections) /// [AttributeUsage(AttributeTargets.Class | AttributeTargets.GenericParameter)] [Conditional("JETBRAINS_ANNOTATIONS")] @@ -260,8 +292,7 @@ public MeansImplicitUseAttribute(ImplicitUseKindFlags useKindFlags) public MeansImplicitUseAttribute(ImplicitUseTargetFlags targetFlags) : this(ImplicitUseKindFlags.Default, targetFlags) { } - public MeansImplicitUseAttribute( - ImplicitUseKindFlags useKindFlags, ImplicitUseTargetFlags targetFlags) + public MeansImplicitUseAttribute(ImplicitUseKindFlags useKindFlags, ImplicitUseTargetFlags targetFlags) { UseKindFlags = useKindFlags; TargetFlags = targetFlags; @@ -277,40 +308,39 @@ public MeansImplicitUseAttribute( internal enum ImplicitUseKindFlags { Default = Access | Assign | InstantiatedWithFixedConstructorSignature, - /// Only entity marked with attribute considered used + /// Only entity marked with attribute considered used. Access = 1, - /// Indicates implicit assignment to a member + /// Indicates implicit assignment to a member. Assign = 2, /// /// Indicates implicit instantiation of a type with fixed constructor signature. /// That means any unused constructor parameters won't be reported as such. /// InstantiatedWithFixedConstructorSignature = 4, - /// Indicates implicit instantiation of a type + /// Indicates implicit instantiation of a type. InstantiatedNoFixedConstructorSignature = 8, } /// - /// Specify what is considered used implicitly - /// when marked with - /// or + /// Specify what is considered used implicitly when marked + /// with or . /// [Flags] internal enum ImplicitUseTargetFlags { Default = Itself, Itself = 1, - /// Members of entity marked with attribute are considered used + /// Members of entity marked with attribute are considered used. Members = 2, - /// Entity marked with attribute and all its members considered used + /// Entity marked with attribute and all its members considered used. WithMembers = Itself | Members } /// /// This attribute is intended to mark publicly available API - /// which should not be removed and so is treated as used + /// which should not be removed and so is treated as used. /// - [MeansImplicitUse] + [MeansImplicitUse(ImplicitUseTargetFlags.WithMembers)] [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class PublicAPIAttribute : Attribute { @@ -320,15 +350,13 @@ public PublicAPIAttribute([NotNull] string comment) Comment = comment; } - public string Comment { get; private set; } + public string Comment { get; } } /// - /// Tells code analysis engine if the parameter is completely handled - /// when the invoked method is on stack. If the parameter is a delegate, - /// indicates that delegate is executed while the method is executed. - /// If the parameter is an enumerable, indicates that it is enumerated - /// while the method is executed + /// Tells code analysis engine if the parameter is completely handled when the invoked method is on stack. + /// If the parameter is a delegate, indicates that delegate is executed while the method is executed. + /// If the parameter is an enumerable, indicates that it is enumerated while the method is executed. /// [AttributeUsage(AttributeTargets.Parameter)] [Conditional("JETBRAINS_ANNOTATIONS")] @@ -336,13 +364,13 @@ internal sealed class InstantHandleAttribute : Attribute { } /// /// Indicates that a method does not make any observable state changes. - /// The same as System.Diagnostics.Contracts.PureAttribute + /// The same as System.Diagnostics.Contracts.PureAttribute. /// /// - /// [Pure] private int Multiply(int x, int y) { return x * y; } - /// public void Foo() { - /// const int a = 2, b = 2; - /// Multiply(a, b); // Waring: Return value of pure method is not used + /// [Pure] int Multiply(int x, int y) => x * y; + /// + /// void M() { + /// Multiply(123, 42); // Waring: Return value of pure method is not used /// } /// [AttributeUsage(AttributeTargets.Method)] @@ -350,10 +378,48 @@ internal sealed class InstantHandleAttribute : Attribute { } internal sealed class PureAttribute : Attribute { } /// - /// Indicates how method invocation affects content of the collection + /// Indicates that the return value of method invocation must be used. /// [AttributeUsage(AttributeTargets.Method)] [Conditional("JETBRAINS_ANNOTATIONS")] + internal sealed class MustUseReturnValueAttribute : Attribute + { + public MustUseReturnValueAttribute() { } + public MustUseReturnValueAttribute([NotNull] string justification) + { + Justification = justification; + } + + public string Justification { get; } + } + + /// + /// Indicates the type member or parameter of some type, that should be used instead of all other ways + /// to get the value that type. This annotation is useful when you have some "context" value evaluated + /// and stored somewhere, meaning that all other ways to get this value must be consolidated with existing one. + /// + /// + /// class Foo { + /// [ProvidesContext] IBarService _barService = ...; + /// + /// void ProcessNode(INode node) { + /// DoSomething(node, node.GetGlobalServices().Bar); + /// // ^ Warning: use value of '_barService' field + /// } + /// } + /// + [AttributeUsage( + AttributeTargets.Field | AttributeTargets.Property | AttributeTargets.Parameter | AttributeTargets.Method | + AttributeTargets.Class | AttributeTargets.Interface | AttributeTargets.Struct | AttributeTargets.GenericParameter)] + [Conditional("JETBRAINS_ANNOTATIONS")] + internal sealed class ProvidesContextAttribute : Attribute { } + + /// + /// Indicates how method, constructor invocation or property access + /// over collection type affects content of the collection. + /// + [AttributeUsage(AttributeTargets.Method | AttributeTargets.Constructor | AttributeTargets.Property)] + [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class CollectionAccessAttribute : Attribute { public CollectionAccessAttribute(CollectionAccessType collectionAccessType) @@ -361,26 +427,26 @@ public CollectionAccessAttribute(CollectionAccessType collectionAccessType) CollectionAccessType = collectionAccessType; } - public CollectionAccessType CollectionAccessType { get; private set; } + public CollectionAccessType CollectionAccessType { get; } } [Flags] internal enum CollectionAccessType { - /// Method does not use or modify content of the collection + /// Method does not use or modify content of the collection. None = 0, - /// Method only reads content of the collection but does not modify it + /// Method only reads content of the collection but does not modify it. Read = 1, - /// Method can change content of the collection but does not add new elements + /// Method can change content of the collection but does not add new elements. ModifyExistingContent = 2, - /// Method can add new elements to the collection + /// Method can add new elements to the collection. UpdatedContent = ModifyExistingContent | 4 } /// /// Indicates that the marked method is assertion method, i.e. it halts control flow if /// one of the conditions is satisfied. To set the condition, mark one of the parameters with - /// attribute + /// attribute. /// [AttributeUsage(AttributeTargets.Method)] [Conditional("JETBRAINS_ANNOTATIONS")] @@ -400,22 +466,22 @@ public AssertionConditionAttribute(AssertionConditionType conditionType) ConditionType = conditionType; } - public AssertionConditionType ConditionType { get; private set; } + public AssertionConditionType ConditionType { get; } } /// /// Specifies assertion type. If the assertion method argument satisfies the condition, - /// then the execution continues. Otherwise, execution is assumed to be halted + /// then the execution continues. Otherwise, execution is assumed to be halted. /// internal enum AssertionConditionType { - /// Marked parameter should be evaluated to true + /// Marked parameter should be evaluated to true. IS_TRUE = 0, - /// Marked parameter should be evaluated to false + /// Marked parameter should be evaluated to false. IS_FALSE = 1, - /// Marked parameter should be evaluated to null value + /// Marked parameter should be evaluated to null value. IS_NULL = 2, - /// Marked parameter should be evaluated to not null value + /// Marked parameter should be evaluated to not null value. IS_NOT_NULL = 3, } @@ -452,4 +518,4 @@ internal sealed class RegexPatternAttribute : Attribute { } [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class NoReorder : Attribute { } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Utils/Contract.cs b/FoundationDB.Client/Utils/Contract.cs index ff171d5a0..f07b1dfdd 100644 --- a/FoundationDB.Client/Utils/Contract.cs +++ b/FoundationDB.Client/Utils/Contract.cs @@ -1,169 +1,1395 @@ -#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. - */ +#region Copyright (c) 2013-2018, Doxense SAS. All rights reserved. +// See License.MD for license information #endregion -namespace FoundationDB.Client.Utils +namespace Doxense.Diagnostics.Contracts { using JetBrains.Annotations; using System; + using System.Collections.Generic; using System.Diagnostics; + using System.Globalization; + using System.Reflection; using System.Runtime.CompilerServices; + using System.Runtime.ConstrainedExecution; using SDC = System.Diagnostics.Contracts; + using SRC = System.Runtime.CompilerServices; - internal static class Contract + internal static class ContractMessages { + public const string PreconditionWasNotMet = "A pre-condition was not met"; + public const string ValueCannotBeNull = "Value cannot be null."; + public const string StringCannotBeEmpty = "String cannot be empty."; + public const string StringCannotBeWhiteSpaces = "String cannot contain only whitespaces."; + public const string CollectionCannotBeEmpty = "Collection cannot be empty."; + public const string BufferCannotBeNull = "Buffer cannot be null."; + public const string BufferCannotBeEmpty = "Buffer cannot be empty."; + public const string PositiveNumberRequired = "Positive number required."; + public const string PowerOfTwoRequired = "Power of two number required."; + public const string AboveZeroNumberRequired = "Non-Zero Positive number required."; + public const string ValueIsTooSmall = "The specified value is too small."; + public const string ValueIsTooBig = "The specified value is too big."; + public const string ValueIsForbidden = "The specified value is not allowed."; + public const string ValueIsExpected = "The specified value is not the expected value."; + public const string ValueMustBeBetween = "The specified value was outside the specified range."; + public const string ValueMustBeMultiple = "The specified value must be a multiple of another value."; + public const string NonNegativeNumberRequired = "Non-negative number required."; + public const string OffsetMustBeWithinBuffer = "Offset and length were out of bounds for the buffer or count is greater than the number of elements from index to the end of the buffer."; - #region Requires + public const string ConditionNotNull = "{0} != null"; + public const string ConditionNotEmptyLength = "{0}.Length > 0"; + public const string ConditionNotWhiteSpace = "{0}.All(c => !char.IsWhiteSpace(c))"; + public const string ConditionNotEmptyCount = "{0}.Count > 0"; + public const string ConditionArgPositive = "{0} >= 0"; + public const string ConditionArgNotEqualTo = "{0} != x"; + public const string ConditionArgEqualTo = "{0} == x"; + public const string ConditionArgGreaterThan = "{0} > x"; + public const string ConditionArgGreaterThanZero = "{0} > 0"; + public const string ConditionArgGreaterOrEqual = "{0} >= x"; + public const string ConditionArgGreaterOrEqualZero = "{0} >= 0"; + public const string ConditionArgMultiple = "{0} % x == 0"; + public const string ConditionArgLessThan = "{0} < x"; + public const string ConditionArgLessThanOrEqual = "{0} <= x"; + public const string ConditionArgBetween = "min <= {0} <= max"; + public const string ConditionArgBufferOverflow = "(buffer.Length - offset) < count"; + } + + /// Classe helper pour la vérification de pré-requis, invariants, assertions, ... + [DebuggerNonUserCode] + public static class Contract + { + + public static bool IsUnitTesting { get; set; } + + private static readonly ConstructorInfo s_constructorNUnitException; + + static Contract() + { + // détermine si on est lancé depuis des tests unitaires (pour désactiver les breakpoints et autres opérations intrusivent qui vont parasiter les tests) + + var nUnitAssert = Type.GetType("NUnit.Framework.AssertionException,nunit.framework"); + if (nUnitAssert != null) + { + // on convertit les échecs "soft" en échec d'assertion NUnit + s_constructorNUnitException = nUnitAssert.GetConstructor(new [] { typeof (string) }); + IsUnitTesting = true; + } + } + + private static Exception MapToNUnitAssertion(string message) + { + return (Exception) s_constructorNUnitException?.Invoke(new object[] { message }); // => new NUnit.Framework.AssertionException(...) + } + + #region DEBUG checks... + + /// [DEBUG ONLY] Dummy method (no-op) + [Conditional("CONTRACTS_FULL")] + public static void EndContractBlock() + { + // cette méthode ne fait rien, et sert juste à émuler la Contract API + } + + /// [DEBUG ONLY] Vérifie qu'une pré-condition est vrai, lors de l'entrée dans une méthode + /// Condition qui ne doit jamais être fausse + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + [DebuggerStepThrough] + public static void Requires([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Precondition, null); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une pré-condition est vrai, lors de l'entrée dans une méthode + /// Condition qui ne doit jamais être fausse + /// Message décrivant l'erreur (optionnel) + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Requires([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string userMessage) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Precondition, userMessage); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, dans le body dans une méthode + /// Condition qui ne doit jamais être fausse + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Assert, null); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, dans le body dans une méthode + /// Condition qui ne doit jamais être fausse + /// Message décrivant l'erreur (optionnel) + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string userMessage) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Assert, userMessage); +#endif + } + +#if DEPRECATED + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, dans le body dans une méthode + /// Valeur observée + /// Valeur attendue + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + [Obsolete("Use Contract.Assert(actual == expected) instead")] + public static void Expect(T actual, T expected) + { + if (!EqualityComparer.Default.Equals(actual, expected)) RaiseContractFailure(SDC.ContractFailureKind.Assert, String.Format(CultureInfo.InvariantCulture, "Expected value {0} but was {1}", expected, actual)); + } +#endif + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, lors de la sortie d'une méthode + /// Condition qui ne doit jamais être fausse + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Ensures([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Postcondition, null); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, lors de la sortie d'une méthode + /// Condition qui ne doit jamais être fausse + /// Message décrivant l'erreur (optionnel) + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Ensures([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string userMessage) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Postcondition, userMessage); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai pendant toute la vie d'une instance + /// Condition qui ne doit jamais être fausse + /// Message décrivant l'erreur (optionnel) + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Invariant([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string userMessage = null) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Invariant, userMessage); +#endif + } + + #endregion + + #region RUNTIME checks... + + #region Contract.NotNull + + /// [RUNTIME] The specified instance must not be null (assert: value != null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] TValue value, + [InvokerParameterName] string paramName) + where TValue : class + { + if (value == null) throw FailArgumentNull(paramName, null); + } + + /// [RUNTIME] The specified instance must not be null (assert: value != null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] string value, + [InvokerParameterName] string paramName) + { + if (value == null) throw FailArgumentNull(paramName, null); + } + + /// [RUNTIME] The specified instance must not be null (assert: value != null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] TValue value, + [InvokerParameterName] string paramName, + string message) + where TValue : class + { + if (value == null) throw FailArgumentNull(paramName, message); + } + + /// [RUNTIME] The specified instance must not be null (assert: value != null) + /// This methods allow structs (that can never be null) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullAllowStructs( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] TValue value, + [InvokerParameterName] string paramName) + { + if (value == null) throw FailArgumentNull(paramName, null); + } + + /// [RUNTIME] The specified instance must not be null (assert: value != null) + /// This methods allow structs (that can never be null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullAllowStructs( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] TValue value, + [InvokerParameterName] string paramName, + string message) + { + if (value == null) throw FailArgumentNull(paramName, message); + } + + /// [RUNTIME] The specified pointer must not be null (assert: pointer != null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static unsafe void PointerNotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] void* pointer, + [InvokerParameterName] string paramName) + { + if (pointer == null) throw FailArgumentNull(paramName, null); + } + + /// [RUNTIME] The specified pointer must not be null (assert: pointer != null) + /// if is null + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static unsafe void PointerNotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] void* pointer, + [InvokerParameterName] string paramName, + string message) + { + if (pointer == null) throw FailArgumentNull(paramName, message); + } + + /// [RUNTIME] The specified value cannot be null (assert: value != null) + /// Passed value, or throws an exception if it was null + /// if is null + /// This method is intended for use in single-line property setters + /// + /// public string FooThatIsNeverNull + /// { + /// get => return m_foo; + /// set => m_foo = Contract.ValueNotNull(value); + /// } + /// + [Pure, NotNull, AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static T ValueNotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] T value + ) + where T : class + { + return value ?? throw FailArgumentNull(nameof(value), null); + } + + /// [RUNTIME] The specified value cannot be null (assert: value != null) + /// Passed value, or throws an exception if it was null + /// if is null + /// This method is intended for use in single-line property setters + /// + /// private string m_fooThatIsNeverNull; + /// public string Foo + /// { + /// get => return m_fooThatIsNeverNull; + /// set => m_fooThatIsNeverNull = Contract.ValueNotNull(value, "Foo cannot be set to null"); + /// } + /// + [Pure, NotNull, AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static T ValueNotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] T value, + string message + ) + where T : class + { + return value ?? throw FailArgumentNull(nameof(value), message); + } + + #endregion + + #region Contract.NotNullOrEmpty + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailStringNullOrEmpty(string value, string paramName, string message = null) + { + if (value == null) + return ReportFailure(typeof(ArgumentNullException), ContractMessages.ValueCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + else + return ReportFailure(typeof(ArgumentException), ContractMessages.StringCannotBeEmpty, message, paramName, ContractMessages.ConditionNotEmptyLength); + } + + /// [RUNTIME] The specified string must not be null or empty (assert: value != null && value.Length != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName + ) + { + if (string.IsNullOrEmpty(value)) throw FailStringNullOrEmpty(value, paramName, null); + } + + /// [RUNTIME] The specified string must not be null or empty (assert: value != null && value.Length != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName, + string message) + { + if (string.IsNullOrEmpty(value)) throw FailStringNullOrEmpty(value, paramName, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailStringNullOrWhiteSpace(string value, string paramName, string message = null) + { + if (value == null) + return ReportFailure(typeof(ArgumentNullException), ContractMessages.ValueCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + else if (value.Length == 0) + return ReportFailure(typeof(ArgumentException), ContractMessages.StringCannotBeEmpty, message, paramName, ContractMessages.ConditionNotEmptyLength); + else + return ReportFailure(typeof(ArgumentException), ContractMessages.StringCannotBeWhiteSpaces, message, paramName, ContractMessages.ConditionNotWhiteSpace); + } + + /// [RUNTIME] The specified string must not be null, empty or contain only whitespaces (assert: value != null && value.Length != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrWhiteSpace( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName) + { + if (string.IsNullOrWhiteSpace(value)) throw FailStringNullOrWhiteSpace(value, paramName, null); + } + + /// [RUNTIME] The specified string must not be null, empty or contain only whitespaces (assert: value != null && value.Length != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrWhiteSpace( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName, + string message) + { + if (string.IsNullOrWhiteSpace(value)) throw FailStringNullOrWhiteSpace(value, paramName, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArrayNullOrEmpty(object collection, string paramName, string message = null) + { + if (collection == null) + return ReportFailure(typeof(ArgumentNullException), ContractMessages.ValueCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + else + return ReportFailure(typeof(ArgumentException), ContractMessages.CollectionCannotBeEmpty, message, paramName, ContractMessages.ConditionNotEmptyCount); + } + + /// [RUNTIME] The specified array must not be null or emtpy (assert: value != null && value.Count != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] T[] value, + [InvokerParameterName] string paramName) + { + if (value == null || value.Length == 0) throw FailArrayNullOrEmpty(value, paramName, null); + } + + /// [RUNTIME] The specified array must not be null or emtpy (assert: value != null && value.Count != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] T[] value, + [InvokerParameterName] string paramName, + string message) + { + if (value == null || value.Length == 0) throw FailArrayNullOrEmpty(value, paramName, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailCollectionNullOrEmpty(object collection, string paramName, string message = null) + { + if (collection == null) + return ReportFailure(typeof(ArgumentNullException), ContractMessages.ValueCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + else + return ReportFailure(typeof(ArgumentException), ContractMessages.CollectionCannotBeEmpty, message, paramName, ContractMessages.ConditionNotEmptyCount); + } + + /// [RUNTIME] The specified collection must not be null or emtpy (assert: value != null && value.Count != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] ICollection value, + [InvokerParameterName] string paramName) + { + if (value == null || value.Count == 0) throw FailCollectionNullOrEmpty(value, paramName, null); + } + + /// [RUNTIME] The specified collection must not be null or emtpy (assert: value != null && value.Count != 0) + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] ICollection value, + [InvokerParameterName] string paramName, + string message) + { + if (value == null || value.Count == 0) throw FailCollectionNullOrEmpty(value, paramName, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailBufferNull(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentNullException), ContractMessages.BufferCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailBufferNullOrEmpty(object array, string paramName, string message = null) + { + if (array == null) + return ReportFailure(typeof(ArgumentNullException), ContractMessages.BufferCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + else + return ReportFailure(typeof(ArgumentException), ContractMessages.BufferCannotBeEmpty, message, paramName, ContractMessages.ConditionNotEmptyCount); + } + + /// [RUNTIME] The specified buffer must not be null or empty (assert: buffer.Array != null && buffer.Count != 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + ArraySegment buffer, + [InvokerParameterName] string paramName) + { + if (buffer.Array == null | buffer.Count == 0) throw FailBufferNullOrEmpty(buffer.Array, paramName, null); + } + + /// [RUNTIME] The specified buffer must not be null or empty (assert: buffer.Array != null && buffer.Count != 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotNullOrEmpty( + ArraySegment buffer, + [InvokerParameterName] string paramName, + string message) + { + if (buffer.Array == null | buffer.Count == 0) throw FailBufferNullOrEmpty(buffer.Array, paramName, message); + } + + #endregion + + #region Contract.Positive, LessThan[OrEqual], GreaterThen[OrEqual], EqualTo, Between, ... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotPositive(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.PositiveNumberRequired, message, paramName, ContractMessages.ConditionArgPositive); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotNonNegative(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.NonNegativeNumberRequired, message, paramName, ContractMessages.ConditionArgPositive); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotPowerOfTwo(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.PowerOfTwoRequired, message, paramName, ContractMessages.ConditionArgPositive); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentForbidden(string paramName, T forbidden, string message = null) + { + //TODO: need support for two format arguments for conditionTxt ! + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueIsForbidden, message, paramName, ContractMessages.ConditionArgNotEqualTo); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentExpected(string paramName, T expected, string message = null) + { + //TODO: need support for two format arguments for conditionTxt ! + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueIsExpected, message, paramName, ContractMessages.ConditionArgEqualTo); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotGreaterThan(string paramName, bool zero, string message = null) + { + return ReportFailure(typeof(ArgumentException), zero ? ContractMessages.AboveZeroNumberRequired : ContractMessages.ValueIsTooSmall, message, paramName, zero ? ContractMessages.ConditionArgGreaterThanZero : ContractMessages.ConditionArgGreaterThan); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotGreaterOrEqual(string paramName, bool zero, string message = null) + { + return ReportFailure(typeof(ArgumentException), zero ? ContractMessages.PositiveNumberRequired : ContractMessages.ValueIsTooSmall, message, paramName, zero ? ContractMessages.ConditionArgGreaterOrEqualZero : ContractMessages.ConditionArgGreaterOrEqual); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotLessThan(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueIsTooBig, message, paramName, ContractMessages.ConditionArgLessThan); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotLessOrEqual(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueIsTooBig, message, paramName, ContractMessages.ConditionArgLessThanOrEqual); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentOutOfBounds(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueMustBeBetween, message, paramName, ContractMessages.ConditionArgBetween); + } + + /// [RUNTIME] The specified value must not be a negative number (assert: value >= 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Positive(int value, [InvokerParameterName] string paramName) + { + if (value < 0) + { + throw FailArgumentNotPositive(paramName, null); + } + } + + /// [RUNTIME] The specified value must not be a negative number (assert: value >= 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Positive(int value, [InvokerParameterName] string paramName, string message) + { + if (value < 0) + { + throw FailArgumentNotPositive(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be a negative number (assert: value >= 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Positive(long value, [InvokerParameterName] string paramName) + { + if (value < 0) + { + throw FailArgumentNotPositive(paramName, null); + } + } + + /// [RUNTIME] The specified value must not be a negative number (assert: value >= 0) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Positive(long value, [InvokerParameterName] string paramName, string message) + { + if (value < 0) + { + throw FailArgumentNotPositive(paramName, message); + } + } + + /// [RUNTIME] The specified value must be a power of two (assert: NextPowerOfTwo(value) == value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void PowerOfTwo(int value, [InvokerParameterName] string paramName, string message = null) + { + if (value < 0 || unchecked((value & (value - 1)) != 0)) + { + throw FailArgumentNotPowerOfTwo(paramName, message); + } + } + + /// [RUNTIME] The specified value must be a power of two (assert: NextPowerOfTwo(value) == value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void PowerOfTwo(uint value, [InvokerParameterName] string paramName, string message = null) + { + if (unchecked((value & (value - 1)) != 0)) + { + throw FailArgumentNotPowerOfTwo(paramName, message); + } + } + + /// [RUNTIME] The specified value must be a power of two (assert: NextPowerOfTwo(value) == value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void PowerOfTwo(long value, [InvokerParameterName] string paramName, string message = null) + { + if (value < 0 || unchecked((value & (value - 1)) != 0)) + { + throw FailArgumentNotPowerOfTwo(paramName, message); + } + } + + /// [RUNTIME] The specified value must be a power of two (assert: NextPowerOfTwo(value) == value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void PowerOfTwo(ulong value, [InvokerParameterName] string paramName, string message = null) + { + if (unchecked((value & (value - 1)) != 0)) + { + throw FailArgumentNotPowerOfTwo(paramName, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(int value, int threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + throw FailArgumentNotGreaterThan(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EqualTo(long value, long expected, [InvokerParameterName] string paramName, string message = null) + { + if (value != expected) + { + throw FailArgumentExpected(paramName, expected, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EqualTo(ulong value, ulong expected, [InvokerParameterName] string paramName, string message = null) + { + if (value != expected) + { + throw FailArgumentExpected(paramName, expected, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EqualTo(string value, string expected, [InvokerParameterName] string paramName, string message = null) + { + if (value != expected) + { + throw FailArgumentExpected(paramName, expected, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EqualTo(T value, T expected, [InvokerParameterName] string paramName, string message = null) + where T : struct, IEquatable + { + if (!value.Equals(expected)) + { + throw FailArgumentExpected(paramName, expected, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotEqualTo(long value, long forbidden, [InvokerParameterName] string paramName, string message = null) + { + if (value == forbidden) + { + throw FailArgumentForbidden(paramName, forbidden, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotEqualTo(ulong value, ulong forbidden, [InvokerParameterName] string paramName, string message = null) + { + if (value == forbidden) + { + throw FailArgumentForbidden(paramName, forbidden, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotEqualTo(string value, string forbidden, [InvokerParameterName] string paramName, string message = null) + { + if (value == forbidden) + { + throw FailArgumentForbidden(paramName, forbidden, message); + } + } + + /// [RUNTIME] The specified value must not equal to the specified constant (assert: value != forbidden) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void NotEqualTo(T value, T forbidden, [InvokerParameterName] string paramName, string message = null) + where T : struct, IEquatable + { + if (value.Equals(forbidden)) + { + throw FailArgumentForbidden(paramName, forbidden, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(uint value, uint threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + throw FailArgumentNotGreaterThan(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(long value, long threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + throw FailArgumentNotGreaterThan(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(ulong value, ulong threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + throw FailArgumentNotGreaterThan(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(float value, float threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + // ReSharper disable once CompareOfFloatsByEqualityOperator + throw FailArgumentNotGreaterThan(paramName, threshold == 0.0f, message); + } + } + + /// [RUNTIME] The specified value must not less than or equal to the specified lower bound (assert: value > threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterThan(double value, double threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value <= threshold) + { + // ReSharper disable once CompareOfFloatsByEqualityOperator + throw FailArgumentNotGreaterThan(paramName, threshold == 0.0d, message); + } + } + + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterOrEqual(int value, int threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value < threshold) + { + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterOrEqual(uint value, uint threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value < threshold) + { + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterOrEqual(long value, long threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value < threshold) + { + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterOrEqual(ulong value, ulong threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value < threshold) + { + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0, message); + } + } + + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void GreaterOrEqual(float value, float threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value < threshold) + { + // ReSharper disable once CompareOfFloatsByEqualityOperator + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0.0f, message); + } + } - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not less than the specified lower bound (assert: value >= threshold) [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - [AssertionMethod] - public static void Requires([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void GreaterOrEqual(double value, double threshold, [InvokerParameterName] string paramName, string message = null) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Precondition, null, _path, _line); + if (value < threshold) + { + // ReSharper disable once CompareOfFloatsByEqualityOperator + throw FailArgumentNotGreaterOrEqual(paramName, threshold == 0.0d, message); + } } - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not greater than or equal to the specified upper bound [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - [AssertionMethod] - public static void Requires([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string message, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void LessThan(int value, int threshold, [InvokerParameterName] string paramName, string message = null) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Precondition, message, _path, _line); + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } } - #endregion + /// [RUNTIME] The specified value must not greater than or equal to the specified upper bound + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessThan(uint value, uint threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } + } - #region Assert + /// [RUNTIME] The specified value must not greater than or equal to the specified uppper bound (assert: value < threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessThan(long value, long threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } + } - /// Assert that a condition is verified, at debug time - /// Condition that must be true - /// Line number of the calling source file - /// Path of the calling source file - /// This method is not compiled on Release builds - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not greater than or equal to the specified uppper bound (assert: value < threshold) [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - [AssertionMethod] - public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void LessThan(ulong value, ulong threshold, [InvokerParameterName] string paramName, string message = null) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Assert, null, _path, _line); + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } } - /// Assert that a condition is verified, at debug time - /// Condition that must be true - /// Error message if the condition does not pass - /// Line number of the calling source file - /// Path of the calling source file - /// This method is not compiled on Release builds - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not greater than or equal to the specified uppper bound (assert: value < threshold) [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - [AssertionMethod] - public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string message, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void LessThan(float value, float threshold, [InvokerParameterName] string paramName, string message = null) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Assert, message, _path, _line); + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } } - #endregion + /// [RUNTIME] The specified value must not greater than or equal to the specified uppper bound (assert: value < threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessThan(double value, double threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value >= threshold) + { + throw FailArgumentNotLessThan(paramName, message); + } + } - #region Ensures + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessOrEqual(int value, int threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } + } - /// Assert that a condition is verified, at debug time - /// Condition that must be true - /// Line number of the calling source file - /// Path of the calling source file - /// This method is not compiled on Release builds - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - [AssertionMethod] - public static void Ensures([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void LessOrEqual(uint value, uint threshold, [InvokerParameterName] string paramName, string message = null) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Postcondition, null, _path, _line); + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } } - /// Assert that a condition is verified, at debug time - /// Condition that must be true - /// Error message if the condition does not pass - /// Line number of the calling source file - /// Path of the calling source file - /// This method is not compiled on Release builds - [DebuggerStepThrough, DebuggerHidden] - [Conditional("DEBUG")] -#if !NET_4_0 + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif + public static void LessOrEqual(long value, long threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } + } + + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessOrEqual(ulong value, ulong threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } + } + + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessOrEqual(float value, float threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } + } + + /// [RUNTIME] The specified value must not greater than the specified upper bound (assert: value <= threshold) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void LessOrEqual(double value, double threshold, [InvokerParameterName] string paramName, string message = null) + { + if (value > threshold) + { + throw FailArgumentNotLessOrEqual(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(int value, int minimumInclusive, int maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(uint value, uint minimumInclusive, uint maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(long value, long minimumInclusive, long maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(ulong value, ulong minimumInclusive, ulong maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(float value, float minimumInclusive, float maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + /// [RUNTIME] The specified value must not be outside of the specified bounds (assert: min <= value <= max) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Between(double value, double minimumInclusive, double maximumInclusive, [InvokerParameterName] string paramName, string message = null) + { + if (value < minimumInclusive || value > maximumInclusive) + { + throw FailArgumentOutOfBounds(paramName, message); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Multiple(int value, int multiple, [InvokerParameterName] string paramName, string message = null) + { + if (value % multiple != 0) + { + throw FailArgumentNotMultiple(paramName, message); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Multiple(uint value, uint multiple, [InvokerParameterName] string paramName, string message = null) + { + if (value % multiple != 0) + { + throw FailArgumentNotMultiple(paramName, message); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Multiple(long value, long multiple, [InvokerParameterName] string paramName, string message = null) + { + if (value % multiple != 0) + { + throw FailArgumentNotMultiple(paramName, message); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void Multiple(ulong value, ulong multiple, [InvokerParameterName] string paramName, string message = null) + { + if (value % multiple != 0) + { + throw FailArgumentNotMultiple(paramName, message); + } + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNotMultiple(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.ValueMustBeMultiple, message, paramName, ContractMessages.ConditionArgMultiple); + } + + #endregion + + #region Contract.DoesNotOverflow + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Buffer (qui ne doit pas être null) + /// Index (qui ne doit pas être négatif) + /// Taille (qui ne doit pas être négative) + /// + [AssertionMethod] + public static void DoesNotOverflow([AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string buffer, int index, int count, string message = null) + { + if (buffer == null) throw FailArgumentNull("buffer", message); + if (index < 0 || count < 0) throw FailArgumentNotNonNegative(index < 0 ? "index" : "count", message); + if ((buffer.Length - index) < count) throw FailBufferTooSmall("count", message); + } + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Taille du buffer + /// Index (qui ne doit pas être négatif) + /// Taille (qui ne doit pas être négative) + [AssertionMethod] + public static void DoesNotOverflow(int bufferLength, int offset, int count) + { + if (offset < 0 || count < 0) throw FailArgumentNotNonNegative(offset < 0 ? "offset" : "count", null); + if ((bufferLength - offset) < count) throw FailBufferTooSmall("count", null); + } + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Taille du buffer + /// Index (qui ne doit pas être négatif) + /// Taille (qui ne doit pas être négative) + [AssertionMethod] + public static void DoesNotOverflow(long bufferLength, long offset, long count) + { + if (offset < 0 || count < 0) throw FailArgumentNotNonNegative(offset < 0 ? "offset" : "count", null); + if ((bufferLength - offset) < count) throw FailBufferTooSmall("count", null); + } + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Buffer (qui ne doit pas être null) + /// Index (qui ne doit pas être négatif) + /// Taille (qui ne doit pas être négative) + /// + [AssertionMethod] + public static void DoesNotOverflow([AssertionCondition(AssertionConditionType.IS_NOT_NULL)] TElement[] buffer, int offset, int count, string message = null) + { + if (buffer == null) throw FailArgumentNull("buffer", message); + if (offset < 0 || count < 0) throw FailArgumentNotNonNegative(offset < 0 ? "offset" : "count", message); + if ((buffer.Length - offset) < count) throw FailBufferTooSmall("count", message); + } + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Buffer (qui ne doit pas être null) + /// + public static void DoesNotOverflow(ArraySegment buffer, string message = null) + { + if (buffer.Offset < 0 || buffer.Count < 0) throw FailArgumentNotNonNegative(buffer.Offset < 0 ? "offset" : "count", message); + if (buffer.Count > 0) + { + if (buffer.Array == null) throw FailBufferNull("buffer", message); + if ((buffer.Array.Length - buffer.Offset) < buffer.Count) throw FailBufferTooSmall("count", message); + } + else + { + if (buffer.Array != null && buffer.Array.Length < buffer.Offset) throw FailBufferTooSmall("count", message); + } + } + + /// Vérifie qu'une couple index/count ne débord pas d'un buffer, et qu'il n'est pas null + /// Buffer (qui ne doit pas être null) + /// Index (qui ne doit pas être négatif) + /// Taille (qui ne doit pas être négative) + /// [AssertionMethod] - public static void Ensures([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string message, [CallerLineNumber] int _line = 0, [CallerFilePath] string _path = "") + public static void DoesNotOverflow([AssertionCondition(AssertionConditionType.IS_NOT_NULL)] ICollection buffer, int offset, int count, string message = null) + { + if (buffer == null) throw FailArgumentNull("buffer", message); + if (offset < 0 || count < 0) throw FailArgumentNotNonNegative(offset < 0 ? "offset" : "count", message); + if ((buffer.Count - offset) < count) throw FailBufferTooSmall("count", message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailBufferTooSmall(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentException), ContractMessages.OffsetMustBeWithinBuffer, message, paramName, ContractMessages.ConditionArgBufferOverflow); + } + + #endregion + + #endregion + + #region Internal Helpers... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception FailArgumentNull(string paramName, string message = null) + { + return ReportFailure(typeof(ArgumentNullException), ContractMessages.ValueCannotBeNull, message, paramName, ContractMessages.ConditionNotNull); + } + + /// Déclenche une exception suite à l'échec d'une condition + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + internal static Exception ReportFailure(Type exceptionType, string msg, string userMessage, string paramName, string conditionTxt) + { + if (conditionTxt != null && conditionTxt.IndexOf('{') >= 0) + { // il y a peut etre un "{0}" dans la condition qu'il faut remplacer le nom du paramètre + conditionTxt = string.Format(conditionTxt, paramName ?? "value"); + } + + string str = SRC.ContractHelper.RaiseContractFailedEvent(SDC.ContractFailureKind.Precondition, userMessage ?? msg, conditionTxt, null); + // si l'appelant retourne null, c'est qu'il a lui même traité l'incident ... + // mais ca n'empeche pas qu'on doit quand même stopper l'execution ! +#if DEBUG + if (str != null) + { + // note: on ne spam les logs si on est en train de unit tester ! (vu qu'on va provoquer intentionellement plein d'erreurs!) + if (!IsUnitTesting) + { + System.Diagnostics.Debug.Fail(str); + } + } +#endif + string description = userMessage ?? str ?? msg; + + var exception = ThrowHelper.TryMapToKnownException(exceptionType, description, paramName); + + if (exception == null) + { // c'est un type compliqué ?? + exception = ThrowHelper.TryMapToComplexException(exceptionType, description, paramName); + } + + if (exception == null) + { // uh? on va quand même envoyer une exception proxy ! + exception = FallbackForUnknownException(description, paramName); + } + + return exception; + } + + [NotNull] + private static Exception FallbackForUnknownException(string description, string paramName) + { +#if DEBUG + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); // README: Si vous tombez ici, c'est que l'appelant a spécifié un type d'Exception qu'on n'arrive pas a construire! il faudrait peut être changer le type... +#endif + if (paramName != null) + return new ArgumentException(description, paramName); + else + return new InvalidOperationException(description); + } + + /// Signale l'échec d'une condition en déclenchant une ContractException + /// Si un debugger est attaché, un breakpoint est déclenché. Sinon, une ContractException est générée + [Pure, NotNull] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + [MethodImpl(MethodImplOptions.NoInlining)] + [DebuggerNonUserCode] + internal static Exception RaiseContractFailure(SDC.ContractFailureKind kind, string msg) { - if (!condition) RaiseContractFailure(SDC.ContractFailureKind.Postcondition, message, _path, _line); + string str = SRC.ContractHelper.RaiseContractFailedEvent(kind, msg, null, null); + if (str != null) + { + if (IsUnitTesting) + { + // throws une AssertionException si on a réussi a se connecter avec NUnit + var ex = MapToNUnitAssertion(str); +#if DEBUG + // README: Si vous breakpointez ici, il faut remonter plus haut dans la callstack, et trouver la fonction invoque Contract.xxx(...) + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); + // note: à partir de VS 2015 Up2, [DebuggerNonUserCode] n'est plus respecté si la regkey AlwaysEnableExceptionCallbacksOutsideMyCode n'est pas égale à 1, pour améliorer les perfs. + // cf "How to Suppress Ignorable Exceptions with DebuggerNonUserCode" dans https://blogs.msdn.microsoft.com/visualstudioalm/2016/02/12/using-the-debuggernonusercode-attribute-in-visual-studio-2015/ +#endif + if (ex != null) return ex; + // sinon, on continue + } +#if DEBUG + else if (kind == SDC.ContractFailureKind.Assert && Debugger.IsAttached) + { + // uniquement si on F5 depuis VS, car sinon cela cause problèmes avec le TestRunner de R# (qui popup les assertion fail!) + System.Diagnostics.Debug.Fail(str); + } +#endif + + return new ContractException(kind, str, msg, null, null); + } + //note: on doit quand même retourner quelque chose! + return new ContractException(kind, "Contract Failed", msg, null, null); } #endregion - [DebuggerStepThrough, DebuggerHidden] - internal static void RaiseContractFailure(SDC.ContractFailureKind kind, string message, string file, int line) + /// Contracts that are only evaluted in Debug builds + public static class Debug { - if (message == null) + // ReSharper disable MemberHidesStaticFromOuterClass + + // contains most of the same contracts as the main class, but only for Debug builds. + // ie: Contract.NotNull(...) will run in both Debug and Release builds, while Contract.Debug.NotNull(...) will NOT be evaluated in Release builds + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName + ) + { +#if DEBUG + if (value == null) + { + throw FailArgumentNull(paramName, null); + } +#endif + } + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName, + string message) + { +#if DEBUG + if (value == null) + { + throw FailArgumentNull(paramName, message); + } +#endif + } + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName + ) + { +#if DEBUG + if (string.IsNullOrEmpty(value)) + { + throw FailArgumentNull(paramName, null); + } +#endif + } + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNullOrEmpty( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL)] string value, + [InvokerParameterName] string paramName, + string message + ) + { +#if DEBUG + if (string.IsNullOrEmpty(value)) + { + throw FailArgumentNull(paramName, message); + } +#endif + } + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNull( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] T value, + [InvokerParameterName] string paramName + ) + where T : class + { +#if DEBUG + if (value == null) + { + throw FailArgumentNull(paramName, null); + } +#endif + } + + [AssertionMethod, Conditional("DEBUG")] + public static void NotNullAllowStructs( + [AssertionCondition(AssertionConditionType.IS_NOT_NULL), NoEnumeration] T value, + [InvokerParameterName] string paramName + ) { - switch(kind) +#if DEBUG + if (value == null) { - case SDC.ContractFailureKind.Assert: message = "An assertion was not met"; break; - case SDC.ContractFailureKind.Precondition: message = "A pre-requisite was not met"; break; - case SDC.ContractFailureKind.Postcondition: message = "A post-condition was not met"; break; - default: message = "An expectation was not met"; break; + throw FailArgumentNull(paramName, null); } +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, dans le body dans une méthode + /// Condition qui ne doit jamais être fausse + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Assert, null); +#endif + } + + /// [DEBUG ONLY] Vérifie qu'une condition est toujours vrai, dans le body dans une méthode + /// Condition qui ne doit jamais être fausse + /// Message décrivant l'erreur (optionnel) + /// Ne fait rien si la condition est vrai. Sinon déclenche une ContractException, après avoir essayé de breakpointer le debugger + [Conditional("DEBUG")] + [AssertionMethod, MethodImpl(MethodImplOptions.AggressiveInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Assert([AssertionCondition(AssertionConditionType.IS_TRUE)] bool condition, string userMessage) + { +#if DEBUG + if (!condition) throw RaiseContractFailure(SDC.ContractFailureKind.Assert, userMessage); +#endif } - if (file != null) - { // add the caller infos - message = String.Format("{0} in {1}:line {2}", message, file, line); + + /// [DEBUG ONLY] Déclenche incontionellement une assertion + [Conditional("DEBUG")] + [AssertionMethod, ContractAnnotation("=>halt"), MethodImpl(MethodImplOptions.NoInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Fail() + { +#if DEBUG + throw RaiseContractFailure(SDC.ContractFailureKind.Assert, null); +#endif } - //TODO: check if we are running under NUnit, and map to an Assert.Fail() instead ? + /// [DEBUG ONLY] Déclenche incontionellement une assertion + [Conditional("DEBUG")] + [AssertionMethod, ContractAnnotation("=>halt"), MethodImpl(MethodImplOptions.NoInlining)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.MayFail)] + public static void Fail(string userMessage) + { +#if DEBUG + throw RaiseContractFailure(SDC.ContractFailureKind.Assert, userMessage); +#endif + } - Debug.Fail(message); - // If you break here, that means that an assertion failed somewhere up the stack. - // TODO: find a way to have the debugger break, but show the caller of Contract.Assert(..) method, instead of here ? - if (Debugger.IsAttached) Debugger.Break(); + // ReSharper restore MemberHidesStaticFromOuterClass - throw new InvalidOperationException(message); } } + } diff --git a/FoundationDB.Client/Utils/ContractException.cs b/FoundationDB.Client/Utils/ContractException.cs new file mode 100644 index 000000000..3240f72ba --- /dev/null +++ b/FoundationDB.Client/Utils/ContractException.cs @@ -0,0 +1,66 @@ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information +#endregion + +namespace Doxense.Diagnostics.Contracts +{ + using System; + using System.Runtime.Serialization; + using System.Security; + using SDC = System.Diagnostics.Contracts; + + [Serializable] + public sealed class ContractException : Exception + { + // copie de l'implémentation "internal" de System.Data.Contracts.ContractException + + #region Constructors... + + private ContractException() + { + base.HResult = -2146233022; + } + + public ContractException(SDC.ContractFailureKind kind, string failure, string userMessage, string condition, Exception innerException) + : base(failure, innerException) + { + base.HResult = -2146233022; + this.Kind = kind; + this.UserMessage = userMessage; + this.Condition = condition; + } + + private ContractException(SerializationInfo info, StreamingContext context) + : base(info, context) + { + this.Kind = (SDC.ContractFailureKind)info.GetInt32("Kind"); + this.UserMessage = info.GetString("UserMessage"); + this.Condition = info.GetString("Condition"); + } + + #endregion + + #region Public Properties... + + public string Condition { get; } + + public SDC.ContractFailureKind Kind { get; } + + public string UserMessage { get; } + + public string Failure { get { return this.Message; } } + + #endregion + + [SecurityCritical] + public override void GetObjectData(SerializationInfo info, StreamingContext context) + { + base.GetObjectData(info, context); + info.AddValue("Kind", (int) this.Kind); + info.AddValue("UserMessage", this.UserMessage); + info.AddValue("Condition", this.Condition); + } + + } + +} diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index 441c2db35..60ea0ae88 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -28,8 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using JetBrains.Annotations; using System; using System.Collections.Generic; @@ -41,6 +39,9 @@ namespace FoundationDB.Client using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client.Utils; /// Delimits a section of a byte array [ImmutableObject(true), PublicAPI, DebuggerDisplay("Count={Count}, Offset={Offset}"), DebuggerTypeProxy(typeof(Slice.DebugView))] diff --git a/FoundationDB.Client/Utils/SliceBuffer.cs b/FoundationDB.Client/Utils/SliceBuffer.cs index d2d31d336..a6968a60a 100644 --- a/FoundationDB.Client/Utils/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/SliceBuffer.cs @@ -28,10 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Utils { - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Buffer that can be used to efficiently store multiple slices into as few chunks as possible /// diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index c4d93a7fb..51e995a81 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -30,13 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Runtime.CompilerServices; using System.Runtime.ConstrainedExecution; using System.Runtime.InteropServices; using System.Security; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; internal static class SliceHelpers { diff --git a/FoundationDB.Client/Utils/SliceListStream.cs b/FoundationDB.Client/Utils/SliceListStream.cs index 6acf1e90f..7f748de17 100644 --- a/FoundationDB.Client/Utils/SliceListStream.cs +++ b/FoundationDB.Client/Utils/SliceListStream.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.IO; using System.Linq; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using JetBrains.Annotations; /// Merge multiple slices into a single stream public sealed class SliceListStream : Stream diff --git a/FoundationDB.Client/Utils/SlicePairComparer.cs b/FoundationDB.Client/Utils/SlicePairComparer.cs index d8e58a17a..bd67e06b4 100644 --- a/FoundationDB.Client/Utils/SlicePairComparer.cs +++ b/FoundationDB.Client/Utils/SlicePairComparer.cs @@ -28,9 +28,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; /// Performs optimized equality and comparison checks on key/value pairs of public sealed class SlicePairComparer : IComparer>, IEqualityComparer> diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/SliceStream.cs index 3c79e3e41..63b860540 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/SliceStream.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using System; using System.IO; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; /// Stream that wraps a Slice for reading /// This stream is optimized for blocking and async reads diff --git a/FoundationDB.Client/Utils/SliceWriter.cs b/FoundationDB.Client/Utils/SliceWriter.cs index 10be371f6..f2a811bcc 100644 --- a/FoundationDB.Client/Utils/SliceWriter.cs +++ b/FoundationDB.Client/Utils/SliceWriter.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Diagnostics; using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Slice buffer that emulates a pseudo-stream using a byte array that will automatically grow in size, if necessary /// IMPORTANT: This struct does not extensively check the parameters! The caller should ensure that everything is valid (this is to get the max performance when serializing keys and values) diff --git a/FoundationDB.Client/Utils/ThrowHelper.cs b/FoundationDB.Client/Utils/ThrowHelper.cs new file mode 100644 index 000000000..5c0c0de8f --- /dev/null +++ b/FoundationDB.Client/Utils/ThrowHelper.cs @@ -0,0 +1,563 @@ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information +#endregion + +namespace Doxense.Diagnostics.Contracts +{ + using JetBrains.Annotations; + using System; + using System.Diagnostics; + using System.Globalization; + using System.Reflection; + using System.Runtime.CompilerServices; + + [DebuggerNonUserCode] + public static class ThrowHelper + { + + #region ArgumentNullException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentNullException([InvokerParameterName] string paramName) + { + return new ArgumentNullException(paramName); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentNullException([InvokerParameterName] string paramName, [NotNull] string message) + { + return new ArgumentNullException(paramName, message); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentNullException([InvokerParameterName] string paramName) + { + throw ArgumentNullException(paramName); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentNullException([InvokerParameterName] string paramName, [NotNull] string message) + { + throw ArgumentNullException(paramName, message); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowArgumentNullException([InvokerParameterName] string paramName, string message = null) + { + throw message != null ? new ArgumentNullException(paramName, message) : new ArgumentNullException(paramName); + } + + #endregion + + #region ArgumentException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentException([InvokerParameterName] string paramName, string message = null) + { + // oui, c'est inversé :) + return new ArgumentException(message, paramName); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentException([InvokerParameterName] string paramName, string message, object arg0) + { + // oui, c'est inversé :) + return new ArgumentException(string.Format(message, arg0), paramName); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentException([InvokerParameterName] string paramName, string message, object arg0, object arg1) + { + // oui, c'est inversé :) + return new ArgumentException(string.Format(message, arg0, arg1), paramName); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentException([InvokerParameterName] string paramName, string message, params object[] args) + { + // oui, c'est inversé :) + return new ArgumentException(string.Format(message, args), paramName); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentException([InvokerParameterName] string paramName, string message = null) + { + // oui, c'est inversé :) + throw ArgumentException(paramName, message); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowArgumentException([InvokerParameterName] string paramName, string message = null) + { + // oui, c'est inversé :) + throw ArgumentException(paramName, message); + } + + #endregion + + #region ArgumentOutOfRangeException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentOutOfRangeException([InvokerParameterName] string paramName, object actualValue, string message = null) + { + return new ArgumentOutOfRangeException(paramName, actualValue, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception ArgumentOutOfRangeException([InvokerParameterName, NotNull] string paramName) + { + return new ArgumentOutOfRangeException(paramName); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentOutOfRangeException() + { + // ReSharper disable once NotResolvedInText + throw ArgumentOutOfRangeException("index", "Index was out of range. Must be non-negative and less than the size of the collection."); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentOutOfRangeException([InvokerParameterName] string paramName) + { + throw ArgumentOutOfRangeException(paramName); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentOutOfRangeException([InvokerParameterName] string paramName, string message) + { + throw ArgumentOutOfRangeException(paramName, message); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentOutOfRangeException([InvokerParameterName] string paramName, object actualValue, string message) + { + throw ArgumentOutOfRangeException(paramName, actualValue, message); + } + + #endregion + + #region ObjectDisposedException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(TDisposed disposed) + { + return new ObjectDisposedException(disposed.GetType().Name); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(Type type) + { + return new ObjectDisposedException(type.Name); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(Type type, string message) + { + return new ObjectDisposedException(type.Name, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(TDisposed disposed, string message) + { + return new ObjectDisposedException(disposed.GetType().Name, message); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(string message) + { + return new ObjectDisposedException(typeof(TDisposed).Name, message); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(string message, object arg0) + { + return new ObjectDisposedException(typeof(TDisposed).Name, string.Format(CultureInfo.InvariantCulture, message, arg0)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(string message, params object[] args) + { + return new ObjectDisposedException(typeof(TDisposed).Name, string.Format(CultureInfo.InvariantCulture, message, args)); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ObjectDisposedException ObjectDisposedException(string message, Exception innnerException) + { + return new ObjectDisposedException(message, innnerException); + } + + [ContractAnnotation("=> halt")] + public static void ThrowObjectDisposedException(Type type) + { + throw ObjectDisposedException(type); + } + + [ContractAnnotation("=> halt")] + public static void ThrowObjectDisposedException(string message, Exception innnerException) + { + throw ObjectDisposedException(message, innnerException); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] //fix .NET < 4.5.2 + public static void ThrowObjectDisposedException(TDisposed disposed) + where TDisposed : IDisposable + { + throw ObjectDisposedException(disposed.GetType()); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] //fix .NET < 4.5.2 + public static void ThrowObjectDisposedException(TDisposed disposed, string message) + where TDisposed : IDisposable + { + throw ObjectDisposedException(disposed.GetType(), message); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] //fix .NET < 4.5.2 + public static void ThrowObjectDisposedException(TDisposed disposed, string message, object arg0) + where TDisposed : IDisposable + { + throw ObjectDisposedException(disposed.GetType(), string.Format(CultureInfo.InvariantCulture, message, arg0)); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] //fix .NET < 4.5.2 + public static void ThrowObjectDisposedException(TDisposed disposed, string message, params object[] args) + where TDisposed : IDisposable + { + throw ObjectDisposedException(disposed.GetType(), string.Format(CultureInfo.InvariantCulture, message, args)); + } + + #endregion + + #region InvalidOperationException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationException(string message) + { + return new InvalidOperationException(message); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationException(string message, object arg0) + { + return new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, message, arg0)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationException(string message, object arg0, object arg1) + { + return new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, message, arg0, arg1)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationException(string message, params object[] args) + { + return new InvalidOperationException(string.Format(CultureInfo.InvariantCulture, message, args)); + } + + [ContractAnnotation("=> halt")] + public static void ThrowInvalidOperationException(string message) + { + throw InvalidOperationException(message); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowInvalidOperationException(string message, object arg0) + { + throw InvalidOperationException(message, arg0); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowInvalidOperationException(string message, object arg0, object arg1) + { + throw InvalidOperationException(message, arg0, arg1); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowInvalidOperationException(string message, object arg0, object arg1, object arg2) + { + throw InvalidOperationException(message, arg0, arg1, arg2); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowInvalidOperationException(string message, params object[] args) + { + throw InvalidOperationException(message, args); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowInvalidOperationException(string message) + { + throw InvalidOperationException(message); + } + + #endregion + + #region FormatException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException FormatException(string message) + { + return new FormatException(message); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException FormatException(string message, object arg0) + { + return new FormatException(String.Format(CultureInfo.InvariantCulture, message, arg0)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException FormatException(string message, object arg0, object arg1) + { + return new FormatException(String.Format(CultureInfo.InvariantCulture, message, arg0, arg1)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException FormatException(string message, params object[] args) + { + return new FormatException(String.Format(CultureInfo.InvariantCulture, message, args)); + } + + [ContractAnnotation("=> halt")] + public static void ThrowFormatException(string message) + { + throw FormatException(message); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowFormatException(string message, object arg0) + { + throw FormatException(message, arg0); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowFormatException(string message, object arg0, object arg1) + { + throw FormatException(message, arg0, arg1); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowFormatException(string message, object arg0, object arg1, object arg2) + { + throw FormatException(message, arg0, arg1, arg2); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowFormatException(string message, params object[] args) + { + throw FormatException(message, args); + } + + #endregion + + #region OperationCanceledException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static OperationCanceledException OperationCanceledException(string message) + { + return new OperationCanceledException(message); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static OperationCanceledException OperationCanceledException(string message, object arg0) + { + return new OperationCanceledException(String.Format(CultureInfo.InvariantCulture, message, arg0)); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static OperationCanceledException OperationCanceledException(string message, params object[] args) + { + return new OperationCanceledException(String.Format(CultureInfo.InvariantCulture, message, args)); + } + + [ContractAnnotation("=> halt")] + public static void ThrowOperationCanceledException(string message) + { + throw OperationCanceledException(message); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowOperationCanceledException(string message, object arg0) + { + throw OperationCanceledException(message, arg0); + } + + [ContractAnnotation("=> halt"), StringFormatMethod("message")] + public static void ThrowOperationCanceledException(string message, params object[] args) + { + throw OperationCanceledException(message, args); + } + + #endregion + + #region NotSupportedException... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static NotSupportedException NotSupportedException(string message) + { + return new NotSupportedException(message); + } + + [Pure, NotNull, StringFormatMethod("message"), MethodImpl(MethodImplOptions.NoInlining)] + public static NotSupportedException NotSupportedException(string message, params object[] args) + { + return new NotSupportedException(String.Format(CultureInfo.InvariantCulture, message, args)); + } + + #endregion + + [CanBeNull, Pure, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception TryMapToKnownException(Type exceptionType, string message, string paramName) + { + // d'abord on regarde si c'est un type "simple" + if (exceptionType == typeof(ArgumentNullException)) + { + return new ArgumentNullException(paramName, message); + } + if (exceptionType == typeof(InvalidOperationException)) + { + return new InvalidOperationException(message); + } + if (exceptionType == typeof(ArgumentException)) + { + return new ArgumentException(message, paramName); + } + if (exceptionType == typeof(ArgumentOutOfRangeException)) + { + return new ArgumentOutOfRangeException(paramName, message); + } + if (exceptionType == typeof(ObjectDisposedException)) + { + return new ObjectDisposedException(paramName, message); + } + if (exceptionType == typeof (FormatException)) + { + return new FormatException(message); + } + return null; + } + + [CanBeNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception TryMapToComplexException(Type exceptionType, string message, string paramName) + { + ConstructorInfo constructor; + + if (paramName != null) + { // essayes de trouver un constructeur qui prenne deux string dont une soit "paramName" + constructor = exceptionType.GetConstructor(new[] { typeof(string), typeof(string) }); + if (constructor != null) + { + if (constructor.GetParameters()[0].Name == "paramName") + { + return constructor.Invoke(new object[] { paramName, message }) as Exception; + } + else if (constructor.GetParameters()[1].Name == "paramName") + { + return constructor.Invoke(new object[] { message, paramName }) as Exception; + } + } + } + + // essayes de trouver un constructeur qui prenne une string + constructor = exceptionType.GetConstructor(new[] { typeof(string) }); + if (constructor != null) + { + return constructor.Invoke(new object[] { message }) as Exception; + } + + // c'est un type d'erreur qui ne prend pas de params ? + constructor = exceptionType.GetConstructor(Type.EmptyTypes); + if (constructor != null) + { + return constructor.Invoke(null) as Exception; + } + + return null; + } + + #region Collection Errors... + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationNoElements() + { + return new InvalidOperationException("Sequence contains no elements."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException InvalidOperationNoMatchingElements() + { + return new InvalidOperationException("Sequence contains no matching element."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static IndexOutOfRangeException IndexOutOfRangeException() + { + return new IndexOutOfRangeException("Index was out of range. Must be non-negative and less than the size of the collection."); + } + + [ContractAnnotation("=> halt")] + public static void ThrowIndexOutOfRangeException() + { + throw IndexOutOfRangeException(); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowIndexOutOfRangeException() + { + throw IndexOutOfRangeException(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentOutOfRangeException ArgumentOutOfRangeIndex(int index) + { + // ArgumentOutOfRange_NeedNonNegNum + // ReSharper disable once UseNameofExpression + return new ArgumentOutOfRangeException("index", index, "Index was out of range. Must be non-negative and less than the size of the collection."); + } + + [ContractAnnotation("=> halt")] + public static void ThrowArgumentOutOfRangeIndex(int index) + { + // ArgumentOutOfRange_NeedNonNegNum + throw ArgumentOutOfRangeIndex(index); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowArgumentOutOfRangeIndex(int index) + { + throw IndexOutOfRangeException(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentOutOfRangeException ArgumentOutOfRangeNeedNonNegNum([InvokerParameterName] string paramName) + { + // ArgumentOutOfRange_NeedNonNegNum + return new ArgumentOutOfRangeException(paramName, "Non-negative number required"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException ArgumentInvalidOffLen() + { + // Argument_InvalidOffLen + return new ArgumentException("Offset and length were out of bounds for the array or count is greater than the number of elements from index to the end of the source collection."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static NotSupportedException NotSupportedReadOnlyCollection() + { + // NotSupported_ReadOnlyCollection + return new NotSupportedException("Collection is read-only."); + } + + [ContractAnnotation("=> halt")] + public static void ThrowNotSupportedReadOnlyCollection() + { + // NotSupported_ReadOnlyCollection + throw NotSupportedReadOnlyCollection(); + } + + #endregion + + } + +} diff --git a/FoundationDB.Client/Utils/TinyJsonParser.cs b/FoundationDB.Client/Utils/TinyJsonParser.cs index 628466a46..399582aa8 100644 --- a/FoundationDB.Client/Utils/TinyJsonParser.cs +++ b/FoundationDB.Client/Utils/TinyJsonParser.cs @@ -1,12 +1,12 @@ -using JetBrains.Annotations; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Text; - + namespace FoundationDB.Client.Utils { + using System; + using System.Collections.Generic; + using System.Globalization; + using System.Text; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Tiny JSON parser internal sealed class TinyJsonParser diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index 876c45446..ea706aca4 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.ComponentModel; using System.Diagnostics; using System.Globalization; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; [DebuggerDisplay("[{ToString()}]")] [ImmutableObject(true), PublicAPI, Serializable] diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 8e96dc6f6..b0fec26a5 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Blobs { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Diagnostics; using System.Globalization; using System.Threading.Tasks; + using FoundationDB.Client; + using JetBrains.Annotations; + using Doxense.Diagnostics.Contracts; /// Represents a potentially large binary value in FoundationDB. [DebuggerDisplay("Subspace={Subspace}")] diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index 12f61b461..220b5251b 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -28,16 +28,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections { - using FoundationDB.Async; - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Async; + using FoundationDB.Client; + using FoundationDB.Linq; + using JetBrains.Annotations; [DebuggerDisplay("Name={Name}, Subspace={Subspace}")] public class FdbMap diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index f72d5b84d..fda4eba21 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Linq; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Linq; + using JetBrains.Annotations; /// /// Provides a high-contention Queue class diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index 6ab0d44ed..f03e3a70c 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Linq; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Linq; + using JetBrains.Annotations; /// Represents a potentially sparse array in FoundationDB. public class FdbVector diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index bd41e125f..e8ca59e12 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -28,18 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Blobs { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; - using System.Globalization; - using System.IO; - using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using FoundationDB.Linq; + using JetBrains.Annotations; // THIS IS NOT AN OFFICIAL LAYER, JUST A PROTOTYPE TO TEST A FEW THINGS ! diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs index cd9f5c322..2ff0af68b 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Builder of compressed bitmaps that can set or clear bits in a random order, in memory public sealed class CompressedBitmapBuilder diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs index c70fe17a7..26d9e67b0 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { - using FoundationDB.Client; - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; /// Iterator that reads 32-bit compressed words from a compressed bitmap public struct CompressedBitmapWordIterator : IEnumerator diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs index 8f7a0ae63..9177fdc00 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { + using System; + using Doxense.Diagnostics.Contracts; using FoundationDB.Client; - using FoundationDB.Client.Utils; using JetBrains.Annotations; - using System; /// Writer that compresses a stream of bits into a , in memory public sealed class CompressedBitmapWriter diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs index c2fdf1871..252fc8e11 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { - using FoundationDB.Client.Utils; using System; using System.Diagnostics; using System.Globalization; + using Doxense.Diagnostics.Contracts; /// Represent a 32-bit word in a Compressed Bitmap [DebuggerDisplay("Literal={IsLiteral}, {WordCount} x {WordValue}")] diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs index dfe077c56..8b75fc8f0 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs @@ -28,15 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Globalization; using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; public static class WordAlignHybridEncoder { diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs index 03600624a..c4c254a3f 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Linq.Expressions; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Expression that uses an async sequence as the source of elements public sealed class FdbQueryAsyncEnumerableExpression : FdbQuerySequenceExpression diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs index 330661957..f830aea7c 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs @@ -28,18 +28,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Linq.Expressions; - using System.Reflection; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Base class of all query expression extensions - public abstract class FdbQueryExpression : Expression - { + public abstract class FdbQueryExpression : Expression + { private readonly Type m_type; /// Base ctor diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs index 5f68c4c81..d80886e21 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs @@ -28,12 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Linq.Expressions; - using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Expression that represent a filter on a source sequence /// Type of elements in the source sequence diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs index 658b41764..7bd9af96a 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs @@ -28,14 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Indexing; - using JetBrains.Annotations; using System; using System.Globalization; using System.Linq.Expressions; - using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using FoundationDB.Layers.Indexing; + using JetBrains.Annotations; /// Expression that represents a lookup on an FdbIndex /// Type of the Id of the enties being indexed diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs index 08c3659fc..837433d93 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs @@ -28,15 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections.Generic; - using System.Linq; using System.Linq.Expressions; - using System.Reflection; - using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Mode of execution of a merge operation public enum FdbQueryMergeType diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs index 30875fd2a..d66c25bf4 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs @@ -28,13 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Linq.Expressions; - using System.Reflection; - using System.Threading; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using JetBrains.Annotations; /// Expression that represent a projection from one type into another /// Type of elements in the inner sequence diff --git a/FoundationDB.Tests/Experimental/JsonNetCodec.cs b/FoundationDB.Tests/Experimental/JsonNetCodec.cs index 86a4ab70b..af07ed72e 100644 --- a/FoundationDB.Tests/Experimental/JsonNetCodec.cs +++ b/FoundationDB.Tests/Experimental/JsonNetCodec.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Types.Json { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using Newtonsoft.Json; using System; using System.IO; using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; + using Newtonsoft.Json; /// Sample codec that uses JSON.Net to serialize values into Slices /// diff --git a/FoundationDB.Tests/Experimental/ProtobufCodec.cs b/FoundationDB.Tests/Experimental/ProtobufCodec.cs index 6270562b2..9a0a99fe7 100644 --- a/FoundationDB.Tests/Experimental/ProtobufCodec.cs +++ b/FoundationDB.Tests/Experimental/ProtobufCodec.cs @@ -28,10 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Types.ProtocolBuffers { - using FoundationDB.Client; - using FoundationDB.Client.Utils; using System; using System.IO; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; public class ProtobufCodec : IValueEncoder, IUnorderedTypeCodec { From 849be964bee72ff6f426dda9ec98decf45bfaec1 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 14:57:05 +0200 Subject: [PATCH 058/153] Renamed all FdbKeySomething into KeySomething - FdbKeyRange => KeyRange - FdbKeySelector => KeySelector - FdbKeySelectorPair => KeySelectorPair - FdbKeyRangeComparer => KeyRangeComparer --- FdbShell/Commands/BasicCommands.cs | 26 ++++---- .../Core/IFdbTransactionHandler.cs | 6 +- FoundationDB.Client/Fdb.Bulk.cs | 12 ++-- FoundationDB.Client/Fdb.System.cs | 32 ++++----- FoundationDB.Client/FdbDatabase.cs | 6 +- FoundationDB.Client/FdbDatabaseExtensions.cs | 17 +++-- .../FdbMergeQueryExtensions.cs | 34 +++++----- .../FdbRangeQuery.PagingIterator.cs | 12 ++-- FoundationDB.Client/FdbRangeQuery.cs | 16 ++--- .../FdbTransaction.Snapshot.cs | 8 +-- FoundationDB.Client/FdbTransaction.cs | 10 +-- .../FdbTransactionExtensions.cs | 42 ++++++------ .../Filters/FdbDatabaseFilter.cs | 6 +- .../Filters/FdbTransactionFilter.cs | 16 ++--- .../Filters/Logging/FdbLoggedTransaction.cs | 24 +++---- .../Logging/FdbTransactionLog.Commands.cs | 14 ++-- .../Filters/PrefixRewriterTransaction.cs | 18 ++--- .../FoundationDB.Client.csproj | 8 +-- .../IFdbReadOnlyTransaction.cs | 8 +-- .../{FdbKeyRange.cs => KeyRange.cs} | 54 +++++++-------- ...eyRangeComparer.cs => KeyRangeComparer.cs} | 16 ++--- .../{FdbKeySelector.cs => KeySelector.cs} | 50 +++++++------- ...bKeySelectorPair.cs => KeySelectorPair.cs} | 52 +++++++-------- .../Layers/Directories/FdbDirectoryLayer.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 12 ++-- .../Layers/Tuples/FdbTupleExtensions.cs | 10 +-- .../Tuples/TypeSystem/TupleKeyEncoder.cs | 20 +++--- FoundationDB.Client/Native/FdbNative.cs | 4 +- .../Native/FdbNativeTransaction.cs | 6 +- .../Subspaces/Fdb.Directory.cs | 8 +-- .../Subspaces/FdbDynamicSubspace.cs | 1 + .../Subspaces/FdbDynamicSubspaceKeys.cs | 8 +-- .../Subspaces/FdbEncoderSubspaceKeys`1.cs | 2 +- .../Subspaces/FdbEncoderSubspaceKeys`2.cs | 2 +- .../Subspaces/FdbEncoderSubspaceKeys`3.cs | 2 +- .../Subspaces/FdbEncoderSubspaceKeys`4.cs | 2 +- FoundationDB.Client/Subspaces/FdbSubspace.cs | 10 +-- .../Subspaces/FdbSubspaceExtensions.cs | 4 +- FoundationDB.Client/Subspaces/IFdbSubspace.cs | 6 +- .../Encoders/DynamicKeyEncoderBase.cs | 22 +++---- .../TypeSystem/IDynamicKeyEncoder.cs | 20 +++--- FoundationDB.Client/Utils/SliceBuffer.cs | 24 +++---- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 6 +- .../Collections/FdbMultimap`2.cs | 6 +- .../Collections/FdbQueue`1.cs | 2 +- .../Collections/FdbRankedSet.cs | 10 +-- .../Collections/FdbVector`1.cs | 2 +- .../Indexes/FdbIndex`2.cs | 14 ++-- .../Documents/FdbDocumentCollection.cs | 8 +-- .../Documents/FdbHashSetCollection.cs | 6 +- .../Messaging/FdbWorkerPool.cs | 4 +- .../Messaging/WorkerPoolTest.cs | 2 +- .../Expressions/FdbQueryExpressions.cs | 8 +-- .../Expressions/FdbQueryRangeExpression.cs | 8 +-- .../FdbAsyncQueryable.cs | 2 +- .../Tutorials/ClassScheduling.cs | 2 +- FoundationDB.Samples/Utils/Sampler.cs | 10 +-- FoundationDB.Tests.Sandbox/Program.cs | 2 +- FoundationDB.Tests/ExoticTestCases.cs | 42 ++++++------ .../Filters/LoggingFilterFacts.cs | 6 +- FoundationDB.Tests/KeyFacts.cs | 66 +++++++++---------- FoundationDB.Tests/RangeQueryFacts.cs | 6 +- FoundationDB.Tests/TestHelpers.cs | 2 +- FoundationDB.Tests/TransactionFacts.cs | 64 +++++++++--------- 64 files changed, 467 insertions(+), 463 deletions(-) rename FoundationDB.Client/{FdbKeyRange.cs => KeyRange.cs} (86%) rename FoundationDB.Client/{FdbKeyRangeComparer.cs => KeyRangeComparer.cs} (81%) rename FoundationDB.Client/{FdbKeySelector.cs => KeySelector.cs} (77%) rename FoundationDB.Client/{FdbKeySelectorPair.cs => KeySelectorPair.cs} (70%) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index ecbeac4ab..8efb32076 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -400,15 +400,15 @@ public static async Task Map(string[] path, IFdbTuple extras, IFdbDatabase db, T var bounds = await db.ReadAsync(async (tr) => { var kvs = await Task.WhenAll( - tr.GetRange(FdbKeyRange.StartsWith(key)).FirstOrDefaultAsync(), - tr.GetRange(FdbKeyRange.StartsWith(key)).LastOrDefaultAsync() + tr.GetRange(KeyRange.StartsWith(key)).FirstOrDefaultAsync(), + tr.GetRange(KeyRange.StartsWith(key)).LastOrDefaultAsync() ); return new { Min = kvs[0].Key, Max = kvs[1].Key }; }, ct); if (bounds.Min.HasValue) { // folder is not empty - shards = await Fdb.System.GetChunksAsync(db, FdbKeyRange.StartsWith(key), ct); + shards = await Fdb.System.GetChunksAsync(db, KeyRange.StartsWith(key), ct); //TODO: we still need to check if the first and last shard really intersect the subspace // we need to check if the shards actually contain data @@ -460,7 +460,7 @@ public static async Task Topology(string[] path, IFdbTuple extras, IFdbDatabase var servers = await db.QueryAsync(tr => tr .WithReadAccessToSystemKeys() - .GetRange(FdbKeyRange.StartsWith(Fdb.System.ServerList)) + .GetRange(KeyRange.StartsWith(Fdb.System.ServerList)) .Select(kvp => new { // Offsets Size Type Name Description @@ -554,7 +554,7 @@ public static async Task Shards(string[] path, IFdbTuple extras, IFdbDatabase db var folder = (await TryOpenCurrentDirectoryAsync(path, db, ct)) as FdbDirectorySubspace; if (folder != null) { - var r = FdbKeyRange.StartsWith(FdbSubspace.Copy(folder).Key); + var r = KeyRange.StartsWith(FdbSubspace.Copy(folder).Key); Console.WriteLine("Searching for shards that intersect with /{0} ...", String.Join("/", path)); ranges = await Fdb.System.GetChunksAsync(db, r, ct); Console.WriteLine("Found {0} ranges intersecting {1}:", ranges.Count, r); @@ -586,16 +586,16 @@ public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase } var folder = await TryOpenCurrentDirectoryAsync(path, db, ct); - FdbKeyRange span; + KeyRange span; if (folder is FdbDirectorySubspace) { - span = FdbKeyRange.StartsWith(FdbSubspace.Copy(folder as FdbDirectorySubspace).Key); + span = KeyRange.StartsWith(FdbSubspace.Copy(folder as FdbDirectorySubspace).Key); log.WriteLine("Reading list of shards for /{0} under {1} ...", String.Join("/", path), FdbKey.Dump(span.Begin)); } else { log.WriteLine("Reading list of shards for the whole cluster ..."); - span = FdbKeyRange.All; + span = KeyRange.All; } // dump keyServers @@ -603,7 +603,7 @@ public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase log.WriteLine("> Found {0:N0} shard(s)", ranges.Count); // take a sample - var samples = new List(); + var samples = new List(); if (ranges.Count <= 32) { // small enough to scan it all @@ -620,7 +620,7 @@ public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase if (sz < 32) sz = Math.Max(sz, Math.Min(32, ranges.Count)); } - var population = new List(ranges); + var population = new List(ranges); for (int i = 0; i < sz; i++) { int p = rnd.Next(population.Count); @@ -667,8 +667,8 @@ public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase long count = 0; int iter = 0; - var beginSelector = FdbKeySelector.FirstGreaterOrEqual(range.Begin); - var endSelector = FdbKeySelector.FirstGreaterOrEqual(range.End); + var beginSelector = KeySelector.FirstGreaterOrEqual(range.Begin); + var endSelector = KeySelector.FirstGreaterOrEqual(range.End); while (true) { FdbRangeChunk data = default(FdbRangeChunk); @@ -707,7 +707,7 @@ public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase if (!data.HasMore) break; - beginSelector = FdbKeySelector.FirstGreaterThan(data.Last.Key); + beginSelector = KeySelector.FirstGreaterThan(data.Last.Key); ++iter; } diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 4dada99aa..06954242d 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.cs @@ -89,7 +89,7 @@ public interface IFdbTransactionHandler : IDisposable /// Set to true for snapshot reads /// Token used to cancel the operation from the outside /// Task that will return the key matching the selector, or an exception - Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken); + Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken cancellationToken); /// Resolves several key selectors against the keys in the database snapshot represented by the current transaction. /// Key selectors to resolve @@ -97,7 +97,7 @@ public interface IFdbTransactionHandler : IDisposable /// Token used to cancel the operation from the outside /// Task that will return an array of keys matching the selectors, or an exception [ItemNotNull] - Task GetKeysAsync([NotNull] FdbKeySelector[] selectors, bool snapshot, CancellationToken cancellationToken); + Task GetKeysAsync([NotNull] KeySelector[] selectors, bool snapshot, CancellationToken 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 @@ -107,7 +107,7 @@ public interface IFdbTransactionHandler : IDisposable /// Set to true for snapshot reads /// Token used to cancel the operation from the outside /// - Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken); + Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken 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. diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index b4a2d8790..83bdb073f 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -1314,7 +1314,7 @@ private static int FillNextBatch(IEnumerator iterator, List batch, int /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. public static Task ExportAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) { - return ExportAsync(db, FdbKeySelector.FirstGreaterOrEqual(beginInclusive), FdbKeySelector.FirstGreaterOrEqual(endExclusive), handler, cancellationToken); + return ExportAsync(db, KeySelector.FirstGreaterOrEqual(beginInclusive), KeySelector.FirstGreaterOrEqual(endExclusive), handler, cancellationToken); } /// Export the content of a potentially large range of keys defined by a pair of begin and end keys. @@ -1324,9 +1324,9 @@ public static Task ExportAsync([NotNull] IFdbDatabase db, Slice beginInclu /// Token used to cancel the operation /// Number of keys exported /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. - public static Task ExportAsync([NotNull] IFdbDatabase db, FdbKeyRange range, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) + public static Task ExportAsync([NotNull] IFdbDatabase db, KeyRange range, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) { - return ExportAsync(db, FdbKeySelector.FirstGreaterOrEqual(range.Begin), FdbKeySelector.FirstGreaterOrEqual(range.End), handler, cancellationToken); + return ExportAsync(db, KeySelector.FirstGreaterOrEqual(range.Begin), KeySelector.FirstGreaterOrEqual(range.End), handler, cancellationToken); } /// Export the content of a potentially large range of keys defined by a pair of selectors. @@ -1337,7 +1337,7 @@ public static Task ExportAsync([NotNull] IFdbDatabase db, FdbKeyRange rang /// Token used to cancel the operation /// Number of keys exported /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. - public static async Task ExportAsync([NotNull] IFdbDatabase db, FdbKeySelector begin, FdbKeySelector end, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) + public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelector begin, KeySelector end, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) { if (db == null) throw new ArgumentNullException("db"); if (handler == null) throw new ArgumentNullException("handler"); @@ -1391,7 +1391,7 @@ public static async Task ExportAsync([NotNull] IFdbDatabase db, FdbKeySele while (page.HasMore) { // prefetch the next one (don't wait for the task yet) - var next = FetchNextBatchAsync(tr, FdbKeySelector.FirstGreaterThan(page.Last.Key), end, options, reset); + var next = FetchNextBatchAsync(tr, KeySelector.FirstGreaterThan(page.Last.Key), end, options, reset); // process the current one if (page.Count > 0) @@ -1428,7 +1428,7 @@ public static async Task ExportAsync([NotNull] IFdbDatabase db, FdbKeySele /// Range read options /// Action (optional) that can reconfigure a transaction whenever it gets reset inside the retry loop. /// Task that will return the next batch - private static async Task FetchNextBatchAsync(IFdbReadOnlyTransaction tr, FdbKeySelector begin, FdbKeySelector end, [NotNull] FdbRangeOptions options, Action onReset = null) + private static async Task FetchNextBatchAsync(IFdbReadOnlyTransaction tr, KeySelector begin, KeySelector end, [NotNull] FdbRangeOptions options, Action onReset = null) { Contract.Requires(tr != null && options != null); diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 9330108da..40b6733c4 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -274,7 +274,7 @@ public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. [ItemNotNull] - public static Task> GetChunksAsync([NotNull] IFdbDatabase db, FdbKeyRange range, CancellationToken cancellationToken) + public static Task> GetChunksAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken cancellationToken) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? return GetChunksAsync(db, range.Begin, range.End, cancellationToken); @@ -288,7 +288,7 @@ public static Task> GetChunksAsync([NotNull] IFdbDatabase db, /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. [ItemNotNull] - public static async Task> GetChunksAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) + public static async Task> GetChunksAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? @@ -298,24 +298,24 @@ public static async Task> GetChunksAsync([NotNull] IFdbDatabas var boundaries = await GetBoundaryKeysAsync(db, beginInclusive, endExclusive, cancellationToken).ConfigureAwait(false); int count = boundaries.Count; - var chunks = new List(count + 2); + var chunks = new List(count + 2); if (count == 0) { // the range does not cross any boundary, and is contained in just one chunk - chunks.Add(new FdbKeyRange(beginInclusive, endExclusive)); + chunks.Add(new KeyRange(beginInclusive, endExclusive)); return chunks; } var k = boundaries[0]; - if (k != beginInclusive) chunks.Add(new FdbKeyRange(beginInclusive, k)); + if (k != beginInclusive) chunks.Add(new KeyRange(beginInclusive, k)); for (int i = 1; i < boundaries.Count; i++) { - chunks.Add(new FdbKeyRange(k, boundaries[i])); + chunks.Add(new KeyRange(k, boundaries[i])); k = boundaries[i]; } - if (k != endExclusive) chunks.Add(new FdbKeyRange(k, endExclusive)); + if (k != endExclusive) chunks.Add(new KeyRange(k, endExclusive)); return chunks; } @@ -326,7 +326,7 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF Contract.Requires(trans != null && end >= begin); #if TRACE_COUNTING - trans.Annotate("Get boundary keys in range {0}", FdbKeyRange.Create(begin, end)); + trans.Annotate("Get boundary keys in range {0}", KeyRange.Create(begin, end)); #endif trans.WithReadAccessToSystemKeys(); @@ -378,7 +378,7 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF #if TRACE_COUNTING if (results.Count == 0) { - trans.Annotate("There is no chunk boundary in range {0}", FdbKeyRange.Create(begin, end)); + trans.Annotate("There is no chunk boundary in range {0}", KeyRange.Create(begin, end)); } else { @@ -395,7 +395,7 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRange range, CancellationToken cancellationToken) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken cancellationToken) { return EstimateCountAsync(db, range.Begin, range.End, null, cancellationToken); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! @@ -408,7 +408,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRan /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, FdbKeyRange range, IProgress> onProgress, CancellationToken cancellationToken) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken cancellationToken) { return EstimateCountAsync(db, range.Begin, range.End, onProgress, cancellationToken); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! @@ -454,13 +454,13 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli using (var tr = db.BeginReadOnlyTransaction(cancellationToken)) { #if TRACE_COUNTING - tr.Annotate("Estimating number of keys in range {0}", FdbKeyRange.Create(beginInclusive, endExclusive)); + tr.Annotate("Estimating number of keys in range {0}", KeyRange.Create(beginInclusive, endExclusive)); #endif tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); // start looking for the first key in the range - cursor = await tr.Snapshot.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(cursor)).ConfigureAwait(false); + cursor = await tr.Snapshot.GetKeyAsync(KeySelector.FirstGreaterOrEqual(cursor)).ConfigureAwait(false); if (cursor >= end) { // the range is empty ! return 0; @@ -479,7 +479,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli { Contract.Assert(windowSize > 0); - var selector = FdbKeySelector.FirstGreaterOrEqual(cursor) + windowSize; + var selector = KeySelector.FirstGreaterOrEqual(cursor) + windowSize; Slice next = Slice.Nil; FdbException error = null; try @@ -530,8 +530,8 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli // Count the keys by reading them. Also, we know that there can not be more than windowSize - 1 remaining int n = await tr.Snapshot .GetRange( - FdbKeySelector.FirstGreaterThan(cursor), // cursor has already been counted once - FdbKeySelector.FirstGreaterOrEqual(end), + KeySelector.FirstGreaterThan(cursor), // cursor has already been counted once + KeySelector.FirstGreaterOrEqual(end), new FdbRangeOptions() { Limit = windowSize - 1 } ) .CountAsync() diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 49b1ef004..c73a673b6 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -597,17 +597,17 @@ IFdbSubspace IFdbSubspace.this[IFdbKey key] } } - FdbKeyRange IFdbSubspace.ToRange() + KeyRange IFdbSubspace.ToRange() { return m_globalSpace.ToRange(); } - FdbKeyRange IFdbSubspace.ToRange(Slice suffix) + KeyRange IFdbSubspace.ToRange(Slice suffix) { return m_globalSpace.ToRange(suffix); } - FdbKeyRange IFdbSubspace.ToRange(TKey key) + KeyRange IFdbSubspace.ToRange(TKey key) { return m_globalSpace.ToRange(key); } diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index 2d808a32f..9b103c72e 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -47,11 +47,14 @@ public static class FdbDatabaseExtensions /// New transaction instance that can read from the database. /// You MUST call Dispose() on the transaction when you are done with it. You SHOULD wrap it in a 'using' statement to ensure that it is disposed in all cases. /// + /// /// using(var tr = db.BeginReadOnlyTransaction(CancellationToken.None)) /// { /// var result = await tr.Get(Slice.FromString("Hello")); - /// var items = await tr.GetRange(FdbKeyRange.StartsWith(Slice.FromString("ABC"))).ToListAsync(); - /// } + /// var items = await tr.GetRange(KeyRange.StartsWith(Slice.FromString("ABC"))).ToListAsync(); + /// } + /// + /// [Pure, NotNull] public static IFdbReadOnlyTransaction BeginReadOnlyTransaction([NotNull] this IFdbDatabase db, CancellationToken cancellationToken) { @@ -257,7 +260,7 @@ public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, FdbKeySelector keySelector, CancellationToken cancellationToken) + public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector keySelector, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetKeyAsync(keySelector), cancellationToken); @@ -270,7 +273,7 @@ public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, F /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] FdbKeySelector[] keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] KeySelector[] keySelectors, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); Contract.NotNull(keySelectors, nameof(keySelectors)); @@ -284,7 +287,7 @@ public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); Contract.NotNull(keySelectors, nameof(keySelectors)); @@ -297,7 +300,7 @@ public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetryable db, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken cancellationToken) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); return db.ReadAsync((tr) => tr.GetRangeAsync(beginInclusive, endExclusive, options, iteration), cancellationToken); @@ -379,7 +382,7 @@ public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, Slice beginK /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, FdbKeyRange range, CancellationToken cancellationToken) + public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, KeyRange range, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => tr.ClearRange(range), cancellationToken); diff --git a/FoundationDB.Client/FdbMergeQueryExtensions.cs b/FoundationDB.Client/FdbMergeQueryExtensions.cs index 1dd141e55..a0926b66a 100644 --- a/FoundationDB.Client/FdbMergeQueryExtensions.cs +++ b/FoundationDB.Client/FdbMergeQueryExtensions.cs @@ -42,7 +42,7 @@ public static class FdbMergeQueryExtensions #region MergeSort (x OR y) [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> MergeSort(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable> MergeSort([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -60,7 +60,7 @@ public static IFdbAsyncEnumerable> MergeSort(th } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable MergeSort(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -79,7 +79,7 @@ public static IFdbAsyncEnumerable MergeSort(this IFdbRea } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Union(IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Union([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); @@ -93,7 +93,7 @@ public static IFdbAsyncEnumerable Union(IEnumerable Union(IEnumerable> sources, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Union([NotNull] IEnumerable> sources, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); return new FdbMergeSortIterator( @@ -110,7 +110,7 @@ public static IFdbAsyncEnumerable Union(IEnumerable> Intersect(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable> Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -128,7 +128,7 @@ public static IFdbAsyncEnumerable> Intersect(th } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? @@ -143,7 +143,7 @@ public static IFdbAsyncEnumerable Intersect(this IFdbRea } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); @@ -157,7 +157,7 @@ public static IFdbAsyncEnumerable Intersect(this IFdbAsy } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) + public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); @@ -171,7 +171,7 @@ public static IFdbAsyncEnumerable Intersect(this IFdbAsyncEnum } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect(IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Intersect([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); @@ -185,7 +185,7 @@ public static IFdbAsyncEnumerable Intersect(IEnumerable< } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect(IEnumerable> sources, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Intersect([NotNull] IEnumerable> sources, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); return new FdbIntersectIterator( @@ -209,7 +209,7 @@ public static IFdbAsyncEnumerable Intersect(IEnumerableInstance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> Except(this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -234,10 +234,10 @@ public static IFdbAsyncEnumerable> Except(this /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { Contract.NotNull(ranges, nameof(ranges)); - return Except(trans, ranges.Select(r => FdbKeySelectorPair.Create(r)), keySelector, keyComparer); + return Except(trans, ranges.Select(r => KeySelectorPair.Create(r)), keySelector, keyComparer); } /// Return the keys that are in the first range, but not in the others @@ -250,7 +250,7 @@ public static IFdbAsyncEnumerable> Except([NotN /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? @@ -274,10 +274,10 @@ public static IFdbAsyncEnumerable Except([NotNull] this /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) + public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { Contract.NotNull(ranges, nameof(ranges)); - return Except(trans, ranges.Select(r => FdbKeySelectorPair.Create(r)), keySelector, resultSelector, keyComparer); + return Except(trans, ranges.Select(r => KeySelectorPair.Create(r)), keySelector, resultSelector, keyComparer); } /// Sequence the return only the elements of that are not in , using a custom key comparison @@ -310,7 +310,7 @@ public static IFdbAsyncEnumerable Except([NotNull] this /// Instance used to compare elements /// Async query that returns only the elements that are in , and not in [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except(this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) + public static IFdbAsyncEnumerable Except([NotNull] this IFdbAsyncEnumerable first, [NotNull] IFdbAsyncEnumerable second, IComparer comparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 74362966c..c6f49647e 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -61,10 +61,10 @@ private sealed class PagingIterator : FdbAsyncIteratorKey selector describing the beginning of the current range (when paging) - private FdbKeySelector Begin { get; set; } + private KeySelector Begin { get; set; } /// Key selector describing the end of the current range (when paging) - private FdbKeySelector End { get; set; } + private KeySelector End { get; set; } /// If non null, contains the remaining allowed number of rows private int? RemainingCount { get; set; } @@ -135,8 +135,8 @@ protected override async Task OnFirstAsync(CancellationToken cancellationT if (min >= max) return false; // range is empty // rewrite the initial selectors with the bounded keys - this.Begin = FdbKeySelector.FirstGreaterOrEqual(min); - this.End = FdbKeySelector.FirstGreaterOrEqual(max); + this.Begin = KeySelector.FirstGreaterOrEqual(min); + this.End = KeySelector.FirstGreaterOrEqual(max); } return true; } @@ -232,11 +232,11 @@ private Task FetchNextPageAsync(CancellationToken cancellationToken) var lastKey = result.Last.Key; if (this.Query.Reversed) { - this.End = FdbKeySelector.FirstGreaterOrEqual(lastKey); + this.End = KeySelector.FirstGreaterOrEqual(lastKey); } else { - this.Begin = FdbKeySelector.FirstGreaterThan(lastKey); + this.Begin = KeySelector.FirstGreaterThan(lastKey); } } #if DEBUG_RANGE_PAGING diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index ffb446bcc..4d17ac9dc 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -44,7 +44,7 @@ public sealed partial class FdbRangeQuery : IFdbAsyncEnumerable { /// Construct a query with a set of initial settings - internal FdbRangeQuery([NotNull] IFdbReadOnlyTransaction transaction, FdbKeySelector begin, FdbKeySelector end, [NotNull] Func, T> transform, bool snapshot, FdbRangeOptions options) + internal FdbRangeQuery([NotNull] IFdbReadOnlyTransaction transaction, KeySelector begin, KeySelector end, [NotNull] Func, T> transform, bool snapshot, FdbRangeOptions options) { Contract.Requires(transaction != null && transform != null); @@ -54,7 +54,7 @@ internal FdbRangeQuery([NotNull] IFdbReadOnlyTransaction transaction, FdbKeySele this.Transform = transform; this.Snapshot = snapshot; this.Options = options ?? new FdbRangeOptions(); - this.OriginalRange = FdbKeySelectorPair.Create(begin, end); + this.OriginalRange = KeySelectorPair.Create(begin, end); } /// Copy constructor @@ -74,20 +74,20 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption #region Public Properties... /// Key selector describing the beginning of the range that will be queried - public FdbKeySelector Begin { get; private set; } + public KeySelector Begin { get; private set; } /// Key selector describing the end of the range that will be queried - public FdbKeySelector End { get; private set; } + public KeySelector End { get; private set; } /// Key selector pair describing the beginning and end of the range that will be queried - public FdbKeySelectorPair Range { get { return new FdbKeySelectorPair(this.Begin, this.End); } } + public KeySelectorPair Range { get { return new KeySelectorPair(this.Begin, this.End); } } /// Stores all the settings for this range query internal FdbRangeOptions Options { get; } /// Original key selector pair describing the bounds of the parent range. All the results returned by the query will be bounded by this original range. /// May differ from when combining certain operators. - internal FdbKeySelectorPair OriginalRange { get; } + internal KeySelectorPair OriginalRange { get; } /// Limit in number of rows to return public int? Limit { get { return this.Options.Limit; } } @@ -217,9 +217,9 @@ public FdbRangeQuery Reverse() /// /// A new query object that will use the specified target bytes size when executed [NotNull] - public FdbRangeQuery WithTargetBytes(int bytes) + public FdbRangeQuery WithTargetBytes([Positive] int bytes) { - if (bytes < 0) throw new ArgumentOutOfRangeException("bytes", "Value cannot be less than zero"); + Contract.Positive(bytes, nameof(bytes)); return new FdbRangeQuery( this, diff --git a/FoundationDB.Client/FdbTransaction.Snapshot.cs b/FoundationDB.Client/FdbTransaction.Snapshot.cs index 9c1e7bce7..725a489e6 100644 --- a/FoundationDB.Client/FdbTransaction.Snapshot.cs +++ b/FoundationDB.Client/FdbTransaction.Snapshot.cs @@ -137,7 +137,7 @@ public Task GetValuesAsync(Slice[] keys) return m_parent.m_handler.GetValuesAsync(keys, snapshot: true, cancellationToken: m_parent.m_cancellation); } - public async Task GetKeyAsync(FdbKeySelector selector) + public async Task GetKeyAsync(KeySelector selector) { EnsureCanRead(); @@ -154,7 +154,7 @@ public async Task GetKeyAsync(FdbKeySelector selector) } - public Task GetKeysAsync(FdbKeySelector[] selectors) + public Task GetKeysAsync(KeySelector[] selectors) { EnsureCanRead(); @@ -170,7 +170,7 @@ public Task GetKeysAsync(FdbKeySelector[] selectors) return m_parent.m_handler.GetKeysAsync(selectors, snapshot: true, cancellationToken: m_parent.m_cancellation); } - public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options, int iteration) + public Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options, int iteration) { EnsureCanRead(); @@ -186,7 +186,7 @@ public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySe return m_parent.m_handler.GetRangeAsync(beginInclusive, endExclusive, options, iteration, snapshot: true, cancellationToken: m_parent.m_cancellation); } - public FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options) + public FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options) { EnsureCanRead(); diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index e9952137e..4707e08c2 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -351,7 +351,7 @@ public Task GetValuesAsync(Slice[] keys) /// Optionnal query options (Limit, TargetBytes, StreamingMode, 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. /// - public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { EnsureCanRead(); @@ -371,7 +371,7 @@ public Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySe #region GetRange... - internal FdbRangeQuery> GetRangeCore(FdbKeySelector begin, FdbKeySelector end, FdbRangeOptions options, bool snapshot) + internal FdbRangeQuery> GetRangeCore(KeySelector begin, KeySelector end, FdbRangeOptions options, bool snapshot) { this.Database.EnsureKeyIsValid(begin.Key); this.Database.EnsureKeyIsValid(end.Key, endExclusive: true); @@ -393,7 +393,7 @@ internal FdbRangeQuery> GetRangeCore(FdbKeySelector b /// key selector defining the end of the range /// Optionnal query options (Limit, TargetBytes, Mode, Reverse, ...) /// Range query that, once executed, will return all the key-value pairs matching the providing selector pair - public FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null) + public FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null) { EnsureCanRead(); @@ -407,7 +407,7 @@ public FdbRangeQuery> GetRange(FdbKeySelector beginIn /// Resolves a key selector against the keys in the database snapshot represented by transaction. /// Key selector to resolve /// Task that will return the key matching the selector, or an exception - public async Task GetKeyAsync(FdbKeySelector selector) + public async Task GetKeyAsync(KeySelector selector) { EnsureCanRead(); @@ -432,7 +432,7 @@ public async Task GetKeyAsync(FdbKeySelector selector) ///
/// Key selectors to resolve /// Task that will return an array of keys matching the selectors, or an exception - public Task GetKeysAsync(FdbKeySelector[] selectors) + public Task GetKeysAsync(KeySelector[] selectors) { EnsureCanRead(); diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index c61712f92..325f35233 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -578,19 +578,19 @@ public static void AtomicMin(this IFdbTransaction trans, TKey key, Slice v #region GetRange... - public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelector beginInclusive, FdbKeySelector endExclusive, int limit, bool reverse = false) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeySelector beginInclusive, KeySelector endExclusive, int limit, bool reverse = false) { Contract.NotNull(trans, nameof(trans)); return trans.GetRange(beginInclusive, endExclusive, new FdbRangeOptions(limit: limit, reverse: reverse)); } - public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeyRange range, FdbRangeOptions options = null) { - return FdbTransactionExtensions.GetRange(trans, FdbKeySelectorPair.Create(range), options); + return FdbTransactionExtensions.GetRange(trans, KeySelectorPair.Create(range), options); } - public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, int limit, bool reverse = false) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeyRange range, int limit, bool reverse = false) { return FdbTransactionExtensions.GetRange(trans, range, new FdbRangeOptions(limit: limit, reverse: reverse)); } @@ -603,8 +603,8 @@ public static FdbRangeQuery> GetRange([NotNull] this if (endKeyExclusive.IsNullOrEmpty) endKeyExclusive = FdbKey.MaxValue; return trans.GetRange( - FdbKeySelector.FirstGreaterOrEqual(beginKeyInclusive), - FdbKeySelector.FirstGreaterOrEqual(endKeyExclusive), + KeySelector.FirstGreaterOrEqual(beginKeyInclusive), + KeySelector.FirstGreaterOrEqual(endKeyExclusive), options ); } @@ -639,7 +639,7 @@ public static FdbRangeQuery> GetRange(this IFdb /// Pair of key selectors defining the beginning and the end of the range /// Optionnal query options (Limit, TargetBytes, Mode, Reverse, ...) /// Range query that, once executed, will return all the key-value pairs matching the providing selector pair - public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeySelectorPair range, FdbRangeOptions options = null) { Contract.NotNull(trans, nameof(trans)); @@ -656,7 +656,7 @@ public static FdbRangeQuery> GetRange([NotNull] this /// 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. /// - public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, FdbKeySelectorPair range, FdbRangeOptions options = null, int iteration = 0) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, KeySelectorPair range, FdbRangeOptions options = null, int iteration = 0) { Contract.NotNull(trans, nameof(trans)); @@ -673,11 +673,11 @@ public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTrans /// 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. /// - public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, FdbKeyRange range, FdbRangeOptions options = null, int iteration = 0) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTransaction trans, KeyRange range, FdbRangeOptions options = null, int iteration = 0) { Contract.NotNull(trans, nameof(trans)); - var sp = FdbKeySelectorPair.Create(range); + var sp = KeySelectorPair.Create(range); return trans.GetRangeAsync(sp.Begin, sp.End, options, iteration); } @@ -696,7 +696,7 @@ public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTrans { Contract.NotNull(trans, nameof(trans)); - var range = FdbKeySelectorPair.Create(beginInclusive, endExclusive); + var range = KeySelectorPair.Create(beginInclusive, endExclusive); return trans.GetRangeAsync(range.Begin, range.End, options, iteration); } @@ -723,7 +723,7 @@ public static void Clear(this IFdbTransaction trans, TKey key) /// /// Transaction to use for the operation /// Pair of keys defining the range to clear. - public static void ClearRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) + public static void ClearRange([NotNull] this IFdbTransaction trans, KeyRange range) { Contract.NotNull(trans, nameof(trans)); @@ -740,7 +740,7 @@ public static void ClearRange([NotNull] this IFdbTransaction trans, FdbKeyRange /// Transaction to use for the operation /// Range of the keys specifying the conflict range. The end key is excluded /// One of the FDBConflictRangeType values indicating what type of conflict range is being set. - public static void AddConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range, FdbConflictRangeType type) + public static void AddConflictRange([NotNull] this IFdbTransaction trans, KeyRange range, FdbConflictRangeType type) { Contract.NotNull(trans, nameof(trans)); @@ -751,7 +751,7 @@ public static void AddConflictRange([NotNull] this IFdbTransaction trans, FdbKey /// /// Adds a range of keys to the transaction’s read conflict ranges as if you had read the range. As a result, other transactions that write a key in this range could cause the transaction to fail with a conflict. /// - public static void AddReadConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) + public static void AddReadConflictRange([NotNull] this IFdbTransaction trans, KeyRange range) { AddConflictRange(trans, range, FdbConflictRangeType.Read); } @@ -784,7 +784,7 @@ public static void AddReadConflictRange(this IFdbTransaction trans, TKey b /// public static void AddReadConflictKey([NotNull] this IFdbTransaction trans, Slice key) { - AddConflictRange(trans, FdbKeyRange.FromKey(key), FdbConflictRangeType.Read); + AddConflictRange(trans, KeyRange.FromKey(key), FdbConflictRangeType.Read); } /// @@ -794,13 +794,13 @@ public static void AddReadConflictKey(this IFdbTransaction trans, TKey key where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); - AddConflictRange(trans, FdbKeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Read); + AddConflictRange(trans, KeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Read); } /// /// Adds a range of keys to the transaction’s write conflict ranges as if you had cleared the range. As a result, other transactions that concurrently read a key in this range could fail with a conflict. /// - public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, FdbKeyRange range) + public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, KeyRange range) { AddConflictRange(trans, range, FdbConflictRangeType.Write); } @@ -833,7 +833,7 @@ public static void AddWriteConflictRange(this IFdbTransaction trans, TKey /// public static void AddWriteConflictKey([NotNull] this IFdbTransaction trans, Slice key) { - AddConflictRange(trans, FdbKeyRange.FromKey(key), FdbConflictRangeType.Write); + AddConflictRange(trans, KeyRange.FromKey(key), FdbConflictRangeType.Write); } /// @@ -843,7 +843,7 @@ public static void AddWriteConflictKey(this IFdbTransaction trans, TKey ke where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); - AddConflictRange(trans, FdbKeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Write); + AddConflictRange(trans, KeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Write); } #endregion @@ -1018,12 +1018,12 @@ public static Task GetValuesAsync(this IFdbReadOnlyTrans /// Sequence of key selectors to resolve /// Task that will return an array of keys matching the selectors, or an exception [ItemNotNull] - public static Task GetKeysAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable selectors) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable selectors) { Contract.NotNull(trans, nameof(trans)); Contract.NotNull(selectors, nameof(selectors)); - var array = selectors as FdbKeySelector[] ?? selectors.ToArray(); + var array = selectors as KeySelector[] ?? selectors.ToArray(); return trans.GetKeysAsync(array); } diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index 9a0c4782d..db9840bc8 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -140,17 +140,17 @@ Slice IFdbSubspace.Key get { return this.GlobalSpace.Key; } } - FdbKeyRange IFdbSubspace.ToRange() + KeyRange IFdbSubspace.ToRange() { return this.GlobalSpace.ToRange(); } - FdbKeyRange IFdbSubspace.ToRange(Slice suffix) + KeyRange IFdbSubspace.ToRange(Slice suffix) { return this.GlobalSpace.ToRange(suffix); } - FdbKeyRange IFdbSubspace.ToRange(TKey key) + KeyRange IFdbSubspace.ToRange(TKey key) { return this.GlobalSpace.ToRange(key); } diff --git a/FoundationDB.Client/Filters/FdbTransactionFilter.cs b/FoundationDB.Client/Filters/FdbTransactionFilter.cs index aace924a8..d702f870f 100644 --- a/FoundationDB.Client/Filters/FdbTransactionFilter.cs +++ b/FoundationDB.Client/Filters/FdbTransactionFilter.cs @@ -150,25 +150,25 @@ public virtual Task GetValuesAsync(Slice[] keys) return m_transaction.GetValuesAsync(keys); } - public virtual Task GetKeyAsync(FdbKeySelector selector) + public virtual Task GetKeyAsync(KeySelector selector) { ThrowIfDisposed(); return m_transaction.GetKeyAsync(selector); } - public virtual Task GetKeysAsync(FdbKeySelector[] selectors) + public virtual Task GetKeysAsync(KeySelector[] selectors) { ThrowIfDisposed(); return m_transaction.GetKeysAsync(selectors); } - public virtual Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public virtual Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { ThrowIfDisposed(); return m_transaction.GetRangeAsync(beginInclusive, endExclusive, options, iteration); } - public virtual FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null) + public virtual FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null) { ThrowIfDisposed(); return m_transaction.GetRange(beginInclusive, endExclusive, options); @@ -375,22 +375,22 @@ public virtual Task GetValuesAsync(Slice[] keys) return m_transaction.GetValuesAsync(keys); } - public virtual Task GetKeyAsync(FdbKeySelector selector) + public virtual Task GetKeyAsync(KeySelector selector) { return m_transaction.GetKeyAsync(selector); } - public virtual Task GetKeysAsync(FdbKeySelector[] selectors) + public virtual Task GetKeysAsync(KeySelector[] selectors) { return m_transaction.GetKeysAsync(selectors); } - public virtual Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public virtual Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { return m_transaction.GetRangeAsync(beginInclusive, endExclusive, options, iteration); } - public virtual FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endInclusive, FdbRangeOptions options = null) + public virtual FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endInclusive, FdbRangeOptions options = null) { return m_transaction.GetRange(beginInclusive, endInclusive, options); } diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index 8f920bf1d..982fd7c96 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -144,16 +144,16 @@ private Slice[] Grab(Slice[] slices) } } - private FdbKeySelector Grab(FdbKeySelector selector) + private KeySelector Grab(KeySelector selector) { - return new FdbKeySelector(Grab(selector.Key), selector.OrEqual, selector.Offset); + return new KeySelector(Grab(selector.Key), selector.OrEqual, selector.Offset); } - private FdbKeySelector[] Grab(FdbKeySelector[] selectors) + private KeySelector[] Grab(KeySelector[] selectors) { if (selectors == null || selectors.Length == 0) return null; - var res = new FdbKeySelector[selectors.Length]; + var res = new KeySelector[selectors.Length]; for (int i = 0; i < selectors.Length; i++) { res[i] = Grab(selectors[i]); @@ -369,7 +369,7 @@ public override Task GetAsync(Slice key) ); } - public override Task GetKeyAsync(FdbKeySelector selector) + public override Task GetKeyAsync(KeySelector selector) { return ExecuteAsync( new FdbTransactionLog.GetKeyCommand(Grab(selector)), @@ -385,7 +385,7 @@ public override Task GetValuesAsync(Slice[] keys) ); } - public override Task GetKeysAsync(FdbKeySelector[] selectors) + public override Task GetKeysAsync(KeySelector[] selectors) { return ExecuteAsync( new FdbTransactionLog.GetKeysCommand(Grab(selectors)), @@ -393,7 +393,7 @@ public override Task GetKeysAsync(FdbKeySelector[] selectors) ); } - public override Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public override Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { return ExecuteAsync( new FdbTransactionLog.GetRangeCommand(Grab(beginInclusive), Grab(endExclusive), options, iteration), @@ -401,7 +401,7 @@ public override Task GetRangeAsync(FdbKeySelector beginInclusive, ); } - public override FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null) + public override FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null) { ThrowIfDisposed(); @@ -478,7 +478,7 @@ public override Task GetAsync(Slice key) ); } - public override Task GetKeyAsync(FdbKeySelector selector) + public override Task GetKeyAsync(KeySelector selector) { return ExecuteAsync( new FdbTransactionLog.GetKeyCommand(m_parent.Grab(selector)), @@ -494,7 +494,7 @@ public override Task GetValuesAsync(Slice[] keys) ); } - public override Task GetKeysAsync(FdbKeySelector[] selectors) + public override Task GetKeysAsync(KeySelector[] selectors) { return ExecuteAsync( new FdbTransactionLog.GetKeysCommand(m_parent.Grab(selectors)), @@ -502,7 +502,7 @@ public override Task GetKeysAsync(FdbKeySelector[] selectors) ); } - public override Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public override Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { return ExecuteAsync( new FdbTransactionLog.GetRangeCommand(m_parent.Grab(beginInclusive), m_parent.Grab(endExclusive), options, iteration), @@ -510,7 +510,7 @@ public override Task GetRangeAsync(FdbKeySelector beginInclusive, ); } - public override FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null) + public override FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null) { m_parent.ThrowIfDisposed(); var query = m_transaction.GetRange(beginInclusive, endExclusive, options); diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index c11a87e97..8ebccc07c 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -637,11 +637,11 @@ protected override string Dump(Slice value) public sealed class GetKeyCommand : Command { /// Selector to a key in the database - public FdbKeySelector Selector { get; private set; } + public KeySelector Selector { get; private set; } public override Operation Op { get { return Operation.GetKey; } } - public GetKeyCommand(FdbKeySelector selector) + public GetKeyCommand(KeySelector selector) { this.Selector = selector; } @@ -722,11 +722,11 @@ public override string GetResult(KeyResolver resolver) public sealed class GetKeysCommand : Command { /// List of selectors looked up in the database - public FdbKeySelector[] Selectors { get; private set; } + public KeySelector[] Selectors { get; private set; } public override Operation Op { get { return Operation.GetKeys; } } - public GetKeysCommand(FdbKeySelector[] selectors) + public GetKeysCommand(KeySelector[] selectors) { this.Selectors = selectors; } @@ -767,9 +767,9 @@ public override string GetArguments(KeyResolver resolver) public sealed class GetRangeCommand : Command { /// Selector to the start of the range - public FdbKeySelector Begin { get; private set; } + public KeySelector Begin { get; private set; } /// Selector to the end of the range - public FdbKeySelector End { get; private set; } + public KeySelector End { get; private set; } /// Options of the range read public FdbRangeOptions Options { get; private set; } /// Iteration number @@ -777,7 +777,7 @@ public sealed class GetRangeCommand : Command public override Operation Op { get { return Operation.GetRange; } } - public GetRangeCommand(FdbKeySelector begin, FdbKeySelector end, FdbRangeOptions options, int iteration) + public GetRangeCommand(KeySelector begin, KeySelector end, FdbRangeOptions options, int iteration) { this.Begin = begin; this.End = end; diff --git a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs index 10b16e438..5f062903b 100644 --- a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs +++ b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs @@ -59,16 +59,16 @@ private Slice[] Encode(Slice[] keys) return m_prefix.ConcatKeys(keys); } - private FdbKeySelector Encode(FdbKeySelector selector) + private KeySelector Encode(KeySelector selector) { - return new FdbKeySelector( + return new KeySelector( m_prefix.ConcatKey(selector.Key), selector.OrEqual, selector.Offset ); } - private FdbKeySelector[] Encode(FdbKeySelector[] selectors) + private KeySelector[] Encode(KeySelector[] selectors) { var keys = new Slice[selectors.Length]; for (int i = 0; i < selectors.Length;i++) @@ -77,10 +77,10 @@ private FdbKeySelector[] Encode(FdbKeySelector[] selectors) } keys = m_prefix.ConcatKeys(keys); - var res = new FdbKeySelector[selectors.Length]; + var res = new KeySelector[selectors.Length]; for (int i = 0; i < selectors.Length; i++) { - res[i] = new FdbKeySelector( + res[i] = new KeySelector( keys[i], selectors[i].OrEqual, selectors[i].Offset @@ -114,12 +114,12 @@ public override Task GetValuesAsync(Slice[] keys) return base.GetValuesAsync(Encode(keys)); } - public override async Task GetKeyAsync(FdbKeySelector selector) + public override async Task GetKeyAsync(KeySelector selector) { return Decode(await base.GetKeyAsync(Encode(selector)).ConfigureAwait(false)); } - public override async Task GetKeysAsync(FdbKeySelector[] selectors) + public override async Task GetKeysAsync(KeySelector[] selectors) { return Decode(await base.GetKeysAsync(Encode(selectors)).ConfigureAwait(false)); } @@ -129,12 +129,12 @@ public override Task GetAddressesForKeyAsync(Slice key) return base.GetAddressesForKeyAsync(Encode(key)); } - public override FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null) + public override FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null) { throw new NotImplementedException(); } - public override Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) + public override Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0) { throw new NotImplementedException(); } diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 4bcb96fe0..ce12c10a1 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -101,7 +101,7 @@ - + @@ -137,7 +137,7 @@ - + @@ -145,7 +145,7 @@ - + @@ -181,7 +181,7 @@ - + diff --git a/FoundationDB.Client/IFdbReadOnlyTransaction.cs b/FoundationDB.Client/IFdbReadOnlyTransaction.cs index 2b24bb197..9f65764af 100644 --- a/FoundationDB.Client/IFdbReadOnlyTransaction.cs +++ b/FoundationDB.Client/IFdbReadOnlyTransaction.cs @@ -83,13 +83,13 @@ public interface IFdbReadOnlyTransaction : IDisposable /// Resolves a key selector against the keys in the database snapshot represented by the current transaction. /// Key selector to resolve /// Task that will return the key matching the selector, or an exception - Task GetKeyAsync(FdbKeySelector selector); + Task GetKeyAsync(KeySelector selector); /// Resolves several key selectors against the keys in the database snapshot represented by the current transaction. /// Key selectors to resolve /// Task that will return an array of keys matching the selectors, or an exception [ItemNotNull] - Task GetKeysAsync([NotNull] FdbKeySelector[] selectors); + Task GetKeysAsync([NotNull] KeySelector[] selectors); /// /// Reads all key-value pairs in the database snapshot represented by transaction (potentially limited by Limit, TargetBytes, or Mode) @@ -101,7 +101,7 @@ public interface IFdbReadOnlyTransaction : 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. /// - Task GetRangeAsync(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0); + Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null, int iteration = 0); /// /// Create a new range query that will read all key-value pairs in the database snapshot represented by the transaction @@ -110,7 +110,7 @@ public interface IFdbReadOnlyTransaction : IDisposable /// key selector defining the end of the range /// Optionnal query options (Limit, TargetBytes, Mode, Reverse, ...) /// Range query that, once executed, will return all the key-value pairs matching the providing selector pair - FdbRangeQuery> GetRange(FdbKeySelector beginInclusive, FdbKeySelector endExclusive, FdbRangeOptions options = null); + FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options = null); /// 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. diff --git a/FoundationDB.Client/FdbKeyRange.cs b/FoundationDB.Client/KeyRange.cs similarity index 86% rename from FoundationDB.Client/FdbKeyRange.cs rename to FoundationDB.Client/KeyRange.cs index 5708188d1..06eb12468 100644 --- a/FoundationDB.Client/FdbKeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -34,13 +34,13 @@ namespace FoundationDB.Client /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] - public struct FdbKeyRange : IEquatable, IComparable + public struct KeyRange : IEquatable, IComparable { /// Returns an empty pair of keys - public static FdbKeyRange Empty { get { return default(FdbKeyRange); } } + public static KeyRange Empty { get { return default(KeyRange); } } /// Returns a range that contains all the keys in the database - public static FdbKeyRange All { get { return new FdbKeyRange(FdbKey.MinValue, FdbKey.MaxValue); } } + public static KeyRange All { get { return new KeyRange(FdbKey.MinValue, FdbKey.MaxValue); } } /// Start of the range public Slice Begin { get { return m_begin; } } @@ -55,7 +55,7 @@ public struct FdbKeyRange : IEquatable, IComparable /// /// Start of range (usually included) /// End of range (usually excluded) - public FdbKeyRange(Slice begin, Slice end) + public KeyRange(Slice begin, Slice end) { m_begin = begin; m_end = end; @@ -63,7 +63,7 @@ public FdbKeyRange(Slice begin, Slice end) Contract.Ensures(m_begin <= m_end, "The range is inverted"); } - public FdbKeyRange(IFdbKey begin, IFdbKey end) + public KeyRange(IFdbKey begin, IFdbKey end) { if (begin == null) throw new ArgumentNullException("begin"); if (end == null) throw new ArgumentNullException("end"); @@ -74,28 +74,28 @@ public FdbKeyRange(IFdbKey begin, IFdbKey end) Contract.Ensures(m_begin <= m_end, "The range is inverted"); } - public static FdbKeyRange Create(Slice a, Slice b) + public static KeyRange Create(Slice a, Slice b) { - return new FdbKeyRange(a, b); + return new KeyRange(a, b); } /// Create a range that will return all keys starting with : ('prefix' <= k < strinc('prefix')) /// /// - public static FdbKeyRange StartsWith(Slice prefix) + public static KeyRange StartsWith(Slice prefix) { if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull("prefix"); - if (prefix.Count == 0) return new FdbKeyRange(Slice.Empty, FdbKey.MaxValue); + if (prefix.Count == 0) return new KeyRange(Slice.Empty, FdbKey.MaxValue); // prefix => [ prefix, prefix + 1 ) - return new FdbKeyRange( + return new KeyRange( prefix, FdbKey.Increment(prefix) ); } - public static FdbKeyRange StartsWith(TKey prefix) + public static KeyRange StartsWith(TKey prefix) where TKey : IFdbKey { if (prefix == null) throw new ArgumentNullException("prefix"); @@ -105,18 +105,18 @@ public static FdbKeyRange StartsWith(TKey prefix) /// Create a range that selects all keys starting with , but not the prefix itself: ('prefix\x00' <= k < string('prefix') /// Key prefix (that will be excluded from the range) /// Range including all keys with the specified prefix. - public static FdbKeyRange PrefixedBy(Slice prefix) + public static KeyRange PrefixedBy(Slice prefix) { if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull("prefix"); // prefix => [ prefix."\0", prefix + 1) - return new FdbKeyRange( + return new KeyRange( prefix + FdbKey.MinValue, FdbKey.Increment(prefix) ); } - public static FdbKeyRange PrefixedBy(TKey prefix) + public static KeyRange PrefixedBy(TKey prefix) where TKey : IFdbKey { if (prefix == null) throw new ArgumentNullException("prefix"); @@ -126,22 +126,22 @@ public static FdbKeyRange PrefixedBy(TKey prefix) /// Create a range that will only return itself ('key' <= k < 'key\x00') /// Key that will be returned by the range /// Range that only return the specified key. - public static FdbKeyRange FromKey(Slice key) + public static KeyRange FromKey(Slice key) { if (key.IsNull) throw Fdb.Errors.KeyCannotBeNull(); if (key.Count == 0) { // "" => [ "", "\x00" ) - return new FdbKeyRange(Slice.Empty, FdbKey.MinValue); + return new KeyRange(Slice.Empty, FdbKey.MinValue); } // key => [ key, key + '\0' ) - return new FdbKeyRange( + return new KeyRange( key, key + FdbKey.MinValue ); } - public static FdbKeyRange FromKey(TKey key) + public static KeyRange FromKey(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -150,7 +150,7 @@ public static FdbKeyRange FromKey(TKey key) public override bool Equals(object obj) { - return (obj is FdbKeyRange) && Equals((FdbKeyRange)obj); + return (obj is KeyRange) && Equals((KeyRange)obj); } public override int GetHashCode() @@ -162,22 +162,22 @@ public override int GetHashCode() return ((h1 << 5) + h1) ^ h2; } - public bool Equals(FdbKeyRange other) + public bool Equals(KeyRange other) { return m_begin.Equals(other.m_begin) && m_end.Equals(other.m_end); } - public static bool operator ==(FdbKeyRange left, FdbKeyRange right) + public static bool operator ==(KeyRange left, KeyRange right) { return left.m_begin.Equals(right.m_begin) && left.m_end.Equals(right.m_end); } - public static bool operator !=(FdbKeyRange left, FdbKeyRange right) + public static bool operator !=(KeyRange left, KeyRange right) { return !left.m_begin.Equals(right.m_begin) || !left.m_end.Equals(right.m_end); } - public int CompareTo(FdbKeyRange other) + public int CompareTo(KeyRange other) { int c = m_begin.CompareTo(other.m_begin); if (c == 0) c = m_end.CompareTo(other.m_end); @@ -188,18 +188,18 @@ public int CompareTo(FdbKeyRange other) /// Range to merge with the current range /// New range where the Begin key is the smallest bound and the End key is the largest bound of both ranges. /// If both range are disjoint, then the resulting range will also contain the keys in between. - public FdbKeyRange Merge(FdbKeyRange other) + public KeyRange Merge(KeyRange other) { Slice begin = m_begin.CompareTo(other.m_begin) <= 0 ? m_begin : other.m_begin; Slice end = m_end.CompareTo(other.m_end) >= 0 ? m_end : other.m_end; - return new FdbKeyRange(begin, end); + return new KeyRange(begin, end); } /// Checks whether the current and the specified range are intersecting (i.e: there exists at at least one key that belongs to both ranges) /// Range that is being checked for interection /// True if the other range intersects the current range. /// Note that ranges [0, 1) and [1, 2) do not intersect, since the end is exclusive by default - public bool Intersects(FdbKeyRange other) + public bool Intersects(KeyRange other) { int c = m_begin.CompareTo(other.m_begin); if (c == 0) @@ -220,7 +220,7 @@ public bool Intersects(FdbKeyRange other) /// /// /// Note that ranges [0, 1) and [1, 2) are not disjoint because, even though they do not intersect, they are both contiguous. - public bool Disjoint(FdbKeyRange other) + public bool Disjoint(KeyRange other) { int c = m_begin.CompareTo(other.m_begin); if (c == 0) diff --git a/FoundationDB.Client/FdbKeyRangeComparer.cs b/FoundationDB.Client/KeyRangeComparer.cs similarity index 81% rename from FoundationDB.Client/FdbKeyRangeComparer.cs rename to FoundationDB.Client/KeyRangeComparer.cs index 0fbe06ff7..94aaabdb9 100644 --- a/FoundationDB.Client/FdbKeyRangeComparer.cs +++ b/FoundationDB.Client/KeyRangeComparer.cs @@ -34,25 +34,25 @@ namespace FoundationDB.Client using Doxense.Diagnostics.Contracts; [DebuggerDisplay("Mode={m_mode}")] - public sealed class FdbKeyRangeComparer : IComparer, IEqualityComparer + public sealed class KeyRangeComparer : IComparer, IEqualityComparer { private const int BOTH = 0; private const int BEGIN = 1; private const int END = 2; - public static readonly FdbKeyRangeComparer Default = new FdbKeyRangeComparer(BOTH); - public static readonly FdbKeyRangeComparer Begin = new FdbKeyRangeComparer(BEGIN); - public static readonly FdbKeyRangeComparer End = new FdbKeyRangeComparer(END); + public static readonly KeyRangeComparer Default = new KeyRangeComparer(BOTH); + public static readonly KeyRangeComparer Begin = new KeyRangeComparer(BEGIN); + public static readonly KeyRangeComparer End = new KeyRangeComparer(END); private readonly int m_mode; - private FdbKeyRangeComparer(int mode) + private KeyRangeComparer(int mode) { Contract.Requires(mode >= BOTH && mode <= END); m_mode = mode; } - public int Compare(FdbKeyRange x, FdbKeyRange y) + public int Compare(KeyRange x, KeyRange y) { switch (m_mode) { @@ -62,7 +62,7 @@ public int Compare(FdbKeyRange x, FdbKeyRange y) } } - public bool Equals(FdbKeyRange x, FdbKeyRange y) + public bool Equals(KeyRange x, KeyRange y) { switch(m_mode) { @@ -72,7 +72,7 @@ public bool Equals(FdbKeyRange x, FdbKeyRange y) } } - public int GetHashCode(FdbKeyRange obj) + public int GetHashCode(KeyRange obj) { switch(m_mode) { diff --git a/FoundationDB.Client/FdbKeySelector.cs b/FoundationDB.Client/KeySelector.cs similarity index 77% rename from FoundationDB.Client/FdbKeySelector.cs rename to FoundationDB.Client/KeySelector.cs index 4c260c918..36f581cd9 100644 --- a/FoundationDB.Client/FdbKeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -35,10 +35,10 @@ namespace FoundationDB.Client /// Defines a selector for a key in the database [DebuggerDisplay("{ToString()}")] - public struct FdbKeySelector : IEquatable + public struct KeySelector : IEquatable { /// Empty key selector - public static readonly FdbKeySelector None = default(FdbKeySelector); + public static readonly KeySelector None = default(KeySelector); /// Key of the selector public Slice Key { get { return m_key; } } @@ -51,7 +51,7 @@ public struct FdbKeySelector : IEquatable public readonly int Offset; /// Creates a new selector - public FdbKeySelector(Slice key, bool orEqual, int offset) + public KeySelector(Slice key, bool orEqual, int offset) { m_key = key; this.OrEqual = orEqual; @@ -59,7 +59,7 @@ public FdbKeySelector(Slice key, bool orEqual, int offset) } /// Creates a new selector - public FdbKeySelector(IFdbKey key, bool orEqual, int offset) + public KeySelector(IFdbKey key, bool orEqual, int offset) { if (key == null) throw new ArgumentNullException("key"); m_key = key.ToFoundationDbKey(); @@ -93,20 +93,20 @@ public string PrettyPrint(FdbKey.PrettyPrintMode mode) return sb.ToString(); } - /// Converts the value of the current object into its equivalent string representation + /// Converts the value of the current object into its equivalent string representation public override string ToString() { return PrettyPrint(FdbKey.PrettyPrintMode.Single); } - public bool Equals(FdbKeySelector other) + public bool Equals(KeySelector other) { return this.Offset == other.Offset && this.OrEqual == other.OrEqual && m_key.Equals(other.m_key); } public override bool Equals(object obj) { - return obj is FdbKeySelector && Equals((FdbKeySelector)obj); + return obj is KeySelector && Equals((KeySelector)obj); } public override int GetHashCode() @@ -116,35 +116,35 @@ public override int GetHashCode() } /// Creates a key selector that will select the last key that is less than - public static FdbKeySelector LastLessThan(Slice key) + public static KeySelector LastLessThan(Slice key) { // #define FDB_KEYSEL_LAST_LESS_THAN(k, l) k, l, 0, 0 - return new FdbKeySelector(key, false, 0); + return new KeySelector(key, false, 0); } /// Creates a key selector that will select the last key that is less than or equal to - public static FdbKeySelector LastLessOrEqual(Slice key) + public static KeySelector LastLessOrEqual(Slice key) { // #define FDB_KEYSEL_LAST_LESS_OR_EQUAL(k, l) k, l, 1, 0 - return new FdbKeySelector(key, true, 0); + return new KeySelector(key, true, 0); } /// Creates a key selector that will select the first key that is greater than - public static FdbKeySelector FirstGreaterThan(Slice key) + public static KeySelector FirstGreaterThan(Slice key) { // #define FDB_KEYSEL_FIRST_GREATER_THAN(k, l) k, l, 1, 1 - return new FdbKeySelector(key, true, 1); + return new KeySelector(key, true, 1); } /// Creates a key selector that will select the first key that is greater than or equal to - public static FdbKeySelector FirstGreaterOrEqual(Slice key) + public static KeySelector FirstGreaterOrEqual(Slice key) { // #define FDB_KEYSEL_FIRST_GREATER_OR_EQUAL(k, l) k, l, 0, 1 - return new FdbKeySelector(key, false, 1); + return new KeySelector(key, false, 1); } /// Creates a key selector that will select the last key that is less than - public static FdbKeySelector LastLessThan(TKey key) + public static KeySelector LastLessThan(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -152,7 +152,7 @@ public static FdbKeySelector LastLessThan(TKey key) } /// Creates a key selector that will select the last key that is less than or equal to - public static FdbKeySelector LastLessOrEqual(TKey key) + public static KeySelector LastLessOrEqual(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -160,7 +160,7 @@ public static FdbKeySelector LastLessOrEqual(TKey key) } /// Creates a key selector that will select the first key that is greater than - public static FdbKeySelector FirstGreaterThan(TKey key) + public static KeySelector FirstGreaterThan(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -168,7 +168,7 @@ public static FdbKeySelector FirstGreaterThan(TKey key) } /// Creates a key selector that will select the first key that is greater than or equal to - public static FdbKeySelector FirstGreaterOrEqual(TKey key) + public static KeySelector FirstGreaterOrEqual(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); @@ -179,26 +179,26 @@ public static FdbKeySelector FirstGreaterOrEqual(TKey key) /// ex: fGE('abc') /// ex: 7 /// fGE('abc')+7 - public static FdbKeySelector operator +(FdbKeySelector selector, int offset) + public static KeySelector operator +(KeySelector selector, int offset) { - return new FdbKeySelector(selector.m_key, selector.OrEqual, selector.Offset + offset); + return new KeySelector(selector.m_key, selector.OrEqual, selector.Offset + offset); } /// Substract a value to the selector's offset /// ex: fGE('abc') /// ex: 7 /// fGE('abc')-7 - public static FdbKeySelector operator -(FdbKeySelector selector, int offset) + public static KeySelector operator -(KeySelector selector, int offset) { - return new FdbKeySelector(selector.m_key, selector.OrEqual, selector.Offset - offset); + return new KeySelector(selector.m_key, selector.OrEqual, selector.Offset - offset); } - public static bool operator ==(FdbKeySelector left, FdbKeySelector right) + public static bool operator ==(KeySelector left, KeySelector right) { return left.Equals(right); } - public static bool operator !=(FdbKeySelector left, FdbKeySelector right) + public static bool operator !=(KeySelector left, KeySelector right) { return !left.Equals(right); } diff --git a/FoundationDB.Client/FdbKeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs similarity index 70% rename from FoundationDB.Client/FdbKeySelectorPair.cs rename to FoundationDB.Client/KeySelectorPair.cs index 0b9b3bb78..40651d844 100644 --- a/FoundationDB.Client/FdbKeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -33,77 +33,77 @@ namespace FoundationDB.Client /// Represents of pair of key selectors that range 'GetKey(Begin) <= key < GetKey(End)' [DebuggerDisplay("[ToString()]")] - public struct FdbKeySelectorPair + public struct KeySelectorPair { /// Start of the range - public FdbKeySelector Begin { get { return m_begin; } } - private FdbKeySelector m_begin; //PERF: readonly struct + public KeySelector Begin { get { return m_begin; } } + private KeySelector m_begin; //PERF: readonly struct /// End of the range - public FdbKeySelector End { get { return m_end; } } - private FdbKeySelector m_end; //PERF: readonly struct + public KeySelector End { get { return m_end; } } + private KeySelector m_end; //PERF: readonly struct /// Create a new pair of key selectors /// Selector for key from which to start iterating /// Selector for key where to stop iterating - public FdbKeySelectorPair(FdbKeySelector beginInclusive, FdbKeySelector endExclusive) + public KeySelectorPair(KeySelector beginInclusive, KeySelector endExclusive) { m_begin = beginInclusive; m_end = endExclusive; } /// Factory method for a pair of key selectors - public static FdbKeySelectorPair Create(FdbKeySelector beginInclusive, FdbKeySelector endExclusive) + public static KeySelectorPair Create(KeySelector beginInclusive, KeySelector endExclusive) { - return new FdbKeySelectorPair( + return new KeySelectorPair( beginInclusive, endExclusive ); } /// Create a new pair of key selectors using FIRST_GREATER_OR_EQUAL on both keys - public static FdbKeySelectorPair Create(Slice begin, Slice end) + public static KeySelectorPair Create(Slice begin, Slice end) { - return new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterOrEqual(begin), - FdbKeySelector.FirstGreaterOrEqual(end) + return new KeySelectorPair( + KeySelector.FirstGreaterOrEqual(begin), + KeySelector.FirstGreaterOrEqual(end) ); } /// Create a new pair of key selectors using FIRST_GREATER_OR_EQUAL on both keys - public static FdbKeySelectorPair Create(TKey begin, TKey end) + public static KeySelectorPair Create(TKey begin, TKey end) where TKey : IFdbKey { if (begin == null) throw new ArgumentNullException("begin"); if (end == null) throw new ArgumentNullException("end"); - return new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterOrEqual(begin.ToFoundationDbKey()), - FdbKeySelector.FirstGreaterOrEqual(end.ToFoundationDbKey()) + return new KeySelectorPair( + KeySelector.FirstGreaterOrEqual(begin.ToFoundationDbKey()), + KeySelector.FirstGreaterOrEqual(end.ToFoundationDbKey()) ); } /// Create a new pair of key selectors using FIRST_GREATER_OR_EQUAL on both keys - public static FdbKeySelectorPair Create(FdbKeyRange range) + public static KeySelectorPair Create(KeyRange range) { - return new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterOrEqual(range.Begin), - FdbKeySelector.FirstGreaterOrEqual(range.End) + return new KeySelectorPair( + KeySelector.FirstGreaterOrEqual(range.Begin), + KeySelector.FirstGreaterOrEqual(range.End) ); } /// Create a new pair of key selectors that will select all the keys that start with the specified prefix - public static FdbKeySelectorPair StartsWith(Slice prefix) + public static KeySelectorPair StartsWith(Slice prefix) { - var range = FdbKeyRange.StartsWith(prefix); + var range = KeyRange.StartsWith(prefix); - return new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterOrEqual(range.Begin), - FdbKeySelector.FirstGreaterOrEqual(range.End) + return new KeySelectorPair( + KeySelector.FirstGreaterOrEqual(range.Begin), + KeySelector.FirstGreaterOrEqual(range.End) ); } /// Create a new pair of key selectors that will select all the keys that start with the specified prefix - public static FdbKeySelectorPair StartsWith(TKey prefix) + public static KeySelectorPair StartsWith(TKey prefix) where TKey : IFdbKey { if (prefix == null) throw new ArgumentNullException("prefix"); diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 5c0d29192..ee0144da0 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -642,7 +642,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly // ensure that there is no data already present under this prefix if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Ensure that there is no data already present under prefix {0}", prefix); - if (await trans.GetRange(FdbKeyRange.StartsWith(prefix)).AnyAsync().ConfigureAwait(false)) + if (await trans.GetRange(KeyRange.StartsWith(prefix)).AnyAsync().ConfigureAwait(false)) { throw new InvalidOperationException(String.Format("The database has keys stored at the prefix chosen by the automatic prefix allocator: {0}", prefix.ToAsciiOrHexaString())); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs index b17d9cf5e..d43e3015e 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs @@ -1079,14 +1079,14 @@ public static bool Equivalent(IFdbTuple left, IFdbTuple right) /// Subspace binary prefix (that will be excluded from the range) /// Range including all possible tuples starting with the specified prefix. /// FdbTuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) - public static FdbKeyRange ToRange(Slice prefix) + public static KeyRange ToRange(Slice prefix) { if (prefix.IsNull) throw new ArgumentNullException("prefix"); //note: there is no guarantee that prefix is a valid packed tuple (could be any exotic binary prefix) // prefix => [ prefix."\0", prefix."\xFF" ) - return new FdbKeyRange( + return new KeyRange( prefix + FdbKey.MinValue, prefix + FdbKey.MaxValue ); @@ -1094,14 +1094,14 @@ public static FdbKeyRange ToRange(Slice prefix) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. - public static FdbKeyRange ToRange([NotNull] IFdbTuple tuple) + public static KeyRange ToRange([NotNull] IFdbTuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); // tuple => [ packed."\0", packed."\xFF" ) var packed = tuple.ToSlice(); - return new FdbKeyRange( + return new KeyRange( packed + FdbKey.MinValue, packed + FdbKey.MaxValue ); @@ -1110,14 +1110,14 @@ public static FdbKeyRange ToRange([NotNull] IFdbTuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(Slice.FromInt32(42), FdbTuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. - public static FdbKeyRange ToRange(Slice prefix, [NotNull] IFdbTuple tuple) + public static KeyRange ToRange(Slice prefix, [NotNull] IFdbTuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = prefix + tuple.ToSlice(); - return new FdbKeyRange( + return new KeyRange( packed + FdbKey.MinValue, packed + FdbKey.MaxValue ); diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs index 71d965e6e..67e5352d6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs @@ -127,7 +127,7 @@ public static IFdbTuple Append([NotNull] this IFdbTuple tuple, T /// Creates a key range containing all children of this tuple, from tuple.pack()+'\0' to tuple.pack()+'\xFF' /// Tuple that is the suffix of all keys /// Range of all keys suffixed by the tuple. The tuple itself will not be included - public static FdbKeyRange ToRange([NotNull] this IFdbTuple tuple) + public static KeyRange ToRange([NotNull] this IFdbTuple tuple) { return ToRange(tuple, false); } @@ -136,7 +136,7 @@ public static FdbKeyRange ToRange([NotNull] this IFdbTuple tuple) /// Tuple that is the prefix of all keys /// If true, the tuple key itself is included, if false only the children keys are included /// Range of all keys suffixed by the tuple. The tuple itself will be included if is true - public static FdbKeyRange ToRange([NotNull] this IFdbTuple tuple, bool includePrefix) + public static KeyRange ToRange([NotNull] this IFdbTuple tuple, bool includePrefix) { if (tuple == null) throw new ArgumentNullException("tuple"); @@ -154,7 +154,7 @@ public static FdbKeyRange ToRange([NotNull] this IFdbTuple tuple, bool includePr writer.Output.WriteByte(0xFF); int p1 = writer.Output.Position; - return new FdbKeyRange( + return new KeyRange( new Slice(writer.Output.Buffer, 0, p0), new Slice(writer.Output.Buffer, p0, p1 - p0) ); @@ -303,11 +303,11 @@ public static Slice Increment([NotNull] this IFdbTuple tuple) } /// Returns a Key Selector pair that defines the range of all items contained under this tuple - public static FdbKeySelectorPair ToSelectorPair([NotNull] this IFdbTuple tuple) + public static KeySelectorPair ToSelectorPair([NotNull] this IFdbTuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); - return FdbKeySelectorPair.StartsWith(tuple.ToSlice()); + return KeySelectorPair.StartsWith(tuple.ToSlice()); } /// Verify that this tuple has the expected size diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs index 7a29d9c96..1ef2ab739 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs @@ -44,7 +44,7 @@ public IFdbKeyEncoding Encoding get { return TypeSystem.Tuples; } } - public FdbKeyRange ToRange(Slice prefix) + public KeyRange ToRange(Slice prefix) { return FdbTuple.ToRange(prefix); } @@ -180,47 +180,47 @@ public FdbTuple DecodeKey(Slice packed) return FdbTuple.DecodeKey(packed); } - public FdbKeyRange ToRange(Slice prefix, IFdbTuple items) + public KeyRange ToRange(Slice prefix, IFdbTuple items) { return FdbTuple.ToRange(prefix, items); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1) + public KeyRange ToKeyRange(Slice prefix, T1 item1) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item2)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7)); } - public FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 8aabe789e..043628887 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -624,7 +624,7 @@ public static FutureHandle TransactionGet(TransactionHandle transaction, Slice k } } - public static FutureHandle TransactionGetRange(TransactionHandle transaction, FdbKeySelector begin, FdbKeySelector end, int limit, int targetBytes, FdbStreamingMode mode, int iteration, bool snapshot, bool reverse) + public static FutureHandle TransactionGetRange(TransactionHandle transaction, KeySelector begin, KeySelector 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) @@ -642,7 +642,7 @@ public static FutureHandle TransactionGetRange(TransactionHandle transaction, Fd } } - public static FutureHandle TransactionGetKey(TransactionHandle transaction, FdbKeySelector selector, bool snapshot) + public static FutureHandle TransactionGetKey(TransactionHandle transaction, KeySelector selector, bool snapshot) { if (selector.Key.IsNull) throw new ArgumentException("Key cannot be null", "selector"); diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index 65db78649..49f74f24c 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -228,7 +228,7 @@ private static KeyValuePair[] GetKeyValueArrayResult(FutureHandle /// Asynchronously fetch a new page of results /// True if Chunk contains a new page of results. False if all results have been read. - public Task GetRangeAsync(FdbKeySelector begin, FdbKeySelector end, FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken) + public Task GetRangeAsync(KeySelector begin, KeySelector end, FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken) { Contract.Requires(options != null); @@ -262,7 +262,7 @@ private static Slice GetKeyResult(FutureHandle h) return result; } - public Task GetKeyAsync(FdbKeySelector selector, bool snapshot, CancellationToken cancellationToken) + public Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken cancellationToken) { var future = FdbNative.TransactionGetKey(m_handle, selector, snapshot); return FdbFuture.CreateTaskFromHandle( @@ -272,7 +272,7 @@ public Task GetKeyAsync(FdbKeySelector selector, bool snapshot, Cancellat ); } - public Task GetKeysAsync(FdbKeySelector[] selectors, bool snapshot, CancellationToken cancellationToken) + public Task GetKeysAsync(KeySelector[] selectors, bool snapshot, CancellationToken cancellationToken) { Contract.Requires(selectors != null); diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index d42a4265a..56fe70713 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -28,10 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Directories; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; @@ -39,6 +35,10 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using SystemIO = System.IO; + using FoundationDB.Client.Utils; + using FoundationDB.Layers.Directories; + using FoundationDB.Linq; + using JetBrains.Annotations; public static partial class Fdb { diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs index cee84e75f..57db84a6b 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Client using System; using System.Diagnostics; using JetBrains.Annotations; + public class FdbDynamicSubspace : FdbSubspace, IFdbDynamicSubspace { /// Encoder for the keys of this subspace diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs index fbd3ef2c7..857b8181d 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs @@ -29,9 +29,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; - using Doxense.Diagnostics.Contracts; using System.Collections.Generic; using System.Linq; + using Doxense.Diagnostics.Contracts; using FoundationDB.Client.Utils; using FoundationDB.Layers.Tuples; using JetBrains.Annotations; @@ -109,21 +109,21 @@ public FdbDynamicSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] IDynami } /// Return a key range that encompass all the keys inside this subspace, according to the current key encoder - public FdbKeyRange ToRange() + public KeyRange ToRange() { return this.Encoder.ToRange(this.Subspace.Key); } /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder /// Tuple used as a prefix for the range - public FdbKeyRange ToRange([NotNull] IFdbTuple tuple) + public KeyRange ToRange([NotNull] IFdbTuple tuple) { return this.Encoder.ToRange(Pack(tuple)); } /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder /// Convertible item used as a prefix for the range - public FdbKeyRange ToRange([NotNull] ITupleFormattable item) + public KeyRange ToRange([NotNull] ITupleFormattable item) { return this.Encoder.ToRange(Pack(item)); } diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs index 42a9b27a7..578eb57e7 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs @@ -76,7 +76,7 @@ public T Decode(Slice packed) return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } - public FdbKeyRange ToRange(T value) + public KeyRange ToRange(T value) { //REVIEW: which semantic for ToRange() should we use? return FdbTuple.ToRange(Encode(value)); diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs index 04585dc81..a0dacd538 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs @@ -76,7 +76,7 @@ public FdbTuple Decode(Slice packed) return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } - public FdbKeyRange ToRange(T1 value1, T2 value2) + public KeyRange ToRange(T1 value1, T2 value2) { //REVIEW: which semantic for ToRange() should we use? return FdbTuple.ToRange(Encode(value1, value2)); diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs index ccaf815f5..d855b4855 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs @@ -71,7 +71,7 @@ public FdbTuple Decode(Slice packed) return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } - public FdbKeyRange ToRange(T1 value1, T2 value2, T3 value3) + public KeyRange ToRange(T1 value1, T2 value2, T3 value3) { //REVIEW: which semantic for ToRange() should we use? return FdbTuple.ToRange(Encode(value1, value2, value3)); diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs index b5ff8db2b..f3ede9f47 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs @@ -77,7 +77,7 @@ public FdbTuple Decode(Slice packed) return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } - public FdbKeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) + public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) { //REVIEW: which semantic for ToRange() should we use? return FdbTuple.ToRange(Encode(value1, value2, value3, value4)); diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/FdbSubspace.cs index 47391d146..b09e0596f 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspace.cs @@ -353,21 +353,21 @@ protected virtual IFdbSubspace CreateChildren(Slice suffix) return new FdbSubspace(ConcatKey(suffix)); } - public FdbKeyRange ToRange() + public KeyRange ToRange() { return ToRange(Slice.Empty); } - public virtual FdbKeyRange ToRange(Slice suffix) + public virtual KeyRange ToRange(Slice suffix) { - return FdbKeyRange.StartsWith(ConcatKey(suffix)); + return KeyRange.StartsWith(ConcatKey(suffix)); } - public virtual FdbKeyRange ToRange(TKey key) + public virtual KeyRange ToRange(TKey key) where TKey : IFdbKey { if (key == null) throw new ArgumentNullException("key"); - return FdbKeyRange.StartsWith(ConcatKey(key.ToFoundationDbKey())); + return KeyRange.StartsWith(ConcatKey(key.ToFoundationDbKey())); } public IFdbSubspace this[Slice suffix] diff --git a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs index 948b744a1..d3b559b41 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs @@ -165,7 +165,7 @@ public static void ClearRange(this IFdbTransaction trans, [NotNull] IFdbSubspace Contract.Requires(trans != null && subspace != null); //BUGBUG: should we call subspace.ToRange() ? - trans.ClearRange(FdbKeyRange.StartsWith(subspace.ToFoundationDbKey())); + trans.ClearRange(KeyRange.StartsWith(subspace.ToFoundationDbKey())); } /// Clear the entire content of a subspace @@ -184,7 +184,7 @@ public static FdbRangeQuery> GetRangeStartsWith(this //REVIEW: should we remove this method? Contract.Requires(trans != null && subspace != null); - return trans.GetRange(FdbKeyRange.StartsWith(subspace.ToFoundationDbKey()), options); + return trans.GetRange(KeyRange.StartsWith(subspace.ToFoundationDbKey()), options); } /// Tests whether the specified starts with this Subspace's prefix, indicating that the Subspace logically contains . diff --git a/FoundationDB.Client/Subspaces/IFdbSubspace.cs b/FoundationDB.Client/Subspaces/IFdbSubspace.cs index 05a0f3b2c..34a6ada06 100644 --- a/FoundationDB.Client/Subspaces/IFdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbSubspace.cs @@ -43,18 +43,18 @@ public interface IFdbSubspace : IFdbKey /// Return a key range that contains all the keys in this subspace, including the prefix itself /// Return the range: Key <= x <= Increment(Key) [Pure] - FdbKeyRange ToRange(); + KeyRange ToRange(); /// Return a key range that contains all the keys under a suffix in this subspace /// Binary suffix that will be appended to the current prefix, before computing the range /// Return the range: (this.Key + suffix) <= x <= Increment(this.Key + suffix) [Pure] - FdbKeyRange ToRange(Slice suffix); + KeyRange ToRange(Slice suffix); /// Return a key range that contains all the keys under a serializable key in this subspace /// Return the range: (this.Key + key.ToFoundationDbKey()) <= x <= Increment(this.Key + key.ToFoundationDbKey()) [Pure] - FdbKeyRange ToRange([NotNull] TKey key) where TKey : IFdbKey; + KeyRange ToRange([NotNull] TKey key) where TKey : IFdbKey; /// Create a new subspace by adding a suffix to the key of the current subspace. /// Binary suffix that will be appended to the current prefix diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index b5921b49a..1a1c0ab7e 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -8,9 +8,9 @@ public abstract class DynamicKeyEncoderBase : IDynamicKeyEncoder public abstract IFdbKeyEncoding Encoding { get; } - public virtual FdbKeyRange ToRange(Slice prefix) + public virtual KeyRange ToRange(Slice prefix) { - return FdbKeyRange.StartsWith(prefix); + return KeyRange.StartsWith(prefix); } public abstract void PackKey(ref SliceWriter writer, IFdbTuple items); @@ -92,49 +92,49 @@ public virtual FdbTuple DecodeKey(Slice return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e) => FdbTuple.Create(a, b, c, d, e)); } - public virtual FdbKeyRange ToRange(Slice prefix, IFdbTuple items) + public virtual KeyRange ToRange(Slice prefix, IFdbTuple items) { var writer = new SliceWriter(prefix, 16); PackKey(ref writer, items); return ToRange(writer.ToSlice()); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1) { return ToRange(prefix, FdbTuple.Create(item1)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { return ToRange(prefix, FdbTuple.Create(item1, item2)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7)); } - public virtual FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs index 2825449cb..bd58015e2 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs @@ -44,7 +44,7 @@ public interface IDynamicKeyEncoder /// Optional binary prefix /// Key range which derives from the semantic of the current encoding /// For example, the Tuple encoding will produce ranges of the form "(Key + \x00) <= x < (Key + \xFF)", while a binary-based encoding would produce ranges of the form "Key <= x < Increment(Key)" - FdbKeyRange ToRange(Slice prefix = default(Slice)); + KeyRange ToRange(Slice prefix = default(Slice)); #region Encoding... @@ -218,13 +218,13 @@ public interface IDynamicKeyEncoder /// Return a key range using a tuple as a prefix /// Optional binary prefix that should be added before encoding the key /// Tuple of any size (0 to N) - FdbKeyRange ToRange(Slice prefix, IFdbTuple items); + KeyRange ToRange(Slice prefix, IFdbTuple items); /// Return a key range using a single element as a prefix /// Type of the element /// Optional binary prefix that should be added before encoding the key /// Element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1); + KeyRange ToKeyRange(Slice prefix, T1 item1); /// Return a key range using two elements as a prefix /// Type of the first element @@ -232,7 +232,7 @@ public interface IDynamicKeyEncoder /// Optional binary prefix that should be added before encoding the key /// First element to encode /// Second element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2); /// Return a key range using three elements as a prefix /// Type of the first element @@ -242,7 +242,7 @@ public interface IDynamicKeyEncoder /// First element to encode /// Second element to encode /// Third element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3); /// Return a key range using four elements as a prefix /// Type of the first element @@ -254,7 +254,7 @@ public interface IDynamicKeyEncoder /// Second element to encode /// Third element to encode /// Fourth element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4); /// Return a key range using five elements as a prefix /// Type of the first element @@ -268,7 +268,7 @@ public interface IDynamicKeyEncoder /// Third element to encode /// Fourth element to encode /// Fifth element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5); /// Return a key range using six elements as a prefix /// Type of the first element @@ -284,7 +284,7 @@ public interface IDynamicKeyEncoder /// Fourth element to encode /// Fifth element to encode /// Sixth element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6); /// Return a key range using seven elements as a prefix /// Type of the first element @@ -302,7 +302,7 @@ public interface IDynamicKeyEncoder /// Fifth element to encode /// Sixth element to encode /// Seventh element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7); /// Return a key range using eight elements as a prefix /// Type of the first element @@ -322,7 +322,7 @@ public interface IDynamicKeyEncoder /// Sixth element to encode /// Seventh element to encode /// Eighth element to encode - FdbKeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8); + KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8); //note: I will be billing $999.99 to anyone who wants up to T11 !!! :( diff --git a/FoundationDB.Client/Utils/SliceBuffer.cs b/FoundationDB.Client/Utils/SliceBuffer.cs index a6968a60a..8df2ebcfd 100644 --- a/FoundationDB.Client/Utils/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/SliceBuffer.cs @@ -105,10 +105,10 @@ public Slice[] GetPages() /// Copy a pair of keys into the buffer, and return a new identical pair /// Key range /// Equivalent pair of keys, that are backed by the buffer. - public FdbKeyRange InternRange(FdbKeyRange range) + public KeyRange InternRange(KeyRange range) { //TODO: if end is prefixed by begin, we could merge both keys (frequent when dealing with ranges on tuples that add \xFF - return new FdbKeyRange( + return new KeyRange( Intern(range.Begin, aligned: true), Intern(range.End, aligned: true) ); @@ -118,10 +118,10 @@ public FdbKeyRange InternRange(FdbKeyRange range) /// Begin key of the range /// End key of the range /// Equivalent pair of keys, that are backed by the buffer. - public FdbKeyRange InternRange(Slice begin, Slice end) + public KeyRange InternRange(Slice begin, Slice end) { //TODO: if end is prefixed by begin, we could merge both keys (frequent when dealing with ranges on tuples that add \xFF - return new FdbKeyRange( + return new KeyRange( Intern(begin, aligned: true), Intern(end, aligned: true) ); @@ -130,12 +130,12 @@ public FdbKeyRange InternRange(Slice begin, Slice end) /// Copy a key into the buffer, and return a new range containing only that key /// Key to copy to the buffer /// Range equivalent to [key, key + '\0') that is backed by the buffer. - public FdbKeyRange InternRangeFromKey(Slice key) + public KeyRange InternRangeFromKey(Slice key) { // Since the end key only adds \0 to the begin key, we can reuse the same bytes by making both overlap var tmp = Intern(key, FdbKey.MinValue, aligned: true); - return new FdbKeyRange( + return new KeyRange( tmp.Substring(0, key.Count), tmp ); @@ -144,9 +144,9 @@ public FdbKeyRange InternRangeFromKey(Slice key) /// Copy a key selector into the buffer, and return a new identical selector /// Key selector to copy to the buffer /// Equivalent key selector that is backed by the buffer. - public FdbKeySelector InternSelector(FdbKeySelector selector) + public KeySelector InternSelector(KeySelector selector) { - return new FdbKeySelector( + return new KeySelector( Intern(selector.Key, aligned: true), selector.OrEqual, selector.Offset @@ -156,14 +156,14 @@ public FdbKeySelector InternSelector(FdbKeySelector selector) /// Copy a pair of key selectors into the buffer, and return a new identical pair /// Pair of key selectors to copy to the buffer /// Equivalent pair of key selectors that is backed by the buffer. - public FdbKeySelectorPair InternSelectorPair(FdbKeySelectorPair pair) + public KeySelectorPair InternSelectorPair(KeySelectorPair pair) { var begin = Intern(pair.Begin.Key, default(Slice), aligned: true); var end = Intern(pair.End.Key, default(Slice), aligned: true); - return new FdbKeySelectorPair( - new FdbKeySelector(begin, pair.Begin.OrEqual, pair.Begin.Offset), - new FdbKeySelector(end, pair.End.OrEqual, pair.End.Offset) + return new KeySelectorPair( + new KeySelector(begin, pair.Begin.OrEqual, pair.Begin.Offset), + new KeySelector(end, pair.End.OrEqual, pair.End.Offset) ); } diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index b0fec26a5..3d45b7603 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -108,7 +108,7 @@ private async Task GetChunkAtAsync([NotNull] IFdbTransaction trans, long { Contract.Requires(trans != null && offset >= 0); - var chunkKey = await trans.GetKeyAsync(FdbKeySelector.LastLessOrEqual(DataKey(offset))).ConfigureAwait(false); + var chunkKey = await trans.GetKeyAsync(KeySelector.LastLessOrEqual(DataKey(offset))).ConfigureAwait(false); if (chunkKey.IsNull) { // nothing before (sparse) return default(Chunk); @@ -242,8 +242,8 @@ public async Task ReadAsync([NotNull] IFdbReadOnlyTransaction trans, long await trans .GetRange( - FdbKeySelector.LastLessOrEqual(DataKey(offset)), - FdbKeySelector.FirstGreaterOrEqual(DataKey(offset + n)) + KeySelector.LastLessOrEqual(DataKey(offset)), + KeySelector.FirstGreaterOrEqual(DataKey(offset + n)) ) .ForEachAsync((chunk) => { diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index 4739173a2..cce139c57 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -168,7 +168,7 @@ public IFdbAsyncEnumerable Get([NotNull] IFdbReadOnlyTransaction trans, { if (trans == null) throw new ArgumentNullException("trans"); - var range = FdbKeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); + var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); if (this.AllowNegativeValues) { return trans @@ -200,7 +200,7 @@ public Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey [NotNull] public IFdbAsyncEnumerable> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) { - var range = FdbKeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); + var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); var query = trans .GetRange(range) @@ -234,7 +234,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key) { if (trans == null) throw new ArgumentNullException("trans"); - trans.ClearRange(FdbKeyRange.StartsWith(this.Location.Partial.Keys.Encode(key))); + trans.ClearRange(KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key))); } /// Remove a value for a specific key diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 86b791899..502a62acf 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -281,7 +281,7 @@ private async Task GetNextIndexAsync([NotNull] IFdbReadOnlyTransaction tr, { var range = subspace.Keys.ToRange(); - var lastKey = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(range.End)).ConfigureAwait(false); + var lastKey = await tr.GetKeyAsync(KeySelector.LastLessThan(range.End)).ConfigureAwait(false); if (lastKey < range.Begin) { diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index fda4eba21..80943b34d 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -169,8 +169,8 @@ public async Task EraseAsync([NotNull] IFdbTransaction trans, Slice key) var lss = this.Subspace.Partition.ByKey(level); long lastCount = 0; var kcs = await trans.GetRange( - FdbKeySelector.FirstGreaterOrEqual(lss.Keys.Encode(rankKey)), - FdbKeySelector.FirstGreaterThan(lss.Keys.Encode(key)) + KeySelector.FirstGreaterOrEqual(lss.Keys.Encode(rankKey)), + KeySelector.FirstGreaterThan(lss.Keys.Encode(key)) ).ToListAsync().ConfigureAwait(false); foreach (var kc in kcs) { @@ -278,12 +278,12 @@ private async Task GetPreviousNodeAsync(IFdbTransaction trans, int level, var k = this.Subspace.Keys.Encode(level, key); //Console.WriteLine(k); //Console.WriteLine("GetPreviousNode(" + level + ", " + key + ")"); - //Console.WriteLine(FdbKeySelector.LastLessThan(k) + " <= x < " + FdbKeySelector.FirstGreaterOrEqual(k)); + //Console.WriteLine(KeySelector.LastLessThan(k) + " <= x < " + KeySelector.FirstGreaterOrEqual(k)); var kv = await trans .Snapshot .GetRange( - FdbKeySelector.LastLessThan(k), - FdbKeySelector.FirstGreaterOrEqual(k) + KeySelector.LastLessThan(k), + KeySelector.FirstGreaterOrEqual(k) ) .FirstAsync() .ConfigureAwait(false); diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index f03e3a70c..ca9c4759e 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -289,7 +289,7 @@ private async Task ComputeSizeAsync(IFdbReadOnlyTransaction tr) var keyRange = this.Subspace.Keys.ToRange(); - var lastKey = await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(keyRange.End)).ConfigureAwait(false); + var lastKey = await tr.GetKeyAsync(KeySelector.LastLessOrEqual(keyRange.End)).ConfigureAwait(false); if (lastKey < keyRange.Begin) { diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index b6276d998..359bb42d4 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -151,7 +151,7 @@ public FdbRangeQuery Lookup(IFdbReadOnlyTransaction trans, TValue value, bo var prefix = this.Location.Partial.Keys.Encode(value); return trans - .GetRange(FdbKeyRange.StartsWith(prefix), new FdbRangeOptions { Reverse = reverse }) + .GetRange(KeyRange.StartsWith(prefix), new FdbRangeOptions { Reverse = reverse }) .Select((kvp) => this.Location.Keys.Decode(kvp.Key).Item2); } @@ -161,9 +161,9 @@ public FdbRangeQuery LookupGreaterThan([NotNull] IFdbReadOnlyTransaction tr var prefix = this.Location.Partial.Keys.Encode(value); if (!orEqual) prefix = FdbKey.Increment(prefix); - var space = new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterThan(prefix), - FdbKeySelector.FirstGreaterOrEqual(this.Location.ToRange().End) + var space = new KeySelectorPair( + KeySelector.FirstGreaterThan(prefix), + KeySelector.FirstGreaterOrEqual(this.Location.ToRange().End) ); return trans @@ -177,9 +177,9 @@ public FdbRangeQuery LookupLessThan([NotNull] IFdbReadOnlyTransaction trans var prefix = this.Location.Partial.Keys.Encode(value); if (orEqual) prefix = FdbKey.Increment(prefix); - var space = new FdbKeySelectorPair( - FdbKeySelector.FirstGreaterOrEqual(this.Location.ToRange().Begin), - FdbKeySelector.FirstGreaterThan(prefix) + var space = new KeySelectorPair( + KeySelector.FirstGreaterOrEqual(this.Location.ToRange().Begin), + KeySelector.FirstGreaterThan(prefix) ); return trans diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index 76611e893..bc611bffd 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -68,7 +68,7 @@ protected virtual Task> LoadPartsAsync(IFdbReadOnlyTransaction trans var key = this.Location.Partial.Keys.Encode(id); return trans - .GetRange(FdbKeyRange.StartsWith(key)) //TODO: options ? + .GetRange(KeyRange.StartsWith(key)) //TODO: options ? .Select(kvp => kvp.Value) .ToListAsync(); } @@ -109,7 +109,7 @@ public void Insert(IFdbTransaction trans, TDocument document) var key = this.Location.Partial.Keys.Encode(id); // clear previous value - trans.ClearRange(FdbKeyRange.StartsWith(key)); + trans.ClearRange(KeyRange.StartsWith(key)); int remaining = packed.Count; if (remaining <= this.ChunkSize) @@ -172,7 +172,7 @@ public void Delete(IFdbTransaction trans, TId id) if (id == null) throw new ArgumentNullException("id"); var key = this.Location.Partial.Keys.Encode(id); - trans.ClearRange(FdbKeyRange.StartsWith(key)); + trans.ClearRange(KeyRange.StartsWith(key)); } @@ -186,7 +186,7 @@ public void DeleteMultiple(IFdbTransaction trans, IEnumerable ids) foreach (var key in this.Location.Partial.Keys.Encode(ids)) { - trans.ClearRange(FdbKeyRange.StartsWith(key)); + trans.ClearRange(KeyRange.StartsWith(key)); } } diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index e8ca59e12..bc4519b0f 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -107,7 +107,7 @@ public async Task> GetAsync([NotNull] IFdbReadOnlyTra var results = new Dictionary(StringComparer.OrdinalIgnoreCase); await trans - .GetRange(FdbKeyRange.StartsWith(prefix)) + .GetRange(KeyRange.StartsWith(prefix)) .ForEachAsync((kvp) => { string field = this.Subspace.Keys.DecodeLast(kvp.Key); @@ -193,7 +193,7 @@ public void Delete(IFdbTransaction trans, IFdbTuple id) if (id == null) throw new ArgumentNullException("id"); // remove all fields of the hash - trans.ClearRange(FdbKeyRange.StartsWith(GetKey(id))); + trans.ClearRange(KeyRange.StartsWith(GetKey(id))); } /// Remove one or more fields of an hashset @@ -233,7 +233,7 @@ public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) var results = new Dictionary(StringComparer.OrdinalIgnoreCase); return trans - .GetRange(FdbKeyRange.StartsWith(prefix)) + .GetRange(KeyRange.StartsWith(prefix)) .Select((kvp) => ParseFieldKey(FdbTuple.Unpack(kvp.Key))) .ToListAsync(cancellationToken); } diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index a9c44da08..fad3168b6 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -162,7 +162,7 @@ private async Task PushQueueAsync(IFdbTransaction tr, IFdbDynamicSubspace queue, // get the current size of the queue var range = queue.Keys.ToRange(); - var lastKey = await tr.Snapshot.GetKeyAsync(FdbKeySelector.LastLessThan(range.End)).ConfigureAwait(false); + var lastKey = await tr.Snapshot.GetKeyAsync(KeySelector.LastLessThan(range.End)).ConfigureAwait(false); int count = lastKey < range.Begin ? 0 : queue.Keys.DecodeFirst(lastKey) + 1; // set the value @@ -188,7 +188,7 @@ private void ClearTask(IFdbTransaction tr, Slice taskId) tr.Annotate("Deleting task {0}", taskId.ToAsciiOrHexaString()); // clear all metadata about the task - tr.ClearRange(FdbKeyRange.StartsWith(this.TaskStore.Keys.Encode(taskId))); + tr.ClearRange(KeyRange.StartsWith(this.TaskStore.Keys.Encode(taskId))); // decrement pending number of tasks this.Counters.Decrement(tr, COUNTER_PENDING_TASKS); } diff --git a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs index 41ea9f191..3b88cd951 100644 --- a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs +++ b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs @@ -138,7 +138,7 @@ private async Task RunAsync(IFdbDatabase db, IFdbDynamicSubspace location, Cance using (var tr = db.BeginTransaction(ct)) { await tr.Snapshot - .GetRange(FdbKeyRange.StartsWith(location.Key)) + .GetRange(KeyRange.StartsWith(location.Key)) .ForEachAsync((kvp) => { Console.WriteLine(" - " + location.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index a2bec47b3..0cc0abc84 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -65,16 +65,16 @@ public static FdbQueryAsyncEnumerableExpression Sequence([NotNull] IFdbAsy /// Execute a Range read from the database, and return all the keys and values [NotNull] - public static FdbQueryRangeExpression Range(FdbKeySelectorPair range, FdbRangeOptions options = null) + public static FdbQueryRangeExpression Range(KeySelectorPair range, FdbRangeOptions options = null) { return new FdbQueryRangeExpression(range, options); } /// Execute a Range read from the database, and return all the keys and values [NotNull] - public static FdbQueryRangeExpression Range(FdbKeySelector start, FdbKeySelector stop, FdbRangeOptions options = null) + public static FdbQueryRangeExpression Range(KeySelector start, KeySelector stop, FdbRangeOptions options = null) { - return Range(new FdbKeySelectorPair(start, stop), options); + return Range(new KeySelectorPair(start, stop), options); } /// Execute a Range read from the database, and return all the keys and values @@ -82,7 +82,7 @@ public static FdbQueryRangeExpression Range(FdbKeySelector start, FdbKeySelector public static FdbQueryRangeExpression RangeStartsWith(Slice prefix, FdbRangeOptions options = null) { // starts_with('A') means ['A', B') - return Range(FdbKeySelectorPair.StartsWith(prefix), options); + return Range(KeySelectorPair.StartsWith(prefix), options); } /// Execute a Range read from the database, and return all the keys and values diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs index 221dbacaa..90cf30a66 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs @@ -40,14 +40,14 @@ namespace FoundationDB.Linq.Expressions public class FdbQueryRangeExpression : FdbQuerySequenceExpression> { - internal FdbQueryRangeExpression(FdbKeySelectorPair range, FdbRangeOptions options) + internal FdbQueryRangeExpression(KeySelectorPair range, FdbRangeOptions options) { this.Range = range; this.Options = options; } /// Returns the pair of key selectors for this range query - public FdbKeySelectorPair Range { get; private set; } + public KeySelectorPair Range { get; private set; } /// Returns the options for this range query public FdbRangeOptions Options { get; private set; } @@ -73,10 +73,10 @@ public override Expression>>>( + var body = FdbExpressionHelpers.RewriteCall>>>( (trans, range, options) => trans.GetRange(range, options), prmTrans, - Expression.Constant(this.Range, typeof(FdbKeySelectorPair)), + Expression.Constant(this.Range, typeof(KeySelectorPair)), Expression.Constant(this.Options, typeof(FdbRangeOptions)) ); diff --git a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs index 6bc03a809..1319149bd 100644 --- a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs +++ b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs @@ -69,7 +69,7 @@ public static IFdbDatabaseQueryable Query(this IFdbDatabase db) /// Source database query /// Pair of key selectors /// Query that will return the keys from the specified - public static IFdbAsyncSequenceQueryable> Range(this IFdbDatabaseQueryable query, FdbKeySelectorPair range) + public static IFdbAsyncSequenceQueryable> Range(this IFdbDatabaseQueryable query, KeySelectorPair range) { if (query == null) throw new ArgumentNullException("query"); diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index 4b237ad70..7d51a8cba 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -46,7 +46,7 @@ protected Slice AttendsKey(string s, string c) return this.Subspace.Keys.Encode("attends", s, c); } - protected FdbKeyRange AttendsKeys(string s) + protected KeyRange AttendsKeys(string s) { return this.Subspace.Keys.ToRange(FdbTuple.Create("attends", s)); } diff --git a/FoundationDB.Samples/Utils/Sampler.cs b/FoundationDB.Samples/Utils/Sampler.cs index 12facd22c..bedfa0d12 100644 --- a/FoundationDB.Samples/Utils/Sampler.cs +++ b/FoundationDB.Samples/Utils/Sampler.cs @@ -47,7 +47,7 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) Console.WriteLine("# Detecting cluster topology..."); var servers = await db.QueryAsync(tr => tr .WithReadAccessToSystemKeys() - .GetRange(FdbKeyRange.StartsWith(Fdb.System.ServerList)) + .GetRange(KeyRange.StartsWith(Fdb.System.ServerList)) .Select(kvp => new { Node = kvp.Value.Substring(8, 16).ToHexaString(), @@ -73,7 +73,7 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) if (sz > 500) sz = 500; //SAFETY if (sz < 50) sz = Math.Max(sz, Math.Min(50, ranges.Count)); - var samples = new List(); + var samples = new List(); for (int i = 0; i < sz; i++) { int p = rnd.Next(ranges.Count); @@ -112,8 +112,8 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) long count = 0; int iter = 0; - var beginSelector = FdbKeySelector.FirstGreaterOrEqual(range.Begin); - var endSelector = FdbKeySelector.FirstGreaterOrEqual(range.End); + var beginSelector = KeySelector.FirstGreaterOrEqual(range.Begin); + var endSelector = KeySelector.FirstGreaterOrEqual(range.End); while (true) { FdbRangeChunk data = default(FdbRangeChunk); @@ -151,7 +151,7 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) if (!data.HasMore) break; - beginSelector = FdbKeySelector.FirstGreaterThan(data.Last.Key); + beginSelector = KeySelector.FirstGreaterThan(data.Last.Key); ++iter; } diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index 7e3eb6559..3a69eefff 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -730,7 +730,7 @@ private static async Task BenchMergeSortAsync(IFdbDatabase db, int N, int K, int { var mergesort = tr .MergeSort( - sources.Select(source => FdbKeySelectorPair.StartsWith(location.Keys.Encode(source))), + sources.Select(source => KeySelectorPair.StartsWith(location.Keys.Encode(source))), (kvp) => location.Keys.DecodeLast(kvp.Key) ) .Take(B) diff --git a/FoundationDB.Tests/ExoticTestCases.cs b/FoundationDB.Tests/ExoticTestCases.cs index 73d54be21..9ea5fcc41 100644 --- a/FoundationDB.Tests/ExoticTestCases.cs +++ b/FoundationDB.Tests/ExoticTestCases.cs @@ -394,8 +394,8 @@ await db.WriteAsync((tr) => tr.ClearRange(subspace.Keys.Encode("K0050"), subspace.Keys.Encode("K0060")); var chunk = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000")), - FdbKeySelector.LastLessOrEqual(subspace.Keys.Encode("K9999")), + KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000")), + KeySelector.LastLessOrEqual(subspace.Keys.Encode("K9999")), new FdbRangeOptions { Mode = FdbStreamingMode.WantAll, Reverse = true } ); @@ -432,14 +432,14 @@ await db.WriteAsync((tr) => tr.Set(subspace.Keys.Encode("K0021"), Slice.Empty); tr.Set(subspace.Keys.Encode("K0042"), Slice.Empty); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0005"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0010"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0015"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0022"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0049"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0050"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0055"))); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0061"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0005"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0010"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0015"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0022"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0049"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0050"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0055"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0061"))); //no commit } @@ -447,13 +447,13 @@ await db.WriteAsync((tr) => using (var tr = db.BeginTransaction(this.Cancellation)) { //tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000"))); // equal=false, offset=1 - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(subspace.Keys.Encode("K0011"))); // equal=true, offset=1 - await tr.GetKeyAsync(FdbKeySelector.LastLessOrEqual(subspace.Keys.Encode("K0022"))); // equal=true, offset=0 - await tr.GetKeyAsync(FdbKeySelector.LastLessThan(subspace.Keys.Encode("K0033"))); // equal=false, offset=0 + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0000"))); // equal=false, offset=1 + await tr.GetKeyAsync(KeySelector.FirstGreaterThan(subspace.Keys.Encode("K0011"))); // equal=true, offset=1 + await tr.GetKeyAsync(KeySelector.LastLessOrEqual(subspace.Keys.Encode("K0022"))); // equal=true, offset=0 + await tr.GetKeyAsync(KeySelector.LastLessThan(subspace.Keys.Encode("K0033"))); // equal=false, offset=0 - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")) + 1000); // equal=false, offset=7 ? - await tr.GetKeyAsync(FdbKeySelector.LastLessThan(subspace.Keys.Encode("K0050")) + 1000); // equal=false, offset=6 ? + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")) + 1000); // equal=false, offset=7 ? + await tr.GetKeyAsync(KeySelector.LastLessThan(subspace.Keys.Encode("K0050")) + 1000); // equal=false, offset=6 ? } } @@ -482,7 +482,7 @@ public async void Test_Case_12() using (var tr = db.BeginTransaction(this.Cancellation)) { tr.SetOption(FdbTransactionOption.ReadYourWritesDisable); - await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("KGETKEY"))); + await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("KGETKEY"))); } using (var tr = db.BeginTransaction(this.Cancellation)) @@ -528,8 +528,8 @@ await db.WriteAsync((tr) => tr.Set(subspace.Keys.Encode("KZZZ"), Slice.FromString("V999")); var r = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0B")), - FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0G")) + KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0B")), + KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0G")) ); await tr.CommitAsync(); @@ -560,8 +560,8 @@ await db.WriteAsync((tr) => tr.ClearRange(subspace.Keys.Encode("K0042"), Slice.FromString("K0069")); var r = await tr.GetRangeAsync( - FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")), - FdbKeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0080")), + KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0040")), + KeySelector.FirstGreaterOrEqual(subspace.Keys.Encode("K0080")), new FdbRangeOptions { Mode = FdbStreamingMode.WantAll } ); // T 1 diff --git a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs index de985b762..49747bc3f 100644 --- a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs +++ b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs @@ -57,7 +57,7 @@ await db.ReadWriteAsync(async (tr) => tr.Set(location.Encode("Warmup", 0), Slice.FromInt32(1)); tr.Clear(location.Encode("Warmup", 1)); await tr.GetAsync(location.Encode("Warmup", 2)); - await tr.GetRange(FdbKeyRange.StartsWith(location.Encode("Warmup", 3))).ToListAsync(); + await tr.GetRange(KeyRange.StartsWith(location.Encode("Warmup", 3))).ToListAsync(); tr.ClearRange(location.Encode("Warmup", 4), location.Encode("Warmup", 5)); }, this.Cancellation); @@ -123,10 +123,10 @@ await logged.ReadWriteAsync(async (tr) => tr.Annotate("This is a comment"); - //await tr.GetRangeAsync(FdbKeySelector.LastLessOrEqual(location.Pack("A")), FdbKeySelector.FirstGreaterThan(location.Pack("Z"))).ConfigureAwait(false); + //await tr.GetRangeAsync(KeySelector.LastLessOrEqual(location.Pack("A")), KeySelector.FirstGreaterThan(location.Pack("Z"))).ConfigureAwait(false); await Task.WhenAll( - tr.GetRange(FdbKeyRange.StartsWith(location.Encode("Range", 0))).ToListAsync(), + tr.GetRange(KeyRange.StartsWith(location.Encode("Range", 0))).ToListAsync(), tr.GetRange(location.Encode("Range", 1, 0), location.Encode("Range", 1, 200)).ToListAsync(), tr.GetRange(location.Encode("Range", 2, 400), location.Encode("Range", 2, 600)).ToListAsync(), tr.GetRange(location.Encode("Range", 3, 800), location.Encode("Range", 3, 1000)).ToListAsync() diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index 80a77704e..d0e0f40b8 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -200,44 +200,44 @@ public async Task Test_FdbKey_Batched() } [Test] - public void Test_FdbKeyRange_Contains() + public void Test_KeyRange_Contains() { - FdbKeyRange range; + KeyRange range; // ["", "") - range = FdbKeyRange.Empty; + range = KeyRange.Empty; Assert.That(range.Contains(Slice.Empty), Is.False); Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.False); Assert.That(range.Contains(Slice.FromAscii("hello")), Is.False); Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); // ["", "\xFF" ) - range = FdbKeyRange.Create(Slice.Empty, Slice.FromAscii("\xFF")); + range = KeyRange.Create(Slice.Empty, Slice.FromAscii("\xFF")); Assert.That(range.Contains(Slice.Empty), Is.True); Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.True); Assert.That(range.Contains(Slice.FromAscii("hello")), Is.True); Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); // ["\x00", "\xFF" ) - range = FdbKeyRange.Create(Slice.FromAscii("\x00"), Slice.FromAscii("\xFF")); + range = KeyRange.Create(Slice.FromAscii("\x00"), Slice.FromAscii("\xFF")); Assert.That(range.Contains(Slice.Empty), Is.False); Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.True); Assert.That(range.Contains(Slice.FromAscii("hello")), Is.True); Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); // corner cases - Assert.That(FdbKeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("A")).Contains(Slice.FromAscii("A")), Is.False, "Equal bounds"); + Assert.That(KeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("A")).Contains(Slice.FromAscii("A")), Is.False, "Equal bounds"); } [Test] - public void Test_FdbKeyRange_Test() + public void Test_KeyRange_Test() { const int BEFORE = -1, INSIDE = 0, AFTER = +1; - FdbKeyRange range; + KeyRange range; // range: [ "A", "Z" ) - range = FdbKeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("Z")); + range = KeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("Z")); // Excluding the end: < "Z" Assert.That(range.Test(Slice.FromAscii("\x00"), endIncluded: false), Is.EqualTo(BEFORE)); @@ -255,7 +255,7 @@ public void Test_FdbKeyRange_Test() Assert.That(range.Test(Slice.FromAscii("Z\x00"), endIncluded: true), Is.EqualTo(AFTER)); Assert.That(range.Test(Slice.FromAscii("\xFF"), endIncluded: true), Is.EqualTo(AFTER)); - range = FdbKeyRange.Create(FdbTuple.EncodeKey("A"), FdbTuple.EncodeKey("Z")); + range = KeyRange.Create(FdbTuple.EncodeKey("A"), FdbTuple.EncodeKey("Z")); Assert.That(range.Test(FdbTuple.Create("@")), Is.EqualTo((BEFORE))); Assert.That(range.Test(FdbTuple.Create("A")), Is.EqualTo((INSIDE))); Assert.That(range.Test(FdbTuple.Create("Z")), Is.EqualTo((AFTER))); @@ -263,66 +263,66 @@ public void Test_FdbKeyRange_Test() } [Test] - public void Test_FdbKeyRange_StartsWith() + public void Test_KeyRange_StartsWith() { - FdbKeyRange range; + KeyRange range; // "abc" => [ "abc", "abd" ) - range = FdbKeyRange.StartsWith(Slice.FromAscii("abc")); + range = KeyRange.StartsWith(Slice.FromAscii("abc")); Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc"))); Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abd"))); // "" => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); // "\xFF" => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); // null => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); } [Test] - public void Test_FdbKeyRange_PrefixedBy() + public void Test_KeyRange_PrefixedBy() { - FdbKeyRange range; + KeyRange range; // "abc" => [ "abc\x00", "abd" ) - range = FdbKeyRange.PrefixedBy(Slice.FromAscii("abc")); + range = KeyRange.PrefixedBy(Slice.FromAscii("abc")); Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc\x00"))); Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abd"))); // "" => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); // "\xFF" => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); // null => ArgumentException - Assert.That(() => FdbKeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); } [Test] - public void Test_FdbKeyRange_FromKey() + public void Test_KeyRange_FromKey() { - FdbKeyRange range; + KeyRange range; // "" => [ "", "\x00" ) - range = FdbKeyRange.FromKey(Slice.Empty); + range = KeyRange.FromKey(Slice.Empty); Assert.That(range.Begin, Is.EqualTo(Slice.Empty)); Assert.That(range.End, Is.EqualTo(Slice.FromAscii("\x00"))); // "abc" => [ "abc", "abc\x00" ) - range = FdbKeyRange.FromKey(Slice.FromAscii("abc")); + range = KeyRange.FromKey(Slice.FromAscii("abc")); Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc"))); Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abc\x00"))); // "\xFF" => [ "\xFF", "\xFF\x00" ) - range = FdbKeyRange.FromKey(Slice.FromAscii("\xFF")); + range = KeyRange.FromKey(Slice.FromAscii("\xFF")); Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("\xFF"))); Assert.That(range.End, Is.EqualTo(Slice.FromAscii("\xFF\x00"))); - Assert.That(() => FdbKeyRange.FromKey(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => KeyRange.FromKey(Slice.Nil), Throws.InstanceOf()); } [Test] @@ -365,7 +365,7 @@ public void Test_FdbKey_PrettyPrint() Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",).<00>")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",).")); - key = FdbKeyRange.StartsWith(FdbTuple.EncodeKey("hello")); + key = KeyRange.StartsWith(FdbTuple.EncodeKey("hello")); // "<02>hello<00>" .. "<02>hello<01>" Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",)")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",) + 1")); @@ -378,9 +378,9 @@ public void Test_FdbKey_PrettyPrint() } [Test] - public void Test_FdbKeyRange_Intersects() + public void Test_KeyRange_Intersects() { - Func range = (x, y) => FdbKeyRange.Create(Slice.FromByte(x), Slice.FromByte(y)); + Func range = (x, y) => KeyRange.Create(Slice.FromByte(x), Slice.FromByte(y)); #region Not Intersecting... @@ -438,9 +438,9 @@ public void Test_FdbKeyRange_Intersects() } [Test] - public void Test_FdbKeyRange_Disjoint() + public void Test_KeyRange_Disjoint() { - Func range = (x, y) => FdbKeyRange.Create(Slice.FromByte(x), Slice.FromByte(y)); + Func range = (x, y) => KeyRange.Create(Slice.FromByte(x), Slice.FromByte(y)); #region Disjoint... diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index b5848c27d..d7485ceda 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -514,7 +514,7 @@ public async Task Test_Can_MergeSort() using (var tr = db.BeginTransaction(this.Cancellation)) { var merge = tr.MergeSort( - lists.Select(list => FdbKeySelectorPair.Create(list.Keys.ToRange())), + lists.Select(list => KeySelectorPair.Create(list.Keys.ToRange())), kvp => location.Keys.DecodeLast(kvp.Key) ); @@ -584,7 +584,7 @@ public async Task Test_Range_Intersect() using (var tr = db.BeginTransaction(this.Cancellation)) { var merge = tr.Intersect( - lists.Select(list => FdbKeySelectorPair.Create(list.Keys.ToRange())), + lists.Select(list => KeySelectorPair.Create(list.Keys.ToRange())), kvp => location.Keys.DecodeLast(kvp.Key) ); @@ -656,7 +656,7 @@ public async Task Test_Range_Except() using (var tr = db.BeginTransaction(this.Cancellation)) { var merge = tr.Except( - lists.Select(list => FdbKeySelectorPair.Create(list.Keys.ToRange())), + lists.Select(list => KeySelectorPair.Create(list.Keys.ToRange())), kvp => location.Keys.DecodeLast(kvp.Key) ); diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index fa943156e..6e4093684 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -103,7 +103,7 @@ public static async Task DumpSubspace([NotNull] IFdbReadOnlyTransaction tr, [Not Console.WriteLine("Dumping content of subspace " + subspace.ToString() + " :"); int count = 0; await tr - .GetRange(FdbKeyRange.StartsWith(subspace.ToFoundationDbKey())) + .GetRange(KeyRange.StartsWith(subspace.ToFoundationDbKey())) .ForEachAsync((kvp) => { var key = subspace.ExtractKey(kvp.Key, boundCheck: true); diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 8a7281c10..94f585d5d 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -416,44 +416,44 @@ public async Task Test_Can_Resolve_Key_Selector() using (var tr = db.BeginTransaction(this.Cancellation)) { - FdbKeySelector sel; + KeySelector sel; // >= 0 - sel = FdbKeySelector.FirstGreaterOrEqual(location.Keys.Encode(0)); + sel = KeySelector.FirstGreaterOrEqual(location.Keys.Encode(0)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(location.Keys.Encode(0)), "fGE(0) should return 0"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(minKey), "fGE(0)-1 should return minKey"); Assert.That(await tr.GetKeyAsync(sel + 1), Is.EqualTo(location.Keys.Encode(1)), "fGE(0)+1 should return 1"); // > 0 - sel = FdbKeySelector.FirstGreaterThan(location.Keys.Encode(0)); + sel = KeySelector.FirstGreaterThan(location.Keys.Encode(0)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(location.Keys.Encode(1)), "fGT(0) should return 1"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(location.Keys.Encode(0)), "fGT(0)-1 should return 0"); Assert.That(await tr.GetKeyAsync(sel + 1), Is.EqualTo(location.Keys.Encode(2)), "fGT(0)+1 should return 2"); // <= 10 - sel = FdbKeySelector.LastLessOrEqual(location.Keys.Encode(10)); + sel = KeySelector.LastLessOrEqual(location.Keys.Encode(10)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(location.Keys.Encode(10)), "lLE(10) should return 10"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(location.Keys.Encode(9)), "lLE(10)-1 should return 9"); Assert.That(await tr.GetKeyAsync(sel + 1), Is.EqualTo(location.Keys.Encode(11)), "lLE(10)+1 should return 11"); // < 10 - sel = FdbKeySelector.LastLessThan(location.Keys.Encode(10)); + sel = KeySelector.LastLessThan(location.Keys.Encode(10)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(location.Keys.Encode(9)), "lLT(10) should return 9"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(location.Keys.Encode(8)), "lLT(10)-1 should return 8"); Assert.That(await tr.GetKeyAsync(sel + 1), Is.EqualTo(location.Keys.Encode(10)), "lLT(10)+1 should return 10"); // < 0 - sel = FdbKeySelector.LastLessThan(location.Keys.Encode(0)); + sel = KeySelector.LastLessThan(location.Keys.Encode(0)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(minKey), "lLT(0) should return minKey"); Assert.That(await tr.GetKeyAsync(sel + 1), Is.EqualTo(location.Keys.Encode(0)), "lLT(0)+1 should return 0"); // >= 20 - sel = FdbKeySelector.FirstGreaterOrEqual(location.Keys.Encode(20)); + sel = KeySelector.FirstGreaterOrEqual(location.Keys.Encode(20)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(maxKey), "fGE(20) should return maxKey"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(location.Keys.Encode(19)), "fGE(20)-1 should return 19"); // > 19 - sel = FdbKeySelector.FirstGreaterThan(location.Keys.Encode(19)); + sel = KeySelector.FirstGreaterThan(location.Keys.Encode(19)); Assert.That(await tr.GetKeyAsync(sel), Is.EqualTo(maxKey), "fGT(19) should return maxKey"); Assert.That(await tr.GetKeyAsync(sel - 1), Is.EqualTo(location.Keys.Encode(19)), "fGT(19)-1 should return 19"); } @@ -488,45 +488,45 @@ public async Task Test_Can_Resolve_Key_Selector_Outside_Boundaries() using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) { // before <00> - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(FdbKey.MinValue)); + key = await tr.GetKeyAsync(KeySelector.LastLessThan(FdbKey.MinValue)); Assert.That(key, Is.EqualTo(Slice.Empty), "lLT(<00>) => ''"); // before the first key in the db - var minKey = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(FdbKey.MinValue)); + var minKey = await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(FdbKey.MinValue)); Assert.That(minKey, Is.Not.Null); - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(minKey)); + key = await tr.GetKeyAsync(KeySelector.LastLessThan(minKey)); Assert.That(key, Is.EqualTo(Slice.Empty), "lLT(min_key) => ''"); // after the last key in the db - var maxKey = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(FdbKey.MaxValue)); + var maxKey = await tr.GetKeyAsync(KeySelector.LastLessThan(FdbKey.MaxValue)); Assert.That(maxKey, Is.Not.Null); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(maxKey)); + key = await tr.GetKeyAsync(KeySelector.FirstGreaterThan(maxKey)); Assert.That(key, Is.EqualTo(FdbKey.MaxValue), "fGT(maxKey) => "); // after - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(FdbKey.MaxValue)); + key = await tr.GetKeyAsync(KeySelector.FirstGreaterThan(FdbKey.MaxValue)); Assert.That(key, Is.EqualTo(FdbKey.MaxValue), "fGT() => "); - Assert.That(async () => await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(FdbKey.MaxValue + FdbKey.MaxValue)), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); - Assert.That(async () => await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Fdb.System.MinValue)), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); + Assert.That(async () => await tr.GetKeyAsync(KeySelector.FirstGreaterThan(FdbKey.MaxValue + FdbKey.MaxValue)), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); + Assert.That(async () => await tr.GetKeyAsync(KeySelector.LastLessThan(Fdb.System.MinValue)), Throws.InstanceOf().With.Property("Code").EqualTo(FdbError.KeyOutsideLegalRange)); tr.WithReadAccessToSystemKeys(); - var firstSystemKey = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(FdbKey.MaxValue)); + var firstSystemKey = await tr.GetKeyAsync(KeySelector.FirstGreaterThan(FdbKey.MaxValue)); // usually the first key in the system space is /backupDataFormat, but that may change in the future version. Assert.That(firstSystemKey, Is.Not.Null); Assert.That(firstSystemKey, Is.GreaterThan(FdbKey.MaxValue), "key should be between and "); Assert.That(firstSystemKey, Is.LessThan(Fdb.System.MaxValue), "key should be between and "); // with access to system keys, the maximum possible key becomes - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(Fdb.System.MaxValue)); + key = await tr.GetKeyAsync(KeySelector.FirstGreaterOrEqual(Fdb.System.MaxValue)); Assert.That(key, Is.EqualTo(Fdb.System.MaxValue), "fGE() => (with access to system keys)"); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(Fdb.System.MaxValue)); + key = await tr.GetKeyAsync(KeySelector.FirstGreaterThan(Fdb.System.MaxValue)); Assert.That(key, Is.EqualTo(Fdb.System.MaxValue), "fGT() => (with access to system keys)"); - key = await tr.GetKeyAsync(FdbKeySelector.LastLessThan(Fdb.System.MinValue)); + key = await tr.GetKeyAsync(KeySelector.LastLessThan(Fdb.System.MinValue)); Assert.That(key, Is.EqualTo(maxKey), "lLT(<00>) => max_key (with access to system keys)"); - key = await tr.GetKeyAsync(FdbKeySelector.FirstGreaterThan(maxKey)); + key = await tr.GetKeyAsync(KeySelector.FirstGreaterThan(maxKey)); Assert.That(key, Is.EqualTo(firstSystemKey), "fGT(max_key) => first_system_key (with access to system keys)"); } @@ -608,7 +608,7 @@ public async Task Test_Get_Multiple_Keys() using (var tr = db.BeginTransaction(this.Cancellation)) { - var selectors = Enumerable.Range(0, N).Select((i) => FdbKeySelector.FirstGreaterOrEqual(location.Keys.Encode(i))).ToArray(); + var selectors = Enumerable.Range(0, N).Select((i) => KeySelector.FirstGreaterOrEqual(location.Keys.Encode(i))).ToArray(); // GetKeysAsync([]) var results = await tr.GetKeysAsync(selectors); @@ -620,7 +620,7 @@ public async Task Test_Get_Multiple_Keys() } // GetKeysAsync(cast to enumerable) - var results2 = await tr.GetKeysAsync((IEnumerable)selectors); + var results2 = await tr.GetKeysAsync((IEnumerable)selectors); Assert.That(results2, Is.EqualTo(results)); // GetKeysAsync(real enumerable) @@ -1041,7 +1041,7 @@ await db.WriteAsync((tr) => using (var tr1 = db.BeginTransaction(this.Cancellation)) { // fGE{0} => 50 - var key = await tr1.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 0))); + var key = await tr1.GetKeyAsync(KeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 0))); Assert.That(key, Is.EqualTo(loc.Keys.Encode("foo", 50))); // 42 < 50 => conflict !!! @@ -1068,7 +1068,7 @@ await db.WriteAsync((tr) => using (var tr1 = db.BeginTransaction(this.Cancellation)) { // fGE{0} => 50 - var key = await tr1.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 0))); + var key = await tr1.GetKeyAsync(KeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 0))); Assert.That(key, Is.EqualTo(loc.Keys.Encode("foo", 50))); // 77 > 50 => no conflict @@ -1097,7 +1097,7 @@ await db.WriteAsync((tr) => using (var tr1 = db.BeginTransaction(this.Cancellation)) { // fGE{50} + 1 => 100 - var key = await tr1.GetKeyAsync(FdbKeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 50)) + 1); + var key = await tr1.GetKeyAsync(KeySelector.FirstGreaterOrEqual(loc.Keys.Encode("foo", 50)) + 1); Assert.That(key, Is.EqualTo(loc.Keys.Encode("foo", 100))); // 77 between 50 and 100 => conflict !!! @@ -1126,7 +1126,7 @@ await db.WriteAsync((tr) => using (var tr1 = db.BeginTransaction(this.Cancellation)) { // fGT{50} => 100 - var key = await tr1.GetKeyAsync(FdbKeySelector.FirstGreaterThan(loc.Keys.Encode("foo", 50))); + var key = await tr1.GetKeyAsync(KeySelector.FirstGreaterThan(loc.Keys.Encode("foo", 50))); Assert.That(key, Is.EqualTo(loc.Keys.Encode("foo", 100))); // another transaction changes the VALUE of 50 and 100 (but does not change the fact that they exist nor add keys in between) @@ -1155,7 +1155,7 @@ await db.WriteAsync((tr) => using (var tr1 = db.BeginTransaction(this.Cancellation)) { // lLT{100} => 50 - var key = await tr1.GetKeyAsync(FdbKeySelector.LastLessThan(loc.Keys.Encode("foo", 100))); + var key = await tr1.GetKeyAsync(KeySelector.LastLessThan(loc.Keys.Encode("foo", 100))); Assert.That(key, Is.EqualTo(loc.Keys.Encode("foo", 50))); // another transaction changes the VALUE of 50 and 100 (but does not change the fact that they exist nor add keys in between) @@ -1912,10 +1912,10 @@ public async Task Test_Simple_Read_Transaction() //tr.AddReadConflictKey(location.Concat(Slice.FromString("READ_CONFLICT"))); //tr.AddWriteConflictKey(location.Concat(Slice.FromString("WRITE_CONFLICT"))); - //tr.AddReadConflictRange(new FdbKeyRange(location.Concat(Slice.FromString("D")), location.Concat(Slice.FromString("E")))); - //tr.AddReadConflictRange(new FdbKeyRange(location.Concat(Slice.FromString("C")), location.Concat(Slice.FromString("G")))); - //tr.AddReadConflictRange(new FdbKeyRange(location.Concat(Slice.FromString("B")), location.Concat(Slice.FromString("F")))); - //tr.AddReadConflictRange(new FdbKeyRange(location.Concat(Slice.FromString("A")), location.Concat(Slice.FromString("Z")))); + //tr.AddReadConflictRange(new KeyRange(location.Concat(Slice.FromString("D")), location.Concat(Slice.FromString("E")))); + //tr.AddReadConflictRange(new KeyRange(location.Concat(Slice.FromString("C")), location.Concat(Slice.FromString("G")))); + //tr.AddReadConflictRange(new KeyRange(location.Concat(Slice.FromString("B")), location.Concat(Slice.FromString("F")))); + //tr.AddReadConflictRange(new KeyRange(location.Concat(Slice.FromString("A")), location.Concat(Slice.FromString("Z")))); await tr.CommitAsync(); From 0c9fb821e7a414c99cf6a73153b21866f31d96e0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 14:58:30 +0200 Subject: [PATCH 059/153] Added missing custom R# attributes [Positive] and [PowerOfTwo] --- FoundationDB.Client/Utils/CodeAnnotations.cs | 25 ++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/FoundationDB.Client/Utils/CodeAnnotations.cs b/FoundationDB.Client/Utils/CodeAnnotations.cs index ad6581a39..8d9fab142 100644 --- a/FoundationDB.Client/Utils/CodeAnnotations.cs +++ b/FoundationDB.Client/Utils/CodeAnnotations.cs @@ -518,4 +518,29 @@ internal sealed class RegexPatternAttribute : Attribute { } [Conditional("JETBRAINS_ANNOTATIONS")] internal sealed class NoReorder : Attribute { } + // ==================================================== + // === CUSTOM CONTRACT ATTRIBUTES + // ==================================================== + + // NOTE: these attributes are not recognize by Resharper (yet?) but can be used + // by Roslyn Analyzers or other static analysis tools to further verify the code. + + // DO NOT OVERWRITE THESE WHEN UPDATING THE OFFICAL CONTRACT ATTRIBUTES! + + /// The value cannot be negative + /// REQUIRES: x >= 0 + [AttributeUsage( + AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | + AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event)] + [Conditional("JETBRAINS_ANNOTATIONS")] + internal sealed class PositiveAttribute : Attribute { } + + /// The value must be a power of two + /// REQUIRES: x == 1 << Round(Log2(X)) + [AttributeUsage( + AttributeTargets.Method | AttributeTargets.Parameter | AttributeTargets.Property | + AttributeTargets.Delegate | AttributeTargets.Field | AttributeTargets.Event)] + [Conditional("JETBRAINS_ANNOTATIONS")] + internal sealed class PowerOfTwoAttribute : Attribute { } + } From 04054b1d488ba5b73643e2d580be2cf92086d3f8 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 15:22:12 +0200 Subject: [PATCH 060/153] Removed IFdbKey interface - This was not a good abstraction in practice. - Reduce API clutter a lot --- FoundationDB.Client/FdbDatabase.cs | 28 -- FoundationDB.Client/FdbDatabaseExtensions.cs | 14 - .../FdbTransactionExtensions.cs | 329 ------------------ .../Filters/FdbDatabaseFilter.cs | 31 -- FoundationDB.Client/IFdbDatabase.cs | 2 +- FoundationDB.Client/IFdbKey.cs | 2 + FoundationDB.Client/KeyRange.cs | 46 --- FoundationDB.Client/KeySelector.cs | 41 --- FoundationDB.Client/KeySelectorPair.cs | 20 -- .../Layers/Tuples/FdbJoinedTuple.cs | 5 - .../Layers/Tuples/FdbLinkedTuple.cs | 5 - .../Layers/Tuples/FdbListTuple.cs | 5 - .../Layers/Tuples/FdbMemoizedTuple.cs | 5 - .../Layers/Tuples/FdbPrefixedTuple.cs | 5 - .../Layers/Tuples/FdbSlicedTuple.cs | 5 - FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 5 - .../Layers/Tuples/FdbTuplePackers.cs | 16 - .../Layers/Tuples/FdbTuple`1.cs | 5 - .../Layers/Tuples/FdbTuple`2.cs | 5 - .../Layers/Tuples/FdbTuple`3.cs | 5 - .../Layers/Tuples/FdbTuple`4.cs | 5 - .../Layers/Tuples/FdbTuple`5.cs | 5 - .../Layers/Tuples/IFdbTuple.cs | 2 +- .../Subspaces/FdbDynamicSubspacePartition.cs | 14 - FoundationDB.Client/Subspaces/FdbSubspace.cs | 83 +---- .../Subspaces/FdbSubspaceExtensions.cs | 16 +- FoundationDB.Client/Subspaces/IFdbSubspace.cs | 24 +- .../Collections/FdbMultimap`2.cs | 6 +- .../Collections/FdbRankedSet.cs | 4 +- .../Messaging/FdbWorkerPool.cs | 2 +- FoundationDB.Tests.Sandbox/Program.cs | 2 +- FoundationDB.Tests/DatabaseFacts.cs | 2 +- FoundationDB.Tests/KeyFacts.cs | 8 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 6 - FoundationDB.Tests/Layers/MultiMapFacts.cs | 6 +- .../Linq/FdbQueryExpressionFacts.cs | 4 +- FoundationDB.Tests/SubspaceFacts.cs | 9 - FoundationDB.Tests/TestHelpers.cs | 4 +- 38 files changed, 35 insertions(+), 746 deletions(-) diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index c73a673b6..268f49c40 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -446,11 +446,6 @@ public void SetOption(FdbDatabaseOption option, long value) #region Key Space Management... - Slice IFdbKey.ToFoundationDbKey() - { - return m_globalSpaceCopy.Key; - } - /// Change the current global namespace. /// Do NOT call this, unless you know exactly what you are doing ! internal void ChangeRoot(IFdbSubspace subspace, IFdbDirectory directory, bool readOnly) @@ -544,21 +539,11 @@ Slice IFdbSubspace.ConcatKey(Slice key) return m_globalSpace.ConcatKey(key); } - Slice IFdbSubspace.ConcatKey(TKey key) - { - return m_globalSpace.ConcatKey(key); - } - Slice[] IFdbSubspace.ConcatKeys(IEnumerable keys) { return m_globalSpace.ConcatKeys(keys); } - Slice[] IFdbSubspace.ConcatKeys(IEnumerable keys) - { - return m_globalSpace.ConcatKeys(keys); - } - /// Remove the database global subspace prefix from a binary key, or throw if the key is outside of the global subspace. Slice IFdbSubspace.ExtractKey(Slice key, bool boundCheck) { @@ -589,14 +574,6 @@ IFdbSubspace IFdbSubspace.this[Slice suffix] } } - IFdbSubspace IFdbSubspace.this[IFdbKey key] - { - get - { - return m_globalSpace[key]; - } - } - KeyRange IFdbSubspace.ToRange() { return m_globalSpace.ToRange(); @@ -607,11 +584,6 @@ KeyRange IFdbSubspace.ToRange(Slice suffix) return m_globalSpace.ToRange(suffix); } - KeyRange IFdbSubspace.ToRange(TKey key) - { - return m_globalSpace.ToRange(key); - } - public FdbDynamicSubspacePartition Partition { //REVIEW: should we hide this on the main db? diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index 9b103c72e..6ed7c433b 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -481,13 +481,6 @@ public static Task GetAndWatch([NotNull] this IFdbRetryable db, Slice }, cancellationToken); } - public static Task GetAndWatch([NotNull] this IFdbRetryable db, TKey key, CancellationToken cancellationToken) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return GetAndWatch(db, key.ToFoundationDbKey(), cancellationToken); - } - /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. /// Database instance. /// Name of the key to be inserted into the database. @@ -510,13 +503,6 @@ await db.WriteAsync((tr) => return watch; } - public static Task SetAndWatch(this IFdbRetryable db, TKey key, Slice value, CancellationToken cancellationToken) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return SetAndWatch(db, key.ToFoundationDbKey(), value, cancellationToken); - } - #endregion } diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 325f35233..cf517d4db 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -184,23 +184,6 @@ public static TTransaction WithMaxRetryDelay([NotNull] this TTrans #region Get... - /// Reads a value from the database snapshot represented by by the current transaction. - /// Type of the key that implements IFdbKey. - /// Transaction instance - /// Key to be looked up in the database - /// Task that will return the value of the key if it is found, Slice.Nil if the key does not exist, or an exception - /// If the is null - /// If the cancellation token is already triggered - /// If the transaction has already been completed - /// If the operation method is called from the Network Thread - public static Task GetAsync(this IFdbReadOnlyTransaction trans, TKey key) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - return trans.GetAsync(key.ToFoundationDbKey()); - } - /// Reads and decode a value from the database snapshot represented by by the current transaction. /// Type of the value. /// Transaction to use for the operation @@ -219,42 +202,10 @@ public static async Task GetAsync([NotNull] this IFdbReadOnlyTra return encoder.DecodeValue(await trans.GetAsync(key).ConfigureAwait(false)); } - /// Reads and decode a value from the database snapshot represented by by the current transaction. - /// Type of the key that implements IFdbKey. - /// Type of the value. - /// Transaction instance - /// Key to be looked up in the database - /// Encoder used to decode the value of the key. - /// Task that will return the value of the key if it is found, Slice.Nil if the key does not exist, or an exception - /// If the is null - /// If the cancellation token is already triggered - /// If the transaction has already been completed - /// If the operation method is called from the Network Thread - public static Task GetAsync([NotNull] this IFdbReadOnlyTransaction trans, TKey key, [NotNull] IValueEncoder encoder) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return GetAsync(trans, key.ToFoundationDbKey(), encoder); - } - #endregion #region Set... - /// Set the value of a key in the database. - /// Type of the key that implements IFdbKey. - /// Transaction instance - /// - /// - public static void Set([NotNull] this IFdbTransaction trans, TKey key, Slice value) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Set(key.ToFoundationDbKey(), value); - } - /// Set the value of a key in the database, using a custom value encoder. /// Type of the value /// Transaction to use for the operation @@ -269,20 +220,6 @@ public static void Set([NotNull] this IFdbTransaction trans, Slice key, trans.Set(key, encoder.EncodeValue(value)); } - /// Set the value of a key in the database, using a custom value encoder. - /// Type of the key that implements IFdbKey. - /// Type of the value - /// Transaction instance - /// Key to set - /// Value of the key - /// Encoder used to convert into a binary slice. - public static void Set([NotNull] this IFdbTransaction trans, TKey key, TValue value, [NotNull] IValueEncoder encoder) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - Set(trans, key.ToFoundationDbKey(), value, encoder); - } - /// Set the value of a key in the database, using the content of a Stream /// Transaction to use for the operation /// Key to set @@ -429,20 +366,6 @@ public static void AtomicAdd([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, value, FdbMutationType.Add); } - /// Modify the database snapshot represented by this transaction to add to the value stored by the given . - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Value to add to existing value of key. - public static void AtomicAdd([NotNull] this IFdbTransaction trans, TKey key, Slice value) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), value, FdbMutationType.Add); - } - /// Modify the database snapshot represented by this transaction to perform a bitwise AND between and the value stored by the given . /// Transaction to use for the operation /// Name of the key whose value is to be mutated. @@ -455,21 +378,6 @@ public static void AtomicAnd([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, mask, FdbMutationType.BitAnd); } - /// Modify the database snapshot represented by this transaction to perform a bitwise AND between and the value stored by the given . - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Bit mask. - public static void AtomicAnd([NotNull] this IFdbTransaction trans, TKey key, Slice mask) - where TKey : IFdbKey - { - //TODO: rename this to AtomicBitAnd(...) ? - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), mask, FdbMutationType.BitAnd); - } - /// Modify the database snapshot represented by this transaction to perform a bitwise OR between and the value stored by the given . /// Transaction to use for the operation /// Name of the key whose value is to be mutated. @@ -482,21 +390,6 @@ public static void AtomicOr([NotNull] this IFdbTransaction trans, Slice key, Sli trans.Atomic(key, mask, FdbMutationType.BitOr); } - /// Modify the database snapshot represented by this transaction to perform a bitwise OR between and the value stored by the given . - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Bit mask. - public static void AtomicOr(this IFdbTransaction trans, TKey key, Slice mask) - where TKey : IFdbKey - { - //TODO: rename this to AtomicBitOr(...) ? - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), mask, FdbMutationType.BitOr); - } - /// Modify the database snapshot represented by this transaction to perform a bitwise XOR between and the value stored by the given . /// Transaction to use for the operation /// Name of the key whose value is to be mutated. @@ -509,21 +402,6 @@ public static void AtomicXor([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, mask, FdbMutationType.BitXor); } - /// Modify the database snapshot represented by this transaction to perform a bitwise XOR between and the value stored by the given . - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Bit mask. - public static void AtomicXor(this IFdbTransaction trans, TKey key, Slice mask) - where TKey : IFdbKey - { - //TODO: rename this to AtomicBitXOr(...) ? - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), mask, FdbMutationType.BitXor); - } - /// Modify the database snapshot represented by this transaction to update a value if it is larger than the value in the database. /// Transaction to use for the operation /// Name of the key whose value is to be mutated. @@ -535,20 +413,6 @@ public static void AtomicMax([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, value, FdbMutationType.Max); } - /// Modify the database snapshot represented by this transaction to update a value if it is larger than the value in the database. - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Bit mask. - public static void AtomicMax(this IFdbTransaction trans, TKey key, Slice value) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), value, FdbMutationType.Max); - } - /// Modify the database snapshot represented by this transaction to update a value if it is smaller than the value in the database. /// Transaction instance /// Name of the key whose value is to be mutated. @@ -560,20 +424,6 @@ public static void AtomicMin([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, value, FdbMutationType.Min); } - /// Modify the database snapshot represented by this transaction to update a value if it is smaller than the value in the database. - /// Type of the key that implements . - /// Transaction instance - /// Name of the key whose value is to be mutated. - /// Bit mask. - public static void AtomicMin(this IFdbTransaction trans, TKey key, Slice value) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Atomic(key.ToFoundationDbKey(), value, FdbMutationType.Min); - } - #endregion #region GetRange... @@ -609,29 +459,11 @@ public static FdbRangeQuery> GetRange([NotNull] this ); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, TKey beginKeyInclusive, TKey endKeyExclusive, FdbRangeOptions options = null) - where TKey : IFdbKey - { - //TODO: TKey in, but Slice out ? Maybe we need to get a ISliceSerializer to convert the slices back to a TKey ? - if (beginKeyInclusive == null) throw new ArgumentNullException("beginKeyInclusive"); - if (endKeyExclusive == null) throw new ArgumentNullException("endKeyExclusive"); - return GetRange(trans, beginKeyInclusive.ToFoundationDbKey(), endKeyExclusive.ToFoundationDbKey(), options); - } - public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, int limit, bool reverse = false) { return GetRange(trans, beginKeyInclusive, endKeyExclusive, new FdbRangeOptions(limit: limit, reverse: reverse)); } - public static FdbRangeQuery> GetRange(this IFdbReadOnlyTransaction trans, TKey beginKeyInclusive, TKey endKeyExclusive, int limit, bool reverse = false) - where TKey : IFdbKey - { - if (beginKeyInclusive == null) throw new ArgumentNullException("beginKeyInclusive"); - if (endKeyExclusive == null) throw new ArgumentNullException("endKeyExclusive"); - - return GetRange(trans, beginKeyInclusive.ToFoundationDbKey(), endKeyExclusive.ToFoundationDbKey(), new FdbRangeOptions(limit: limit, reverse: reverse)); - } - /// /// Create a new range query that will read all key-value pairs in the database snapshot represented by the transaction /// @@ -704,19 +536,6 @@ public static Task GetRangeAsync([NotNull] this IFdbReadOnlyTrans #region Clear... - /// - /// Modify the database snapshot represented by this transaction to remove the given key from the database. If the key was not previously present in the database, there is no effect. - /// - /// Key to be removed from the database. - public static void Clear(this IFdbTransaction trans, TKey key) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (key == null) throw new ArgumentNullException("key"); - - trans.Clear(key.ToFoundationDbKey()); - } - /// /// Modify the database snapshot represented by this transaction to remove all keys (if any) which are lexicographically greater than or equal to the given begin key and lexicographically less than the given end_key. /// Sets and clears affect the actual database only if transaction is later committed with CommitAsync(). @@ -766,19 +585,6 @@ public static void AddReadConflictRange([NotNull] this IFdbTransaction trans, Sl trans.AddConflictRange(beginKeyInclusive, endKeyExclusive, FdbConflictRangeType.Read); } - /// - /// Adds a range of keys to the transaction’s read conflict ranges as if you had read the range. As a result, other transactions that write a key in this range could cause the transaction to fail with a conflict. - /// - public static void AddReadConflictRange(this IFdbTransaction trans, TKey beginKeyInclusive, TKey endKeyExclusive) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (beginKeyInclusive == null) throw new ArgumentNullException("beginKeyInclusive"); - if (endKeyExclusive == null) throw new ArgumentNullException("endKeyExclusive"); - - trans.AddConflictRange(beginKeyInclusive.ToFoundationDbKey(), endKeyExclusive.ToFoundationDbKey(), FdbConflictRangeType.Read); - } - /// /// Adds a key to the transaction’s read conflict ranges as if you had read the key. As a result, other transactions that write to this key could cause the transaction to fail with a conflict. /// @@ -787,16 +593,6 @@ public static void AddReadConflictKey([NotNull] this IFdbTransaction trans, Slic AddConflictRange(trans, KeyRange.FromKey(key), FdbConflictRangeType.Read); } - /// - /// Adds a key to the transaction’s read conflict ranges as if you had read the key. As a result, other transactions that write to this key could cause the transaction to fail with a conflict. - /// - public static void AddReadConflictKey(this IFdbTransaction trans, TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - AddConflictRange(trans, KeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Read); - } - /// /// Adds a range of keys to the transaction’s write conflict ranges as if you had cleared the range. As a result, other transactions that concurrently read a key in this range could fail with a conflict. /// @@ -815,19 +611,6 @@ public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, S trans.AddConflictRange(beginKeyInclusive, endKeyExclusive, FdbConflictRangeType.Write); } - /// - /// Adds a range of keys to the transaction’s write conflict ranges as if you had cleared the range. As a result, other transactions that concurrently read a key in this range could fail with a conflict. - /// - public static void AddWriteConflictRange(this IFdbTransaction trans, TKey beginKeyInclusive, TKey endKeyExclusive) - where TKey : IFdbKey - { - if (trans == null) throw new ArgumentNullException("trans"); - if (beginKeyInclusive == null) throw new ArgumentNullException("beginKeyInclusive"); - if (endKeyExclusive == null) throw new ArgumentNullException("endKeyExclusive"); - - trans.AddConflictRange(beginKeyInclusive.ToFoundationDbKey(), endKeyExclusive.ToFoundationDbKey(), FdbConflictRangeType.Write); - } - /// /// Adds a key to the transaction’s write conflict ranges as if you had cleared the key. As a result, other transactions that concurrently read this key could fail with a conflict. /// @@ -836,16 +619,6 @@ public static void AddWriteConflictKey([NotNull] this IFdbTransaction trans, Sli AddConflictRange(trans, KeyRange.FromKey(key), FdbConflictRangeType.Write); } - /// - /// Adds a key to the transaction’s write conflict ranges as if you had cleared the key. As a result, other transactions that concurrently read this key could fail with a conflict. - /// - public static void AddWriteConflictKey(this IFdbTransaction trans, TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - AddConflictRange(trans, KeyRange.FromKey(key.ToFoundationDbKey()), FdbConflictRangeType.Write); - } - #endregion #region Watches... @@ -867,19 +640,6 @@ public static async Task GetAndWatchAsync([NotNull] this IFdbTransacti return watch; } - /// Reads the value associated with , and returns a Watch that will complete after a subsequent change to key in the database. - /// Type of the key, which must implement the IFdbKey interface - /// Transaction to use for the operation - /// Key to be looked up in the database - /// Token that can be used to cancel the Watch from the outside. - /// A new Watch that will track any changes to in the database, and whose Value property contains the current value of the key. - public static Task GetAndWatchAsync(this IFdbTransaction trans, TKey key, CancellationToken cancellationToken) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return GetAndWatchAsync(trans, key.ToFoundationDbKey(), cancellationToken); - } - /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. /// Transaction to use for the operation /// Name of the key to be inserted into the database. @@ -898,20 +658,6 @@ public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice k return watch; } - /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. - /// Type of the key, which must implement the IFdbKey interface - /// Transaction to use for the operation - /// Name of the key to be inserted into the database. - /// Value to be inserted into the database. - /// Token that can be used to cancel the Watch from the outside. - /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch(this IFdbTransaction trans, TKey key, Slice value, CancellationToken cancellationToken) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return SetAndWatch(trans, key.ToFoundationDbKey(), value, cancellationToken); - } - /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. /// Type of the value /// Transaction to use for the operation @@ -927,24 +673,6 @@ public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, return SetAndWatch(trans, key, encoder.EncodeValue(value), cancellationToken); } - /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. - /// Type of the key, which must implement the IFdbKey interface - /// Type of the value - /// Transaction to use for the operation - /// Name of the key to be inserted into the database. - /// Value to be inserted into the database. - /// Encoder use to convert into a slice - /// Token that can be used to cancel the Watch from the outside. - /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch(this IFdbTransaction trans, TKey key, TValue value, [NotNull] IValueEncoder encoder, CancellationToken cancellationToken) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - if (encoder == null) throw new ArgumentNullException("encoder"); - cancellationToken.ThrowIfCancellationRequested(); - return SetAndWatch(trans, key.ToFoundationDbKey(), encoder.EncodeValue(value), cancellationToken); - } - #endregion #region Batching... @@ -982,35 +710,6 @@ public static async Task GetValuesAsync([NotNull] this IFdbRea return decoder.DecodeValues(await GetValuesAsync(trans, keys).ConfigureAwait(false)); } - /// - /// Reads several values from the database snapshot represented by the current transaction - /// - /// Sequence of keys to be looked up in the database - /// Task that will return an array of values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value will be Slice.Nil. - [ItemNotNull] - public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) - where TKey : IFdbKey - { - if (keys == null) throw new ArgumentNullException("keys"); - - return GetValuesAsync(trans, keys.Select(key => key.ToFoundationDbKey())); - } - - /// - /// Reads several values from the database snapshot represented by the current transaction. - /// - /// Sequence of keys to be looked up in the database - /// Decoder used to decoded the results into values of type - /// Task that will return an array of decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . - [ItemNotNull] - public static Task GetValuesAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) - where TKey : IFdbKey - { - if (keys == null) throw new ArgumentNullException("keys"); - - return GetValuesAsync(trans, keys.Select(key => key.ToFoundationDbKey()), decoder); - } - /// /// Resolves several key selectors against the keys in the database snapshot represented by the current transaction. /// @@ -1113,34 +812,6 @@ public static async Task[]> GetBatchAsync([N return array; } - /// - /// Reads several values from the database snapshot represented by the current transaction. - /// - /// Sequence of keys to be looked up in the database - /// Task that will return an array of key/value pairs, or an exception. Each pair in the array will contain the key at the same index in , and its corresponding value in the database or Slice.Nil if that key does not exist. - /// This method is equivalent to calling , except that it will return the keys in addition to the values. - [ItemNotNull] - public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys) - where TKey : IFdbKey - { - if (keys == null) throw new ArgumentNullException("keys"); - return GetBatchAsync(trans, keys.Select(key => key.ToFoundationDbKey()).ToArray()); - } - - /// - /// Reads several values from the database snapshot represented by the current transaction. - /// - /// Sequence of keys to be looked up in the database - /// Decoder used to decoded the results into values of type - /// Task that will return an array of pairs of key and decoded values, or an exception. The position of each item in the array is the same as its coresponding key in . If a key does not exist in the database, its value depends on the behavior of . - [ItemNotNull] - public static Task[]> GetBatchAsync(this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable keys, [NotNull] IValueEncoder decoder) - where TKey : IFdbKey - { - if (keys == null) throw new ArgumentNullException("keys"); - return GetBatchAsync(trans, keys.Select(key => key.ToFoundationDbKey()).ToArray(), decoder); - } - #endregion #region Queries... diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index db9840bc8..f4431c11c 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -150,21 +150,11 @@ KeyRange IFdbSubspace.ToRange(Slice suffix) return this.GlobalSpace.ToRange(suffix); } - KeyRange IFdbSubspace.ToRange(TKey key) - { - return this.GlobalSpace.ToRange(key); - } - IFdbSubspace IFdbSubspace.this[Slice suffix] { get { return this.GlobalSpace[suffix]; } } - IFdbSubspace IFdbSubspace.this[IFdbKey key] - { - get { return this.GlobalSpace[key]; } - } - public virtual FdbDynamicSubspacePartition Partition { get { return m_database.Partition; } @@ -190,23 +180,11 @@ public virtual Slice ConcatKey(Slice key) return m_database.ConcatKey(key); } - public virtual Slice ConcatKey(TKey key) - where TKey : IFdbKey - { - return m_database.ConcatKey(key); - } - public virtual Slice[] ConcatKeys(IEnumerable keys) { return m_database.ConcatKeys(keys); } - public virtual Slice[] ConcatKeys(IEnumerable keys) - where TKey : IFdbKey - { - return m_database.ConcatKeys(keys); - } - public virtual Slice ExtractKey(Slice key, bool boundCheck = false) { return m_database.ExtractKey(key, boundCheck); @@ -406,15 +384,6 @@ protected virtual void Dispose(bool disposing) #endregion - #region IFdbSubspace Members... - - Slice IFdbKey.ToFoundationDbKey() - { - return m_database.ToFoundationDbKey(); - } - - #endregion - } } diff --git a/FoundationDB.Client/IFdbDatabase.cs b/FoundationDB.Client/IFdbDatabase.cs index 736e062bf..7958b4d7a 100644 --- a/FoundationDB.Client/IFdbDatabase.cs +++ b/FoundationDB.Client/IFdbDatabase.cs @@ -34,7 +34,7 @@ namespace FoundationDB.Client /// Database connection context. [PublicAPI] - public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IFdbDynamicSubspace, IFdbKey, IDisposable + public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IFdbDynamicSubspace, IDisposable { /// Name of the database string Name { [NotNull] get; } diff --git a/FoundationDB.Client/IFdbKey.cs b/FoundationDB.Client/IFdbKey.cs index f18ea95ee..1b5d15d50 100644 --- a/FoundationDB.Client/IFdbKey.cs +++ b/FoundationDB.Client/IFdbKey.cs @@ -31,9 +31,11 @@ namespace FoundationDB.Client using System; /// Allows a layer or custom class to be used as keys in a FoundationDB database + [Obsolete("Remove Me")] public interface IFdbKey { /// Returns a binary representation of this instance, to be used as a complete key in the database + [Obsolete("Remove Me", error: true)] Slice ToFoundationDbKey(); } diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index 06eb12468..a6c133751 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -63,17 +63,6 @@ public KeyRange(Slice begin, Slice end) Contract.Ensures(m_begin <= m_end, "The range is inverted"); } - public KeyRange(IFdbKey begin, IFdbKey end) - { - if (begin == null) throw new ArgumentNullException("begin"); - if (end == null) throw new ArgumentNullException("end"); - - m_begin = begin.ToFoundationDbKey(); - m_end = end.ToFoundationDbKey(); - - Contract.Ensures(m_begin <= m_end, "The range is inverted"); - } - public static KeyRange Create(Slice a, Slice b) { return new KeyRange(a, b); @@ -95,13 +84,6 @@ public static KeyRange StartsWith(Slice prefix) ); } - public static KeyRange StartsWith(TKey prefix) - where TKey : IFdbKey - { - if (prefix == null) throw new ArgumentNullException("prefix"); - return StartsWith(prefix.ToFoundationDbKey()); - } - /// Create a range that selects all keys starting with , but not the prefix itself: ('prefix\x00' <= k < string('prefix') /// Key prefix (that will be excluded from the range) /// Range including all keys with the specified prefix. @@ -116,13 +98,6 @@ public static KeyRange PrefixedBy(Slice prefix) ); } - public static KeyRange PrefixedBy(TKey prefix) - where TKey : IFdbKey - { - if (prefix == null) throw new ArgumentNullException("prefix"); - return PrefixedBy(prefix.ToFoundationDbKey()); - } - /// Create a range that will only return itself ('key' <= k < 'key\x00') /// Key that will be returned by the range /// Range that only return the specified key. @@ -141,13 +116,6 @@ public static KeyRange FromKey(Slice key) ); } - public static KeyRange FromKey(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return FromKey(key.ToFoundationDbKey()); - } - public override bool Equals(object obj) { return (obj is KeyRange) && Equals((KeyRange)obj); @@ -245,13 +213,6 @@ public bool Contains(Slice key) return key.CompareTo(m_begin) >= 0 && key.CompareTo(m_end) < 0; } - public bool Contains(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return Contains(key.ToFoundationDbKey()); - } - /// Test if is contained inside the range /// Key that will be compared with the the range's bounds /// If true, the End bound is inclusive, otherwise it is exclusive @@ -265,13 +226,6 @@ public int Test(Slice key, bool endIncluded = false) return 0; } - public int Test(TKey key, bool endIncluded = false) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return Test(key.ToFoundationDbKey(), endIncluded); - } - /// Returns a printable version of the range public override string ToString() { diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index 36f581cd9..51168d896 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -58,15 +58,6 @@ public KeySelector(Slice key, bool orEqual, int offset) this.Offset = offset; } - /// Creates a new selector - public KeySelector(IFdbKey key, bool orEqual, int offset) - { - if (key == null) throw new ArgumentNullException("key"); - m_key = key.ToFoundationDbKey(); - this.OrEqual = orEqual; - this.Offset = offset; - } - /// Returns a displayable representation of the key selector [Pure] public string PrettyPrint(FdbKey.PrettyPrintMode mode) @@ -143,38 +134,6 @@ public static KeySelector FirstGreaterOrEqual(Slice key) return new KeySelector(key, false, 1); } - /// Creates a key selector that will select the last key that is less than - public static KeySelector LastLessThan(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return LastLessThan(key.ToFoundationDbKey()); - } - - /// Creates a key selector that will select the last key that is less than or equal to - public static KeySelector LastLessOrEqual(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return LastLessOrEqual(key.ToFoundationDbKey()); - } - - /// Creates a key selector that will select the first key that is greater than - public static KeySelector FirstGreaterThan(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return FirstGreaterThan(key.ToFoundationDbKey()); - } - - /// Creates a key selector that will select the first key that is greater than or equal to - public static KeySelector FirstGreaterOrEqual(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return FirstGreaterOrEqual(key.ToFoundationDbKey()); - } - /// Add a value to the selector's offset /// ex: fGE('abc') /// ex: 7 diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index 40651d844..d72e32b96 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -70,18 +70,6 @@ public static KeySelectorPair Create(Slice begin, Slice end) ); } - /// Create a new pair of key selectors using FIRST_GREATER_OR_EQUAL on both keys - public static KeySelectorPair Create(TKey begin, TKey end) - where TKey : IFdbKey - { - if (begin == null) throw new ArgumentNullException("begin"); - if (end == null) throw new ArgumentNullException("end"); - return new KeySelectorPair( - KeySelector.FirstGreaterOrEqual(begin.ToFoundationDbKey()), - KeySelector.FirstGreaterOrEqual(end.ToFoundationDbKey()) - ); - } - /// Create a new pair of key selectors using FIRST_GREATER_OR_EQUAL on both keys public static KeySelectorPair Create(KeyRange range) { @@ -102,14 +90,6 @@ public static KeySelectorPair StartsWith(Slice prefix) ); } - /// Create a new pair of key selectors that will select all the keys that start with the specified prefix - public static KeySelectorPair StartsWith(TKey prefix) - where TKey : IFdbKey - { - if (prefix == null) throw new ArgumentNullException("prefix"); - return StartsWith(prefix.ToFoundationDbKey()); - } - /// Returns a printable version of the pair of key selectors public override string ToString() { diff --git a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs index 60e7bb181..1155b2116 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs @@ -78,11 +78,6 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return FdbTuple.ToString(this); diff --git a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs index 80abb61f6..856b3378a 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs @@ -80,11 +80,6 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - /// Returns the number of elements in this tuple public int Count { diff --git a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs index 260ad099e..52223a753 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs @@ -251,11 +251,6 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return FdbTuple.ToString(m_items, m_offset, m_count); diff --git a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs index a906f3dad..8d0b74743 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs @@ -88,11 +88,6 @@ public Slice ToSlice() return m_packed; } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public FdbMemoizedTuple Copy() { return new FdbMemoizedTuple( diff --git a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs index aa986c60b..7ede05619 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs @@ -74,11 +74,6 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public int Count { get { return m_items.Count; } diff --git a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs index 90c4ff6dd..a0d233b16 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs @@ -79,11 +79,6 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public int Count { get { return m_count; } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs index d43e3015e..5841128c7 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs @@ -99,11 +99,6 @@ public Slice ToSlice() return Slice.Empty; } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public void CopyTo(object[] array, int offset) { //NO-OP diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs index 488616fce..965c0c1a0 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs @@ -95,15 +95,6 @@ private static Delegate GetSerializerFor([NotNull] Type type) } } - if (typeof(IFdbKey).IsAssignableFrom(type)) - { - method = typeof(FdbTuplePackers).GetMethod("SerializeFdbKeyTo", BindingFlags.Static | BindingFlags.Public); - if (method != null) - { - return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); - } - } - var nullableType = Nullable.GetUnderlyingType(type); if (nullableType != null) { // nullable types can reuse the underlying type serializer @@ -509,13 +500,6 @@ public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormatta FdbTupleParser.EndTuple(ref writer); } - public static void SerializeFdbKeyTo(ref TupleWriter writer, IFdbKey key) - { - Contract.Requires(key != null); - var slice = key.ToFoundationDbKey(); - FdbTupleParser.WriteBytes(ref writer, slice); - } - #endregion #region Deserializers... diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs index 1f58f922d..1567fb8c4 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs @@ -151,11 +151,6 @@ public Slice ToSlice() return FdbTuple.EncodeKey(this.Item1); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { // singleton tuples end with a trailing ',' diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs index 1ef16cb2d..4a2894f87 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs @@ -175,11 +175,6 @@ public Slice ToSlice() return FdbTuple.EncodeKey(this.Item1, this.Item2); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return "(" + FdbTuple.Stringify(this.Item1) + ", " + FdbTuple.Stringify(this.Item2) + ")"; diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs index 75b8d23e4..244df2c22 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs @@ -195,11 +195,6 @@ public Slice ToSlice() return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return new StringBuilder(32).Append('(') diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs index 45e3a3fd1..66fabb4c0 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs @@ -196,11 +196,6 @@ public Slice ToSlice() return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return new StringBuilder(48).Append('(') diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs index 94a9e47b0..fed8161e1 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs @@ -206,11 +206,6 @@ public Slice ToSlice() return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); } - Slice IFdbKey.ToFoundationDbKey() - { - return this.ToSlice(); - } - public override string ToString() { return new StringBuilder(48).Append('(') diff --git a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs b/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs index 28988b554..65316b3fe 100644 --- a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs @@ -37,7 +37,7 @@ namespace FoundationDB.Layers.Tuples /// Represents a Tuple of N elements [ImmutableObject(true)] [CannotApplyEqualityOperator] - public interface IFdbTuple : IEnumerable, IEquatable, IReadOnlyCollection, IFdbKey + public interface IFdbTuple : IEnumerable, IEquatable, IReadOnlyCollection #if !NET_4_0 , IReadOnlyList , System.Collections.IStructuralEquatable diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs index 1008c4a7c..09b0995c0 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs @@ -91,20 +91,6 @@ public IFdbDynamicSubspace this[Slice suffix] } } - /// Create a new subspace by adding a to the current subspace's prefix - /// Key that will be appended to the current prefix - /// New subspace whose prefix is the concatenation of the parent prefix, and the packed representation of - public IFdbDynamicSubspace this[IFdbKey key] - { - [ContractAnnotation("null => halt; notnull => notnull")] - get - { - if (key == null) throw new ArgumentNullException("key"); - var packed = key.ToFoundationDbKey(); - return this[packed]; - } - } - public IFdbDynamicSubspace this[IFdbTuple tuple] { [ContractAnnotation("null => halt; notnull => notnull")] diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/FdbSubspace.cs index b09e0596f..e31ca3858 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspace.cs @@ -38,7 +38,7 @@ namespace FoundationDB.Client /// Adds a prefix on every keys, to group them inside a common subspace [PublicAPI] - public class FdbSubspace : IFdbSubspace, IFdbKey, IEquatable, IComparable + public class FdbSubspace : IFdbSubspace, IEquatable, IComparable { /// Empty subspace, that does not add any prefix to the keys public static readonly IFdbSubspace Empty = new FdbSubspace(Slice.Empty); @@ -60,7 +60,7 @@ protected FdbSubspace([NotNull] IFdbSubspace copy) { Contract.NotNull(copy, nameof(copy)); var sub = copy as FdbSubspace; - Slice key = sub != null ? sub.m_rawPrefix : copy.ToFoundationDbKey(); + Slice key = sub != null ? sub.m_rawPrefix : copy.Key; if (key.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(copy)); m_rawPrefix = key; } @@ -94,13 +94,6 @@ public static IFdbSubspace Create(Slice slice) return new FdbDynamicSubspace(slice, TypeSystem.Default.GetDynamicEncoder()); } - public static IFdbSubspace Create([NotNull] TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return new FdbSubspace(key.ToFoundationDbKey()); - } - /// Create a new Subspace using a binary key as the prefix /// Prefix of the new subspace /// Type System used to encode the keys of this subspace @@ -123,22 +116,6 @@ public static IFdbDynamicSubspace CreateDynamic(Slice slice, [NotNull] IDynamicK return new FdbDynamicSubspace(slice, encoder); } - public static IFdbDynamicSubspace CreateDynamic([NotNull] TKey key, IFdbKeyEncoding encoding = null) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new FdbDynamicSubspace(key.ToFoundationDbKey(), encoder); - } - - public static IFdbDynamicSubspace CreateDynamic([NotNull] TKey key, IDynamicKeyEncoder encoder) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - if (encoder == null) throw new ArgumentNullException("encoder"); - return new FdbDynamicSubspace(key.ToFoundationDbKey(), encoder); - } - /// Create a new Subspace using a tuples as the prefix /// Tuple that represents the prefix of the new subspace /// Optional type encoding used by this subspace. @@ -261,7 +238,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subsp [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(subspace.Key, true, encoder); } @@ -279,7 +256,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubs [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(subspace.Key, true, encoder); } @@ -297,7 +274,7 @@ public static IFdbEncoderSubspace CopyEncoder([NotNull] [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(subspace.Key, true, encoder); } @@ -315,21 +292,12 @@ public static IFdbEncoderSubspace CopyEncoder([N [Pure, NotNull] public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(encoder, nameof(encoder)); return new FdbEncoderSubspace(subspace.Key, true, encoder); } #endregion - #region IFdbKey... - - Slice IFdbKey.ToFoundationDbKey() - { - return GetKeyPrefix(); - } - - #endregion - #region IFdbSubspace... /// Returns the raw prefix of this subspace @@ -363,27 +331,11 @@ public virtual KeyRange ToRange(Slice suffix) return KeyRange.StartsWith(ConcatKey(suffix)); } - public virtual KeyRange ToRange(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - return KeyRange.StartsWith(ConcatKey(key.ToFoundationDbKey())); - } - public IFdbSubspace this[Slice suffix] { get { return CreateChildren(suffix); } } - public IFdbSubspace this[IFdbKey key] - { - get - { - if (key == null) throw new ArgumentNullException("key"); - return CreateChildren(key.ToFoundationDbKey()); - } - } - /// Tests whether the specified starts with this Subspace's prefix, indicating that the Subspace logically contains . /// The key to be tested /// The key Slice.Nil is not contained by any Subspace, so subspace.Contains(Slice.Nil) will always return false @@ -400,14 +352,6 @@ public Slice ConcatKey(Slice suffix) return GetKeyPrefix().Concat(suffix); } - public Slice ConcatKey(TKey key) - where TKey : IFdbKey - { - if (key == null) throw new ArgumentNullException("key"); - var suffix = key.ToFoundationDbKey(); - return GetKeyPrefix().Concat(suffix); - } - /// Merge an array of keys with the subspace's prefix, all sharing the same buffer /// Array of keys to pack /// Array of slices (for all keys) that share the same underlying buffer @@ -418,17 +362,6 @@ public Slice[] ConcatKeys(IEnumerable keys) return Slice.ConcatRange(GetKeyPrefix(), keys); } - /// Merge an array of keys with the subspace's prefix, all sharing the same buffer - /// Array of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - public Slice[] ConcatKeys(IEnumerable keys) - where TKey : IFdbKey - { - if (keys == null) throw new ArgumentNullException("keys"); - //REVIEW: what to do with keys that are Slice.Nil ? - return Slice.ConcatRange(GetKeyPrefix(), keys.Select(key => key.ToFoundationDbKey())); - } - /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace /// Complete key that contains the current subspace prefix, and a binary suffix /// If true, verify that is inside the bounds of the subspace @@ -523,7 +456,7 @@ public int CompareTo(IFdbSubspace other) if (sub != null) return this.InternalKey.CompareTo(sub.InternalKey); else - return this.InternalKey.CompareTo(other.ToFoundationDbKey()); + return this.InternalKey.CompareTo(other.Key); } /// Test if both subspaces have the same prefix @@ -535,7 +468,7 @@ public bool Equals(IFdbSubspace other) if (sub != null) return this.InternalKey.Equals(sub.InternalKey); else - return this.InternalKey.Equals(other.ToFoundationDbKey()); + return this.InternalKey.Equals(other.Key); } /// Test if an object is a subspace with the same prefix diff --git a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs index d3b559b41..5de658b91 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs @@ -165,7 +165,7 @@ public static void ClearRange(this IFdbTransaction trans, [NotNull] IFdbSubspace Contract.Requires(trans != null && subspace != null); //BUGBUG: should we call subspace.ToRange() ? - trans.ClearRange(KeyRange.StartsWith(subspace.ToFoundationDbKey())); + trans.ClearRange(subspace.ToRange()); } /// Clear the entire content of a subspace @@ -184,19 +184,7 @@ public static FdbRangeQuery> GetRangeStartsWith(this //REVIEW: should we remove this method? Contract.Requires(trans != null && subspace != null); - return trans.GetRange(KeyRange.StartsWith(subspace.ToFoundationDbKey()), options); - } - - /// Tests whether the specified starts with this Subspace's prefix, indicating that the Subspace logically contains . - /// - /// The key to be tested - /// If is null - public static bool Contains([NotNull] this IFdbSubspace subspace, [NotNull] TKey key) - where TKey : IFdbKey - { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (key == null) throw new ArgumentNullException("key"); - return subspace.Contains(key.ToFoundationDbKey()); + return trans.GetRange(subspace.ToRange(), options); } } diff --git a/FoundationDB.Client/Subspaces/IFdbSubspace.cs b/FoundationDB.Client/Subspaces/IFdbSubspace.cs index 34a6ada06..596168246 100644 --- a/FoundationDB.Client/Subspaces/IFdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbSubspace.cs @@ -33,7 +33,7 @@ namespace FoundationDB.Client using System.Collections.Generic; [PublicAPI] - public interface IFdbSubspace : IFdbKey + public interface IFdbSubspace { // This interface helps solve some type resolution ambiguities at compile time between types that all implement IFdbKey but have different semantics for partitionning and concatenation @@ -51,21 +51,11 @@ public interface IFdbSubspace : IFdbKey [Pure] KeyRange ToRange(Slice suffix); - /// Return a key range that contains all the keys under a serializable key in this subspace - /// Return the range: (this.Key + key.ToFoundationDbKey()) <= x <= Increment(this.Key + key.ToFoundationDbKey()) - [Pure] - KeyRange ToRange([NotNull] TKey key) where TKey : IFdbKey; - /// Create a new subspace by adding a suffix to the key of the current subspace. /// Binary suffix that will be appended to the current prefix /// New subspace whose prefix is the concatenation of the parent prefix, and IFdbSubspace this[Slice suffix] { [Pure, NotNull] get; } - /// Create a new subspace by adding a suffix to the key of the current subspace. - /// Item that can serialize itself into a binary suffix, that will be appended to the current subspace's prefix - /// New subspace whose prefix is the concatenation of the parent prefix, and - IFdbSubspace this[[NotNull] IFdbKey key] { [Pure, NotNull] get; } - /// Test if a key is inside the range of keys logically contained by this subspace /// Key to test /// True if the key can exist inside the current subspace. @@ -85,24 +75,12 @@ public interface IFdbSubspace : IFdbKey [Pure] Slice ConcatKey(Slice suffix); - /// Return the key that is composed of the subspace's prefix and a serializable key - /// Item that can serialize itself into a binary suffix, that will be appended to the current prefix - /// Full binary key - [Pure] - Slice ConcatKey([NotNull] TKey key) where TKey : IFdbKey; - /// Concatenate a batch of keys under this subspace /// List of suffixes to process /// Array of which is equivalent to calling on each entry in [Pure, NotNull] Slice[] ConcatKeys([NotNull] IEnumerable suffixes); - /// Concatenate a batch of serializable keys under this subspace - /// List of serializable keys to process - /// Array of which is equivalent to calling on each entry in - [Pure, NotNull] - Slice[] ConcatKeys([NotNull, ItemNotNull] IEnumerable keys) where TKey : IFdbKey; - /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace /// Complete key that contains the current subspace prefix, and a binary suffix /// If true, verify that is inside the bounds of the subspace diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index cce139c57..2891409bf 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -75,13 +75,13 @@ public FdbMultiMap(IFdbSubspace subspace, bool allowNegativeValues, ICompositeKe #region Public Properties... /// Subspace used as a prefix for all items in this map - public IFdbSubspace Subspace { [NotNull] get; private set; } + public IFdbSubspace Subspace { [NotNull] get; } /// If true, allow negative or zero values to stay in the map. - public bool AllowNegativeValues { get; private set; } + public bool AllowNegativeValues { get; } /// Subspace used to encoded the keys for the items - protected IFdbEncoderSubspace Location { [NotNull] get; private set; } + protected IFdbEncoderSubspace Location { [NotNull] get; } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index 80943b34d..cd76e0383 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -98,7 +98,7 @@ public async Task InsertAsync([NotNull] IFdbTransaction trans, Slice key) if ((keyHash & ((1 << (level * LEVEL_FAN_POW)) - 1)) != 0) { //Console.WriteLine("> [" + level + "] Incrementing previous key: " + FdbKey.Dump(prevKey)); - trans.AtomicAdd(this.Subspace.Partition.ByKey(level, prevKey), EncodeCount(1)); + trans.AtomicAdd(this.Subspace.Keys.Encode(level, prevKey), EncodeCount(1)); } else { @@ -138,7 +138,7 @@ public async Task EraseAsync([NotNull] IFdbTransaction trans, Slice key) for (int level = 0; level < MAX_LEVELS; level++) { // This could be optimized with hash - var k = this.Subspace.Partition.ByKey(level, key); + var k = this.Subspace.Keys.Encode(level, key); var c = await trans.GetAsync(k).ConfigureAwait(false); if (c.HasValue) trans.Clear(k); if (level == 0) continue; diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index fad3168b6..446775e10 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -322,7 +322,7 @@ await db.ReadWriteAsync( var prefix = this.TaskStore.Partition.ByKey(msg.Id); //TODO: replace this with a get_range ? var data = await tr.GetValuesAsync(new [] { - prefix.ToFoundationDbKey(), + prefix.Key, prefix.Keys.Encode(TASK_META_SCHEDULED) }).ConfigureAwait(false); diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index 3a69eefff..356b7f760 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -180,7 +180,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, FdbSubspace.Create(FdbTuple.Create(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, FdbSubspace.Create(FdbTuple.EncodeKey(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index 4de3fed67..24265b65f 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -270,7 +270,7 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", FdbSubspace.Create(FdbTuple.Create("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", FdbSubspace.Create(FdbTuple.EncodeKey("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index d0e0f40b8..cc7012231 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -256,10 +256,10 @@ public void Test_KeyRange_Test() Assert.That(range.Test(Slice.FromAscii("\xFF"), endIncluded: true), Is.EqualTo(AFTER)); range = KeyRange.Create(FdbTuple.EncodeKey("A"), FdbTuple.EncodeKey("Z")); - Assert.That(range.Test(FdbTuple.Create("@")), Is.EqualTo((BEFORE))); - Assert.That(range.Test(FdbTuple.Create("A")), Is.EqualTo((INSIDE))); - Assert.That(range.Test(FdbTuple.Create("Z")), Is.EqualTo((AFTER))); - Assert.That(range.Test(FdbTuple.Create("Z"), endIncluded: true), Is.EqualTo(INSIDE)); + Assert.That(range.Test(FdbTuple.EncodeKey("@")), Is.EqualTo((BEFORE))); + Assert.That(range.Test(FdbTuple.EncodeKey("A")), Is.EqualTo((INSIDE))); + Assert.That(range.Test(FdbTuple.EncodeKey("Z")), Is.EqualTo((AFTER))); + Assert.That(range.Test(FdbTuple.EncodeKey("Z"), endIncluded: true), Is.EqualTo(INSIDE)); } [Test] diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 1a2218329..8c47a0c11 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -874,11 +874,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K // Key shouldFail(() => { var _ = partition.Key; }); - // ToFoundationDBKey - shouldFail(() => ((IFdbKey)partition).ToFoundationDbKey()); - // Contains - shouldFail(() => partition.Contains(subdir)); shouldFail(() => partition.Contains(barKey)); // Extract / ExtractAndCheck / BoundCheck @@ -900,12 +896,10 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => { var _ = partition[Slice.FromString("hello")]; }); shouldFail(() => { var _ = partition[location.Key]; }); - shouldFail(() => { var _ = partition[location]; }); shouldFail(() => partition.ToRange()); shouldFail(() => partition.ToRange(Slice.FromString("hello"))); shouldFail(() => partition.ToRange(FdbTuple.EncodeKey("hello"))); - shouldFail(() => partition.ToRange(location)); // Tuples diff --git a/FoundationDB.Tests/Layers/MultiMapFacts.cs b/FoundationDB.Tests/Layers/MultiMapFacts.cs index af11ff7ee..e716ceb88 100644 --- a/FoundationDB.Tests/Layers/MultiMapFacts.cs +++ b/FoundationDB.Tests/Layers/MultiMapFacts.cs @@ -89,7 +89,8 @@ public async Task Test_FdbMultiMap_Read_Write_Delete() // directly read the value, behind the table's back using (var tr = db.BeginTransaction(this.Cancellation)) { - var value = await tr.GetAsync(map.Subspace[FdbTuple.Create("hello", "world")]); + var loc = map.Subspace.Using(TypeSystem.Tuples); + var value = await tr.GetAsync(loc.Keys.Encode("hello", "world")); Assert.That(value, Is.Not.EqualTo(Slice.Nil)); Assert.That(value.ToInt64(), Is.EqualTo(1)); } @@ -112,7 +113,8 @@ public async Task Test_FdbMultiMap_Read_Write_Delete() Assert.That(count, Is.Null); // also check directly - var data = await tr.GetAsync(map.Subspace[FdbTuple.Create("hello", "world")]); + var loc = map.Subspace.Using(TypeSystem.Tuples); + var data = await tr.GetAsync(loc.Keys.Encode("hello", "world")); Assert.That(data, Is.EqualTo(Slice.Nil)); } diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index 93e38037d..812cf6d45 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -40,8 +40,8 @@ namespace FoundationDB.Linq.Expressions.Tests public class FdbQueryExpressionFacts { - private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", FdbSubspace.Create(FdbTuple.Create("Foos", 1))); - private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", FdbSubspace.Create(FdbTuple.Create("Foos", 2))); + private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", FdbSubspace.Create(FdbTuple.EncodeKey("Foos", 1))); + private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", FdbSubspace.Create(FdbTuple.EncodeKey("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index fb7b54de9..b95a506a2 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -114,15 +114,6 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() //Assert.That(() => FdbSubspace.Create(FdbKey.Directory).Partition[Slice.Nil], Throws.ArgumentException); } - [Test] - public void Test_Cannot_Create_Or_Partition_Subspace_With_Null_Tuple() - { - Assert.That(() => FdbSubspace.Create(default(IFdbTuple)), Throws.InstanceOf()); - //FIXME: typed subspaces refactoring ! - //Assert.That(() => FdbSubspace.Empty.Partition[default(IFdbTuple)], Throws.InstanceOf()); - //Assert.That(() => FdbSubspace.Create(FdbKey.Directory).Partition[default(IFdbTuple)], Throws.InstanceOf()); - } - [Test] [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index 6e4093684..fd4844a32 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -85,7 +85,7 @@ public static async Task GetCleanDirectory([NotNull] IFdbD public static async Task DumpSubspace([NotNull] IFdbDatabase db, [NotNull] IFdbSubspace subspace, CancellationToken ct) { Assert.That(db, Is.Not.Null); - Assert.That(db.GlobalSpace.Contains(subspace.ToFoundationDbKey()), Is.True, "Using a location outside of the test database partition!!! This is probably a bug in the test..."); + Assert.That(db.GlobalSpace.Contains(subspace.Key), Is.True, "Using a location outside of the test database partition!!! This is probably a bug in the test..."); // do not log db = db.WithoutLogging(); @@ -103,7 +103,7 @@ public static async Task DumpSubspace([NotNull] IFdbReadOnlyTransaction tr, [Not Console.WriteLine("Dumping content of subspace " + subspace.ToString() + " :"); int count = 0; await tr - .GetRange(KeyRange.StartsWith(subspace.ToFoundationDbKey())) + .GetRange(KeyRange.StartsWith(subspace.Key)) .ForEachAsync((kvp) => { var key = subspace.ExtractKey(kvp.Key, boundCheck: true); From 678e17c8921436054d0a417f161ed9a6d52f2b20 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 15:38:25 +0200 Subject: [PATCH 061/153] Code cleanup --- Common/VersionInfo.cs | 2 - .../Documents/FdbDocumentCollection.cs | 51 ++++++++--------- .../Documents/FdbDocumentHandlers.cs | 14 ++--- .../Documents/FdbHashSetCollection.cs | 56 +++++++++---------- .../Indexes/Bitmaps/BitRange.cs | 11 +--- .../Indexes/Bitmaps/CompressedBitmap.cs | 25 ++++----- .../Bitmaps/CompressedBitmapBitView.cs | 6 +- .../Bitmaps/CompressedBitmapBuilder.cs | 29 ++++------ .../Bitmaps/CompressedBitmapWordIterator.cs | 10 +--- .../Indexes/Bitmaps/CompressedBitmapWriter.cs | 2 +- .../Indexes/Bitmaps/CompressedWord.cs | 27 ++------- .../Indexes/Bitmaps/WordAlignHybridCoding.cs | 32 +++++------ .../Indexes/FdbCompressedBitmapIndex.cs | 24 ++++---- .../Messaging/FdbWorkerPool.cs | 36 ++++++------ .../Messaging/WorkerPoolTest.cs | 2 +- .../Properties/AssemblyInfo.cs | 1 - .../Benchmarks/BenchRunner.cs | 26 +++------ FoundationDB.Samples/Benchmarks/LeakTest.cs | 16 +++--- .../MessageQueue/MessageQueueRunner.cs | 30 ++++------ FoundationDB.Samples/PerfCounters.cs | 19 +------ FoundationDB.Samples/Program.cs | 30 +++++----- .../Properties/AssemblyInfo.cs | 1 - .../Tutorials/ClassScheduling.cs | 14 ++--- FoundationDB.Samples/Utils/Sampler.cs | 10 +--- 24 files changed, 194 insertions(+), 280 deletions(-) diff --git a/Common/VersionInfo.cs b/Common/VersionInfo.cs index 5262018c7..6d4a21662 100644 --- a/Common/VersionInfo.cs +++ b/Common/VersionInfo.cs @@ -27,8 +27,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion using System.Reflection; -using System.Runtime.CompilerServices; -using System.Runtime.InteropServices; [assembly: AssemblyCompany("Doxense")] [assembly: AssemblyProduct("FoundationDB.Client")] diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index bc611bffd..3ddc9c587 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -28,16 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Documents { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using System; using System.Collections.Generic; using System.Linq; - using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; /// Represents a collection of dictionaries of fields. public class FdbDocumentCollection @@ -52,10 +47,10 @@ public FdbDocumentCollection(FdbSubspace subspace, Func selector public FdbDocumentCollection(FdbSubspace subspace, Func selector, ICompositeKeyEncoder keyEncoder, IValueEncoder valueEncoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (selector == null) throw new ArgumentNullException("selector"); - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); - if (valueEncoder == null) throw new ArgumentNullException("valueEncoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); + if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); this.Subspace = subspace; this.IdSelector = selector; @@ -80,15 +75,15 @@ protected virtual TDocument DecodeParts(List parts) } /// Subspace used as a prefix for all hashsets in this collection - public FdbSubspace Subspace { get; private set; } + public FdbSubspace Subspace { get; } - protected IFdbEncoderSubspace Location { get; private set; } + protected IFdbEncoderSubspace Location { get; } /// Encoder that packs/unpacks the documents - public IValueEncoder ValueEncoder { get; private set; } + public IValueEncoder ValueEncoder { get; } /// Lambda function used to extract the ID from a document - public Func IdSelector { get; private set; } + public Func IdSelector { get; } /// Maximum size of a document chunk (1 MB by default) public int ChunkSize { get; private set; } @@ -96,8 +91,8 @@ protected virtual TDocument DecodeParts(List parts) /// Insert a new document in the collection public void Insert(IFdbTransaction trans, TDocument document) { - if (trans == null) throw new ArgumentNullException("trans"); - if (document == null) throw new ArgumentNullException("document"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (document == null) throw new ArgumentNullException(nameof(document)); var id = this.IdSelector(document); if (id == null) throw new InvalidOperationException("Cannot insert a document with a null identifier"); @@ -141,8 +136,8 @@ public void Insert(IFdbTransaction trans, TDocument document) /// public async Task LoadAsync(IFdbReadOnlyTransaction trans, TId id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); // only for ref types + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); // only for ref types var parts = await LoadPartsAsync(trans, id).ConfigureAwait(false); @@ -155,8 +150,8 @@ public async Task LoadAsync(IFdbReadOnlyTransaction trans, TId id) /// public async Task> LoadMultipleAsync(IFdbReadOnlyTransaction trans, IEnumerable ids) { - if (trans == null) throw new ArgumentNullException("trans"); - if (ids == null) throw new ArgumentNullException("ids"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (ids == null) throw new ArgumentNullException(nameof(ids)); var results = await Task.WhenAll(ids.Select(id => LoadPartsAsync(trans, id))); @@ -168,8 +163,8 @@ public async Task> LoadMultipleAsync(IFdbReadOnlyTransaction tra /// public void Delete(IFdbTransaction trans, TId id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); var key = this.Location.Partial.Keys.Encode(id); trans.ClearRange(KeyRange.StartsWith(key)); @@ -181,8 +176,8 @@ public void Delete(IFdbTransaction trans, TId id) /// public void DeleteMultiple(IFdbTransaction trans, IEnumerable ids) { - if (trans == null) throw new ArgumentNullException("trans"); - if (ids == null) throw new ArgumentNullException("ids"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (ids == null) throw new ArgumentNullException(nameof(ids)); foreach (var key in this.Location.Partial.Keys.Encode(ids)) { @@ -195,8 +190,8 @@ public void DeleteMultiple(IFdbTransaction trans, IEnumerable ids) /// public void Delete(IFdbTransaction trans, TDocument document) { - if (trans == null) throw new ArgumentNullException("trans"); - if (document == null) throw new ArgumentNullException("document"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (document == null) throw new ArgumentNullException(nameof(document)); var id = this.IdSelector(document); if (id == null) throw new InvalidOperationException(); @@ -209,8 +204,8 @@ public void Delete(IFdbTransaction trans, TDocument document) /// public void DeleteMultiple(IFdbTransaction trans, IEnumerable documents) { - if (trans == null) throw new ArgumentNullException("trans"); - if (documents == null) throw new ArgumentNullException("documents"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (documents == null) throw new ArgumentNullException(nameof(documents)); DeleteMultiple(trans, documents.Select(document => this.IdSelector(document))); } diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs index 35a301f3b..062edc631 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs @@ -28,15 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Documents { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using System; using System.Collections.Generic; using System.Linq; - using System.Threading; - using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; /// Interface that defines a class that knows of to chop instances of into slices /// Type of documents @@ -95,11 +91,11 @@ public DictionaryHandler(string idName = null, IEqualityComparer compare private readonly IEqualityComparer m_keyComparer; - public string IdName { get; private set; } + public string IdName { get; } public KeyValuePair[] Split(List> document) { - if (document == null) throw new ArgumentNullException("document"); + if (document == null) throw new ArgumentNullException(nameof(document)); return document // don't include the id @@ -114,7 +110,7 @@ public KeyValuePair[] Split(List> Build(KeyValuePair[] parts) { - if (parts == null) throw new ArgumentNullException("parts"); + if (parts == null) throw new ArgumentNullException(nameof(parts)); var list = new List>(parts.Length); foreach(var part in parts) diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index bc4519b0f..97a2286ea 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -46,13 +46,13 @@ public class FdbHashSetCollection public FdbHashSetCollection(IFdbSubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace.Using(TypeSystem.Tuples); } /// Subspace used as a prefix for all hashsets in this collection - public IFdbDynamicSubspace Subspace { get; private set; } + public IFdbDynamicSubspace Subspace { get; } /// Returns the key prefix of an HashSet: (subspace, id, ) /// @@ -87,9 +87,9 @@ protected virtual string ParseFieldKey(IFdbTuple key) /// Value of the corresponding field, or Slice.Nil if it the hashset does not exist, or doesn't have a field with this name public Task GetValueAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id, string field) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (string.IsNullOrEmpty(field)) throw new ArgumentNullException("field"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (string.IsNullOrEmpty(field)) throw new ArgumentNullException(nameof(field)); return trans.GetAsync(GetFieldKey(id, field)); } @@ -100,8 +100,8 @@ public Task GetValueAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNu /// Dictionary containing, for all fields, their associated values public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); var prefix = GetKey(id); var results = new Dictionary(StringComparer.OrdinalIgnoreCase); @@ -125,9 +125,9 @@ await trans /// Dictionary containing the values of the selected fields, or Slice.Empty if that particular field does not exist. public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id, [NotNull] params string[] fields) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (fields == null) throw new ArgumentNullException("fields"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (fields == null) throw new ArgumentNullException(nameof(fields)); var keys = FdbTuple.EncodePrefixedKeys(GetKey(id), fields); @@ -148,22 +148,22 @@ public async Task> GetAsync([NotNull] IFdbReadOnlyTra public void SetValue(IFdbTransaction trans, IFdbTuple id, string field, Slice value) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (string.IsNullOrEmpty(field)) throw new ArgumentNullException("field"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (string.IsNullOrEmpty(field)) throw new ArgumentNullException(nameof(field)); trans.Set(GetFieldKey(id, field), value); } public void Set(IFdbTransaction trans, IFdbTuple id, IEnumerable> fields) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (fields == null) throw new ArgumentNullException("fields"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (fields == null) throw new ArgumentNullException(nameof(fields)); foreach (var field in fields) { - if (string.IsNullOrEmpty(field.Key)) throw new ArgumentException("Field cannot have an empty name", "fields"); + if (string.IsNullOrEmpty(field.Key)) throw new ArgumentException("Field cannot have an empty name", nameof(fields)); trans.Set(GetFieldKey(id, field.Key), field.Value); } } @@ -178,9 +178,9 @@ public void Set(IFdbTransaction trans, IFdbTuple id, IEnumerable public void DeleteValue(IFdbTransaction trans, IFdbTuple id, string field) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (string.IsNullOrEmpty(field)) throw new ArgumentNullException("field"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (string.IsNullOrEmpty(field)) throw new ArgumentNullException(nameof(field)); trans.Clear(GetFieldKey(id, field)); } @@ -189,8 +189,8 @@ public void DeleteValue(IFdbTransaction trans, IFdbTuple id, string field) /// public void Delete(IFdbTransaction trans, IFdbTuple id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); // remove all fields of the hash trans.ClearRange(KeyRange.StartsWith(GetKey(id))); @@ -202,13 +202,13 @@ public void Delete(IFdbTransaction trans, IFdbTuple id) /// public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); - if (fields == null) throw new ArgumentNullException("fields"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); + if (fields == null) throw new ArgumentNullException(nameof(fields)); foreach (var field in fields) { - if (string.IsNullOrEmpty(field)) throw new ArgumentException("Field cannot have an empty name", "fields"); + if (string.IsNullOrEmpty(field)) throw new ArgumentException("Field cannot have an empty name", nameof(fields)); trans.Clear(GetFieldKey(id, field)); } } @@ -226,8 +226,8 @@ public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) //note: As of Beta2, FDB does not have a fdb_get_range that only return the keys. That means that we will have to also read the values from the db, in order to just get the names of the fields :( //TODO: find a way to optimize this ? - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); var prefix = GetKey(id); var results = new Dictionary(StringComparer.OrdinalIgnoreCase); diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs index 54bf20d80..ff32e7a47 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs @@ -28,14 +28,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; using System; - using System.Collections.Generic; using System.Diagnostics; - using System.Globalization; - using System.Text; /// Bounds of a Compressed Bitmaps, from the Lowest Set Bit to the Highest Set Bit [DebuggerDisplay("[{Lowest}, {Highest}]")] @@ -44,14 +38,15 @@ public struct BitRange : IEquatable private const int LOWEST_UNDEFINED = 0; private const int HIGHEST_UNDEFINED = -1; - public static BitRange Empty { get { return new BitRange(LOWEST_UNDEFINED, HIGHEST_UNDEFINED); } } + public static BitRange Empty => new BitRange(LOWEST_UNDEFINED, HIGHEST_UNDEFINED); /// Index of the lowest bit that is set to 1 in the source Bitmap public readonly int Lowest; + /// Index of the highest bit that is set to 1 in the source Bitmap public readonly int Highest; - public bool IsEmpty { get { return this.Highest < this.Lowest; } } + public bool IsEmpty => this.Highest < this.Lowest; public BitRange(int lowest, int highest) { diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs index 622701aa5..47dea01a3 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs @@ -48,9 +48,9 @@ public sealed class CompressedBitmap : IEnumerable public CompressedBitmap(Slice data) { - if (data.IsNull) throw new ArgumentNullException("data"); - if (data.Count > 0 && data.Count < 8) throw new ArgumentException("A compressed bitmap must either be empty, or at least 8 bytes long", "data"); - if ((data.Count & 3) != 0) throw new ArgumentException("A compressed bitmap size must be a multiple of 4 bytes", "data"); + if (data.IsNull) throw new ArgumentNullException(nameof(data)); + if (data.Count > 0 && data.Count < 8) throw new ArgumentException("A compressed bitmap must either be empty, or at least 8 bytes long", nameof(data)); + if ((data.Count & 3) != 0) throw new ArgumentException("A compressed bitmap size must be a multiple of 4 bytes", nameof(data)); if (data.Count == 0) { @@ -66,7 +66,7 @@ public CompressedBitmap(Slice data) internal CompressedBitmap(Slice data, BitRange bounds) { - if (data.IsNull) throw new ArgumentNullException("data"); + if (data.IsNull) throw new ArgumentNullException(nameof(data)); if (data.Count == 0) { @@ -75,8 +75,8 @@ internal CompressedBitmap(Slice data, BitRange bounds) } else { - if ((data.Count & 3) != 0) throw new ArgumentException("A compressed bitmap size must be a multiple of 4 bytes", "data"); - if (data.Count < 4) throw new ArgumentException("A compressed bitmap must be at least 4 bytes long", "data"); + if ((data.Count & 3) != 0) throw new ArgumentException("A compressed bitmap size must be a multiple of 4 bytes", nameof(data)); + if (data.Count < 4) throw new ArgumentException("A compressed bitmap must be at least 4 bytes long", nameof(data)); m_data = data; m_bounds = bounds; } @@ -93,16 +93,13 @@ public CompressedBitmapBuilder ToBuilder() /// Gets the underlying buffer of the compressed bitmap /// The content of the buffer MUST NOT be modified directly - internal Slice Data { get { return m_data; } } + internal Slice Data => m_data; /// Gets the bounds of the compressed bitmap - public BitRange Bounds { get { return m_bounds; } } + public BitRange Bounds => m_bounds; /// Number of Data Words in the compressed bitmap - public int Count - { - get { return m_data.IsNullOrEmpty ? 0 : (m_data.Count >> 2) - 1; } - } + public int Count => m_data.IsNullOrEmpty ? 0 : (m_data.Count >> 2) - 1; /// Test if the specified bit is set /// Offset of the bit to test @@ -149,8 +146,8 @@ public int CountBits() internal static BitRange ComputeBounds(Slice data, int words = -1) { int count = data.Count; - if (count > 0 && count < 8) throw new ArgumentException("Bitmap buffer size is too small", "data"); - if ((count & 3) != 0) throw new ArgumentException("Bitmap buffer size must be a multiple of 4 bytes", "data"); + if (count > 0 && count < 8) throw new ArgumentException("Bitmap buffer size is too small", nameof(data)); + if ((count & 3) != 0) throw new ArgumentException("Bitmap buffer size must be a multiple of 4 bytes", nameof(data)); // if the bitmap is empty, return 0..0 if (count == 0) return BitRange.Empty; diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs index 06ebad93c..b7070b2e6 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs @@ -26,11 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Layers.Experimental.Indexing { - using JetBrains.Annotations; using System; using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// View that reads the indexes of all the set bits in a bitmap public class CompressedBitmapBitView : IEnumerable @@ -39,7 +41,7 @@ public class CompressedBitmapBitView : IEnumerable public CompressedBitmapBitView(CompressedBitmap bitmap) { - if (bitmap == null) throw new ArgumentNullException("bitmap"); + Contract.NotNull(bitmap, nameof(bitmap)); m_bitmap = bitmap; } diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs index 2ff0af68b..2933c36f5 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs @@ -37,31 +37,30 @@ namespace FoundationDB.Layers.Experimental.Indexing /// Builder of compressed bitmaps that can set or clear bits in a random order, in memory public sealed class CompressedBitmapBuilder { - private static readonly CompressedWord[] s_emptyArray = new CompressedWord[0]; /// Returns a new instance of an empty bitmap builder - public static CompressedBitmapBuilder Empty - { - get { return new CompressedBitmapBuilder(s_emptyArray, 0, BitRange.Empty); } - } + public static CompressedBitmapBuilder Empty => new CompressedBitmapBuilder(Array.Empty(), 0, BitRange.Empty); /// Buffer of compressed words private CompressedWord[] m_words; + /// Number of words used in the buffer private int m_size; + /// Index of the lowest bit that is set (or int.MaxValue) private int m_lowest; + /// Index of the highest bit that is set (or -1) private int m_highest; public CompressedBitmapBuilder(CompressedBitmap bitmap) { - if (bitmap == null) throw new ArgumentNullException("bitmap"); - if ((bitmap.Data.Count & 3) != 0) throw new ArgumentException("Bitmap's underlying buffer size should be a multiple of 4 bytes", "bitmap"); + if (bitmap == null) throw new ArgumentNullException(nameof(bitmap)); + if ((bitmap.Data.Count & 3) != 0) throw new ArgumentException("Bitmap's underlying buffer size should be a multiple of 4 bytes", nameof(bitmap)); if (bitmap.Count == 0) { - m_words = s_emptyArray; + m_words = Array.Empty(); var range = BitRange.Empty; m_lowest = range.Lowest; m_highest = range.Highest; @@ -110,10 +109,7 @@ internal static CompressedWord[] DecodeWords(Slice data, int size, BitRange boun } /// Returns the number of compressed words in the builder - public int Count - { - get { return m_size; } - } + public int Count => m_size; /// Compute the word index, and mask of a bit offset /// Bit offset (0-based) @@ -161,13 +157,8 @@ public void EnsureCapacity(int minSize) int newSize = SliceHelpers.NextPowerOfTwo(minSize); if (newSize < 0) newSize = minSize; if (newSize < 8) newSize = 8; - //Console.WriteLine("> resize buffer to {0} words", newSize); Array.Resize(ref m_words, newSize); } - //else - //{ - // Console.WriteLine("> buffer has enough capacity {0} for min size {1}", m_words.Length, minSize); - //} } /// Gets or sets the value of a bit in the bitmap. @@ -203,7 +194,7 @@ internal void Shift(int position, int count) /// True if the bit was changed from 0 to 1; or false if it was already set. public bool Set(int index) { - if (index < 0) throw new ArgumentException("Bit index cannot be less than zero.", "index"); + if (index < 0) throw new ArgumentException("Bit index cannot be less than zero.", nameof(index)); //Console.WriteLine("Set({0}) on {1}-words bitmap", index, m_size); @@ -288,7 +279,7 @@ public bool Set(int index) /// True if the bit was changed from 1 to 0; or false if it was already unset. public bool Clear(int index) { - if (index < 0) throw new ArgumentException("Bit index cannot be less than zero.", "index"); + if (index < 0) throw new ArgumentException("Bit index cannot be less than zero.", nameof(index)); uint mask; int wordIndex = GetWordIndex(index, out mask); diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs index 26d9e67b0..b987b634d 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs @@ -65,15 +65,9 @@ public bool MoveNext() return true; } - public CompressedWord Current - { - get { return new CompressedWord(m_current); } - } + public CompressedWord Current => new CompressedWord(m_current); - object System.Collections.IEnumerator.Current - { - get { return this.Current; } - } + object System.Collections.IEnumerator.Current => this.Current; public void Reset() { diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs index 9177fdc00..b3bbbd5c2 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs @@ -73,7 +73,7 @@ public CompressedBitmapWriter() public CompressedBitmapWriter(int capacity) : this(new SliceWriter(Math.Max(4 + capacity * 4, 20)), true) { - if (capacity < 0) throw new ArgumentOutOfRangeException("capacity"); + if (capacity < 0) throw new ArgumentOutOfRangeException(nameof(capacity)); } /// Create a new compressed bitmap writer, with a specific underlying buffer diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs index 252fc8e11..f018b8113 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs @@ -52,10 +52,7 @@ public CompressedWord(uint raw) /// Checks if this word is a literal /// Literal words have their MSB unset (0) - public bool IsLiteral - { - get { return (this.RawValue & WordAlignHybridEncoder.TYPE_MASK) == WordAlignHybridEncoder.BIT_TYPE_LITERAL; } - } + public bool IsLiteral => (this.RawValue & WordAlignHybridEncoder.TYPE_MASK) == WordAlignHybridEncoder.BIT_TYPE_LITERAL; /// Value of the 31-bit uncompressed word /// This word is repeated times in the in the uncompressed bitmap. @@ -88,31 +85,19 @@ public int WordValue /// Number of times the value is repeated in the uncompressed bitmap /// This value is 1 for literal words, and for filler words - public int WordCount - { - get { return this.IsLiteral ? 1 : this.FillCount; } - } + public int WordCount => this.IsLiteral ? 1 : this.FillCount; /// Value of a literal word /// Only valid if is true - public int Literal - { - get { return (int)(this.RawValue & WordAlignHybridEncoder.LITERAL_MASK); } - } + public int Literal => (int)(this.RawValue & WordAlignHybridEncoder.LITERAL_MASK); /// Value of the fill bit (either 0 or 1) /// Only valid if is false - public int FillBit - { - get { return (int)((this.RawValue & WordAlignHybridEncoder.FILL_MASK) >> WordAlignHybridEncoder.FILL_SHIFT); } - } + public int FillBit => (int)((this.RawValue & WordAlignHybridEncoder.FILL_MASK) >> WordAlignHybridEncoder.FILL_SHIFT); - /// Number of 31-bit words that are filled by + /// Number of 31-bit words that are filled by /// Only valid if is false - public int FillCount - { - get { return 1 + (int)(this.RawValue & WordAlignHybridEncoder.LENGTH_MASK); } - } + public int FillCount => 1 + (int)(this.RawValue & WordAlignHybridEncoder.LENGTH_MASK); /// Return the position of the lowest set bit, or -1 /// Index from 0 to 30 of the lowest set bit, or -1 if the word is empty diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs index 8b75fc8f0..cde6a4efe 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs @@ -107,7 +107,7 @@ public static class WordAlignHybridEncoder #endregion /// Helper class to read 31-bit words from an uncompressed source - private unsafe sealed class UncompressedWordReader + private sealed unsafe class UncompressedWordReader { /// Value returned by or when the input have less than 31 bits remaining public const uint NotEnough = 0xFFFFFFFF; @@ -241,10 +241,10 @@ public bool ReadIf(uint expected) } /// Returns the number of bits left in the register (0 if emtpy) - public int Bits { get { return m_bits; } } + public int Bits => m_bits; /// Returns the last word, padded with 0s - /// If there is at least one full word remaning + /// If there is at least one full word remaning public uint ReadLast() { if (m_bits >= 31) throw new InvalidOperationException("There are still words left to read in the source"); @@ -379,7 +379,7 @@ internal enum LogicalOperation [NotNull] public static CompressedBitmap Not([NotNull] this CompressedBitmap bitmap, int size) { - if (bitmap == null) throw new ArgumentNullException("bitmap"); + if (bitmap == null) throw new ArgumentNullException(nameof(bitmap)); // there is a high change that the final bitmap will have the same size, with an optional extra filler word at the end var writer = new CompressedBitmapWriter(bitmap.Count + 1); @@ -417,8 +417,8 @@ public static CompressedBitmap Not([NotNull] this CompressedBitmap bitmap, int s [NotNull] public static CompressedBitmap And([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0 || right.Count == 0) return CompressedBitmap.Empty; return CompressedBinaryExpression(left, right, LogicalOperation.And); @@ -431,8 +431,8 @@ public static CompressedBitmap And([NotNull] this CompressedBitmap left, [NotNul [NotNull] public static CompressedBitmap Or([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0) return right.Count == 0 ? CompressedBitmap.Empty : right; if (right.Count == 0) return left; @@ -446,8 +446,8 @@ public static CompressedBitmap Or([NotNull] this CompressedBitmap left, [NotNull [NotNull] public static CompressedBitmap Xor([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0) return right.Count == 0 ? CompressedBitmap.Empty : right; if (right.Count == 0) return left; @@ -461,8 +461,8 @@ public static CompressedBitmap Xor([NotNull] this CompressedBitmap left, [NotNul [NotNull] public static CompressedBitmap AndNot([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0 || right.Count == 0) return CompressedBitmap.Empty; return CompressedBinaryExpression(left, right, LogicalOperation.AndNot); @@ -475,8 +475,8 @@ public static CompressedBitmap AndNot([NotNull] this CompressedBitmap left, [Not [NotNull] public static CompressedBitmap OrNot([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0) return right.Count == 0 ? CompressedBitmap.Empty : right; if (right.Count == 0) return left; @@ -490,8 +490,8 @@ public static CompressedBitmap OrNot([NotNull] this CompressedBitmap left, [NotN [NotNull] public static CompressedBitmap XorNot([NotNull] this CompressedBitmap left, [NotNull] CompressedBitmap right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); if (left.Count == 0) return right.Count == 0 ? CompressedBitmap.Empty : right; if (right.Count == 0) return left; diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index cafdf3d70..175e60595 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense.Diagnostics.Contracts; + namespace FoundationDB.Layers.Experimental.Indexing { using FoundationDB.Client; @@ -50,9 +52,9 @@ public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace sub public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] IKeyEncoder encoder) { - if (name == null) throw new ArgumentNullException("name"); - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + Contract.NotNull(name, nameof(name)); + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); this.Name = name; this.Subspace = subspace; @@ -61,17 +63,17 @@ public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace sub this.Location = subspace.UsingEncoder(encoder); } - public string Name { [NotNull] get; private set; } + public string Name { [NotNull] get; } - public FdbSubspace Subspace { [NotNull] get; private set; } + public FdbSubspace Subspace { [NotNull] get; } - protected IFdbEncoderSubspace Location { [NotNull] get; private set; } + protected IFdbEncoderSubspace Location { [NotNull] get; } - public IEqualityComparer ValueComparer { [NotNull] get; private set; } + public IEqualityComparer ValueComparer { [NotNull] get; } /// If true, null values are inserted in the index. If false (default), they are ignored /// This has no effect if is not a reference type - public bool IndexNullValues { get; private set; } + public bool IndexNullValues { get; } /// Insert a newly created entity to the index /// Transaction to use @@ -80,7 +82,7 @@ public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace sub /// True if a value was inserted into the index; or false if is null and is false, or if this was already indexed at this . public async Task AddAsync([NotNull] IFdbTransaction trans, long id, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (this.IndexNullValues || value != null) { @@ -107,7 +109,7 @@ public async Task AddAsync([NotNull] IFdbTransaction trans, long id, TValu /// If and are identical, then nothing will be done. Otherwise, the old index value will be deleted and the new value will be added public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TValue newValue, TValue previousValue) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (!this.ValueComparer.Equals(newValue, previousValue)) { @@ -146,7 +148,7 @@ public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TV /// Previous value of the entity in the index public async Task RemoveAsync([NotNull] IFdbTransaction trans, long id, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); var key = this.Location.Keys.Encode(value); var data = await trans.GetAsync(key).ConfigureAwait(false); diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 446775e10..937fb81ed 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -26,19 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client; -using FoundationDB.Layers.Tuples; -using FoundationDB.Filters.Logging; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Security.Cryptography; -using System.Threading; -using System.Threading.Tasks; -using FoundationDB.Layers.Counters; - namespace FoundationDB.Layers.Messaging { + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Security.Cryptography; + using System.Threading; + using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Filters.Logging; + using FoundationDB.Layers.Counters; public class FdbWorkerMessage { @@ -64,17 +62,17 @@ public class FdbWorkerPool private readonly RandomNumberGenerator m_rng = RandomNumberGenerator.Create(); - public IFdbDynamicSubspace Subspace { get; private set; } + public IFdbDynamicSubspace Subspace { get; } - internal IFdbDynamicSubspace TaskStore { get; private set; } + internal IFdbDynamicSubspace TaskStore { get; } - internal IFdbDynamicSubspace IdleRing { get; private set; } + internal IFdbDynamicSubspace IdleRing { get; } - internal IFdbDynamicSubspace BusyRing { get; private set; } + internal IFdbDynamicSubspace BusyRing { get; } - internal IFdbDynamicSubspace UnassignedTaskRing { get; private set; } + internal IFdbDynamicSubspace UnassignedTaskRing { get; } - internal FdbCounterMap Counters { get; private set; } + internal FdbCounterMap Counters { get; } #region Profiling... @@ -110,7 +108,7 @@ public class FdbWorkerPool public FdbWorkerPool(IFdbSubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace.Using(TypeSystem.Tuples); @@ -201,7 +199,7 @@ private void ClearTask(IFdbTransaction tr, Slice taskId) /// public async Task ScheduleTaskAsync(IFdbRetryable db, Slice taskId, Slice taskBody, CancellationToken ct = default(CancellationToken)) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); var now = DateTime.UtcNow; await db.ReadWriteAsync(async (tr) => diff --git a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs index 3b88cd951..2cbeb5658 100644 --- a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs +++ b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs @@ -60,7 +60,7 @@ public void Main() private async Task RunAsync(IFdbDatabase db, IFdbDynamicSubspace location, CancellationToken ct, Action done, int N, int K, int W) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); StringBuilder sb = new StringBuilder(); diff --git a/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs b/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs index c4f33a5ad..e86b441dd 100644 --- a/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs +++ b/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs @@ -1,5 +1,4 @@ using System.Reflection; -using System.Runtime.CompilerServices; using System.Runtime.InteropServices; [assembly: AssemblyTitle("FoundationDB.Layers.Messaging")] diff --git a/FoundationDB.Samples/Benchmarks/BenchRunner.cs b/FoundationDB.Samples/Benchmarks/BenchRunner.cs index 9e1d67694..aca7925c4 100644 --- a/FoundationDB.Samples/Benchmarks/BenchRunner.cs +++ b/FoundationDB.Samples/Benchmarks/BenchRunner.cs @@ -2,22 +2,16 @@ namespace FoundationDB.Samples.Benchmarks { - using Doxense.Mathematics.Statistics; - using FoundationDB.Client; - using FoundationDB.Client.Native; - using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Messaging; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using System; - using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.IO; using System.Linq; - using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Mathematics.Statistics; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; public class BenchRunner : IAsyncTest { @@ -40,15 +34,15 @@ public BenchRunner(BenchMode mode, int value = 1) this.Histo = new RobustHistogram(); } - public string Name { get { return "Bench" + this.Mode.ToString(); } } + public string Name => "Bench" + this.Mode.ToString(); - public int Value { get; private set; } + public int Value { get; set; } - public BenchMode Mode { get; private set; } + public BenchMode Mode { get; } public IFdbDynamicSubspace Subspace { get; private set; } - public RobustHistogram Histo { get; private set; } + public RobustHistogram Histo { get; } /// @@ -132,10 +126,8 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) var w = await db.GetAndWatch(foo, ct); var v = w.Value; - if (v == bar) - v = barf; - else - v = bar; + // swap + v = v == bar ? barf : bar; await db.WriteAsync((tr) => tr.Set(foo, v), ct); diff --git a/FoundationDB.Samples/Benchmarks/LeakTest.cs b/FoundationDB.Samples/Benchmarks/LeakTest.cs index 54e180f30..1cc3cdec9 100644 --- a/FoundationDB.Samples/Benchmarks/LeakTest.cs +++ b/FoundationDB.Samples/Benchmarks/LeakTest.cs @@ -2,14 +2,13 @@ namespace FoundationDB.Samples.Benchmarks { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; using System; using System.IO; using System.Text; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Client.Utils; public class LeakTest : IAsyncTest { @@ -22,10 +21,13 @@ public LeakTest(int k, int m, int n, TimeSpan delay) this.Delay = delay; } - public int K { get; private set; } - public int M { get; private set; } - public int N { get; private set; } - public TimeSpan Delay { get; private set; } + public int K { get; } + + public int M { get; } + + public int N { get; } + + public TimeSpan Delay { get; } public IFdbDynamicSubspace Subspace { get; private set; } diff --git a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs index 3b3753f05..94e853aea 100644 --- a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs +++ b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs @@ -46,16 +46,19 @@ public MessageQueueRunner(string id, AgentRole role, TimeSpan delayMin, TimeSpan ); } - public string Id { get; private set; } - public AgentRole Role { get; private set; } - public TimeSpan DelayMin { get; private set; } - public TimeSpan DelayMax { get; private set; } + public string Id { get; } + + public AgentRole Role { get; } + + public TimeSpan DelayMin { get; } + + public TimeSpan DelayMax { get; } public FdbSubspace Subspace { get; private set; } public FdbWorkerPool WorkerPool { get; private set; } - public RobustTimeLine TimeLine { get; private set; } + public RobustTimeLine TimeLine { get; } /// /// Setup the initial state of the database @@ -76,18 +79,13 @@ public async Task RunProducer(IFdbDatabase db, CancellationToken ct) { int cnt = 0; - var rnd = new Random(123456); - - DateTime last = DateTime.Now; - - rnd = new Random(); + var rnd = new Random(); this.TimeLine.Start(); while (!ct.IsCancellationRequested) { int k = cnt++; Slice taskId = FdbTuple.EncodeKey(this.Id.GetHashCode(), k); - var ts = Stopwatch.GetTimestamp(); string msg = "Message #" + k + " from producer " + this.Id + " (" + DateTime.UtcNow.ToString("O") + ")"; var latency = Stopwatch.StartNew(); @@ -98,7 +96,7 @@ public async Task RunProducer(IFdbDatabase db, CancellationToken ct) this.TimeLine.Add(latency.Elapsed.TotalMilliseconds); TimeSpan delay = TimeSpan.FromTicks(rnd.Next((int)this.DelayMin.Ticks, (int)this.DelayMax.Ticks)); - await Task.Delay(delay).ConfigureAwait(false); + await Task.Delay(delay, ct).ConfigureAwait(false); } this.TimeLine.Stop(); @@ -110,14 +108,11 @@ public async Task RunConsumer(IFdbDatabase db, CancellationToken ct) { var rnd = new Random(); - DateTime last = DateTime.Now; int received = 0; this.TimeLine.Start(); await this.WorkerPool.RunWorkerAsync(db, async (msg, _ct) => { - long ts = Stopwatch.GetTimestamp(); - var latency = msg.Received - msg.Scheduled; Interlocked.Increment(ref received); @@ -125,9 +120,8 @@ await this.WorkerPool.RunWorkerAsync(db, async (msg, _ct) => this.TimeLine.Add(latency.TotalMilliseconds); - //Console.Write("."); TimeSpan delay = TimeSpan.FromTicks(rnd.Next((int)this.DelayMin.Ticks, (int)this.DelayMax.Ticks)); - await Task.Delay(delay).ConfigureAwait(false); + await Task.Delay(delay, ct).ConfigureAwait(false); }, ct); this.TimeLine.Stop(); @@ -184,7 +178,7 @@ await tr.Snapshot.GetRange(tasksLocation.Keys.ToRange()).ForEachAsync((kvp) => #region IAsyncTest... - public string Name { get { return "MessageQueueTest"; } } + public string Name => "MessageQueueTest"; public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) { diff --git a/FoundationDB.Samples/PerfCounters.cs b/FoundationDB.Samples/PerfCounters.cs index 3c90d03a5..7da654142 100644 --- a/FoundationDB.Samples/PerfCounters.cs +++ b/FoundationDB.Samples/PerfCounters.cs @@ -1,22 +1,9 @@ -using FoundationDB.Async; -using FoundationDB.Client; -using FoundationDB.Filters.Logging; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using FoundationDB.Samples.Benchmarks; -using FoundationDB.Samples.Tutorials; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Linq; -using System.Net; -using System.Text; -using System.Threading; -using System.Threading.Tasks; +//TODO: License for samples/tutorials ??? namespace FoundationDB.Samples { + using System; + using System.Diagnostics; public static class PerfCounters { diff --git a/FoundationDB.Samples/Program.cs b/FoundationDB.Samples/Program.cs index c2ba07494..076eadd2f 100644 --- a/FoundationDB.Samples/Program.cs +++ b/FoundationDB.Samples/Program.cs @@ -1,22 +1,18 @@ -using FoundationDB.Async; -using FoundationDB.Client; -using FoundationDB.Filters.Logging; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using FoundationDB.Samples.Benchmarks; -using FoundationDB.Samples.Tutorials; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Linq; -using System.Net; -using System.Text; -using System.Threading; -using System.Threading.Tasks; +//TODO: License for samples/tutorials ??? namespace FoundationDB.Samples { + using System; + using System.Diagnostics; + using System.IO; + using System.Linq; + using System.Threading; + using System.Threading.Tasks; + using FoundationDB.Async; + using FoundationDB.Client; + using FoundationDB.Filters.Logging; + using FoundationDB.Samples.Benchmarks; + using FoundationDB.Samples.Tutorials; public interface IAsyncTest { @@ -28,7 +24,7 @@ public class Program { private static IFdbDatabase Db; - private static bool LogEnabled = false; + private static bool LogEnabled; private static string CurrentDirectoryPath = "/"; static StreamWriter GetLogFile(string name) diff --git a/FoundationDB.Samples/Properties/AssemblyInfo.cs b/FoundationDB.Samples/Properties/AssemblyInfo.cs index 97fb8218a..e319b87d0 100644 --- a/FoundationDB.Samples/Properties/AssemblyInfo.cs +++ b/FoundationDB.Samples/Properties/AssemblyInfo.cs @@ -27,7 +27,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion using System.Reflection; -using System.Runtime.CompilerServices; using System.Runtime.InteropServices; [assembly: AssemblyTitle("FoundationDB.Samples")] diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index 7d51a8cba..3802fe6f9 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -2,19 +2,15 @@ namespace FoundationDB.Samples.Tutorials { - using FoundationDB.Client; - using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using System; using System.Collections.Generic; - using System.Diagnostics; - using System.Globalization; using System.IO; using System.Linq; - using System.Text; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using FoundationDB.Linq; public class ClassScheduling : IAsyncTest { @@ -32,7 +28,7 @@ public ClassScheduling() .ToArray(); } - public string[] ClassNames { get; private set; } + public string[] ClassNames { get; } public IFdbDynamicSubspace Subspace { get; private set; } @@ -204,7 +200,7 @@ public async Task IndecisiveStudent(IFdbDatabase db, int id, int ops, Cancellati #region IAsyncTest... - public string Name { get { return "ClassScheduling"; } } + public string Name => "ClassScheduling"; public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) { diff --git a/FoundationDB.Samples/Utils/Sampler.cs b/FoundationDB.Samples/Utils/Sampler.cs index bedfa0d12..1b6b3968e 100644 --- a/FoundationDB.Samples/Utils/Sampler.cs +++ b/FoundationDB.Samples/Utils/Sampler.cs @@ -2,19 +2,15 @@ namespace FoundationDB.Samples.Benchmarks { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using FoundationDB.Layers.Directories; using System; using System.Collections.Generic; + using System.Diagnostics; using System.Linq; using System.IO; - using System.Text; using System.Threading; using System.Threading.Tasks; using Doxense.Mathematics.Statistics; - using System.Diagnostics; + using FoundationDB.Client; public class SamplerTest : IAsyncTest { @@ -24,7 +20,7 @@ public SamplerTest(double ratio) this.Ratio = ratio; } - public double Ratio { get; private set; } + public double Ratio { get; } #region IAsyncTest... From 40dce5a3c6246015ca524fd7cc04d97a2e106399 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 15:40:47 +0200 Subject: [PATCH 062/153] Updated Copyright year to 2018 --- Common/VersionInfo.cs | 2 +- FoundationDB.Client/Async/AsyncBuffer.cs | 2 +- FoundationDB.Client/Async/AsyncCancellableMutex.cs | 2 +- FoundationDB.Client/Async/AsyncHelpers.cs | 2 +- FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs | 2 +- FoundationDB.Client/Async/AsyncPump.cs | 2 +- FoundationDB.Client/Async/AsyncTaskBuffer.cs | 2 +- FoundationDB.Client/Async/AsyncTransform.cs | 2 +- FoundationDB.Client/Async/AsyncTransformQueue.cs | 2 +- FoundationDB.Client/Async/IAsyncEnumerable.cs | 2 +- FoundationDB.Client/Async/IAsyncEnumerator.cs | 2 +- FoundationDB.Client/Async/IAsyncPump.cs | 2 +- FoundationDB.Client/Async/Maybe.cs | 2 +- FoundationDB.Client/Async/TaskHelpers.cs | 2 +- FoundationDB.Client/Converters/ConversionHelper.cs | 2 +- FoundationDB.Client/Converters/FdbConverters.cs | 2 +- FoundationDB.Client/Core/IFdbClusterHandler.cs | 2 +- FoundationDB.Client/Core/IFdbDatabaseHandler.cs | 2 +- FoundationDB.Client/Core/IFdbTransactionHandler.cs | 2 +- FoundationDB.Client/Fdb.Bulk.cs | 2 +- FoundationDB.Client/Fdb.Errors.cs | 2 +- FoundationDB.Client/Fdb.Options.cs | 2 +- FoundationDB.Client/Fdb.System.cs | 2 +- FoundationDB.Client/Fdb.cs | 2 +- FoundationDB.Client/FdbCluster.cs | 2 +- FoundationDB.Client/FdbDatabase.cs | 2 +- FoundationDB.Client/FdbDatabaseExtensions.cs | 2 +- FoundationDB.Client/FdbDatabaseOption.cs | 2 +- FoundationDB.Client/FdbIsolationLevel.cs | 2 +- FoundationDB.Client/FdbKey.cs | 2 +- FoundationDB.Client/FdbMutationType.cs | 2 +- FoundationDB.Client/FdbOperationContext.cs | 2 +- FoundationDB.Client/FdbRangeQuery.PagingIterator.cs | 2 +- FoundationDB.Client/FdbRangeQuery.ResultIterator.cs | 2 +- FoundationDB.Client/FdbRangeQuery.cs | 2 +- FoundationDB.Client/FdbTransaction.Snapshot.cs | 2 +- FoundationDB.Client/FdbTransaction.cs | 2 +- FoundationDB.Client/FdbTransactionExtensions.cs | 2 +- FoundationDB.Client/FdbTransactionMode.cs | 2 +- FoundationDB.Client/FdbWatch.cs | 2 +- FoundationDB.Client/Filters/FdbDatabaseFilter.cs | 2 +- FoundationDB.Client/Filters/FdbFilterExtensions.cs | 2 +- FoundationDB.Client/Filters/FdbTransactionFilter.cs | 2 +- FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs | 2 +- .../Filters/Logging/FdbTransactionLog.Commands.cs | 2 +- FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs | 2 +- FoundationDB.Client/IFdbCluster.cs | 2 +- FoundationDB.Client/IFdbDatabase.cs | 2 +- FoundationDB.Client/IFdbKey.cs | 2 +- FoundationDB.Client/IFdbReadOnlyRetryable.cs | 2 +- FoundationDB.Client/IFdbReadOnlyTransaction.cs | 2 +- FoundationDB.Client/IFdbRetryable.cs | 2 +- FoundationDB.Client/IFdbTransaction.cs | 2 +- FoundationDB.Client/KeySelector.cs | 2 +- .../Layers/Directories/FdbDirectoryExtensions.cs | 2 +- FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs | 2 +- FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs | 2 +- FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs | 2 +- .../Layers/Directories/FdbHighContentionAllocator.cs | 2 +- FoundationDB.Client/Layers/Directories/IFdbDirectory.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbListTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs | 2 +- FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs | 2 +- FoundationDB.Client/Layers/Tuples/IFdbTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/TupleReader.cs | 2 +- FoundationDB.Client/Layers/Tuples/TupleWriter.cs | 2 +- FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs | 2 +- .../Layers/Tuples/TypeSystem/TupleKeyEncoding.cs | 2 +- FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs | 2 +- FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs | 2 +- .../Linq/Expressions/AsyncTransformExpression.cs | 2 +- FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs | 2 +- FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs | 2 +- FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs | 2 +- FoundationDB.Client/Linq/FdbAsyncEnumerable.cs | 2 +- FoundationDB.Client/Linq/FdbAsyncSequence.cs | 2 +- FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs | 2 +- FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs | 2 +- FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs | 2 +- .../Linq/Iterators/FdbAnonymousAsyncGenerator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs | 2 +- .../Linq/Iterators/FdbParallelSelectAsyncIterator.cs | 2 +- .../Linq/Iterators/FdbSelectManyAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs | 2 +- .../Linq/Iterators/FdbWhereSelectAsyncIterator.cs | 2 +- FoundationDB.Client/Native/FdbFuture.cs | 2 +- FoundationDB.Client/Native/FdbFutureArray.cs | 2 +- FoundationDB.Client/Native/FdbFutureSingle.cs | 2 +- FoundationDB.Client/Native/FdbNative.cs | 2 +- FoundationDB.Client/Native/FdbNativeCluster.cs | 2 +- FoundationDB.Client/Native/FdbNativeDatabase.cs | 2 +- FoundationDB.Client/Native/FdbNativeTransaction.cs | 2 +- FoundationDB.Client/Native/Handles/ClusterHandle.cs | 2 +- FoundationDB.Client/Native/Handles/DatabaseHandle.cs | 2 +- FoundationDB.Client/Native/Handles/FdbSafeHandle.cs | 2 +- FoundationDB.Client/Native/Handles/FutureHandle.cs | 2 +- FoundationDB.Client/Native/Handles/TransactionHandle.cs | 2 +- FoundationDB.Client/Native/UnmanagedLibrary.cs | 2 +- FoundationDB.Client/Subspaces/Fdb.Directory.cs | 2 +- FoundationDB.Client/Subspaces/FdbDatabasePartition.cs | 2 +- FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs | 2 +- FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs | 2 +- FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs | 2 +- FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs | 2 +- FoundationDB.Client/Subspaces/FdbSubspace.cs | 2 +- FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs | 2 +- FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs | 2 +- FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs | 2 +- FoundationDB.Client/Subspaces/IFdbSubspace.cs | 2 +- FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs | 2 +- FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs | 2 +- FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs | 2 +- FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs | 2 +- FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs | 2 +- FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs | 2 +- FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs | 2 +- FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs | 2 +- FoundationDB.Client/TypeSystem/IValueEncoder.cs | 2 +- FoundationDB.Client/TypeSystem/TypeSystem.cs | 2 +- FoundationDB.Client/Utils/Logging.cs | 2 +- FoundationDB.Client/Utils/Slice.cs | 2 +- FoundationDB.Client/Utils/SliceBuffer.cs | 2 +- FoundationDB.Client/Utils/SliceComparer.cs | 2 +- FoundationDB.Client/Utils/SliceHelpers.cs | 2 +- FoundationDB.Client/Utils/SlicePairComparer.cs | 2 +- FoundationDB.Client/Utils/SliceReader.cs | 2 +- FoundationDB.Client/Utils/SliceStream.cs | 2 +- FoundationDB.Client/Utils/SliceWriter.cs | 2 +- FoundationDB.Client/Utils/Uuid128.cs | 2 +- FoundationDB.Client/Utils/Uuid64.cs | 2 +- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 2 +- FoundationDB.Layers.Common/Collections/FdbMap`2.cs | 2 +- FoundationDB.Layers.Common/Collections/FdbQueue`1.cs | 2 +- FoundationDB.Layers.Common/Collections/FdbRankedSet.cs | 2 +- FoundationDB.Layers.Common/Collections/FdbVector`1.cs | 2 +- FoundationDB.Layers.Common/Counters/FdbCounterMap.cs | 2 +- FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs | 2 +- FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs | 2 +- FoundationDB.Layers.Common/Interning/FdbStringIntern.cs | 2 +- FoundationDB.Layers.Common/Optional`1.cs | 2 +- FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmap.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmapBitView.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmapBuilder.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmapWordIterator.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmapWriter.cs | 2 +- .../Indexes/Bitmaps/CompressedWord.cs | 2 +- .../Indexes/Bitmaps/WordAlignHybridCoding.cs | 2 +- .../Indexes/FdbCompressedBitmapIndex.cs | 2 +- .../Expressions/FdbDebugStatementWriter.cs | 2 +- FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs | 2 +- .../Expressions/FdbQueryAsyncEnumerableExpression.cs | 2 +- FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs | 2 +- FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs | 2 +- .../Expressions/FdbQueryFilterExpression.cs | 2 +- .../Expressions/FdbQueryIndexLookupExpression.cs | 2 +- .../Expressions/FdbQueryIntersectExpression.cs | 2 +- FoundationDB.Linq.Providers/Expressions/FdbQueryNodeType.cs | 2 +- .../Expressions/FdbQueryRangeExpression.cs | 2 +- .../Expressions/FdbQuerySequenceExpression.cs | 2 +- .../Expressions/FdbQuerySingleExpression.cs | 2 +- .../Expressions/FdbQueryTransformExpression.cs | 2 +- FoundationDB.Linq.Providers/FdbAsyncQueryable.cs | 2 +- FoundationDB.Linq.Providers/Interfaces.cs | 2 +- FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs | 2 +- FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs | 2 +- FoundationDB.Linq.Providers/Providers/FdbAsyncSingleQuery.cs | 2 +- FoundationDB.Linq.Providers/Providers/FdbDatabaseQuery.cs | 2 +- FoundationDB.Linq.Providers/Providers/FdbIndexQuery`2.cs | 2 +- FoundationDB.Tests.Sandbox/Program.cs | 2 +- FoundationDB.Tests/Async/AsyncBufferFacts.cs | 2 +- .../Experimental/Indexing/CompressedBitmapsFacts.cs | 2 +- .../Experimental/Indexing/SuperSlowUncompressedBitmap.cs | 2 +- FoundationDB.Tests/Experimental/JsonNetCodec.cs | 2 +- FoundationDB.Tests/Experimental/ProtobufCodec.cs | 2 +- FoundationDB.Tests/FdbTest.cs | 2 +- FoundationDB.Tests/Layers/CounterFacts.cs | 2 +- FoundationDB.Tests/Layers/MapFacts.cs | 2 +- FoundationDB.Tests/Layers/MultiMapFacts.cs | 2 +- FoundationDB.Tests/Layers/RankedSetFacts.cs | 2 +- FoundationDB.Tests/Layers/StringInternFacts.cs | 2 +- FoundationDB.Tests/Layers/VectorFacts.cs | 2 +- FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs | 2 +- FoundationDB.Tests/RangeQueryFacts.cs | 2 +- FoundationDB.Tests/SubspaceFacts.cs | 2 +- FoundationDB.Tests/Utils/FdbConvertersFacts.cs | 2 +- FoundationDB.Tests/Utils/SliceComparerFacts.cs | 2 +- FoundationDB.Tests/Utils/SliceFacts.cs | 2 +- FoundationDB.Tests/Utils/SliceHelperFacts.cs | 2 +- FoundationDB.Tests/Utils/SliceStreamFacts.cs | 2 +- FoundationDB.Tests/Utils/SliceWriterFacts.cs | 2 +- LICENSE.md | 2 +- 224 files changed, 224 insertions(+), 224 deletions(-) diff --git a/Common/VersionInfo.cs b/Common/VersionInfo.cs index 6d4a21662..9a2fceed4 100644 --- a/Common/VersionInfo.cs +++ b/Common/VersionInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Async/AsyncBuffer.cs index 421ac78da..107e2b83e 100644 --- a/FoundationDB.Client/Async/AsyncBuffer.cs +++ b/FoundationDB.Client/Async/AsyncBuffer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncCancellableMutex.cs b/FoundationDB.Client/Async/AsyncCancellableMutex.cs index 2e4a16990..8fdd48d95 100644 --- a/FoundationDB.Client/Async/AsyncCancellableMutex.cs +++ b/FoundationDB.Client/Async/AsyncCancellableMutex.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Async/AsyncHelpers.cs index 4ea5fec01..1008deb1b 100644 --- a/FoundationDB.Client/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Async/AsyncHelpers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs index 83cdce814..68c10f003 100644 --- a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs +++ b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncPump.cs b/FoundationDB.Client/Async/AsyncPump.cs index a98dc6afe..98b93cf6f 100644 --- a/FoundationDB.Client/Async/AsyncPump.cs +++ b/FoundationDB.Client/Async/AsyncPump.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncTaskBuffer.cs b/FoundationDB.Client/Async/AsyncTaskBuffer.cs index 1c04ab714..6e89e2097 100644 --- a/FoundationDB.Client/Async/AsyncTaskBuffer.cs +++ b/FoundationDB.Client/Async/AsyncTaskBuffer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncTransform.cs b/FoundationDB.Client/Async/AsyncTransform.cs index 4c32628eb..1a76ab222 100644 --- a/FoundationDB.Client/Async/AsyncTransform.cs +++ b/FoundationDB.Client/Async/AsyncTransform.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Async/AsyncTransformQueue.cs index 76e527c3b..9a303aaaa 100644 --- a/FoundationDB.Client/Async/AsyncTransformQueue.cs +++ b/FoundationDB.Client/Async/AsyncTransformQueue.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncEnumerable.cs b/FoundationDB.Client/Async/IAsyncEnumerable.cs index 8564718ce..e4488b6c1 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerable.cs +++ b/FoundationDB.Client/Async/IAsyncEnumerable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncEnumerator.cs b/FoundationDB.Client/Async/IAsyncEnumerator.cs index f7ae14e81..40823eefa 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerator.cs +++ b/FoundationDB.Client/Async/IAsyncEnumerator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncPump.cs b/FoundationDB.Client/Async/IAsyncPump.cs index 37019e5ef..192ed854d 100644 --- a/FoundationDB.Client/Async/IAsyncPump.cs +++ b/FoundationDB.Client/Async/IAsyncPump.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/Maybe.cs b/FoundationDB.Client/Async/Maybe.cs index 55bb6df31..3846dc03c 100644 --- a/FoundationDB.Client/Async/Maybe.cs +++ b/FoundationDB.Client/Async/Maybe.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Async/TaskHelpers.cs index ce5ad2cd0..d1a5fe6ac 100644 --- a/FoundationDB.Client/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Async/TaskHelpers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Converters/ConversionHelper.cs b/FoundationDB.Client/Converters/ConversionHelper.cs index d7f28d736..b632ef649 100644 --- a/FoundationDB.Client/Converters/ConversionHelper.cs +++ b/FoundationDB.Client/Converters/ConversionHelper.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Converters/FdbConverters.cs b/FoundationDB.Client/Converters/FdbConverters.cs index dee9c62c2..505d4c3d6 100644 --- a/FoundationDB.Client/Converters/FdbConverters.cs +++ b/FoundationDB.Client/Converters/FdbConverters.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Core/IFdbClusterHandler.cs b/FoundationDB.Client/Core/IFdbClusterHandler.cs index e5bba6094..2d88d4608 100644 --- a/FoundationDB.Client/Core/IFdbClusterHandler.cs +++ b/FoundationDB.Client/Core/IFdbClusterHandler.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Core/IFdbDatabaseHandler.cs b/FoundationDB.Client/Core/IFdbDatabaseHandler.cs index 3859bd262..5b7bfcfa0 100644 --- a/FoundationDB.Client/Core/IFdbDatabaseHandler.cs +++ b/FoundationDB.Client/Core/IFdbDatabaseHandler.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 06954242d..84503c585 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index 83bdb073f..553638344 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Fdb.Errors.cs b/FoundationDB.Client/Fdb.Errors.cs index 9bac05602..7a9b78a77 100644 --- a/FoundationDB.Client/Fdb.Errors.cs +++ b/FoundationDB.Client/Fdb.Errors.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Fdb.Options.cs b/FoundationDB.Client/Fdb.Options.cs index 738b3c1ee..216b724a4 100644 --- a/FoundationDB.Client/Fdb.Options.cs +++ b/FoundationDB.Client/Fdb.Options.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 40b6733c4..75eb674e3 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 2cd256481..2f6562a5a 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbCluster.cs b/FoundationDB.Client/FdbCluster.cs index c5da090ae..d54d68d4d 100644 --- a/FoundationDB.Client/FdbCluster.cs +++ b/FoundationDB.Client/FdbCluster.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 268f49c40..c806bfc47 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index 6ed7c433b..a51fb2bfb 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbDatabaseOption.cs b/FoundationDB.Client/FdbDatabaseOption.cs index f11db6541..ccd519515 100644 --- a/FoundationDB.Client/FdbDatabaseOption.cs +++ b/FoundationDB.Client/FdbDatabaseOption.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbIsolationLevel.cs b/FoundationDB.Client/FdbIsolationLevel.cs index 80e3adf3c..d09c65f6b 100644 --- a/FoundationDB.Client/FdbIsolationLevel.cs +++ b/FoundationDB.Client/FdbIsolationLevel.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 6ed2571ea..21d8b5ccf 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbMutationType.cs b/FoundationDB.Client/FdbMutationType.cs index 645be1510..4899b5a61 100644 --- a/FoundationDB.Client/FdbMutationType.cs +++ b/FoundationDB.Client/FdbMutationType.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index 364a8a08e..3d1d6979d 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index c6f49647e..2dbb8da9a 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 2141c7250..415ce3dcb 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index 4d17ac9dc..07b950c23 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbTransaction.Snapshot.cs b/FoundationDB.Client/FdbTransaction.Snapshot.cs index 725a489e6..d114e6459 100644 --- a/FoundationDB.Client/FdbTransaction.Snapshot.cs +++ b/FoundationDB.Client/FdbTransaction.Snapshot.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 4707e08c2..e0fa7d1df 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index cf517d4db..c668661de 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbTransactionMode.cs b/FoundationDB.Client/FdbTransactionMode.cs index c94ec61e6..96e3490a7 100644 --- a/FoundationDB.Client/FdbTransactionMode.cs +++ b/FoundationDB.Client/FdbTransactionMode.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbWatch.cs b/FoundationDB.Client/FdbWatch.cs index e113cf397..d23aab4f1 100644 --- a/FoundationDB.Client/FdbWatch.cs +++ b/FoundationDB.Client/FdbWatch.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index f4431c11c..ed057a75e 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/FdbFilterExtensions.cs b/FoundationDB.Client/Filters/FdbFilterExtensions.cs index e93109a53..6d25aaab1 100644 --- a/FoundationDB.Client/Filters/FdbFilterExtensions.cs +++ b/FoundationDB.Client/Filters/FdbFilterExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/FdbTransactionFilter.cs b/FoundationDB.Client/Filters/FdbTransactionFilter.cs index d702f870f..6f6567a7f 100644 --- a/FoundationDB.Client/Filters/FdbTransactionFilter.cs +++ b/FoundationDB.Client/Filters/FdbTransactionFilter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index 982fd7c96..b0c127c74 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index 8ebccc07c..f9b32bb4b 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs index 497c215a3..c6c166072 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbCluster.cs b/FoundationDB.Client/IFdbCluster.cs index a2ee5bbe3..40e32dce8 100644 --- a/FoundationDB.Client/IFdbCluster.cs +++ b/FoundationDB.Client/IFdbCluster.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbDatabase.cs b/FoundationDB.Client/IFdbDatabase.cs index 7958b4d7a..85179208c 100644 --- a/FoundationDB.Client/IFdbDatabase.cs +++ b/FoundationDB.Client/IFdbDatabase.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbKey.cs b/FoundationDB.Client/IFdbKey.cs index 1b5d15d50..da2bec25d 100644 --- a/FoundationDB.Client/IFdbKey.cs +++ b/FoundationDB.Client/IFdbKey.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbReadOnlyRetryable.cs b/FoundationDB.Client/IFdbReadOnlyRetryable.cs index e303cfa7f..fc5b332f5 100644 --- a/FoundationDB.Client/IFdbReadOnlyRetryable.cs +++ b/FoundationDB.Client/IFdbReadOnlyRetryable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbReadOnlyTransaction.cs b/FoundationDB.Client/IFdbReadOnlyTransaction.cs index 9f65764af..1ed8ac0c9 100644 --- a/FoundationDB.Client/IFdbReadOnlyTransaction.cs +++ b/FoundationDB.Client/IFdbReadOnlyTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbRetryable.cs b/FoundationDB.Client/IFdbRetryable.cs index 015631b9f..a67396ac9 100644 --- a/FoundationDB.Client/IFdbRetryable.cs +++ b/FoundationDB.Client/IFdbRetryable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/IFdbTransaction.cs b/FoundationDB.Client/IFdbTransaction.cs index b56681168..911403fab 100644 --- a/FoundationDB.Client/IFdbTransaction.cs +++ b/FoundationDB.Client/IFdbTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index 51168d896..d670ebfba 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs index 2aea8bb71..06bba208a 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index ee0144da0..b8ff917cd 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 7ccc2fd6a..7554d1ace 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index 3a1cf408f..fb484f821 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs index 32cc1be74..7fbe97ffc 100644 --- a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs +++ b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Directories/IFdbDirectory.cs b/FoundationDB.Client/Layers/Directories/IFdbDirectory.cs index 927a13192..12319f5b4 100644 --- a/FoundationDB.Client/Layers/Directories/IFdbDirectory.cs +++ b/FoundationDB.Client/Layers/Directories/IFdbDirectory.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs index 1155b2116..a31ca1da6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs index 856b3378a..5952260f8 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs index 52223a753..114b3f97e 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs index 8d0b74743..e3366f7df 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs index 7ede05619..6d3ad31b3 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs index a0d233b16..5bc091469 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs index 5841128c7..8d383f5a9 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs index 18f377534..bfa27aae1 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs index f9bf577b1..859a3a363 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs index a75534672..7d053333a 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs index 67e5352d6..2da6f33b3 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs b/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs index 01962aaad..a5dda0f3c 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs index 965c0c1a0..341ebd58a 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs index dca0460f9..092a9ed15 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs index 3fd25bc45..d52438caa 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs index 1567fb8c4..d00cbb7ad 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs index 4a2894f87..75b91cca3 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs index 244df2c22..4262c2666 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs index 66fabb4c0..1adc807e6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs b/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs index fed8161e1..8e07e2583 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs b/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs index 65316b3fe..57e103f4b 100644 --- a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/TupleReader.cs b/FoundationDB.Client/Layers/Tuples/TupleReader.cs index 14e1b9895..0a2e83f78 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleReader.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleReader.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/TupleWriter.cs b/FoundationDB.Client/Layers/Tuples/TupleWriter.cs index 9d33d227a..093176d42 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleWriter.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleWriter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs index 1ef2ab739..2f0fc05b3 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs index 9db59e1f7..ca4b0baf0 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs b/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs index 18caf33d4..61f6089cb 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs +++ b/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs b/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs index f292a930f..8920e87b5 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs +++ b/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs b/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs index e35ac589c..c0207c4e9 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs +++ b/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs index 9cd1318f5..8df8c4673 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs index 488eef83e..ee02eedce 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs index 575fb0a5f..a79f810b7 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs index 342368a4b..c0ac7006a 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/FdbAsyncSequence.cs b/FoundationDB.Client/Linq/FdbAsyncSequence.cs index 0acedcd17..87ff3095b 100644 --- a/FoundationDB.Client/Linq/FdbAsyncSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncSequence.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs index e426f8a8d..fb258399d 100644 --- a/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs b/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs index 166b83334..1fe5794ee 100644 --- a/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs +++ b/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs b/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs index 97644cfdb..369e5b129 100644 --- a/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs +++ b/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs b/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs index 3f7af64b5..752f5c5d7 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs index 320a4b3f1..f84155d62 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs index 46aa7dccc..5ecd89b4f 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs index 544e6465c..767baf138 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs index ffafd8aef..b080e9672 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs index d44d9dc5b..473d7d5d2 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs index 802f62260..f565ff394 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs index 62e79a731..17af0110a 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs index ca2f3dadf..82f18ebd0 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs index 84da430f5..11ffbc1e9 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/FdbFuture.cs index e17fcd1bb..f54c925fb 100644 --- a/FoundationDB.Client/Native/FdbFuture.cs +++ b/FoundationDB.Client/Native/FdbFuture.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbFutureArray.cs b/FoundationDB.Client/Native/FdbFutureArray.cs index 0a7989478..3f6e3296e 100644 --- a/FoundationDB.Client/Native/FdbFutureArray.cs +++ b/FoundationDB.Client/Native/FdbFutureArray.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbFutureSingle.cs b/FoundationDB.Client/Native/FdbFutureSingle.cs index 43572ca71..1d1e3982b 100644 --- a/FoundationDB.Client/Native/FdbFutureSingle.cs +++ b/FoundationDB.Client/Native/FdbFutureSingle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 043628887..41f341305 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbNativeCluster.cs b/FoundationDB.Client/Native/FdbNativeCluster.cs index 1976acf77..0e28ae7d0 100644 --- a/FoundationDB.Client/Native/FdbNativeCluster.cs +++ b/FoundationDB.Client/Native/FdbNativeCluster.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbNativeDatabase.cs b/FoundationDB.Client/Native/FdbNativeDatabase.cs index 6427cac69..a996996f8 100644 --- a/FoundationDB.Client/Native/FdbNativeDatabase.cs +++ b/FoundationDB.Client/Native/FdbNativeDatabase.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index 49f74f24c..74174e25d 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/Handles/ClusterHandle.cs b/FoundationDB.Client/Native/Handles/ClusterHandle.cs index 9f5b74abe..925a46a6d 100644 --- a/FoundationDB.Client/Native/Handles/ClusterHandle.cs +++ b/FoundationDB.Client/Native/Handles/ClusterHandle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/Handles/DatabaseHandle.cs b/FoundationDB.Client/Native/Handles/DatabaseHandle.cs index d74aa286c..23507c172 100644 --- a/FoundationDB.Client/Native/Handles/DatabaseHandle.cs +++ b/FoundationDB.Client/Native/Handles/DatabaseHandle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/Handles/FdbSafeHandle.cs b/FoundationDB.Client/Native/Handles/FdbSafeHandle.cs index 2b329869d..9863a6ed6 100644 --- a/FoundationDB.Client/Native/Handles/FdbSafeHandle.cs +++ b/FoundationDB.Client/Native/Handles/FdbSafeHandle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/Handles/FutureHandle.cs b/FoundationDB.Client/Native/Handles/FutureHandle.cs index cf5c18674..20eac078d 100644 --- a/FoundationDB.Client/Native/Handles/FutureHandle.cs +++ b/FoundationDB.Client/Native/Handles/FutureHandle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/Handles/TransactionHandle.cs b/FoundationDB.Client/Native/Handles/TransactionHandle.cs index e4f80d0c3..643c34da4 100644 --- a/FoundationDB.Client/Native/Handles/TransactionHandle.cs +++ b/FoundationDB.Client/Native/Handles/TransactionHandle.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/UnmanagedLibrary.cs b/FoundationDB.Client/Native/UnmanagedLibrary.cs index 4696f3298..62659bbdd 100644 --- a/FoundationDB.Client/Native/UnmanagedLibrary.cs +++ b/FoundationDB.Client/Native/UnmanagedLibrary.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index 56fe70713..921fa72da 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs index c9f0ac156..25d811dcc 100644 --- a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs index 57db84a6b..1a5f4e229 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs index 857b8181d..3faee77b3 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs index 09b0995c0..489bcbac2 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs index 578eb57e7..6f7ea7a2a 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs index a0dacd538..53760d30a 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs index d855b4855..958972eea 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs index f3ede9f47..f4971e8d0 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs index 0e969fb6f..b33e6c3d9 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs index 0c99c2788..d56fe24c1 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs index a66e9d73a..cc90d8bff 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs index 5e60d3c36..7bcf0d137 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs index 1bcd94cd1..7099dcfa4 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs index 9e4af1197..d3c698a84 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs index 213c8cea4..5a642c0ab 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs index 3c28c1ad2..ff5c33e54 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/FdbSubspace.cs index e31ca3858..d9c8d01f3 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs index 5de658b91..af706bb87 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs index af1d657fd..2d2adc923 100644 --- a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs b/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs index 172af8455..6e650f6c5 100644 --- a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Subspaces/IFdbSubspace.cs b/FoundationDB.Client/Subspaces/IFdbSubspace.cs index 596168246..f32964658 100644 --- a/FoundationDB.Client/Subspaces/IFdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/IFdbSubspace.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index ae20aa552..ec1a4379f 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs b/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs index d323b24d1..6bdbc589b 100644 --- a/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs +++ b/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs index 4b2621897..3f90ad738 100644 --- a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs index bd58015e2..2fc38d5f1 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs b/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs index 28ac7c288..3417b27a4 100644 --- a/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs +++ b/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs b/FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs index b8d561bfd..33f40c62b 100644 --- a/FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs index c2b1edd44..d26d6136d 100644 --- a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs index a51a126ac..75112738f 100644 --- a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/IValueEncoder.cs b/FoundationDB.Client/TypeSystem/IValueEncoder.cs index 9f9fc1967..a677f671c 100644 --- a/FoundationDB.Client/TypeSystem/IValueEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IValueEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/TypeSystem/TypeSystem.cs b/FoundationDB.Client/TypeSystem/TypeSystem.cs index da2b9e61e..0fe33ef11 100644 --- a/FoundationDB.Client/TypeSystem/TypeSystem.cs +++ b/FoundationDB.Client/TypeSystem/TypeSystem.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/Logging.cs b/FoundationDB.Client/Utils/Logging.cs index 03e655620..3e1968410 100644 --- a/FoundationDB.Client/Utils/Logging.cs +++ b/FoundationDB.Client/Utils/Logging.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index 60ea0ae88..2122271ee 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceBuffer.cs b/FoundationDB.Client/Utils/SliceBuffer.cs index 8df2ebcfd..48bd07807 100644 --- a/FoundationDB.Client/Utils/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/SliceBuffer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceComparer.cs b/FoundationDB.Client/Utils/SliceComparer.cs index 83d911b51..2971bfe33 100644 --- a/FoundationDB.Client/Utils/SliceComparer.cs +++ b/FoundationDB.Client/Utils/SliceComparer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index 51e995a81..7576d37e7 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SlicePairComparer.cs b/FoundationDB.Client/Utils/SlicePairComparer.cs index bd67e06b4..b7f1081c8 100644 --- a/FoundationDB.Client/Utils/SlicePairComparer.cs +++ b/FoundationDB.Client/Utils/SlicePairComparer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceReader.cs b/FoundationDB.Client/Utils/SliceReader.cs index b86bae06f..6637554e8 100644 --- a/FoundationDB.Client/Utils/SliceReader.cs +++ b/FoundationDB.Client/Utils/SliceReader.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/SliceStream.cs index 63b860540..afdb376d4 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/SliceStream.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/SliceWriter.cs b/FoundationDB.Client/Utils/SliceWriter.cs index f2a811bcc..01b7f4528 100644 --- a/FoundationDB.Client/Utils/SliceWriter.cs +++ b/FoundationDB.Client/Utils/SliceWriter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Utils/Uuid128.cs index f28485f0c..1bd9a60cc 100644 --- a/FoundationDB.Client/Utils/Uuid128.cs +++ b/FoundationDB.Client/Utils/Uuid128.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index ea706aca4..e507c78e6 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 3d45b7603..4bc98a833 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index 220b5251b..e5dd2c532 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 502a62acf..e05f531b7 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index cd76e0383..db54e1923 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index ca9c4759e..9db163605 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs index 5e774248a..32869753e 100644 --- a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs +++ b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index cb8e7c21b..397837e27 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index 359bb42d4..6bad9e106 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index 67a10dd3a..769959163 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Optional`1.cs b/FoundationDB.Layers.Common/Optional`1.cs index 8a734a334..62d301494 100644 --- a/FoundationDB.Layers.Common/Optional`1.cs +++ b/FoundationDB.Layers.Common/Optional`1.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs index ff32e7a47..f96190ae4 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs index 47dea01a3..33505928c 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs index b7070b2e6..c9d48465d 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBitView.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs index 2933c36f5..632b67762 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs index b987b634d..9474919e1 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs index b3bbbd5c2..3a812b954 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs index f018b8113..241a4d397 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs index cde6a4efe..4d501db3d 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index 175e60595..681d3ff99 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbDebugStatementWriter.cs b/FoundationDB.Linq.Providers/Expressions/FdbDebugStatementWriter.cs index 1fd4d46ae..ce8f535ec 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbDebugStatementWriter.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbDebugStatementWriter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs b/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs index 0f4ee241b..4a06b6cc2 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs index c4c254a3f..0d5b40f7b 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs index f830aea7c..4b4744892 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index 0cc0abc84..dbfa633a7 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs index d80886e21..49c131e9b 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs index 7bd9af96a..adf6cc479 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs index 837433d93..d07480e30 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryNodeType.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryNodeType.cs index 681c8a53b..914923670 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryNodeType.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryNodeType.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs index 90cf30a66..5d177db9b 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs index 04c38dd89..92733490a 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs index 812f0e8b7..975a8e489 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs index d66c25bf4..3732f6403 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryTransformExpression.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs index 1319149bd..9545dea31 100644 --- a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs +++ b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Interfaces.cs b/FoundationDB.Linq.Providers/Interfaces.cs index b5c38f41a..4224cb89b 100644 --- a/FoundationDB.Linq.Providers/Interfaces.cs +++ b/FoundationDB.Linq.Providers/Interfaces.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs index b6a3751f1..d2848a200 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs index 2e04c4fdc..017b5772a 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncSingleQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncSingleQuery.cs index 91a598185..4860cb927 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncSingleQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncSingleQuery.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Providers/FdbDatabaseQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbDatabaseQuery.cs index 97395d358..280839bd9 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbDatabaseQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbDatabaseQuery.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Providers/FdbIndexQuery`2.cs b/FoundationDB.Linq.Providers/Providers/FdbIndexQuery`2.cs index ec508f1d2..d4f54fc1d 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbIndexQuery`2.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbIndexQuery`2.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index 356b7f760..03fd2055c 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Async/AsyncBufferFacts.cs b/FoundationDB.Tests/Async/AsyncBufferFacts.cs index a4b85bb65..0d312b906 100644 --- a/FoundationDB.Tests/Async/AsyncBufferFacts.cs +++ b/FoundationDB.Tests/Async/AsyncBufferFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs index 665844f76..d021186b5 100644 --- a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs +++ b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs b/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs index e175183b2..41e5e5141 100644 --- a/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs +++ b/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Experimental/JsonNetCodec.cs b/FoundationDB.Tests/Experimental/JsonNetCodec.cs index af07ed72e..b5b530473 100644 --- a/FoundationDB.Tests/Experimental/JsonNetCodec.cs +++ b/FoundationDB.Tests/Experimental/JsonNetCodec.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Experimental/ProtobufCodec.cs b/FoundationDB.Tests/Experimental/ProtobufCodec.cs index 9a0a99fe7..8ae2d14fc 100644 --- a/FoundationDB.Tests/Experimental/ProtobufCodec.cs +++ b/FoundationDB.Tests/Experimental/ProtobufCodec.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/FdbTest.cs b/FoundationDB.Tests/FdbTest.cs index c8aca0b78..bc6ba321c 100644 --- a/FoundationDB.Tests/FdbTest.cs +++ b/FoundationDB.Tests/FdbTest.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/CounterFacts.cs b/FoundationDB.Tests/Layers/CounterFacts.cs index 09c5ceba1..50c079cc7 100644 --- a/FoundationDB.Tests/Layers/CounterFacts.cs +++ b/FoundationDB.Tests/Layers/CounterFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index bde98dbf7..6f01111bb 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/MultiMapFacts.cs b/FoundationDB.Tests/Layers/MultiMapFacts.cs index e716ceb88..9d81dc565 100644 --- a/FoundationDB.Tests/Layers/MultiMapFacts.cs +++ b/FoundationDB.Tests/Layers/MultiMapFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/RankedSetFacts.cs b/FoundationDB.Tests/Layers/RankedSetFacts.cs index c3980cf25..397f127de 100644 --- a/FoundationDB.Tests/Layers/RankedSetFacts.cs +++ b/FoundationDB.Tests/Layers/RankedSetFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/StringInternFacts.cs b/FoundationDB.Tests/Layers/StringInternFacts.cs index 4cd5b2460..ba9396502 100644 --- a/FoundationDB.Tests/Layers/StringInternFacts.cs +++ b/FoundationDB.Tests/Layers/StringInternFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/VectorFacts.cs b/FoundationDB.Tests/Layers/VectorFacts.cs index 6a9e8a6a7..4d8aaf839 100644 --- a/FoundationDB.Tests/Layers/VectorFacts.cs +++ b/FoundationDB.Tests/Layers/VectorFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs b/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs index 836913402..5fdf3a057 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs +++ b/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index d7485ceda..80a14d469 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index b95a506a2..33b74b11c 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/FdbConvertersFacts.cs b/FoundationDB.Tests/Utils/FdbConvertersFacts.cs index 9f9cd1863..66da1b771 100644 --- a/FoundationDB.Tests/Utils/FdbConvertersFacts.cs +++ b/FoundationDB.Tests/Utils/FdbConvertersFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2015, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/SliceComparerFacts.cs b/FoundationDB.Tests/Utils/SliceComparerFacts.cs index 499fcfcd1..b54c7fcb5 100644 --- a/FoundationDB.Tests/Utils/SliceComparerFacts.cs +++ b/FoundationDB.Tests/Utils/SliceComparerFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/SliceFacts.cs b/FoundationDB.Tests/Utils/SliceFacts.cs index 448554d92..fef7ed960 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-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/SliceHelperFacts.cs b/FoundationDB.Tests/Utils/SliceHelperFacts.cs index 22e1ec768..58306a129 100644 --- a/FoundationDB.Tests/Utils/SliceHelperFacts.cs +++ b/FoundationDB.Tests/Utils/SliceHelperFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/SliceStreamFacts.cs b/FoundationDB.Tests/Utils/SliceStreamFacts.cs index cad5b974f..0bfe73560 100644 --- a/FoundationDB.Tests/Utils/SliceStreamFacts.cs +++ b/FoundationDB.Tests/Utils/SliceStreamFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/SliceWriterFacts.cs b/FoundationDB.Tests/Utils/SliceWriterFacts.cs index ec1b87b1c..d7c6c9f75 100644 --- a/FoundationDB.Tests/Utils/SliceWriterFacts.cs +++ b/FoundationDB.Tests/Utils/SliceWriterFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2014, Doxense SAS +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/LICENSE.md b/LICENSE.md index 2a92e2397..40d8b56cd 100644 --- a/LICENSE.md +++ b/LICENSE.md @@ -1,4 +1,4 @@ -Copyright (c) 2013-2014, Doxense SAS +Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without From 058f155b2d6f0f5f591325fdc147e3c08b1ab2ee Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 20 Apr 2018 16:06:36 +0200 Subject: [PATCH 063/153] Renamed Fdb Tuples to drop the "Fdb" prefix - Renamed IFdbTuple into ITuple - All "struct-based" tuples have been renamed from FdbTuple<..> to STuple<...> (S for Struct). They are similar to ValueTuple<...> except that they are immutable, and behave more like a vector of items - Dropped the Fdb prefix from all other helper classes --- FdbShell/Commands/BasicCommands.cs | 28 +- FdbShell/Program.cs | 4 +- FoundationDB.Client/Fdb.System.cs | 8 +- FoundationDB.Client/FdbKey.cs | 10 +- .../FoundationDB.Client.csproj | 52 +- .../Layers/Directories/FdbDirectoryLayer.cs | 74 +- .../Directories/FdbDirectoryPartition.cs | 8 +- .../Directories/FdbDirectorySubspace.cs | 24 +- .../PrefixedTuple.cs} | 30 +- .../SlicedTuple.cs} | 42 +- .../TupleCodec`1.cs} | 18 +- .../TuplePacker.cs} | 6 +- .../TuplePackers.cs} | 378 ++--- .../TupleParser.cs} | 136 +- .../Tuples/{ => Encoding}/TupleReader.cs | 0 .../TupleTypes.cs} | 28 +- .../Tuples/{ => Encoding}/TupleWriter.cs | 0 ...ormatter.cs => AnonymousTupleFormatter.cs} | 18 +- ...matter.cs => FormattableTupleFormatter.cs} | 8 +- ...eFormatter.cs => GenericTupleFormatter.cs} | 8 +- .../Tuples/Formatters/ITupleFormattable.cs | 4 +- .../Tuples/Formatters/ITupleFormatter.cs | 4 +- ...FdbTupleFormatter.cs => TupleFormatter.cs} | 24 +- .../Layers/Tuples/{IFdbTuple.cs => ITuple.cs} | 14 +- .../{FdbJoinedTuple.cs => JoinedTuple.cs} | 44 +- .../{FdbLinkedTuple.cs => LinkedTuple.cs} | 32 +- .../Tuples/{FdbListTuple.cs => ListTuple.cs} | 58 +- .../{FdbMemoizedTuple.cs => MemoizedTuple.cs} | 38 +- .../Layers/Tuples/{FdbTuple.cs => STuple.cs} | 442 +++--- .../Tuples/{FdbTuple`1.cs => STuple`1.cs} | 60 +- .../Tuples/{FdbTuple`2.cs => STuple`2.cs} | 70 +- .../Tuples/{FdbTuple`3.cs => STuple`3.cs} | 82 +- .../Tuples/{FdbTuple`4.cs => STuple`4.cs} | 74 +- .../Tuples/{FdbTuple`5.cs => STuple`5.cs} | 79 +- ...upleComparisons.cs => TupleComparisons.cs} | 50 +- ...bTupleExtensions.cs => TupleExtensions.cs} | 176 +-- .../Tuples/TypeSystem/TupleKeyEncoder.cs | 124 +- .../Subspaces/FdbDynamicSubspaceKeys.cs | 86 +- .../Subspaces/FdbDynamicSubspacePartition.cs | 2 +- .../Subspaces/FdbEncoderSubspaceKeys`1.cs | 2 +- .../Subspaces/FdbEncoderSubspaceKeys`2.cs | 4 +- .../Subspaces/FdbEncoderSubspaceKeys`3.cs | 4 +- .../Subspaces/FdbEncoderSubspaceKeys`4.cs | 4 +- FoundationDB.Client/Subspaces/FdbSubspace.cs | 2 +- .../Encoders/DynamicKeyEncoderBase.cs | 54 +- .../TypeSystem/Encoders/KeyValueEncoders.cs | 124 +- .../TypeSystem/ICompositeKeyEncoder.cs | 8 +- .../TypeSystem/IDynamicKeyEncoder.cs | 14 +- .../Documents/FdbDocumentHandlers.cs | 34 +- .../Documents/FdbHashSetCollection.cs | 28 +- .../Expressions/FdbQueryExpressions.cs | 2 +- .../Benchmarks/BenchRunner.cs | 2 +- .../MessageQueue/MessageQueueRunner.cs | 2 +- .../Tutorials/ClassScheduling.cs | 4 +- FoundationDB.Tests.Sandbox/Program.cs | 6 +- FoundationDB.Tests/DatabaseBulkFacts.cs | 12 +- FoundationDB.Tests/DatabaseFacts.cs | 4 +- FoundationDB.Tests/Encoders/EncoderFacts.cs | 10 +- FoundationDB.Tests/Encoders/TypeCodecFacts.cs | 36 +- .../Indexing/CompressedBitmapsFacts.cs | 2 +- FoundationDB.Tests/KeyFacts.cs | 42 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 14 +- FoundationDB.Tests/Layers/MapFacts.cs | 4 +- FoundationDB.Tests/Layers/RankedSetFacts.cs | 2 +- FoundationDB.Tests/Layers/TupleFacts.cs | 1224 ++++++++--------- .../Linq/FdbAsyncEnumerableFacts.cs | 6 +- .../Linq/FdbQueryExpressionFacts.cs | 10 +- FoundationDB.Tests/RangeQueryFacts.cs | 24 +- FoundationDB.Tests/SubspaceFacts.cs | 6 +- README.md | 6 +- Tuples.md | 86 +- 71 files changed, 2056 insertions(+), 2069 deletions(-) rename FoundationDB.Client/Layers/Tuples/{FdbPrefixedTuple.cs => Encoding/PrefixedTuple.cs} (87%) rename FoundationDB.Client/Layers/Tuples/{FdbSlicedTuple.cs => Encoding/SlicedTuple.cs} (79%) rename FoundationDB.Client/Layers/Tuples/{FdbTupleCodec`1.cs => Encoding/TupleCodec`1.cs} (85%) rename FoundationDB.Client/Layers/Tuples/{FdbTuplePacker.cs => Encoding/TuplePacker.cs} (92%) rename FoundationDB.Client/Layers/Tuples/{FdbTuplePackers.cs => Encoding/TuplePackers.cs} (74%) rename FoundationDB.Client/Layers/Tuples/{FdbTupleParser.cs => Encoding/TupleParser.cs} (88%) rename FoundationDB.Client/Layers/Tuples/{ => Encoding}/TupleReader.cs (100%) rename FoundationDB.Client/Layers/Tuples/{FdbTupleTypes.cs => Encoding/TupleTypes.cs} (85%) rename FoundationDB.Client/Layers/Tuples/{ => Encoding}/TupleWriter.cs (100%) rename FoundationDB.Client/Layers/Tuples/Formatters/{FdbAnonymousTupleFormatter.cs => AnonymousTupleFormatter.cs} (78%) rename FoundationDB.Client/Layers/Tuples/Formatters/{FdbFormattableTupleFormatter.cs => FormattableTupleFormatter.cs} (89%) rename FoundationDB.Client/Layers/Tuples/Formatters/{FdbGenericTupleFormatter.cs => GenericTupleFormatter.cs} (90%) rename FoundationDB.Client/Layers/Tuples/Formatters/{FdbTupleFormatter.cs => TupleFormatter.cs} (80%) rename FoundationDB.Client/Layers/Tuples/{IFdbTuple.cs => ITuple.cs} (91%) rename FoundationDB.Client/Layers/Tuples/{FdbJoinedTuple.cs => JoinedTuple.cs} (83%) rename FoundationDB.Client/Layers/Tuples/{FdbLinkedTuple.cs => LinkedTuple.cs} (87%) rename FoundationDB.Client/Layers/Tuples/{FdbListTuple.cs => ListTuple.cs} (83%) rename FoundationDB.Client/Layers/Tuples/{FdbMemoizedTuple.cs => MemoizedTuple.cs} (81%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple.cs => STuple.cs} (75%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple`1.cs => STuple`1.cs} (79%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple`2.cs => STuple`2.cs} (78%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple`3.cs => STuple`3.cs} (74%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple`4.cs => STuple`4.cs} (80%) rename FoundationDB.Client/Layers/Tuples/{FdbTuple`5.cs => STuple`5.cs} (79%) rename FoundationDB.Client/Layers/Tuples/{FdbTupleComparisons.cs => TupleComparisons.cs} (89%) rename FoundationDB.Client/Layers/Tuples/{FdbTupleExtensions.cs => TupleExtensions.cs} (78%) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 8efb32076..c912e6b33 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -37,7 +37,7 @@ public static async Task TryOpenCurrentDirectoryAsync(string[] pa } } - public static async Task Dir(string[] path, IFdbTuple extras, DirectoryBrowseOptions options, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Dir(string[] path, ITuple extras, DirectoryBrowseOptions options, IFdbDatabase db, TextWriter log, CancellationToken ct) { if (log == null) log = Console.Out; @@ -96,7 +96,7 @@ public static async Task Dir(string[] path, IFdbTuple extras, DirectoryBrowseOpt } /// Creates a new directory - public static async Task CreateDirectory(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task CreateDirectory(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { if (log == null) log = Console.Out; @@ -124,7 +124,7 @@ public static async Task CreateDirectory(string[] path, IFdbTuple extras, IFdbDa } /// Remove a directory and all its data - public static async Task RemoveDirectory(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task RemoveDirectory(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { if (log == null) log = Console.Out; @@ -153,7 +153,7 @@ public static async Task RemoveDirectory(string[] path, IFdbTuple extras, IFdbDa } /// Move/Rename a directory - public static async Task MoveDirectory(string[] srcPath, string[] dstPath, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task MoveDirectory(string[] srcPath, string[] dstPath, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { var folder = await db.Directory.TryOpenAsync(srcPath, cancellationToken: ct); if (folder == null) @@ -173,7 +173,7 @@ public static async Task MoveDirectory(string[] srcPath, string[] dstPath, IFdbT Console.WriteLine("Moved {0} to {1}", string.Join("/", srcPath), string.Join("/", dstPath)); } - public static async Task ShowDirectoryLayer(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task ShowDirectoryLayer(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { var dir = await BasicCommands.TryOpenCurrentDirectoryAsync(path, db, ct); if (dir == null) @@ -191,7 +191,7 @@ public static async Task ShowDirectoryLayer(string[] path, IFdbTuple extras, IFd } } - public static async Task ChangeDirectoryLayer(string[] path, string layer, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task ChangeDirectoryLayer(string[] path, string layer, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { var dir = await BasicCommands.TryOpenCurrentDirectoryAsync(path, db, ct); if (dir == null) @@ -206,7 +206,7 @@ public static async Task ChangeDirectoryLayer(string[] path, string layer, IFdbT } /// Counts the number of keys inside a directory - public static async Task Count(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Count(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { // look if there is something under there var folder = (await TryOpenCurrentDirectoryAsync(path, db, ct)) as FdbDirectorySubspace; @@ -219,7 +219,7 @@ public static async Task Count(string[] path, IFdbTuple extras, IFdbDatabase db, var copy = FdbSubspace.Copy(folder); log.WriteLine("# Counting keys under {0} ...", FdbKey.Dump(copy.Key)); - var progress = new Progress>((state) => + var progress = new Progress>((state) => { log.Write("\r# Found {0:N0} keys...", state.Item1); }); @@ -229,7 +229,7 @@ public static async Task Count(string[] path, IFdbTuple extras, IFdbDatabase db, } /// Shows the first few keys of a directory - public static async Task Show(string[] path, IFdbTuple extras, bool reverse, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Show(string[] path, ITuple extras, bool reverse, IFdbDatabase db, TextWriter log, CancellationToken ct) { int count = 20; if (extras.Count > 0) @@ -270,7 +270,7 @@ public static async Task Show(string[] path, IFdbTuple extras, bool reverse, IFd } /// Display a tree of a directory's children - public static async Task Tree(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Tree(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { if (log == null) log = Console.Out; @@ -317,7 +317,7 @@ private static async Task TreeDirectoryWalk(FdbDirectorySubspace folder, ListFind the DCs, machines and processes in the cluster - public static async Task Topology(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Topology(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { var coords = await Fdb.System.GetCoordinatorsAsync(db, ct); log.WriteLine("[Cluster] {0}", coords.Id); @@ -545,7 +545,7 @@ public static async Task Topology(string[] path, IFdbTuple extras, IFdbDatabase log.WriteLine(); } - public static async Task Shards(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Shards(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { var ranges = await Fdb.System.GetChunksAsync(db, FdbKey.MinValue, FdbKey.MaxValue, ct); Console.WriteLine("Found {0} shards in the whole cluster", ranges.Count); @@ -574,7 +574,7 @@ public static async Task Shards(string[] path, IFdbTuple extras, IFdbDatabase db //TODO: shards that intersect the current directory } - public static async Task Sampling(string[] path, IFdbTuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) + public static async Task Sampling(string[] path, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { double ratio = 0.1d; bool auto = true; diff --git a/FdbShell/Program.cs b/FdbShell/Program.cs index bbc08976a..5fbbc26de 100644 --- a/FdbShell/Program.cs +++ b/FdbShell/Program.cs @@ -305,7 +305,7 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) var tokens = s.Trim().Split(new [] { ' ' }, StringSplitOptions.RemoveEmptyEntries); string cmd = tokens.Length > 0 ? tokens[0] : String.Empty; string prm = tokens.Length > 1 ? tokens[1] : String.Empty; - var extras = tokens.Length > 2 ? FdbTuple.FromEnumerable(tokens.Skip(2)) : FdbTuple.Empty; + var extras = tokens.Length > 2 ? STuple.FromEnumerable(tokens.Skip(2)) : STuple.Empty; var trimmedCommand = cmd.Trim().ToLowerInvariant(); switch (trimmedCommand) @@ -467,7 +467,7 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) if (!string.IsNullOrEmpty(prm)) { var path = ParsePath(CombinePath(CurrentDirectoryPath, prm)); - await RunAsyncCommand((db, log, ct) => BasicCommands.CreateDirectory(path, FdbTuple.Create(FdbDirectoryPartition.LayerId).Concat(extras), db, log, ct), cancel); + await RunAsyncCommand((db, log, ct) => BasicCommands.CreateDirectory(path, STuple.Create(FdbDirectoryPartition.LayerId).Concat(extras), db, log, ct), cancel); } break; diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 75eb674e3..c45e9659b 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -408,7 +408,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken cancellationToken) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken cancellationToken) { return EstimateCountAsync(db, range.Begin, range.End, onProgress, cancellationToken); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! @@ -422,7 +422,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Token used to cancel the operation /// Number of keys k such that <= k > /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress> onProgress, CancellationToken cancellationToken) + public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress> onProgress, CancellationToken cancellationToken) { const int INIT_WINDOW_SIZE = 1 << 8; // start at 256 //1024 const int MAX_WINDOW_SIZE = 1 << 13; // never use more than 4096 @@ -538,7 +538,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli .ConfigureAwait(false); counter += n; - if (onProgress != null) onProgress.Report(FdbTuple.Create(counter, end)); + if (onProgress != null) onProgress.Report(STuple.Create(counter, end)); #if TRACE_COUNTING ++iter; #endif @@ -552,7 +552,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli // the range is not finished, advance the cursor counter += windowSize; cursor = next; - if (onProgress != null) onProgress.Report(FdbTuple.Create(counter, cursor)); + if (onProgress != null) onProgress.Report(STuple.Create(counter, cursor)); if (!last) { // double the size of the window if we are not in the last segment diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 21d8b5ccf..d47c577da 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -284,7 +284,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) { // it could be a tuple... try { - IFdbTuple tuple = null; + ITuple tuple = null; string suffix = null; bool skip = false; @@ -302,7 +302,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) case 0xFF: { //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.FdbTuple.Unpack(key[0, -1]); + tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key[0, -1]); suffix = "."; break; } @@ -310,7 +310,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) { var tmp = key[0, -1] + (byte)0; //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.FdbTuple.Unpack(tmp); + tuple = FoundationDB.Layers.Tuples.STuple.Unpack(tmp); suffix = " + 1"; break; } @@ -327,7 +327,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) if (key.Count > 2 && key[-1] == 0 && key[-2] != 0xFF) { //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.FdbTuple.Unpack(key[0, -1]); + tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key[0, -1]); suffix = ".<00>"; } break; @@ -343,7 +343,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) if (tuple == null && !skip) { // attempt a regular decoding //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.FdbTuple.Unpack(key); + tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key); } if (tuple != null) return tuple.ToString() + suffix; diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index ce12c10a1..407a4ddb8 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -85,9 +85,9 @@ - - - + + + @@ -154,21 +154,21 @@ - - + + - + - - - + + + - - + + @@ -177,10 +177,10 @@ - - + + - + @@ -215,7 +215,7 @@ - + @@ -257,18 +257,18 @@ - + - - - - - - - - - - + + + + + + + + + + diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index b8ff917cd..49f2a0e11 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -75,7 +75,7 @@ public class FdbDirectoryLayer : IFdbDirectory internal FdbHighContentionAllocator Allocator { [NotNull] get; } /// Gets the path for the root node of this FdbDirectoryLayer. - internal IFdbTuple Location { [NotNull] get; } + internal ITuple Location { [NotNull] get; } /// Name of root directory of this layer /// Returns String.Empty for the root Directory Layer, or the name of the partition @@ -100,7 +100,7 @@ public string Name /// Convert a relative path in this Directory Layer, into an absolute path from the root of partition of the database [NotNull] - internal IFdbTuple PartitionSubPath(IFdbTuple path = null) + internal ITuple PartitionSubPath(ITuple path = null) { // If the DL is the root, the path is already absolute // If the DL is used by a partition, then the path of the partition will be prepended to the path @@ -128,7 +128,7 @@ Task IFdbDirectory.ChangeLayerAsync(IFdbTransaction trans, /// Subspace where all the node metadata will be stored ('\xFE' by default) /// Subspace where all automatically allocated directories will be stored (empty by default) /// Location of the root of all the directories managed by this Directory Layer. Ususally empty for the root partition of the database. - internal FdbDirectoryLayer(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace contentSubspace, IFdbTuple location) + internal FdbDirectoryLayer(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace contentSubspace, ITuple location) { Contract.Requires(nodeSubspace != null && contentSubspace != null); @@ -141,7 +141,7 @@ internal FdbDirectoryLayer(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace this.Allocator = new FdbHighContentionAllocator(this.RootNode.Partition.ByKey(HcaKey)); if (location == null || location.Count == 0) { - this.Location = FdbTuple.Empty; + this.Location = STuple.Empty; this.Path = new string[0]; } else @@ -165,7 +165,7 @@ public static FdbDirectoryLayer Create() public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null) { var subspace = FdbSubspace.CreateDynamic(prefix, TypeSystem.Tuples); - var location = path != null ? ParsePath(path) : FdbTuple.Empty; + var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } @@ -177,7 +177,7 @@ public static FdbDirectoryLayer Create(IFdbSubspace subspace, IEnumerable MoveAsync(IFdbTransaction trans, IEnumerable TryMoveAsync(IFdbTransaction trans, IEnumerabl if (oldPath == null) throw new ArgumentNullException("oldPath"); if (newPath == null) throw new ArgumentNullException("newPath"); - var oldLocation = FdbTuple.FromEnumerable(oldPath); + var oldLocation = STuple.FromEnumerable(oldPath); VerifyPath(oldLocation, "oldPath"); - var newLocation = FdbTuple.FromEnumerable(newPath); + var newLocation = STuple.FromEnumerable(newPath); VerifyPath(newLocation, "newPath"); return MoveInternalAsync(trans, oldLocation, newLocation, throwOnError: false); @@ -436,7 +436,7 @@ public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans) { if (trans == null) throw new ArgumentNullException("trans"); - return ListInternalAsync(trans, FdbTuple.Empty, throwIfMissing: true); + return ListInternalAsync(trans, STuple.Empty, throwIfMissing: true); } /// Returns the list of subdirectories of directory at , if it exists. @@ -454,7 +454,7 @@ public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans, public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans) { if (trans == null) throw new ArgumentNullException("trans"); - return ListInternalAsync(trans, FdbTuple.Empty, throwIfMissing: false); + return ListInternalAsync(trans, STuple.Empty, throwIfMissing: false); } #endregion @@ -490,7 +490,7 @@ public override string ToString() private struct Node { - public Node(IFdbDynamicSubspace subspace, IFdbTuple path, IFdbTuple targetPath, Slice layer) + public Node(IFdbDynamicSubspace subspace, ITuple path, ITuple targetPath, Slice layer) { this.Subspace = subspace; this.Path = path; @@ -499,13 +499,13 @@ public Node(IFdbDynamicSubspace subspace, IFdbTuple path, IFdbTuple targetPath, } public readonly IFdbDynamicSubspace Subspace; - public readonly IFdbTuple Path; - public readonly IFdbTuple TargetPath; + public readonly ITuple Path; + public readonly ITuple TargetPath; public Slice Layer; //PERF: readonly struct public bool Exists { get { return this.Subspace != null; } } - public IFdbTuple PartitionSubPath { [NotNull] get { return this.TargetPath.Substring(this.Path.Count); } } + public ITuple PartitionSubPath { [NotNull] get { return this.TargetPath.Substring(this.Path.Count); } } public bool IsInPartition(bool includeEmptySubPath) { @@ -521,9 +521,9 @@ private static void SetLayer([NotNull] IFdbTransaction trans, [NotNull] IFdbDyna } [NotNull] - internal static IFdbTuple ParsePath(IEnumerable path, string argName = null) + internal static ITuple ParsePath(IEnumerable path, string argName = null) { - if (path == null) return FdbTuple.Empty; + if (path == null) return STuple.Empty; var pathCopy = path.ToArray(); foreach (var s in pathCopy) @@ -533,19 +533,19 @@ internal static IFdbTuple ParsePath(IEnumerable path, string argName = n throw new ArgumentException("The path of a directory cannot contain null elements", argName ?? "path"); } } - return FdbTuple.FromArray(pathCopy); + return STuple.FromArray(pathCopy); } [NotNull] - internal static IFdbTuple ParsePath([NotNull] string name, string argName = null) + internal static ITuple ParsePath([NotNull] string name, string argName = null) { if (name == null) throw new ArgumentNullException(argName ?? "name"); - return FdbTuple.Create(name); + return STuple.Create(name); } [NotNull] - internal static IFdbTuple VerifyPath([NotNull] IFdbTuple path, string argName = null) + internal static ITuple VerifyPath([NotNull] ITuple path, string argName = null) { // The path should not contain any null strings if (path == null) throw new ArgumentNullException(argName ?? "path"); @@ -561,7 +561,7 @@ internal static IFdbTuple VerifyPath([NotNull] IFdbTuple path, string argName = } [NotNull] - internal IReadOnlyList ToAbsolutePath([NotNull] IFdbTuple path) + internal IReadOnlyList ToAbsolutePath([NotNull] ITuple path) { if (path.Count == 0) return this.Path; var converted = path.ToArray(); @@ -571,7 +571,7 @@ internal IReadOnlyList ToAbsolutePath([NotNull] IFdbTuple path) /// Maps an absolute path to a relative path within this directory layer [NotNull] - internal IFdbTuple ToRelativePath([NotNull] IFdbTuple path) + internal ITuple ToRelativePath([NotNull] ITuple path) { if (path == null) throw new ArgumentNullException("path"); @@ -580,7 +580,7 @@ internal IFdbTuple ToRelativePath([NotNull] IFdbTuple path) } [ItemCanBeNull] - internal async Task CreateOrOpenInternalAsync(IFdbReadOnlyTransaction readTrans, IFdbTransaction trans, [NotNull] IFdbTuple path, Slice layer, Slice prefix, bool allowCreate, bool allowOpen, bool throwOnError) + internal async Task CreateOrOpenInternalAsync(IFdbReadOnlyTransaction readTrans, IFdbTransaction trans, [NotNull] ITuple path, Slice layer, Slice prefix, bool allowCreate, bool allowOpen, bool throwOnError) { Contract.Requires(readTrans != null || trans != null, "Need at least one transaction"); Contract.Requires(path != null, "Path must be specified"); @@ -687,7 +687,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly } [ItemCanBeNull] - internal async Task MoveInternalAsync([NotNull] IFdbTransaction trans, [NotNull] IFdbTuple oldPath, [NotNull] IFdbTuple newPath, bool throwOnError) + internal async Task MoveInternalAsync([NotNull] IFdbTransaction trans, [NotNull] ITuple oldPath, [NotNull] ITuple newPath, bool throwOnError) { Contract.Requires(trans != null && oldPath != null && newPath != null); @@ -718,7 +718,7 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa // we have already checked that old and new are under this partition path, but one of them (or both?) could be under a sub-partition.. if (oldNode.IsInPartition(false) || newNode.IsInPartition(false)) { - if (!oldNode.IsInPartition(false) || !newNode.IsInPartition(false) || !FdbTuple.Equals(oldNode.Path, newNode.Path)) + if (!oldNode.IsInPartition(false) || !newNode.IsInPartition(false) || !STuple.Equals(oldNode.Path, newNode.Path)) { throw new InvalidOperationException("Cannot move between partitions."); } @@ -746,7 +746,7 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa return ContentsOfNode(oldNode.Subspace, newPath, oldNode.Layer); } - internal async Task RemoveInternalAsync([NotNull] IFdbTransaction trans, [NotNull] IFdbTuple path, bool throwIfMissing) + internal async Task RemoveInternalAsync([NotNull] IFdbTransaction trans, [NotNull] ITuple path, bool throwIfMissing) { Contract.Requires(trans != null && path != null); @@ -777,7 +777,7 @@ internal async Task RemoveInternalAsync([NotNull] IFdbTransaction trans, [ } [ItemCanBeNull] - internal async Task> ListInternalAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple path, bool throwIfMissing) + internal async Task> ListInternalAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] ITuple path, bool throwIfMissing) { Contract.Requires(trans != null && path != null); @@ -802,7 +802,7 @@ internal async Task> ListInternalAsync([NotNull] IFdbReadOnlyTransa .ConfigureAwait(false); } - internal async Task ExistsInternalAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple path) + internal async Task ExistsInternalAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] ITuple path) { Contract.Requires(trans != null && path != null); @@ -820,7 +820,7 @@ internal async Task ExistsInternalAsync([NotNull] IFdbReadOnlyTransaction return true; } - internal async Task ChangeLayerInternalAsync([NotNull] IFdbTransaction trans, [NotNull] IFdbTuple path, Slice newLayer) + internal async Task ChangeLayerInternalAsync([NotNull] IFdbTransaction trans, [NotNull] ITuple path, Slice newLayer) { Contract.Requires(trans != null && path != null); @@ -934,7 +934,7 @@ private IFdbDynamicSubspace NodeWithPrefix(Slice prefix) /// Returns a new Directory Subspace given its node subspace, path and layer id [NotNull] - private FdbDirectorySubspace ContentsOfNode([NotNull] IFdbSubspace node, [NotNull] IFdbTuple relativePath, Slice layer) + private FdbDirectorySubspace ContentsOfNode([NotNull] IFdbSubspace node, [NotNull] ITuple relativePath, Slice layer) { Contract.Requires(node != null); @@ -959,7 +959,7 @@ private FdbDirectoryPartition GetPartitionForNode(Node node) /// Finds a node subspace, given its path, by walking the tree from the root. /// Node if it was found, or null - private async Task FindAsync([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IFdbTuple path) + private async Task FindAsync([NotNull] IFdbReadOnlyTransaction tr, [NotNull] ITuple path) { Contract.Requires(tr != null && path != null); @@ -1006,7 +1006,7 @@ private IFdbAsyncEnumerable> SubdirNam /// Remove an existing node from its parents /// True if the parent node was found, otherwise false - private async Task RemoveFromParent([NotNull] IFdbTransaction tr, [NotNull] IFdbTuple path) + private async Task RemoveFromParent([NotNull] IFdbTransaction tr, [NotNull] ITuple path) { Contract.Requires(tr != null && path != null); @@ -1030,7 +1030,7 @@ private async Task RemoveRecursive([NotNull] IFdbTransaction tr, [NotNull] IFdbD // remove ALL the contents if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all content located under {0}", node.Key); - tr.ClearRange(ContentsOfNode(node, FdbTuple.Empty, Slice.Empty).ToRange()); + tr.ClearRange(ContentsOfNode(node, STuple.Empty, Slice.Empty).ToRange()); // and all the metadata for this folder if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all metadata for folder under {0}", node.Key); tr.ClearRange(node.Keys.ToRange()); @@ -1073,7 +1073,7 @@ private static Slice GetSubDirKey([NotNull] IFdbDynamicSubspace parent, [NotNull /// Tuple that should only contain strings /// Array of strings [NotNull] - public static string[] ParsePath([NotNull] IFdbTuple path) + public static string[] ParsePath([NotNull] ITuple path) { if (path == null) throw new ArgumentNullException("path"); var tmp = new string[path.Count]; diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 7554d1ace..1733d2619 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -39,7 +39,7 @@ public class FdbDirectoryPartition : FdbDirectorySubspace private readonly FdbDirectoryLayer m_parentDirectoryLayer; - internal FdbDirectoryPartition(IFdbTuple location, IFdbTuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer) + internal FdbDirectoryPartition(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer) : base(location, relativeLocation, prefix, new FdbDirectoryLayer(FdbSubspace.CreateDynamic(prefix + FdbKey.Directory, TypeSystem.Tuples), FdbSubspace.CreateDynamic(prefix, TypeSystem.Tuples), location), LayerId, TypeSystem.Tuples.GetDynamicEncoder()) { m_parentDirectoryLayer = directoryLayer; @@ -57,12 +57,12 @@ public override bool Contains(Slice key) throw new InvalidOperationException("Cannot check whether a key belongs to the root of a directory partition."); } - protected override IFdbTuple ToRelativePath(IFdbTuple location) + protected override ITuple ToRelativePath(ITuple location) { - return location ?? FdbTuple.Empty; + return location ?? STuple.Empty; } - protected override FdbDirectoryLayer GetLayerForPath(IFdbTuple relativeLocation) + protected override FdbDirectoryLayer GetLayerForPath(ITuple relativeLocation) { if (relativeLocation.Count == 0) { // Forward all actions on the Partition itself (empty path) to its parent's DL diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index fb484f821..5edb70f5f 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -43,7 +43,7 @@ namespace FoundationDB.Layers.Directories public class FdbDirectorySubspace : FdbDynamicSubspace, IFdbDirectory { - internal FdbDirectorySubspace(IFdbTuple location, IFdbTuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IDynamicKeyEncoder encoder) + internal FdbDirectorySubspace(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IDynamicKeyEncoder encoder) : base(prefix, encoder) { Contract.Requires(location != null && relativeLocation != null && prefix != null && directoryLayer != null); @@ -60,10 +60,10 @@ internal FdbDirectorySubspace(IFdbTuple location, IFdbTuple relativeLocation, Sl } /// Absolute location of the directory - protected IFdbTuple Location { [NotNull] get; private set; } + protected ITuple Location { [NotNull] get; private set; } /// Location of the directory relative to its parent Directory Layer - protected IFdbTuple RelativeLocation { [NotNull] get; private set; } + protected ITuple RelativeLocation { [NotNull] get; private set; } /// Absolute path of this directory public IReadOnlyList Path { [NotNull] get; private set; } @@ -90,7 +90,7 @@ public string FullName /// Return the DirectoryLayer instance that should be called for the given path /// Location relative to this directory subspace - protected virtual FdbDirectoryLayer GetLayerForPath(IFdbTuple relativeLocation) + protected virtual FdbDirectoryLayer GetLayerForPath(ITuple relativeLocation) { // for regular directories, always returns its DL. return this.DirectoryLayer; @@ -100,7 +100,7 @@ protected virtual FdbDirectoryLayer GetLayerForPath(IFdbTuple relativeLocation) /// Path relative from this directory /// Path relative to the path of the current partition [NotNull] - protected virtual IFdbTuple ToRelativePath(IFdbTuple location) + protected virtual ITuple ToRelativePath(ITuple location) { return location == null ? this.RelativeLocation : this.RelativeLocation.Concat(location); } @@ -109,9 +109,9 @@ protected virtual IFdbTuple ToRelativePath(IFdbTuple location) /// Path relative from this directory /// Path relative to the path of the current partition [NotNull] - protected IFdbTuple ToRelativePath(IEnumerable path) + protected ITuple ToRelativePath(IEnumerable path) { - return ToRelativePath(path == null ? null : FdbTuple.FromEnumerable(path)); + return ToRelativePath(path == null ? null : STuple.FromEnumerable(path)); } /// Ensure that this directory was registered with the correct layer id @@ -246,7 +246,7 @@ public Task MoveToAsync([NotNull] IFdbTransaction trans, [ if (newAbsolutePath == null) throw new ArgumentNullException("newAbsolutePath"); // if 'this' is a Directory Partition, we need to move it via the parent DL ! - var directoryLayer = GetLayerForPath(FdbTuple.Empty); + var directoryLayer = GetLayerForPath(STuple.Empty); // verify that it is still inside the same partition var location = FdbDirectoryLayer.ParsePath(newAbsolutePath, "newAbsolutePath"); @@ -280,7 +280,7 @@ public Task TryMoveToAsync([NotNull] IFdbTransaction trans if (newPath == null) throw new ArgumentNullException("newPath"); // if 'this' is a Directory Partition, we need to move it via the parent DL ! - var directoryLayer = GetLayerForPath(FdbTuple.Empty); + var directoryLayer = GetLayerForPath(STuple.Empty); var location = FdbDirectoryLayer.ParsePath(newPath, "newPath"); if (!location.StartsWith(directoryLayer.Location)) throw new InvalidOperationException("Cannot move between partitions."); @@ -311,7 +311,7 @@ public Task RemoveAsync([NotNull] IFdbTransaction trans) if (trans == null) throw new ArgumentNullException("trans"); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! - var directoryLayer = GetLayerForPath(FdbTuple.Empty); + var directoryLayer = GetLayerForPath(STuple.Empty); return directoryLayer.RemoveInternalAsync(trans, this.RelativeLocation, throwIfMissing: true); } @@ -344,7 +344,7 @@ public Task TryRemoveAsync([NotNull] IFdbTransaction trans) if (trans == null) throw new ArgumentNullException("trans"); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! - var directoryLayer = GetLayerForPath(FdbTuple.Empty); + var directoryLayer = GetLayerForPath(STuple.Empty); return directoryLayer.RemoveInternalAsync(trans, this.RelativeLocation, throwIfMissing: false); } @@ -375,7 +375,7 @@ public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans) if (trans == null) throw new ArgumentNullException("trans"); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! - var directoryLayer = GetLayerForPath(FdbTuple.Empty); + var directoryLayer = GetLayerForPath(STuple.Empty); return directoryLayer.ExistsInternalAsync(trans, this.RelativeLocation); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs similarity index 87% rename from FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs index 6d3ad31b3..b1e1ebb42 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbPrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs @@ -39,15 +39,15 @@ namespace FoundationDB.Layers.Tuples /// Tuple that has a fixed abitrary binary prefix [DebuggerDisplay("{ToString()}")] - public sealed class FdbPrefixedTuple : IFdbTuple + public sealed class PrefixedTuple : ITuple { // Used in scenario where we will append keys to a common base tuple // note: linked list are not very efficient, but we do not expect a very long chain, and the head will usually be a subspace or memoized tuple private Slice m_prefix; //PERF: readonly struct - private readonly IFdbTuple m_items; + private readonly ITuple m_items; - internal FdbPrefixedTuple(Slice prefix, IFdbTuple items) + internal PrefixedTuple(Slice prefix, ITuple items) { Contract.Requires(!prefix.IsNull && items != null); @@ -84,7 +84,7 @@ public object this[int index] get { return m_items[index]; } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { get { return m_items[fromIncluded, toExcluded]; } } @@ -99,29 +99,29 @@ public R Last() return m_items.Last(); } - IFdbTuple IFdbTuple.Append(R value) + ITuple ITuple.Append(R value) { return this.Append(value); } - IFdbTuple IFdbTuple.Concat(IFdbTuple tuple) + ITuple ITuple.Concat(ITuple tuple) { return this.Concat(tuple); } [NotNull] - public FdbPrefixedTuple Append(R value) + public PrefixedTuple Append(R value) { - return new FdbPrefixedTuple(m_prefix, m_items.Append(value)); + return new PrefixedTuple(m_prefix, m_items.Append(value)); } [NotNull] - public FdbPrefixedTuple Concat([NotNull] IFdbTuple tuple) + public PrefixedTuple Concat([NotNull] ITuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); if (tuple.Count == 0) return this; - return new FdbPrefixedTuple(m_prefix, m_items.Concat(tuple)); + return new PrefixedTuple(m_prefix, m_items.Concat(tuple)); } public void CopyTo([NotNull] object[] array, int offset) @@ -143,7 +143,7 @@ public override string ToString() { //TODO: should we add the prefix to the string representation ? // => something like "(123, 'abc', true)" - return FdbTuple.ToString(this); + return STuple.ToString(this); } public override bool Equals(object obj) @@ -151,7 +151,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -166,7 +166,7 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect if (object.ReferenceEquals(this, other)) return true; if (other == null) return false; - var linked = other as FdbPrefixedTuple; + var linked = other as PrefixedTuple; if (!object.ReferenceEquals(linked, null)) { // Should all of these tuples be considered equal ? @@ -188,12 +188,12 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect return comparer.Equals(m_items, linked.m_items); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( m_prefix.GetHashCode(), comparer.GetHashCode(m_items) ); diff --git a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs similarity index 79% rename from FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs index 5bc091469..8439a1a3c 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbSlicedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs @@ -36,9 +36,9 @@ namespace FoundationDB.Layers.Tuples using FoundationDB.Client.Converters; /// Lazily-evaluated tuple that was unpacked from a key - internal sealed class FdbSlicedTuple : IFdbTuple + internal sealed class SlicedTuple : ITuple { - // FdbTuple.Unpack() splits a key into an array of slices (one for each item). We hold onto these slices, and only deserialize them if needed. + // STuple.Unpack() splits a key into an array of slices (one for each item). We hold onto these slices, and only deserialize them if needed. // This is helpful because in most cases, the app code will only want to get the last few items (e.g: tuple[-1]) or skip the first few items (some subspace). // We also support offset/count so that Splicing is efficient (used a lot to remove the suffixes from keys) @@ -51,7 +51,7 @@ internal sealed class FdbSlicedTuple : IFdbTuple private int? m_hashCode; - public FdbSlicedTuple(Slice[] slices, int offset, int count) + public SlicedTuple(Slice[] slices, int offset, int count) { Contract.Requires(slices != null && offset >= 0 && count >= 0); Contract.Requires(offset + count <= slices.Length); @@ -86,45 +86,45 @@ public int Count public object this[int index] { - get { return FdbTuplePackers.DeserializeBoxed(GetSlice(index)); } + get { return TuplePackers.DeserializeBoxed(GetSlice(index)); } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { get { - int begin = fromIncluded.HasValue ? FdbTuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? FdbTuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; int len = end - begin; - if (len <= 0) return FdbTuple.Empty; + if (len <= 0) return STuple.Empty; if (begin == 0 && len == m_count) return this; - return new FdbSlicedTuple(m_slices, m_offset + begin, len); + return new SlicedTuple(m_slices, m_offset + begin, len); } } public R Get(int index) { - return FdbTuplePacker.Deserialize(GetSlice(index)); + return TuplePacker.Deserialize(GetSlice(index)); } public R Last() { if (m_count == 0) throw new InvalidOperationException("Tuple is empty"); - return FdbTuplePacker.Deserialize(m_slices[m_offset + m_count - 1]); + return TuplePacker.Deserialize(m_slices[m_offset + m_count - 1]); } public Slice GetSlice(int index) { - return m_slices[m_offset + FdbTuple.MapIndex(index, m_count)]; + return m_slices[m_offset + STuple.MapIndex(index, m_count)]; } - IFdbTuple IFdbTuple.Append(T value) + ITuple ITuple.Append(T value) { throw new NotSupportedException(); } - IFdbTuple IFdbTuple.Concat(IFdbTuple tuple) + ITuple ITuple.Concat(ITuple tuple) { throw new NotSupportedException(); } @@ -133,7 +133,7 @@ public void CopyTo(object[] array, int offset) { for (int i = 0; i < m_count;i++) { - array[i + offset] = FdbTuplePackers.DeserializeBoxed(m_slices[i + m_offset]); + array[i + offset] = TuplePackers.DeserializeBoxed(m_slices[i + m_offset]); } } @@ -141,7 +141,7 @@ public IEnumerator GetEnumerator() { for (int i = 0; i < m_count; i++) { - yield return FdbTuplePackers.DeserializeBoxed(m_slices[i + m_offset]); + yield return TuplePackers.DeserializeBoxed(m_slices[i + m_offset]); } } @@ -154,7 +154,7 @@ public override string ToString() { //OPTIMIZE: this could be optimized, because it may be called a lot when logging is enabled on keys parsed from range reads // => each slice has a type prefix that could be used to format it to a StringBuilder faster, maybe? - return FdbTuple.ToString(this); + return STuple.ToString(this); } public override bool Equals(object obj) @@ -162,7 +162,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -177,7 +177,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) if (object.ReferenceEquals(this, other)) return true; if (other == null) return false; - var sliced = other as FdbSlicedTuple; + var sliced = other as SlicedTuple; if (!object.ReferenceEquals(sliced, null)) { if (sliced.m_count != m_count) return false; @@ -190,7 +190,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) return false; } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) @@ -205,7 +205,7 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) int h = 0; for (int i = 0; i < m_count; i++) { - h = FdbTuple.CombineHashCodes(h, comparer.GetHashCode(m_slices[i + m_offset])); + h = STuple.CombineHashCodes(h, comparer.GetHashCode(m_slices[i + m_offset])); } if (canUseCache) m_hashCode = h; return h; diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs similarity index 85% rename from FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index 859a3a363..add745128 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -34,40 +34,40 @@ namespace FoundationDB.Layers.Tuples /// Type codec that uses the Tuple Encoding format /// Type of the values encoded by this codec - public sealed class FdbTupleCodec : FdbTypeCodec, IValueEncoder + public sealed class TupleCodec : FdbTypeCodec, IValueEncoder { - private static volatile FdbTupleCodec s_defaultSerializer; + private static volatile TupleCodec s_defaultSerializer; - public static FdbTupleCodec Default + public static TupleCodec Default { [NotNull] - get { return s_defaultSerializer ?? (s_defaultSerializer = new FdbTupleCodec(default(T))); } + get { return s_defaultSerializer ?? (s_defaultSerializer = new TupleCodec(default(T))); } } private readonly T m_missingValue; - public FdbTupleCodec(T missingValue) + public TupleCodec(T missingValue) { m_missingValue = missingValue; } public override Slice EncodeOrdered(T value) { - return FdbTuple.EncodeKey(value); + return STuple.EncodeKey(value); } public override void EncodeOrderedSelfTerm(ref SliceWriter output, T value) { //HACKHACK: we lose the current depth! var writer = new TupleWriter(output); - FdbTuplePacker.Encoder(ref writer, value); + TuplePacker.Encoder(ref writer, value); output = writer.Output; } public override T DecodeOrdered(Slice input) { - return FdbTuple.DecodeKey(input); + return STuple.DecodeKey(input); } public override T DecodeOrderedSelfTerm(ref SliceReader input) @@ -75,7 +75,7 @@ public override T DecodeOrderedSelfTerm(ref SliceReader input) //HACKHACK: we lose the current depth! var reader = new TupleReader(input); T value; - bool res = FdbTuple.DecodeNext(ref reader, out value); + bool res = STuple.DecodeNext(ref reader, out value); input = reader.Input; return res ? value : m_missingValue; } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs similarity index 92% rename from FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs index a5dda0f3c..bc12202c6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePacker.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs @@ -34,12 +34,12 @@ namespace FoundationDB.Layers.Tuples /// Helper class that can serialize values of type to the tuple binary format /// Type of values to be serialized - public static class FdbTuplePacker + public static class TuplePacker { - internal static readonly FdbTuplePackers.Encoder Encoder = FdbTuplePackers.GetSerializer(required: true); + internal static readonly TuplePackers.Encoder Encoder = TuplePackers.GetSerializer(required: true); - internal static readonly Func Decoder = FdbTuplePackers.GetDeserializer(required: true); + internal static readonly Func Decoder = TuplePackers.GetDeserializer(required: true); /// Serialize a into a binary buffer /// Target buffer diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs similarity index 74% rename from FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index 341ebd58a..67f8639cb 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -39,7 +39,7 @@ namespace FoundationDB.Layers.Tuples using JetBrains.Annotations; /// Helper methods used during serialization of values to the tuple binary format - public static class FdbTuplePackers + public static class TuplePackers { #region Serializers... @@ -66,20 +66,20 @@ private static Delegate GetSerializerFor([NotNull] Type type) if (type == typeof(object)) { // return a generic serializer that will inspect the runtime type of the object - return new Encoder(FdbTuplePackers.SerializeObjectTo); + return new Encoder(TuplePackers.SerializeObjectTo); } var typeArgs = new[] { typeof(TupleWriter).MakeByRefType(), type }; - var method = typeof(FdbTuplePackers).GetMethod("SerializeTo", BindingFlags.Static | BindingFlags.Public, null, typeArgs, null); + var method = typeof(TuplePackers).GetMethod("SerializeTo", BindingFlags.Static | BindingFlags.Public, null, typeArgs, null); if (method != null) { // we have a direct serializer return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } // maybe if it is a tuple ? - if (typeof(IFdbTuple).IsAssignableFrom(type)) + if (typeof(ITuple).IsAssignableFrom(type)) { - method = typeof(FdbTuplePackers).GetMethod("SerializeTupleTo", BindingFlags.Static | BindingFlags.Public); + method = typeof(TuplePackers).GetMethod("SerializeTupleTo", BindingFlags.Static | BindingFlags.Public); if (method != null) { return method.MakeGenericMethod(type).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); @@ -88,7 +88,7 @@ private static Delegate GetSerializerFor([NotNull] Type type) if (typeof(ITupleFormattable).IsAssignableFrom(type)) { - method = typeof(FdbTuplePackers).GetMethod("SerializeFormattableTo", BindingFlags.Static | BindingFlags.Public); + method = typeof(TuplePackers).GetMethod("SerializeFormattableTo", BindingFlags.Static | BindingFlags.Public); if (method != null) { return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); @@ -98,7 +98,7 @@ private static Delegate GetSerializerFor([NotNull] Type type) var nullableType = Nullable.GetUnderlyingType(type); if (nullableType != null) { // nullable types can reuse the underlying type serializer - method = typeof(FdbTuplePackers).GetMethod("SerializeNullableTo", BindingFlags.Static | BindingFlags.Public); + method = typeof(TuplePackers).GetMethod("SerializeNullableTo", BindingFlags.Static | BindingFlags.Public); if (method != null) { return method.MakeGenericMethod(nullableType).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); @@ -120,9 +120,9 @@ public static void SerializeNullableTo(ref TupleWriter writer, T? value) where T : struct { if (value == null) - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); else - FdbTuplePacker.Encoder(ref writer, value.Value); + TuplePacker.Encoder(ref writer, value.Value); } /// Serialize an untyped object, by checking its type at runtime @@ -134,7 +134,7 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) if (value == null) { // null value // includes all null references to ref types, as nullables where HasValue == false - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); return; } @@ -190,7 +190,7 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) } case TypeCode.DBNull: { // same as null - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); return; } case TypeCode.Boolean: @@ -266,7 +266,7 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) } } - var tuple = value as IFdbTuple; + var tuple = value as ITuple; if (tuple != null) { SerializeTupleTo(ref writer, tuple); @@ -291,22 +291,22 @@ public static void SerializeTo(ref TupleWriter writer, Slice value) { if (value.IsNull) { - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); } else if (value.Offset == 0 && value.Count == value.Array.Length) { - FdbTupleParser.WriteBytes(ref writer, value.Array); + TupleParser.WriteBytes(ref writer, value.Array); } else { - FdbTupleParser.WriteBytes(ref writer, value.Array, value.Offset, value.Count); + TupleParser.WriteBytes(ref writer, value.Array, value.Offset, value.Count); } } /// Writes a byte[] array public static void SerializeTo(ref TupleWriter writer, byte[] value) { - FdbTupleParser.WriteBytes(ref writer, value); + TupleParser.WriteBytes(ref writer, value); } /// Writes an array segment as a byte[] array @@ -318,14 +318,14 @@ public static void SerializeTo(ref TupleWriter writer, ArraySegment value) /// Writes a char as Unicode string public static void SerializeTo(ref TupleWriter writer, char value) { - FdbTupleParser.WriteChar(ref writer, value); + TupleParser.WriteChar(ref writer, value); } /// Writes a boolean as an integer /// Uses 0 for false, and -1 for true public static void SerializeTo(ref TupleWriter writer, bool value) { - FdbTupleParser.WriteBool(ref writer, value); + TupleParser.WriteBool(ref writer, value); } /// Writes a boolean as an integer or null @@ -333,78 +333,78 @@ public static void SerializeTo(ref TupleWriter writer, bool? value) { if (value == null) { // null => 00 - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); } else { - FdbTupleParser.WriteBool(ref writer, value.Value); + TupleParser.WriteBool(ref writer, value.Value); } } /// Writes a signed byte as an integer public static void SerializeTo(ref TupleWriter writer, sbyte value) { - FdbTupleParser.WriteInt32(ref writer, value); + TupleParser.WriteInt32(ref writer, value); } /// Writes an unsigned byte as an integer public static void SerializeTo(ref TupleWriter writer, byte value) { - FdbTupleParser.WriteByte(ref writer, value); + TupleParser.WriteByte(ref writer, value); } /// Writes a signed word as an integer public static void SerializeTo(ref TupleWriter writer, short value) { - FdbTupleParser.WriteInt32(ref writer, value); + TupleParser.WriteInt32(ref writer, value); } /// Writes an unsigned word as an integer public static void SerializeTo(ref TupleWriter writer, ushort value) { - FdbTupleParser.WriteUInt32(ref writer, value); + TupleParser.WriteUInt32(ref writer, value); } /// Writes a signed int as an integer public static void SerializeTo(ref TupleWriter writer, int value) { - FdbTupleParser.WriteInt32(ref writer, value); + TupleParser.WriteInt32(ref writer, value); } /// Writes an unsigned int as an integer public static void SerializeTo(ref TupleWriter writer, uint value) { - FdbTupleParser.WriteUInt32(ref writer, value); + TupleParser.WriteUInt32(ref writer, value); } /// Writes a signed long as an integer public static void SerializeTo(ref TupleWriter writer, long value) { - FdbTupleParser.WriteInt64(ref writer, value); + TupleParser.WriteInt64(ref writer, value); } /// Writes an unsigned long as an integer public static void SerializeTo(ref TupleWriter writer, ulong value) { - FdbTupleParser.WriteUInt64(ref writer, value); + TupleParser.WriteUInt64(ref writer, value); } /// Writes a 32-bit IEEE floating point number public static void SerializeTo(ref TupleWriter writer, float value) { - FdbTupleParser.WriteSingle(ref writer, value); + TupleParser.WriteSingle(ref writer, value); } /// Writes a 64-bit IEEE floating point number public static void SerializeTo(ref TupleWriter writer, double value) { - FdbTupleParser.WriteDouble(ref writer, value); + TupleParser.WriteDouble(ref writer, value); } /// Writes a string as an Unicode string public static void SerializeTo(ref TupleWriter writer, string value) { - FdbTupleParser.WriteString(ref writer, value); + TupleParser.WriteString(ref writer, value); } /// Writes a DateTime converted to the number of days since the Unix Epoch and stored as a 64-bit decimal @@ -424,7 +424,7 @@ public static void SerializeTo(ref TupleWriter writer, DateTime value) const long UNIX_EPOCH_EPOCH = 621355968000000000L; double ms = (value.ToUniversalTime().Ticks - UNIX_EPOCH_EPOCH) / (double)TimeSpan.TicksPerDay; - FdbTupleParser.WriteDouble(ref writer, ms); + TupleParser.WriteDouble(ref writer, ms); } /// Writes a TimeSpan converted to to a number seconds encoded as a 64-bit decimal @@ -438,7 +438,7 @@ public static void SerializeTo(ref TupleWriter writer, TimeSpan value) // Right now, we will store the duration as the number of seconds, using a 64-bit float - FdbTupleParser.WriteDouble(ref writer, value.TotalSeconds); + TupleParser.WriteDouble(ref writer, value.TotalSeconds); } /// Writes a Guid as a 128-bit UUID @@ -446,25 +446,25 @@ public static void SerializeTo(ref TupleWriter writer, Guid value) { //REVIEW: should we consider serializing Guid.Empty as <14> (integer 0) ? or maybe <01><00> (empty bytestring) ? // => could spare ~16 bytes per key in indexes on GUID properties that are frequently missing or empty (== default(Guid)) - FdbTupleParser.WriteGuid(ref writer, value); + TupleParser.WriteGuid(ref writer, value); } /// Writes a Uuid as a 128-bit UUID public static void SerializeTo(ref TupleWriter writer, Uuid128 value) { - FdbTupleParser.WriteUuid128(ref writer, value); + TupleParser.WriteUuid128(ref writer, value); } /// Writes a Uuid as a 64-bit UUID public static void SerializeTo(ref TupleWriter writer, Uuid64 value) { - FdbTupleParser.WriteUuid64(ref writer, value); + TupleParser.WriteUuid64(ref writer, value); } /// Writes an IPaddress as a 32-bit (IPv4) or 128-bit (IPv6) byte array public static void SerializeTo(ref TupleWriter writer, System.Net.IPAddress value) { - FdbTupleParser.WriteBytes(ref writer, value != null ? value.GetAddressBytes() : null); + TupleParser.WriteBytes(ref writer, value != null ? value.GetAddressBytes() : null); } public static void SerializeTo(ref TupleWriter writer, FdbTupleAlias value) @@ -475,29 +475,29 @@ public static void SerializeTo(ref TupleWriter writer, FdbTupleAlias value) } public static void SerializeTupleTo(ref TupleWriter writer, TTuple tuple) - where TTuple : IFdbTuple + where TTuple : ITuple { Contract.Requires(tuple != null); - FdbTupleParser.BeginTuple(ref writer); + TupleParser.BeginTuple(ref writer); tuple.PackTo(ref writer); - FdbTupleParser.EndTuple(ref writer); + TupleParser.EndTuple(ref writer); } public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormattable formattable) { if (formattable == null) { - FdbTupleParser.WriteNil(ref writer); + TupleParser.WriteNil(ref writer); return; } var tuple = formattable.ToTuple(); if (tuple == null) throw new InvalidOperationException(String.Format("Custom formatter {0}.ToTuple() cannot return null", formattable.GetType().Name)); - FdbTupleParser.BeginTuple(ref writer); + TupleParser.BeginTuple(ref writer); tuple.PackTo(ref writer); - FdbTupleParser.EndTuple(ref writer); + TupleParser.EndTuple(ref writer); } #endregion @@ -511,26 +511,26 @@ private static Dictionary InitializeDefaultUnpackers() { var map = new Dictionary(); - map[typeof(Slice)] = new Func(FdbTuplePackers.DeserializeSlice); - map[typeof(byte[])] = new Func(FdbTuplePackers.DeserializeBytes); - map[typeof(bool)] = new Func(FdbTuplePackers.DeserializeBoolean); - map[typeof(string)] = new Func(FdbTuplePackers.DeserializeString); - map[typeof(sbyte)] = new Func(FdbTuplePackers.DeserializeSByte); - map[typeof(short)] = new Func(FdbTuplePackers.DeserializeInt16); - map[typeof(int)] = new Func(FdbTuplePackers.DeserializeInt32); - map[typeof(long)] = new Func(FdbTuplePackers.DeserializeInt64); - map[typeof(byte)] = new Func(FdbTuplePackers.DeserializeByte); - map[typeof(ushort)] = new Func(FdbTuplePackers.DeserializeUInt16); - map[typeof(uint)] = new Func(FdbTuplePackers.DeserializeUInt32); - map[typeof(ulong)] = new Func(FdbTuplePackers.DeserializeUInt64); - map[typeof(float)] = new Func(FdbTuplePackers.DeserializeSingle); - map[typeof(double)] = new Func(FdbTuplePackers.DeserializeDouble); - map[typeof(Guid)] = new Func(FdbTuplePackers.DeserializeGuid); - map[typeof(Uuid128)] = new Func(FdbTuplePackers.DeserializeUuid128); - map[typeof(Uuid64)] = new Func(FdbTuplePackers.DeserializeUuid64); - map[typeof(TimeSpan)] = new Func(FdbTuplePackers.DeserializeTimeSpan); - map[typeof(DateTime)] = new Func(FdbTuplePackers.DeserializeDateTime); - map[typeof(System.Net.IPAddress)] = new Func(FdbTuplePackers.DeserializeIPAddress); + map[typeof(Slice)] = new Func(TuplePackers.DeserializeSlice); + map[typeof(byte[])] = new Func(TuplePackers.DeserializeBytes); + map[typeof(bool)] = new Func(TuplePackers.DeserializeBoolean); + map[typeof(string)] = new Func(TuplePackers.DeserializeString); + map[typeof(sbyte)] = new Func(TuplePackers.DeserializeSByte); + map[typeof(short)] = new Func(TuplePackers.DeserializeInt16); + map[typeof(int)] = new Func(TuplePackers.DeserializeInt32); + map[typeof(long)] = new Func(TuplePackers.DeserializeInt64); + map[typeof(byte)] = new Func(TuplePackers.DeserializeByte); + map[typeof(ushort)] = new Func(TuplePackers.DeserializeUInt16); + map[typeof(uint)] = new Func(TuplePackers.DeserializeUInt32); + map[typeof(ulong)] = new Func(TuplePackers.DeserializeUInt64); + map[typeof(float)] = new Func(TuplePackers.DeserializeSingle); + map[typeof(double)] = new Func(TuplePackers.DeserializeDouble); + map[typeof(Guid)] = new Func(TuplePackers.DeserializeGuid); + map[typeof(Uuid128)] = new Func(TuplePackers.DeserializeUuid128); + map[typeof(Uuid64)] = new Func(TuplePackers.DeserializeUuid64); + map[typeof(TimeSpan)] = new Func(TuplePackers.DeserializeTimeSpan); + map[typeof(DateTime)] = new Func(TuplePackers.DeserializeDateTime); + map[typeof(System.Net.IPAddress)] = new Func(TuplePackers.DeserializeIPAddress); // add Nullable versions for all these types return map; @@ -571,7 +571,7 @@ internal static Func GetDeserializer(bool required) /// Check if a tuple segment is the equivalent of 'Nil' internal static bool IsNilSegment(Slice slice) { - return slice.IsNullOrEmpty || slice[0] == FdbTupleTypes.Nil; + return slice.IsNullOrEmpty || slice[0] == TupleTypes.Nil; } private static Delegate MakeNullableDeserializer([NotNull] Type nullableType, [NotNull] Type type, [NotNull] Delegate decoder) @@ -582,7 +582,7 @@ private static Delegate MakeNullableDeserializer([NotNull] Type nullableType, [N var prmSlice = Expression.Parameter(typeof(Slice), "slice"); var body = Expression.Condition( // IsNilSegment(slice) ? - Expression.Call(typeof(FdbTuplePackers).GetMethod("IsNilSegment", BindingFlags.Static | BindingFlags.NonPublic), prmSlice), + Expression.Call(typeof(TuplePackers).GetMethod("IsNilSegment", BindingFlags.Static | BindingFlags.NonPublic), prmSlice), // True => default(Nullable) Expression.Default(nullableType), // False => decoder(slice) @@ -602,28 +602,28 @@ public static object DeserializeBoxed(Slice slice) if (slice.IsNullOrEmpty) return null; int type = slice[0]; - if (type <= FdbTupleTypes.IntPos8) + if (type <= TupleTypes.IntPos8) { - if (type >= FdbTupleTypes.IntNeg8) return FdbTupleParser.ParseInt64(type, slice); + if (type >= TupleTypes.IntNeg8) return TupleParser.ParseInt64(type, slice); switch (type) { - case FdbTupleTypes.Nil: return null; - case FdbTupleTypes.Bytes: return FdbTupleParser.ParseBytes(slice); - case FdbTupleTypes.Utf8: return FdbTupleParser.ParseUnicode(slice); - case FdbTupleTypes.TupleStart: return FdbTupleParser.ParseTuple(slice); + case TupleTypes.Nil: return null; + case TupleTypes.Bytes: return TupleParser.ParseBytes(slice); + case TupleTypes.Utf8: return TupleParser.ParseUnicode(slice); + case TupleTypes.TupleStart: return TupleParser.ParseTuple(slice); } } else { switch (type) { - case FdbTupleTypes.Single: return FdbTupleParser.ParseSingle(slice); - case FdbTupleTypes.Double: return FdbTupleParser.ParseDouble(slice); - case FdbTupleTypes.Uuid128: return FdbTupleParser.ParseGuid(slice); - case FdbTupleTypes.Uuid64: return FdbTupleParser.ParseUuid64(slice); - case FdbTupleTypes.AliasDirectory: return FdbTupleAlias.Directory; - case FdbTupleTypes.AliasSystem: return FdbTupleAlias.System; + case TupleTypes.Single: return TupleParser.ParseSingle(slice); + case TupleTypes.Double: return TupleParser.ParseDouble(slice); + case TupleTypes.Uuid128: return TupleParser.ParseGuid(slice); + case TupleTypes.Uuid64: return TupleParser.ParseUuid64(slice); + case TupleTypes.AliasDirectory: return FdbTupleAlias.Directory; + case TupleTypes.AliasSystem: return FdbTupleAlias.System; } } @@ -638,12 +638,12 @@ public static object DeserializeBoxed(Slice slice) public static T DeserializeFormattable(Slice slice) where T : ITupleFormattable, new() { - if (FdbTuplePackers.IsNilSegment(slice)) + if (TuplePackers.IsNilSegment(slice)) { return default(T); } - var tuple = FdbTupleParser.ParseTuple(slice); + var tuple = TupleParser.ParseTuple(slice); var value = new T(); value.FromTuple(tuple); return value; @@ -657,7 +657,7 @@ public static T DeserializeFormattable(Slice slice) public static T DeserializeFormattable(Slice slice, [NotNull] Func factory) where T : ITupleFormattable { - var tuple = FdbTupleParser.ParseTuple(slice); + var tuple = TupleParser.ParseTuple(slice); var value = factory(); value.FromTuple(tuple); return value; @@ -674,20 +674,20 @@ public static Slice DeserializeSlice(Slice slice) byte type = slice[0]; switch(type) { - case FdbTupleTypes.Nil: return Slice.Nil; - case FdbTupleTypes.Bytes: return FdbTupleParser.ParseBytes(slice); - case FdbTupleTypes.Utf8: return Slice.FromString(FdbTupleParser.ParseUnicode(slice)); + case TupleTypes.Nil: return Slice.Nil; + case TupleTypes.Bytes: return TupleParser.ParseBytes(slice); + case TupleTypes.Utf8: return Slice.FromString(TupleParser.ParseUnicode(slice)); - case FdbTupleTypes.Single: return Slice.FromSingle(FdbTupleParser.ParseSingle(slice)); - case FdbTupleTypes.Double: return Slice.FromDouble(FdbTupleParser.ParseDouble(slice)); + case TupleTypes.Single: return Slice.FromSingle(TupleParser.ParseSingle(slice)); + case TupleTypes.Double: return Slice.FromDouble(TupleParser.ParseDouble(slice)); - case FdbTupleTypes.Uuid128: return Slice.FromGuid(FdbTupleParser.ParseGuid(slice)); - case FdbTupleTypes.Uuid64: return Slice.FromUuid64(FdbTupleParser.ParseUuid64(slice)); + case TupleTypes.Uuid128: return Slice.FromGuid(TupleParser.ParseGuid(slice)); + case TupleTypes.Uuid64: return Slice.FromUuid64(TupleParser.ParseUuid64(slice)); } - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { - if (type >= FdbTupleTypes.IntBase) return Slice.FromInt64(DeserializeInt64(slice)); + if (type >= TupleTypes.IntBase) return Slice.FromInt64(DeserializeInt64(slice)); return Slice.FromUInt64(DeserializeUInt64(slice)); } @@ -703,24 +703,24 @@ public static byte[] DeserializeBytes(Slice slice) /// Deserialize a tuple segment into a tuple [CanBeNull] - public static IFdbTuple DeserializeTuple(Slice slice) + public static ITuple DeserializeTuple(Slice slice) { if (slice.IsNullOrEmpty) return null; byte type = slice[0]; switch(type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return null; } - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { - return FdbTuple.Unpack(FdbTupleParser.ParseBytes(slice)); + return STuple.Unpack(TupleParser.ParseBytes(slice)); } - case FdbTupleTypes.TupleStart: + case TupleTypes.TupleStart: { - return FdbTupleParser.ParseTuple(slice); + return TupleParser.ParseTuple(slice); } } @@ -736,7 +736,7 @@ public static bool DeserializeBoolean(Slice slice) byte type = slice[0]; // Booleans are usually encoded as integers, with 0 for False (<14>) and 1 for True (<15><01>) - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { //note: DeserializeInt64 handles most cases return 0 != DeserializeInt64(slice); @@ -744,23 +744,23 @@ public static bool DeserializeBoolean(Slice slice) switch (type) { - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { // empty is false, all other is true return slice.Count != 2; // <01><00> } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: {// empty is false, all other is true return slice.Count != 2; // <02><00> } - case FdbTupleTypes.Single: + case TupleTypes.Single: { //TODO: should NaN considered to be false ? - return 0f != FdbTupleParser.ParseSingle(slice); + return 0f != TupleParser.ParseSingle(slice); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { //TODO: should NaN considered to be false ? - return 0f != FdbTupleParser.ParseDouble(slice); + return 0f != TupleParser.ParseDouble(slice); } } @@ -797,15 +797,15 @@ public static long DeserializeInt64(Slice slice) if (slice.IsNullOrEmpty) return 0L; //TODO: fail ? int type = slice[0]; - if (type <= FdbTupleTypes.IntPos8) + if (type <= TupleTypes.IntPos8) { - if (type >= FdbTupleTypes.IntNeg8) return FdbTupleParser.ParseInt64(type, slice); + if (type >= TupleTypes.IntNeg8) return TupleParser.ParseInt64(type, slice); switch (type) { - case FdbTupleTypes.Nil: return 0; - case FdbTupleTypes.Bytes: return long.Parse(FdbTupleParser.ParseAscii(slice), CultureInfo.InvariantCulture); - case FdbTupleTypes.Utf8: return long.Parse(FdbTupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + case TupleTypes.Nil: return 0; + case TupleTypes.Bytes: return long.Parse(TupleParser.ParseAscii(slice), CultureInfo.InvariantCulture); + case TupleTypes.Utf8: return long.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } } @@ -840,16 +840,16 @@ public static ulong DeserializeUInt64(Slice slice) if (slice.IsNullOrEmpty) return 0UL; //TODO: fail ? int type = slice[0]; - if (type <= FdbTupleTypes.IntPos8) + if (type <= TupleTypes.IntPos8) { - if (type >= FdbTupleTypes.IntZero) return (ulong)FdbTupleParser.ParseInt64(type, slice); - if (type < FdbTupleTypes.IntZero) throw new OverflowException(); // negative values + if (type >= TupleTypes.IntZero) return (ulong)TupleParser.ParseInt64(type, slice); + if (type < TupleTypes.IntZero) throw new OverflowException(); // negative values switch (type) { - case FdbTupleTypes.Nil: return 0; - case FdbTupleTypes.Bytes: return ulong.Parse(FdbTupleParser.ParseAscii(slice), CultureInfo.InvariantCulture); - case FdbTupleTypes.Utf8: return ulong.Parse(FdbTupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + case TupleTypes.Nil: return 0; + case TupleTypes.Bytes: return ulong.Parse(TupleParser.ParseAscii(slice), CultureInfo.InvariantCulture); + case TupleTypes.Utf8: return ulong.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } } @@ -863,25 +863,25 @@ public static float DeserializeSingle(Slice slice) byte type = slice[0]; switch (type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return 0; } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { - return Single.Parse(FdbTupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + return Single.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } - case FdbTupleTypes.Single: + case TupleTypes.Single: { - return FdbTupleParser.ParseSingle(slice); + return TupleParser.ParseSingle(slice); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { - return (float)FdbTupleParser.ParseDouble(slice); + return (float)TupleParser.ParseDouble(slice); } } - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { return checked((float)DeserializeInt64(slice)); } @@ -896,25 +896,25 @@ public static double DeserializeDouble(Slice slice) byte type = slice[0]; switch(type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return 0; } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { - return Double.Parse(FdbTupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + return Double.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } - case FdbTupleTypes.Single: + case TupleTypes.Single: { - return (double)FdbTupleParser.ParseSingle(slice); + return (double)TupleParser.ParseSingle(slice); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { - return FdbTupleParser.ParseDouble(slice); + return TupleParser.ParseDouble(slice); } } - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { return checked((double)DeserializeInt64(slice)); } @@ -934,28 +934,28 @@ public static DateTime DeserializeDateTime(Slice slice) switch(type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return DateTime.MinValue; } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { // we only support ISO 8601 dates. For ex: YYYY-MM-DDTHH:MM:SS.fffff" - string str = FdbTupleParser.ParseUnicode(slice); + string str = TupleParser.ParseUnicode(slice); return DateTime.Parse(str, CultureInfo.InvariantCulture, DateTimeStyles.RoundtripKind); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { // Number of days since Epoch const long UNIX_EPOCH_TICKS = 621355968000000000L; //note: we can't user TimeSpan.FromDays(...) because it rounds to the nearest millisecond! - long ticks = UNIX_EPOCH_TICKS + (long)(FdbTupleParser.ParseDouble(slice) * TimeSpan.TicksPerDay); + long ticks = UNIX_EPOCH_TICKS + (long)(TupleParser.ParseDouble(slice) * TimeSpan.TicksPerDay); return new DateTime(ticks, DateTimeKind.Utc); } } // If we have an integer, we consider it to be a number of Ticks (Windows Only) - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { return new DateTime(DeserializeInt64(slice), DateTimeKind.Utc); } @@ -975,23 +975,23 @@ public static TimeSpan DeserializeTimeSpan(Slice slice) switch(type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return TimeSpan.Zero; } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { // "HH:MM:SS.fffff" - return TimeSpan.Parse(FdbTupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + return TimeSpan.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { // Number of seconds //note: We can't use TimeSpan.FromSeconds(...) because it rounds to the nearest millisecond! - return new TimeSpan((long)(FdbTupleParser.ParseDouble(slice) * (double)TimeSpan.TicksPerSecond)); + return new TimeSpan((long)(TupleParser.ParseDouble(slice) * (double)TimeSpan.TicksPerSecond)); } } // If we have an integer, we consider it to be a number of Ticks (Windows Only) - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { return new TimeSpan(DeserializeInt64(slice)); } @@ -1009,39 +1009,39 @@ public static string DeserializeString(Slice slice) byte type = slice[0]; switch (type) { - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { return null; } - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { - return FdbTupleParser.ParseAscii(slice); + return TupleParser.ParseAscii(slice); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { - return FdbTupleParser.ParseUnicode(slice); + return TupleParser.ParseUnicode(slice); } - case FdbTupleTypes.Single: + case TupleTypes.Single: { - return FdbTupleParser.ParseSingle(slice).ToString(CultureInfo.InvariantCulture); + return TupleParser.ParseSingle(slice).ToString(CultureInfo.InvariantCulture); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { - return FdbTupleParser.ParseDouble(slice).ToString(CultureInfo.InvariantCulture); + return TupleParser.ParseDouble(slice).ToString(CultureInfo.InvariantCulture); } - case FdbTupleTypes.Uuid128: + case TupleTypes.Uuid128: { - return FdbTupleParser.ParseGuid(slice).ToString(); + return TupleParser.ParseGuid(slice).ToString(); } - case FdbTupleTypes.Uuid64: + case TupleTypes.Uuid64: { - return FdbTupleParser.ParseUuid64(slice).ToString(); + return TupleParser.ParseUuid64(slice).ToString(); } } - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { - return FdbTupleParser.ParseInt64(type, slice).ToString(CultureInfo.InvariantCulture); + return TupleParser.ParseInt64(type, slice).ToString(CultureInfo.InvariantCulture); } throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a String", type)); @@ -1057,17 +1057,17 @@ public static Guid DeserializeGuid(Slice slice) switch (type) { - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { - return Guid.Parse(FdbTupleParser.ParseAscii(slice)); + return Guid.Parse(TupleParser.ParseAscii(slice)); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { - return Guid.Parse(FdbTupleParser.ParseUnicode(slice)); + return Guid.Parse(TupleParser.ParseUnicode(slice)); } - case FdbTupleTypes.Uuid128: + case TupleTypes.Uuid128: { - return FdbTupleParser.ParseGuid(slice); + return TupleParser.ParseGuid(slice); } //REVIEW: should we allow converting a Uuid64 into a Guid? This looks more like a bug than an expected behavior... } @@ -1085,17 +1085,17 @@ public static Uuid128 DeserializeUuid128(Slice slice) switch (type) { - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { // expect binary representation as a 16-byte array - return new Uuid128(FdbTupleParser.ParseBytes(slice)); + return new Uuid128(TupleParser.ParseBytes(slice)); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { // expect text representation - return new Uuid128(FdbTupleParser.ParseUnicode(slice)); + return new Uuid128(TupleParser.ParseUnicode(slice)); } - case FdbTupleTypes.Uuid128: + case TupleTypes.Uuid128: { - return FdbTupleParser.ParseUuid128(slice); + return TupleParser.ParseUuid128(slice); } //REVIEW: should we allow converting a Uuid64 into a Uuid128? This looks more like a bug than an expected behavior... } @@ -1113,23 +1113,23 @@ public static Uuid64 DeserializeUuid64(Slice slice) switch (type) { - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { // expect binary representation as a 16-byte array - return new Uuid64(FdbTupleParser.ParseBytes(slice)); + return new Uuid64(TupleParser.ParseBytes(slice)); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { // expect text representation - return new Uuid64(FdbTupleParser.ParseUnicode(slice)); + return new Uuid64(TupleParser.ParseUnicode(slice)); } - case FdbTupleTypes.Uuid64: + case TupleTypes.Uuid64: { - return FdbTupleParser.ParseUuid64(slice); + return TupleParser.ParseUuid64(slice); } } - if (type >= FdbTupleTypes.IntZero && type <= FdbTupleTypes.IntPos8) + if (type >= TupleTypes.IntZero && type <= TupleTypes.IntPos8) { // expect 64-bit number - return new Uuid64(FdbTupleParser.ParseInt64(type, slice)); + return new Uuid64(TupleParser.ParseInt64(type, slice)); } // we don't support negative numbers! @@ -1147,23 +1147,23 @@ public static System.Net.IPAddress DeserializeIPAddress(Slice slice) switch (type) { - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { - return new System.Net.IPAddress(FdbTupleParser.ParseBytes(slice).GetBytes()); + return new System.Net.IPAddress(TupleParser.ParseBytes(slice).GetBytes()); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { - return System.Net.IPAddress.Parse(FdbTupleParser.ParseUnicode(slice)); + return System.Net.IPAddress.Parse(TupleParser.ParseUnicode(slice)); } - case FdbTupleTypes.Uuid128: + case TupleTypes.Uuid128: { // could be an IPv6 encoded as a 128-bits UUID return new System.Net.IPAddress(slice.GetBytes()); } } - if (type >= FdbTupleTypes.IntPos1 && type <= FdbTupleTypes.IntPos4) + if (type >= TupleTypes.IntPos1 && type <= TupleTypes.IntPos4) { // could be an IPv4 encoded as a 32-bit unsigned integer - var value = FdbTupleParser.ParseInt64(type, slice); + var value = TupleParser.ParseInt64(type, slice); Contract.Assert(value >= 0 && value <= uint.MaxValue); return new System.Net.IPAddress(value); } @@ -1181,7 +1181,7 @@ public static FdbTupleAlias DeserializeAlias(Slice slice) /// Slice that contains the packed representation of a tuple with zero or more elements /// Decoded tuple [NotNull] - internal static FdbSlicedTuple Unpack(Slice buffer, bool embedded) + internal static SlicedTuple Unpack(Slice buffer, bool embedded) { var reader = new TupleReader(buffer); if (embedded) reader.Depth = 1; @@ -1191,7 +1191,7 @@ internal static FdbSlicedTuple Unpack(Slice buffer, bool embedded) Slice item; int p = 0; - while ((item = FdbTupleParser.ParseNext(ref reader)).HasValue) + while ((item = TupleParser.ParseNext(ref reader)).HasValue) { if (p >= items.Length) { @@ -1202,7 +1202,7 @@ internal static FdbSlicedTuple Unpack(Slice buffer, bool embedded) } if (reader.Input.HasMore) throw new FormatException("Parsing of tuple failed failed before reaching the end of the key"); - return new FdbSlicedTuple(p == 0 ? Slice.EmptySliceArray : items, 0, p); + return new SlicedTuple(p == 0 ? Slice.EmptySliceArray : items, 0, p); } /// Ensure that a slice is a packed tuple that contains a single and valid element @@ -1212,7 +1212,7 @@ public static Slice UnpackSingle(Slice buffer) { var slicer = new TupleReader(buffer); - var current = FdbTupleParser.ParseNext(ref slicer); + var current = TupleParser.ParseNext(ref slicer); if (slicer.Input.HasMore) throw new FormatException("Parsing of singleton tuple failed before reaching the end of the key"); return current; @@ -1225,7 +1225,7 @@ public static Slice UnpackFirst(Slice buffer) { var slicer = new TupleReader(buffer); - return FdbTupleParser.ParseNext(ref slicer); + return TupleParser.ParseNext(ref slicer); } /// Only returns the last item of a packed tuple @@ -1238,7 +1238,7 @@ public static Slice UnpackLast(Slice buffer) Slice item = Slice.Nil; Slice current; - while ((current = FdbTupleParser.ParseNext(ref slicer)).HasValue) + while ((current = TupleParser.ParseNext(ref slicer)).HasValue) { item = current; } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs similarity index 88% rename from FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs index 092a9ed15..576400166 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs @@ -35,7 +35,7 @@ namespace FoundationDB.Layers.Tuples using JetBrains.Annotations; /// Helper class that contains low-level encoders for the tuple binary format - public static class FdbTupleParser + public static class TupleParser { #region Serialization... @@ -44,11 +44,11 @@ public static void WriteNil(ref TupleWriter writer) { if (writer.Depth == 0) { // at the top level, NILs are escaped as <00> - writer.Output.WriteByte(FdbTupleTypes.Nil); + writer.Output.WriteByte(TupleTypes.Nil); } else { // inside a tuple, NILs are escaped as <00> - writer.Output.WriteByte2(FdbTupleTypes.Nil, 0xFF); + writer.Output.WriteByte2(TupleTypes.Nil, 0xFF); } } @@ -57,11 +57,11 @@ public static void WriteBool(ref TupleWriter writer, bool value) // To be compatible with other bindings, we will encode False as the number 0, and True as the number 1 if (value) { // true => 15 01 - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, 1); + writer.Output.WriteByte2(TupleTypes.IntPos1, 1); } else { // false => 14 - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); } } @@ -72,11 +72,11 @@ public static void WriteByte(ref TupleWriter writer, byte value) { if (value == 0) { // zero - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); } else { // 1..255: frequent for array index - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, value); + writer.Output.WriteByte2(TupleTypes.IntPos1, value); } } @@ -89,19 +89,19 @@ public static void WriteInt32(ref TupleWriter writer, int value) { if (value == 0) { // zero - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); return; } if (value > 0) { // 1..255: frequent for array index - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, (byte)value); + writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); return; } if (value > -256) { // -255..-1 - writer.Output.WriteByte2(FdbTupleTypes.IntNeg1, (byte)(255 + value)); + writer.Output.WriteByte2(TupleTypes.IntNeg1, (byte)(255 + value)); return; } } @@ -118,19 +118,19 @@ public static void WriteInt64(ref TupleWriter writer, long value) { if (value == 0) { // zero - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); return; } if (value > 0) { // 1..255: frequent for array index - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, (byte)value); + writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); return; } if (value > -256) { // -255..-1 - writer.Output.WriteByte2(FdbTupleTypes.IntNeg1, (byte)(255 + value)); + writer.Output.WriteByte2(TupleTypes.IntNeg1, (byte)(255 + value)); return; } } @@ -153,7 +153,7 @@ private static void WriteInt64Slow(ref TupleWriter writer, long value) ulong v; if (value > 0) { // simple case - buffer[p++] = (byte)(FdbTupleTypes.IntBase + bytes); + buffer[p++] = (byte)(TupleTypes.IntBase + bytes); v = (ulong)value; } else @@ -161,7 +161,7 @@ private static void WriteInt64Slow(ref TupleWriter writer, long value) // -1 => 0xFE // -256 => 0xFFFE // -65536 => 0xFFFFFE - buffer[p++] = (byte)(FdbTupleTypes.IntBase - bytes); + buffer[p++] = (byte)(TupleTypes.IntBase - bytes); v = (ulong)(~(-value)); } @@ -191,11 +191,11 @@ public static void WriteUInt32(ref TupleWriter writer, uint value) { if (value == 0) { // 0 - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); } else { // 1..255 - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, (byte)value); + writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); } } else @@ -213,11 +213,11 @@ public static void WriteUInt64(ref TupleWriter writer, ulong value) { if (value == 0) { // 0 - writer.Output.WriteByte(FdbTupleTypes.IntZero); + writer.Output.WriteByte(TupleTypes.IntZero); } else { // 1..255 - writer.Output.WriteByte2(FdbTupleTypes.IntPos1, (byte)value); + writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); } } else @@ -239,7 +239,7 @@ private static void WriteUInt64Slow(ref TupleWriter writer, ulong value) int p = writer.Output.Position; // simple case (ulong can only be positive) - buffer[p++] = (byte)(FdbTupleTypes.IntBase + bytes); + buffer[p++] = (byte)(TupleTypes.IntBase + bytes); if (bytes > 0) { @@ -287,7 +287,7 @@ public static void WriteSingle(ref TupleWriter writer, float value) writer.Output.EnsureBytes(5); var buffer = writer.Output.Buffer; int p = writer.Output.Position; - buffer[p + 0] = FdbTupleTypes.Single; + buffer[p + 0] = TupleTypes.Single; buffer[p + 1] = (byte)(bits >> 24); buffer[p + 2] = (byte)(bits >> 16); buffer[p + 3] = (byte)(bits >> 8); @@ -323,7 +323,7 @@ public static void WriteDouble(ref TupleWriter writer, double value) writer.Output.EnsureBytes(9); var buffer = writer.Output.Buffer; int p = writer.Output.Position; - buffer[p] = FdbTupleTypes.Double; + buffer[p] = TupleTypes.Double; buffer[p + 1] = (byte)(bits >> 56); buffer[p + 2] = (byte)(bits >> 48); buffer[p + 3] = (byte)(bits >> 40); @@ -344,7 +344,7 @@ public static void WriteBytes(ref TupleWriter writer, byte[] value) } else { - WriteNulEscapedBytes(ref writer, FdbTupleTypes.Bytes, value); + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value); } } @@ -357,7 +357,7 @@ public static unsafe void WriteString(ref TupleWriter writer, string value) } else if (value.Length == 0) { // "02 00" - writer.Output.WriteByte2(FdbTupleTypes.Utf8, 0x00); + writer.Output.WriteByte2(TupleTypes.Utf8, 0x00); } else { @@ -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, TupleTypes.Utf8, Encoding.UTF8.GetBytes(value)); } } } @@ -384,7 +384,7 @@ internal static unsafe void WriteChars(ref TupleWriter writer, char[] value, int } else { // "02 00" - writer.Output.WriteByte2(FdbTupleTypes.Utf8, 0x00); + writer.Output.WriteByte2(TupleTypes.Utf8, 0x00); } } else @@ -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, TupleTypes.Utf8, Encoding.UTF8.GetBytes(value, 0, count)); } } } @@ -410,7 +410,7 @@ private static unsafe void WriteUnescapedAsciiChars(ref TupleWriter writer, char char* end = chars + count; fixed (byte* buffer = writer.Output.Buffer) { - buffer[pos++] = FdbTupleTypes.Utf8; + buffer[pos++] = TupleTypes.Utf8; //OPTIMIZE: copy 2 or 4 chars at once, unroll loop? while(chars < end) { @@ -478,7 +478,7 @@ private static unsafe bool TryWriteUnescapedUtf8String(ref TupleWriter writer, c // * Western languages have a few chars that usually need 2 bytes. If we pre-allocate 50% more bytes, it should fit most of the time, without too much waste // * Eastern langauges will have all chars encoded to 3 bytes. If we also pre-allocated 50% more, we should only need one resize of the buffer (150% x 2 = 300%), which is acceptable writer.Output.EnsureBytes(checked(2 + count + (count >> 1))); // preallocate 150% of the string + 2 bytes - writer.Output.UnsafeWriteByte(FdbTupleTypes.Utf8); + writer.Output.UnsafeWriteByte(TupleTypes.Utf8); var encoder = Encoding.UTF8.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. @@ -510,15 +510,15 @@ public static void WriteChar(ref TupleWriter writer, char value) if (value == 0) { // NUL => "00 0F" // note: \0 is the only unicode character that will produce a zero byte when converted in UTF-8 - writer.Output.WriteByte4(FdbTupleTypes.Utf8, 0x00, 0xFF, 0x00); + writer.Output.WriteByte4(TupleTypes.Utf8, 0x00, 0xFF, 0x00); } else if (value < 0x80) { // 0x00..0x7F => 0xxxxxxx - writer.Output.WriteByte3(FdbTupleTypes.Utf8, (byte)value, 0x00); + writer.Output.WriteByte3(TupleTypes.Utf8, (byte)value, 0x00); } else if (value < 0x800) { // 0x80..0x7FF => 110xxxxx 10xxxxxx => two bytes - writer.Output.WriteByte4(FdbTupleTypes.Utf8, (byte)(0xC0 | (value >> 6)), (byte)(0x80 | (value & 0x3F)), 0x00); + writer.Output.WriteByte4(TupleTypes.Utf8, (byte)(0xC0 | (value >> 6)), (byte)(0x80 | (value & 0x3F)), 0x00); } else { // 0x800..0xFFFF => 11110xxx 10xxxxxx 10xxxxxx 10xxxxxx @@ -526,7 +526,7 @@ public static void WriteChar(ref TupleWriter writer, char value) // => 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)); writer.Output.EnsureBytes(tmp.Length + 2); - writer.Output.UnsafeWriteByte(FdbTupleTypes.Utf8); + writer.Output.UnsafeWriteByte(TupleTypes.Utf8); writer.Output.UnsafeWriteBytes(tmp, 0, tmp.Length); writer.Output.UnsafeWriteByte(0x00); } @@ -535,13 +535,13 @@ public static void WriteChar(ref TupleWriter writer, char value) /// Writes a binary string public static void WriteBytes(ref TupleWriter writer, [NotNull] byte[] value, int offset, int count) { - WriteNulEscapedBytes(ref writer, FdbTupleTypes.Bytes, value, offset, count); + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value, offset, count); } /// Writes a binary string public static void WriteBytes(ref TupleWriter writer, ArraySegment value) { - WriteNulEscapedBytes(ref writer, FdbTupleTypes.Bytes, value.Array, value.Offset, value.Count); + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value.Array, value.Offset, value.Count); } /// Writes a buffer with all instances of 0 escaped as '00 FF' @@ -620,7 +620,7 @@ private static void WriteNulEscapedBytes(ref TupleWriter writer, byte type, [Not public static void WriteGuid(ref TupleWriter writer, Guid value) { writer.Output.EnsureBytes(17); - writer.Output.UnsafeWriteByte(FdbTupleTypes.Uuid128); + writer.Output.UnsafeWriteByte(TupleTypes.Uuid128); unsafe { // UUIDs are stored using the RFC 4122 standard, so we need to swap some parts of the System.Guid @@ -635,7 +635,7 @@ public static void WriteGuid(ref TupleWriter writer, Guid value) public static void WriteUuid128(ref TupleWriter writer, Uuid128 value) { writer.Output.EnsureBytes(17); - writer.Output.UnsafeWriteByte(FdbTupleTypes.Uuid128); + writer.Output.UnsafeWriteByte(TupleTypes.Uuid128); unsafe { byte* ptr = stackalloc byte[16]; @@ -648,7 +648,7 @@ public static void WriteUuid128(ref TupleWriter writer, Uuid128 value) public static void WriteUuid64(ref TupleWriter writer, Uuid64 value) { writer.Output.EnsureBytes(9); - writer.Output.UnsafeWriteByte(FdbTupleTypes.Uuid64); + writer.Output.UnsafeWriteByte(TupleTypes.Uuid64); unsafe { byte* ptr = stackalloc byte[8]; @@ -661,7 +661,7 @@ public static void WriteUuid64(ref TupleWriter writer, Uuid64 value) public static void BeginTuple(ref TupleWriter writer) { writer.Depth++; - writer.Output.WriteByte(FdbTupleTypes.TupleStart); + writer.Output.WriteByte(TupleTypes.TupleStart); } /// Mark the end of an embedded tuple @@ -679,7 +679,7 @@ public static void EndTuple(ref TupleWriter writer) /// This method should only be used by custom decoders. public static long ParseInt64(int type, Slice slice) { - int bytes = type - FdbTupleTypes.IntBase; + int bytes = type - TupleTypes.IntBase; if (bytes == 0) return 0L; bool neg = false; @@ -755,7 +755,7 @@ internal static ArraySegment UnescapeByteStringSlow([NotNull] byte[] buffe /// Parse a tuple segment containing a byte array public static Slice ParseBytes(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Bytes && slice[-1] == 0); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Bytes && slice[-1] == 0); if (slice.Count <= 2) return Slice.Empty; var decoded = UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); @@ -766,7 +766,7 @@ public static Slice ParseBytes(Slice slice) /// Parse a tuple segment containing an ASCII string stored as a byte array public static string ParseAscii(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Bytes && slice[-1] == 0); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Bytes && slice[-1] == 0); if (slice.Count <= 2) return String.Empty; @@ -778,7 +778,7 @@ public static string ParseAscii(Slice slice) /// Parse a tuple segment containing a unicode string public static string ParseUnicode(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Utf8 && slice[-1] == 0); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Utf8 && slice[-1] == 0); if (slice.Count <= 2) return String.Empty; //TODO: check args @@ -787,18 +787,18 @@ public static string ParseUnicode(Slice slice) } /// Parse a tuple segment containing an embedded tuple - public static IFdbTuple ParseTuple(Slice slice) + public static ITuple ParseTuple(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.TupleStart && slice[-1] == 0); - if (slice.Count <= 2) return FdbTuple.Empty; + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.TupleStart && slice[-1] == 0); + if (slice.Count <= 2) return STuple.Empty; - return FdbTuplePackers.Unpack(slice.Substring(1, slice.Count - 2), true); + return TuplePackers.Unpack(slice.Substring(1, slice.Count - 2), true); } /// Parse a tuple segment containing a single precision number (float32) public static float ParseSingle(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Single); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Single); if (slice.Count != 5) { @@ -828,7 +828,7 @@ public static float ParseSingle(Slice slice) /// Parse a tuple segment containing a double precision number (float64) public static double ParseDouble(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Double); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Double); if (slice.Count != 9) { @@ -859,7 +859,7 @@ public static double ParseDouble(Slice slice) /// Parse a tuple segment containing a 128-bit GUID public static Guid ParseGuid(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Uuid128); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid128); if (slice.Count != 17) { @@ -873,7 +873,7 @@ public static Guid ParseGuid(Slice slice) /// Parse a tuple segment containing a 128-bit UUID public static Uuid128 ParseUuid128(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Uuid128); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid128); if (slice.Count != 17) { @@ -886,7 +886,7 @@ public static Uuid128 ParseUuid128(Slice slice) /// Parse a tuple segment containing a 64-bit UUID public static Uuid64 ParseUuid64(Slice slice) { - Contract.Requires(slice.HasValue && slice[0] == FdbTupleTypes.Uuid64); + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid64); if (slice.Count != 9) { @@ -913,7 +913,7 @@ public static Slice ParseNext(ref TupleReader reader) return Slice.Nil; } - case FdbTupleTypes.Nil: + case TupleTypes.Nil: { // <00> / <00> => null if (reader.Depth > 0) { // must be <00> inside an embedded tuple @@ -935,53 +935,53 @@ public static Slice ParseNext(ref TupleReader reader) } } - case FdbTupleTypes.Bytes: + case TupleTypes.Bytes: { // <01>(bytes)<00> return reader.Input.ReadByteString(); } - case FdbTupleTypes.Utf8: + case TupleTypes.Utf8: { // <02>(utf8 bytes)<00> return reader.Input.ReadByteString(); } - case FdbTupleTypes.TupleStart: + case TupleTypes.TupleStart: { // <03>(packed tuple)<04> //PERF: currently, we will first scan to get all the bytes of this tuple, and parse it later. // This means that we may need to scan multiple times the bytes, which may not be efficient if there are multiple embedded tuples inside each other return ReadEmbeddedTupleBytes(ref reader); } - case FdbTupleTypes.Single: + case TupleTypes.Single: { // <20>(4 bytes) return reader.Input.ReadBytes(5); } - case FdbTupleTypes.Double: + case TupleTypes.Double: { // <21>(8 bytes) return reader.Input.ReadBytes(9); } - case FdbTupleTypes.Uuid128: + case TupleTypes.Uuid128: { // <30>(16 bytes) return reader.Input.ReadBytes(17); } - case FdbTupleTypes.Uuid64: + case TupleTypes.Uuid64: { // <31>(8 bytes) return reader.Input.ReadBytes(9); } - case FdbTupleTypes.AliasDirectory: - case FdbTupleTypes.AliasSystem: + case TupleTypes.AliasDirectory: + case TupleTypes.AliasSystem: { // or return reader.Input.ReadBytes(1); } } - if (type <= FdbTupleTypes.IntPos8 && type >= FdbTupleTypes.IntNeg8) + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { - int bytes = type - FdbTupleTypes.IntZero; + int bytes = type - TupleTypes.IntZero; if (bytes < 0) bytes = -bytes; return reader.Input.ReadBytes(1 + bytes); @@ -1028,7 +1028,7 @@ public static bool Skip(ref TupleReader reader, int count) while (count-- > 0) { if (!reader.Input.HasMore) return false; - var token = FdbTupleParser.ParseNext(ref reader); + var token = TupleParser.ParseNext(ref reader); if (token.IsNull) return false; } return true; @@ -1038,11 +1038,11 @@ public static bool Skip(ref TupleReader reader, int count) /// Reader positionned at the start of a packed tuple /// Lambda called for each segment of a tuple. Returns true to continue parsing, or false to stop /// Number of tokens that have been visited until either returned false, or reached the end. - public static T VisitNext(ref TupleReader reader, Func visitor) + public static T VisitNext(ref TupleReader reader, Func visitor) { if (!reader.Input.HasMore) throw new InvalidOperationException("The reader has already reached the end"); - var token = FdbTupleParser.ParseNext(ref reader); - return visitor(token, FdbTupleTypes.DecodeSegmentType(ref token)); + var token = TupleParser.ParseNext(ref reader); + return visitor(token, TupleTypes.DecodeSegmentType(ref token)); } #endregion diff --git a/FoundationDB.Client/Layers/Tuples/TupleReader.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TupleReader.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs similarity index 85% rename from FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs index d52438caa..71e133b9a 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs @@ -34,7 +34,7 @@ namespace FoundationDB.Layers.Tuples /// /// Constants for the various tuple value types /// - internal static class FdbTupleTypes + internal static class TupleTypes { /// Null/Empty/Void internal const byte Nil = 0; @@ -88,34 +88,34 @@ internal static class FdbTupleTypes internal const byte AliasSystem = 255; /// Return the type of a tuple segment, from its header - public static FdbTupleSegmentType DecodeSegmentType(ref Slice segment) + public static TupleSegmentType DecodeSegmentType(ref Slice segment) { - if (segment.Count == 0) return FdbTupleSegmentType.Nil; + if (segment.Count == 0) return TupleSegmentType.Nil; int type = segment[0]; switch(type) { - case Nil: return FdbTupleSegmentType.Nil; - case Bytes: return FdbTupleSegmentType.ByteString; - case Utf8: return FdbTupleSegmentType.UnicodeString; - case TupleStart: return FdbTupleSegmentType.Tuple; - case Single: return FdbTupleSegmentType.Single; - case Double: return FdbTupleSegmentType.Double; - case Uuid128: return FdbTupleSegmentType.Uuid128; - case Uuid64: return FdbTupleSegmentType.Uuid64; + case Nil: return TupleSegmentType.Nil; + case Bytes: return TupleSegmentType.ByteString; + case Utf8: return TupleSegmentType.UnicodeString; + case TupleStart: return TupleSegmentType.Tuple; + case Single: return TupleSegmentType.Single; + case Double: return TupleSegmentType.Double; + case Uuid128: return TupleSegmentType.Uuid128; + case Uuid64: return TupleSegmentType.Uuid64; } if (type <= IntPos8 && type >= IntNeg8) { - return FdbTupleSegmentType.Integer; + return TupleSegmentType.Integer; } - return FdbTupleSegmentType.Invalid; + return TupleSegmentType.Invalid; } } /// Logical type of packed element of a tuple - public enum FdbTupleSegmentType + public enum TupleSegmentType { Invalid = -1, Nil = 0, diff --git a/FoundationDB.Client/Layers/Tuples/TupleWriter.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TupleWriter.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FdbAnonymousTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs similarity index 78% rename from FoundationDB.Client/Layers/Tuples/Formatters/FdbAnonymousTupleFormatter.cs rename to FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs index 806d8c228..45cf18398 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FdbAnonymousTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs @@ -31,28 +31,28 @@ namespace FoundationDB.Layers.Tuples using System; /// Customer formatter that will called the provided lambda functions to convert to and from a tuple - internal sealed class FdbAnonymousTupleFormatter : ITupleFormatter + internal sealed class AnonymousTupleFormatter : ITupleFormatter { - private readonly Func m_to; - private readonly Func m_from; + private readonly Func m_to; + private readonly Func m_from; - public FdbAnonymousTupleFormatter(Func to, Func from) + public AnonymousTupleFormatter(Func to, Func from) { - if (to == null) throw new ArgumentNullException("to"); - if (from == null) throw new ArgumentNullException("from"); + if (to == null) throw new ArgumentNullException(nameof(to)); + if (from == null) throw new ArgumentNullException(nameof(@from)); m_to = to; m_from = from; } - public IFdbTuple ToTuple(T key) + public ITuple ToTuple(T key) { return m_to(key); } - public T FromTuple(IFdbTuple tuple) + public T FromTuple(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return m_from(tuple); } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FdbFormattableTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs similarity index 89% rename from FoundationDB.Client/Layers/Tuples/Formatters/FdbFormattableTupleFormatter.cs rename to FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs index fe5ed5b0d..19a18bc39 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FdbFormattableTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs @@ -31,18 +31,18 @@ namespace FoundationDB.Layers.Tuples using System; /// Specialized formatter for types that implement ITupleFormattable - internal sealed class FdbFormattableTupleFormatter : ITupleFormatter + internal sealed class FormattableTupleFormatter : ITupleFormatter where T : ITupleFormattable, new() { - public IFdbTuple ToTuple(T key) + public ITuple ToTuple(T key) { if (key == null) return null; return key.ToTuple(); } - public T FromTuple(IFdbTuple tuple) + public T FromTuple(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); var key = new T(); key.FromTuple(tuple); return key; diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FdbGenericTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs similarity index 90% rename from FoundationDB.Client/Layers/Tuples/Formatters/FdbGenericTupleFormatter.cs rename to FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs index 29edfb904..7b432f075 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FdbGenericTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs @@ -31,15 +31,15 @@ namespace FoundationDB.Layers.Tuples using System; /// Simple key formatter that maps a value into a singleton tuple, and back - internal sealed class FdbGenericTupleFormatter : ITupleFormatter + internal sealed class GenericTupleFormatter : ITupleFormatter { - public IFdbTuple ToTuple(T key) + public ITuple ToTuple(T key) { - return FdbTuple.Create(key); + return STuple.Create(key); } - public T FromTuple(IFdbTuple tuple) + public T FromTuple(ITuple tuple) { return tuple.OfSize(1).Get(0); } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs index 1e1c86b6d..cb2645056 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs @@ -36,11 +36,11 @@ namespace FoundationDB.Layers.Tuples public interface ITupleFormattable { /// Return the tuple representation of this instance - IFdbTuple ToTuple(); + ITuple ToTuple(); /// Load a tuple representation into a newly created instance /// - void FromTuple(IFdbTuple tuple); + void FromTuple(ITuple tuple); } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs index 6e3dea0f0..9e11904ab 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs @@ -39,12 +39,12 @@ public interface ITupleFormatter /// Convert a key into a tuple sequence /// Key to convert to a tuple /// Tuple that represent the key (can contain a single item for primitive keys, or several items for composite keys) - IFdbTuple ToTuple(TKey key); + ITuple ToTuple(TKey key); /// Convert a tuple sequence into a key /// Tuple to convert back into a key /// Key that corresponds to the tuple - TKey FromTuple(IFdbTuple tuple); + TKey FromTuple(ITuple tuple); } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FdbTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs similarity index 80% rename from FoundationDB.Client/Layers/Tuples/Formatters/FdbTupleFormatter.cs rename to FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs index 64dd5a875..49d3ab565 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FdbTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs @@ -31,7 +31,7 @@ namespace FoundationDB.Layers.Tuples using System; /// Helper class to get or create tuple formatters - public static class FdbTupleFormatter + public static class TupleFormatter { private static ITupleFormatter s_default; @@ -54,22 +54,22 @@ public static ITupleFormatter Default /// Lambda that is called to convert a value into a tuple. It SHOULD NOT return null. /// Lambda that is called to convert a tuple back into a value. It CAN return null. /// Custom formatter - public static ITupleFormatter Create(Func from, Func to) + public static ITupleFormatter Create(Func from, Func to) { - return new FdbAnonymousTupleFormatter(from, to); + return new AnonymousTupleFormatter(from, to); } /// Create a formatter that just add or remove a prefix to values - public static ITupleFormatter CreateAppender(IFdbTuple prefix) + public static ITupleFormatter CreateAppender(ITuple prefix) { - if (prefix == null) throw new ArgumentNullException("prefix"); + if (prefix == null) throw new ArgumentNullException(nameof(prefix)); - return new FdbAnonymousTupleFormatter( + return new AnonymousTupleFormatter( (value) => prefix.Append(value), (tuple) => { - if (tuple.Count != prefix.Count + 1) throw new ArgumentException("Tuple size is invalid", "tuple"); - if (!FdbTuple.StartsWith(tuple, prefix)) throw new ArgumentException("Tuple does not start with the expected prefix", "tuple"); + if (tuple.Count != prefix.Count + 1) throw new ArgumentException("Tuple size is invalid", nameof(tuple)); + if (!STuple.StartsWith(tuple, prefix)) throw new ArgumentException("Tuple does not start with the expected prefix", nameof(tuple)); return tuple.Last(); } ); @@ -81,19 +81,19 @@ private static ITupleFormatter CreateDefaultFormatter() { var type = typeof(T); - if (typeof(IFdbTuple).IsAssignableFrom(type)) + if (typeof(ITuple).IsAssignableFrom(type)) { - return new FdbAnonymousTupleFormatter((x) => (IFdbTuple)x, (x) => (T)x); + return new AnonymousTupleFormatter((x) => (ITuple)x, (x) => (T)x); } if (typeof(ITupleFormattable).IsAssignableFrom(type)) { // note: we cannot call directlty 'new FormattableFormatter()' because of the generic type constraints, so we have to use reflection... // => this WILL fail if someone implements 'ITupleFormattable' on a class that does not have public parameterless constructor ! - return (ITupleFormatter)Activator.CreateInstance(typeof(FdbFormattableTupleFormatter<>).MakeGenericType(type)); + return (ITupleFormatter)Activator.CreateInstance(typeof(FormattableTupleFormatter<>).MakeGenericType(type)); } - return new FdbGenericTupleFormatter(); + return new GenericTupleFormatter(); } } diff --git a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs b/FoundationDB.Client/Layers/Tuples/ITuple.cs similarity index 91% rename from FoundationDB.Client/Layers/Tuples/IFdbTuple.cs rename to FoundationDB.Client/Layers/Tuples/ITuple.cs index 57e103f4b..cfe12bf17 100644 --- a/FoundationDB.Client/Layers/Tuples/IFdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/ITuple.cs @@ -37,7 +37,7 @@ namespace FoundationDB.Layers.Tuples /// Represents a Tuple of N elements [ImmutableObject(true)] [CannotApplyEqualityOperator] - public interface IFdbTuple : IEnumerable, IEquatable, IReadOnlyCollection + public interface ITuple : IEnumerable, IEquatable, IReadOnlyCollection #if !NET_4_0 , IReadOnlyList , System.Collections.IStructuralEquatable @@ -56,8 +56,8 @@ public interface IFdbTuple : IEnumerable, IEquatable, IReadOn // - Accessing the Count and Last item should be fast, if possible in O(1) // - Appending should also be fast, if possible O(1) // - Getting the substring of a tuple should as fast as possible, if possible O(1). For list-based tuples, it should return a view of the list (offset/count) and avoid copying the list - // - If an operation returns an empty tuple, then it should return the FdbTuple.Empty singleton instance - // - If an operation does not change the tuple (like Append(FdbTuple.Empty), or tuple.Substring(0)), then the tuple should return itself + // - If an operation returns an empty tuple, then it should return the STuple.Empty singleton instance + // - If an operation does not change the tuple (like Append(STuple.Empty), or tuple.Substring(0)), then the tuple should return itself // - If the same tuple will be packed frequently, it should be memoized (converted into a FdbMemoizedTuple) #if NET_4_0 @@ -77,7 +77,7 @@ public interface IFdbTuple : IEnumerable, IEquatable, IReadOn /// Starting offset of the sub-tuple to return, or null to select from the start. Negative values means from the end /// Ending offset (excluded) of the sub-tuple to return or null to select until the end. Negative values means from the end. /// Tuple that include all items in the current tuple whose offset are greather than or equal to and strictly less than . The tuple may be smaller than expected if the range is larger than the parent tuple. If the range does not intersect with the tuple, the Empty tuple will be returned. - IFdbTuple this[int? fromIncluded, int? toExcluded] { [NotNull] get; } + ITuple this[int? fromIncluded, int? toExcluded] { [NotNull] get; } /// Return the typed value of an item of the tuple, given its position /// Expected type of the item @@ -102,15 +102,15 @@ public interface IFdbTuple : IEnumerable, IEquatable, IReadOn /// Value that will be appended at the end /// New tuple with the new value /// ("Hello,").Append("World") => ("Hello", "World",) - /// If is an , then it will be appended as a single element. If you need to append the *items* of a tuple, you must call + /// If is an , then it will be appended as a single element. If you need to append the *items* of a tuple, you must call [NotNull] - IFdbTuple Append(T value); + ITuple Append(T value); /// Create a new Tuple by appending the items of another tuple at the end of this tuple /// Tuple whose items must be appended at the end of the current tuple /// New tuple with the new values, or the same instance if is empty. [NotNull] - IFdbTuple Concat([NotNull] IFdbTuple tuple); + ITuple Concat([NotNull] ITuple tuple); /// Copy all items of the tuple into an array at a specific location /// Destination array (must be big enough to contains all the items) diff --git a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs b/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs similarity index 83% rename from FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs rename to FoundationDB.Client/Layers/Tuples/JoinedTuple.cs index a31ca1da6..daed93900 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbJoinedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs @@ -38,15 +38,15 @@ namespace FoundationDB.Layers.Tuples /// Tuple that represents the concatenation of two tuples [DebuggerDisplay("{ToString()}")] - public sealed class FdbJoinedTuple : IFdbTuple + public sealed class JoinedTuple : ITuple { // Uses cases: joining a 'subspace' tuple (customerId, 'Users', ) with a 'key' tuple (userId, 'Contacts', 123, ) /// First tuple (first N items) - public readonly IFdbTuple Head; + public readonly ITuple Head; /// Second tuple (last M items) - public readonly IFdbTuple Tail; + public readonly ITuple Tail; /// Offset at which the Tail tuple starts. Items are in Head tuple if index < split. Items are in Tail tuple if index >= split. private readonly int m_split; @@ -54,7 +54,7 @@ public sealed class FdbJoinedTuple : IFdbTuple /// Total size of the tuple (sum of the size of the two inner tuples) private readonly int m_count; - public FdbJoinedTuple(IFdbTuple head, IFdbTuple tail) + public JoinedTuple(ITuple head, ITuple tail) { if (head == null) throw new ArgumentNullException("head"); if (tail == null) throw new ArgumentNullException("tail"); @@ -80,7 +80,7 @@ public Slice ToSlice() public override string ToString() { - return FdbTuple.ToString(this); + return STuple.ToString(this); } public int Count @@ -92,19 +92,19 @@ public object this[int index] { get { - index = FdbTuple.MapIndex(index, m_count); + index = STuple.MapIndex(index, m_count); return index < m_split ? this.Head[index] : this.Tail[index - m_split]; } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { get { - int begin = fromIncluded.HasValue ? FdbTuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? FdbTuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; - if (end <= begin) return FdbTuple.Empty; + if (end <= begin) return STuple.Empty; int p = this.Head.Count; if (begin >= p) @@ -117,14 +117,14 @@ public object this[int index] } else { // selected items are both in head and tail - return new FdbJoinedTuple(this.Head[begin, null], this.Tail[null, end - p]); + return new JoinedTuple(this.Head[begin, null], this.Tail[null, end - p]); } } } public T Get(int index) { - index = FdbTuple.MapIndex(index, m_count); + index = STuple.MapIndex(index, m_count); return index < m_split ? this.Head.Get(index) : this.Tail.Get(index - m_split); } @@ -136,19 +136,19 @@ public T Last() return this.Head.Last(); } - IFdbTuple IFdbTuple.Append(T value) + ITuple ITuple.Append(T value) { - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } [NotNull] - public FdbLinkedTuple Append(T value) + public LinkedTuple Append(T value) { - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); @@ -159,12 +159,12 @@ public IFdbTuple Concat([NotNull] IFdbTuple tuple) if (n1 + n2 >= 10) { // it's getting bug, merge to a new List tuple - return new FdbListTuple(this.Head, this.Tail, tuple); + return new ListTuple(this.Head, this.Tail, tuple); } else { // REVIEW: should we always concat with the tail? - return new FdbJoinedTuple(this.Head, this.Tail.Concat(tuple)); + return new JoinedTuple(this.Head, this.Tail.Concat(tuple)); } } @@ -196,7 +196,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -211,7 +211,7 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect if (object.ReferenceEquals(this, other)) return true; if (other == null) return false; - var tuple = other as IFdbTuple; + var tuple = other as ITuple; if (!object.ReferenceEquals(tuple, null)) { if (tuple.Count != m_count) return false; @@ -235,7 +235,7 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( this.Head != null ? this.Head.GetHashCode(comparer) : 0, this.Tail != null ? this.Tail.GetHashCode(comparer) : 0 ); diff --git a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs b/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs similarity index 87% rename from FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs rename to FoundationDB.Client/Layers/Tuples/LinkedTuple.cs index 5952260f8..a52bfd45d 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbLinkedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs @@ -39,7 +39,7 @@ namespace FoundationDB.Layers.Tuples /// Tuple that adds a value at the end of an already existing tuple /// Type of the last value of the tuple [DebuggerDisplay("{ToString()}")] - public sealed class FdbLinkedTuple : IFdbTuple + public sealed class LinkedTuple : ITuple { //TODO: consider changing this to a struct ? @@ -50,13 +50,13 @@ public sealed class FdbLinkedTuple : IFdbTuple public readonly T Tail; /// Link to the parent tuple that contains the head. - public readonly IFdbTuple Head; + public readonly ITuple Head; /// Cached size of the size of the Head tuple. Add 1 to get the size of this tuple. public readonly int Depth; /// Append a new value at the end of an existing tuple - internal FdbLinkedTuple(IFdbTuple head, T tail) + internal LinkedTuple(ITuple head, T tail) { Contract.Requires(head != null); @@ -69,7 +69,7 @@ internal FdbLinkedTuple(IFdbTuple head, T tail) public void PackTo(ref TupleWriter writer) { this.Head.PackTo(ref writer); - FdbTuplePacker.SerializeTo(ref writer, this.Tail); + TuplePacker.SerializeTo(ref writer, this.Tail); } /// Pack this tuple into a slice @@ -96,9 +96,9 @@ public object this[int index] } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } public R Get(int index) @@ -113,21 +113,21 @@ public R Last() return FdbConverters.Convert(this.Tail); } - IFdbTuple IFdbTuple.Append(R value) + ITuple ITuple.Append(R value) { return this.Append(value); } [NotNull] - public FdbLinkedTuple Append(R value) + public LinkedTuple Append(R value) { - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } public void CopyTo([NotNull] object[] array, int offset) @@ -152,7 +152,7 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public override string ToString() { - return FdbTuple.ToString(this); + return STuple.ToString(this); } public override bool Equals(object obj) @@ -160,7 +160,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -175,7 +175,7 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect if (object.ReferenceEquals(this, other)) return true; if (other == null) return false; - var linked = other as FdbLinkedTuple; + var linked = other as LinkedTuple; if (!object.ReferenceEquals(linked, null)) { // must have same length @@ -186,12 +186,12 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect return this.Head.Equals(linked.Tail, comparer); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( this.Head != null ? this.Head.GetHashCode(comparer) : 0, comparer.GetHashCode(this.Tail) ); diff --git a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs b/FoundationDB.Client/Layers/Tuples/ListTuple.cs similarity index 83% rename from FoundationDB.Client/Layers/Tuples/FdbListTuple.cs rename to FoundationDB.Client/Layers/Tuples/ListTuple.cs index 114b3f97e..4fa95de86 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbListTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/ListTuple.cs @@ -37,10 +37,10 @@ namespace FoundationDB.Layers.Tuples using FoundationDB.Client.Converters; /// Tuple that can hold any number of untyped items - public sealed class FdbListTuple : IFdbTuple + public sealed class ListTuple : ITuple { // We could use a FdbListTuple for tuples where all items are of type T, and FdbListTuple could derive from FdbListTuple. - // => this could speed up a bit the use case of FdbTuple.FromArray or FdbTuple.FromSequence + // => this could speed up a bit the use case of STuple.FromArray or STuple.FromSequence /// List of the items in the tuple. /// It is supposed to be immutable! @@ -53,7 +53,7 @@ public sealed class FdbListTuple : IFdbTuple private int? m_hashCode; /// Create a new tuple from a sequence of items (copied) - internal FdbListTuple(IEnumerable items) + internal ListTuple(IEnumerable items) { m_items = items.ToArray(); m_count = m_items.Length; @@ -61,7 +61,7 @@ internal FdbListTuple(IEnumerable items) /// Wrap a List of items /// The list should not mutate and should not be exposed to anyone else! - internal FdbListTuple(object[] items, int offset, int count) + internal ListTuple(object[] items, int offset, int count) { Contract.Requires(items != null && offset >= 0 && count >= 0); Contract.Requires(offset + count <= items.Length, "inner item array is too small"); @@ -74,7 +74,7 @@ internal FdbListTuple(object[] items, int offset, int count) /// Create a new list tuple by merging the items of two tuples together /// /// - internal FdbListTuple(IFdbTuple a, IFdbTuple b) + internal ListTuple(ITuple a, ITuple b) { if (a == null) throw new ArgumentNullException("a"); if (b == null) throw new ArgumentNullException("b"); @@ -91,7 +91,7 @@ internal FdbListTuple(IFdbTuple a, IFdbTuple b) } /// Create a new list tuple by merging the items of three tuples together - internal FdbListTuple(IFdbTuple a, IFdbTuple b, IFdbTuple c) + internal ListTuple(ITuple a, ITuple b, ITuple c) { if (a == null) throw new ArgumentNullException("a"); if (b == null) throw new ArgumentNullException("b"); @@ -119,25 +119,25 @@ public object this[int index] { get { - return m_items[m_offset + FdbTuple.MapIndex(index, m_count)]; + return m_items[m_offset + STuple.MapIndex(index, m_count)]; } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { get { - int begin = fromIncluded.HasValue ? FdbTuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? FdbTuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; int len = end - begin; - if (len <= 0) return FdbTuple.Empty; + if (len <= 0) return STuple.Empty; if (begin == 0 && len == m_count) return this; Contract.Assert(m_offset + begin >= m_offset); Contract.Assert(len >= 0 && len <= m_count); - return new FdbListTuple(m_items, m_offset + begin, len); + return new ListTuple(m_items, m_offset + begin, len); } } @@ -152,20 +152,20 @@ public R Last() return FdbConverters.ConvertBoxed(m_items[m_offset + m_count - 1]); } - IFdbTuple IFdbTuple.Append(T value) + ITuple ITuple.Append(T value) { return this.Append(value); } - public FdbListTuple Append(T value) + public ListTuple Append(T value) { var list = new object[m_count + 1]; Array.Copy(m_items, m_offset, list, 0, m_count); list[m_count] = value; - return new FdbListTuple(list, 0, list.Length); + return new ListTuple(list, 0, list.Length); } - public FdbListTuple AppendRange(object[] items) + public ListTuple AppendRange(object[] items) { if (items == null) throw new ArgumentNullException("items"); @@ -174,10 +174,10 @@ public FdbListTuple AppendRange(object[] items) var list = new object[m_count + items.Length]; Array.Copy(m_items, m_offset, list, 0, m_count); Array.Copy(items, 0, list, m_count, items.Length); - return new FdbListTuple(list, 0, list.Length); + return new ListTuple(list, 0, list.Length); } - public FdbListTuple Concat(FdbListTuple tuple) + public ListTuple Concat(ListTuple tuple) { if (tuple == null) throw new ArgumentNullException("tuple"); @@ -187,12 +187,12 @@ public FdbListTuple Concat(FdbListTuple tuple) var list = new object[m_count + tuple.m_count]; Array.Copy(m_items, m_offset, list, 0, m_count); Array.Copy(tuple.m_items, tuple.m_offset, list, m_count, tuple.m_count); - return new FdbListTuple(list, 0, list.Length); + return new ListTuple(list, 0, list.Length); } - public FdbListTuple Concat(IFdbTuple tuple) + public ListTuple Concat(ITuple tuple) { - var _ = tuple as FdbListTuple; + var _ = tuple as ListTuple; if (_ != null) return Concat(_); int count = tuple.Count; @@ -201,10 +201,10 @@ public FdbListTuple Concat(IFdbTuple tuple) var list = new object[m_count + count]; Array.Copy(m_items, m_offset, list, 0, m_count); tuple.CopyTo(list, m_count); - return new FdbListTuple(list, 0, list.Length); + return new ListTuple(list, 0, list.Length); } - IFdbTuple IFdbTuple.Concat(IFdbTuple tuple) + ITuple ITuple.Concat(ITuple tuple) { return this.Concat(tuple); } @@ -240,7 +240,7 @@ public void PackTo(ref TupleWriter writer) { for (int i = 0; i < m_count; i++) { - FdbTuplePackers.SerializeObjectTo(ref writer, m_items[i + m_offset]); + TuplePackers.SerializeObjectTo(ref writer, m_items[i + m_offset]); } } @@ -253,7 +253,7 @@ public Slice ToSlice() public override string ToString() { - return FdbTuple.ToString(m_items, m_offset, m_count); + return STuple.ToString(m_items, m_offset, m_count); } private bool CompareItems(IEnumerable theirs, IEqualityComparer comparer) @@ -281,7 +281,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -296,7 +296,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) if (object.ReferenceEquals(this, other)) return true; if (other == null) return false; - var list = other as FdbListTuple; + var list = other as ListTuple; if (!object.ReferenceEquals(list, null)) { if (list.m_count != m_count) return false; @@ -311,7 +311,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) } } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) @@ -328,7 +328,7 @@ int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer compar { var item = m_items[i + m_offset]; - h = FdbTuple.CombineHashCodes(h, comparer.GetHashCode(item)); + h = STuple.CombineHashCodes(h, comparer.GetHashCode(item)); } if (canUseCache) m_hashCode = h; return h; diff --git a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs similarity index 81% rename from FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs rename to FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs index e3366f7df..32740bfa8 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbMemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs @@ -38,7 +38,7 @@ namespace FoundationDB.Layers.Tuples /// Represents an immutable tuple where the packed bytes are cached [DebuggerDisplay("{ToString()}")] - public sealed class FdbMemoizedTuple : IFdbTuple + public sealed class MemoizedTuple : ITuple { /// Items of the tuple private readonly object[] m_items; @@ -46,7 +46,7 @@ public sealed class FdbMemoizedTuple : IFdbTuple /// Packed version of the tuple private Slice m_packed; //PERF: readonly struct - internal FdbMemoizedTuple(object[] items, Slice packed) + internal MemoizedTuple(object[] items, Slice packed) { Contract.Requires(items != null); Contract.Requires(packed.HasValue); @@ -67,12 +67,12 @@ public int Count public object this[int index] { - get { return m_items[FdbTuple.MapIndex(index, m_items.Length)]; } + get { return m_items[STuple.MapIndex(index, m_items.Length)]; } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } public void PackTo(ref TupleWriter writer) @@ -88,9 +88,9 @@ public Slice ToSlice() return m_packed; } - public FdbMemoizedTuple Copy() + public MemoizedTuple Copy() { - return new FdbMemoizedTuple( + return new MemoizedTuple( (object[])(m_items.Clone()), m_packed.Memoize() ); @@ -115,19 +115,19 @@ public R Last() return FdbConverters.ConvertBoxed(m_items[n - 1]); } - IFdbTuple IFdbTuple.Append(T value) + ITuple ITuple.Append(T value) { return this.Append(value); } - public FdbLinkedTuple Append(T value) + public LinkedTuple Append(T value) { - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } - public IFdbTuple Concat(IFdbTuple tuple) + public ITuple Concat(ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } public void CopyTo(object[] array, int offset) @@ -147,25 +147,25 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public override string ToString() { - return FdbTuple.ToString(m_items, 0, m_items.Length); + return STuple.ToString(m_items, 0, m_items.Length); } public override bool Equals(object obj) { - return Equals(obj as IFdbTuple); + return Equals(obj as ITuple); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { if (object.ReferenceEquals(other, null)) return false; - var memoized = other as FdbMemoizedTuple; + var memoized = other as MemoizedTuple; if (!object.ReferenceEquals(memoized, null)) { return m_packed.Equals(memoized.m_packed); } - return FdbTuple.Equals(this, other, SimilarValueComparer.Default); + return STuple.Equals(this, other, SimilarValueComparer.Default); } public override int GetHashCode() @@ -175,12 +175,12 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) { - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return FdbTuple.StructuralGetHashCode(this, comparer); + return STuple.StructuralGetHashCode(this, comparer); } } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs similarity index 75% rename from FoundationDB.Client/Layers/Tuples/FdbTuple.cs rename to FoundationDB.Client/Layers/Tuples/STuple.cs index 8d383f5a9..a57d499a6 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -41,50 +41,40 @@ namespace FoundationDB.Layers.Tuples /// Factory class for Tuples [PublicAPI] - public static class FdbTuple + public static class STuple { /// Empty tuple /// Not to be mistaken with a 1-tuple containing 'null' ! - public static readonly IFdbTuple Empty = new EmptyTuple(); + public static readonly ITuple Empty = new EmptyTuple(); /// Empty tuple (singleton that is used as a base for other tuples) - private sealed class EmptyTuple : IFdbTuple + private sealed class EmptyTuple : ITuple { - public int Count - { - get { return 0; } - } + public int Count => 0; - object IReadOnlyList.this[int index] - { - get { throw new InvalidOperationException("Tuple is empty"); } - } - - public IFdbTuple this[int? from, int? to] - { - //REVIEW: should we throw if from/to are not null, 0 or -1 ? - get { return this; } - } + object IReadOnlyList.this[int index] => throw new InvalidOperationException("Tuple is empty"); + public ITuple this[int? from, int? to] => this; + //REVIEW: should we throw if from/to are not null, 0 or -1 ? public R Get(int index) { throw new InvalidOperationException("Tuple is empty"); } - R IFdbTuple.Last() + R ITuple.Last() { throw new InvalidOperationException("Tuple is empty"); } - public IFdbTuple Append(T1 value) + public ITuple Append(T1 value) { - return new FdbTuple(value); + return new STuple(value); } - public IFdbTuple Concat(IFdbTuple tuple) + public ITuple Concat(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); if (tuple is EmptyTuple || tuple.Count == 0) return this; return tuple; } @@ -124,19 +114,19 @@ public override int GetHashCode() return 0; } - public bool Equals(IFdbTuple value) + public bool Equals(ITuple value) { return value != null && value.Count == 0; } public override bool Equals(object obj) { - return Equals(obj as IFdbTuple); + return Equals(obj as ITuple); } bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) { - var tuple = other as IFdbTuple; + var tuple = other as ITuple; return tuple != null && tuple.Count == 0; } @@ -150,68 +140,68 @@ int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqua #region Creation /// Create a new 1-tuple, holding only one item - /// This is the non-generic equivalent of FdbTuple.Create<object>() + /// This is the non-generic equivalent of STuple.Create<object>() [NotNull] - public static IFdbTuple CreateBoxed(object item) + public static ITuple CreateBoxed(object item) { - return new FdbTuple(item); + return new STuple(item); } /// Create a new 1-tuple, holding only one item [DebuggerStepThrough] - public static FdbTuple Create(T1 item1) + public static STuple Create(T1 item1) { - return new FdbTuple(item1); + return new STuple(item1); } /// Create a new 2-tuple, holding two items [DebuggerStepThrough] - public static FdbTuple Create(T1 item1, T2 item2) + public static STuple Create(T1 item1, T2 item2) { - return new FdbTuple(item1, item2); + return new STuple(item1, item2); } /// Create a new 3-tuple, holding three items [DebuggerStepThrough] - public static FdbTuple Create(T1 item1, T2 item2, T3 item3) + public static STuple Create(T1 item1, T2 item2, T3 item3) { - return new FdbTuple(item1, item2, item3); + return new STuple(item1, item2, item3); } /// Create a new 4-tuple, holding four items [DebuggerStepThrough] - public static FdbTuple Create(T1 item1, T2 item2, T3 item3, T4 item4) + public static STuple Create(T1 item1, T2 item2, T3 item3, T4 item4) { - return new FdbTuple(item1, item2, item3, item4); + return new STuple(item1, item2, item3, item4); } /// Create a new 5-tuple, holding five items [DebuggerStepThrough] - public static FdbTuple Create(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public static STuple Create(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return new FdbTuple(item1, item2, item3, item4, item5); + return new STuple(item1, item2, item3, item4, item5); } /// Create a new N-tuple, from N items /// Items to wrap in a tuple /// If you already have an array of items, you should call instead. Mutating the array, would also mutate the tuple! [NotNull] - public static IFdbTuple Create([NotNull] params object[] items) + public static ITuple Create([NotNull] params object[] items) { - if (items == null) throw new ArgumentNullException("items"); + if (items == null) throw new ArgumentNullException(nameof(items)); //note: this is a convenience method for people that wants to pass more than 3 args arguments, and not have to call CreateRange(object[]) method - if (items.Length == 0) return FdbTuple.Empty; + if (items.Length == 0) return STuple.Empty; // We don't copy the array, and rely on the fact that the array was created by the compiler and that nobody will get a reference on it. - return new FdbListTuple(items, 0, items.Length); + return new ListTuple(items, 0, items.Length); } /// Create a new N-tuple that wraps an array of untyped items /// If the original array is mutated, the tuple will reflect the changes! [NotNull] - public static IFdbTuple Wrap([NotNull] object[] items) + public static ITuple Wrap([NotNull] object[] items) { //note: this method only exists to differentiate between Create(object[]) and Create() if (items == null) throw new ArgumentException("items"); @@ -221,14 +211,14 @@ public static IFdbTuple Wrap([NotNull] object[] items) /// Create a new N-tuple that wraps a section of an array of untyped items /// If the original array is mutated, the tuple will reflect the changes! [NotNull] - public static IFdbTuple Wrap([NotNull] object[] items, int offset, int count) + public static ITuple Wrap([NotNull] object[] items, int offset, int count) { return FromObjects(items, offset, count, copy: false); } /// Create a new N-tuple by copying the content of an array of untyped items [NotNull] - public static IFdbTuple FromObjects([NotNull] object[] items) + public static ITuple FromObjects([NotNull] object[] items) { //note: this method only exists to differentiate between Create(object[]) and Create() if (items == null) throw new ArgumentException("items"); @@ -237,7 +227,7 @@ public static IFdbTuple FromObjects([NotNull] object[] items) /// Create a new N-tuple by copying a section of an array of untyped items [NotNull] - public static IFdbTuple FromObjects([NotNull] object[] items, int offset, int count) + public static ITuple FromObjects([NotNull] object[] items, int offset, int count) { return FromObjects(items, offset, count, copy: true); } @@ -245,25 +235,25 @@ public static IFdbTuple FromObjects([NotNull] object[] items, int offset, int co /// Create a new N-tuple that wraps a section of an array of untyped items /// If is true, and the original array is mutated, the tuple will reflect the changes! [NotNull] - public static IFdbTuple FromObjects([NotNull] object[] items, int offset, int count, bool copy) + public static ITuple FromObjects([NotNull] object[] items, int offset, int count, bool copy) { - if (items == null) throw new ArgumentNullException("items"); - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - if (offset + count > items.Length) throw new ArgumentOutOfRangeException("count", "Source array is too small"); + if (items == null) throw new ArgumentNullException(nameof(items)); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); + if (offset + count > items.Length) throw new ArgumentOutOfRangeException(nameof(count), "Source array is too small"); - if (count == 0) return FdbTuple.Empty; + if (count == 0) return STuple.Empty; if (copy) { var tmp = new object[count]; Array.Copy(items, offset, tmp, 0, count); - return new FdbListTuple(tmp, 0, count); + return new ListTuple(tmp, 0, count); } else { // can mutate if passed a pre-allocated array: { var foo = new objec[123]; Create(foo); foo[42] = "bad"; } - return new FdbListTuple(items, offset, count); + return new ListTuple(items, offset, count); } } @@ -271,44 +261,44 @@ public static IFdbTuple FromObjects([NotNull] object[] items, int offset, int co /// Array of items /// Tuple with the same size as and where all the items are of type [NotNull] - public static IFdbTuple FromArray([NotNull] T[] items) + public static ITuple FromArray([NotNull] T[] items) { - if (items == null) throw new ArgumentNullException("items"); + if (items == null) throw new ArgumentNullException(nameof(items)); return FromArray(items, 0, items.Length); } /// Create a new tuple, from a section of an array of typed items [NotNull] - public static IFdbTuple FromArray([NotNull] T[] items, int offset, int count) + public static ITuple FromArray([NotNull] T[] items, int offset, int count) { - if (items == null) throw new ArgumentNullException("items"); - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - if (offset + count > items.Length) throw new ArgumentOutOfRangeException("count", "Source array is too small"); + if (items == null) throw new ArgumentNullException(nameof(items)); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); + if (offset + count > items.Length) throw new ArgumentOutOfRangeException(nameof(count), "Source array is too small"); switch(count) { - case 0: return FdbTuple.Empty; - case 1: return FdbTuple.Create(items[offset]); - case 2: return FdbTuple.Create(items[offset], items[offset + 1]); - case 3: return FdbTuple.Create(items[offset], items[offset + 1], items[offset + 2]); - case 4: return FdbTuple.Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3]); + case 0: return STuple.Empty; + case 1: return STuple.Create(items[offset]); + case 2: return STuple.Create(items[offset], items[offset + 1]); + case 3: return STuple.Create(items[offset], items[offset + 1], items[offset + 2]); + case 4: return STuple.Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3]); default: { // copy the items in a temp array //TODO: we would probably benefit from having an FdbListTuple here! var tmp = new object[count]; Array.Copy(items, offset, tmp, 0, count); - return new FdbListTuple(tmp, 0, count); + return new ListTuple(tmp, 0, count); } } } /// Create a new tuple from a sequence of typed items [NotNull] - public static IFdbTuple FromEnumerable([NotNull] IEnumerable items) + public static ITuple FromEnumerable([NotNull] IEnumerable items) { - if (items == null) throw new ArgumentNullException("items"); + if (items == null) throw new ArgumentNullException(nameof(items)); var arr = items as T[]; if (arr != null) @@ -317,7 +307,7 @@ public static IFdbTuple FromEnumerable([NotNull] IEnumerable items) } // may already be a tuple (because it implements IE) - var tuple = items as IFdbTuple; + var tuple = items as ITuple; if (tuple != null) { return tuple; @@ -325,15 +315,15 @@ public static IFdbTuple FromEnumerable([NotNull] IEnumerable items) object[] tmp = items.Cast().ToArray(); //TODO: we would probably benefit from having an FdbListTuple here! - return new FdbListTuple(tmp, 0, tmp.Length); + return new ListTuple(tmp, 0, tmp.Length); } /// Concatenates two tuples together [NotNull] - public static IFdbTuple Concat([NotNull] IFdbTuple head, [NotNull] IFdbTuple tail) + public static ITuple Concat([NotNull] ITuple head, [NotNull] ITuple tail) { - if (head == null) throw new ArgumentNullException("head"); - if (tail == null) throw new ArgumentNullException("tail"); + if (head == null) throw new ArgumentNullException(nameof(head)); + if (tail == null) throw new ArgumentNullException(nameof(tail)); int n1 = head.Count; if (n1 == 0) return tail; @@ -341,7 +331,7 @@ public static IFdbTuple Concat([NotNull] IFdbTuple head, [NotNull] IFdbTuple tai int n2 = tail.Count; if (n2 == 0) return head; - return new FdbJoinedTuple(head, tail); + return new JoinedTuple(head, tail); } #endregion @@ -352,12 +342,12 @@ public static IFdbTuple Concat([NotNull] IFdbTuple head, [NotNull] IFdbTuple tai /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice - public static Slice Pack([NotNull] IFdbTuple tuple) + public static Slice Pack([NotNull] ITuple tuple) { //note: this is redundant with tuple.ToSlice() // => maybe we should remove this method? - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return tuple.ToSlice(); } @@ -366,7 +356,7 @@ public static Slice Pack([NotNull] IFdbTuple tuple) /// Array containing the buffer segment of each packed tuple /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] [NotNull] - public static Slice[] Pack([NotNull] params IFdbTuple[] tuples) + public static Slice[] Pack([NotNull] params ITuple[] tuples) { return Pack(Slice.Nil, tuples); } @@ -376,21 +366,21 @@ public static Slice[] Pack([NotNull] params IFdbTuple[] tuples) /// Array containing the buffer segment of each packed tuple /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] [NotNull] - public static Slice[] Pack([NotNull] IEnumerable tuples) + public static Slice[] Pack([NotNull] IEnumerable tuples) { return Pack(Slice.Nil, tuples); } // With prefix - public static void Pack(ref TupleWriter writer, [CanBeNull] IFdbTuple tuple) + public static void Pack(ref TupleWriter writer, [CanBeNull] ITuple tuple) { if (tuple == null || tuple.Count == 0) return; tuple.PackTo(ref writer); } /// Efficiently concatenate a prefix with the packed representation of a tuple - public static Slice Pack(Slice prefix, [CanBeNull] IFdbTuple tuple) + public static Slice Pack(Slice prefix, [CanBeNull] ITuple tuple) { if (tuple == null || tuple.Count == 0) return prefix; @@ -406,9 +396,9 @@ public static Slice Pack(Slice prefix, [CanBeNull] IFdbTuple tuple) /// Array containing the buffer segment of each packed tuple /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] params IFdbTuple[] tuples) + public static Slice[] Pack(Slice prefix, [NotNull] params ITuple[] tuples) { - if (tuples == null) throw new ArgumentNullException("tuples"); + if (tuples == null) throw new ArgumentNullException(nameof(tuples)); // pre-allocate by supposing that each tuple will take at least 16 bytes var writer = new TupleWriter(tuples.Length * (16 + prefix.Count)); @@ -432,12 +422,12 @@ public static Slice[] Pack(Slice prefix, [NotNull] params IFdbTuple[] tuples) /// Array containing the buffer segment of each packed tuple /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable tuples) + public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable tuples) { - if (tuples == null) throw new ArgumentNullException("tuples"); + if (tuples == null) throw new ArgumentNullException(nameof(tuples)); // use optimized version for arrays - var array = tuples as IFdbTuple[]; + var array = tuples as ITuple[]; if (array != null) return Pack(prefix, array); var next = new List(); @@ -456,10 +446,10 @@ public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable tuples } [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] TElement[] elements, Func transform) + public static Slice[] Pack(Slice prefix, [NotNull] TElement[] elements, Func transform) { - if (elements == null) throw new ArgumentNullException("elements"); - if (transform == null) throw new ArgumentNullException("transform"); + if (elements == null) throw new ArgumentNullException(nameof(elements)); + if (transform == null) throw new ArgumentNullException(nameof(transform)); var next = new List(elements.Length); var writer = new TupleWriter(); @@ -485,10 +475,10 @@ public static Slice[] Pack(Slice prefix, [NotNull] TElement[] elements } [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable elements, Func transform) + public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable elements, Func transform) { - if (elements == null) throw new ArgumentNullException("elements"); - if (transform == null) throw new ArgumentNullException("transform"); + if (elements == null) throw new ArgumentNullException(nameof(elements)); + if (transform == null) throw new ArgumentNullException(nameof(transform)); // use optimized version for arrays var array = elements as TElement[]; @@ -527,7 +517,7 @@ public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable(T1 item1) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item1); return writer.Output.ToSlice(); } @@ -535,8 +525,8 @@ public static Slice EncodeKey(T1 item1) public static Slice EncodeKey(T1 item1, T2 item2) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); return writer.Output.ToSlice(); } @@ -544,9 +534,9 @@ public static Slice EncodeKey(T1 item1, T2 item2) public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); return writer.Output.ToSlice(); } @@ -554,10 +544,10 @@ public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); - FdbTuplePacker.SerializeTo(ref writer, item4); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item4); return writer.Output.ToSlice(); } @@ -565,11 +555,11 @@ public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 i public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); - FdbTuplePacker.SerializeTo(ref writer, item4); - FdbTuplePacker.SerializeTo(ref writer, item5); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item4); + TuplePacker.SerializeTo(ref writer, item5); return writer.Output.ToSlice(); } @@ -577,12 +567,12 @@ public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); - FdbTuplePacker.SerializeTo(ref writer, item4); - FdbTuplePacker.SerializeTo(ref writer, item5); - FdbTuplePacker.SerializeTo(ref writer, item6); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item4); + TuplePacker.SerializeTo(ref writer, item5); + TuplePacker.SerializeTo(ref writer, item6); return writer.Output.ToSlice(); } @@ -590,13 +580,13 @@ public static Slice EncodeKey(T1 item1, T2 item2, T3 ite public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); - FdbTuplePacker.SerializeTo(ref writer, item4); - FdbTuplePacker.SerializeTo(ref writer, item5); - FdbTuplePacker.SerializeTo(ref writer, item6); - FdbTuplePacker.SerializeTo(ref writer, item7); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item4); + TuplePacker.SerializeTo(ref writer, item5); + TuplePacker.SerializeTo(ref writer, item6); + TuplePacker.SerializeTo(ref writer, item7); return writer.Output.ToSlice(); } @@ -604,14 +594,14 @@ public static Slice EncodeKey(T1 item1, T2 item2, T3 public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { var writer = new TupleWriter(); - FdbTuplePacker.SerializeTo(ref writer, item1); - FdbTuplePacker.SerializeTo(ref writer, item2); - FdbTuplePacker.SerializeTo(ref writer, item3); - FdbTuplePacker.SerializeTo(ref writer, item4); - FdbTuplePacker.SerializeTo(ref writer, item5); - FdbTuplePacker.SerializeTo(ref writer, item6); - FdbTuplePacker.SerializeTo(ref writer, item7); - FdbTuplePacker.SerializeTo(ref writer, item8); + TuplePacker.SerializeTo(ref writer, item1); + TuplePacker.SerializeTo(ref writer, item2); + TuplePacker.SerializeTo(ref writer, item3); + TuplePacker.SerializeTo(ref writer, item4); + TuplePacker.SerializeTo(ref writer, item5); + TuplePacker.SerializeTo(ref writer, item6); + TuplePacker.SerializeTo(ref writer, item7); + TuplePacker.SerializeTo(ref writer, item8); return writer.Output.ToSlice(); } @@ -629,8 +619,8 @@ public static Slice[] EncodeKeys([NotNull] IEnumerable keys) [NotNull] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) { - if (prefix == null) throw new ArgumentNullException("prefix"); - if (keys == null) throw new ArgumentNullException("keys"); + if (prefix == null) throw new ArgumentNullException(nameof(prefix)); + if (keys == null) throw new ArgumentNullException(nameof(keys)); // use optimized version for arrays var array = keys as T[]; @@ -638,7 +628,7 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable< var next = new List(); var writer = new TupleWriter(); - var packer = FdbTuplePacker.Encoder; + var packer = TuplePacker.Encoder; //TODO: use multiple buffers if item count is huge ? @@ -666,12 +656,12 @@ public static Slice[] EncodeKeys([NotNull] params T[] keys) [NotNull] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) { - if (keys == null) throw new ArgumentNullException("keys"); + if (keys == null) throw new ArgumentNullException(nameof(keys)); // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize var writer = new TupleWriter(keys.Length * (prefix.Count + 8)); var next = new List(keys.Length); - var packer = FdbTuplePacker.Encoder; + var packer = TuplePacker.Encoder; //TODO: use multiple buffers if item count is huge ? @@ -707,13 +697,13 @@ public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) { - if (elements == null) throw new ArgumentNullException("elements"); - if (selector == null) throw new ArgumentNullException("selector"); + if (elements == null) throw new ArgumentNullException(nameof(elements)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize var writer = new TupleWriter(elements.Length * (prefix.Count + 8)); var next = new List(elements.Length); - var packer = FdbTuplePacker.Encoder; + var packer = TuplePacker.Encoder; //TODO: use multiple buffers if item count is huge ? @@ -733,9 +723,9 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] IFdbTuple prefix, [NotNull] IEnumerable keys) + public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] IEnumerable keys) { - if (prefix == null) throw new ArgumentNullException("prefix"); + if (prefix == null) throw new ArgumentNullException(nameof(prefix)); return EncodePrefixedKeys(prefix.ToSlice(), keys); } @@ -746,9 +736,9 @@ public static Slice[] EncodePrefixedKeys([NotNull] IFdbTuple prefix, [NotNull /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] IFdbTuple prefix, [NotNull] params T[] keys) + public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] params T[] keys) { - if (prefix == null) throw new ArgumentNullException("prefix"); + if (prefix == null) throw new ArgumentNullException(nameof(prefix)); return EncodePrefixedKeys(prefix.ToSlice(), keys); } @@ -762,23 +752,23 @@ public static Slice[] EncodePrefixedKeys([NotNull] IFdbTuple prefix, [NotNull /// Unpacked tuple, or the empty tuple if the key is /// If is equal to [NotNull] - public static IFdbTuple Unpack(Slice packedKey) + public static ITuple Unpack(Slice packedKey) { - if (packedKey.IsNull) throw new ArgumentNullException("packedKey"); - if (packedKey.Count == 0) return FdbTuple.Empty; + if (packedKey.IsNull) throw new ArgumentNullException(nameof(packedKey)); + if (packedKey.Count == 0) return STuple.Empty; - return FdbTuplePackers.Unpack(packedKey, false); + return TuplePackers.Unpack(packedKey, false); } /// Unpack a tuple from a binary representation /// Binary key containing a previously packed tuple, or Slice.Nil /// Unpacked tuple, the empty tuple if is equal to , or null if the key is [CanBeNull] - public static IFdbTuple UnpackOrDefault(Slice packedKey) + public static ITuple UnpackOrDefault(Slice packedKey) { if (packedKey.IsNull) return null; - if (packedKey.Count == 0) return FdbTuple.Empty; - return FdbTuplePackers.Unpack(packedKey, false); + if (packedKey.Count == 0) return STuple.Empty; + return TuplePackers.Unpack(packedKey, false); } /// Unpack a tuple and only return its first element @@ -789,10 +779,10 @@ public static T DecodeFirst(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the first element of an empty tuple"); - var slice = FdbTuplePackers.UnpackFirst(packedKey); + var slice = TuplePackers.UnpackFirst(packedKey); if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); - return FdbTuplePacker.Deserialize(slice); + return TuplePacker.Deserialize(slice); } /// Unpack a tuple and only return its last element @@ -803,10 +793,10 @@ public static T DecodeLast(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the last element of an empty tuple"); - var slice = FdbTuplePackers.UnpackLast(packedKey); + var slice = TuplePackers.UnpackLast(packedKey); if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); - return FdbTuplePacker.Deserialize(slice); + return TuplePacker.Deserialize(slice); } /// Unpack the value of a singleton tuple @@ -817,16 +807,16 @@ public static T DecodeKey(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack a single value out of an empty tuple"); - var slice = FdbTuplePackers.UnpackSingle(packedKey); + var slice = TuplePackers.UnpackSingle(packedKey); if (slice.IsNull) throw new InvalidOperationException("Failed to unpack singleton tuple"); - return FdbTuplePacker.Deserialize(slice); + return TuplePacker.Deserialize(slice); } /// Unpack a key containing two elements /// Slice that should contain the packed representation of a tuple with two elements /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static FdbTuple DecodeKey(Slice packedKey) + public static STuple DecodeKey(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); @@ -846,7 +836,7 @@ public static FdbTuple DecodeKey(Slice packedKey) /// Unpack a key containing three elements /// Slice that should contain the packed representation of a tuple with three elements /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static FdbTuple DecodeKey(Slice packedKey) + public static STuple DecodeKey(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); @@ -869,7 +859,7 @@ public static FdbTuple DecodeKey(Slice packedKey) /// Unpack a key containing four elements /// Slice that should contain the packed representation of a tuple with four elements /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static FdbTuple DecodeKey(Slice packedKey) + public static STuple DecodeKey(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); @@ -895,7 +885,7 @@ public static FdbTuple DecodeKey(Slice packedKey /// Unpack a key containing five elements /// Slice that should contain the packed representation of a tuple with five elements /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static FdbTuple DecodeKey(Slice packedKey) + public static STuple DecodeKey(Slice packedKey) { if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); @@ -934,8 +924,8 @@ public static bool DecodeNext(ref TupleReader input, out T value) return false; } - var slice = FdbTupleParser.ParseNext(ref input); - value = FdbTuplePacker.Deserialize(slice); + var slice = TupleParser.ParseNext(ref input); + value = TuplePacker.Deserialize(slice); return true; } @@ -950,7 +940,7 @@ public static Slice EncodePrefixedKey(Slice prefix, T value) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value); + TuplePacker.Encoder(ref writer, value); return writer.Output.ToSlice(); } @@ -959,8 +949,8 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2 { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); return writer.Output.ToSlice(); } @@ -969,9 +959,9 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 va { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); return writer.Output.ToSlice(); } @@ -980,10 +970,10 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); - FdbTuplePacker.Encoder(ref writer, value4); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value4); return writer.Output.ToSlice(); } @@ -992,11 +982,11 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); - FdbTuplePacker.Encoder(ref writer, value4); - FdbTuplePacker.Encoder(ref writer, value5); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value4); + TuplePacker.Encoder(ref writer, value5); return writer.Output.ToSlice(); } @@ -1005,12 +995,12 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 v { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); - FdbTuplePacker.Encoder(ref writer, value4); - FdbTuplePacker.Encoder(ref writer, value5); - FdbTuplePacker.Encoder(ref writer, value6); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value4); + TuplePacker.Encoder(ref writer, value5); + TuplePacker.Encoder(ref writer, value6); return writer.Output.ToSlice(); } @@ -1019,13 +1009,13 @@ public static Slice EncodePrefixedKey(Slice prefix, { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); - FdbTuplePacker.Encoder(ref writer, value4); - FdbTuplePacker.Encoder(ref writer, value5); - FdbTuplePacker.Encoder(ref writer, value6); - FdbTuplePacker.Encoder(ref writer, value7); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value4); + TuplePacker.Encoder(ref writer, value5); + TuplePacker.Encoder(ref writer, value6); + TuplePacker.Encoder(ref writer, value7); return writer.Output.ToSlice(); } @@ -1034,14 +1024,14 @@ public static Slice EncodePrefixedKey(Slice pref { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - FdbTuplePacker.Encoder(ref writer, value1); - FdbTuplePacker.Encoder(ref writer, value2); - FdbTuplePacker.Encoder(ref writer, value3); - FdbTuplePacker.Encoder(ref writer, value4); - FdbTuplePacker.Encoder(ref writer, value5); - FdbTuplePacker.Encoder(ref writer, value6); - FdbTuplePacker.Encoder(ref writer, value7); - FdbTuplePacker.Encoder(ref writer, value8); + TuplePacker.Encoder(ref writer, value1); + TuplePacker.Encoder(ref writer, value2); + TuplePacker.Encoder(ref writer, value3); + TuplePacker.Encoder(ref writer, value4); + TuplePacker.Encoder(ref writer, value5); + TuplePacker.Encoder(ref writer, value6); + TuplePacker.Encoder(ref writer, value7); + TuplePacker.Encoder(ref writer, value8); return writer.Output.ToSlice(); } @@ -1054,7 +1044,7 @@ public static Slice EncodePrefixedKey(Slice pref /// Right tuple /// True if the tuples are considered equal; otherwise, false. If both and are null, the methods returns true; /// This method is equivalent of calling left.Equals(right), - public static bool Equals(IFdbTuple left, IFdbTuple right) + public static bool Equals(ITuple left, ITuple right) { if (object.ReferenceEquals(left, null)) return object.ReferenceEquals(right, null); return left.Equals(right); @@ -1064,19 +1054,19 @@ public static bool Equals(IFdbTuple left, IFdbTuple right) /// Left tuple /// Right tuple /// True if the tuples are considered similar; otherwise, false. If both and are null, the methods returns true; - public static bool Equivalent(IFdbTuple left, IFdbTuple right) + public static bool Equivalent(ITuple left, ITuple right) { if (object.ReferenceEquals(left, null)) return object.ReferenceEquals(right, null); - return !object.ReferenceEquals(right, null) && Equals(left, right, FdbTupleComparisons.Default); + return !object.ReferenceEquals(right, null) && Equals(left, right, TupleComparisons.Default); } /// Create a range that selects all tuples that are stored under the specified subspace: 'prefix\x00' <= k < 'prefix\xFF' /// Subspace binary prefix (that will be excluded from the range) /// Range including all possible tuples starting with the specified prefix. - /// FdbTuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) + /// STuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) public static KeyRange ToRange(Slice prefix) { - if (prefix.IsNull) throw new ArgumentNullException("prefix"); + if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); //note: there is no guarantee that prefix is a valid packed tuple (could be any exotic binary prefix) @@ -1088,10 +1078,10 @@ public static KeyRange ToRange(Slice prefix) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. - public static KeyRange ToRange([NotNull] IFdbTuple tuple) + /// STuple.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + public static KeyRange ToRange([NotNull] ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = tuple.ToSlice(); @@ -1103,11 +1093,11 @@ public static KeyRange ToRange([NotNull] IFdbTuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(Slice.FromInt32(42), FdbTuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. + /// STuple.ToRange(Slice.FromInt32(42), STuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. - public static KeyRange ToRange(Slice prefix, [NotNull] IFdbTuple tuple) + public static KeyRange ToRange(Slice prefix, [NotNull] ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = prefix + tuple.ToSlice(); @@ -1178,7 +1168,7 @@ internal static string Stringify(object item) /// Start offset of the items to convert /// Number of items to convert /// String representation of the tuple in the form "(item1, item2, ... itemN,)" - /// ToString(FdbTuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\",) + /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\",) [NotNull] internal static string ToString(object[] items, int offset, int count) { @@ -1208,7 +1198,7 @@ internal static string ToString(object[] items, int offset, int count) /// Converts a sequence of object into a displaying string, for loggin/debugging purpose /// Sequence of items to stringfy /// String representation of the tuple in the form "(item1, item2, ... itemN,)" - /// ToString(FdbTuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\") + /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\") [NotNull] internal static string ToString(IEnumerable items) { @@ -1233,29 +1223,29 @@ internal static string ToString(IEnumerable items) } } - /// Default (non-optimized) implementation of IFdbTuple.this[long?, long?] + /// Default (non-optimized) implementation of ITuple.this[long?, long?] /// Tuple to slice /// Start offset of the section (included) /// End offset of the section (included) /// New tuple only containing items inside this section [NotNull] - internal static IFdbTuple Splice([NotNull] IFdbTuple tuple, int? fromIncluded, int? toExcluded) + internal static ITuple Splice([NotNull] ITuple tuple, int? fromIncluded, int? toExcluded) { Contract.Requires(tuple != null); int count = tuple.Count; - if (count == 0) return FdbTuple.Empty; + if (count == 0) return STuple.Empty; int start = fromIncluded.HasValue ? MapIndexBounded(fromIncluded.Value, count) : 0; int end = toExcluded.HasValue ? MapIndexBounded(toExcluded.Value, count) : count; int len = end - start; - if (len <= 0) return FdbTuple.Empty; + if (len <= 0) return STuple.Empty; if (start == 0 && len == count) return tuple; switch(len) { - case 1: return new FdbListTuple(new object[] { tuple[start] }, 0, 1); - case 2: return new FdbListTuple(new object[] { tuple[start], tuple[start + 1] }, 0, 2); + case 1: return new ListTuple(new object[] { tuple[start] }, 0, 1); + case 2: return new ListTuple(new object[] { tuple[start], tuple[start + 1] }, 0, 2); default: { var items = new object[len]; @@ -1265,16 +1255,16 @@ internal static IFdbTuple Splice([NotNull] IFdbTuple tuple, int? fromIncluded, i { items[p] = tuple[q++]; } - return new FdbListTuple(items, 0, len); + return new ListTuple(items, 0, len); } } } - /// Default (non-optimized) implementation for IFdbTuple.StartsWith() + /// Default (non-optimized) implementation for ITuple.StartsWith() /// Larger tuple /// Smaller tuple /// True if starts with (or is equal to) - internal static bool StartsWith([NotNull] IFdbTuple a, [NotNull] IFdbTuple b) + internal static bool StartsWith([NotNull] ITuple a, [NotNull] ITuple b) { Contract.Requires(a != null && b != null); if (object.ReferenceEquals(a, b)) return true; @@ -1291,11 +1281,11 @@ internal static bool StartsWith([NotNull] IFdbTuple a, [NotNull] IFdbTuple b) return true; } - /// Default (non-optimized) implementation for IFdbTuple.EndsWith() + /// Default (non-optimized) implementation for ITuple.EndsWith() /// Larger tuple /// Smaller tuple /// True if starts with (or is equal to) - internal static bool EndsWith([NotNull] IFdbTuple a, [NotNull] IFdbTuple b) + internal static bool EndsWith([NotNull] ITuple a, [NotNull] ITuple b) { Contract.Requires(a != null && b != null); if (object.ReferenceEquals(a, b)) return true; @@ -1315,7 +1305,7 @@ internal static bool EndsWith([NotNull] IFdbTuple a, [NotNull] IFdbTuple b) /// Helper to copy the content of a tuple at a specific position in an array /// Updated offset just after the last element of the copied tuple - internal static int CopyTo([NotNull] IFdbTuple tuple, [NotNull] object[] array, int offset) + internal static int CopyTo([NotNull] ITuple tuple, [NotNull] object[] array, int offset) { Contract.Requires(tuple != null && array != null && offset >= 0); @@ -1376,12 +1366,12 @@ internal static int CombineHashCodes(int h1, int h2, int h3, int h4, int h5) return CombineHashCodes(CombineHashCodes(h1, h2, h3), CombineHashCodes(h4, h5)); } - internal static bool Equals(IFdbTuple left, object other, [NotNull] IEqualityComparer comparer) + internal static bool Equals(ITuple left, object other, [NotNull] IEqualityComparer comparer) { - return object.ReferenceEquals(left, null) ? other == null : FdbTuple.Equals(left, other as IFdbTuple, comparer); + return object.ReferenceEquals(left, null) ? other == null : STuple.Equals(left, other as ITuple, comparer); } - internal static bool Equals(IFdbTuple x, IFdbTuple y, [NotNull] IEqualityComparer comparer) + internal static bool Equals(ITuple x, ITuple y, [NotNull] IEqualityComparer comparer) { if (object.ReferenceEquals(x, y)) return true; if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; @@ -1389,7 +1379,7 @@ internal static bool Equals(IFdbTuple x, IFdbTuple y, [NotNull] IEqualityCompare return x.Count == y.Count && DeepEquals(x, y, comparer); } - internal static bool DeepEquals([NotNull] IFdbTuple x, [NotNull] IFdbTuple y, [NotNull] IEqualityComparer comparer) + internal static bool DeepEquals([NotNull] ITuple x, [NotNull] ITuple y, [NotNull] IEqualityComparer comparer) { Contract.Requires(x != null && y != null && comparer != null); @@ -1407,7 +1397,7 @@ internal static bool DeepEquals([NotNull] IFdbTuple x, [NotNull] IFdbTuple y, [N } } - internal static int StructuralGetHashCode(IFdbTuple tuple, [NotNull] IEqualityComparer comparer) + internal static int StructuralGetHashCode(ITuple tuple, [NotNull] IEqualityComparer comparer) { Contract.Requires(comparer != null); @@ -1424,7 +1414,7 @@ internal static int StructuralGetHashCode(IFdbTuple tuple, [NotNull] IEqualityCo return h; } - internal static int StructuralCompare(IFdbTuple x, IFdbTuple y, [NotNull] IComparer comparer) + internal static int StructuralCompare(ITuple x, ITuple y, [NotNull] IComparer comparer) { Contract.Requires(comparer != null); diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs similarity index 79% rename from FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs rename to FoundationDB.Client/Layers/Tuples/STuple`1.cs index d00cbb7ad..6b1e57759 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -28,28 +28,28 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that holds only one item /// Type of the item - [ImmutableObject(true), DebuggerDisplay("{ToString()}")] - public struct FdbTuple : IFdbTuple + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple { // This is mostly used by code that create a lot of temporary singleton, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an FdbTuple as an IFdbTuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First and only item in the tuple public readonly T1 Item1; [DebuggerStepThrough] - public FdbTuple(T1 item1) + public STuple(T1 item1) { this.Item1 = item1; } @@ -60,14 +60,14 @@ public object this[int index] { get { - if (index > 0 || index < -1) FdbTuple.FailIndexOutOfRange(index, 1); + if (index > 0 || index < -1) STuple.FailIndexOutOfRange(index, 1); return this.Item1; } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position @@ -76,24 +76,24 @@ public object this[int index] /// Value of the item at position , adapted into type . public R Get(int index) { - if (index > 0 || index < -1) FdbTuple.FailIndexOutOfRange(index, 1); + if (index > 0 || index < -1) STuple.FailIndexOutOfRange(index, 1); return FdbConverters.Convert(this.Item1); } /// Return the typed value of the last item in the tuple - R IFdbTuple.Last() + R ITuple.Last() { return FdbConverters.Convert(this.Item1); } public void PackTo(ref TupleWriter writer) { - FdbTuplePacker.Encoder(ref writer, this.Item1); + TuplePacker.Encoder(ref writer, this.Item1); } - IFdbTuple IFdbTuple.Append(T2 value) + ITuple ITuple.Append(T2 value) { - return new FdbTuple(this.Item1, value); + return new STuple(this.Item1, value); } /// Appends a tuple as a single new item at the end of the current tuple. @@ -101,18 +101,18 @@ IFdbTuple IFdbTuple.Append(T2 value) /// New tuple with one extra item /// If you want to append the *items* of , and not the tuple itself, please call ! [NotNull] - public FdbTuple Append(T2 value) + public STuple Append(T2 value) { - return new FdbTuple(this.Item1, value); + return new STuple(this.Item1, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } /// Copy the item of this singleton into an array at the specified offset @@ -148,13 +148,13 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public Slice ToSlice() { - return FdbTuple.EncodeKey(this.Item1); + return STuple.EncodeKey(this.Item1); } public override string ToString() { // singleton tuples end with a trailing ',' - return "(" + FdbTuple.Stringify(this.Item1) + ",)"; + return "(" + STuple.Stringify(this.Item1) + ",)"; } public override bool Equals(object obj) @@ -162,7 +162,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -172,12 +172,12 @@ public override int GetHashCode() return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } - public static bool operator ==(FdbTuple left, FdbTuple right) + public static bool operator ==(STuple left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1); } - public static bool operator !=(FdbTuple left, FdbTuple right) + public static bool operator !=(STuple left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1); } @@ -185,11 +185,11 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is FdbTuple) + if (other is STuple) { - return comparer.Equals(this.Item1, ((FdbTuple)other).Item1); + return comparer.Equals(this.Item1, ((STuple)other).Item1); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) @@ -197,13 +197,13 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) return comparer.GetHashCode(this.Item1); } - public static implicit operator FdbTuple(Tuple t) + public static implicit operator STuple(Tuple t) { if (t == null) throw new ArgumentNullException("t"); - return new FdbTuple(t.Item1); + return new STuple(t.Item1); } - public static explicit operator Tuple(FdbTuple t) + public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs similarity index 78% rename from FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs rename to FoundationDB.Client/Layers/Tuples/STuple`2.cs index 75b91cca3..3cf96db23 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -28,23 +28,23 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that holds a pair of items /// Type of the first item /// Type of the second item - [ImmutableObject(true), DebuggerDisplay("{ToString()}")] - public struct FdbTuple : IFdbTuple + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple { // This is mostly used by code that create a lot of temporary pair, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an FdbTuple as an IFdbTuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the pair public readonly T1 Item1; @@ -52,7 +52,7 @@ public struct FdbTuple : IFdbTuple public readonly T2 Item2; [DebuggerStepThrough] - public FdbTuple(T1 item1, T2 item2) + public STuple(T1 item1, T2 item2) { this.Item1 = item1; this.Item2 = item2; @@ -68,14 +68,14 @@ public object this[int index] { case 0: case -2: return this.Item1; case 1: case -1: return this.Item2; - default: FdbTuple.FailIndexOutOfRange(index, 2); return null; + default: STuple.FailIndexOutOfRange(index, 2); return null; } } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position @@ -88,7 +88,7 @@ public R Get(int index) { case 0: case -2: return FdbConverters.Convert(this.Item1); case 1: case -1: return FdbConverters.Convert(this.Item2); - default: FdbTuple.FailIndexOutOfRange(index, 2); return default(R); + default: STuple.FailIndexOutOfRange(index, 2); return default(R); } } @@ -99,42 +99,40 @@ public T2 Last } /// Return the typed value of the last item in the tuple - R IFdbTuple.Last() + R ITuple.Last() { return FdbConverters.Convert(this.Item2); } public void PackTo(ref TupleWriter writer) { - FdbTuplePacker.Encoder(ref writer, this.Item1); - FdbTuplePacker.Encoder(ref writer, this.Item2); + TuplePacker.Encoder(ref writer, this.Item1); + TuplePacker.Encoder(ref writer, this.Item2); } - IFdbTuple IFdbTuple.Append(T3 value) + ITuple ITuple.Append(T3 value) { - return new FdbTuple(this.Item1, this.Item2, value); + return new STuple(this.Item1, this.Item2, value); } /// Appends a single new item at the end of the current tuple. /// Value that will be added as an embedded item /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! - [NotNull] - public FdbTuple Append(T3 value) + public STuple Append(T3 value) { - return new FdbTuple(this.Item1, this.Item2, value); - // Note: By create a FdbTuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). - // ex: if we have N possible types, then there could be N^3 possible variants of FdbTuple that the JIT has to deal with. + return new STuple(this.Item1, this.Item2, value); + // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). + // ex: if we have N possible types, then there could be N^3 possible variants of STuple that the JIT has to deal with. // => if this starts becoming a problem, then we should return a list tuple ! } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } /// Copy both items of this pair into an array at the specified offset @@ -172,12 +170,12 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public Slice ToSlice() { - return FdbTuple.EncodeKey(this.Item1, this.Item2); + return STuple.EncodeKey(this.Item1, this.Item2); } public override string ToString() { - return "(" + FdbTuple.Stringify(this.Item1) + ", " + FdbTuple.Stringify(this.Item2) + ")"; + return "(" + STuple.Stringify(this.Item1) + ", " + STuple.Stringify(this.Item2) + ")"; } public override bool Equals(object obj) @@ -185,7 +183,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -195,13 +193,13 @@ public override int GetHashCode() return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } - public static bool operator ==(FdbTuple left, FdbTuple right) + public static bool operator ==(STuple left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } - public static bool operator !=(FdbTuple left, FdbTuple right) + public static bool operator !=(STuple left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); @@ -210,30 +208,30 @@ public override int GetHashCode() bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) { if (other == null) return false; - if (other is FdbTuple) + if (other is STuple) { - var tuple = (FdbTuple)other; + var tuple = (STuple)other; return comparer.Equals(this.Item1, tuple.Item1) && comparer.Equals(this.Item2, tuple.Item2); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2) ); } - public static implicit operator FdbTuple(Tuple t) + public static implicit operator STuple(Tuple t) { if (t == null) throw new ArgumentNullException("t"); - return new FdbTuple(t.Item1, t.Item2); + return new STuple(t.Item1, t.Item2); } - public static explicit operator Tuple(FdbTuple t) + public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs similarity index 74% rename from FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs rename to FoundationDB.Client/Layers/Tuples/STuple`3.cs index 4262c2666..8966d7fd8 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -28,25 +28,25 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; using System.Text; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that can hold three items /// Type of the first item /// Type of the second item /// Type of the third item - [ImmutableObject(true), DebuggerDisplay("{ToString()}")] - public struct FdbTuple : IFdbTuple + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple { // This is mostly used by code that create a lot of temporary triplet, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an FdbTuple as an IFdbTuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the triplet public readonly T1 Item1; @@ -56,7 +56,7 @@ public struct FdbTuple : IFdbTuple public readonly T3 Item3; [DebuggerStepThrough] - public FdbTuple(T1 item1, T2 item2, T3 item3) + public STuple(T1 item1, T2 item2, T3 item3) { this.Item1 = item1; this.Item2 = item2; @@ -74,14 +74,14 @@ public object this[int index] case 0: case -3: return this.Item1; case 1: case -2: return this.Item2; case 2: case -1: return this.Item3; - default: FdbTuple.FailIndexOutOfRange(index, 3); return null; + default: STuple.FailIndexOutOfRange(index, 3); return null; } } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position @@ -95,7 +95,7 @@ public R Get(int index) case 0: case -3: return FdbConverters.Convert(this.Item1); case 1: case -2: return FdbConverters.Convert(this.Item2); case 2: case -1: return FdbConverters.Convert(this.Item3); - default: FdbTuple.FailIndexOutOfRange(index, 3); return default(R); + default: STuple.FailIndexOutOfRange(index, 3); return default(R); } } @@ -106,22 +106,22 @@ public T3 Last } /// Return the typed value of the last item in the tuple - R IFdbTuple.Last() + R ITuple.Last() { return FdbConverters.Convert(this.Item3); } public void PackTo(ref TupleWriter writer) { - FdbTuplePacker.Encoder(ref writer, this.Item1); - FdbTuplePacker.Encoder(ref writer, this.Item2); - FdbTuplePacker.Encoder(ref writer, this.Item3); + TuplePacker.Encoder(ref writer, this.Item1); + TuplePacker.Encoder(ref writer, this.Item2); + TuplePacker.Encoder(ref writer, this.Item3); } - IFdbTuple IFdbTuple.Append(T4 value) + ITuple ITuple.Append(T4 value) { // here, the caller doesn't care about the exact tuple type, so we simply return a boxed List Tuple. - return new FdbListTuple(new object[4] { this.Item1, this.Item2, this.Item3, value }, 0, 4); + return new ListTuple(new object[4] { this.Item1, this.Item2, this.Item3, value }, 0, 4); } /// Appends a single new item at the end of the current tuple. @@ -129,31 +129,31 @@ IFdbTuple IFdbTuple.Append(T4 value) /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! [NotNull] - public FdbTuple Append(T4 value) + public STuple Append(T4 value) { - // Here, the caller was explicitly using the FdbTuple struct so probably care about memory footprint, so we keep returning a struct - return new FdbTuple(this.Item1, this.Item2, this.Item3, value); + // Here, the caller was explicitly using the STuple struct so probably care about memory footprint, so we keep returning a struct + return new STuple(this.Item1, this.Item2, this.Item3, value); - // Note: By create a FdbTuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). - // ex: if we have N possible types, then there could be N^4 possible variants of FdbTuple that the JIT has to deal with. + // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). + // ex: if we have N possible types, then there could be N^4 possible variants of STuple that the JIT has to deal with. // => if this starts becoming a problem, then we should return a list tuple ! } /// Copy all the items of this tuple into an array at the specified offset [NotNull] - public FdbTuple Append(IFdbTuple value) + public STuple Append(ITuple value) { - //note: this override exists to prevent the explosion of tuple types such as FdbTuple, FdbTuple, FdbTuple> ! - return new FdbTuple(this.Item1, this.Item2, this.Item3, value); + //note: this override exists to prevent the explosion of tuple types such as STuple, STuple, STuple> ! + return new STuple(this.Item1, this.Item2, this.Item3, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } public void CopyTo(object[] array, int offset) @@ -192,15 +192,15 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public Slice ToSlice() { - return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3); + return STuple.EncodeKey(this.Item1, this.Item2, this.Item3); } public override string ToString() { return new StringBuilder(32).Append('(') - .Append(FdbTuple.Stringify(this.Item1)).Append(", ") - .Append(FdbTuple.Stringify(this.Item2)).Append(", ") - .Append(FdbTuple.Stringify(this.Item3)).Append(')') + .Append(STuple.Stringify(this.Item1)).Append(", ") + .Append(STuple.Stringify(this.Item2)).Append(", ") + .Append(STuple.Stringify(this.Item3)).Append(')') .ToString(); } @@ -209,7 +209,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -219,7 +219,7 @@ public override int GetHashCode() return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } - public static bool operator ==(FdbTuple left, FdbTuple right) + public static bool operator ==(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return comparer.Equals(left.Item1, right.Item1) @@ -227,7 +227,7 @@ public override int GetHashCode() && comparer.Equals(left.Item3, right.Item3); } - public static bool operator !=(FdbTuple left, FdbTuple right) + public static bool operator !=(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return !comparer.Equals(left.Item1, right.Item1) @@ -238,32 +238,32 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is FdbTuple) + if (other is STuple) { - var tuple = (FdbTuple)other; + var tuple = (STuple)other; return comparer.Equals(this.Item1, tuple.Item1) && comparer.Equals(this.Item2, tuple.Item2) && comparer.Equals(this.Item3, tuple.Item3); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3) ); } - public static implicit operator FdbTuple(Tuple t) + public static implicit operator STuple(Tuple t) { if (t == null) throw new ArgumentNullException("t"); - return new FdbTuple(t.Item1, t.Item2, t.Item3); + return new STuple(t.Item1, t.Item2, t.Item3); } - public static explicit operator Tuple(FdbTuple t) + public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs similarity index 80% rename from FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs rename to FoundationDB.Client/Layers/Tuples/STuple`4.cs index 1adc807e6..a6bc908a3 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -28,26 +28,26 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; using System.Text; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that can hold four items /// Type of the first item /// Type of the second item /// Type of the third item /// Type of the fourth item - [ImmutableObject(true), DebuggerDisplay("{ToString()}")] - public struct FdbTuple : IFdbTuple + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an FdbTuple as an IFdbTuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the quartet public readonly T1 Item1; @@ -60,7 +60,7 @@ public struct FdbTuple : IFdbTuple /// Create a tuple containing for items [DebuggerStepThrough] - public FdbTuple(T1 item1, T2 item2, T3 item3, T4 item4) + public STuple(T1 item1, T2 item2, T3 item3, T4 item4) { this.Item1 = item1; this.Item2 = item2; @@ -82,14 +82,14 @@ public object this[int index] case 1: case -3: return this.Item2; case 2: case -2: return this.Item3; case 3: case -1: return this.Item4; - default: FdbTuple.FailIndexOutOfRange(index, 4); return null; + default: STuple.FailIndexOutOfRange(index, 4); return null; } } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position @@ -104,7 +104,7 @@ public R Get(int index) case 1: case -3: return FdbConverters.Convert(this.Item2); case 2: case -2: return FdbConverters.Convert(this.Item3); case 3: case -1: return FdbConverters.Convert(this.Item4); - default: FdbTuple.FailIndexOutOfRange(index, 4); return default(R); + default: STuple.FailIndexOutOfRange(index, 4); return default(R); } } @@ -115,23 +115,23 @@ public T4 Last } /// Return the typed value of the last item in the tuple - R IFdbTuple.Last() + R ITuple.Last() { return FdbConverters.Convert(this.Item4); } public void PackTo(ref TupleWriter writer) { - FdbTuplePacker.Encoder(ref writer, this.Item1); - FdbTuplePacker.Encoder(ref writer, this.Item2); - FdbTuplePacker.Encoder(ref writer, this.Item3); - FdbTuplePacker.Encoder(ref writer, this.Item4); + TuplePacker.Encoder(ref writer, this.Item1); + TuplePacker.Encoder(ref writer, this.Item2); + TuplePacker.Encoder(ref writer, this.Item3); + TuplePacker.Encoder(ref writer, this.Item4); } - IFdbTuple IFdbTuple.Append(T5 value) + ITuple ITuple.Append(T5 value) { // the caller doesn't care about the return type, so just box everything into a list tuple - return new FdbListTuple(new object[5] { this.Item1, this.Item2, this.Item3, this.Item4, value }, 0, 5); + return new ListTuple(new object[5] { this.Item1, this.Item2, this.Item3, this.Item4, value }, 0, 5); } /// Appends a single new item at the end of the current tuple. @@ -139,19 +139,19 @@ IFdbTuple IFdbTuple.Append(T5 value) /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! [NotNull] - public FdbLinkedTuple Append(T5 value) + public LinkedTuple Append(T5 value) { // the caller probably cares about the return type, since it is using a struct, but whatever tuple type we use will end up boxing this tuple on the heap, and we will loose type information. // but, by returning a FdbLinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - public IFdbTuple Concat(IFdbTuple tuple) + public ITuple Concat(ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } /// Copy all the items of this tuple into an array at the specified offset @@ -193,16 +193,16 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public Slice ToSlice() { - return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4); + return STuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4); } public override string ToString() { return new StringBuilder(48).Append('(') - .Append(FdbTuple.Stringify(this.Item1)).Append(", ") - .Append(FdbTuple.Stringify(this.Item2)).Append(", ") - .Append(FdbTuple.Stringify(this.Item3)).Append(", ") - .Append(FdbTuple.Stringify(this.Item4)).Append(')') + .Append(STuple.Stringify(this.Item1)).Append(", ") + .Append(STuple.Stringify(this.Item2)).Append(", ") + .Append(STuple.Stringify(this.Item3)).Append(", ") + .Append(STuple.Stringify(this.Item4)).Append(')') .ToString(); } @@ -211,7 +211,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -221,7 +221,7 @@ public override int GetHashCode() return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } - public static bool operator ==(FdbTuple left, FdbTuple right) + public static bool operator ==(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return comparer.Equals(left.Item1, right.Item1) @@ -230,7 +230,7 @@ public override int GetHashCode() && comparer.Equals(left.Item4, right.Item4); } - public static bool operator !=(FdbTuple left, FdbTuple right) + public static bool operator !=(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return !comparer.Equals(left.Item1, right.Item1) @@ -242,20 +242,20 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is FdbTuple) + if (other is STuple) { - var tuple = (FdbTuple)other; + var tuple = (STuple)other; return comparer.Equals(this.Item1, tuple.Item1) && comparer.Equals(this.Item2, tuple.Item2) && comparer.Equals(this.Item3, tuple.Item3) && comparer.Equals(this.Item4, tuple.Item4); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3), @@ -263,13 +263,13 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) ); } - public static implicit operator FdbTuple(Tuple t) + public static implicit operator STuple(Tuple t) { if (t == null) throw new ArgumentNullException("t"); - return new FdbTuple(t.Item1, t.Item2, t.Item3, t.Item4); + return new STuple(t.Item1, t.Item2, t.Item3, t.Item4); } - public static explicit operator Tuple(FdbTuple t) + public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs similarity index 79% rename from FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs rename to FoundationDB.Client/Layers/Tuples/STuple`5.cs index 8e07e2583..0625f103b 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -28,16 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; using System.Text; + using FoundationDB.Client; + using FoundationDB.Client.Converters; + using JetBrains.Annotations; /// Tuple that can hold four items /// Type of the 1st item @@ -45,11 +44,11 @@ namespace FoundationDB.Layers.Tuples /// Type of the 3rd item /// Type of the 4th item /// Type of the 5th item - [ImmutableObject(true), DebuggerDisplay("{ToString()}")] - public struct FdbTuple : IFdbTuple + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an FdbTuple as an IFdbTuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the tuple public readonly T1 Item1; @@ -64,7 +63,7 @@ public struct FdbTuple : IFdbTuple /// Create a tuple containing for items [DebuggerStepThrough] - public FdbTuple(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public STuple(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { this.Item1 = item1; this.Item2 = item2; @@ -88,14 +87,14 @@ public object this[int index] case 2: case -3: return this.Item3; case 3: case -2: return this.Item4; case 4: case -1: return this.Item5; - default: FdbTuple.FailIndexOutOfRange(index, 5); return null; + default: STuple.FailIndexOutOfRange(index, 5); return null; } } } - public IFdbTuple this[int? fromIncluded, int? toExcluded] + public ITuple this[int? fromIncluded, int? toExcluded] { - get { return FdbTuple.Splice(this, fromIncluded, toExcluded); } + get { return STuple.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position @@ -111,7 +110,7 @@ public R Get(int index) case 2: case -3: return FdbConverters.Convert(this.Item3); case 3: case -2: return FdbConverters.Convert(this.Item4); case 4: case -1: return FdbConverters.Convert(this.Item5); - default: FdbTuple.FailIndexOutOfRange(index, 5); return default(R); + default: STuple.FailIndexOutOfRange(index, 5); return default(R); } } @@ -122,24 +121,24 @@ public T5 Last } /// Return the typed value of the last item in the tuple - R IFdbTuple.Last() + R ITuple.Last() { return FdbConverters.Convert(this.Item5); } public void PackTo(ref TupleWriter writer) { - FdbTuplePacker.Encoder(ref writer, this.Item1); - FdbTuplePacker.Encoder(ref writer, this.Item2); - FdbTuplePacker.Encoder(ref writer, this.Item3); - FdbTuplePacker.Encoder(ref writer, this.Item4); - FdbTuplePacker.Encoder(ref writer, this.Item5); + TuplePacker.Encoder(ref writer, this.Item1); + TuplePacker.Encoder(ref writer, this.Item2); + TuplePacker.Encoder(ref writer, this.Item3); + TuplePacker.Encoder(ref writer, this.Item4); + TuplePacker.Encoder(ref writer, this.Item5); } - IFdbTuple IFdbTuple.Append(T6 value) + ITuple ITuple.Append(T6 value) { // the caller doesn't care about the return type, so just box everything into a list tuple - return new FdbListTuple(new object[6] { this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, value }, 0, 6); + return new ListTuple(new object[6] { this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, value }, 0, 6); } /// Appends a single new item at the end of the current tuple. @@ -147,19 +146,19 @@ IFdbTuple IFdbTuple.Append(T6 value) /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! [NotNull] - public FdbLinkedTuple Append(T6 value) + public LinkedTuple Append(T6 value) { // the caller probably cares about the return type, since it is using a struct, but whatever tuple type we use will end up boxing this tuple on the heap, and we will loose type information. // but, by returning a FdbLinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) - return new FdbLinkedTuple(this, value); + return new LinkedTuple(this, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - public IFdbTuple Concat(IFdbTuple tuple) + public ITuple Concat(ITuple tuple) { - return FdbTuple.Concat(this, tuple); + return STuple.Concat(this, tuple); } /// Copy all the items of this tuple into an array at the specified offset @@ -203,17 +202,17 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public Slice ToSlice() { - return FdbTuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); + return STuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); } public override string ToString() { return new StringBuilder(48).Append('(') - .Append(FdbTuple.Stringify(this.Item1)).Append(", ") - .Append(FdbTuple.Stringify(this.Item2)).Append(", ") - .Append(FdbTuple.Stringify(this.Item3)).Append(", ") - .Append(FdbTuple.Stringify(this.Item4)).Append(", ") - .Append(FdbTuple.Stringify(this.Item5)).Append(')') + .Append(STuple.Stringify(this.Item1)).Append(", ") + .Append(STuple.Stringify(this.Item2)).Append(", ") + .Append(STuple.Stringify(this.Item3)).Append(", ") + .Append(STuple.Stringify(this.Item4)).Append(", ") + .Append(STuple.Stringify(this.Item5)).Append(')') .ToString(); } @@ -222,7 +221,7 @@ public override bool Equals(object obj) return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } - public bool Equals(IFdbTuple other) + public bool Equals(ITuple other) { return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } @@ -232,7 +231,7 @@ public override int GetHashCode() return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } - public static bool operator ==(FdbTuple left, FdbTuple right) + public static bool operator ==(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return comparer.Equals(left.Item1, right.Item1) @@ -242,7 +241,7 @@ public override int GetHashCode() && comparer.Equals(left.Item5, right.Item5); } - public static bool operator !=(FdbTuple left, FdbTuple right) + public static bool operator !=(STuple left, STuple right) { var comparer = SimilarValueComparer.Default; return !comparer.Equals(left.Item1, right.Item1) @@ -255,21 +254,21 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is FdbTuple) + if (other is STuple) { - var tuple = (FdbTuple)other; + var tuple = (STuple)other; return comparer.Equals(this.Item1, tuple.Item1) && comparer.Equals(this.Item2, tuple.Item2) && comparer.Equals(this.Item3, tuple.Item3) && comparer.Equals(this.Item4, tuple.Item4) && comparer.Equals(this.Item5, tuple.Item5); } - return FdbTuple.Equals(this, other, comparer); + return STuple.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return FdbTuple.CombineHashCodes( + return STuple.CombineHashCodes( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3), @@ -278,13 +277,13 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) ); } - public static implicit operator FdbTuple(Tuple t) + public static implicit operator STuple(Tuple t) { if (t == null) throw new ArgumentNullException("t"); - return new FdbTuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); + return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } - public static explicit operator Tuple(FdbTuple t) + public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs similarity index 89% rename from FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs rename to FoundationDB.Client/Layers/Tuples/TupleComparisons.cs index 7d053333a..cb3559d1d 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleComparisons.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs @@ -36,7 +36,7 @@ namespace FoundationDB.Layers.Tuples using System.Runtime.CompilerServices; /// Helper class for tuple comparisons - public static class FdbTupleComparisons + public static class TupleComparisons { /// Tuple comparer that treats similar values as equal ("123" = 123 = 123L = 123.0d) public static readonly EqualityComparer Default = new EqualityComparer(SimilarValueComparer.Default); @@ -47,7 +47,7 @@ public static class FdbTupleComparisons /// Tuple comparer that compared the packed bytes (slow!) public static readonly BinaryComparer Binary = new BinaryComparer(); - public sealed class EqualityComparer : IEqualityComparer, IEqualityComparer + public sealed class EqualityComparer : IEqualityComparer, IEqualityComparer { private readonly IEqualityComparer m_comparer; @@ -56,7 +56,7 @@ internal EqualityComparer(IEqualityComparer comparer) m_comparer = comparer; } - public bool Equals(IFdbTuple x, IFdbTuple y) + public bool Equals(ITuple x, ITuple y) { if (object.ReferenceEquals(x, y)) return true; if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; @@ -64,7 +64,7 @@ public bool Equals(IFdbTuple x, IFdbTuple y) return x.Equals(y, m_comparer); } - public int GetHashCode(IFdbTuple obj) + public int GetHashCode(ITuple obj) { return obj != null ? obj.GetHashCode(m_comparer) : 0; } @@ -74,10 +74,10 @@ public int GetHashCode(IFdbTuple obj) if (object.ReferenceEquals(x, y)) return true; if (x == null || y == null) return false; - var t = x as IFdbTuple; + var t = x as ITuple; if (t != null) return t.Equals(y, m_comparer); - t = y as IFdbTuple; + t = y as ITuple; if (t != null) return t.Equals(x, m_comparer); return false; @@ -87,7 +87,7 @@ public int GetHashCode(object obj) { if (obj == null) return 0; - var t = obj as IFdbTuple; + var t = obj as ITuple; if (!object.ReferenceEquals(t, null)) return t.GetHashCode(m_comparer); // returns a hash base on the pointers @@ -95,13 +95,13 @@ public int GetHashCode(object obj) } } - public sealed class BinaryComparer : IEqualityComparer, IEqualityComparer + public sealed class BinaryComparer : IEqualityComparer, IEqualityComparer { internal BinaryComparer() { } - public bool Equals(IFdbTuple x, IFdbTuple y) + public bool Equals(ITuple x, ITuple y) { if (object.ReferenceEquals(x, y)) return true; if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; @@ -109,7 +109,7 @@ public bool Equals(IFdbTuple x, IFdbTuple y) return x.ToSlice().Equals(y.ToSlice()); } - public int GetHashCode(IFdbTuple obj) + public int GetHashCode(ITuple obj) { return object.ReferenceEquals(obj, null) ? 0 : obj.ToSlice().GetHashCode(); } @@ -119,8 +119,8 @@ public int GetHashCode(IFdbTuple obj) if (object.ReferenceEquals(x, y)) return true; if (x == null || y == null) return false; - var tx = x as IFdbTuple; - var ty = y as IFdbTuple; + var tx = x as ITuple; + var ty = y as ITuple; if (object.ReferenceEquals(tx, null) || object.ReferenceEquals(ty, null)) return false; return tx.ToSlice().Equals(ty.ToSlice()); } @@ -129,7 +129,7 @@ public int GetHashCode(object obj) { if (obj == null) return 0; - var tuple = obj as IFdbTuple; + var tuple = obj as ITuple; if (!object.ReferenceEquals(tuple, null)) return tuple.ToSlice().GetHashCode(); return RuntimeHelpers.GetHashCode(obj); @@ -142,7 +142,7 @@ public int GetHashCode(object obj) /// Comparer for the item's type /// New comparer instance [NotNull] - public static IComparer Composite(int offset = 0, IComparer comparer = null) + public static IComparer Composite(int offset = 0, IComparer comparer = null) { return new CompositeComparer(offset, comparer); } @@ -155,7 +155,7 @@ public static IComparer Composite(int offset = 0, IComparer c /// Comparer for the second item's type /// New comparer instance [NotNull] - public static IComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null) + public static IComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null) { return new CompositeComparer(offset, comparer1, comparer2); } @@ -170,17 +170,17 @@ public static IComparer Composite(int offset = 0, IComparerComparer for the third item's type /// New comparer instance [NotNull] - public static IComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null, IComparer comparer3 = null) + public static IComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null, IComparer comparer3 = null) { return new CompositeComparer(offset, comparer1, comparer2, comparer3); } /// Comparer that compares tuples with at least 1 item /// Type of the item - public sealed class CompositeComparer : IComparer + public sealed class CompositeComparer : IComparer { - public static readonly IComparer Default = new CompositeComparer(); + public static readonly IComparer Default = new CompositeComparer(); /// Constructor for a new tuple comparer public CompositeComparer() @@ -212,7 +212,7 @@ public CompositeComparer(int offset, IComparer comparer) /// First tuple /// Second tuple /// Returns a positive value if x is greater than y, a negative value if x is less than y and 0 if x is equal to y. - public int Compare(IFdbTuple x, IFdbTuple y) + public int Compare(ITuple x, ITuple y) { if (y == null) return x == null ? 0 : +1; if (x == null) return -1; @@ -230,10 +230,10 @@ public int Compare(IFdbTuple x, IFdbTuple y) /// Comparer that compares tuples with at least 2 items /// Type of the first item /// Type of the second item - public sealed class CompositeComparer : IComparer + public sealed class CompositeComparer : IComparer { - public static readonly IComparer Default = new CompositeComparer(); + public static readonly IComparer Default = new CompositeComparer(); /// Constructor for a new tuple comparer public CompositeComparer() @@ -270,7 +270,7 @@ public CompositeComparer(int offset, IComparer comparer1, IComparer comp /// First tuple /// Second tuple /// Returns a positive value if x is greater than y, a negative value if x is less than y and 0 if x is equal to y. - public int Compare(IFdbTuple x, IFdbTuple y) + public int Compare(ITuple x, ITuple y) { if (y == null) return x == null ? 0 : +1; if (x == null) return -1; @@ -296,10 +296,10 @@ public int Compare(IFdbTuple x, IFdbTuple y) /// Type of the first item /// Type of the second item /// Type of the thrid item - public sealed class CompositeComparer : IComparer + public sealed class CompositeComparer : IComparer { - public static readonly IComparer Default = new CompositeComparer(); + public static readonly IComparer Default = new CompositeComparer(); /// Constructor for a new tuple comparer public CompositeComparer() @@ -341,7 +341,7 @@ public CompositeComparer(int offset, IComparer comparer1, IComparer comp /// First tuple /// Second tuple /// Returns a positive value if x is greater than y, a negative value if x is less than y and 0 if x is equal to y. - public int Compare(IFdbTuple x, IFdbTuple y) + public int Compare(ITuple x, ITuple y) { if (y == null) return x == null ? 0 : +1; if (x == null) return -1; diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs similarity index 78% rename from FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs rename to FoundationDB.Client/Layers/Tuples/TupleExtensions.cs index 2da6f33b3..8a3e8797c 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs @@ -28,36 +28,36 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples { - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; + using FoundationDB.Client; + using JetBrains.Annotations; /// Add extensions methods that deal with tuples on various types - public static class FdbTupleExtensions + public static class TupleExtensions { - #region IFdbTuple extensions... + #region ITuple extensions... /// Returns true if the tuple is either null or empty [ContractAnnotation("null => true")] - public static bool IsNullOrEmpty(this IFdbTuple tuple) + public static bool IsNullOrEmpty(this ITuple tuple) { return tuple == null || tuple.Count == 0; } /// Returns true if the tuple is not null, and contains only one item [ContractAnnotation("null => false")] - public static bool IsSingleton(this IFdbTuple tuple) + public static bool IsSingleton(this ITuple tuple) { return tuple != null && tuple.Count == 1; } /// Returns an array containing all the objects of a tuple [NotNull, ItemCanBeNull] - public static object[] ToArray([NotNull] this IFdbTuple tuple) + public static object[] ToArray([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); var items = new object[tuple.Count]; if (items.Length > 0) @@ -69,9 +69,9 @@ public static object[] ToArray([NotNull] this IFdbTuple tuple) /// Returns a typed array containing all the items of a tuple [NotNull] - public static T[] ToArray([NotNull] this IFdbTuple tuple) + public static T[] ToArray([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); var items = new T[tuple.Count]; if (items.Length > 0) @@ -86,7 +86,7 @@ public static T[] ToArray([NotNull] this IFdbTuple tuple) /// Returns a byte array containing the packed version of a tuple [CanBeNull] - public static byte[] GetBytes([NotNull] this IFdbTuple tuple) + public static byte[] GetBytes([NotNull] this ITuple tuple) { return tuple.ToSlice().GetBytes(); } @@ -94,40 +94,40 @@ public static byte[] GetBytes([NotNull] this IFdbTuple tuple) /// Returns the typed value of the first item in this tuple /// Expected type of the first item /// Value of the first item, adapted into type . - public static T First([NotNull] this IFdbTuple tuple) + public static T First([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return tuple.Get(0); } /// Appends two values at the end of a tuple [NotNull] - public static IFdbTuple Append([NotNull] this IFdbTuple tuple, T1 value1, T2 value2) + public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2) { - if (tuple == null) throw new ArgumentNullException("tuple"); - return new FdbJoinedTuple(tuple, FdbTuple.Create(value1, value2)); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + return new JoinedTuple(tuple, STuple.Create(value1, value2)); } /// Appends three values at the end of a tuple [NotNull] - public static IFdbTuple Append([NotNull] this IFdbTuple tuple, T1 value1, T2 value2, T3 value3) + public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2, T3 value3) { - if (tuple == null) throw new ArgumentNullException("tuple"); - return new FdbJoinedTuple(tuple, FdbTuple.Create(value1, value2, value3)); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + return new JoinedTuple(tuple, STuple.Create(value1, value2, value3)); } /// Appends four values at the end of a tuple [NotNull] - public static IFdbTuple Append([NotNull] this IFdbTuple tuple, T1 value1, T2 value2, T3 value3, T4 value4) + public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2, T3 value3, T4 value4) { - if (tuple == null) throw new ArgumentNullException("tuple"); - return new FdbJoinedTuple(tuple, FdbTuple.Create(value1, value2, value3, value4)); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + return new JoinedTuple(tuple, STuple.Create(value1, value2, value3, value4)); } /// Creates a key range containing all children of this tuple, from tuple.pack()+'\0' to tuple.pack()+'\xFF' /// Tuple that is the suffix of all keys /// Range of all keys suffixed by the tuple. The tuple itself will not be included - public static KeyRange ToRange([NotNull] this IFdbTuple tuple) + public static KeyRange ToRange([NotNull] this ITuple tuple) { return ToRange(tuple, false); } @@ -136,9 +136,9 @@ public static KeyRange ToRange([NotNull] this IFdbTuple tuple) /// Tuple that is the prefix of all keys /// If true, the tuple key itself is included, if false only the children keys are included /// Range of all keys suffixed by the tuple. The tuple itself will be included if is true - public static KeyRange ToRange([NotNull] this IFdbTuple tuple, bool includePrefix) + public static KeyRange ToRange([NotNull] this ITuple tuple, bool includePrefix) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); // We want to allocate only one byte[] to store both keys, and map both Slice to each chunk // So we will serialize the tuple two times in the same writer @@ -165,39 +165,39 @@ public static KeyRange ToRange([NotNull] this IFdbTuple tuple, bool includePrefi /// Create a copy of the tuple that can be reused frequently to pack values /// If the tuple is already memoized, the current instance will be returned [CanBeNull, ContractAnnotation("null => null")] - public static FdbMemoizedTuple Memoize(this IFdbTuple tuple) + public static MemoizedTuple Memoize(this ITuple tuple) { if (tuple == null) return null; - var memoized = tuple as FdbMemoizedTuple ?? new FdbMemoizedTuple(tuple.ToArray(), tuple.ToSlice()); + var memoized = tuple as MemoizedTuple ?? new MemoizedTuple(tuple.ToArray(), tuple.ToSlice()); return memoized; } /// Unpack a tuple from this slice /// - /// Unpacked tuple if the slice contains data, FdbTuple.Empty if the slice is empty, or null if the slice is Slice.Nil + /// Unpacked tuple if the slice contains data, STuple.Empty if the slice is empty, or null if the slice is Slice.Nil [NotNull] - public static IFdbTuple ToTuple(this Slice slice) + public static ITuple ToTuple(this Slice slice) { //note: this method is here to allow a fluent API with method chaining, like "something.ToFoundationDbKey().ToTuple().With((int x, int y) => .....)" - return FdbTuple.Unpack(slice); + return STuple.Unpack(slice); } /// Unpack a tuple from this slice /// - /// Unpacked tuple if the slice contains data, FdbTuple.Empty if the slice is empty, or null if the slice is Slice.Nil + /// Unpacked tuple if the slice contains data, STuple.Empty if the slice is empty, or null if the slice is Slice.Nil [CanBeNull] - public static IFdbTuple ToTupleOrDefault(this Slice slice) + public static ITuple ToTupleOrDefault(this Slice slice) { //note: this method is here to allow a fluent API with method chaining, like "something.ToFoundationDbKey().ToTuple().With((int x, int y) => .....)" if (slice.IsNullOrEmpty) { - return slice.HasValue ? FdbTuple.Empty : null; + return slice.HasValue ? STuple.Empty : null; } - return FdbTuple.Unpack(slice); + return STuple.Unpack(slice); } /// Returns a substring of the current tuple @@ -205,9 +205,9 @@ public static IFdbTuple ToTupleOrDefault(this Slice slice) /// Offset from the start of the current tuple (negative value means from the end) /// Tuple that contains only the items past the first items of the current tuple [NotNull] - public static IFdbTuple Substring([NotNull] this IFdbTuple tuple, int offset) + public static ITuple Substring([NotNull] this ITuple tuple, int offset) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return tuple[offset, null]; } @@ -218,12 +218,12 @@ public static IFdbTuple Substring([NotNull] this IFdbTuple tuple, int offset) /// Number of items to keep /// Tuple that contains only the selected items from the current tuple [NotNull] - public static IFdbTuple Substring([NotNull] this IFdbTuple tuple, int offset, int count) + public static ITuple Substring([NotNull] this ITuple tuple, int offset, int count) { - if (tuple == null) throw new ArgumentNullException("tuple"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count cannot be negative."); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative."); - if (count == 0) return FdbTuple.Empty; + if (count == 0) return STuple.Empty; return tuple[offset, offset + count]; } @@ -236,7 +236,7 @@ public static IFdbTuple Substring([NotNull] this IFdbTuple tuple, int offset, in /// (a, b, c).Truncate(2) => (a, b) /// (a, b, c).Truncate(-2) => (b, c) /// - public static IFdbTuple Truncate([NotNull] this IFdbTuple tuple, int count) + public static ITuple Truncate([NotNull] this ITuple tuple, int count) { tuple.OfSizeAtLeast(Math.Abs(count)); @@ -255,35 +255,35 @@ public static IFdbTuple Truncate([NotNull] this IFdbTuple tuple, int count) /// Larger tuple /// Smaller tuple /// True if the beginning of is equal to or if both tuples are identical - public static bool StartsWith([NotNull] this IFdbTuple left, [NotNull] IFdbTuple right) + public static bool StartsWith([NotNull] this ITuple left, [NotNull] ITuple right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); - //REVIEW: move this on IFdbTuple interface ? - return FdbTuple.StartsWith(left, right); + //REVIEW: move this on ITuple interface ? + return STuple.StartsWith(left, right); } /// Test if the end of current tuple is equal to another tuple /// Larger tuple /// Smaller tuple /// True if the end of is equal to or if both tuples are identical - public static bool EndsWith([NotNull] this IFdbTuple left, [NotNull] IFdbTuple right) + public static bool EndsWith([NotNull] this ITuple left, [NotNull] ITuple right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + if (left == null) throw new ArgumentNullException(nameof(left)); + if (right == null) throw new ArgumentNullException(nameof(right)); - //REVIEW: move this on IFdbTuple interface ? - return FdbTuple.EndsWith(left, right); + //REVIEW: move this on ITuple interface ? + return STuple.EndsWith(left, right); } /// Transform a tuple of N elements into a list of N singletons /// Tuple that contains any number of elements /// Sequence of tuples that contains a single element /// (123, ABC, false,).Explode() => [ (123,), (ABC,), (false,) ] - public static IEnumerable Explode([NotNull] this IFdbTuple tuple) + public static IEnumerable Explode([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); int p = 0; int n = tuple.Count; @@ -296,16 +296,16 @@ public static IEnumerable Explode([NotNull] this IFdbTuple tuple) /// Returns a key that is immediately after the packed representation of this tuple /// This is the equivalent of manually packing the tuple and incrementing the resulting slice - public static Slice Increment([NotNull] this IFdbTuple tuple) + public static Slice Increment([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return FdbKey.Increment(tuple.ToSlice()); } /// Returns a Key Selector pair that defines the range of all items contained under this tuple - public static KeySelectorPair ToSelectorPair([NotNull] this IFdbTuple tuple) + public static KeySelectorPair ToSelectorPair([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return KeySelectorPair.StartsWith(tuple.ToSlice()); } @@ -318,7 +318,7 @@ public static KeySelectorPair ToSelectorPair([NotNull] this IFdbTuple tuple) /// If is smaller or larger than [ContractAnnotation("halt <= tuple:null")] [NotNull] - public static IFdbTuple OfSize(this IFdbTuple tuple, int size) + public static ITuple OfSize(this ITuple tuple, int size) { if (tuple == null || tuple.Count != size) ThrowInvalidTupleSize(tuple, size, 0); return tuple; @@ -332,7 +332,7 @@ public static IFdbTuple OfSize(this IFdbTuple tuple, int size) /// If is smaller than [ContractAnnotation("halt <= tuple:null")] [NotNull] - public static IFdbTuple OfSizeAtLeast(this IFdbTuple tuple, int size) + public static ITuple OfSizeAtLeast(this ITuple tuple, int size) { if (tuple == null || tuple.Count < size) ThrowInvalidTupleSize(tuple, size, -1); return tuple; @@ -346,18 +346,18 @@ public static IFdbTuple OfSizeAtLeast(this IFdbTuple tuple, int size) /// If is larger than [ContractAnnotation("halt <= tuple:null")] [NotNull] - public static IFdbTuple OfSizeAtMost(this IFdbTuple tuple, int size) + public static ITuple OfSizeAtMost(this ITuple tuple, int size) { if (tuple == null || tuple.Count > size) ThrowInvalidTupleSize(tuple, size, 1); return tuple; } [ContractAnnotation("=> halt")] - internal static void ThrowInvalidTupleSize(IFdbTuple tuple, int expected, int test) + internal static void ThrowInvalidTupleSize(ITuple tuple, int expected, int test) { if (tuple == null) { - throw new ArgumentNullException("tuple"); + throw new ArgumentNullException(nameof(tuple)); } switch(test) { @@ -371,10 +371,10 @@ internal static void ThrowInvalidTupleSize(IFdbTuple tuple, int expected, int te /// Expected type of the single element /// Tuple that must be of size 1 /// Equivalent tuple, with its element converted to the specified type - public static FdbTuple As([NotNull] this IFdbTuple tuple) + public static STuple As([NotNull] this ITuple tuple) { tuple.OfSize(1); - return new FdbTuple(tuple.Get(0)); + return new STuple(tuple.Get(0)); } /// Returns a typed version of a tuple of size 2 @@ -382,10 +382,10 @@ public static FdbTuple As([NotNull] this IFdbTuple tuple) /// Expected type of the second element /// Tuple that must be of size 2 /// Equivalent tuple, with its elements converted to the specified types - public static FdbTuple As([NotNull] this IFdbTuple tuple) + public static STuple As([NotNull] this ITuple tuple) { tuple.OfSize(2); - return new FdbTuple( + return new STuple( tuple.Get(0), tuple.Get(1) ); @@ -397,10 +397,10 @@ public static FdbTuple As([NotNull] this IFdbTuple tuple) /// Expected type of the third element /// Tuple that must be of size 3 /// Equivalent tuple, with its elements converted to the specified types - public static FdbTuple As([NotNull] this IFdbTuple tuple) + public static STuple As([NotNull] this ITuple tuple) { tuple.OfSize(3); - return new FdbTuple( + return new STuple( tuple.Get(0), tuple.Get(1), tuple.Get(2) @@ -414,10 +414,10 @@ public static FdbTuple As([NotNull] this IFdbTuple tuple /// Expected type of the fourth element /// Tuple that must be of size 4 /// Equivalent tuple, with its elements converted to the specified types - public static FdbTuple As([NotNull] this IFdbTuple tuple) + public static STuple As([NotNull] this ITuple tuple) { tuple.OfSize(4); - return new FdbTuple( + return new STuple( tuple.Get(0), tuple.Get(1), tuple.Get(2), @@ -433,10 +433,10 @@ public static FdbTuple As([NotNull] this IFdbTup /// Expected type of the fifth element /// Tuple that must be of size 5 /// Equivalent tuple, with its elements converted to the specified types - public static FdbTuple As([NotNull] this IFdbTuple tuple) + public static STuple As([NotNull] this ITuple tuple) { tuple.OfSize(5); - return new FdbTuple( + return new STuple( tuple.Get(0), tuple.Get(1), tuple.Get(2), @@ -449,7 +449,7 @@ public static FdbTuple As([NotNull] this /// Tuple of size 1 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 1); lambda(tuple.Get(0)); @@ -459,7 +459,7 @@ public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action /// Tuple of size 2 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 2); lambda(tuple.Get(0), tuple.Get(1)); @@ -469,7 +469,7 @@ public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action /// Tuple of size 3 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 3); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2)); @@ -479,7 +479,7 @@ public static void With([NotNull] this IFdbTuple tuple, [NotNull] Ac /// Tuple of size 4 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 4); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3)); @@ -489,7 +489,7 @@ public static void With([NotNull] this IFdbTuple tuple, [NotNull /// Tuple of size 5 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 5); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4)); @@ -499,7 +499,7 @@ public static void With([NotNull] this IFdbTuple tuple, [Not /// Tuple of size 6 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 6); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5)); @@ -509,7 +509,7 @@ public static void With([NotNull] this IFdbTuple tuple, /// Tuple of size 7 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 7); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5), tuple.Get(6)); @@ -519,7 +519,7 @@ public static void With([NotNull] this IFdbTuple tup /// Tuple of size 8 /// Action that will be passed the content of this tuple as parameters /// If has not the expected size - public static void With([NotNull] this IFdbTuple tuple, [NotNull] Action lambda) + public static void With([NotNull] this ITuple tuple, [NotNull] Action lambda) { OfSize(tuple, 8); lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5), tuple.Get(6), tuple.Get(7)); @@ -530,7 +530,7 @@ public static void With([NotNull] this IFdbTuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 1); return lambda(tuple.Get(0)); @@ -541,7 +541,7 @@ public static TResult With([NotNull] this IFdbTuple tuple, [NotNull /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 2); return lambda(tuple.Get(0), tuple.Get(1)); @@ -552,7 +552,7 @@ public static TResult With([NotNull] this IFdbTuple tuple, [Not /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 3); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2)); @@ -563,7 +563,7 @@ public static TResult With([NotNull] this IFdbTuple tuple, /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 4); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3)); @@ -574,7 +574,7 @@ public static TResult With([NotNull] this IFdbTuple tup /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 5); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4)); @@ -585,7 +585,7 @@ public static TResult With([NotNull] this IFdbTuple /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 6); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5)); @@ -596,7 +596,7 @@ public static TResult With([NotNull] this IFdbT /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 7); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5), tuple.Get(6)); @@ -607,7 +607,7 @@ public static TResult With([NotNull] this I /// Function that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple /// If has not the expected size - public static TResult With([NotNull] this IFdbTuple tuple, [NotNull] Func lambda) + public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { OfSize(tuple, 8); return lambda(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3), tuple.Get(4), tuple.Get(5), tuple.Get(6), tuple.Get(7)); diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs index 2f0fc05b3..59b731477 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs @@ -46,183 +46,183 @@ public IFdbKeyEncoding Encoding public KeyRange ToRange(Slice prefix) { - return FdbTuple.ToRange(prefix); + return STuple.ToRange(prefix); } - public void PackKey(ref SliceWriter writer, IFdbTuple items) + public void PackKey(ref SliceWriter writer, ITuple items) { var tw = new TupleWriter(writer); - FdbTuple.Pack(ref tw, items); + STuple.Pack(ref tw, items); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item1); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); - FdbTuplePacker.SerializeTo(ref tw, item4); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item4); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); - FdbTuplePacker.SerializeTo(ref tw, item4); - FdbTuplePacker.SerializeTo(ref tw, item5); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item4); + TuplePacker.SerializeTo(ref tw, item5); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); - FdbTuplePacker.SerializeTo(ref tw, item4); - FdbTuplePacker.SerializeTo(ref tw, item5); - FdbTuplePacker.SerializeTo(ref tw, item6); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item4); + TuplePacker.SerializeTo(ref tw, item5); + TuplePacker.SerializeTo(ref tw, item6); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); - FdbTuplePacker.SerializeTo(ref tw, item4); - FdbTuplePacker.SerializeTo(ref tw, item5); - FdbTuplePacker.SerializeTo(ref tw, item6); - FdbTuplePacker.SerializeTo(ref tw, item7); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item4); + TuplePacker.SerializeTo(ref tw, item5); + TuplePacker.SerializeTo(ref tw, item6); + TuplePacker.SerializeTo(ref tw, item7); writer = tw.Output; } public void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { var tw = new TupleWriter(writer); - FdbTuplePacker.SerializeTo(ref tw, item1); - FdbTuplePacker.SerializeTo(ref tw, item2); - FdbTuplePacker.SerializeTo(ref tw, item3); - FdbTuplePacker.SerializeTo(ref tw, item4); - FdbTuplePacker.SerializeTo(ref tw, item5); - FdbTuplePacker.SerializeTo(ref tw, item6); - FdbTuplePacker.SerializeTo(ref tw, item7); - FdbTuplePacker.SerializeTo(ref tw, item8); + TuplePacker.SerializeTo(ref tw, item1); + TuplePacker.SerializeTo(ref tw, item2); + TuplePacker.SerializeTo(ref tw, item3); + TuplePacker.SerializeTo(ref tw, item4); + TuplePacker.SerializeTo(ref tw, item5); + TuplePacker.SerializeTo(ref tw, item6); + TuplePacker.SerializeTo(ref tw, item7); + TuplePacker.SerializeTo(ref tw, item8); writer = tw.Output; } - public IFdbTuple UnpackKey(Slice packed) + public ITuple UnpackKey(Slice packed) { - return FdbTuple.Unpack(packed); + return STuple.Unpack(packed); } public T DecodeKey(Slice packed) { - return FdbTuple.DecodeKey(packed); + return STuple.DecodeKey(packed); } public T DecodeKeyFirst(Slice packed) { - return FdbTuple.DecodeFirst(packed); + return STuple.DecodeFirst(packed); } public T DecodeKeyLast(Slice packed) { - return FdbTuple.DecodeLast(packed); + return STuple.DecodeLast(packed); } - public FdbTuple DecodeKey(Slice packed) + public STuple DecodeKey(Slice packed) { - return FdbTuple.DecodeKey(packed); + return STuple.DecodeKey(packed); } - public FdbTuple DecodeKey(Slice packed) + public STuple DecodeKey(Slice packed) { - return FdbTuple.DecodeKey(packed); + return STuple.DecodeKey(packed); } - public FdbTuple DecodeKey(Slice packed) + public STuple DecodeKey(Slice packed) { - return FdbTuple.DecodeKey(packed); + return STuple.DecodeKey(packed); } - public FdbTuple DecodeKey(Slice packed) + public STuple DecodeKey(Slice packed) { - return FdbTuple.DecodeKey(packed); + return STuple.DecodeKey(packed); } - public KeyRange ToRange(Slice prefix, IFdbTuple items) + public KeyRange ToRange(Slice prefix, ITuple items) { - return FdbTuple.ToRange(prefix, items); + return STuple.ToRange(prefix, items); } public KeyRange ToKeyRange(Slice prefix, T1 item1) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1)); + return STuple.ToRange(prefix, STuple.Create(item1)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item2)); + return STuple.ToRange(prefix, STuple.Create(item1, item2)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - return FdbTuple.ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); + return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } } diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs index 3faee77b3..8dec069fe 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs @@ -93,7 +93,7 @@ public static Slice[] Convert(SliceWriter writer, [NotNull, ItemNotNull] IEnumer /// Key helper for a dynamic TypeSystem public struct FdbDynamicSubspaceKeys { - //NOTE: everytime an IFdbTuple is used here, it is as a container (vector of objects), and NOT as the Tuple Encoding scheme ! (separate concept) + //NOTE: everytime an ITuple is used here, it is as a container (vector of objects), and NOT as the Tuple Encoding scheme ! (separate concept) /// Parent subspace [NotNull] public readonly IFdbSubspace Subspace; @@ -116,7 +116,7 @@ public KeyRange ToRange() /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder /// Tuple used as a prefix for the range - public KeyRange ToRange([NotNull] IFdbTuple tuple) + public KeyRange ToRange([NotNull] ITuple tuple) { return this.Encoder.ToRange(Pack(tuple)); } @@ -130,8 +130,8 @@ public KeyRange ToRange([NotNull] ITupleFormattable item) /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix - /// This is a shortcut for - public Slice this[[NotNull] IFdbTuple tuple] + /// This is a shortcut for + public Slice this[[NotNull] ITuple tuple] { get { return Pack(tuple); } } @@ -146,7 +146,7 @@ public Slice this[[NotNull] ITupleFormattable item] /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix - public Slice Pack([NotNull] IFdbTuple tuple) + public Slice Pack([NotNull] ITuple tuple) { Contract.NotNull(tuple, nameof(tuple)); @@ -157,14 +157,14 @@ public Slice Pack([NotNull] IFdbTuple tuple) /// Convert a batch of tuples into keys of this subspace, in an optimized way. /// Sequence of tuple that will be packed and appended to the subspace prefix - public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable tuples) + public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable tuples) { if (tuples == null) throw new ArgumentNullException("tuples"); - return Batched.Convert( + return Batched.Convert( this.Subspace.GetWriter(), tuples, - (ref SliceWriter writer, IFdbTuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), + (ref SliceWriter writer, ITuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), this.Encoder ); } @@ -184,10 +184,10 @@ public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable it { if (items == null) throw new ArgumentNullException("items"); - return Batched.Convert( + return Batched.Convert( this.Subspace.GetWriter(), items.Select(item => item.ToTuple()), - (ref SliceWriter writer, IFdbTuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), + (ref SliceWriter writer, ITuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), this.Encoder ); } @@ -356,9 +356,9 @@ public Slice[] EncodeMany(IEnumerableUnpack a key of this subspace, back into a tuple - /// Key that was produced by a previous call to + /// Key that was produced by a previous call to /// Original tuple - public IFdbTuple Unpack(Slice packed) + public ITuple Unpack(Slice packed) { return this.Encoder.UnpackKey(this.Subspace.ExtractKey(packed)); } @@ -389,9 +389,9 @@ private static T[] BatchDecode(IEnumerable packed, IFdbSubspace subspa } /// Unpack a batch of keys of this subspace, back into an array of tuples - /// Sequence of keys that were produced by a previous call to or + /// Sequence of keys that were produced by a previous call to or /// Array containing the original tuples - public IFdbTuple[] UnpackMany(IEnumerable packed) + public ITuple[] UnpackMany(IEnumerable packed) { return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.UnpackKey(data)); } @@ -409,49 +409,49 @@ public IEnumerable DecodeMany(IEnumerable packed) } /// Decode a key of this subspace, composed of exactly two elements - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } /// Decode a batch of keys of this subspace, each one composed of exactly two elements - public IEnumerable> DecodeMany(IEnumerable packed) + public IEnumerable> DecodeMany(IEnumerable packed) { return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); } /// Decode a key of this subspace, composed of exactly three elements - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } /// Decode a batch of keys of this subspace, each one composed of exactly three elements - public IEnumerable> DecodeMany(IEnumerable packed) + public IEnumerable> DecodeMany(IEnumerable packed) { return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); } /// Decode a key of this subspace, composed of exactly four elements - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } /// Decode a batch of keys of this subspace, each one composed of exactly four elements - public IEnumerable> DecodeMany(IEnumerable packed) + public IEnumerable> DecodeMany(IEnumerable packed) { return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); } /// Decode a key of this subspace, composed of exactly five elements - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } /// Decode a batch of keys of this subspace, each one composed of exactly five elements - public IEnumerable> DecodeMany(IEnumerable packed) + public IEnumerable> DecodeMany(IEnumerable packed) { return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); } @@ -489,18 +489,18 @@ public IEnumerable DecodeLastMany(Slice[] packed) /// Return an empty tuple that is attached to this subspace /// Empty tuple that can be extended, and whose packed representation will always be prefixed by the subspace key [NotNull] - public IFdbTuple ToTuple() + public ITuple ToTuple() { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Empty); + return new PrefixedTuple(this.Subspace.Key, STuple.Empty); } /// Attach a tuple to an existing subspace. /// Tuple whose items will be appended at the end of the current subspace /// Tuple that wraps the items of and whose packed representation will always be prefixed by the subspace key. [NotNull] - public IFdbTuple Concat([NotNull] IFdbTuple tuple) + public ITuple Concat([NotNull] ITuple tuple) { - return new FdbPrefixedTuple(this.Subspace.Key, tuple); + return new PrefixedTuple(this.Subspace.Key, tuple); } /// Convert a formattable item into a tuple that is attached to this subspace. @@ -508,23 +508,23 @@ public IFdbTuple Concat([NotNull] IFdbTuple tuple) /// Tuple that is the logical representation of the item, and whose packed representation will always be prefixed by the subspace key. /// This is the equivalent of calling 'subspace.Create(formattable.ToTuple())' [NotNull] - public IFdbTuple Concat([NotNull] ITupleFormattable formattable) + public ITuple Concat([NotNull] ITupleFormattable formattable) { if (formattable == null) throw new ArgumentNullException("formattable"); var tuple = formattable.ToTuple(); if (tuple == null) throw new InvalidOperationException("Formattable item cannot return an empty tuple"); - return new FdbPrefixedTuple(this.Subspace.Key, tuple); + return new PrefixedTuple(this.Subspace.Key, tuple); } /// Create a new 1-tuple that is attached to this subspace /// Type of the value to append /// Value that will be appended /// Tuple of size 1 that contains , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(FdbTuple.Create<T>(value))' + /// This is the equivalent of calling 'subspace.Create(STuple.Create<T>(value))' [NotNull] - public IFdbTuple Append(T value) + public ITuple Append(T value) { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Create(value)); + return new PrefixedTuple(this.Subspace.Key, STuple.Create(value)); } /// Create a new 2-tuple that is attached to this subspace @@ -533,11 +533,11 @@ public IFdbTuple Append(T value) /// First value that will be appended /// Second value that will be appended /// Tuple of size 2 that contains and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(FdbTuple.Create<T1, T2>(item1, item2))' + /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2>(item1, item2))' [NotNull] - public IFdbTuple Append(T1 item1, T2 item2) + public ITuple Append(T1 item1, T2 item2) { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Create(item1, item2)); + return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2)); } /// Create a new 3-tuple that is attached to this subspace @@ -548,11 +548,11 @@ public IFdbTuple Append(T1 item1, T2 item2) /// Second value that will be appended /// Third value that will be appended /// Tuple of size 3 that contains , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(FdbTuple.Create<T1, T2, T3>(item1, item2, item3))' + /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3>(item1, item2, item3))' [NotNull] - public IFdbTuple Append(T1 item1, T2 item2, T3 item3) + public ITuple Append(T1 item1, T2 item2, T3 item3) { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Create(item1, item2, item3)); + return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3)); } /// Create a new 4-tuple that is attached to this subspace @@ -565,11 +565,11 @@ public IFdbTuple Append(T1 item1, T2 item2, T3 item3) /// Third value that will be appended /// Fourth value that will be appended /// Tuple of size 4 that contains , , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(FdbTuple.Create<T1, T2, T3, T4>(item1, item2, item3, item4))' + /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3, T4>(item1, item2, item3, item4))' [NotNull] - public IFdbTuple Append(T1 item1, T2 item2, T3 item3, T4 item4) + public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4) { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Create(item1, item2, item3, item4)); + return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3, item4)); } /// Create a new 5-tuple that is attached to this subspace @@ -584,11 +584,11 @@ public IFdbTuple Append(T1 item1, T2 item2, T3 item3, T4 item4) /// Fourth value that will be appended /// Fifth value that will be appended /// Tuple of size 5 that contains , , , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(FdbTuple.Create<T1, T2, T3, T4, T5>(item1, item2, item3, item4, item5))' + /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3, T4, T5>(item1, item2, item3, item4, item5))' [NotNull] - public IFdbTuple Append(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return new FdbPrefixedTuple(this.Subspace.Key, FdbTuple.Create(item1, item2, item3, item4, item5)); + return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3, item4, item5)); } #endregion diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs index 489bcbac2..865656f68 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs @@ -91,7 +91,7 @@ public IFdbDynamicSubspace this[Slice suffix] } } - public IFdbDynamicSubspace this[IFdbTuple tuple] + public IFdbDynamicSubspace this[ITuple tuple] { [ContractAnnotation("null => halt; notnull => notnull")] get diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs index 6f7ea7a2a..549a92705 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs @@ -79,7 +79,7 @@ public T Decode(Slice packed) public KeyRange ToRange(T value) { //REVIEW: which semantic for ToRange() should we use? - return FdbTuple.ToRange(Encode(value)); + return STuple.ToRange(Encode(value)); } } diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs index 53760d30a..ce97fc2a4 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs @@ -71,7 +71,7 @@ public Slice[] Encode([NotNull] IEnumerable values, [NotNull] ); } - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } @@ -79,7 +79,7 @@ public FdbTuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2) { //REVIEW: which semantic for ToRange() should we use? - return FdbTuple.ToRange(Encode(value1, value2)); + return STuple.ToRange(Encode(value1, value2)); } } diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs index 958972eea..01f1742e1 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs @@ -66,7 +66,7 @@ public Slice[] Encode([NotNull] IEnumerable values, [NotNull] ); } - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } @@ -74,7 +74,7 @@ public FdbTuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2, T3 value3) { //REVIEW: which semantic for ToRange() should we use? - return FdbTuple.ToRange(Encode(value1, value2, value3)); + return STuple.ToRange(Encode(value1, value2, value3)); } } diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs index f4971e8d0..49e322355 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs +++ b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs @@ -72,7 +72,7 @@ public Slice[] Encode([NotNull] IEnumerable values, [NotNull] ); } - public FdbTuple Decode(Slice packed) + public STuple Decode(Slice packed) { return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); } @@ -80,7 +80,7 @@ public FdbTuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) { //REVIEW: which semantic for ToRange() should we use? - return FdbTuple.ToRange(Encode(value1, value2, value3, value4)); + return STuple.ToRange(Encode(value1, value2, value3, value4)); } } diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/FdbSubspace.cs index d9c8d01f3..dd8aab308 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/FdbSubspace.cs @@ -121,7 +121,7 @@ public static IFdbDynamicSubspace CreateDynamic(Slice slice, [NotNull] IDynamicK /// Optional type encoding used by this subspace. /// New subspace instance that will use the packed representation of as its prefix [Pure, NotNull] - public static IFdbDynamicSubspace CreateDynamic([NotNull] IFdbTuple tuple, IFdbKeyEncoding encoding = null) + public static IFdbDynamicSubspace CreateDynamic([NotNull] ITuple tuple, IFdbKeyEncoding encoding = null) { Contract.NotNull(tuple, nameof(tuple)); var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index 1a1c0ab7e..de76de445 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -13,49 +13,49 @@ public virtual KeyRange ToRange(Slice prefix) return KeyRange.StartsWith(prefix); } - public abstract void PackKey(ref SliceWriter writer, IFdbTuple items); + public abstract void PackKey(ref SliceWriter writer, ITuple items); public virtual void EncodeKey(ref SliceWriter writer, T1 item1) { - PackKey(ref writer, FdbTuple.Create(item1)); + PackKey(ref writer, STuple.Create(item1)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2) { - PackKey(ref writer, FdbTuple.Create(item1, item2)); + PackKey(ref writer, STuple.Create(item1, item2)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3)); + PackKey(ref writer, STuple.Create(item1, item2, item3)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3, item4)); + PackKey(ref writer, STuple.Create(item1, item2, item3, item4)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3, item4, item5)); + PackKey(ref writer, STuple.Create(item1, item2, item3, item4, item5)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3, item4, item5, item6)); + PackKey(ref writer, STuple.Create(item1, item2, item3, item4, item5, item6)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3, item4, item5, item6, item7)); + PackKey(ref writer, STuple.Create(item1, item2, item3, item4, item5, item6, item7)); } public virtual void EncodeKey(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - PackKey(ref writer, FdbTuple.Create(item1, item2, item3, item4, item5, item6, item7, item8)); + PackKey(ref writer, STuple.Create(item1, item2, item3, item4, item5, item6, item7, item8)); } - public abstract IFdbTuple UnpackKey(Slice packed); + public abstract ITuple UnpackKey(Slice packed); public virtual T DecodeKey(Slice packed) { @@ -72,27 +72,27 @@ public virtual T DecodeKeyLast(Slice packed) return UnpackKey(packed).OfSizeAtLeast(1).Get(-1); } - public virtual FdbTuple DecodeKey(Slice packed) + public virtual STuple DecodeKey(Slice packed) { - return UnpackKey(packed).With((T1 a, T2 b) => FdbTuple.Create(a, b)); + return UnpackKey(packed).With((T1 a, T2 b) => STuple.Create(a, b)); } - public virtual FdbTuple DecodeKey(Slice packed) + public virtual STuple DecodeKey(Slice packed) { - return UnpackKey(packed).With((T1 a, T2 b, T3 c) => FdbTuple.Create(a, b, c)); + return UnpackKey(packed).With((T1 a, T2 b, T3 c) => STuple.Create(a, b, c)); } - public virtual FdbTuple DecodeKey(Slice packed) + public virtual STuple DecodeKey(Slice packed) { - return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d) => FdbTuple.Create(a, b, c, d)); + return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d) => STuple.Create(a, b, c, d)); } - public virtual FdbTuple DecodeKey(Slice packed) + public virtual STuple DecodeKey(Slice packed) { - return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e) => FdbTuple.Create(a, b, c, d, e)); + return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e) => STuple.Create(a, b, c, d, e)); } - public virtual KeyRange ToRange(Slice prefix, IFdbTuple items) + public virtual KeyRange ToRange(Slice prefix, ITuple items) { var writer = new SliceWriter(prefix, 16); PackKey(ref writer, items); @@ -101,42 +101,42 @@ public virtual KeyRange ToRange(Slice prefix, IFdbTuple items) public virtual KeyRange ToKeyRange(Slice prefix, T1 item1) { - return ToRange(prefix, FdbTuple.Create(item1)); + return ToRange(prefix, STuple.Create(item1)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { - return ToRange(prefix, FdbTuple.Create(item1, item2)); + return ToRange(prefix, STuple.Create(item1, item2)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3)); + return ToRange(prefix, STuple.Create(item1, item3, item3)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4)); + return ToRange(prefix, STuple.Create(item1, item3, item3, item4)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5)); + return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6)); + return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7)); + return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7)); } public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - return ToRange(prefix, FdbTuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); + return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } } } \ No newline at end of file diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index ec1a4379f..e04ac2c3b 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -116,21 +116,21 @@ public T DecodeValue(Slice encoded) public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(FdbTuple key, int items); + public abstract Slice EncodeComposite(STuple key, int items); - public abstract FdbTuple DecodeComposite(Slice encoded, int items); + public abstract STuple DecodeComposite(Slice encoded, int items); - public Slice EncodeKey(FdbTuple key) + public Slice EncodeKey(STuple key) { return EncodeComposite(key, 2); } public virtual Slice EncodeKey(T1 item1, T2 item2) { - return EncodeComposite(FdbTuple.Create(item1, item2), 2); + return EncodeComposite(STuple.Create(item1, item2), 2); } - public virtual FdbTuple DecodeKey(Slice encoded) + public virtual STuple DecodeKey(Slice encoded) { return DecodeComposite(encoded, 2); } @@ -151,26 +151,26 @@ public HeadEncoder Head() public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(FdbTuple items, int count); + public abstract Slice EncodeComposite(STuple items, int count); - public abstract FdbTuple DecodeComposite(Slice encoded, int count); + public abstract STuple DecodeComposite(Slice encoded, int count); - public Slice EncodeKey(FdbTuple key) + public Slice EncodeKey(STuple key) { return EncodeComposite(key, 3); } public virtual Slice EncodeKey(T1 item1, T2 item2, T3 item3) { - return EncodeComposite(FdbTuple.Create(item1, item2, item3), 3); + return EncodeComposite(STuple.Create(item1, item2, item3), 3); } - public virtual FdbTuple DecodeKey(Slice encoded) + public virtual STuple DecodeKey(Slice encoded) { return DecodeComposite(encoded, 3); } - public FdbTuple DecodeKey(Slice encoded, int items) + public STuple DecodeKey(Slice encoded, int items) { return DecodeComposite(encoded, items); } @@ -191,26 +191,26 @@ public PairEncoder Pair() public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(FdbTuple items, int count); + public abstract Slice EncodeComposite(STuple items, int count); - public abstract FdbTuple DecodeComposite(Slice encoded, int count); + public abstract STuple DecodeComposite(Slice encoded, int count); - public Slice EncodeKey(FdbTuple key) + public Slice EncodeKey(STuple key) { return EncodeComposite(key, 4); } public virtual Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) { - return EncodeComposite(FdbTuple.Create(item1, item2, item3, item4), 4); + return EncodeComposite(STuple.Create(item1, item2, item3, item4), 4); } - public virtual FdbTuple DecodeKey(Slice encoded) + public virtual STuple DecodeKey(Slice encoded) { return DecodeComposite(encoded, 4); } - public FdbTuple DecodeKey(Slice encoded, int items) + public STuple DecodeKey(Slice encoded, int items) { return DecodeComposite(encoded, items); } @@ -230,7 +230,7 @@ public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) public Slice EncodeKey(T1 value) { - return this.Encoder.EncodeComposite(new FdbTuple(value, default(T2)), 1); + return this.Encoder.EncodeComposite(new STuple(value, default(T2)), 1); } public T1 DecodeKey(Slice encoded) @@ -253,7 +253,7 @@ public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) public Slice EncodeKey(T1 value) { - return this.Encoder.EncodeComposite(new FdbTuple(value, default(T2), default(T3)), 1); + return this.Encoder.EncodeComposite(new STuple(value, default(T2), default(T3)), 1); } public T1 DecodeKey(Slice encoded) @@ -276,7 +276,7 @@ public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) public Slice EncodeKey(T1 value) { - return this.Encoder.EncodeComposite(new FdbTuple(value, default(T2), default(T3), default(T4)), 1); + return this.Encoder.EncodeComposite(new STuple(value, default(T2), default(T3), default(T4)), 1); } public T1 DecodeKey(Slice encoded) @@ -299,26 +299,26 @@ public PairEncoder([NotNull] ICompositeKeyEncoder encoder) public Slice EncodeKey(T1 value1, T2 value2) { - return this.Encoder.EncodeComposite(new FdbTuple(value1, value2, default(T3)), 2); + return this.Encoder.EncodeComposite(new STuple(value1, value2, default(T3)), 2); } - public Slice EncodeComposite(FdbTuple key, int items) + public Slice EncodeComposite(STuple key, int items) { - return this.Encoder.EncodeComposite(new FdbTuple(key.Item1, key.Item2, default(T3)), items); + return this.Encoder.EncodeComposite(new STuple(key.Item1, key.Item2, default(T3)), items); } - public FdbTuple DecodeComposite(Slice encoded, int items) + public STuple DecodeComposite(Slice encoded, int items) { var t = this.Encoder.DecodeComposite(encoded, items); - return new FdbTuple(t.Item1, t.Item2); + return new STuple(t.Item1, t.Item2); } - public Slice EncodeKey(FdbTuple value) + public Slice EncodeKey(STuple value) { return EncodeComposite(value, 2); } - public FdbTuple DecodeKey(Slice encoded) + public STuple DecodeKey(Slice encoded) { return DecodeComposite(encoded, 2); } @@ -343,26 +343,26 @@ public PairEncoder([NotNull] ICompositeKeyEncoder encoder) public Slice EncodeKey(T1 value1, T2 value2) { - return this.Encoder.EncodeComposite(new FdbTuple(value1, value2, default(T3), default(T4)), 2); + return this.Encoder.EncodeComposite(new STuple(value1, value2, default(T3), default(T4)), 2); } - public Slice EncodeComposite(FdbTuple key, int items) + public Slice EncodeComposite(STuple key, int items) { - return this.Encoder.EncodeComposite(new FdbTuple(key.Item1, key.Item2, default(T3), default(T4)), items); + return this.Encoder.EncodeComposite(new STuple(key.Item1, key.Item2, default(T3), default(T4)), items); } - public FdbTuple DecodeComposite(Slice encoded, int items) + public STuple DecodeComposite(Slice encoded, int items) { var t = this.Encoder.DecodeComposite(encoded, items); - return new FdbTuple(t.Item1, t.Item2); + return new STuple(t.Item1, t.Item2); } - public Slice EncodeKey(FdbTuple value) + public Slice EncodeKey(STuple value) { return EncodeComposite(value, 2); } - public FdbTuple DecodeKey(Slice encoded) + public STuple DecodeKey(Slice encoded) { return DecodeComposite(encoded, 2); } @@ -440,7 +440,7 @@ public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec< m_codec2 = codec2; } - public override Slice EncodeComposite(FdbTuple items, int count) + public override Slice EncodeComposite(STuple items, int count) { Contract.Requires(count > 0); @@ -450,7 +450,7 @@ public override Slice EncodeComposite(FdbTuple items, int count) return writer.ToSlice(); } - public override FdbTuple DecodeComposite(Slice encoded, int count) + public override STuple DecodeComposite(Slice encoded, int count) { Contract.Requires(count > 0); @@ -460,7 +460,7 @@ public override FdbTuple DecodeComposite(Slice encoded, int count) if (count >= 1) key1 = m_codec1.DecodeOrderedSelfTerm(ref reader); if (count >= 2) key2 = m_codec2.DecodeOrderedSelfTerm(ref reader); if (reader.HasMore) throw new InvalidOperationException(String.Format("Unexpected data at the end of composite key after {0} items", count)); - return FdbTuple.Create(key1, key2); + return STuple.Create(key1, key2); } } @@ -478,7 +478,7 @@ public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec< m_codec3 = codec3; } - public override Slice EncodeComposite(FdbTuple items, int count) + public override Slice EncodeComposite(STuple items, int count) { Contract.Requires(count > 0 && count <= 3); @@ -489,7 +489,7 @@ public override Slice EncodeComposite(FdbTuple items, int count) return writer.ToSlice(); } - public override FdbTuple DecodeComposite(Slice encoded, int count) + public override STuple DecodeComposite(Slice encoded, int count) { Contract.Requires(count > 0); @@ -501,7 +501,7 @@ public override FdbTuple DecodeComposite(Slice encoded, int count) if (count >= 2) key2 = m_codec2.DecodeOrderedSelfTerm(ref reader); if (count >= 3) key3 = m_codec3.DecodeOrderedSelfTerm(ref reader); if (reader.HasMore) throw new InvalidOperationException(String.Format("Unexpected data at the end of composite key after {0} items", count)); - return FdbTuple.Create(key1, key2, key3); + return STuple.Create(key1, key2, key3); } } @@ -700,24 +700,24 @@ private TupleKeyEncoder() { } public Slice EncodeKey(T key) { - return FdbTuple.EncodeKey(key); + return STuple.EncodeKey(key); } public T DecodeKey(Slice encoded) { if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return FdbTuple.DecodeKey(encoded); + return STuple.DecodeKey(encoded); } public Slice EncodeValue(T key) { - return FdbTuple.EncodeKey(key); + return STuple.EncodeKey(key); } public T DecodeValue(Slice encoded) { if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return FdbTuple.DecodeKey(encoded); + return STuple.DecodeKey(encoded); } } @@ -729,24 +729,24 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder private TupleCompositeEncoder() { } - public override Slice EncodeComposite(FdbTuple key, int items) + public override Slice EncodeComposite(STuple key, int items) { switch (items) { case 2: return key.ToSlice(); - case 1: return FdbTuple.EncodeKey(key.Item1); + case 1: return STuple.EncodeKey(key.Item1); default: throw new ArgumentOutOfRangeException("items", items, "Item count must be either 1 or 2"); } } - public override FdbTuple DecodeComposite(Slice encoded, int items) + public override STuple DecodeComposite(Slice encoded, int items) { if (items < 1 || items > 2) throw new ArgumentOutOfRangeException("items", items, "Item count must be either 1 or 2"); - var t = FdbTuple.Unpack(encoded).OfSize(items); + var t = STuple.Unpack(encoded).OfSize(items); Contract.Assert(t != null); - return FdbTuple.Create( + return STuple.Create( t.Get(0), items >= 2 ? t.Get(1) : default(T2) ); @@ -760,25 +760,25 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder key, int items) + public override Slice EncodeComposite(STuple key, int items) { switch (items) { case 3: return key.ToSlice(); - case 2: return FdbTuple.EncodeKey(key.Item1, key.Item2); - case 1: return FdbTuple.EncodeKey(key.Item1); + case 2: return STuple.EncodeKey(key.Item1, key.Item2); + case 1: return STuple.EncodeKey(key.Item1); default: throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 3"); } } - public override FdbTuple DecodeComposite(Slice encoded, int items) + public override STuple DecodeComposite(Slice encoded, int items) { if (items < 1 || items > 3) throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 3"); - var t = FdbTuple.Unpack(encoded).OfSize(items); + var t = STuple.Unpack(encoded).OfSize(items); Contract.Assert(t != null); - return FdbTuple.Create( + return STuple.Create( t.Get(0), items >= 2 ? t.Get(1) : default(T2), items >= 3 ? t.Get(2) : default(T3) @@ -793,25 +793,25 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder key, int items) + public override Slice EncodeComposite(STuple key, int items) { switch (items) { case 4: return key.ToSlice(); - case 3: return FdbTuple.EncodeKey(key.Item1, key.Item2, key.Item3); - case 2: return FdbTuple.EncodeKey(key.Item1, key.Item2); - case 1: return FdbTuple.EncodeKey(key.Item1); + case 3: return STuple.EncodeKey(key.Item1, key.Item2, key.Item3); + case 2: return STuple.EncodeKey(key.Item1, key.Item2); + case 1: return STuple.EncodeKey(key.Item1); default: throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 4"); } } - public override FdbTuple DecodeComposite(Slice encoded, int items) + public override STuple DecodeComposite(Slice encoded, int items) { if (items < 1 || items > 4) throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 4"); - var t = FdbTuple.Unpack(encoded).OfSize(items); + var t = STuple.Unpack(encoded).OfSize(items); - return FdbTuple.Create( + return STuple.Create( t.Get(0), items >= 2 ? t.Get(1) : default(T2), items >= 3 ? t.Get(2) : default(T3), diff --git a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs index 3f90ad738..3d3f4c6d0 100644 --- a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs @@ -31,23 +31,23 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { Slice EncodeKey(T1 value1, T2 value2, T3 value3, T4 value4); } - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { Slice EncodeKey(T1 value1, T2 value2, T3 value3); } - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { Slice EncodeKey(T1 value1, T2 value2); } public interface ICompositeKeyEncoder : IKeyEncoder - where TTuple : IFdbTuple + where TTuple : ITuple { Slice EncodeComposite(TTuple key, int items); diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs index 2fc38d5f1..3929d8ee2 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs @@ -52,7 +52,7 @@ public interface IDynamicKeyEncoder /// Buffer where to append the binary representation /// Tuple of any size (0 to N) /// If some elements in are not supported by this type system - void PackKey(ref SliceWriter writer, IFdbTuple items); + void PackKey(ref SliceWriter writer, ITuple items); /// Encode a key composed of a single element into a binary slice /// Type of the element @@ -165,7 +165,7 @@ public interface IDynamicKeyEncoder /// Decode a binary slice into a tuple or arbitrary length /// Binary slice produced by a previous call to /// Tuple of any size (0 to N) - IFdbTuple UnpackKey(Slice packed); + ITuple UnpackKey(Slice packed); /// Decode a binary slice containing exactly on element /// Expected type of the element @@ -182,7 +182,7 @@ public interface IDynamicKeyEncoder /// Expected type of the second element /// Binary slice produced by a previous call to or /// Tuple containing two elements, or an exception if the data is invalid, or the tuples has less or more than two elements - FdbTuple DecodeKey(Slice packed); + STuple DecodeKey(Slice packed); /// Decode a binary slice containing exactly three elements /// Expected type of the first element @@ -190,7 +190,7 @@ public interface IDynamicKeyEncoder /// Expected type of the third element /// Binary slice produced by a previous call to or /// Tuple containing three elements, or an exception if the data is invalid, or the tuples has less or more than three elements - FdbTuple DecodeKey(Slice packed); + STuple DecodeKey(Slice packed); /// Decode a binary slice containing exactly four elements /// Expected type of the first element @@ -199,7 +199,7 @@ public interface IDynamicKeyEncoder /// Expected type of the fourth element /// Binary slice produced by a previous call to or /// Tuple containing four elements, or an exception if the data is invalid, or the tuples has less or more than four elements - FdbTuple DecodeKey(Slice packed); + STuple DecodeKey(Slice packed); /// Decode a binary slice containing exactly five elements /// Expected type of the first element @@ -209,7 +209,7 @@ public interface IDynamicKeyEncoder /// Expected type of the fifth element /// Binary slice produced by a previous call to or /// Tuple containing five elements, or an exception if the data is invalid, or the tuples has less or more than five elements - FdbTuple DecodeKey(Slice packed); + STuple DecodeKey(Slice packed); #endregion @@ -218,7 +218,7 @@ public interface IDynamicKeyEncoder /// Return a key range using a tuple as a prefix /// Optional binary prefix that should be added before encoding the key /// Tuple of any size (0 to N) - KeyRange ToRange(Slice prefix, IFdbTuple items); + KeyRange ToRange(Slice prefix, ITuple items); /// Return a key range using a single element as a prefix /// Type of the element diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs index 062edc631..747a8c2e5 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs @@ -38,7 +38,7 @@ namespace FoundationDB.Layers.Documents /// Type of documents public interface IDocumentSplitter { - KeyValuePair[] Split(TDocument document); + KeyValuePair[] Split(TDocument document); } /// Interface that defines a class that knows of to reconstruct instances of from slices @@ -46,7 +46,7 @@ public interface IDocumentSplitter public interface IDocumentBuilder { - TDocument Build(KeyValuePair[] parts); + TDocument Build(KeyValuePair[] parts); } /// Interface that defines a class that knows of to store and retrieve serialized versions of instances into a document collection @@ -79,7 +79,7 @@ public static class FdbDocumentHandlers /// Docuemnt handler that handle dictionarys of string to objects /// /// - public sealed class DictionaryHandler : IDocumentHandler>> + public sealed class DictionaryHandler : IDocumentHandler>> where TDictionary : IDictionary, new() { @@ -93,7 +93,7 @@ public DictionaryHandler(string idName = null, IEqualityComparer compare public string IdName { get; } - public KeyValuePair[] Split(List> document) + public KeyValuePair[] Split(List> document) { if (document == null) throw new ArgumentNullException(nameof(document)); @@ -101,23 +101,23 @@ public KeyValuePair[] Split(List !m_keyComparer.Equals(kvp.Key, this.IdName)) // convert into tuples - .Select(kvp => new KeyValuePair( - FdbTuple.Create(kvp.Key), - FdbTuple.Create(kvp.Value).ToSlice() + .Select(kvp => new KeyValuePair( + STuple.Create(kvp.Key), + STuple.Create(kvp.Value).ToSlice() )) .ToArray(); } - public List> Build(KeyValuePair[] parts) + public List> Build(KeyValuePair[] parts) { if (parts == null) throw new ArgumentNullException(nameof(parts)); - var list = new List>(parts.Length); + var list = new List>(parts.Length); foreach(var part in parts) { - list.Add(new KeyValuePair( + list.Add(new KeyValuePair( part.Key.Last(), - FdbTuple.Unpack(part.Value) + STuple.Unpack(part.Value) )); } return list; @@ -128,28 +128,28 @@ public TId GetId(TDictionary document) return (TId)document[this.IdName]; } - public void SetId(Dictionary document, TId id) + public void SetId(Dictionary document, TId id) { - document[this.IdName] = FdbTuple.Create(id); + document[this.IdName] = STuple.Create(id); } - public List> Pack(TDictionary document) + public List> Pack(TDictionary document) { - var dic = new List>(document.Count); + var dic = new List>(document.Count); // convert everything, except the Id foreach(var kvp in document) { if (!m_keyComparer.Equals(kvp.Key, this.IdName)) { - dic.Add(new KeyValuePair(kvp.Key, FdbTuple.Create(kvp.Key))); + dic.Add(new KeyValuePair(kvp.Key, STuple.Create(kvp.Key))); } } return dic; } - public TDictionary Unpack(List> packed, TId id) + public TDictionary Unpack(List> packed, TId id) { var dic = new TDictionary(); dic.Add(this.IdName, id); diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 97a2286ea..d5fadbaf6 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -57,7 +57,7 @@ public FdbHashSetCollection(IFdbSubspace subspace) /// Returns the key prefix of an HashSet: (subspace, id, ) /// /// - protected virtual Slice GetKey(IFdbTuple id) + protected virtual Slice GetKey(ITuple id) { //REVIEW: should the id be encoded as a an embedded tuple or not? return this.Subspace.Keys.Pack(id); @@ -67,13 +67,13 @@ protected virtual Slice GetKey(IFdbTuple id) /// /// /// - protected virtual Slice GetFieldKey(IFdbTuple id, string field) + protected virtual Slice GetFieldKey(ITuple id, string field) { //REVIEW: should the id be encoded as a an embedded tuple or not? return this.Subspace.Keys.Pack(id.Append(field)); } - protected virtual string ParseFieldKey(IFdbTuple key) + protected virtual string ParseFieldKey(ITuple key) { return key.Last(); } @@ -85,7 +85,7 @@ protected virtual string ParseFieldKey(IFdbTuple key) /// Unique identifier of the hashset /// Name of the field to read /// Value of the corresponding field, or Slice.Nil if it the hashset does not exist, or doesn't have a field with this name - public Task GetValueAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id, string field) + public Task GetValueAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] ITuple id, string field) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -98,7 +98,7 @@ public Task GetValueAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNu /// Transaction that will be used for this request /// Unique identifier of the hashset /// Dictionary containing, for all fields, their associated values - public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id) + public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] ITuple id) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -123,13 +123,13 @@ await trans /// Unique identifier of the hashset /// List of the fields to read /// Dictionary containing the values of the selected fields, or Slice.Empty if that particular field does not exist. - public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IFdbTuple id, [NotNull] params string[] fields) + public async Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] ITuple id, [NotNull] params string[] fields) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); if (fields == null) throw new ArgumentNullException(nameof(fields)); - var keys = FdbTuple.EncodePrefixedKeys(GetKey(id), fields); + var keys = STuple.EncodePrefixedKeys(GetKey(id), fields); var values = await trans.GetValuesAsync(keys).ConfigureAwait(false); Contract.Assert(values != null && values.Length == fields.Length); @@ -146,7 +146,7 @@ public async Task> GetAsync([NotNull] IFdbReadOnlyTra #region Set - public void SetValue(IFdbTransaction trans, IFdbTuple id, string field, Slice value) + public void SetValue(IFdbTransaction trans, ITuple id, string field, Slice value) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -155,7 +155,7 @@ public void SetValue(IFdbTransaction trans, IFdbTuple id, string field, Slice va trans.Set(GetFieldKey(id, field), value); } - public void Set(IFdbTransaction trans, IFdbTuple id, IEnumerable> fields) + public void Set(IFdbTransaction trans, ITuple id, IEnumerable> fields) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -176,7 +176,7 @@ public void Set(IFdbTransaction trans, IFdbTuple id, IEnumerable /// /// - public void DeleteValue(IFdbTransaction trans, IFdbTuple id, string field) + public void DeleteValue(IFdbTransaction trans, ITuple id, string field) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -187,7 +187,7 @@ public void DeleteValue(IFdbTransaction trans, IFdbTuple id, string field) /// Remove all fields of an hashset /// - public void Delete(IFdbTransaction trans, IFdbTuple id) + public void Delete(IFdbTransaction trans, ITuple id) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -200,7 +200,7 @@ public void Delete(IFdbTransaction trans, IFdbTuple id) /// /// /// - public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) + public void Delete(IFdbTransaction trans, ITuple id, params string[] fields) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); @@ -221,7 +221,7 @@ public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) /// Transaction that will be used for this request /// Unique identifier of the hashset /// List of all fields. If the list is empty, the hashset does not exist - public Task> GetKeys(IFdbReadOnlyTransaction trans, IFdbTuple id, CancellationToken cancellationToken = default(CancellationToken)) + public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id, CancellationToken cancellationToken = default(CancellationToken)) { //note: As of Beta2, FDB does not have a fdb_get_range that only return the keys. That means that we will have to also read the values from the db, in order to just get the names of the fields :( //TODO: find a way to optimize this ? @@ -234,7 +234,7 @@ public void Delete(IFdbTransaction trans, IFdbTuple id, params string[] fields) return trans .GetRange(KeyRange.StartsWith(prefix)) - .Select((kvp) => ParseFieldKey(FdbTuple.Unpack(kvp.Key))) + .Select((kvp) => ParseFieldKey(STuple.Unpack(kvp.Key))) .ToListAsync(cancellationToken); } diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index dbfa633a7..116d83b52 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -87,7 +87,7 @@ public static FdbQueryRangeExpression RangeStartsWith(Slice prefix, FdbRangeOpti /// Execute a Range read from the database, and return all the keys and values [NotNull] - public static FdbQueryRangeExpression RangeStartsWith(IFdbTuple tuple, FdbRangeOptions options = null) + public static FdbQueryRangeExpression RangeStartsWith(ITuple tuple, FdbRangeOptions options = null) { return Range(tuple.ToSelectorPair(), options); } diff --git a/FoundationDB.Samples/Benchmarks/BenchRunner.cs b/FoundationDB.Samples/Benchmarks/BenchRunner.cs index aca7925c4..1f5e660e3 100644 --- a/FoundationDB.Samples/Benchmarks/BenchRunner.cs +++ b/FoundationDB.Samples/Benchmarks/BenchRunner.cs @@ -111,7 +111,7 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) } else { - var foos = FdbTuple.EncodePrefixedKeys(foo, Enumerable.Range(1, this.Value).ToArray()); + var foos = STuple.EncodePrefixedKeys(foo, Enumerable.Range(1, this.Value).ToArray()); await db.ReadAsync(tr => tr.GetValuesAsync(foos), ct); } break; diff --git a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs index 94e853aea..a0c1ae9f3 100644 --- a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs +++ b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs @@ -84,7 +84,7 @@ public async Task RunProducer(IFdbDatabase db, CancellationToken ct) while (!ct.IsCancellationRequested) { int k = cnt++; - Slice taskId = FdbTuple.EncodeKey(this.Id.GetHashCode(), k); + Slice taskId = STuple.EncodeKey(this.Id.GetHashCode(), k); string msg = "Message #" + k + " from producer " + this.Id + " (" + DateTime.UtcNow.ToString("O") + ")"; diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index 3802fe6f9..01fc43301 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -44,7 +44,7 @@ protected Slice AttendsKey(string s, string c) protected KeyRange AttendsKeys(string s) { - return this.Subspace.Keys.ToRange(FdbTuple.Create("attends", s)); + return this.Subspace.Keys.ToRange(STuple.Create("attends", s)); } /// @@ -74,7 +74,7 @@ await db.WriteAsync((tr) => /// public Task> AvailableClasses(IFdbReadOnlyTransaction tr) { - return tr.GetRange(this.Subspace.Keys.ToRange(FdbTuple.Create("class"))) + return tr.GetRange(this.Subspace.Keys.ToRange(STuple.Create("class"))) .Where(kvp => { int _; return Int32.TryParse(kvp.Value.ToAscii(), out _); }) // (step 3) .Select(kvp => this.Subspace.Keys.Decode(kvp.Key)) .ToListAsync(); diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index 03fd2055c..e757d32e9 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -180,7 +180,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, FdbSubspace.Create(FdbTuple.EncodeKey(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, FdbSubspace.Create(STuple.EncodeKey(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); @@ -266,8 +266,8 @@ private static async Task HelloWorld(CancellationToken ct) // Writes some data in to the database using (var tr = db.BeginTransaction(ct)) { - tr.Set(FdbTuple.EncodeKey("Test", 123), Slice.FromString("Hello World!")); - tr.Set(FdbTuple.EncodeKey("Test", 456), Slice.FromInt64(DateTime.UtcNow.Ticks)); + tr.Set(STuple.EncodeKey("Test", 123), Slice.FromString("Hello World!")); + tr.Set(STuple.EncodeKey("Test", 456), Slice.FromInt64(DateTime.UtcNow.Ticks)); } } diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index e6e754e8d..e7cc6156a 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -211,7 +211,7 @@ await Fdb.Bulk.WriteAsync( await Fdb.Bulk.ForEachAsync( db, Enumerable.Range(1, N).Select(x => location.Keys.Encode(x)), - () => FdbTuple.Create(0L, 0L), + () => STuple.Create(0L, 0L), async (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -226,7 +226,7 @@ await Fdb.Bulk.ForEachAsync( { sum += results[i].ToInt32(); } - return FdbTuple.Create(state.Item1 + sum, state.Item2 + results.Length); + return STuple.Create(state.Item1 + sum, state.Item2 + results.Length); }, (state) => { @@ -362,7 +362,7 @@ await Fdb.Bulk.WriteAsync( await Fdb.Bulk.ForEachAsync( db, Enumerable.Range(1, N).Select(x => location.Keys.Encode(x)), - () => FdbTuple.Create(0L, 0L), // (sum, count) + () => STuple.Create(0L, 0L), // (sum, count) (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -377,7 +377,7 @@ await Fdb.Bulk.ForEachAsync( { sum += results[i].ToInt32(); } - return FdbTuple.Create( + return STuple.Create( state.Item1 + sum, // updated sum state.Item2 + results.Length // updated count ); @@ -546,7 +546,7 @@ await Fdb.Bulk.WriteAsync( double average = await Fdb.Bulk.AggregateAsync( db, source.Select(x => location.Keys.Encode(x.Key)), - () => FdbTuple.Create(0L, 0L), + () => STuple.Create(0L, 0L), async (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -561,7 +561,7 @@ await Fdb.Bulk.WriteAsync( { sum += results[i].ToInt32(); } - return FdbTuple.Create(state.Item1 + sum, state.Item2 + results.Length); + return STuple.Create(state.Item1 + sum, state.Item2 + results.Length); }, (state) => (double)state.Item1 / state.Item2, this.Cancellation diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index 24265b65f..c4cee30ca 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -270,7 +270,7 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", FdbSubspace.Create(FdbTuple.EncodeKey("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", FdbSubspace.Create(STuple.EncodeKey("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); @@ -280,7 +280,7 @@ public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() Assert.That(subspace.Key.ToString(), Is.EqualTo("<02>test<00><02>hello<00>")); // keys inside the global space are valid - Assert.That(db.IsKeyValid(FdbTuple.EncodeKey("test", 123)), Is.True); + Assert.That(db.IsKeyValid(STuple.EncodeKey("test", 123)), Is.True); // keys outside the global space are invalid Assert.That(db.IsKeyValid(Slice.Create(new byte[] { 42 })), Is.False); diff --git a/FoundationDB.Tests/Encoders/EncoderFacts.cs b/FoundationDB.Tests/Encoders/EncoderFacts.cs index 66cd7a218..ffc7d748b 100644 --- a/FoundationDB.Tests/Encoders/EncoderFacts.cs +++ b/FoundationDB.Tests/Encoders/EncoderFacts.cs @@ -115,7 +115,7 @@ public void Test_Tuple_Composite_Encoder() // note: EncodeKey(...) is just a shortcurt for packing all items in a tuple, and EncodeComposite(..., count = 3) var data = encoder.EncodeKey(x, y, z); - Assert.That(data, Is.EqualTo(FdbTuple.EncodeKey(x, y, z))); + Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y, z))); var items = encoder.DecodeKey(data); Assert.That(items.Item1, Is.EqualTo(x)); @@ -125,15 +125,15 @@ public void Test_Tuple_Composite_Encoder() // partial key encoding data = encoder.EncodeComposite(items, 2); - Assert.That(data, Is.EqualTo(FdbTuple.EncodeKey(x, y))); - items = encoder.DecodeComposite(FdbTuple.EncodeKey(x, y), 2); + Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y))); + items = encoder.DecodeComposite(STuple.EncodeKey(x, y), 2); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(y)); Assert.That(items.Item3, Is.EqualTo(default(Guid))); data = encoder.EncodeComposite(items, 1); - Assert.That(data, Is.EqualTo(FdbTuple.EncodeKey(x))); - items = encoder.DecodeComposite(FdbTuple.EncodeKey(x), 1); + Assert.That(data, Is.EqualTo(STuple.EncodeKey(x))); + items = encoder.DecodeComposite(STuple.EncodeKey(x), 1); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(default(long))); Assert.That(items.Item3, Is.EqualTo(default(Guid))); diff --git a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs index 77f53c905..67f88ca4e 100644 --- a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs +++ b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs @@ -45,31 +45,31 @@ public class TypeCodecFacts [Test] public void Test_Simple_Integer_Codec() { - var codec = FdbTupleCodec.Default; + var codec = TupleCodec.Default; Assert.That(codec, Is.Not.Null); - Assert.That(codec.EncodeOrdered(0), Is.EqualTo(FdbTuple.EncodeKey(0))); - Assert.That(codec.EncodeOrdered(123), Is.EqualTo(FdbTuple.EncodeKey(123))); - Assert.That(codec.EncodeOrdered(123456), Is.EqualTo(FdbTuple.EncodeKey(123456))); + Assert.That(codec.EncodeOrdered(0), Is.EqualTo(STuple.EncodeKey(0))); + Assert.That(codec.EncodeOrdered(123), Is.EqualTo(STuple.EncodeKey(123))); + Assert.That(codec.EncodeOrdered(123456), Is.EqualTo(STuple.EncodeKey(123456))); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey(0)), Is.EqualTo(0)); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey(123)), Is.EqualTo(123)); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey(123456)), Is.EqualTo(123456)); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey(0)), Is.EqualTo(0)); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey(123)), Is.EqualTo(123)); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey(123456)), Is.EqualTo(123456)); } [Test] public void Test_Simple_String_Codec() { - var codec = FdbTupleCodec.Default; + var codec = TupleCodec.Default; Assert.That(codec, Is.Not.Null); - Assert.That(codec.EncodeOrdered("héllø Wörld"), Is.EqualTo(FdbTuple.EncodeKey("héllø Wörld"))); - Assert.That(codec.EncodeOrdered(String.Empty), Is.EqualTo(FdbTuple.EncodeKey(""))); - Assert.That(codec.EncodeOrdered(null), Is.EqualTo(FdbTuple.EncodeKey(default(string)))); + Assert.That(codec.EncodeOrdered("héllø Wörld"), Is.EqualTo(STuple.EncodeKey("héllø Wörld"))); + Assert.That(codec.EncodeOrdered(String.Empty), Is.EqualTo(STuple.EncodeKey(""))); + Assert.That(codec.EncodeOrdered(null), Is.EqualTo(STuple.EncodeKey(default(string)))); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey("héllø Wörld")), Is.EqualTo("héllø Wörld")); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey(String.Empty)), Is.EqualTo("")); - Assert.That(codec.DecodeOrdered(FdbTuple.EncodeKey(default(string))), Is.Null); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey("héllø Wörld")), Is.EqualTo("héllø Wörld")); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey(String.Empty)), Is.EqualTo("")); + Assert.That(codec.DecodeOrdered(STuple.EncodeKey(default(string))), Is.Null); } [Test] @@ -81,16 +81,16 @@ public void Test_Simple_SelfTerms_Codecs() long y = 123; Guid z = Guid.NewGuid(); - var first = FdbTupleCodec.Default; - var second = FdbTupleCodec.Default; - var third = FdbTupleCodec.Default; + var first = TupleCodec.Default; + var second = TupleCodec.Default; + var third = TupleCodec.Default; var writer = SliceWriter.Empty; first.EncodeOrderedSelfTerm(ref writer, x); second.EncodeOrderedSelfTerm(ref writer, y); third.EncodeOrderedSelfTerm(ref writer, z); var data = writer.ToSlice(); - Assert.That(data, Is.EqualTo(FdbTuple.EncodeKey(x, y, z))); + Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y, z))); var reader = new SliceReader(data); Assert.That(first.DecodeOrderedSelfTerm(ref reader), Is.EqualTo(x)); diff --git a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs index d021186b5..033b20c6a 100644 --- a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs +++ b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs @@ -436,7 +436,7 @@ private static void DumpIndex(string label, MemoryIndex index, Console.WriteLine("|:------------------------|-------:|--------:|------:|-------:|-------:|-------:|---------:|--------:|----------:|--------:|" + (heatMaps ? ":-----------------------------------------------------------------------|" : "")); foreach (var kv in index.Values.OrderBy((kv) => orderBy(kv.Key, index.Count(kv.Key)), comparer)) { - var t = FdbTuple.Create(kv.Key); + var t = STuple.Create(kv.Key); var tk = t.ToSlice(); int bits, words, literals, fillers; diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index cc7012231..e526456cd 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -255,11 +255,11 @@ public void Test_KeyRange_Test() Assert.That(range.Test(Slice.FromAscii("Z\x00"), endIncluded: true), Is.EqualTo(AFTER)); Assert.That(range.Test(Slice.FromAscii("\xFF"), endIncluded: true), Is.EqualTo(AFTER)); - range = KeyRange.Create(FdbTuple.EncodeKey("A"), FdbTuple.EncodeKey("Z")); - Assert.That(range.Test(FdbTuple.EncodeKey("@")), Is.EqualTo((BEFORE))); - Assert.That(range.Test(FdbTuple.EncodeKey("A")), Is.EqualTo((INSIDE))); - Assert.That(range.Test(FdbTuple.EncodeKey("Z")), Is.EqualTo((AFTER))); - Assert.That(range.Test(FdbTuple.EncodeKey("Z"), endIncluded: true), Is.EqualTo(INSIDE)); + range = KeyRange.Create(STuple.EncodeKey("A"), STuple.EncodeKey("Z")); + Assert.That(range.Test(STuple.EncodeKey("@")), Is.EqualTo((BEFORE))); + Assert.That(range.Test(STuple.EncodeKey("A")), Is.EqualTo((INSIDE))); + Assert.That(range.Test(STuple.EncodeKey("Z")), Is.EqualTo((AFTER))); + Assert.That(range.Test(STuple.EncodeKey("Z"), endIncluded: true), Is.EqualTo(INSIDE)); } [Test] @@ -344,36 +344,36 @@ public void Test_FdbKey_PrettyPrint() // tuples should be decoded properly - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(123)), Is.EqualTo("(123,)"), "Singleton tuples should end with a ','"); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(Slice.FromAscii("hello"))), Is.EqualTo("('hello',)"), "ASCII strings should use single quotes"); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey("héllø")), Is.EqualTo("(\"héllø\",)"), "Unicode strings should use double quotes"); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(Slice.Create(new byte[] { 1, 2, 3 }))), Is.EqualTo("(<01 02 03>,)")); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(123, 456)), Is.EqualTo("(123, 456)"), "Elements should be separated with a space, and not end up with ','"); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(true, false, default(object))), Is.EqualTo("(1, 0, null)"), "Booleans should be displayed as numbers, and null should be in lowercase"); //note: even though it's tempting to using Python's "Nil", it's not very ".NETty" - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(1.0d, Math.PI, Math.E)), Is.EqualTo("(1, 3.1415926535897931, 2.7182818284590451)"), "Doubles should used dot and have full precision (17 digits)"); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(1.0f, (float)Math.PI, (float)Math.E)), Is.EqualTo("(1, 3.14159274, 2.71828175)"), "Singles should used dot and have full precision (10 digits)"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(123)), Is.EqualTo("(123,)"), "Singleton tuples should end with a ','"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(Slice.FromAscii("hello"))), Is.EqualTo("('hello',)"), "ASCII strings should use single quotes"); + Assert.That(FdbKey.Dump(STuple.EncodeKey("héllø")), Is.EqualTo("(\"héllø\",)"), "Unicode strings should use double quotes"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(Slice.Create(new byte[] { 1, 2, 3 }))), Is.EqualTo("(<01 02 03>,)")); + Assert.That(FdbKey.Dump(STuple.EncodeKey(123, 456)), Is.EqualTo("(123, 456)"), "Elements should be separated with a space, and not end up with ','"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(true, false, default(object))), Is.EqualTo("(1, 0, null)"), "Booleans should be displayed as numbers, and null should be in lowercase"); //note: even though it's tempting to using Python's "Nil", it's not very ".NETty" + Assert.That(FdbKey.Dump(STuple.EncodeKey(1.0d, Math.PI, Math.E)), Is.EqualTo("(1, 3.1415926535897931, 2.7182818284590451)"), "Doubles should used dot and have full precision (17 digits)"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(1.0f, (float)Math.PI, (float)Math.E)), Is.EqualTo("(1, 3.14159274, 2.71828175)"), "Singles should used dot and have full precision (10 digits)"); var guid = Guid.NewGuid(); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {...}, and without quotes"); var uuid128 = Uuid128.NewUuid(); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {...}, and without quotes"); var uuid64 = Uuid64.NewUuid(); - Assert.That(FdbKey.Dump(FdbTuple.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(STuple.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {...}, and without quotes"); // ranges should be decoded when possible - var key = FdbTuple.ToRange(FdbTuple.Create("hello")); + var key = STuple.ToRange(STuple.Create("hello")); // "<02>hello<00><00>" .. "<02>hello<00>" Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",).<00>")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",).")); - key = KeyRange.StartsWith(FdbTuple.EncodeKey("hello")); + key = KeyRange.StartsWith(STuple.EncodeKey("hello")); // "<02>hello<00>" .. "<02>hello<01>" Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",)")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",) + 1")); - var t = FdbTuple.EncodeKey(123); + var t = STuple.EncodeKey(123); Assert.That(FdbKey.PrettyPrint(t, FdbKey.PrettyPrintMode.Single), Is.EqualTo("(123,)")); - Assert.That(FdbKey.PrettyPrint(FdbTuple.ToRange(t).Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(123,).<00>")); - Assert.That(FdbKey.PrettyPrint(FdbTuple.ToRange(t).End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(123,).")); + Assert.That(FdbKey.PrettyPrint(STuple.ToRange(t).Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(123,).<00>")); + Assert.That(FdbKey.PrettyPrint(STuple.ToRange(t).End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(123,).")); } diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 8c47a0c11..16a925943 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -537,7 +537,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() Assert.That(folder2, Is.Not.Null); Assert.That(folder2.Layer.ToUnicode(), Is.EqualTo("bar")); Assert.That(folder2.FullName, Is.EqualTo("Test")); - Assert.That(folder2.Path, Is.EqualTo(FdbTuple.Create("Test"))); + Assert.That(folder2.Path, Is.EqualTo(STuple.Create("Test"))); Assert.That(folder2.Key, Is.EqualTo(folder.Key)); // opening the directory with the new layer should succeed @@ -899,7 +899,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => partition.ToRange()); shouldFail(() => partition.ToRange(Slice.FromString("hello"))); - shouldFail(() => partition.ToRange(FdbTuple.EncodeKey("hello"))); + shouldFail(() => partition.ToRange(STuple.EncodeKey("hello"))); // Tuples @@ -915,13 +915,13 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => partition.Keys.EncodeMany((IEnumerable)new object[] { 123, "hello", true })); shouldFail(() => partition.Keys.Unpack(barKey)); - shouldFail(() => partition.Keys.UnpackMany(new[] { barKey, barKey + FdbTuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.UnpackMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); shouldFail(() => partition.Keys.Decode(barKey)); shouldFail(() => partition.Keys.DecodeMany(new[] { barKey, barKey })); shouldFail(() => partition.Keys.DecodeLast(barKey)); - shouldFail(() => partition.Keys.DecodeLastMany(new[] { barKey, barKey + FdbTuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.DecodeLastMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); shouldFail(() => partition.Keys.DecodeFirst(barKey)); - shouldFail(() => partition.Keys.DecodeFirstMany(new[] { barKey, barKey + FdbTuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.DecodeFirstMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); //FIXME: need to re-enable this code! #if REFACTORING_IN_PROGRESS @@ -931,13 +931,13 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => partition.Keys.Append(123, "hello")); shouldFail(() => partition.Keys.Append(123, "hello", false)); shouldFail(() => partition.Keys.Append(123, "hello", false, "world")); - shouldFail(() => partition.Keys.Concat(FdbTuple.Create(123, "hello", false, "world"))); + shouldFail(() => partition.Keys.Concat(STuple.Create(123, "hello", false, "world"))); shouldFail(() => partition.Keys.Append(new object[] { 123, "hello", false, "world" })); #endif shouldFail(() => partition.Keys.ToRange()); shouldFail(() => partition.ToRange(Slice.FromString("hello"))); - shouldFail(() => partition.Keys.ToRange(FdbTuple.Create("hello"))); + shouldFail(() => partition.Keys.ToRange(STuple.Create("hello"))); } } diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index 6f01111bb..430843a94 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -168,11 +168,11 @@ public async Task Test_FdbMap_With_Custom_Key_Encoder() // Encode IPEndPoint as the (IP, Port,) encoded with the Tuple codec // note: there is a much simpler way or creating composite keys, this is just a quick and dirty test! var keyEncoder = KeyValueEncoders.Bind( - (ipe) => ipe == null ? Slice.Empty : FdbTuple.EncodeKey(ipe.Address, ipe.Port), + (ipe) => ipe == null ? Slice.Empty : STuple.EncodeKey(ipe.Address, ipe.Port), (packed) => { if (packed.IsNullOrEmpty) return default(IPEndPoint); - var t = FdbTuple.Unpack(packed); + var t = STuple.Unpack(packed); return new IPEndPoint(t.Get(0), t.Get(1)); } ); diff --git a/FoundationDB.Tests/Layers/RankedSetFacts.cs b/FoundationDB.Tests/Layers/RankedSetFacts.cs index 397f127de..38d22a8c4 100644 --- a/FoundationDB.Tests/Layers/RankedSetFacts.cs +++ b/FoundationDB.Tests/Layers/RankedSetFacts.cs @@ -65,7 +65,7 @@ await db.ReadWriteAsync(async (tr) => for (int i = 0; i < 100; i++) { Console.Write("\rInserting " + i); - await db.ReadWriteAsync((tr) => vector.InsertAsync(tr, FdbTuple.EncodeKey(rnd.Next())), this.Cancellation); + await db.ReadWriteAsync((tr) => vector.InsertAsync(tr, STuple.EncodeKey(rnd.Next())), this.Cancellation); } sw.Stop(); Console.WriteLine("\rDone in {0:N3} sec", sw.Elapsed.TotalSeconds); diff --git a/FoundationDB.Tests/Layers/TupleFacts.cs b/FoundationDB.Tests/Layers/TupleFacts.cs index 3a189739a..610fc9004 100644 --- a/FoundationDB.Tests/Layers/TupleFacts.cs +++ b/FoundationDB.Tests/Layers/TupleFacts.cs @@ -47,15 +47,15 @@ public class TupleFacts : FdbTest #region General Use... [Test] - public void Test_FdbTuple_Create() + public void Test_STuple_Create() { - var t1 = FdbTuple.Create("hello world"); + var t1 = STuple.Create("hello world"); Assert.That(t1.Count, Is.EqualTo(1)); Assert.That(t1.Item1, Is.EqualTo("hello world")); Assert.That(t1.Get(0), Is.EqualTo("hello world")); Assert.That(t1[0], Is.EqualTo("hello world")); - var t2 = FdbTuple.Create("hello world", 123); + var t2 = STuple.Create("hello world", 123); Assert.That(t2.Count, Is.EqualTo(2)); Assert.That(t2.Item1, Is.EqualTo("hello world")); Assert.That(t2.Item2, Is.EqualTo(123)); @@ -64,7 +64,7 @@ public void Test_FdbTuple_Create() Assert.That(t2[0], Is.EqualTo("hello world")); Assert.That(t2[1], Is.EqualTo(123)); - var t3 = FdbTuple.Create("hello world", 123, false); + var t3 = STuple.Create("hello world", 123, false); Assert.That(t3.Count, Is.EqualTo(3)); Assert.That(t3.Item1, Is.EqualTo("hello world")); Assert.That(t3.Item2, Is.EqualTo(123)); @@ -76,7 +76,7 @@ public void Test_FdbTuple_Create() Assert.That(t3[1], Is.EqualTo(123)); Assert.That(t3[2], Is.EqualTo(false)); - var t4 = FdbTuple.Create("hello world", 123, false, 1234L); + var t4 = STuple.Create("hello world", 123, false, 1234L); Assert.That(t4.Count, Is.EqualTo(4)); Assert.That(t4.Item1, Is.EqualTo("hello world")); Assert.That(t4.Item2, Is.EqualTo(123)); @@ -91,7 +91,7 @@ public void Test_FdbTuple_Create() Assert.That(t4[2], Is.EqualTo(false)); Assert.That(t4[3], Is.EqualTo(1234L)); - var t5 = FdbTuple.Create("hello world", 123, false, 1234L, -1234); + var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); Assert.That(t5.Count, Is.EqualTo(5)); Assert.That(t5.Item1, Is.EqualTo("hello world")); Assert.That(t5.Item2, Is.EqualTo(123)); @@ -109,7 +109,7 @@ public void Test_FdbTuple_Create() Assert.That(t5[3], Is.EqualTo(1234L)); Assert.That(t5[4], Is.EqualTo(-1234)); - var tn = FdbTuple.Create(new object[] { "hello world", 123, false, 1234L, -1234, "six" }); + var tn = STuple.Create(new object[] { "hello world", 123, false, 1234L, -1234, "six" }); Assert.That(tn.Count, Is.EqualTo(6)); Assert.That(tn.Get(0), Is.EqualTo("hello world")); Assert.That(tn.Get(1), Is.EqualTo(123)); @@ -120,13 +120,13 @@ public void Test_FdbTuple_Create() } [Test] - public void Test_FdbTuple_Wrap() + public void Test_STuple_Wrap() { - // FdbTuple.Wrap(...) does not copy the items of the array + // STuple.Wrap(...) does not copy the items of the array var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; - var t = FdbTuple.Wrap(arr); + var t = STuple.Wrap(arr); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(4)); Assert.That(t[0], Is.EqualTo("Hello")); @@ -134,7 +134,7 @@ public void Test_FdbTuple_Wrap() Assert.That(t[2], Is.EqualTo(false)); Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); - t = FdbTuple.Wrap(arr, 1, 2); + t = STuple.Wrap(arr, 1, 2); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(2)); Assert.That(t[0], Is.EqualTo(123)); @@ -152,13 +152,13 @@ public void Test_FdbTuple_Wrap() } [Test] - public void Test_FdbTuple_FromObjects() + public void Test_STuple_FromObjects() { - // FdbTuple.FromObjects(...) does a copy of the items of the array + // STuple.FromObjects(...) does a copy of the items of the array var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; - var t = FdbTuple.FromObjects(arr); + var t = STuple.FromObjects(arr); Log("t = {0}", t); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(4)); @@ -167,7 +167,7 @@ public void Test_FdbTuple_FromObjects() Assert.That(t[2], Is.EqualTo(false)); Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); - t = FdbTuple.FromObjects(arr, 1, 2); + t = STuple.FromObjects(arr, 1, 2); Log("t = {0}", t); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(2)); @@ -185,11 +185,11 @@ public void Test_FdbTuple_FromObjects() } [Test] - public void Test_FdbTuple_FromArray() + public void Test_STuple_FromArray() { var items = new string[] { "Bonjour", "le", "Monde" }; - var t = FdbTuple.FromArray(items); + var t = STuple.FromArray(items); Log("t = {0}", t); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(3)); @@ -197,7 +197,7 @@ public void Test_FdbTuple_FromArray() Assert.That(t[1], Is.EqualTo("le")); Assert.That(t[2], Is.EqualTo("Monde")); - t = FdbTuple.FromArray(items, 1, 2); + t = STuple.FromArray(items, 1, 2); Log("t = {0}", t); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(2)); @@ -212,19 +212,19 @@ public void Test_FdbTuple_FromArray() } [Test] - public void Test_FdbTuple_Negative_Indexing() + public void Test_STuple_Negative_Indexing() { - var t1 = FdbTuple.Create("hello world"); + var t1 = STuple.Create("hello world"); Assert.That(t1.Get(-1), Is.EqualTo("hello world")); Assert.That(t1[-1], Is.EqualTo("hello world")); - var t2 = FdbTuple.Create("hello world", 123); + var t2 = STuple.Create("hello world", 123); Assert.That(t2.Get(-1), Is.EqualTo(123)); Assert.That(t2.Get(-2), Is.EqualTo("hello world")); Assert.That(t2[-1], Is.EqualTo(123)); Assert.That(t2[-2], Is.EqualTo("hello world")); - var t3 = FdbTuple.Create("hello world", 123, false); + var t3 = STuple.Create("hello world", 123, false); Assert.That(t3.Get(-1), Is.EqualTo(false)); Assert.That(t3.Get(-2), Is.EqualTo(123)); Assert.That(t3.Get(-3), Is.EqualTo("hello world")); @@ -232,7 +232,7 @@ public void Test_FdbTuple_Negative_Indexing() Assert.That(t3[-2], Is.EqualTo(123)); Assert.That(t3[-3], Is.EqualTo("hello world")); - var t4 = FdbTuple.Create("hello world", 123, false, 1234L); + var t4 = STuple.Create("hello world", 123, false, 1234L); Assert.That(t4.Get(-1), Is.EqualTo(1234L)); Assert.That(t4.Get(-2), Is.EqualTo(false)); Assert.That(t4.Get(-3), Is.EqualTo(123)); @@ -242,7 +242,7 @@ public void Test_FdbTuple_Negative_Indexing() Assert.That(t4[-3], Is.EqualTo(123)); Assert.That(t4[-4], Is.EqualTo("hello world")); - var t5 = FdbTuple.Create("hello world", 123, false, 1234L, -1234); + var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); Assert.That(t5.Get(-1), Is.EqualTo(-1234)); Assert.That(t5.Get(-2), Is.EqualTo(1234L)); Assert.That(t5.Get(-3), Is.EqualTo(false)); @@ -254,7 +254,7 @@ public void Test_FdbTuple_Negative_Indexing() Assert.That(t5[-4], Is.EqualTo(123)); Assert.That(t5[-5], Is.EqualTo("hello world")); - var tn = FdbTuple.Create(new object[] { "hello world", 123, false, 1234, -1234, "six" }); + var tn = STuple.Create(new object[] { "hello world", 123, false, 1234, -1234, "six" }); Assert.That(tn.Get(-1), Is.EqualTo("six")); Assert.That(tn.Get(-2), Is.EqualTo(-1234)); Assert.That(tn.Get(-3), Is.EqualTo(1234)); @@ -270,182 +270,182 @@ public void Test_FdbTuple_Negative_Indexing() } [Test] - public void Test_FdbTuple_First_And_Last() + public void Test_STuple_First_And_Last() { // tuple.First() should be equivalent to tuple.Get(0) // tuple.Last() should be equivalent to tuple.Get(-1) - var t1 = FdbTuple.Create(1); + var t1 = STuple.Create(1); Assert.That(t1.First(), Is.EqualTo(1)); Assert.That(t1.First(), Is.EqualTo("1")); - Assert.That(((IFdbTuple)t1).Last(), Is.EqualTo(1)); - Assert.That(((IFdbTuple)t1).Last(), Is.EqualTo("1")); + Assert.That(((ITuple)t1).Last(), Is.EqualTo(1)); + Assert.That(((ITuple)t1).Last(), Is.EqualTo("1")); - var t2 = FdbTuple.Create(1, 2); + var t2 = STuple.Create(1, 2); Assert.That(t2.First(), Is.EqualTo(1)); Assert.That(t2.First(), Is.EqualTo("1")); Assert.That(t2.Last, Is.EqualTo(2)); - Assert.That(((IFdbTuple)t2).Last(), Is.EqualTo(2)); - Assert.That(((IFdbTuple)t2).Last(), Is.EqualTo("2")); + Assert.That(((ITuple)t2).Last(), Is.EqualTo(2)); + Assert.That(((ITuple)t2).Last(), Is.EqualTo("2")); - var t3 = FdbTuple.Create(1, 2, 3); + var t3 = STuple.Create(1, 2, 3); Assert.That(t3.First(), Is.EqualTo(1)); Assert.That(t3.First(), Is.EqualTo("1")); Assert.That(t3.Last, Is.EqualTo(3)); - Assert.That(((IFdbTuple)t3).Last(), Is.EqualTo(3)); - Assert.That(((IFdbTuple)t3).Last(), Is.EqualTo("3")); + Assert.That(((ITuple)t3).Last(), Is.EqualTo(3)); + Assert.That(((ITuple)t3).Last(), Is.EqualTo("3")); - var t4 = FdbTuple.Create(1, 2, 3, 4); + var t4 = STuple.Create(1, 2, 3, 4); Assert.That(t4.First(), Is.EqualTo(1)); Assert.That(t4.First(), Is.EqualTo("1")); Assert.That(t4.Last, Is.EqualTo(4)); - Assert.That(((IFdbTuple)t4).Last(), Is.EqualTo(4)); - Assert.That(((IFdbTuple)t4).Last(), Is.EqualTo("4")); + Assert.That(((ITuple)t4).Last(), Is.EqualTo(4)); + Assert.That(((ITuple)t4).Last(), Is.EqualTo("4")); - var t5 = FdbTuple.Create(1, 2, 3, 4, 5); + var t5 = STuple.Create(1, 2, 3, 4, 5); Assert.That(t5.First(), Is.EqualTo(1)); Assert.That(t5.First(), Is.EqualTo("1")); Assert.That(t5.Last, Is.EqualTo(5)); - Assert.That(((IFdbTuple)t5).Last(), Is.EqualTo(5)); - Assert.That(((IFdbTuple)t5).Last(), Is.EqualTo("5")); + Assert.That(((ITuple)t5).Last(), Is.EqualTo(5)); + Assert.That(((ITuple)t5).Last(), Is.EqualTo("5")); - var tn = FdbTuple.Create(1, 2, 3, 4, 5, 6); + var tn = STuple.Create(1, 2, 3, 4, 5, 6); Assert.That(tn.First(), Is.EqualTo(1)); Assert.That(tn.First(), Is.EqualTo("1")); Assert.That(tn.Last(), Is.EqualTo(6)); Assert.That(tn.Last(), Is.EqualTo("6")); - Assert.That(() => FdbTuple.Empty.First(), Throws.InstanceOf()); - Assert.That(() => FdbTuple.Empty.Last(), Throws.InstanceOf()); + Assert.That(() => STuple.Empty.First(), Throws.InstanceOf()); + Assert.That(() => STuple.Empty.Last(), Throws.InstanceOf()); } [Test] - public void Test_FdbTuple_Unpack_First_And_Last() + public void Test_STuple_Unpack_First_And_Last() { // should only work with tuples having at least one element Slice packed; - packed = FdbTuple.EncodeKey(1); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("1")); + packed = STuple.EncodeKey(1); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("1")); - packed = FdbTuple.EncodeKey(1, 2); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(2)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("2")); + packed = STuple.EncodeKey(1, 2); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(2)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("2")); - packed = FdbTuple.EncodeKey(1, 2, 3); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(3)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("3")); + packed = STuple.EncodeKey(1, 2, 3); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(3)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("3")); - packed = FdbTuple.EncodeKey(1, 2, 3, 4); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(4)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("4")); + packed = STuple.EncodeKey(1, 2, 3, 4); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(4)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("4")); - packed = FdbTuple.EncodeKey(1, 2, 3, 4, 5); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(5)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("5")); + packed = STuple.EncodeKey(1, 2, 3, 4, 5); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(5)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("5")); - packed = FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(6)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("6")); + packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(6)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("6")); - packed = FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6, 7); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(7)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("7")); + packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(7)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("7")); - packed = FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo(8)); - Assert.That(FdbTuple.DecodeLast(packed), Is.EqualTo("8")); + packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo(8)); + Assert.That(STuple.DecodeLast(packed), Is.EqualTo("8")); - Assert.That(() => FdbTuple.DecodeFirst(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeFirst(Slice.Empty), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeLast(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeLast(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeFirst(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeFirst(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeLast(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeLast(Slice.Empty), Throws.InstanceOf()); } [Test] - public void Test_FdbTuple_UnpackSingle() + public void Test_STuple_UnpackSingle() { // should only work with tuples having exactly one element Slice packed; - packed = FdbTuple.EncodeKey(1); - Assert.That(FdbTuple.DecodeKey(packed), Is.EqualTo(1)); - Assert.That(FdbTuple.DecodeKey(packed), Is.EqualTo("1")); + packed = STuple.EncodeKey(1); + Assert.That(STuple.DecodeKey(packed), Is.EqualTo(1)); + Assert.That(STuple.DecodeKey(packed), Is.EqualTo("1")); - packed = FdbTuple.EncodeKey("Hello\0World"); - Assert.That(FdbTuple.DecodeKey(packed), Is.EqualTo("Hello\0World")); + packed = STuple.EncodeKey("Hello\0World"); + Assert.That(STuple.DecodeKey(packed), Is.EqualTo("Hello\0World")); - Assert.That(() => FdbTuple.DecodeKey(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(Slice.Empty), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3, 4)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3, 4, 5)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6, 7)), Throws.InstanceOf()); - Assert.That(() => FdbTuple.DecodeKey(FdbTuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7)), Throws.InstanceOf()); + Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8)), Throws.InstanceOf()); } [Test] - public void Test_FdbTuple_Embedded_Tuples() + public void Test_STuple_Embedded_Tuples() { // (A,B).Append((C,D)) should return (A,B,(C,D)) (length 3) and not (A,B,C,D) (length 4) - FdbTuple x = FdbTuple.Create("A", "B"); - FdbTuple y = FdbTuple.Create("C", "D"); + STuple x = STuple.Create("A", "B"); + STuple y = STuple.Create("C", "D"); - // using the instance method that returns a FdbTuple - IFdbTuple z = x.Append(y); + // using the instance method that returns an STuple + ITuple z = x.Append(y); Log(z); Assert.That(z, Is.Not.Null); Assert.That(z.Count, Is.EqualTo(3)); Assert.That(z[0], Is.EqualTo("A")); Assert.That(z[1], Is.EqualTo("B")); Assert.That(z[2], Is.EqualTo(y)); - var t = z.Get(2); + var t = z.Get(2); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(2)); Assert.That(t[0], Is.EqualTo("C")); Assert.That(t[1], Is.EqualTo("D")); - // casted down to the interface IFdbTuple - z = ((IFdbTuple)x).Append((IFdbTuple)y); + // casted down to the interface ITuple + z = ((ITuple)x).Append((ITuple)y); Log(z); Assert.That(z, Is.Not.Null); Assert.That(z.Count, Is.EqualTo(3)); Assert.That(z[0], Is.EqualTo("A")); Assert.That(z[1], Is.EqualTo("B")); Assert.That(z[2], Is.EqualTo(y)); - t = z.Get(2); + t = z.Get(2); Assert.That(t, Is.Not.Null); Assert.That(t.Count, Is.EqualTo(2)); Assert.That(t[0], Is.EqualTo("C")); Assert.That(t[1], Is.EqualTo("D")); // composite index key "(prefix, value, id)" - IFdbTuple subspace = FdbTuple.Create(123, 42); - IFdbTuple value = FdbTuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) + ITuple subspace = STuple.Create(123, 42); + ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) string id = "Doc123"; z = subspace.Append(value, id); Log(z); @@ -453,14 +453,14 @@ public void Test_FdbTuple_Embedded_Tuples() } [Test] - public void Test_FdbTuple_With() + public void Test_STuple_With() { - //note: important to always cast to (IFdbTuple) to be sure that we don't call specialized instance methods (tested elsewhere) - IFdbTuple t; + //note: important to always cast to (ITuple) to be sure that we don't call specialized instance methods (tested elsewhere) + ITuple t; // Size 1 - t = FdbTuple.Create(123); + t = STuple.Create(123); t.With((int a) => { Assert.That(a, Is.EqualTo(123)); @@ -621,11 +621,11 @@ public void Test_FdbTuple_With() } [Test] - public void Test_FdbTuple_With_Struct() + public void Test_STuple_With_Struct() { // calling With() on the structs is faster - FdbTuple t1 = FdbTuple.Create(123); + STuple t1 = STuple.Create(123); t1.With((a) => { Assert.That(a, Is.EqualTo(123)); @@ -636,7 +636,7 @@ public void Test_FdbTuple_With_Struct() return 42; }), Is.EqualTo(42)); - FdbTuple t2 = FdbTuple.Create(123, "abc"); + STuple t2 = STuple.Create(123, "abc"); t2.With((a, b) => { Assert.That(a, Is.EqualTo(123)); @@ -649,7 +649,7 @@ public void Test_FdbTuple_With_Struct() return 42; }), Is.EqualTo(42)); - FdbTuple t3 = FdbTuple.Create(123, "abc", 3.14f); + STuple t3 = STuple.Create(123, "abc", 3.14f); t3.With((a, b, c) => { Assert.That(a, Is.EqualTo(123)); @@ -664,7 +664,7 @@ public void Test_FdbTuple_With_Struct() return 42; }), Is.EqualTo(42)); - FdbTuple t4 = FdbTuple.Create(123, "abc", 3.14f, true); + STuple t4 = STuple.Create(123, "abc", 3.14f, true); t4.With((a, b, c, d) => { Assert.That(a, Is.EqualTo(123)); @@ -681,7 +681,7 @@ public void Test_FdbTuple_With_Struct() return 42; }), Is.EqualTo(42)); - FdbTuple t5 = FdbTuple.Create(123, "abc", 3.14f, true, 'z'); + STuple t5 = STuple.Create(123, "abc", 3.14f, true, 'z'); t5.With((a, b, c, d, e) => { Assert.That(a, Is.EqualTo(123)); @@ -704,13 +704,13 @@ public void Test_FdbTuple_With_Struct() } [Test] - public void Test_FdbTuple_Of_Size() + public void Test_STuple_Of_Size() { // OfSize(n) check the size and return the tuple if it passed // VerifySize(n) only check the size // Both should throw if tuple is null, or not the expected size - Action verify = (t) => + Action verify = (t) => { for (int i = 0; i <= 10; i++) { @@ -735,29 +735,29 @@ public void Test_FdbTuple_Of_Size() } }; - verify(FdbTuple.Empty); - verify(FdbTuple.Create(123)); - verify(FdbTuple.Create(123, "abc")); - verify(FdbTuple.Create(123, "abc", 3.14f)); - verify(FdbTuple.Create(123, "abc", 3.14f, true)); - verify(FdbTuple.Create(123, "abc", 3.14f, true, 'z')); - verify(FdbTuple.FromArray(new[] { "hello", "world", "!" })); - verify(FdbTuple.FromEnumerable(Enumerable.Range(0, 10))); + verify(STuple.Empty); + verify(STuple.Create(123)); + verify(STuple.Create(123, "abc")); + verify(STuple.Create(123, "abc", 3.14f)); + verify(STuple.Create(123, "abc", 3.14f, true)); + verify(STuple.Create(123, "abc", 3.14f, true, 'z')); + verify(STuple.FromArray(new[] { "hello", "world", "!" })); + verify(STuple.FromEnumerable(Enumerable.Range(0, 10))); - verify(FdbTuple.Create(123, "abc", 3.14f, true, 'z')[0, 2]); - verify(FdbTuple.Create(123, "abc", 3.14f, true, 'z')[1, 4]); - verify(FdbTuple.FromEnumerable(Enumerable.Range(0, 50)).Substring(15, 6)); + verify(STuple.Create(123, "abc", 3.14f, true, 'z')[0, 2]); + verify(STuple.Create(123, "abc", 3.14f, true, 'z')[1, 4]); + verify(STuple.FromEnumerable(Enumerable.Range(0, 50)).Substring(15, 6)); - IFdbTuple none = null; + ITuple none = null; Assert.That(() => none.OfSize(0), Throws.InstanceOf()); Assert.That(() => none.OfSizeAtLeast(0), Throws.InstanceOf()); Assert.That(() => none.OfSizeAtMost(0), Throws.InstanceOf()); } [Test] - public void Test_FdbTuple_Truncate() + public void Test_STuple_Truncate() { - IFdbTuple t = FdbTuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + ITuple t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); var head = t.Truncate(1); Assert.That(head, Is.Not.Null); @@ -787,37 +787,37 @@ public void Test_FdbTuple_Truncate() tail = t.Truncate(-5); Assert.That(tail, Is.EqualTo(t)); - Assert.That(t.Truncate(0), Is.EqualTo(FdbTuple.Empty)); + Assert.That(t.Truncate(0), Is.EqualTo(STuple.Empty)); Assert.That(() => t.Truncate(6), Throws.InstanceOf()); Assert.That(() => t.Truncate(-6), Throws.InstanceOf()); - Assert.That(() => FdbTuple.Empty.Truncate(1), Throws.InstanceOf()); - Assert.That(() => FdbTuple.Create("Hello", "World").Truncate(3), Throws.InstanceOf()); - Assert.That(() => FdbTuple.Create("Hello", "World").Truncate(-3), Throws.InstanceOf()); + Assert.That(() => STuple.Empty.Truncate(1), Throws.InstanceOf()); + Assert.That(() => STuple.Create("Hello", "World").Truncate(3), Throws.InstanceOf()); + Assert.That(() => STuple.Create("Hello", "World").Truncate(-3), Throws.InstanceOf()); } [Test] - public void Test_FdbTuple_As() + public void Test_STuple_As() { - // IFdbTuple.As<...>() adds types to an untyped IFdbTuple - IFdbTuple t; + // ITuple.As<...>() adds types to an untyped ITuple + ITuple t; - t = FdbTuple.Create("Hello"); - var t1 = t.As(); + t = STuple.Create("Hello"); + var t1 = t.As(); Assert.That(t1.Item1, Is.EqualTo("Hello")); - t = FdbTuple.Create("Hello", 123); - var t2 = t.As(); + t = STuple.Create("Hello", 123); + var t2 = t.As(); Assert.That(t2.Item1, Is.EqualTo("Hello")); Assert.That(t2.Item2, Is.EqualTo(123)); - t = FdbTuple.Create("Hello", 123, false); - var t3 = t.As(); + t = STuple.Create("Hello", 123, false); + var t3 = t.As(); Assert.That(t3.Item1, Is.EqualTo("Hello")); Assert.That(t3.Item2, Is.EqualTo(123)); Assert.That(t3.Item3, Is.EqualTo(false)); - var t4 = FdbTuple + var t4 = STuple .Create("Hello", 123, false, TimeSpan.FromSeconds(5)) .As(); Assert.That(t4.Item1, Is.EqualTo("Hello")); @@ -825,7 +825,7 @@ public void Test_FdbTuple_As() Assert.That(t4.Item3, Is.EqualTo(false)); Assert.That(t4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - t = FdbTuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); var t5 = t.As(); Assert.That(t5.Item1, Is.EqualTo("Hello")); Assert.That(t5.Item2, Is.EqualTo(123)); @@ -837,50 +837,50 @@ public void Test_FdbTuple_As() [Test] public void Test_Cast_To_BCL_Tuples() { - // implicit: Tuple => FdbTuple - // explicit: FdbTuple => Tuple + // implicit: Tuple => STuple + // explicit: STuple => Tuple - var t1 = FdbTuple.Create("Hello"); + var t1 = STuple.Create("Hello"); var b1 = (Tuple) t1; // explicit Assert.That(b1, Is.Not.Null); Assert.That(b1.Item1, Is.EqualTo("Hello")); - FdbTuple r1 = t1; // implicit + STuple r1 = t1; // implicit Assert.That(r1.Item1, Is.EqualTo("Hello")); - var t2 = FdbTuple.Create("Hello", 123); + var t2 = STuple.Create("Hello", 123); var b2 = (Tuple)t2; // explicit Assert.That(b2, Is.Not.Null); Assert.That(b2.Item1, Is.EqualTo("Hello")); Assert.That(b2.Item2, Is.EqualTo(123)); - FdbTuple r2 = t2; // implicit + STuple r2 = t2; // implicit Assert.That(r2.Item1, Is.EqualTo("Hello")); Assert.That(r2.Item2, Is.EqualTo(123)); - var t3 = FdbTuple.Create("Hello", 123, false); + var t3 = STuple.Create("Hello", 123, false); var b3 = (Tuple)t3; // explicit Assert.That(b3, Is.Not.Null); Assert.That(b3.Item1, Is.EqualTo("Hello")); Assert.That(b3.Item2, Is.EqualTo(123)); Assert.That(b3.Item3, Is.EqualTo(false)); - FdbTuple r3 = t3; // implicit + STuple r3 = t3; // implicit Assert.That(r3.Item1, Is.EqualTo("Hello")); Assert.That(r3.Item2, Is.EqualTo(123)); Assert.That(r3.Item3, Is.EqualTo(false)); - var t4 = FdbTuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5)); + var t4 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5)); var b4 = (Tuple)t4; // explicit Assert.That(b4, Is.Not.Null); Assert.That(b4.Item1, Is.EqualTo("Hello")); Assert.That(b4.Item2, Is.EqualTo(123)); Assert.That(b4.Item3, Is.EqualTo(false)); Assert.That(b4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - FdbTuple r4 = t4; // implicit + STuple r4 = t4; // implicit Assert.That(r4.Item1, Is.EqualTo("Hello")); Assert.That(r4.Item2, Is.EqualTo(123)); Assert.That(r4.Item3, Is.EqualTo(false)); Assert.That(r4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - var t5 = FdbTuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + var t5 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); var b5 = (Tuple)t5; // explicit Assert.That(b5, Is.Not.Null); Assert.That(b5.Item1, Is.EqualTo("Hello")); @@ -888,7 +888,7 @@ public void Test_Cast_To_BCL_Tuples() Assert.That(b5.Item3, Is.EqualTo(false)); Assert.That(b5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); Assert.That(b5.Item5, Is.EqualTo("World")); - FdbTuple r5 = t5; // implicit + STuple r5 = t5; // implicit Assert.That(r5.Item1, Is.EqualTo("Hello")); Assert.That(r5.Item2, Is.EqualTo(123)); Assert.That(r5.Item3, Is.EqualTo(false)); @@ -901,7 +901,7 @@ public void Test_Cast_To_BCL_Tuples() #region Splicing... - private static void VerifyTuple(string message, IFdbTuple t, object[] expected) + private static void VerifyTuple(string message, ITuple t, object[] expected) { // count if (t.Count != expected.Length) @@ -909,7 +909,7 @@ private static void VerifyTuple(string message, IFdbTuple t, object[] expected) #if DEBUG if (Debugger.IsAttached) Debugger.Break(); #endif - Assert.Fail("{0}: Count mismatch between observed {1} and expected {2} for tuple of type {3}", message, t, FdbTuple.ToString(expected), t.GetType().Name); + Assert.Fail("{0}: Count mismatch between observed {1} and expected {2} for tuple of type {3}", message, t, STuple.ToString(expected), t.GetType().Name); } // direct access @@ -944,7 +944,7 @@ private static void VerifyTuple(string message, IFdbTuple t, object[] expected) } // Append - if (!(t is FdbSlicedTuple)) + if (!(t is SlicedTuple)) { var u = t.Append("last"); Assert.That(u.Get(-1), Is.EqualTo("last")); @@ -963,7 +963,7 @@ public void Test_Can_Splice_FdbListTuple() // 0 1 2 3 4 5 // -6 -5 -4 -3 -2 -1 - var tuple = new FdbListTuple(items); + var tuple = new ListTuple(items); Assert.That(tuple.Count, Is.EqualTo(6)); // get all @@ -1042,21 +1042,21 @@ public void Test_Randomized_Splices() const int N = 100 * 1000; - var tuples = new IFdbTuple[14]; - tuples[0] = FdbTuple.Empty; - tuples[1] = FdbTuple.Create("A"); - tuples[2] = FdbTuple.Create("A", "B"); - tuples[3] = FdbTuple.Create("A", "B", "C"); - tuples[4] = FdbTuple.Create("A", "B", "C", "D"); - tuples[5] = FdbTuple.Create("A", "B", "C", "D", "E"); - tuples[6] = FdbTuple.Create("A", "B", "C", "D", "E", "F"); - tuples[7] = FdbTuple.Create("A", "B", "C", "D", "E", "F", "G"); - tuples[8] = FdbTuple.Create("A", "B", "C", "D", "E", "F", "G", "H"); - tuples[9] = FdbTuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I"); - tuples[10]= FdbTuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J"); - tuples[11] = new FdbJoinedTuple(tuples[6], FdbTuple.Create("G", "H", "I", "J", "K")); - tuples[12] = new FdbLinkedTuple(tuples[11], "L"); - tuples[13] = new FdbLinkedTuple(FdbTuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L"), "M"); + var tuples = new ITuple[14]; + tuples[0] = STuple.Empty; + tuples[1] = STuple.Create("A"); + tuples[2] = STuple.Create("A", "B"); + tuples[3] = STuple.Create("A", "B", "C"); + tuples[4] = STuple.Create("A", "B", "C", "D"); + tuples[5] = STuple.Create("A", "B", "C", "D", "E"); + tuples[6] = STuple.Create("A", "B", "C", "D", "E", "F"); + tuples[7] = STuple.Create("A", "B", "C", "D", "E", "F", "G"); + tuples[8] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H"); + tuples[9] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I"); + tuples[10]= STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J"); + tuples[11] = new JoinedTuple(tuples[6], STuple.Create("G", "H", "I", "J", "K")); + tuples[12] = new LinkedTuple(tuples[11], "L"); + tuples[13] = new LinkedTuple(STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L"), "M"); #if false Console.Write("Checking tuples"); @@ -1081,7 +1081,7 @@ public void Test_Randomized_Splices() if (rnd.Next(5) == 0) { // randomly pack/unpack - tuple = FdbTuple.Unpack(tuple.ToSlice()); + tuple = STuple.Unpack(tuple.ToSlice()); prefix = "unpacked:" + prefix; } else if (rnd.Next(5) == 0) @@ -1144,7 +1144,7 @@ public void Test_Randomized_Splices() #region Serialization... [Test] - public void Test_FdbTuple_Serialize_Bytes() + public void Test_STuple_Serialize_Bytes() { // Byte arrays are stored with prefix '01' followed by the bytes, and terminated by '00'. All occurences of '00' in the byte array are escaped with '00 FF' // - Best case: packed_size = 2 + array_len @@ -1152,98 +1152,98 @@ public void Test_FdbTuple_Serialize_Bytes() Slice packed; - packed = FdbTuple.EncodeKey(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }); + packed = STuple.EncodeKey(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }); Assert.That(packed.ToString(), Is.EqualTo("<01><12>4Vx<9A><00>")); - packed = FdbTuple.EncodeKey(new byte[] { 0x00, 0x42 }); + packed = STuple.EncodeKey(new byte[] { 0x00, 0x42 }); Assert.That(packed.ToString(), Is.EqualTo("<01><00>B<00>")); - packed = FdbTuple.EncodeKey(new byte[] { 0x42, 0x00 }); + packed = STuple.EncodeKey(new byte[] { 0x42, 0x00 }); Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>")); - packed = FdbTuple.EncodeKey(new byte[] { 0x42, 0x00, 0x42 }); + packed = STuple.EncodeKey(new byte[] { 0x42, 0x00, 0x42 }); Assert.That(packed.ToString(), Is.EqualTo("<01>B<00>B<00>")); - packed = FdbTuple.EncodeKey(new byte[] { 0x42, 0x00, 0x00, 0x42 }); + packed = STuple.EncodeKey(new byte[] { 0x42, 0x00, 0x00, 0x42 }); Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>B<00>")); } [Test] - public void Test_FdbTuple_Deserialize_Bytes() + public void Test_STuple_Deserialize_Bytes() { - IFdbTuple t; + ITuple t; - t = FdbTuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); + t = STuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF })); Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF")); - t = FdbTuple.Unpack(Slice.Unescape("<01><42><00><00>")); + t = STuple.Unpack(Slice.Unescape("<01><42><00><00>")); Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00 })); Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00")); - t = FdbTuple.Unpack(Slice.Unescape("<01><00><42><00>")); + t = STuple.Unpack(Slice.Unescape("<01><00><42><00>")); Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x00, 0x42 })); Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("00 42")); - t = FdbTuple.Unpack(Slice.Unescape("<01><42><00><42><00>")); + t = STuple.Unpack(Slice.Unescape("<01><42><00><42><00>")); Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00, 0x42 })); Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 42")); - t = FdbTuple.Unpack(Slice.Unescape("<01><42><00><00><42><00>")); + t = STuple.Unpack(Slice.Unescape("<01><42><00><00><42><00>")); Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00, 0x00, 0x42 })); Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 00 42")); } [Test] - public void Test_FdbTuple_Serialize_Unicode_Strings() + public void Test_STuple_Serialize_Unicode_Strings() { // Unicode strings are stored with prefix '02' followed by the utf8 bytes, and terminated by '00'. All occurences of '00' in the UTF8 bytes are escaped with '00 FF' Slice packed; // simple string - packed = FdbTuple.Create("hello world").ToSlice(); + packed = STuple.Create("hello world").ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("<02>hello world<00>")); // empty - packed = FdbTuple.Create(String.Empty).ToSlice(); + packed = STuple.Create(String.Empty).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("<02><00>")); // null - packed = FdbTuple.Create(default(string)).ToSlice(); + packed = STuple.Create(default(string)).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("<00>")); // unicode - packed = FdbTuple.Create("こんにちは世界").ToSlice(); + packed = STuple.Create("こんにちは世界").ToSlice(); // note: Encoding.UTF8.GetBytes("こんにちは世界") => { e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c } Assert.That(packed.ToString(), Is.EqualTo("<02><81><93><82><93><81><81><81><96><95><8C><00>")); } [Test] - public void Test_FdbTuple_Deserialize_Unicode_Strings() + public void Test_STuple_Deserialize_Unicode_Strings() { - IFdbTuple t; + ITuple t; // simple string - t = FdbTuple.Unpack(Slice.Unescape("<02>hello world<00>")); + t = STuple.Unpack(Slice.Unescape("<02>hello world<00>")); Assert.That(t.Get(0), Is.EqualTo("hello world")); Assert.That(t[0], Is.EqualTo("hello world")); // empty - t = FdbTuple.Unpack(Slice.Unescape("<02><00>")); + t = STuple.Unpack(Slice.Unescape("<02><00>")); Assert.That(t.Get(0), Is.EqualTo(String.Empty)); Assert.That(t[0], Is.EqualTo(String.Empty)); // null - t = FdbTuple.Unpack(Slice.Unescape("<00>")); + t = STuple.Unpack(Slice.Unescape("<00>")); Assert.That(t.Get(0), Is.EqualTo(default(string))); Assert.That(t[0], Is.Null); // unicode - t = FdbTuple.Unpack(Slice.Unescape("<02><81><93><82><93><81><81><81><96><95><8C><00>")); + t = STuple.Unpack(Slice.Unescape("<02><81><93><82><93><81><81><81><96><95><8C><00>")); // note: Encoding.UTF8.GetString({ e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c }) => "こんにちは世界" Assert.That(t.Get(0), Is.EqualTo("こんにちは世界")); Assert.That(t[0], Is.EqualTo("こんにちは世界")); } [Test] - public void Test_FdbTuple_Serialize_Guids() + public void Test_STuple_Serialize_Guids() { // 128-bit Guids are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 @@ -1252,146 +1252,146 @@ public void Test_FdbTuple_Serialize_Guids() Slice packed; // note: new Guid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; - packed = FdbTuple.Create(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); + packed = STuple.Create(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - packed = FdbTuple.Create(Guid.Empty).ToSlice(); + packed = STuple.Create(Guid.Empty).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); } [Test] - public void Test_FdbTuple_Deserialize_Guids() + public void Test_STuple_Deserialize_Guids() { // 128-bit Guids are stored with prefix '30' followed by 16 bytes // we also accept byte arrays (prefix '01') if they are of length 16 - IFdbTuple packed; + ITuple packed; - packed = FdbTuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + packed = STuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); Assert.That(packed[0], Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - packed = FdbTuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + packed = STuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); Assert.That(packed[0], Is.EqualTo(Guid.Empty)); // unicode string - packed = FdbTuple.Unpack(Slice.Unescape("<02>03020100-0504-0706-0809-0a0b0c0d0e0f<00>")); + packed = STuple.Unpack(Slice.Unescape("<02>03020100-0504-0706-0809-0a0b0c0d0e0f<00>")); Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("03020100-0504-0706-0809-0a0b0c0d0e0f"))); //note: t[0] returns a string, not a GUID // null maps to Guid.Empty - packed = FdbTuple.Unpack(Slice.Unescape("<00>")); + packed = STuple.Unpack(Slice.Unescape("<00>")); Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); //note: t[0] returns null, not a GUID } [Test] - public void Test_FdbTuple_Serialize_Uuid128s() + public void Test_STuple_Serialize_Uuid128s() { // UUID128s are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 Slice packed; // note: new Uuid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; - packed = FdbTuple.Create(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); + packed = STuple.Create(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - packed = FdbTuple.Create(Uuid128.Empty).ToSlice(); + packed = STuple.Create(Uuid128.Empty).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); } [Test] - public void Test_FdbTuple_Deserialize_Uuid128s() + public void Test_STuple_Deserialize_Uuid128s() { // UUID128s are stored with prefix '30' followed by 16 bytes (the result of uuid.ToByteArray()) // we also accept byte arrays (prefix '01') if they are of length 16 - IFdbTuple packed; + ITuple packed; // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; - packed = FdbTuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + packed = STuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); Assert.That(packed[0], Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - packed = FdbTuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + packed = STuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); Assert.That(packed[0], Is.EqualTo(Uuid128.Empty)); // unicode string - packed = FdbTuple.Unpack(Slice.Unescape("<02>00010203-0405-0607-0809-0a0b0c0d0e0f<00>")); + packed = STuple.Unpack(Slice.Unescape("<02>00010203-0405-0607-0809-0a0b0c0d0e0f<00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); //note: t[0] returns a string, not a UUID // null maps to Uuid.Empty - packed = FdbTuple.Unpack(Slice.Unescape("<00>")); + packed = STuple.Unpack(Slice.Unescape("<00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); //note: t[0] returns null, not a UUID } [Test] - public void Test_FdbTuple_Serialize_Uuid64s() + public void Test_STuple_Serialize_Uuid64s() { // UUID64s are stored with prefix '31' followed by 8 bytes formatted according to RFC 4122 Slice packed; // note: new Uuid(bytes from 0 to 7) => "00010203-04050607"; - packed = FdbTuple.Create(Uuid64.Parse("00010203-04050607")).ToSlice(); + packed = STuple.Create(Uuid64.Parse("00010203-04050607")).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("1<00><01><02><03><04><05><06><07>")); - packed = FdbTuple.Create(Uuid64.Parse("01234567-89ABCDEF")).ToSlice(); + packed = STuple.Create(Uuid64.Parse("01234567-89ABCDEF")).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("1<01>#Eg<89>")); - packed = FdbTuple.Create(Uuid64.Empty).ToSlice(); + packed = STuple.Create(Uuid64.Empty).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00><00><00><00><00>")); - packed = FdbTuple.Create(new Uuid64(0xBADC0FFEE0DDF00DUL)).ToSlice(); + packed = STuple.Create(new Uuid64(0xBADC0FFEE0DDF00DUL)).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("1<0F>
<0D>")); - packed = FdbTuple.Create(new Uuid64(0xDEADBEEFL)).ToSlice(); + packed = STuple.Create(new Uuid64(0xDEADBEEFL)).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00>")); } [Test] - public void Test_FdbTuple_Deserialize_Uuid64s() + public void Test_STuple_Deserialize_Uuid64s() { // UUID64s are stored with prefix '31' followed by 8 bytes (the result of uuid.ToByteArray()) // we also accept byte arrays (prefix '01') if they are of length 8, and unicode strings (prefix '02') - IFdbTuple packed; + ITuple packed; // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; - packed = FdbTuple.Unpack(Slice.Unescape("<31><01><23><45><67><89>")); + packed = STuple.Unpack(Slice.Unescape("<31><01><23><45><67><89>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); Assert.That(packed[0], Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); - packed = FdbTuple.Unpack(Slice.Unescape("<31><00><00><00><00><00><00><00><00>")); + packed = STuple.Unpack(Slice.Unescape("<31><00><00><00><00><00><00><00><00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); Assert.That(packed[0], Is.EqualTo(Uuid64.Empty)); // 8 bytes - packed = FdbTuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); + packed = STuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); //note: t[0] returns a string, not a UUID // unicode string - packed = FdbTuple.Unpack(Slice.Unescape("<02>01234567-89abcdef<00>")); + packed = STuple.Unpack(Slice.Unescape("<02>01234567-89abcdef<00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); //note: t[0] returns a string, not a UUID // null maps to Uuid.Empty - packed = FdbTuple.Unpack(Slice.Unescape("<00>")); + packed = STuple.Unpack(Slice.Unescape("<00>")); Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); //note: t[0] returns null, not a UUID } [Test] - public void Test_FdbTuple_Serialize_Integers() + public void Test_STuple_Serialize_Integers() { // Positive integers are stored with a variable-length encoding. // - The prefix is 0x14 + the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x14 to 0x1C @@ -1403,141 +1403,141 @@ public void Test_FdbTuple_Serialize_Integers() // - ulong.MaxValue => <1C> Assert.That( - FdbTuple.Create(0).ToSlice().ToString(), + STuple.Create(0).ToSlice().ToString(), Is.EqualTo("<14>") ); Assert.That( - FdbTuple.Create(1).ToSlice().ToString(), + STuple.Create(1).ToSlice().ToString(), Is.EqualTo("<15><01>") ); Assert.That( - FdbTuple.Create(255).ToSlice().ToString(), + STuple.Create(255).ToSlice().ToString(), Is.EqualTo("<15>") ); Assert.That( - FdbTuple.Create(256).ToSlice().ToString(), + STuple.Create(256).ToSlice().ToString(), Is.EqualTo("<16><01><00>") ); Assert.That( - FdbTuple.Create(65535).ToSlice().ToString(), + STuple.Create(65535).ToSlice().ToString(), Is.EqualTo("<16>") ); Assert.That( - FdbTuple.Create(65536).ToSlice().ToString(), + STuple.Create(65536).ToSlice().ToString(), Is.EqualTo("<17><01><00><00>") ); Assert.That( - FdbTuple.Create(int.MaxValue).ToSlice().ToString(), + STuple.Create(int.MaxValue).ToSlice().ToString(), Is.EqualTo("<18><7F>") ); // signed max Assert.That( - FdbTuple.Create(long.MaxValue).ToSlice().ToString(), + STuple.Create(long.MaxValue).ToSlice().ToString(), Is.EqualTo("<1C><7F>") ); // unsigned max Assert.That( - FdbTuple.Create(ulong.MaxValue).ToSlice().ToString(), + STuple.Create(ulong.MaxValue).ToSlice().ToString(), Is.EqualTo("<1C>") ); } [Test] - public void Test_FdbTuple_Deserialize_Integers() + public void Test_STuple_Deserialize_Integers() { Action verify = (encoded, value) => { var slice = Slice.Unescape(encoded); - Assert.That(FdbTuplePackers.DeserializeBoxed(slice), Is.EqualTo(value), "DeserializeBoxed({0})", encoded); + Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(value), "DeserializeBoxed({0})", encoded); // int64 - Assert.That(FdbTuplePackers.DeserializeInt64(slice), Is.EqualTo(value), "DeserializeInt64({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo(value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeInt64(slice), Is.EqualTo(value), "DeserializeInt64({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo(value), "Deserialize({0})", encoded); // uint64 if (value >= 0) { - Assert.That(FdbTuplePackers.DeserializeUInt64(slice), Is.EqualTo((ulong)value), "DeserializeUInt64({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((ulong)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeUInt64(slice), Is.EqualTo((ulong)value), "DeserializeUInt64({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ulong)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeUInt64(slice), Throws.InstanceOf(), "DeserializeUInt64({0})", encoded); + Assert.That(() => TuplePackers.DeserializeUInt64(slice), Throws.InstanceOf(), "DeserializeUInt64({0})", encoded); } // int32 if (value <= int.MaxValue && value >= int.MinValue) { - Assert.That(FdbTuplePackers.DeserializeInt32(slice), Is.EqualTo((int)value), "DeserializeInt32({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((int)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeInt32(slice), Is.EqualTo((int)value), "DeserializeInt32({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((int)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeInt32(slice), Throws.InstanceOf(), "DeserializeInt32({0})", encoded); + Assert.That(() => TuplePackers.DeserializeInt32(slice), Throws.InstanceOf(), "DeserializeInt32({0})", encoded); } // uint32 if (value <= uint.MaxValue && value >= 0) { - Assert.That(FdbTuplePackers.DeserializeUInt32(slice), Is.EqualTo((uint)value), "DeserializeUInt32({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((uint)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeUInt32(slice), Is.EqualTo((uint)value), "DeserializeUInt32({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((uint)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeUInt32(slice), Throws.InstanceOf(), "DeserializeUInt32({0})", encoded); + Assert.That(() => TuplePackers.DeserializeUInt32(slice), Throws.InstanceOf(), "DeserializeUInt32({0})", encoded); } // int16 if (value <= short.MaxValue && value >= short.MinValue) { - Assert.That(FdbTuplePackers.DeserializeInt16(slice), Is.EqualTo((short)value), "DeserializeInt16({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((short)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeInt16(slice), Is.EqualTo((short)value), "DeserializeInt16({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((short)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeInt16(slice), Throws.InstanceOf(), "DeserializeInt16({0})", encoded); + Assert.That(() => TuplePackers.DeserializeInt16(slice), Throws.InstanceOf(), "DeserializeInt16({0})", encoded); } // uint16 if (value <= ushort.MaxValue && value >= 0) { - Assert.That(FdbTuplePackers.DeserializeUInt16(slice), Is.EqualTo((ushort)value), "DeserializeUInt16({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((ushort)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeUInt16(slice), Is.EqualTo((ushort)value), "DeserializeUInt16({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ushort)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeUInt16(slice), Throws.InstanceOf(), "DeserializeUInt16({0})", encoded); + Assert.That(() => TuplePackers.DeserializeUInt16(slice), Throws.InstanceOf(), "DeserializeUInt16({0})", encoded); } // sbyte if (value <= sbyte.MaxValue && value >= sbyte.MinValue) { - Assert.That(FdbTuplePackers.DeserializeSByte(slice), Is.EqualTo((sbyte)value), "DeserializeSByte({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((sbyte)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeSByte(slice), Is.EqualTo((sbyte)value), "DeserializeSByte({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((sbyte)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeSByte(slice), Throws.InstanceOf(), "DeserializeSByte({0})", encoded); + Assert.That(() => TuplePackers.DeserializeSByte(slice), Throws.InstanceOf(), "DeserializeSByte({0})", encoded); } // byte if (value <= 255 && value >= 0) { - Assert.That(FdbTuplePackers.DeserializeByte(slice), Is.EqualTo((byte)value), "DeserializeByte({0})", encoded); - Assert.That(FdbTuplePacker.Deserialize(slice), Is.EqualTo((byte)value), "Deserialize({0})", encoded); + Assert.That(TuplePackers.DeserializeByte(slice), Is.EqualTo((byte)value), "DeserializeByte({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((byte)value), "Deserialize({0})", encoded); } else { - Assert.That(() => FdbTuplePackers.DeserializeByte(slice), Throws.InstanceOf(), "DeserializeByte({0})", encoded); + Assert.That(() => TuplePackers.DeserializeByte(slice), Throws.InstanceOf(), "DeserializeByte({0})", encoded); } }; @@ -1562,7 +1562,7 @@ public void Test_FdbTuple_Deserialize_Integers() } [Test] - public void Test_FdbTuple_Serialize_Negative_Integers() + public void Test_STuple_Serialize_Negative_Integers() { // Negative integers are stored with a variable-length encoding. // - The prefix is 0x14 - the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x0C to 0x13 @@ -1574,65 +1574,65 @@ public void Test_FdbTuple_Serialize_Negative_Integers() // - long.MinValue => <0C><7F> Assert.That( - FdbTuple.Create(-1).ToSlice().ToString(), + STuple.Create(-1).ToSlice().ToString(), Is.EqualTo("<13>") ); Assert.That( - FdbTuple.Create(-255).ToSlice().ToString(), + STuple.Create(-255).ToSlice().ToString(), Is.EqualTo("<13><00>") ); Assert.That( - FdbTuple.Create(-256).ToSlice().ToString(), + STuple.Create(-256).ToSlice().ToString(), Is.EqualTo("<12>") ); Assert.That( - FdbTuple.Create(-257).ToSlice().ToString(), + STuple.Create(-257).ToSlice().ToString(), Is.EqualTo("<12>") ); Assert.That( - FdbTuple.Create(-65535).ToSlice().ToString(), + STuple.Create(-65535).ToSlice().ToString(), Is.EqualTo("<12><00><00>") ); Assert.That( - FdbTuple.Create(-65536).ToSlice().ToString(), + STuple.Create(-65536).ToSlice().ToString(), Is.EqualTo("<11>") ); Assert.That( - FdbTuple.Create(int.MinValue).ToSlice().ToString(), + STuple.Create(int.MinValue).ToSlice().ToString(), Is.EqualTo("<10><7F>") ); Assert.That( - FdbTuple.Create(long.MinValue).ToSlice().ToString(), + STuple.Create(long.MinValue).ToSlice().ToString(), Is.EqualTo("<0C><7F>") ); } [Test] - public void Test_FdbTuple_Serialize_Singles() + public void Test_STuple_Serialize_Singles() { // 32-bit floats are stored in 5 bytes, using the prefix 0x20 followed by the High-Endian representation of their normalized form - Assert.That(FdbTuple.Create(0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 00")); - Assert.That(FdbTuple.Create(42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 C2 28 00 00")); - Assert.That(FdbTuple.Create(-42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 3D D7 FF FF")); + Assert.That(STuple.Create(0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 00")); + Assert.That(STuple.Create(42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 C2 28 00 00")); + Assert.That(STuple.Create(-42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 3D D7 FF FF")); - Assert.That(FdbTuple.Create((float)Math.Sqrt(2)).ToSlice().ToHexaString(' '), Is.EqualTo("20 BF B5 04 F3")); + Assert.That(STuple.Create((float)Math.Sqrt(2)).ToSlice().ToHexaString(' '), Is.EqualTo("20 BF B5 04 F3")); - Assert.That(FdbTuple.Create(float.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 80 00 00"), "float.MinValue"); - Assert.That(FdbTuple.Create(float.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 7F FF FF"), "float.MaxValue"); - Assert.That(FdbTuple.Create(-0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FF"), "-0f"); - Assert.That(FdbTuple.Create(float.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 7F FF FF"), "float.NegativeInfinity"); - Assert.That(FdbTuple.Create(float.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 80 00 00"), "float.PositiveInfinity"); - Assert.That(FdbTuple.Create(float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 01"), "+float.Epsilon"); - Assert.That(FdbTuple.Create(-float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FE"), "-float.Epsilon"); + Assert.That(STuple.Create(float.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 80 00 00"), "float.MinValue"); + Assert.That(STuple.Create(float.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 7F FF FF"), "float.MaxValue"); + Assert.That(STuple.Create(-0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FF"), "-0f"); + Assert.That(STuple.Create(float.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 7F FF FF"), "float.NegativeInfinity"); + Assert.That(STuple.Create(float.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 80 00 00"), "float.PositiveInfinity"); + Assert.That(STuple.Create(float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 01"), "+float.Epsilon"); + Assert.That(STuple.Create(-float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FE"), "-float.Epsilon"); // all possible variants of NaN should all be equal - Assert.That(FdbTuple.Create(float.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 3F FF FF"), "float.NaN"); + Assert.That(STuple.Create(float.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 3F FF FF"), "float.NaN"); // cook up a non standard NaN (with some bits set in the fraction) float f = float.NaN; // defined as 1f / 0f @@ -1642,7 +1642,7 @@ public void Test_FdbTuple_Serialize_Singles() unsafe { f = *((float*)&nan); } Assert.That(float.IsNaN(f), Is.True); Assert.That( - FdbTuple.Create(f).ToSlice().ToHexaString(' '), + STuple.Create(f).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 3F FF FF"), "All variants of NaN must be normalized" //note: if we have 20 00 3F FF 84, that means that the NaN was not normalized @@ -1651,52 +1651,52 @@ public void Test_FdbTuple_Serialize_Singles() } [Test] - public void Test_FdbTuple_Deserialize_Singles() + public void Test_STuple_Deserialize_Singles() { - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 80 00 00 00")), Is.EqualTo(0f), "0f"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 C2 28 00 00")), Is.EqualTo(42f), "42f"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 3D D7 FF FF")), Is.EqualTo(-42f), "-42f"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 80 00 00 00")), Is.EqualTo(0f), "0f"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 C2 28 00 00")), Is.EqualTo(42f), "42f"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 3D D7 FF FF")), Is.EqualTo(-42f), "-42f"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 BF B5 04 F3")), Is.EqualTo((float)Math.Sqrt(2)), "Sqrt(2)"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 BF B5 04 F3")), Is.EqualTo((float)Math.Sqrt(2)), "Sqrt(2)"); // well known values - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.MinValue"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 FF 7F FF FF")), Is.EqualTo(float.MaxValue), "float.MaxValue"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 7F FF FF FF")), Is.EqualTo(-0f), "-0f"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 00 7F FF FF")), Is.EqualTo(float.NegativeInfinity), "float.NegativeInfinity"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 FF 80 00 00")), Is.EqualTo(float.PositiveInfinity), "float.PositiveInfinity"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.Epsilon"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 80 00 00 01")), Is.EqualTo(float.Epsilon), "+float.Epsilon"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 7F FF FF FE")), Is.EqualTo(-float.Epsilon), "-float.Epsilon"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.MinValue"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 FF 7F FF FF")), Is.EqualTo(float.MaxValue), "float.MaxValue"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 7F FF FF FF")), Is.EqualTo(-0f), "-0f"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 7F FF FF")), Is.EqualTo(float.NegativeInfinity), "float.NegativeInfinity"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 FF 80 00 00")), Is.EqualTo(float.PositiveInfinity), "float.PositiveInfinity"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.Epsilon"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 80 00 00 01")), Is.EqualTo(float.Epsilon), "+float.Epsilon"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 7F FF FF FE")), Is.EqualTo(-float.Epsilon), "-float.Epsilon"); // all possible variants of NaN should end up equal and normalized to float.NaN - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); } [Test] - public void Test_FdbTuple_Serialize_Doubles() + public void Test_STuple_Serialize_Doubles() { // 64-bit floats are stored in 9 bytes, using the prefix 0x21 followed by the High-Endian representation of their normalized form - Assert.That(FdbTuple.Create(0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 00")); - Assert.That(FdbTuple.Create(42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 45 00 00 00 00 00 00")); - Assert.That(FdbTuple.Create(-42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 3F BA FF FF FF FF FF FF")); + Assert.That(STuple.Create(0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 00")); + Assert.That(STuple.Create(42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 45 00 00 00 00 00 00")); + Assert.That(STuple.Create(-42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 3F BA FF FF FF FF FF FF")); - Assert.That(FdbTuple.Create(Math.PI).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 09 21 FB 54 44 2D 18")); - Assert.That(FdbTuple.Create(Math.E).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 05 BF 0A 8B 14 57 69")); + Assert.That(STuple.Create(Math.PI).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 09 21 FB 54 44 2D 18")); + Assert.That(STuple.Create(Math.E).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 05 BF 0A 8B 14 57 69")); - Assert.That(FdbTuple.Create(double.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 10 00 00 00 00 00 00"), "double.MinValue"); - Assert.That(FdbTuple.Create(double.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF EF FF FF FF FF FF FF"), "double.MaxValue"); - Assert.That(FdbTuple.Create(-0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FF"), "-0d"); - Assert.That(FdbTuple.Create(double.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 0F FF FF FF FF FF FF"), "double.NegativeInfinity"); - Assert.That(FdbTuple.Create(double.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF F0 00 00 00 00 00 00"), "double.PositiveInfinity"); - Assert.That(FdbTuple.Create(double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 01"), "+double.Epsilon"); - Assert.That(FdbTuple.Create(-double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FE"), "-double.Epsilon"); + Assert.That(STuple.Create(double.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 10 00 00 00 00 00 00"), "double.MinValue"); + Assert.That(STuple.Create(double.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF EF FF FF FF FF FF FF"), "double.MaxValue"); + Assert.That(STuple.Create(-0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FF"), "-0d"); + Assert.That(STuple.Create(double.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 0F FF FF FF FF FF FF"), "double.NegativeInfinity"); + Assert.That(STuple.Create(double.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF F0 00 00 00 00 00 00"), "double.PositiveInfinity"); + Assert.That(STuple.Create(double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 01"), "+double.Epsilon"); + Assert.That(STuple.Create(-double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FE"), "-double.Epsilon"); // all possible variants of NaN should all be equal - Assert.That(FdbTuple.Create(double.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 07 FF FF FF FF FF FF"), "double.NaN"); + Assert.That(STuple.Create(double.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 07 FF FF FF FF FF FF"), "double.NaN"); // cook up a non standard NaN (with some bits set in the fraction) double d = double.NaN; // defined as 1d / 0d @@ -1706,114 +1706,114 @@ public void Test_FdbTuple_Serialize_Doubles() unsafe { d = *((double*)&nan); } Assert.That(double.IsNaN(d), Is.True); Assert.That( - FdbTuple.Create(d).ToSlice().ToHexaString(' '), + STuple.Create(d).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 07 FF FF FF FF FF FF") //note: if we have 21 00 07 FF FF FF FF FF 84, that means that the NaN was not normalized ); // roundtripping vectors of doubles - var tuple = FdbTuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)); - Assert.That(FdbTuple.Unpack(FdbTuple.EncodeKey(Math.PI, Math.E, Math.Log(1), Math.Log(2))), Is.EqualTo(tuple)); - Assert.That(FdbTuple.Unpack(FdbTuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); - Assert.That(FdbTuple.Unpack(FdbTuple.Empty.Append(Math.PI).Append(Math.E).Append(Math.Log(1)).Append(Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); + var tuple = STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)); + Assert.That(STuple.Unpack(STuple.EncodeKey(Math.PI, Math.E, Math.Log(1), Math.Log(2))), Is.EqualTo(tuple)); + Assert.That(STuple.Unpack(STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); + Assert.That(STuple.Unpack(STuple.Empty.Append(Math.PI).Append(Math.E).Append(Math.Log(1)).Append(Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); } [Test] - public void Test_FdbTuple_Deserialize_Doubles() + public void Test_STuple_Deserialize_Doubles() { - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 00")), Is.EqualTo(0d), "0d"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 C0 45 00 00 00 00 00 00")), Is.EqualTo(42d), "42d"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 3F BA FF FF FF FF FF FF")), Is.EqualTo(-42d), "-42d"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 00")), Is.EqualTo(0d), "0d"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 45 00 00 00 00 00 00")), Is.EqualTo(42d), "42d"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 3F BA FF FF FF FF FF FF")), Is.EqualTo(-42d), "-42d"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 C0 09 21 FB 54 44 2D 18")), Is.EqualTo(Math.PI), "Math.PI"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 C0 05 BF 0A 8B 14 57 69")), Is.EqualTo(Math.E), "Math.E"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 09 21 FB 54 44 2D 18")), Is.EqualTo(Math.PI), "Math.PI"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 05 BF 0A 8B 14 57 69")), Is.EqualTo(Math.E), "Math.E"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 00 10 00 00 00 00 00 00")), Is.EqualTo(double.MinValue), "double.MinValue"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 FF EF FF FF FF FF FF FF")), Is.EqualTo(double.MaxValue), "double.MaxValue"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FF")), Is.EqualTo(-0d), "-0d"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 00 0F FF FF FF FF FF FF")), Is.EqualTo(double.NegativeInfinity), "double.NegativeInfinity"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 FF F0 00 00 00 00 00 00")), Is.EqualTo(double.PositiveInfinity), "double.PositiveInfinity"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 01")), Is.EqualTo(double.Epsilon), "+double.Epsilon"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FE")), Is.EqualTo(-double.Epsilon), "-double.Epsilon"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 10 00 00 00 00 00 00")), Is.EqualTo(double.MinValue), "double.MinValue"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 FF EF FF FF FF FF FF FF")), Is.EqualTo(double.MaxValue), "double.MaxValue"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FF")), Is.EqualTo(-0d), "-0d"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 0F FF FF FF FF FF FF")), Is.EqualTo(double.NegativeInfinity), "double.NegativeInfinity"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 FF F0 00 00 00 00 00 00")), Is.EqualTo(double.PositiveInfinity), "double.PositiveInfinity"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 01")), Is.EqualTo(double.Epsilon), "+double.Epsilon"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FE")), Is.EqualTo(-double.Epsilon), "-double.Epsilon"); // all possible variants of NaN should end up equal and normalized to double.NaN - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF FF")), Is.EqualTo(double.NaN), "double.NaN"); - Assert.That(FdbTuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF 84")), Is.EqualTo(double.NaN), "double.NaN"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF FF")), Is.EqualTo(double.NaN), "double.NaN"); + Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF 84")), Is.EqualTo(double.NaN), "double.NaN"); } [Test] - public void Test_FdbTuple_Serialize_Booleans() + public void Test_STuple_Serialize_Booleans() { // Booleans are stored as interger 0 (<14>) for false, and integer 1 (<15><01>) for true Slice packed; // bool - packed = FdbTuple.EncodeKey(false); + packed = STuple.EncodeKey(false); Assert.That(packed.ToString(), Is.EqualTo("<14>")); - packed = FdbTuple.EncodeKey(true); + packed = STuple.EncodeKey(true); Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); // bool? - packed = FdbTuple.EncodeKey(default(bool?)); + packed = STuple.EncodeKey(default(bool?)); Assert.That(packed.ToString(), Is.EqualTo("<00>")); - packed = FdbTuple.EncodeKey((bool?)false); + packed = STuple.EncodeKey((bool?)false); Assert.That(packed.ToString(), Is.EqualTo("<14>")); - packed = FdbTuple.EncodeKey((bool?)true); + packed = STuple.EncodeKey((bool?)true); Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); // tuple containing bools - packed = FdbTuple.Create(true).ToSlice(); + packed = STuple.Create(true).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); - packed = FdbTuple.Create(true, null, false).ToSlice(); + packed = STuple.Create(true, null, false).ToSlice(); Assert.That(packed.ToString(), Is.EqualTo("<15><01><00><14>")); } [Test] - public void Test_FdbTuple_Deserialize_Booleans() + public void Test_STuple_Deserialize_Booleans() { // Null, 0, and empty byte[]/strings are equivalent to False. All others are equivalent to True // Falsy... - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.EqualTo(false), "Null => False"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(false), "0 => False"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<01><00>")), Is.EqualTo(false), "byte[0] => False"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<02><00>")), Is.EqualTo(false), "String.Empty => False"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.EqualTo(false), "Null => False"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(false), "0 => False"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<01><00>")), Is.EqualTo(false), "byte[0] => False"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<02><00>")), Is.EqualTo(false), "String.Empty => False"); // Truthy - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<15><01>")), Is.EqualTo(true), "1 => True"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<13>")), Is.EqualTo(true), "-1 => True"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<01>Hello<00>")), Is.EqualTo(true), "'Hello' => True"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<02>Hello<00>")), Is.EqualTo(true), "\"Hello\" => True"); - Assert.That(FdbTuple.DecodeKey(FdbTuple.EncodeKey(123456789)), Is.EqualTo(true), "random int => True"); - - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<02>True<00>")), Is.EqualTo(true), "\"True\" => True"); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<02>False<00>")), Is.EqualTo(true), "\"False\" => True ***"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<15><01>")), Is.EqualTo(true), "1 => True"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<13>")), Is.EqualTo(true), "-1 => True"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<01>Hello<00>")), Is.EqualTo(true), "'Hello' => True"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<02>Hello<00>")), Is.EqualTo(true), "\"Hello\" => True"); + Assert.That(STuple.DecodeKey(STuple.EncodeKey(123456789)), Is.EqualTo(true), "random int => True"); + + Assert.That(STuple.DecodeKey(Slice.Unescape("<02>True<00>")), Is.EqualTo(true), "\"True\" => True"); + Assert.That(STuple.DecodeKey(Slice.Unescape("<02>False<00>")), Is.EqualTo(true), "\"False\" => True ***"); // note: even though it would be tempting to convert the string "false" to False, it is not a standard behavior accross all bindings // When decoded to object, though, they should return 0 and 1 - Assert.That(FdbTuplePackers.DeserializeBoxed(FdbTuple.EncodeKey(false)), Is.EqualTo(0)); - Assert.That(FdbTuplePackers.DeserializeBoxed(FdbTuple.EncodeKey(true)), Is.EqualTo(1)); + Assert.That(TuplePackers.DeserializeBoxed(STuple.EncodeKey(false)), Is.EqualTo(0)); + Assert.That(TuplePackers.DeserializeBoxed(STuple.EncodeKey(true)), Is.EqualTo(1)); } [Test] - public void Test_FdbTuple_Serialize_IPAddress() + public void Test_STuple_Serialize_IPAddress() { // IP Addresses are stored as a byte array (<01>..<00>), in network order (big-endian) // They will take from 6 to 10 bytes, depending on the number of '.0' in them. Assert.That( - FdbTuple.Create(IPAddress.Loopback).ToSlice().ToHexaString(' '), + STuple.Create(IPAddress.Loopback).ToSlice().ToHexaString(' '), Is.EqualTo("01 7F 00 FF 00 FF 01 00") ); Assert.That( - FdbTuple.Create(IPAddress.Any).ToSlice().ToHexaString(' '), + STuple.Create(IPAddress.Any).ToSlice().ToHexaString(' '), Is.EqualTo("01 00 FF 00 FF 00 FF 00 FF 00") ); Assert.That( - FdbTuple.Create(IPAddress.Parse("1.2.3.4")).ToSlice().ToHexaString(' '), + STuple.Create(IPAddress.Parse("1.2.3.4")).ToSlice().ToHexaString(' '), Is.EqualTo("01 01 02 03 04 00") ); @@ -1821,380 +1821,380 @@ public void Test_FdbTuple_Serialize_IPAddress() [Test] - public void Test_FdbTuple_Deserialize_IPAddress() + public void Test_STuple_Deserialize_IPAddress() { - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<01><7F><00><00><01><00>")), Is.EqualTo(IPAddress.Parse("127.0.0.1"))); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<01><00><00><00><00><00>")), Is.EqualTo(IPAddress.Parse("0.0.0.0"))); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<01><01><02><03><04><00>")), Is.EqualTo(IPAddress.Parse("1.2.3.4"))); + Assert.That(STuple.DecodeKey(Slice.Unescape("<01><7F><00><00><01><00>")), Is.EqualTo(IPAddress.Parse("127.0.0.1"))); + Assert.That(STuple.DecodeKey(Slice.Unescape("<01><00><00><00><00><00>")), Is.EqualTo(IPAddress.Parse("0.0.0.0"))); + Assert.That(STuple.DecodeKey(Slice.Unescape("<01><01><02><03><04><00>")), Is.EqualTo(IPAddress.Parse("1.2.3.4"))); - Assert.That(FdbTuple.DecodeKey(FdbTuple.EncodeKey("127.0.0.1")), Is.EqualTo(IPAddress.Loopback)); + Assert.That(STuple.DecodeKey(STuple.EncodeKey("127.0.0.1")), Is.EqualTo(IPAddress.Loopback)); var ip = IPAddress.Parse("192.168.0.1"); - Assert.That(FdbTuple.DecodeKey(FdbTuple.EncodeKey(ip.ToString())), Is.EqualTo(ip)); - Assert.That(FdbTuple.DecodeKey(FdbTuple.EncodeKey(ip.GetAddressBytes())), Is.EqualTo(ip)); - Assert.That(FdbTuple.DecodeKey(FdbTuple.EncodeKey(ip.Address)), Is.EqualTo(ip)); + Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.ToString())), Is.EqualTo(ip)); + Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.GetAddressBytes())), Is.EqualTo(ip)); + Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.Address)), Is.EqualTo(ip)); } [Test] - public void Test_FdbTuple_NullableTypes() + public void Test_STuple_NullableTypes() { // Nullable types will either be encoded as <14> for null, or their regular encoding if not null // serialize - Assert.That(FdbTuple.EncodeKey(0), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(FdbTuple.EncodeKey(123), Is.EqualTo(Slice.Unescape("<15>{"))); - Assert.That(FdbTuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + Assert.That(STuple.EncodeKey(0), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(STuple.EncodeKey(123), Is.EqualTo(Slice.Unescape("<15>{"))); + Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - Assert.That(FdbTuple.EncodeKey(0L), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(FdbTuple.EncodeKey(123L), Is.EqualTo(Slice.Unescape("<15>{"))); - Assert.That(FdbTuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + Assert.That(STuple.EncodeKey(0L), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(STuple.EncodeKey(123L), Is.EqualTo(Slice.Unescape("<15>{"))); + Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - Assert.That(FdbTuple.EncodeKey(true), Is.EqualTo(Slice.Unescape("<15><01>"))); - Assert.That(FdbTuple.EncodeKey(false), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(FdbTuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>")), "Maybe it was File Not Found?"); + Assert.That(STuple.EncodeKey(true), Is.EqualTo(Slice.Unescape("<15><01>"))); + Assert.That(STuple.EncodeKey(false), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>")), "Maybe it was File Not Found?"); - Assert.That(FdbTuple.EncodeKey(Guid.Empty), Is.EqualTo(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>"))); - Assert.That(FdbTuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + Assert.That(STuple.EncodeKey(Guid.Empty), Is.EqualTo(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>"))); + Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - Assert.That(FdbTuple.EncodeKey(TimeSpan.Zero), Is.EqualTo(Slice.Unescape("!<80><00><00><00><00><00><00><00>"))); - Assert.That(FdbTuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + Assert.That(STuple.EncodeKey(TimeSpan.Zero), Is.EqualTo(Slice.Unescape("!<80><00><00><00><00><00><00><00>"))); + Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); // deserialize - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); + Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0L)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123L)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); + Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0L)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123L)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<15><01>")), Is.True); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<14>")), Is.False); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); + Assert.That(STuple.DecodeKey(Slice.Unescape("<15><01>")), Is.True); + Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.False); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")), Is.EqualTo(Guid.Empty)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); + Assert.That(STuple.DecodeKey(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")), Is.EqualTo(Guid.Empty)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(TimeSpan.Zero)); - Assert.That(FdbTuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); + Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(TimeSpan.Zero)); + Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); } [Test] - public void Test_FdbTuple_Serialize_Alias() + public void Test_STuple_Serialize_Alias() { Assert.That( - FdbTuple.EncodeKey(FdbTupleAlias.System).ToString(), + STuple.EncodeKey(FdbTupleAlias.System).ToString(), Is.EqualTo("") ); Assert.That( - FdbTuple.EncodeKey(FdbTupleAlias.Directory).ToString(), + STuple.EncodeKey(FdbTupleAlias.Directory).ToString(), Is.EqualTo("") ); Assert.That( - FdbTuple.EncodeKey(FdbTupleAlias.Zero).ToString(), + STuple.EncodeKey(FdbTupleAlias.Zero).ToString(), Is.EqualTo("<00>") ); } [Test] - public void Test_FdbTuple_Deserialize_Alias() + public void Test_STuple_Deserialize_Alias() { Slice slice; slice = Slice.Unescape(""); - Assert.That(FdbTuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.System)); + Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.System)); slice = Slice.Unescape(""); - Assert.That(FdbTuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.Directory)); + Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.Directory)); //note: FdbTupleAlias.Start is <00> and will be deserialized as null } [Test] - public void Test_FdbTuple_Serialize_Embedded_Tuples() + public void Test_STuple_Serialize_Embedded_Tuples() { - Action verify = (t, expected) => + Action verify = (t, expected) => { var key = t.ToSlice(); Assert.That(key.ToHexaString(' '), Is.EqualTo(expected)); - Assert.That(FdbTuple.Pack(t), Is.EqualTo(key)); - var t2 = FdbTuple.Unpack(key); + Assert.That(STuple.Pack(t), Is.EqualTo(key)); + var t2 = STuple.Unpack(key); Assert.That(t2, Is.Not.Null); Assert.That(t2.Count, Is.EqualTo(t.Count), "{0}", t2); Assert.That(t2, Is.EqualTo(t)); }; // Index composite key - IFdbTuple value = FdbTuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) + ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) string docId = "Doc123"; // key would be "(..., value, id)" verify( - FdbTuple.Create(42, value, docId), + STuple.Create(42, value, docId), "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" ); verify( - FdbTuple.Create(new object[] { 42, value, docId }), + STuple.Create(new object[] { 42, value, docId }), "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" ); verify( - FdbTuple.Create(42).Append(value).Append(docId), + STuple.Create(42).Append(value).Append(docId), "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" ); verify( - FdbTuple.Create(42).Append(value, docId), + STuple.Create(42).Append(value, docId), "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" ); // multiple depth verify( - FdbTuple.Create(1, FdbTuple.Create(2, 3), FdbTuple.Create(FdbTuple.Create(4, 5, 6)), 7), + STuple.Create(1, STuple.Create(2, 3), STuple.Create(STuple.Create(4, 5, 6)), 7), "15 01 03 15 02 15 03 00 03 03 15 04 15 05 15 06 00 00 15 07" ); // corner cases verify( - FdbTuple.Create(FdbTuple.Empty), + STuple.Create(STuple.Empty), "03 00" // empty tumple should have header and footer ); verify( - FdbTuple.Create(FdbTuple.Empty, default(string)), + STuple.Create(STuple.Empty, default(string)), "03 00 00" // outer null should not be escaped ); verify( - FdbTuple.Create(FdbTuple.Create(default(string)), default(string)), + STuple.Create(STuple.Create(default(string)), default(string)), "03 00 FF 00 00" // inner null should be escaped, but not outer ); verify( - FdbTuple.Create(FdbTuple.Create(0x100, 0x10000, 0x1000000)), + STuple.Create(STuple.Create(0x100, 0x10000, 0x1000000)), "03 16 01 00 17 01 00 00 18 01 00 00 00 00" ); verify( - FdbTuple.Create(default(string), FdbTuple.Empty, default(string), FdbTuple.Create(default(string)), default(string)), + STuple.Create(default(string), STuple.Empty, default(string), STuple.Create(default(string)), default(string)), "00 03 00 00 03 00 FF 00 00" ); } [Test] - public void Test_FdbTuple_SameBytes() + public void Test_STuple_SameBytes() { - IFdbTuple t1 = FdbTuple.Create("hello world"); - IFdbTuple t2 = FdbTuple.Create(new object[] { "hello world" }); + ITuple t1 = STuple.Create("hello world"); + ITuple t2 = STuple.Create(new object[] { "hello world" }); Assert.That(t1.ToSlice(), Is.EqualTo(t2.ToSlice())); - t1 = FdbTuple.Create("hello world", 1234); - t2 = FdbTuple.Create("hello world").Append(1234); + t1 = STuple.Create("hello world", 1234); + t2 = STuple.Create("hello world").Append(1234); Assert.That(t1.ToSlice(), Is.EqualTo(t2.ToSlice())); } [Test] - public void Test_FdbTuple_Create_ToSlice() + public void Test_STuple_Create_ToSlice() { Assert.That( - FdbTuple.Create("hello world").ToSlice().ToString(), + STuple.Create("hello world").ToSlice().ToString(), Is.EqualTo("<02>hello world<00>") ); Assert.That( - FdbTuple.Create("hello", "world").ToSlice().ToString(), + STuple.Create("hello", "world").ToSlice().ToString(), Is.EqualTo("<02>hello<00><02>world<00>") ); Assert.That( - FdbTuple.Create("hello world", 123).ToSlice().ToString(), + STuple.Create("hello world", 123).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><15>{") ); Assert.That( - FdbTuple.Create("hello world", 1234, -1234).ToSlice().ToString(), + STuple.Create("hello world", 1234, -1234).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><16><04><12>-") ); Assert.That( - FdbTuple.Create("hello world", 123, false).ToSlice().ToString(), + STuple.Create("hello world", 123, false).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><15>{<14>") ); Assert.That( - FdbTuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToSlice().ToString(), + STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") ); Assert.That( - FdbTuple.Create(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), + STuple.Create(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") ); Assert.That( - FdbTuple.FromArray(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }, 1, 2).ToSlice().ToString(), + STuple.FromArray(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }, 1, 2).ToSlice().ToString(), Is.EqualTo("<15>{<14>") ); Assert.That( - FdbTuple.FromEnumerable(new List { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), + STuple.FromEnumerable(new List { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") ); } [Test] - public void Test_FdbTuple_EncodeKey() + public void Test_STuple_EncodeKey() { Assert.That( - FdbTuple.EncodeKey("hello world").ToString(), + STuple.EncodeKey("hello world").ToString(), Is.EqualTo("<02>hello world<00>") ); Assert.That( - FdbTuple.EncodeKey("hello", "world").ToString(), + STuple.EncodeKey("hello", "world").ToString(), Is.EqualTo("<02>hello<00><02>world<00>") ); Assert.That( - FdbTuple.EncodeKey("hello world", 123).ToString(), + STuple.EncodeKey("hello world", 123).ToString(), Is.EqualTo("<02>hello world<00><15>{") ); Assert.That( - FdbTuple.EncodeKey("hello world", 1234, -1234).ToString(), + STuple.EncodeKey("hello world", 1234, -1234).ToString(), Is.EqualTo("<02>hello world<00><16><04><12>-") ); Assert.That( - FdbTuple.EncodeKey("hello world", 123, false).ToString(), + STuple.EncodeKey("hello world", 123, false).ToString(), Is.EqualTo("<02>hello world<00><15>{<14>") ); Assert.That( - FdbTuple.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToString(), + STuple.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToString(), Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") ); } [Test] - public void Test_FdbTuple_Unpack() + public void Test_STuple_Unpack() { - var packed = FdbTuple.Create("hello world").ToSlice(); + var packed = STuple.Create("hello world").ToSlice(); Log(packed); - var tuple = FdbTuple.Unpack(packed); + var tuple = STuple.Unpack(packed); Assert.That(tuple, Is.Not.Null); Log(tuple); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple.Get(0), Is.EqualTo("hello world")); - packed = FdbTuple.Create("hello world", 123).ToSlice(); + packed = STuple.Create("hello world", 123).ToSlice(); Log(packed); - tuple = FdbTuple.Unpack(packed); + tuple = STuple.Unpack(packed); Assert.That(tuple, Is.Not.Null); Log(tuple); Assert.That(tuple.Count, Is.EqualTo(2)); Assert.That(tuple.Get(0), Is.EqualTo("hello world")); Assert.That(tuple.Get(1), Is.EqualTo(123)); - packed = FdbTuple.Create(1, 256, 257, 65536, int.MaxValue, long.MaxValue).ToSlice(); + packed = STuple.Create(1, 256, 257, 65536, int.MaxValue, long.MaxValue).ToSlice(); Log(packed); - tuple = FdbTuple.Unpack(packed); + tuple = STuple.Unpack(packed); Assert.That(tuple, Is.Not.Null); Assert.That(tuple.Count, Is.EqualTo(6)); Assert.That(tuple.Get(0), Is.EqualTo(1)); Assert.That(tuple.Get(1), Is.EqualTo(256)); - Assert.That(tuple.Get(2), Is.EqualTo(257), ((FdbSlicedTuple)tuple).GetSlice(2).ToString()); + Assert.That(tuple.Get(2), Is.EqualTo(257), ((SlicedTuple)tuple).GetSlice(2).ToString()); Assert.That(tuple.Get(3), Is.EqualTo(65536)); Assert.That(tuple.Get(4), Is.EqualTo(int.MaxValue)); Assert.That(tuple.Get(5), Is.EqualTo(long.MaxValue)); - packed = FdbTuple.Create(-1, -256, -257, -65536, int.MinValue, long.MinValue).ToSlice(); + packed = STuple.Create(-1, -256, -257, -65536, int.MinValue, long.MinValue).ToSlice(); Log(packed); - tuple = FdbTuple.Unpack(packed); + tuple = STuple.Unpack(packed); Assert.That(tuple, Is.Not.Null); - Assert.That(tuple, Is.InstanceOf()); + Assert.That(tuple, Is.InstanceOf()); Log(tuple); Assert.That(tuple.Count, Is.EqualTo(6)); Assert.That(tuple.Get(0), Is.EqualTo(-1)); Assert.That(tuple.Get(1), Is.EqualTo(-256)); - Assert.That(tuple.Get(2), Is.EqualTo(-257), "Slice is " + ((FdbSlicedTuple)tuple).GetSlice(2).ToString()); + Assert.That(tuple.Get(2), Is.EqualTo(-257), "Slice is " + ((SlicedTuple)tuple).GetSlice(2).ToString()); Assert.That(tuple.Get(3), Is.EqualTo(-65536)); Assert.That(tuple.Get(4), Is.EqualTo(int.MinValue)); Assert.That(tuple.Get(5), Is.EqualTo(long.MinValue)); } [Test] - public void Test_FdbTuple_CreateBoxed() + public void Test_STuple_CreateBoxed() { - IFdbTuple tuple; + ITuple tuple; - tuple = FdbTuple.CreateBoxed(default(object)); + tuple = STuple.CreateBoxed(default(object)); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.Null); - tuple = FdbTuple.CreateBoxed(1); + tuple = STuple.CreateBoxed(1); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.EqualTo(1)); - tuple = FdbTuple.CreateBoxed(1L); + tuple = STuple.CreateBoxed(1L); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.EqualTo(1L)); - tuple = FdbTuple.CreateBoxed(false); + tuple = STuple.CreateBoxed(false); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.EqualTo(false)); - tuple = FdbTuple.CreateBoxed("hello"); + tuple = STuple.CreateBoxed("hello"); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.EqualTo("hello")); - tuple = FdbTuple.CreateBoxed(new byte[] { 1, 2, 3 }); + tuple = STuple.CreateBoxed(new byte[] { 1, 2, 3 }); Assert.That(tuple.Count, Is.EqualTo(1)); Assert.That(tuple[0], Is.EqualTo(Slice.Create(new byte[] { 1, 2, 3 }))); } [Test] - public void Test_FdbTuple_EncodeKey_Boxed() + public void Test_STuple_EncodeKey_Boxed() { Slice slice; - slice = FdbTuple.EncodeKey(default(object)); + slice = STuple.EncodeKey(default(object)); Assert.That(slice.ToString(), Is.EqualTo("<00>")); - slice = FdbTuple.EncodeKey(1); + slice = STuple.EncodeKey(1); Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - slice = FdbTuple.EncodeKey(1L); + slice = STuple.EncodeKey(1L); Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - slice = FdbTuple.EncodeKey(1U); + slice = STuple.EncodeKey(1U); Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - slice = FdbTuple.EncodeKey(1UL); + slice = STuple.EncodeKey(1UL); Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - slice = FdbTuple.EncodeKey(false); + slice = STuple.EncodeKey(false); Assert.That(slice.ToString(), Is.EqualTo("<14>")); - slice = FdbTuple.EncodeKey(new byte[] { 4, 5, 6 }); + slice = STuple.EncodeKey(new byte[] { 4, 5, 6 }); Assert.That(slice.ToString(), Is.EqualTo("<01><04><05><06><00>")); - slice = FdbTuple.EncodeKey("hello"); + slice = STuple.EncodeKey("hello"); Assert.That(slice.ToString(), Is.EqualTo("<02>hello<00>")); } [Test] - public void Test_FdbTuple_Numbers_Are_Sorted_Lexicographically() + public void Test_STuple_Numbers_Are_Sorted_Lexicographically() { // pick two numbers 'x' and 'y' at random, and check that the order of 'x' compared to 'y' is the same as 'pack(tuple(x))' compared to 'pack(tuple(y))' @@ -2213,8 +2213,8 @@ public void Test_FdbTuple_Numbers_Are_Sorted_Lexicographically() y = rnd.Next() - 1073741824; } - var t1 = FdbTuple.Create(x).ToSlice(); - var t2 = FdbTuple.Create(y).ToSlice(); + var t1 = STuple.Create(x).ToSlice(); + var t2 = STuple.Create(y).ToSlice(); int dint = x.CompareTo(y); int dtup = t1.CompareTo(t2); @@ -2233,100 +2233,100 @@ public void Test_FdbTuple_Numbers_Are_Sorted_Lexicographically() } [Test] - public void Test_FdbTuple_Serialize_ITupleFormattable() + public void Test_STuple_Serialize_ITupleFormattable() { // types that implement ITupleFormattable should be packed by calling ToTuple() and then packing the returned tuple Slice packed; - packed = FdbTuplePacker.Serialize(new Thing { Foo = 123, Bar = "hello" }); + packed = TuplePacker.Serialize(new Thing { Foo = 123, Bar = "hello" }); Assert.That(packed.ToString(), Is.EqualTo("<03><15>{<02>hello<00><00>")); - packed = FdbTuplePacker.Serialize(new Thing()); + packed = TuplePacker.Serialize(new Thing()); Assert.That(packed.ToString(), Is.EqualTo("<03><14><00><00>")); - packed = FdbTuplePacker.Serialize(default(Thing)); + packed = TuplePacker.Serialize(default(Thing)); Assert.That(packed.ToString(), Is.EqualTo("<00>")); } [Test] - public void Test_FdbTuple_Deserialize_ITupleFormattable() + public void Test_STuple_Deserialize_ITupleFormattable() { Slice slice; Thing thing; slice = Slice.Unescape("<03><16><01><02>world<00><00>"); - thing = FdbTuplePackers.DeserializeFormattable(slice); + thing = TuplePackers.DeserializeFormattable(slice); Assert.That(thing, Is.Not.Null); Assert.That(thing.Foo, Is.EqualTo(456)); Assert.That(thing.Bar, Is.EqualTo("world")); slice = Slice.Unescape("<03><14><00><00>"); - thing = FdbTuplePackers.DeserializeFormattable(slice); + thing = TuplePackers.DeserializeFormattable(slice); Assert.That(thing, Is.Not.Null); Assert.That(thing.Foo, Is.EqualTo(0)); Assert.That(thing.Bar, Is.EqualTo(null)); slice = Slice.Unescape("<00>"); - thing = FdbTuplePackers.DeserializeFormattable(slice); + thing = TuplePackers.DeserializeFormattable(slice); Assert.That(thing, Is.Null); } [Test] - public void Test_FdbTuple_BatchPack_Of_Tuples() + public void Test_STuple_BatchPack_Of_Tuples() { Slice[] slices; - var tuples = new IFdbTuple[] { - FdbTuple.Create("hello"), - FdbTuple.Create(123), - FdbTuple.Create(false), - FdbTuple.Create("world", 456, true) + var tuples = new ITuple[] { + STuple.Create("hello"), + STuple.Create(123), + STuple.Create(false), + STuple.Create("world", 456, true) }; // array version - slices = FdbTuple.Pack(tuples); + slices = STuple.Pack(tuples); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(tuples.Length)); Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); // IEnumerable version that is passed an array - slices = FdbTuple.Pack((IEnumerable)tuples); + slices = STuple.Pack((IEnumerable)tuples); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(tuples.Length)); Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); // IEnumerable version but with a "real" enumerable - slices = FdbTuple.Pack(tuples.Select(t => t)); + slices = STuple.Pack(tuples.Select(t => t)); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(tuples.Length)); Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); } [Test] - public void Test_FdbTuple_EncodeKeys_Of_T() + public void Test_STuple_EncodeKeys_Of_T() { Slice[] slices; #region PackRange(Tuple, ...) - var tuple = FdbTuple.Create("hello"); + var tuple = STuple.Create("hello"); int[] items = new int[] { 1, 2, 3, 123, -1, int.MaxValue }; // array version - slices = FdbTuple.EncodePrefixedKeys(tuple, items); + slices = STuple.EncodePrefixedKeys(tuple, items); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); // IEnumerable version that is passed an array - slices = FdbTuple.EncodePrefixedKeys(tuple, (IEnumerable)items); + slices = STuple.EncodePrefixedKeys(tuple, (IEnumerable)items); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); // IEnumerable version but with a "real" enumerable - slices = FdbTuple.EncodePrefixedKeys(tuple, items.Select(t => t)); + slices = STuple.EncodePrefixedKeys(tuple, items.Select(t => t)); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); @@ -2337,13 +2337,13 @@ public void Test_FdbTuple_EncodeKeys_Of_T() string[] words = new string[] { "hello", "world", "très bien", "断トツ", "abc\0def", null, String.Empty }; - var merged = FdbTuple.EncodePrefixedKeys(Slice.FromByte(42), words); + var merged = STuple.EncodePrefixedKeys(Slice.FromByte(42), words); Assert.That(merged, Is.Not.Null); Assert.That(merged.Length, Is.EqualTo(words.Length)); for (int i = 0; i < words.Length; i++) { - var expected = Slice.FromByte(42) + FdbTuple.EncodeKey(words[i]); + var expected = Slice.FromByte(42) + STuple.EncodeKey(words[i]); Assert.That(merged[i], Is.EqualTo(expected)); Assert.That(merged[i].Array, Is.SameAs(merged[0].Array), "All slices should be stored in the same buffer"); @@ -2351,33 +2351,33 @@ public void Test_FdbTuple_EncodeKeys_Of_T() } // corner cases - Assert.That(() => FdbTuple.EncodePrefixedKeys(Slice.Empty, default(int[])), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); - Assert.That(() => FdbTuple.EncodePrefixedKeys(Slice.Empty, default(IEnumerable)), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); + Assert.That(() => STuple.EncodePrefixedKeys(Slice.Empty, default(int[])), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); + Assert.That(() => STuple.EncodePrefixedKeys(Slice.Empty, default(IEnumerable)), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); #endregion } [Test] - public void Test_FdbTuple_EncodeKeys_Boxed() + public void Test_STuple_EncodeKeys_Boxed() { Slice[] slices; - var tuple = FdbTuple.Create("hello"); + var tuple = STuple.Create("hello"); object[] items = new object[] { "world", 123, false, Guid.NewGuid(), long.MinValue }; // array version - slices = FdbTuple.EncodePrefixedKeys(tuple, items); + slices = STuple.EncodePrefixedKeys(tuple, items); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); // IEnumerable version that is passed an array - slices = FdbTuple.EncodePrefixedKeys(tuple, (IEnumerable)items); + slices = STuple.EncodePrefixedKeys(tuple, (IEnumerable)items); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); // IEnumerable version but with a "real" enumerable - slices = FdbTuple.EncodePrefixedKeys(tuple, items.Select(t => t)); + slices = STuple.EncodePrefixedKeys(tuple, items.Select(t => t)); Assert.That(slices, Is.Not.Null); Assert.That(slices.Length, Is.EqualTo(items.Length)); Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); @@ -2385,7 +2385,7 @@ public void Test_FdbTuple_EncodeKeys_Boxed() #endregion - #region FdbTupleParser + #region TupleParser private static string Clean(string value) { @@ -2397,7 +2397,7 @@ private static string Clean(string value) return sb.ToString(); } - private static void PerformWriterTest(FdbTuplePackers.Encoder action, T value, string expectedResult, string message = null) + private static void PerformWriterTest(TuplePackers.Encoder action, T value, string expectedResult, string message = null) { var writer = new TupleWriter(); action(ref writer, value); @@ -2409,9 +2409,9 @@ private static void PerformWriterTest(FdbTuplePackers.Encoder action, T va } [Test] - public void Test_FdbTupleParser_WriteInt64() + public void Test_TupleParser_WriteInt64() { - var test = new FdbTuplePackers.Encoder(FdbTupleParser.WriteInt64); + var test = new TuplePackers.Encoder(TupleParser.WriteInt64); PerformWriterTest(test, 0L, "14"); @@ -2446,14 +2446,14 @@ public void Test_FdbTupleParser_WriteInt64() } [Test] - public void Test_FdbTupleParser_WriteInt64_Respects_Ordering() + public void Test_TupleParser_WriteInt64_Respects_Ordering() { var list = new List>(); Action test = (x) => { var writer = new TupleWriter(); - FdbTupleParser.WriteInt64(ref writer, x); + TupleParser.WriteInt64(ref writer, x); var res = new KeyValuePair(x, writer.Output.ToSlice()); list.Add(res); Log("{0,20} : {0:x16} {1}", res.Key, res.Value.ToString()); @@ -2509,9 +2509,9 @@ public void Test_FdbTupleParser_WriteInt64_Respects_Ordering() } [Test] - public void Test_FdbTupleParser_WriteUInt64() + public void Test_TupleParser_WriteUInt64() { - var test = new FdbTuplePackers.Encoder(FdbTupleParser.WriteUInt64); + var test = new TuplePackers.Encoder(TupleParser.WriteUInt64); PerformWriterTest(test, 0UL, "14"); @@ -2536,14 +2536,14 @@ public void Test_FdbTupleParser_WriteUInt64() } [Test] - public void Test_FdbTupleParser_WriteUInt64_Respects_Ordering() + public void Test_TupleParser_WriteUInt64_Respects_Ordering() { var list = new List>(); Action test = (x) => { var writer = new TupleWriter(); - FdbTupleParser.WriteUInt64(ref writer, x); + TupleParser.WriteUInt64(ref writer, x); var res = new KeyValuePair(x, writer.Output.ToSlice()); list.Add(res); #if DEBUG @@ -2584,10 +2584,10 @@ public void Test_FdbTupleParser_WriteUInt64_Respects_Ordering() } [Test] - public void Test_FdbTupleParser_WriteString() + public void Test_TupleParser_WriteString() { string s; - var test = new FdbTuplePackers.Encoder(FdbTupleParser.WriteString); + var test = new TuplePackers.Encoder(TupleParser.WriteString); Func encodeSimple = (value) => "02 " + Slice.Create(Encoding.UTF8.GetBytes(value)).ToHexaString(' ') + " 00"; Func encodeWithZeroes = (value) => "02 " + Slice.Create(Encoding.UTF8.GetBytes(value)).ToHexaString(' ').Replace("00", "00 FF") + " 00"; @@ -2667,9 +2667,9 @@ public void Test_FdbTupleParser_WriteString() } [Test] - public void Test_FdbTupleParser_WriteChar() + public void Test_TupleParser_WriteChar() { - var test = new FdbTuplePackers.Encoder(FdbTupleParser.WriteChar); + var test = new TuplePackers.Encoder(TupleParser.WriteChar); // 1 bytes PerformWriterTest(test, 'A', "02 41 00", "Unicode chars in the ASCII table take only one byte in UTF-8"); @@ -2694,7 +2694,7 @@ public void Test_FdbTupleParser_WriteChar() { char c = (char)i; var writer = new TupleWriter(); - FdbTupleParser.WriteChar(ref writer, c); + TupleParser.WriteChar(ref writer, c); string s = new string(c, 1); Assert.That(writer.Output.ToSlice().ToString(), Is.EqualTo("<02>" + Slice.Create(Encoding.UTF8.GetBytes(s)).ToString() + "<00>"), "{0} '{1}'", i, c); } @@ -2704,7 +2704,7 @@ public void Test_FdbTupleParser_WriteChar() #region Equality / Comparison - private static void AssertEquality(IFdbTuple x, IFdbTuple y) + private static void AssertEquality(ITuple x, ITuple y) { Assert.That(x.Equals(y), Is.True, "x.Equals(y)"); Assert.That(x.Equals((object)y), Is.True, "x.Equals((object)y)"); @@ -2712,7 +2712,7 @@ private static void AssertEquality(IFdbTuple x, IFdbTuple y) Assert.That(y.Equals((object)x), Is.True, "y.Equals((object)y"); } - private static void AssertInequality(IFdbTuple x, IFdbTuple y) + private static void AssertInequality(ITuple x, ITuple y) { Assert.That(x.Equals(y), Is.False, "!x.Equals(y)"); Assert.That(x.Equals((object)y), Is.False, "!x.Equals((object)y)"); @@ -2721,33 +2721,33 @@ private static void AssertInequality(IFdbTuple x, IFdbTuple y) } [Test] - public void Test_FdbTuple_Equals() + public void Test_STuple_Equals() { - var t1 = FdbTuple.Create(1, 2); + var t1 = STuple.Create(1, 2); // self equality AssertEquality(t1, t1); - var t2 = FdbTuple.Create(1, 2); + var t2 = STuple.Create(1, 2); // same type equality AssertEquality(t1, t2); - var t3 = FdbTuple.Create(new object[] { 1, 2 }); + var t3 = STuple.Create(new object[] { 1, 2 }); // other tuple type equality AssertEquality(t1, t3); - var t4 = FdbTuple.Create(1).Append(2); + var t4 = STuple.Create(1).Append(2); // multi step AssertEquality(t1, t4); } [Test] - public void Test_FdbTuple_Similar() + public void Test_STuple_Similar() { - var t1 = FdbTuple.Create(1, 2); - var t2 = FdbTuple.Create((long)1, (short)2); - var t3 = FdbTuple.Create("1", "2"); - var t4 = FdbTuple.Create(new object[] { 1, 2L }); - var t5 = FdbTuple.Unpack(Slice.Unescape("<02>1<00><15><02>")); + var t1 = STuple.Create(1, 2); + var t2 = STuple.Create((long)1, (short)2); + var t3 = STuple.Create("1", "2"); + var t4 = STuple.Create(new object[] { 1, 2L }); + var t5 = STuple.Unpack(Slice.Unescape("<02>1<00><15><02>")); AssertEquality(t1, t1); AssertEquality(t1, t2); @@ -2767,14 +2767,14 @@ public void Test_FdbTuple_Similar() } [Test] - public void Test_FdbTuple_Not_Equal() + public void Test_STuple_Not_Equal() { - var t1 = FdbTuple.Create(1, 2); + var t1 = STuple.Create(1, 2); - var x1 = FdbTuple.Create(2, 1); - var x2 = FdbTuple.Create("11", "22"); - var x3 = FdbTuple.Create(1, 2, 3); - var x4 = FdbTuple.Unpack(Slice.Unescape("<15><01>")); + var x1 = STuple.Create(2, 1); + var x2 = STuple.Create("11", "22"); + var x3 = STuple.Create(1, 2, 3); + var x4 = STuple.Unpack(Slice.Unescape("<15><01>")); AssertInequality(t1, x1); AssertInequality(t1, x2); @@ -2790,47 +2790,47 @@ public void Test_FdbTuple_Not_Equal() } [Test] - public void Test_FdbTuple_Substring_Equality() + public void Test_STuple_Substring_Equality() { - var x = FdbTuple.FromArray(new [] { "A", "C" }); - var y = FdbTuple.FromArray(new[] { "A", "B", "C" }); + var x = STuple.FromArray(new [] { "A", "C" }); + var y = STuple.FromArray(new[] { "A", "B", "C" }); Assert.That(x.Substring(0, 1), Is.EqualTo(y.Substring(0, 1))); Assert.That(x.Substring(1, 1), Is.EqualTo(y.Substring(2, 1))); - var aa = FdbTuple.Create("A"); - var bb = FdbTuple.Create("A"); + var aa = STuple.Create("A"); + var bb = STuple.Create("A"); Assert.That(aa == bb, Is.True); var a = x.Substring(0, 1); var b = y.Substring(0, 1); - Assert.That(a.Equals((IFdbTuple)b), Is.True); + Assert.That(a.Equals((ITuple)b), Is.True); Assert.That(a.Equals((object)b), Is.True); Assert.That(object.Equals(a, b), Is.True); - Assert.That(FdbTuple.Equals(a, b), Is.True); - Assert.That(FdbTuple.Equivalent(a, b), Is.True); + Assert.That(STuple.Equals(a, b), Is.True); + Assert.That(STuple.Equivalent(a, b), Is.True); - // this is very unfortunate, but 'a == b' does NOT work because IFdbTuple is an interface, and there is no known way to make it work :( + // this is very unfortunate, but 'a == b' does NOT work because ITuple is an interface, and there is no known way to make it work :( //Assert.That(a == b, Is.True); } [Test] - public void Test_FdbTuple_String_AutoCast() + public void Test_STuple_String_AutoCast() { // 'a' ~= "A" - AssertEquality(FdbTuple.Create("A"), FdbTuple.Create('A')); - AssertInequality(FdbTuple.Create("A"), FdbTuple.Create('B')); - AssertInequality(FdbTuple.Create("A"), FdbTuple.Create('a')); + AssertEquality(STuple.Create("A"), STuple.Create('A')); + AssertInequality(STuple.Create("A"), STuple.Create('B')); + AssertInequality(STuple.Create("A"), STuple.Create('a')); // ASCII ~= Unicode - AssertEquality(FdbTuple.Create("ABC"), FdbTuple.Create(Slice.FromAscii("ABC"))); - AssertInequality(FdbTuple.Create("ABC"), FdbTuple.Create(Slice.FromAscii("DEF"))); - AssertInequality(FdbTuple.Create("ABC"), FdbTuple.Create(Slice.FromAscii("abc"))); + AssertEquality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("ABC"))); + AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("DEF"))); + AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("abc"))); // 'a' ~= ASCII 'a' - AssertEquality(FdbTuple.Create(Slice.FromAscii("A")), FdbTuple.Create('A')); - AssertInequality(FdbTuple.Create(Slice.FromAscii("A")), FdbTuple.Create('B')); - AssertInequality(FdbTuple.Create(Slice.FromAscii("A")), FdbTuple.Create('a')); + AssertEquality(STuple.Create(Slice.FromAscii("A")), STuple.Create('A')); + AssertInequality(STuple.Create(Slice.FromAscii("A")), STuple.Create('B')); + AssertInequality(STuple.Create(Slice.FromAscii("A")), STuple.Create('a')); } #endregion @@ -2838,42 +2838,42 @@ public void Test_FdbTuple_String_AutoCast() #region Formatters [Test] - public void Test_Default_FdbTupleFormatter_For_Common_Types() + public void Test_Default_TupleFormatter_For_Common_Types() { // common simple types - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); // corner cases - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); // ITupleFormattable types - Assert.That(FdbTupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); } [Test] public void Test_Format_Common_Types() { - Assert.That(FdbTupleFormatter.Default.ToTuple(123), Is.EqualTo(FdbTuple.Create(123))); - Assert.That(FdbTupleFormatter.Default.FromTuple(FdbTuple.Create(123)), Is.EqualTo(123)); + Assert.That(TupleFormatter.Default.ToTuple(123), Is.EqualTo(STuple.Create(123))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(123)), Is.EqualTo(123)); - Assert.That(FdbTupleFormatter.Default.ToTuple(true), Is.EqualTo(FdbTuple.Create(true))); - Assert.That(FdbTupleFormatter.Default.FromTuple(FdbTuple.Create(true)), Is.True); + Assert.That(TupleFormatter.Default.ToTuple(true), Is.EqualTo(STuple.Create(true))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(true)), Is.True); - Assert.That(FdbTupleFormatter.Default.ToTuple("hello"), Is.EqualTo(FdbTuple.Create("hello"))); - Assert.That(FdbTupleFormatter.Default.FromTuple(FdbTuple.Create("hello")), Is.EqualTo("hello")); + Assert.That(TupleFormatter.Default.ToTuple("hello"), Is.EqualTo(STuple.Create("hello"))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create("hello")), Is.EqualTo("hello")); - var t = FdbTuple.Create(new object[] { "hello", 123, false }); - Assert.That(FdbTupleFormatter.Default.ToTuple(t), Is.SameAs(t)); - Assert.That(FdbTupleFormatter.Default.FromTuple(t), Is.SameAs(t)); + var t = STuple.Create(new object[] { "hello", 123, false }); + Assert.That(TupleFormatter.Default.ToTuple(t), Is.SameAs(t)); + Assert.That(TupleFormatter.Default.FromTuple(t), Is.SameAs(t)); var thing = new Thing { Foo = 123, Bar = "hello" }; - Assert.That(FdbTupleFormatter.Default.ToTuple(thing), Is.EqualTo(FdbTuple.Create(123, "hello"))); + Assert.That(TupleFormatter.Default.ToTuple(thing), Is.EqualTo(STuple.Create(123, "hello"))); - var thing2 = FdbTupleFormatter.Default.FromTuple(FdbTuple.Create(456, "world")); + var thing2 = TupleFormatter.Default.FromTuple(STuple.Create(456, "world")); Assert.That(thing2, Is.Not.Null); Assert.That(thing2.Foo, Is.EqualTo(456)); Assert.That(thing2.Bar, Is.EqualTo("world")); @@ -2885,24 +2885,24 @@ public void Test_Create_Appender_Formatter() { // create an appender formatter that will always add the values after the same prefix - var fmtr = FdbTupleFormatter.CreateAppender(FdbTuple.Create("hello", "world")); - Assert.That(fmtr, Is.InstanceOf>()); + var fmtr = TupleFormatter.CreateAppender(STuple.Create("hello", "world")); + Assert.That(fmtr, Is.InstanceOf>()); - Assert.That(fmtr.ToTuple(123), Is.EqualTo(FdbTuple.Create("hello", "world", 123))); - Assert.That(fmtr.ToTuple(456), Is.EqualTo(FdbTuple.Create("hello", "world", 456))); - Assert.That(fmtr.ToTuple(-1), Is.EqualTo(FdbTuple.Create("hello", "world", -1))); + Assert.That(fmtr.ToTuple(123), Is.EqualTo(STuple.Create("hello", "world", 123))); + Assert.That(fmtr.ToTuple(456), Is.EqualTo(STuple.Create("hello", "world", 456))); + Assert.That(fmtr.ToTuple(-1), Is.EqualTo(STuple.Create("hello", "world", -1))); - Assert.That(fmtr.FromTuple(FdbTuple.Create("hello", "world", 42)), Is.EqualTo(42)); - Assert.That(fmtr.FromTuple(FdbTuple.Create("hello", "world", -1)), Is.EqualTo(-1)); + Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", 42)), Is.EqualTo(42)); + Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", -1)), Is.EqualTo(-1)); Assert.That(() => fmtr.FromTuple(null), Throws.InstanceOf()); - Assert.That(() => fmtr.FromTuple(FdbTuple.Empty), Throws.InstanceOf()); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create("hello", "world", 42, 77)), Throws.InstanceOf(), "Too many values"); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create("hello_world", 42)), Throws.InstanceOf(), "not enough values"); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create("world", "hello", "42")), Throws.InstanceOf(), "incorrect type"); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create(42)), Throws.InstanceOf(), "missing prefix"); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create("extra", "hello", "world", 42)), Throws.InstanceOf(), "prefix must match exactly"); - Assert.That(() => fmtr.FromTuple(FdbTuple.Create("Hello", "World", 42)), Throws.InstanceOf(), "case sensitive"); + Assert.That(() => fmtr.FromTuple(STuple.Empty), Throws.InstanceOf()); + Assert.That(() => fmtr.FromTuple(STuple.Create("hello", "world", 42, 77)), Throws.InstanceOf(), "Too many values"); + Assert.That(() => fmtr.FromTuple(STuple.Create("hello_world", 42)), Throws.InstanceOf(), "not enough values"); + Assert.That(() => fmtr.FromTuple(STuple.Create("world", "hello", "42")), Throws.InstanceOf(), "incorrect type"); + Assert.That(() => fmtr.FromTuple(STuple.Create(42)), Throws.InstanceOf(), "missing prefix"); + Assert.That(() => fmtr.FromTuple(STuple.Create("extra", "hello", "world", 42)), Throws.InstanceOf(), "prefix must match exactly"); + Assert.That(() => fmtr.FromTuple(STuple.Create("Hello", "World", 42)), Throws.InstanceOf(), "case sensitive"); } #endregion @@ -2910,7 +2910,7 @@ public void Test_Create_Appender_Formatter() #region Bench.... [Test] - public void Bench_FdbTuple_Unpack_Random() + public void Bench_STuple_Unpack_Random() { const int N = 100 * 1000; @@ -2919,7 +2919,7 @@ public void Bench_FdbTuple_Unpack_Random() string UNICODE_STRING = "héllø 世界"; Console.Write("Creating {0:N0} random tuples", N); - var tuples = new List(N); + var tuples = new List(N); var rnd = new Random(777); var guids = Enumerable.Range(0, 10).Select(_ => Guid.NewGuid()).ToArray(); var uuid128s = Enumerable.Range(0, 10).Select(_ => Uuid128.NewUuid()).ToArray(); @@ -2928,7 +2928,7 @@ public void Bench_FdbTuple_Unpack_Random() var sw = Stopwatch.StartNew(); for (int i = 0; i < N; i++) { - IFdbTuple tuple = FdbTuple.Empty; + ITuple tuple = STuple.Empty; int s = 1 + (int)Math.Sqrt(rnd.Next(128)); if (i % (N / 100) == 0) Console.Write("."); for (int j = 0; j < s; j++) @@ -2964,7 +2964,7 @@ public void Bench_FdbTuple_Unpack_Random() Console.Write("Packing tuples..."); sw.Restart(); - var slices = FdbTuple.Pack(tuples); + var slices = STuple.Pack(tuples); sw.Stop(); Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); @@ -2974,7 +2974,7 @@ public void Bench_FdbTuple_Unpack_Random() Console.Write("Unpacking tuples..."); sw.Restart(); - var unpacked = slices.Select(slice => FdbTuple.Unpack(slice)).ToList(); + var unpacked = slices.Select(slice => STuple.Unpack(slice)).ToList(); sw.Stop(); Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); @@ -3023,12 +3023,12 @@ public Thing() public int Foo { get; set; } public string Bar { get; set; } - IFdbTuple ITupleFormattable.ToTuple() + ITuple ITupleFormattable.ToTuple() { - return FdbTuple.Create(this.Foo, this.Bar); + return STuple.Create(this.Foo, this.Bar); } - void ITupleFormattable.FromTuple(IFdbTuple tuple) + void ITupleFormattable.FromTuple(ITuple tuple) { this.Foo = tuple.Get(0); this.Bar = tuple.Get(1); diff --git a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs index babd12852..185650f49 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs @@ -967,7 +967,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() // record the timing and call history to ensure that inner is called at least twice before the first item gets out - Func> record = (x) => FdbTuple.Create(x, Volatile.Read(ref called)); + Func> record = (x) => STuple.Create(x, Volatile.Read(ref called)); // without prefetching, the number of calls should match for the producer and the consumer called = 0; @@ -1012,9 +1012,9 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() return Task.FromResult(Maybe.Return((int)index)); }); - Func> record = (x) => + Func> record = (x) => { - var res = FdbTuple.Create(x, Volatile.Read(ref called), sw.Elapsed); + var res = STuple.Create(x, Volatile.Read(ref called), sw.Elapsed); sw.Restart(); return res; }; diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index 812cf6d45..e69c99ae5 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -40,8 +40,8 @@ namespace FoundationDB.Linq.Expressions.Tests public class FdbQueryExpressionFacts { - private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", FdbSubspace.Create(FdbTuple.EncodeKey("Foos", 1))); - private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", FdbSubspace.Create(FdbTuple.EncodeKey("Foos", 2))); + private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", FdbSubspace.Create(STuple.EncodeKey("Foos", 1))); + private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", FdbSubspace.Create(STuple.EncodeKey("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() @@ -92,7 +92,7 @@ public void Test_FdbQueryIndexLookupExpression_From_Lambda() public void Test_FdbQueryRangeExpression() { var expr = FdbQueryExpressions.Range( - FdbTuple.Create("Foo").ToSelectorPair() + STuple.Create("Foo").ToSelectorPair() ); Console.WriteLine(expr); @@ -170,7 +170,7 @@ public void Test_FdbQueryUnionExpression() public void Test_FdbQueryTransformExpression() { var expr = FdbQueryExpressions.Transform( - FdbQueryExpressions.RangeStartsWith(FdbTuple.Create("Hello", "World")), + FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), (kvp) => kvp.Value.ToUnicode() ); Console.WriteLine(expr); @@ -189,7 +189,7 @@ public void Test_FdbQueryTransformExpression() public void Test_FdbQueryFilterExpression() { var expr = FdbQueryExpressions.Filter( - FdbQueryExpressions.RangeStartsWith(FdbTuple.Create("Hello", "World")), + FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), (kvp) => kvp.Value.ToInt32() % 2 == 0 ); Console.WriteLine(expr); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 80a14d469..5dd064b14 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -502,7 +502,7 @@ public async Task Test_Can_MergeSort() { for (int i = 0; i < N; i++) { - tr.Set(lists[k].Keys.Encode((i * K) + k), FdbTuple.EncodeKey(k, i)); + tr.Set(lists[k].Keys.Encode((i * K) + k), STuple.EncodeKey(k, i)); } await tr.CommitAsync(); } @@ -527,8 +527,8 @@ public async Task Test_Can_MergeSort() for (int i = 0; i < K * N; i++) { - Assert.That(location.ExtractKey(results[i].Key), Is.EqualTo(FdbTuple.EncodeKey(i % K, i))); - Assert.That(results[i].Value, Is.EqualTo(FdbTuple.EncodeKey(i % K, i / K))); + Assert.That(location.ExtractKey(results[i].Key), Is.EqualTo(STuple.EncodeKey(i % K, i))); + Assert.That(results[i].Value, Is.EqualTo(STuple.EncodeKey(i % K, i / K))); } } } @@ -567,7 +567,7 @@ public async Task Test_Range_Intersect() for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); - var value = FdbTuple.EncodeKey(k, i); + var value = STuple.EncodeKey(k, i); //Console.WriteLine("> " + key + " = " + value); tr.Set(key, value); } @@ -639,7 +639,7 @@ public async Task Test_Range_Except() for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); - var value = FdbTuple.EncodeKey(k, i); + var value = STuple.EncodeKey(k, i); //Console.WriteLine("> " + key + " = " + value); tr.Set(key, value); } @@ -712,8 +712,8 @@ await db.WriteAsync((tr) => { var query = tr.Except( new[] { locItems.Keys.ToRange(), locProcessed.Keys.ToRange() }, - (kv) => FdbTuple.Unpack(kv.Key).Substring(-2), // note: keys come from any of the two ranges, so we must only keep the last 2 elements of the tuple - FdbTupleComparisons.Composite() // compares t[0] as a string, and t[1] as an int + (kv) => STuple.Unpack(kv.Key).Substring(-2), // note: keys come from any of the two ranges, so we must only keep the last 2 elements of the tuple + TupleComparisons.Composite() // compares t[0] as a string, and t[1] as an int ); // problem: Except() still returns the original (Slice,Slice) pairs from the first range, @@ -726,8 +726,8 @@ await db.WriteAsync((tr) => Trace.WriteLine(r); } Assert.That(results.Count, Is.EqualTo(2)); - Assert.That(results[0], Is.EqualTo(FdbTuple.Create("userA", 10093))); - Assert.That(results[1], Is.EqualTo(FdbTuple.Create("userB", 20003))); + Assert.That(results[0], Is.EqualTo(STuple.Create("userA", 10093))); + Assert.That(results[1], Is.EqualTo(STuple.Create("userB", 20003))); // Second Method: pre-parse the queries, and merge on the results directly Trace.WriteLine("Method 2:"); @@ -742,7 +742,7 @@ await db.WriteAsync((tr) => .Select(kv => locProcessed.Keys.Unpack(kv.Key)); // items and processed are lists of (string, int) tuples, we can compare them directly - var query = items.Except(processed, FdbTupleComparisons.Composite()); + var query = items.Except(processed, TupleComparisons.Composite()); // query is already a list of tuples, nothing more to do return query; @@ -753,8 +753,8 @@ await db.WriteAsync((tr) => Trace.WriteLine(r); } Assert.That(results.Count, Is.EqualTo(2)); - Assert.That(results[0], Is.EqualTo(FdbTuple.Create("userA", 10093))); - Assert.That(results[1], Is.EqualTo(FdbTuple.Create("userB", 20003))); + Assert.That(results[0], Is.EqualTo(STuple.Create("userA", 10093))); + Assert.That(results[1], Is.EqualTo(STuple.Create("userB", 20003))); } diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 33b74b11c..8bd26db3a 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -68,7 +68,7 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(subspace.Keys.Encode(123).ToString(), Is.EqualTo("*<00><7F><15>{")); Assert.That(subspace.Keys.Encode("hello").ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); Assert.That(subspace.Keys.Encode(Slice.FromAscii("world")).ToString(), Is.EqualTo("*<00><7F><01>world<00>")); - Assert.That(subspace.Keys.Pack(FdbTuple.Create("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); + Assert.That(subspace.Keys.Pack(STuple.Create("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); // if we derive a tuple from this subspace, it should keep the binary prefix when converted to a key var t = subspace.Keys.Append("world", 123, false); @@ -118,7 +118,7 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = FdbSubspace.CreateDynamic(FdbTuple.Create("hello")); + var subspace = FdbSubspace.CreateDynamic(STuple.Create("hello")); Assert.That(subspace.Key.ToString(), Is.EqualTo("<02>hello<00>")); Assert.That(FdbSubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -208,7 +208,7 @@ public void Test_Subspace_Partitioning_With_Tuple_Suffix() Assert.That(t2.ToSlice(), Is.EqualTo(t1.ToSlice())); // cornercase - Assert.That(child.Partition[FdbTuple.Empty].Key, Is.EqualTo(child.Key)); + Assert.That(child.Partition[STuple.Empty].Key, Is.EqualTo(child.Key)); } diff --git a/README.md b/README.md index b4205492f..616f0e679 100644 --- a/README.md +++ b/README.md @@ -109,14 +109,14 @@ using (var db = await Fdb.OpenAsync()) // and 5 bytes). The values are raw slices, which means that your // application MUST KNOW that they are strings in order to decode // them. If you wan't any tool to be able to find out the type of - // your values, you can also use FdbTuple.Pack("AAA") to create + // your values, you can also use STuple.Pack("AAA") to create // the values, at the cost of 2 extra bytes per entry. // This is always a good idea to maintain a counter of keys in our array. // The cheapest way to do that, is to reuse the subspace key itself, which // is 'in' the subspace, but not 'inside': trans.Set(list.Key, Slice.FromFixed32(3)); - // We could use FdbTuple.Pack(3) here, but have a fixed size counter + // We could use STuple.Pack(3) here, but have a fixed size counter // makes it easy to use AtomicAdd(...) to increment (or decrement) the value // when adding or removing entries in the array. @@ -277,7 +277,7 @@ There were a few design goals, that you may agree with or not: However, there are some key differences between Python and .NET that may cause problems: * Python's dynamic types and auto casting of Tuples values, are difficult to model in .NET (without relying on the DLR). The Tuple implementation try to be as dynamic as possible, but if you want to be safe, please try to only use strings, longs, booleans and byte[] to be 100% compatible with other bindings. You should refrain from using the untyped `tuple[index]` indexer (that returns an object), and instead use the generic `tuple.Get(index)` that will try to adapt the underlying type into a T. -* The Tuple layer uses ASCII and Unicode strings, while .NET only have Unicode strings. That means that all strings in .NET will be packed with prefix type 0x02 and byte arrays with prefix type 0x01. An ASCII string packed in Python will be seen as a byte[] unless you use `IFdbTuple.Get()` that will automatically convert it to Unicode. +* The Tuple layer uses ASCII and Unicode strings, while .NET only have Unicode strings. That means that all strings in .NET will be packed with prefix type 0x02 and byte arrays with prefix type 0x01. An ASCII string packed in Python will be seen as a byte[] unless you use `ITuple.Get()` that will automatically convert it to Unicode. * There is no dedicated 'UUID' type prefix, so that means that System.Guid would be serialized as byte arrays, and all instances of byte 0 would need to be escaped. Since `System.Guid` are frequently used as primary keys, I added a new custom type prefix (0x30) for 128-bits UUIDs and (0x31) for 64-bits UUIDs. This simplifies packing/unpacking and speeds up writing/reading/comparing Guid keys. The following files will be required by your application diff --git a/Tuples.md b/Tuples.md index d1f9f855d..f73ce47d0 100644 --- a/Tuples.md +++ b/Tuples.md @@ -68,13 +68,13 @@ And quite frankly, if you have used other languages where tuples are first-class That's why we need a better API, in order to help us be more productive. -## IFdbTuple +## ITuple -The `IFdbTuple` interface, defined in `FoundationDB.Layers.Tuples` (TODO: update this if we rename it!), is the base of all the different tuples implementation, all targetting a specific use case. +The `ITuple` interface, defined in `FoundationDB.Layers.Tuples` (TODO: update this if we rename it!), is the base of all the different tuples implementation, all targetting a specific use case. This interface has the bare minimum API, thats must be implemented by each variant, and is in turn used by a set of extension methods that add more generic behavior that does NOT need to be replicated in all the variants. -There is also a static class, called `FdbTuple`, which holds a bunch of methods to create and handle all the different variants of tuples. +There is also a static class, called `STuple`, which holds a bunch of methods to create and handle all the different variants of tuples. _note: the interface is not called `ITuple` because 1) there is already an `ITuple` interface in the BCL (even though it is internal), and 2) we wouldn't be able to call our static helper class `Tuple` since it would collide with the BCL._ @@ -82,55 +82,55 @@ _note: the interface is not called `ITuple` because 1) there is already an `ITup Tuples need to adapt to different use case: some tuples should have a fixed size and types (like the BCL Tuples), some should have a variable length (like a vector or list). Some tuples should probably be structs (to reduce the number of allocation in tight loops), while others need to be reference types. And finally, some tuples could be thin wrappers around encoded binary blobs, and defer the decoding of items until they are accessed. -That's why there is multiple variants of tuples, all implementing the `IFdbTuple` interface: +That's why there is multiple variants of tuples, all implementing the `ITuple` interface: -- `FdbTuple`, `FdbTuple` (up to T5 right now) are the equivalent of the BCL's `Tuple` except that they are implemented as a struct. They are efficient when used as a temporary step to create bigger tuples, or when you have control of the actual type (in LINQ queries, inside your own private methods, ...). They are also ideal if you want type safety and nice intellisense support, since the types are known at compile time. -- `FdbListTuple` wraps an array of object[] and exposes a subset of this array. Getting a substring of this cheap since it does not have to copy the items. -- `FdbJoinedTuple` is a wrapper that glues together two tuples (of any type). -- `FdbLinkedTuple` is a special case of an FdbJoinedTupel, where we are only adding one value to an existing tuple. -- `FdbSlicedTuple` is a wrapper around a half-parsed binary representation of a tuple, and which will only decode items if they are accessed. In cases where you are only interested in part of a key, you won't waste CPU cycles decoding the other items. -- `FdbMemoizedTuple` will cache its binary representation, which is usefull when you have a common tuple prefix which is used everytime to construct other tuples. -- `FdbPrefixedTuple` is some sort of hybrid tuples whose binary representation always have a constant binary prefix, which may or may not be a valid binary tuple representation itself (need to use tuples with prefixes generated from a different encoding). +- `STuple`, `STuple` (up to T5 right now) are the equivalent of the BCL's `Tuple` except that they are implemented as a struct. They are efficient when used as a temporary step to create bigger tuples, or when you have control of the actual type (in LINQ queries, inside your own private methods, ...). They are also ideal if you want type safety and nice intellisense support, since the types are known at compile time. +- `ListTuple` wraps an array of object[] and exposes a subset of this array. Getting a substring of this cheap since it does not have to copy the items. +- `JoinedTuple` is a wrapper that glues together two tuples (of any type). +- `LinkedTuple` is a special case of an FdbJoinedTupel, where we are only adding one value to an existing tuple. +- `SlicedTuple` is a wrapper around a half-parsed binary representation of a tuple, and which will only decode items if they are accessed. In cases where you are only interested in part of a key, you won't waste CPU cycles decoding the other items. +- `MemoizedTuple` will cache its binary representation, which is usefull when you have a common tuple prefix which is used everytime to construct other tuples. +- `PrefixedTuple` is some sort of hybrid tuples whose binary representation always have a constant binary prefix, which may or may not be a valid binary tuple representation itself (need to use tuples with prefixes generated from a different encoding). ### Creating a tuple The most simple way to create a tuple, is from its elements: ```CSharp -var t = FdbTuple.Create("Hello", 123, Guid.NewGuid()); +var t = STuple.Create("Hello", 123, Guid.NewGuid()); ``` -The actual type of the tuple will be `FdbTuple` which is a struct. Since we are using the `var` keyword, then as long as `t` stays inside the method, it will not be boxed. +The actual type of the tuple will be `STuple` which is a struct. Since we are using the `var` keyword, then as long as `t` stays inside the method, it will not be boxed. We can also create a tuple by adding something to an existing tuples, even starting with the Empty tuple: ```CSharp -var t = FdbTuple.Empty.Append("Hello").Append(123).Append(Guid.NewGuid()); +var t = STuple.Empty.Append("Hello").Append(123).Append(Guid.NewGuid()); ``` -The good news here is that _t_ is still a struct of type `FdbTuple` and we did not produce any allocations: the Empty tuple is a singleton, and all the intermediate Append() returned structs of type `FdbTuple` and `FdbTuple`. There is of course a limit to the number of elements that can be added, before we have to switch to an array-based tuple variant. +The good news here is that _t_ is still a struct of type `STuple` and we did not produce any allocations: the Empty tuple is a singleton, and all the intermediate Append() returned structs of type `STuple` and `STuple`. There is of course a limit to the number of elements that can be added, before we have to switch to an array-based tuple variant. If we have a variable-size list of items, we can also create a tuple from it: ```CSharp IEnumerable xs = ....; // xs is a sequence of MyFoo objects, with an Id property (of type Guid) -var t = FdbTuple.FromSequence(xs.Select(x => x.Id)); +var t = STuple.FromSequence(xs.Select(x => x.Id)); ``` When all the elements or a tuple are of the same type, you can use specialized versions: ```CSharp var xs = new [] { "Bonjour", "le", "Monde!" }; -var t = FdbTuple.FromArray(xs); +var t = STuple.FromArray(xs); ``` If you were already using the BCL's Tuple, you can easily convert from one to the other, via a set of implicit and explicit cast operators: ```CSharp var bcl = Tuple.Create("Hello", 123, Guid.NewGuid()); -FdbTuple t = bcl; // implicit cast +STuple t = bcl; // implicit cast -var t = FdbTuple.Create("Hello", 123, Guid.NewGuid()); +var t = STuple.Create("Hello", 123, Guid.NewGuid()); Tuple bcl = (Tuple) t; // explicit cast ``` @@ -138,8 +138,8 @@ And for the more adventurous, you can of course create a tuple by copying the el ```CSharp var xs = new object[] { "Hello", 123, Guid.NewGuid() }; -var t1 = FdbTuple.FromObjects(xs); // => ("hello", 123, guid) -var t2 = FdbTuple.FromObjects(xs, 1, 2); // => (123, guid) +var t1 = STuple.FromObjects(xs); // => ("hello", 123, guid) +var t2 = STuple.FromObjects(xs, 1, 2); // => (123, guid) xs[1] = 456; // won't change the content of the tuples // t[1] => 123 ``` @@ -148,8 +148,8 @@ If you really want to push it, you can skip copying the items by wrapping an exi ```CSharp var xs = new object[] { "Hello", 123, Guid.NewGuid() }; -var t1 = FdbTuple.Wrap(xs); // no copy! -var t2 = FdbTuple.Wrap(xs, 1, 2); // no copy! +var t1 = STuple.Wrap(xs); // no copy! +var t2 = STuple.Wrap(xs, 1, 2); // no copy! xs[1] = 456; // will change the content of the tuples!! // t[1] => 456 ``` @@ -166,12 +166,12 @@ To help you verify that a tuple has the correct size before accessing its elemen - `t.OfSize(3)` checks that `t` is not null, and that `t.Count` is equal to 3, and then returns the tuple itself, so you can write: `t.OfSize(3).DoSomethingWichExceptsThreeElements()` - `t.OfSizeAtLeast(3)` (and `t.OfSizeAtMost(3)`) work the same, except they check that `t.Count >= 3` (or `t.Count <= 3`) -Of course, if you have one of the `FdbTuple` struct, you can skip this step, since the size if known at compile time. +Of course, if you have one of the `STuple` struct, you can skip this step, since the size if known at compile time. To read the content of a tuple, you can simply call `t.Get(index)`, where `index` is the offset _in the tuple_ of the element, and `T` is the type into which the value will be converted. ```CSharp -var t = FdbTuple.Create("hello", 123, Guid.NewGuid()); +var t = STuple.Create("hello", 123, Guid.NewGuid()); var x = t.Get(0); // => "hello" var y = t.Get(1); // => 123 var z = t.Get(2); // => guid @@ -180,7 +180,7 @@ var z = t.Get(2); // => guid If `index` is negative, then it is relative to the end of the tuple, where -1 is the last element, -2 is the next-to-last element, and -N is the first element. ```CSharp -var t = FdbTuple.Create("hello", 123, Guid.NewGuid()); +var t = STuple.Create("hello", 123, Guid.NewGuid()); var x = t.Get(-3); // => "hello" var y = t.Get(-2); // => 123 var z = t.Get(-1); // => guid @@ -193,13 +193,13 @@ Code that manipulate tuples can get complex pretty fast, so you need a way to di For that, every tuple overrides `ToString()` to return a nicely formatted string with a standardized format. ```CSharp -var t1 = FdbTuple.Create("hello", 123, Guid.NewGuid()); +var t1 = STuple.Create("hello", 123, Guid.NewGuid()); Console.WriteLine("t1 = {0}", t1); // => t1 = ("hello", 123, {773166b7-de74-4fcc-845c-84080cc89533}) -var t2 = FdbTuple.Create("hello"); +var t2 = STuple.Create("hello"); Console.WriteLine("t1 = {0}", t2); // => t2 = ("hello",) -var t3 = FdbTuple.Empty; +var t3 = STuple.Empty; Console.WriteLine("t3 = {0}", t3); // => t3 = () ``` @@ -213,11 +213,11 @@ Since a tuple is just a vector of elements, you can of course put a tuple inside This works: ```CSharp -var t1 = FdbTuple.Create("hello", FdbTuple(123, 456), Guid.NewGuid()); +var t1 = STuple.Create("hello", STuple(123, 456), Guid.NewGuid()); // t1 = ("hello", (123, 456), {773166b7-de74-4fcc-845c-84080cc89533}) -var t2 = FdbTuple.Create(FdbTuple.Create("a", "b")); +var t2 = STuple.Create(STuple.Create("a", "b")); // t2 = ((a, b),) -var t3 = FdbTuple.Create("hello", FdbTuple.Empty, "world"); +var t3 = STuple.Create("hello", STuple.Empty, "world"); // t3 = ("hello", (), "world"); ``` @@ -230,7 +230,7 @@ var productId = "B00CS8QSSK"; var locationId = new [] { "Europe", "France", "Lille" }; var orderId = Guid.NewGuid(); -var t = FdbTuple.Create(productId, FdbTuple.FromArray(locationId), orderId); +var t = STuple.Create(productId, STuple.FromArray(locationId), orderId); // t.Count => 3 // t[0] => "B00CS8QSSK" // t[1] => ("Europe", "France", "Lille") @@ -248,7 +248,7 @@ You can, though, modify tuples by returning a new tuple, with or without copying The most common case is to simply add a value to a tuple via the `t.Append(T value)` method. For example you have a base tuple (cached value), and you want to add a document ID. ```CSharp -var location = FdbTuple.Create("MyAwesomeApp", "Documents"); +var location = STuple.Create("MyAwesomeApp", "Documents"); var documentId = Guid.NewGuid(); var t = location.Append(document); @@ -260,11 +260,11 @@ Don't forget that if you Append a tuple, it will be added as a nested tuple! If you actually want to merge the elements of two tuples, when you can use the `t1.Concat(t2)` method, which return a new tuple with the elements of both t1 and t2. ```CSharp -var location = FdbTuple.Create("MyAwesomeApp", "OrdersByProduct"); +var location = STuple.Create("MyAwesomeApp", "OrdersByProduct"); var productId = "B00CS8QSSK"; var orderId = Guid.NewGuid(); -var t1 = FdbTuple.Create(productId, orderId) +var t1 = STuple.Create(productId, orderId) // t1 => ("B00CS8QSSK", {773166b7-de74-4fcc-845c-84080cc89533}) var t2 = location.Concat(t1); @@ -280,7 +280,7 @@ First, you can return a subset of a tuple via on of the `t.Substring(...)` metho The `Substring()` method works exactly the same way as for regulard strings. ```CSharp -var t = FdbTuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); +var t = STuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); var u = t.Substring(0, 3); // => (1, 2, 3) var v = t.Substring(5, 2); // => (6, 7) var w = t.Substring(7); // => (8, 9, 10) @@ -292,7 +292,7 @@ var w = v.Substring(-3); // => (8, 9, 10) The `t[from, to]` indexer gets some getting used to. If actual returns all the elements in the tuple with position `from <= p < to`, which means that the `to` is excluded. ```CSharp -var t = FdbTuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); +var t = STuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); var u = t[0, 3]; // => (1, 2, 3) var v = t[5, 7]; // => (6, 7) // rember that 'to' is excluded! @@ -307,7 +307,7 @@ var w = v[-3, null]; // => (8, 9, 10) If you are tired of writing `t.Substring(0, 3)` all the time, you can also use `t.Truncate(3)` which does the same thing. ```CSharp -var t = FdbTuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); +var t = STuple.Create(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); var u = t.Truncate(3); // u => (1, 2, 3); var v = t.Truncate(-3); @@ -319,7 +319,7 @@ var v = t.Truncate(-3); When decoding keys using tuple, you wil often find yourself extracting a fixed number of arguments into local variables, and then constructing an instance of a Model class from your application. ```CSharp -public MyFooBar DecodeFoobar(IFdbTuple tuple) +public MyFooBar DecodeFoobar(ITuple tuple) { var x = tuple.Get(0); var y = tuple.Get(1); @@ -335,10 +335,10 @@ The keen eye will see the problems with this method: - what if tuple.Count is only 2 ? - you probably copy/pasted `var x = tuple.Get<...>(0)` two more times, and forgot to change the index to 1 and 2! _(even Notch does it!)_ -One solution is to use the set of `t.As()` helper methods to convert a tuple of type `IFdbTuple` into a more friendly `FdbTuple` introducing tape safety and intellisence. +One solution is to use the set of `t.As()` helper methods to convert a tuple of type `ITuple` into a more friendly `STuple` introducing tape safety and intellisence. ```CSharp -public MyFooBar DecodeFoobar(IFdbTuple tuple) +public MyFooBar DecodeFoobar(ITuple tuple) { var t = tuple.As(); // this throws if tuple is null, or not of size 3 @@ -351,7 +351,7 @@ That's better, but you can still swap two arguments by mistake, if they have the To combat this, you can use on of the `t.With(Action)` or `t.With(Func)` which can give names to the elements. ```CSharp -public MyFooBar DecodeFoobar(IFdbTuple tuple) +public MyFooBar DecodeFoobar(ITuple tuple) { return tuple.With((Guid productId, Guid categoryId, Guid orderId) => new MyFooBar(productId, categoriyId, orderId)); // all three elements are GUID, but adding name help you catch argument inversion errors From 53ec71244814843c49593515f555f2ce69e06689 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 12:35:15 +0200 Subject: [PATCH 064/153] Refactored and renamed IFdbSubspaces into IKeySpaces - "KeySpace" because it deals with keys only (not values!) - IDynamicKeySubspace for keys with variable shape (usually encoded as tuples) - ITypedKeySubspace for keys with fixed types and number of elements - IFdbSubspace => IKeySubspace, and changed Key property into GetPrrefix() method - EncoderSubspace => TypedKeySubspace --- FdbShell/Commands/BasicCommands.cs | 20 +- FoundationDB.Client/Fdb.cs | 12 +- FoundationDB.Client/FdbCluster.cs | 4 +- FoundationDB.Client/FdbDatabase.cs | 46 +-- FoundationDB.Client/FdbRangeChunk.cs | 4 +- .../Filters/FdbDatabaseFilter.cs | 16 +- .../Filters/PrefixRewriterTransaction.cs | 6 +- .../FoundationDB.Client.csproj | 37 +-- FoundationDB.Client/IFdbCluster.cs | 2 +- FoundationDB.Client/IFdbDatabase.cs | 4 +- .../Layers/Directories/FdbDirectoryLayer.cs | 58 ++-- .../Directories/FdbDirectoryPartition.cs | 2 +- .../Directories/FdbDirectorySubspace.cs | 2 +- .../Directories/FdbHighContentionAllocator.cs | 12 +- .../Layers/Tuples/Encoding/TupleCodec`1.cs | 2 +- .../Tuples/TypeSystem/TupleKeyEncoder.cs | 2 +- .../Tuples/TypeSystem/TupleKeyEncoding.cs | 2 +- ...cSubspaceKeys.cs => DynamicKeySubspace.cs} | 280 +++++++++++++----- .../Subspaces/Fdb.Directory.cs | 4 +- .../Subspaces/FdbDatabasePartition.cs | 154 ++++------ .../Subspaces/FdbDynamicSubspace.cs | 80 ----- .../Subspaces/FdbDynamicSubspacePartition.cs | 184 ------------ .../Subspaces/FdbEncoderSubspaceKeys`1.cs | 86 ------ .../Subspaces/FdbEncoderSubspaceKeys`3.cs | 81 ----- .../Subspaces/FdbEncoderSubspaceKeys`4.cs | 87 ------ .../FdbEncoderSubspacePartition`1.cs | 82 ----- .../FdbEncoderSubspacePartition`2.cs | 86 ------ .../FdbEncoderSubspacePartition`3.cs | 87 ------ .../FdbEncoderSubspacePartition`4.cs | 88 ------ .../Subspaces/FdbEncoderSubspace`1.cs | 72 ----- .../Subspaces/FdbEncoderSubspace`2.cs | 81 ----- .../Subspaces/FdbEncoderSubspace`3.cs | 87 ------ .../Subspaces/FdbEncoderSubspace`4.cs | 88 ------ ...amicSubspace.cs => IDynamicKeySubspace.cs} | 24 +- .../{IFdbSubspace.cs => IKeySubspace.cs} | 29 +- ...ncoderSubspace.cs => ITypedKeySubspace.cs} | 34 +-- .../{FdbSubspace.cs => KeySubspace.cs} | 156 +++++----- ...Extensions.cs => KeySubspaceExtensions.cs} | 50 ++-- .../Subspaces/TypedKeySubspace`1.cs | 162 ++++++++++ .../Subspaces/TypedKeySubspace`2.cs | 166 +++++++++++ .../Subspaces/TypedKeySubspace`3.cs | 169 +++++++++++ .../Subspaces/TypedKeySubspace`4.cs | 178 +++++++++++ .../Encoders/DynamicKeyEncoderBase.cs | 2 +- .../TypeSystem/IDynamicKeyEncoder.cs | 2 +- .../{IKeyValueEncoder.cs => IKeyEncoder`1.cs} | 0 .../{IFdbKeyEncoding.cs => IKeyEncoding.cs} | 8 +- .../TypeSystem/IOrderedTypeCodec.cs | 2 + .../TypeSystem/IUnorderedTypeCodec.cs | 2 + .../{FdbTypeCodec`1.cs => TypeCodec`1.cs} | 5 +- FoundationDB.Client/TypeSystem/TypeSystem.cs | 4 +- .../Batched.cs} | 83 +++--- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 4 +- .../Collections/FdbMap`2.cs | 8 +- .../Collections/FdbMultimap`2.cs | 8 +- .../Collections/FdbQueue`1.cs | 16 +- .../Collections/FdbRankedSet.cs | 4 +- .../Collections/FdbVector`1.cs | 8 +- .../Counters/FdbCounterMap.cs | 8 +- .../Counters/FdbHighContentionCounter.cs | 6 +- .../Indexes/FdbIndex`2.cs | 8 +- .../Interning/FdbStringIntern.cs | 4 +- .../Documents/FdbDocumentCollection.cs | 8 +- .../Documents/FdbHashSetCollection.cs | 4 +- .../Indexes/FdbCompressedBitmapIndex.cs | 8 +- .../Messaging/FdbWorkerPool.cs | 20 +- .../Messaging/WorkerPoolTest.cs | 8 +- .../Benchmarks/BenchRunner.cs | 2 +- FoundationDB.Samples/Benchmarks/LeakTest.cs | 6 +- .../MessageQueue/MessageQueueRunner.cs | 2 +- .../Tutorials/ClassScheduling.cs | 2 +- FoundationDB.Tests.Sandbox/Program.cs | 2 +- FoundationDB.Tests/DatabaseFacts.cs | 30 +- FoundationDB.Tests/FdbTest.cs | 4 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 56 ++-- FoundationDB.Tests/Layers/QueuesFacts.cs | 2 +- .../Linq/FdbAsyncQueryableFacts.cs | 2 +- .../Linq/FdbQueryExpressionFacts.cs | 4 +- FoundationDB.Tests/RangeQueryFacts.cs | 4 +- FoundationDB.Tests/SubspaceFacts.cs | 56 ++-- FoundationDB.Tests/TestHelpers.cs | 12 +- FoundationDB.Tests/TransactionFacts.cs | 10 +- 81 files changed, 1427 insertions(+), 1823 deletions(-) rename FoundationDB.Client/Subspaces/{FdbDynamicSubspaceKeys.cs => DynamicKeySubspace.cs} (71%) delete mode 100644 FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs delete mode 100644 FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs rename FoundationDB.Client/Subspaces/{IFdbDynamicSubspace.cs => IDynamicKeySubspace.cs} (58%) rename FoundationDB.Client/Subspaces/{IFdbSubspace.cs => IKeySubspace.cs} (84%) rename FoundationDB.Client/Subspaces/{IFdbEncoderSubspace.cs => ITypedKeySubspace.cs} (73%) rename FoundationDB.Client/Subspaces/{FdbSubspace.cs => KeySubspace.cs} (75%) rename FoundationDB.Client/Subspaces/{FdbSubspaceExtensions.cs => KeySubspaceExtensions.cs} (76%) create mode 100644 FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs create mode 100644 FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs create mode 100644 FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs create mode 100644 FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs rename FoundationDB.Client/TypeSystem/{IKeyValueEncoder.cs => IKeyEncoder`1.cs} (100%) rename FoundationDB.Client/TypeSystem/{IFdbKeyEncoding.cs => IKeyEncoding.cs} (89%) rename FoundationDB.Client/TypeSystem/{FdbTypeCodec`1.cs => TypeCodec`1.cs} (94%) rename FoundationDB.Client/{Subspaces/FdbEncoderSubspaceKeys`2.cs => Utils/Batched.cs} (51%) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index c912e6b33..262f2ae99 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -69,16 +69,16 @@ public static async Task Dir(string[] path, ITuple extras, DirectoryBrowseOption if (!(subfolder is FdbDirectoryPartition)) { long count = await Fdb.System.EstimateCountAsync(db, subfolder.Keys.ToRange(), ct); - log.WriteLine(" {0,-12} {1,-12} {3,9:N0} {2}", FdbKey.Dump(FdbSubspace.Copy(subfolder).Key), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, count); + log.WriteLine(" {0,-12} {1,-12} {3,9:N0} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, count); } else { - log.WriteLine(" {0,-12} {1,-12} {3,9} {2}", FdbKey.Dump(FdbSubspace.Copy(subfolder).Key), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, "-"); + log.WriteLine(" {0,-12} {1,-12} {3,9} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, "-"); } } else { - log.WriteLine(" {0,-12} {1,-12} {2}", FdbKey.Dump(FdbSubspace.Copy(subfolder).Key), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name); + log.WriteLine(" {0,-12} {1,-12} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name); } } else @@ -112,7 +112,7 @@ public static async Task CreateDirectory(string[] path, ITuple extras, IFdbDatab } folder = await db.Directory.TryCreateAsync(path, Slice.FromString(layer), cancellationToken: ct); - log.WriteLine("- Created under {0} [{1}]", FdbKey.Dump(folder.Key), folder.Key.ToHexaString(' ')); + log.WriteLine("- Created under {0} [{1}]", FdbKey.Dump(folder.GetPrefix()), folder.GetPrefix().ToHexaString(' ')); // look if there is already stuff under there var stuff = await db.ReadAsync((tr) => tr.GetRange(folder.Keys.ToRange()).FirstOrDefaultAsync(), cancellationToken: ct); @@ -216,8 +216,8 @@ public static async Task Count(string[] path, ITuple extras, IFdbDatabase db, Te return; } - var copy = FdbSubspace.Copy(folder); - log.WriteLine("# Counting keys under {0} ...", FdbKey.Dump(copy.Key)); + var copy = KeySubspace.Copy(folder); + log.WriteLine("# Counting keys under {0} ...", FdbKey.Dump(copy.GetPrefix())); var progress = new Progress>((state) => { @@ -242,7 +242,7 @@ public static async Task Show(string[] path, ITuple extras, bool reverse, IFdbDa var folder = await db.Directory.TryOpenAsync(path, cancellationToken: ct); if (folder != null) { - log.WriteLine("# Content of {0} [{1}]", FdbKey.Dump(folder.Key), folder.Key.ToHexaString(' ')); + log.WriteLine("# Content of {0} [{1}]", FdbKey.Dump(folder.GetPrefix()), folder.GetPrefix().ToHexaString(' ')); var keys = await db.QueryAsync((tr) => { var query = tr.GetRange(folder.Keys.ToRange()); @@ -394,7 +394,7 @@ public static async Task Map(string[] path, ITuple extras, IFdbDatabase db, Text n = dir.Name.Length; var p = dir.Path.ToArray(); - var key = ((FdbSubspace)dir).Key; + var key = ((KeySubspace)dir).GetPrefix(); // verify that the subspace has at least one key inside var bounds = await db.ReadAsync(async (tr) => @@ -554,7 +554,7 @@ public static async Task Shards(string[] path, ITuple extras, IFdbDatabase db, T var folder = (await TryOpenCurrentDirectoryAsync(path, db, ct)) as FdbDirectorySubspace; if (folder != null) { - var r = KeyRange.StartsWith(FdbSubspace.Copy(folder).Key); + var r = KeyRange.StartsWith(KeySubspace.Copy(folder).GetPrefix()); Console.WriteLine("Searching for shards that intersect with /{0} ...", String.Join("/", path)); ranges = await Fdb.System.GetChunksAsync(db, r, ct); Console.WriteLine("Found {0} ranges intersecting {1}:", ranges.Count, r); @@ -589,7 +589,7 @@ public static async Task Sampling(string[] path, ITuple extras, IFdbDatabase db, KeyRange span; if (folder is FdbDirectorySubspace) { - span = KeyRange.StartsWith(FdbSubspace.Copy(folder as FdbDirectorySubspace).Key); + span = KeyRange.StartsWith(KeySubspace.Copy(folder as FdbDirectorySubspace).GetPrefix()); log.WriteLine("Reading list of shards for /{0} under {1} ...", String.Join("/", path), FdbKey.Dump(span.Begin)); } else diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 2f6562a5a..fab98017e 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -489,7 +489,7 @@ private static async Task CreateClusterInternalAsync(string clusterF [ItemNotNull] public static Task OpenAsync(CancellationToken cancellationToken = default(CancellationToken)) { - return OpenAsync(clusterFile: null, dbName: null, globalSpace: FdbSubspace.Empty, cancellationToken: cancellationToken); + return OpenAsync(clusterFile: null, dbName: null, globalSpace: KeySubspace.Empty, cancellationToken: cancellationToken); } /// Create a new connection with the "DB" database on the cluster specified by the default cluster file, and with the specified global subspace @@ -499,7 +499,7 @@ private static async Task CreateClusterInternalAsync(string clusterF /// 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. [ItemNotNull] - public static Task OpenAsync(IFdbSubspace globalSpace, CancellationToken cancellationToken = default(CancellationToken)) + public static Task OpenAsync(IKeySubspace globalSpace, CancellationToken cancellationToken = default(CancellationToken)) { return OpenAsync(clusterFile: null, dbName: null, globalSpace: globalSpace, cancellationToken: cancellationToken); } @@ -516,7 +516,7 @@ private static async Task CreateClusterInternalAsync(string clusterF [ItemNotNull] public static Task OpenAsync(string clusterFile, string dbName, CancellationToken cancellationToken = default(CancellationToken)) { - return OpenAsync(clusterFile, dbName, FdbSubspace.Empty, readOnly: false, cancellationToken: cancellationToken); + return OpenAsync(clusterFile, dbName, KeySubspace.Empty, readOnly: false, cancellationToken: cancellationToken); } /// Create a new connection with a database on the specified cluster @@ -531,19 +531,19 @@ private static async Task CreateClusterInternalAsync(string clusterF /// 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. [ItemNotNull] - public static async Task OpenAsync(string clusterFile, string dbName, IFdbSubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task OpenAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) { return await 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 [ItemNotNull] - internal static async Task OpenInternalAsync(string clusterFile, string dbName, IFdbSubspace globalSpace, bool readOnly, CancellationToken cancellationToken) + internal static async Task OpenInternalAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly, CancellationToken cancellationToken) { cancellationToken.ThrowIfCancellationRequested(); dbName = dbName ?? "DB"; - globalSpace = globalSpace ?? FdbSubspace.Empty; + globalSpace = globalSpace ?? KeySubspace.Empty; if (Logging.On) Logging.Info(typeof(Fdb), "OpenAsync", String.Format("Connecting to database '{0}' using cluster file '{1}' and subspace '{2}' ...", dbName, clusterFile, globalSpace)); diff --git a/FoundationDB.Client/FdbCluster.cs b/FoundationDB.Client/FdbCluster.cs index d54d68d4d..37f0da3f4 100644 --- a/FoundationDB.Client/FdbCluster.cs +++ b/FoundationDB.Client/FdbCluster.cs @@ -112,7 +112,7 @@ protected virtual void Dispose(bool disposing) /// If the token is cancelled /// Any attempt to use a key outside the specified subspace will throw an exception [ItemNotNull] - public async Task OpenDatabaseAsync(string databaseName, IFdbSubspace subspace, bool readOnly, CancellationToken cancellationToken) + public async Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken cancellationToken) { if (subspace == null) throw new ArgumentNullException("subspace"); return await OpenDatabaseInternalAsync(databaseName, subspace, readOnly: readOnly, ownsCluster: false, cancellationToken: cancellationToken).ConfigureAwait(false); @@ -129,7 +129,7 @@ public async Task OpenDatabaseAsync(string databaseName, IFdbSubsp /// If the token is cancelled /// As of Beta2, the only supported database name is 'DB' [ItemNotNull] - internal async Task OpenDatabaseInternalAsync(string databaseName, IFdbSubspace subspace, bool readOnly, bool ownsCluster, CancellationToken cancellationToken) + internal async Task OpenDatabaseInternalAsync(string databaseName, IKeySubspace subspace, bool readOnly, bool ownsCluster, CancellationToken cancellationToken) { ThrowIfDisposed(); if (string.IsNullOrEmpty(databaseName)) throw new ArgumentNullException("databaseName"); diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index c806bfc47..ada696fd2 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -78,9 +78,9 @@ public class FdbDatabase : IFdbDatabase, IFdbRetryable /// Global namespace used to prefix ALL keys and subspaces accessible by this database instance (default is empty) /// This is readonly and is set when creating the database instance - private IFdbDynamicSubspace m_globalSpace; + private IDynamicKeySubspace m_globalSpace; /// Copy of the namespace, that is exposed to the outside. - private IFdbDynamicSubspace m_globalSpaceCopy; + private IDynamicKeySubspace m_globalSpaceCopy; /// Default Timeout value for all transactions private int m_defaultTimeout; @@ -106,7 +106,7 @@ public class FdbDatabase : IFdbDatabase, IFdbRetryable /// Root directory of the database instance /// If true, the database instance will only allow read-only transactions /// If true, the cluster instance lifetime is linked with the database instance - protected FdbDatabase(IFdbCluster cluster, IFdbDatabaseHandler handler, string name, IFdbSubspace contentSubspace, IFdbDirectory directory, bool readOnly, bool ownsCluster) + protected FdbDatabase(IFdbCluster cluster, IFdbDatabaseHandler handler, string name, IKeySubspace contentSubspace, IFdbDirectory directory, bool readOnly, bool ownsCluster) { Contract.Requires(cluster != null && handler != null && name != null && contentSubspace != null); @@ -126,7 +126,7 @@ protected FdbDatabase(IFdbCluster cluster, IFdbDatabaseHandler handler, string n /// Root directory of the database instance /// If true, the database instance will only allow read-only transactions /// If true, the cluster instance lifetime is linked with the database instance - public static FdbDatabase Create(IFdbCluster cluster, IFdbDatabaseHandler handler, string name, IFdbSubspace contentSubspace, IFdbDirectory directory, bool readOnly, bool ownsCluster) + public static FdbDatabase Create(IFdbCluster cluster, IFdbDatabaseHandler handler, string name, IKeySubspace contentSubspace, IFdbDirectory directory, bool readOnly, bool ownsCluster) { if (cluster == null) throw new ArgumentNullException(nameof(cluster)); if (handler == null) throw new ArgumentNullException(nameof(handler)); @@ -448,21 +448,21 @@ public void SetOption(FdbDatabaseOption option, long value) /// Change the current global namespace. /// Do NOT call this, unless you know exactly what you are doing ! - internal void ChangeRoot(IFdbSubspace subspace, IFdbDirectory directory, bool readOnly) + internal void ChangeRoot(IKeySubspace subspace, IFdbDirectory directory, bool readOnly) { //REVIEW: rename to "ChangeRootSubspace" ? - subspace = subspace ?? FdbSubspace.Empty; + subspace = subspace ?? KeySubspace.Empty; lock (this)//TODO: don't use this for locking { m_readOnly = readOnly; - m_globalSpace = FdbSubspace.CopyDynamic(subspace, TypeSystem.Tuples); - m_globalSpaceCopy = FdbSubspace.CopyDynamic(subspace, TypeSystem.Tuples); // keep another copy + m_globalSpace = KeySubspace.CopyDynamic(subspace, TypeSystem.Tuples); + m_globalSpaceCopy = KeySubspace.CopyDynamic(subspace, TypeSystem.Tuples); // keep another copy m_directory = directory == null ? null : new FdbDatabasePartition(this, directory); } } /// Returns the global namespace used by this database instance - public IFdbDynamicSubspace GlobalSpace + public IDynamicKeySubspace GlobalSpace { //REVIEW: rename to just "Subspace" ? [NotNull] @@ -511,7 +511,7 @@ internal static bool ValidateKey(IFdbDatabase database, ref Slice key, bool endE // special case: if endExclusive is true (we are validating the end key of a ClearRange), // and the key is EXACTLY equal to strinc(globalSpace.Prefix), we let is slide if (!endExclusive - || !key.Equals(FdbKey.Increment(database.GlobalSpace.Key))) //TODO: cache this? + || !key.Equals(FdbKey.Increment(database.GlobalSpace.GetPrefix()))) //TODO: cache this? { if (!ignoreError) error = Fdb.Errors.InvalidKeyOutsideDatabaseNamespace(database, key); return false; @@ -534,39 +534,39 @@ public Slice BoundCheck(Slice key, bool allowSystemKeys) return m_globalSpace.BoundCheck(key, allowSystemKeys); } - Slice IFdbSubspace.ConcatKey(Slice key) + Slice IKeySubspace.ConcatKey(Slice key) { return m_globalSpace.ConcatKey(key); } - Slice[] IFdbSubspace.ConcatKeys(IEnumerable keys) + Slice[] IKeySubspace.ConcatKeys(IEnumerable keys) { return m_globalSpace.ConcatKeys(keys); } /// Remove the database global subspace prefix from a binary key, or throw if the key is outside of the global subspace. - Slice IFdbSubspace.ExtractKey(Slice key, bool boundCheck) + Slice IKeySubspace.ExtractKey(Slice key, bool boundCheck) { return m_globalSpace.ExtractKey(key, boundCheck); } /// Remove the database global subspace prefix from a binary key, or throw if the key is outside of the global subspace. - Slice[] IFdbSubspace.ExtractKeys(IEnumerable keys, bool boundCheck) + Slice[] IKeySubspace.ExtractKeys(IEnumerable keys, bool boundCheck) { return m_globalSpace.ExtractKeys(keys, boundCheck); } - SliceWriter IFdbSubspace.GetWriter(int capacity) + SliceWriter IKeySubspace.GetWriter(int capacity) { return m_globalSpace.GetWriter(capacity); } - Slice IFdbSubspace.Key + Slice IKeySubspace.GetPrefix() { - get { return m_globalSpace.Key; } + return m_globalSpace.GetPrefix(); } - IFdbSubspace IFdbSubspace.this[Slice suffix] + IKeySubspace IKeySubspace.this[Slice suffix] { get { @@ -574,28 +574,28 @@ IFdbSubspace IFdbSubspace.this[Slice suffix] } } - KeyRange IFdbSubspace.ToRange() + KeyRange IKeySubspace.ToRange() { return m_globalSpace.ToRange(); } - KeyRange IFdbSubspace.ToRange(Slice suffix) + KeyRange IKeySubspace.ToRange(Slice suffix) { return m_globalSpace.ToRange(suffix); } - public FdbDynamicSubspacePartition Partition + public DynamicPartition Partition { //REVIEW: should we hide this on the main db? get { return m_globalSpace.Partition; } } - IDynamicKeyEncoder IFdbDynamicSubspace.Encoder + IDynamicKeyEncoder IDynamicKeySubspace.Encoder { get { return m_globalSpace.Encoder; } } - public FdbDynamicSubspaceKeys Keys + public DynamicKeys Keys { get { return m_globalSpace.Keys; } } diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index f46863e07..4ed43c223 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -138,7 +138,7 @@ public KeyValuePair[] Decode([NotNull] FuncIf at least on key in the result is outside . /// If either , or is null. [NotNull] - public KeyValuePair[] Decode([NotNull] FdbSubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) + public KeyValuePair[] Decode([NotNull] KeySubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); @@ -207,7 +207,7 @@ public T[] DecodeKeys([NotNull] Func handler) /// Instance used to decode the keys of this chunk /// Array of decoded keys, or an empty array if the chunk doesn't have any results [NotNull] - public T[] DecodeKeys([NotNull] FdbSubspace subspace, [NotNull] IKeyEncoder keyEncoder) + public T[] DecodeKeys([NotNull] KeySubspace subspace, [NotNull] IKeyEncoder keyEncoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index ed057a75e..76510d82e 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -110,7 +110,7 @@ public CancellationToken Cancellation } /// Returns the global namespace used by this database instance - public virtual IFdbDynamicSubspace GlobalSpace + public virtual IDynamicKeySubspace GlobalSpace { [NotNull] get { return m_database.GlobalSpace; } @@ -135,32 +135,32 @@ public virtual bool IsReadOnly get { return m_readOnly; } } - Slice IFdbSubspace.Key + Slice IKeySubspace.GetPrefix() { - get { return this.GlobalSpace.Key; } + return this.GlobalSpace.GetPrefix(); } - KeyRange IFdbSubspace.ToRange() + KeyRange IKeySubspace.ToRange() { return this.GlobalSpace.ToRange(); } - KeyRange IFdbSubspace.ToRange(Slice suffix) + KeyRange IKeySubspace.ToRange(Slice suffix) { return this.GlobalSpace.ToRange(suffix); } - IFdbSubspace IFdbSubspace.this[Slice suffix] + IKeySubspace IKeySubspace.this[Slice suffix] { get { return this.GlobalSpace[suffix]; } } - public virtual FdbDynamicSubspacePartition Partition + public virtual DynamicPartition Partition { get { return m_database.Partition; } } - public virtual FdbDynamicSubspaceKeys Keys + public virtual DynamicKeys Keys { get { return m_database.Keys; } } diff --git a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs index 5f062903b..0feb35de1 100644 --- a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs +++ b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs @@ -38,16 +38,16 @@ public sealed class PrefixRewriterTransaction : FdbTransactionFilter { // We will add a prefix to all keys sent to the db, and remove it on the way back - private readonly IFdbSubspace m_prefix; + private readonly IKeySubspace m_prefix; - public PrefixRewriterTransaction(IFdbSubspace prefix, IFdbTransaction trans, bool ownsTransaction) + public PrefixRewriterTransaction(IKeySubspace prefix, IFdbTransaction trans, bool ownsTransaction) : base(trans, false, ownsTransaction) { if (prefix == null) throw new ArgumentNullException("prefix"); m_prefix = prefix; } - public IFdbSubspace Prefix { get { return m_prefix; } } + public IKeySubspace Prefix { get { return m_prefix; } } private Slice Encode(Slice key) { diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 407a4ddb8..8ee988e68 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -65,17 +65,11 @@ - - - - - - - - - + + + - + @@ -169,17 +163,14 @@ - - - - - - - - + + + + + - + @@ -216,11 +207,11 @@ - - + + - + @@ -231,7 +222,7 @@ - + diff --git a/FoundationDB.Client/IFdbCluster.cs b/FoundationDB.Client/IFdbCluster.cs index 40e32dce8..02b580cf5 100644 --- a/FoundationDB.Client/IFdbCluster.cs +++ b/FoundationDB.Client/IFdbCluster.cs @@ -61,7 +61,7 @@ public interface IFdbCluster : IDisposable /// Cancellation Token (optionnal) for the connect operation /// Task that will return an FdbDatabase, or an exception [ItemNotNull] - Task OpenDatabaseAsync(string databaseName, IFdbSubspace subspace, bool readOnly, CancellationToken cancellationToken); + Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken cancellationToken); } } diff --git a/FoundationDB.Client/IFdbDatabase.cs b/FoundationDB.Client/IFdbDatabase.cs index 85179208c..69471c02b 100644 --- a/FoundationDB.Client/IFdbDatabase.cs +++ b/FoundationDB.Client/IFdbDatabase.cs @@ -34,7 +34,7 @@ namespace FoundationDB.Client /// Database connection context. [PublicAPI] - public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IFdbDynamicSubspace, IDisposable + public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IDynamicKeySubspace, IDisposable { /// Name of the database string Name { [NotNull] get; } @@ -48,7 +48,7 @@ public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IFdbDynami /// Returns the global namespace used by this database instance /// Makes a copy of the subspace tuple, so you should not call this property a lot. Use any of the Partition(..) methods to create a subspace of the database - IFdbDynamicSubspace GlobalSpace { [NotNull] get; } + IDynamicKeySubspace GlobalSpace { [NotNull] get; } /// Directory partition of this database instance FdbDatabasePartition Directory { [NotNull] get; } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 49f2a0e11..35b85c315 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -63,13 +63,13 @@ public class FdbDirectoryLayer : IFdbDirectory public static bool AnnotateTransactions { get; set; } /// Subspace where the content of each folder will be stored - public IFdbDynamicSubspace ContentSubspace { [NotNull] get; } + public IDynamicKeySubspace ContentSubspace { [NotNull] get; } /// Subspace where all the metadata nodes for each folder will be stored - public IFdbDynamicSubspace NodeSubspace { [NotNull] get; } + public IDynamicKeySubspace NodeSubspace { [NotNull] get; } /// Root node of the directory - internal IFdbDynamicSubspace RootNode { [NotNull] get; } + internal IDynamicKeySubspace RootNode { [NotNull] get; } /// Allocated used to generated prefix for new content internal FdbHighContentionAllocator Allocator { [NotNull] get; } @@ -128,7 +128,7 @@ Task IFdbDirectory.ChangeLayerAsync(IFdbTransaction trans, /// Subspace where all the node metadata will be stored ('\xFE' by default) /// Subspace where all automatically allocated directories will be stored (empty by default) /// Location of the root of all the directories managed by this Directory Layer. Ususally empty for the root partition of the database. - internal FdbDirectoryLayer(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace contentSubspace, ITuple location) + internal FdbDirectoryLayer(IDynamicKeySubspace nodeSubspace, IDynamicKeySubspace contentSubspace, ITuple location) { Contract.Requires(nodeSubspace != null && contentSubspace != null); @@ -137,7 +137,7 @@ internal FdbDirectoryLayer(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace this.NodeSubspace = nodeSubspace; // The root node is the one whose contents are the node subspace - this.RootNode = nodeSubspace.Partition.ByKey(nodeSubspace.Key); + this.RootNode = nodeSubspace.Partition.ByKey(nodeSubspace.GetPrefix()); this.Allocator = new FdbHighContentionAllocator(this.RootNode.Partition.ByKey(HcaKey)); if (location == null || location.Count == 0) { @@ -164,7 +164,7 @@ public static FdbDirectoryLayer Create() [NotNull] public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null) { - var subspace = FdbSubspace.CreateDynamic(prefix, TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(prefix, TypeSystem.Tuples); var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } @@ -173,7 +173,7 @@ public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = /// Subspace for the content. The nodes will be stored under .Key + <FE> /// Optional path, if the Directory Layer is not located at the root of the database. [NotNull] - public static FdbDirectoryLayer Create(IFdbSubspace subspace, IEnumerable path = null) + public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null) { if (subspace == null) throw new ArgumentNullException("subspace"); @@ -187,7 +187,7 @@ public static FdbDirectoryLayer Create(IFdbSubspace subspace, IEnumerableSubspace for the content of the Directory Layer. /// Optional path, if the Directory Layer is not located at the root of the database [NotNull] - public static FdbDirectoryLayer Create(IFdbDynamicSubspace nodeSubspace, IFdbDynamicSubspace contentSubspace, IEnumerable path = null) + public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynamicKeySubspace contentSubspace, IEnumerable path = null) { if (nodeSubspace == null) throw new ArgumentNullException("nodeSubspace"); if (contentSubspace == null) throw new ArgumentNullException("contentSubspace"); @@ -480,7 +480,7 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti public override string ToString() { - return String.Format("DirectoryLayer(path={0}, contents={1}, nodes={2})", this.FullName, this.ContentSubspace.Key.ToAsciiOrHexaString(), this.NodeSubspace.Key.ToAsciiOrHexaString()); + return String.Format("DirectoryLayer(path={0}, contents={1}, nodes={2})", this.FullName, this.ContentSubspace.GetPrefix().ToAsciiOrHexaString(), this.NodeSubspace.GetPrefix().ToAsciiOrHexaString()); } #endregion @@ -490,7 +490,7 @@ public override string ToString() private struct Node { - public Node(IFdbDynamicSubspace subspace, ITuple path, ITuple targetPath, Slice layer) + public Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice layer) { this.Subspace = subspace; this.Path = path; @@ -498,7 +498,7 @@ public Node(IFdbDynamicSubspace subspace, ITuple path, ITuple targetPath, Slice this.Layer = layer; } - public readonly IFdbDynamicSubspace Subspace; + public readonly IDynamicKeySubspace Subspace; public readonly ITuple Path; public readonly ITuple TargetPath; public Slice Layer; //PERF: readonly struct @@ -514,7 +514,7 @@ public bool IsInPartition(bool includeEmptySubPath) } - private static void SetLayer([NotNull] IFdbTransaction trans, [NotNull] IFdbDynamicSubspace subspace, Slice layer) + private static void SetLayer([NotNull] IFdbTransaction trans, [NotNull] IDynamicKeySubspace subspace, Slice layer) { if (layer.IsNull) layer = Slice.Empty; trans.Set(subspace.Keys.Encode(LayerSuffix), layer); @@ -665,11 +665,11 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly } // we need to recursively create any missing parents - IFdbDynamicSubspace parentNode; + IDynamicKeySubspace parentNode; if (path.Count > 1) { var parentSubspace = await CreateOrOpenInternalAsync(readTrans, trans, path.Substring(0, path.Count - 1), Slice.Nil, Slice.Nil, true, true, true).ConfigureAwait(false); - parentNode = NodeWithPrefix(parentSubspace.Key); + parentNode = NodeWithPrefix(parentSubspace.GetPrefix()); } else { @@ -739,8 +739,8 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa return null; } - if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Register the prefix {0} to its new location in the folder sub-tree", oldNode.Subspace.Key); - trans.Set(GetSubDirKey(parentNode.Subspace, newPath.Get(-1)), this.NodeSubspace.Keys.Decode(oldNode.Subspace.Key)); + if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Register the prefix {0} to its new location in the folder sub-tree", oldNode.Subspace.GetPrefix()); + trans.Set(GetSubDirKey(parentNode.Subspace, newPath.Get(-1)), this.NodeSubspace.Keys.Decode(oldNode.Subspace.GetPrefix())); await RemoveFromParent(trans, oldPath).ConfigureAwait(false); return ContentsOfNode(oldNode.Subspace, newPath, oldNode.Layer); @@ -893,7 +893,7 @@ private void InitializeDirectory([NotNull] IFdbTransaction trans) } [ItemCanBeNull] - private async Task NodeContainingKey([NotNull] IFdbReadOnlyTransaction tr, Slice key) + private async Task NodeContainingKey([NotNull] IFdbReadOnlyTransaction tr, Slice key) { Contract.Requires(tr != null); @@ -926,7 +926,7 @@ private async Task NodeContainingKey([NotNull] IFdbReadOnlyTransac /// Returns the subspace to a node metadata, given its prefix [CanBeNull] - private IFdbDynamicSubspace NodeWithPrefix(Slice prefix) + private IDynamicKeySubspace NodeWithPrefix(Slice prefix) { if (prefix.IsNullOrEmpty) return null; return this.NodeSubspace.Partition.ByKey(prefix); @@ -934,12 +934,12 @@ private IFdbDynamicSubspace NodeWithPrefix(Slice prefix) /// Returns a new Directory Subspace given its node subspace, path and layer id [NotNull] - private FdbDirectorySubspace ContentsOfNode([NotNull] IFdbSubspace node, [NotNull] ITuple relativePath, Slice layer) + private FdbDirectorySubspace ContentsOfNode([NotNull] IKeySubspace node, [NotNull] ITuple relativePath, Slice layer) { Contract.Requires(node != null); var path = this.Location.Concat(relativePath); - var prefix = this.NodeSubspace.Keys.Decode(node.Key); + var prefix = this.NodeSubspace.Keys.Decode(node.GetPrefix()); if (layer == FdbDirectoryPartition.LayerId) { return new FdbDirectoryPartition(path, relativePath, prefix, this); @@ -970,14 +970,14 @@ private async Task FindAsync([NotNull] IFdbReadOnlyTransaction tr, [NotNul Slice layer = Slice.Nil; while (i < path.Count) { - if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Looking for child {0} under node {1}...", path.Get(i), n.Key); + if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Looking for child {0} under node {1}...", path.Get(i), n.GetPrefix()); n = NodeWithPrefix(await tr.GetAsync(GetSubDirKey(n, path.Get(i))).ConfigureAwait(false)); if (n == null) { return new Node(null, path.Substring(0, i + 1), path, Slice.Empty); } - if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Reading Layer value for subfolder {0} found at {1}", path, n.Key); + if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Reading Layer value for subfolder {0} found at {1}", path, n.GetPrefix()); layer = await tr.GetAsync(n.Keys.Encode(LayerSuffix)).ConfigureAwait(false); if (layer == FdbDirectoryPartition.LayerId) { // stop when reaching a partition @@ -991,14 +991,14 @@ private async Task FindAsync([NotNull] IFdbReadOnlyTransaction tr, [NotNul /// Returns the list of names and nodes of all children of the specified node [NotNull] - private IFdbAsyncEnumerable> SubdirNamesAndNodes([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IFdbDynamicSubspace node) + private IFdbAsyncEnumerable> SubdirNamesAndNodes([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IDynamicKeySubspace node) { Contract.Requires(tr != null && node != null); var sd = node.Partition.ByKey(SUBDIRS); return tr .GetRange(sd.Keys.ToRange()) - .Select(kvp => new KeyValuePair( + .Select(kvp => new KeyValuePair( sd.Keys.Decode(kvp.Key), NodeWithPrefix(kvp.Value) )); @@ -1013,7 +1013,7 @@ private async Task RemoveFromParent([NotNull] IFdbTransaction tr, [NotNull var parent = await FindAsync(tr, path.Substring(0, path.Count - 1)).ConfigureAwait(false); if (parent.Exists) { - if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing path {0} from its parent folder at {1}", path, parent.Subspace.Key); + if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing path {0} from its parent folder at {1}", path, parent.Subspace.GetPrefix()); tr.Clear(GetSubDirKey(parent.Subspace, path.Get(-1))); return true; } @@ -1021,7 +1021,7 @@ private async Task RemoveFromParent([NotNull] IFdbTransaction tr, [NotNull } /// Resursively remove a node (including the content), all its children - private async Task RemoveRecursive([NotNull] IFdbTransaction tr, [NotNull] IFdbDynamicSubspace node) + private async Task RemoveRecursive([NotNull] IFdbTransaction tr, [NotNull] IDynamicKeySubspace node) { Contract.Requires(tr != null && node != null); @@ -1029,10 +1029,10 @@ private async Task RemoveRecursive([NotNull] IFdbTransaction tr, [NotNull] IFdbD await SubdirNamesAndNodes(tr, node).ForEachAsync((kvp) => RemoveRecursive(tr, kvp.Value)).ConfigureAwait(false); // remove ALL the contents - if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all content located under {0}", node.Key); + if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all content located under {0}", node.GetPrefix()); tr.ClearRange(ContentsOfNode(node, STuple.Empty, Slice.Empty).ToRange()); // and all the metadata for this folder - if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all metadata for folder under {0}", node.Key); + if (FdbDirectoryLayer.AnnotateTransactions) tr.Annotate("Removing all metadata for folder under {0}", node.GetPrefix()); tr.ClearRange(node.Keys.ToRange()); } @@ -1056,7 +1056,7 @@ private async Task IsPrefixFree([NotNull] IFdbReadOnlyTransaction tr, Slic .ConfigureAwait(false); } - private static Slice GetSubDirKey([NotNull] IFdbDynamicSubspace parent, [NotNull] string path) + private static Slice GetSubDirKey([NotNull] IDynamicKeySubspace parent, [NotNull] string path) { Contract.Requires(parent != null && path != null); diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 1733d2619..0a60e3256 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -40,7 +40,7 @@ public class FdbDirectoryPartition : FdbDirectorySubspace private readonly FdbDirectoryLayer m_parentDirectoryLayer; internal FdbDirectoryPartition(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer) - : base(location, relativeLocation, prefix, new FdbDirectoryLayer(FdbSubspace.CreateDynamic(prefix + FdbKey.Directory, TypeSystem.Tuples), FdbSubspace.CreateDynamic(prefix, TypeSystem.Tuples), location), LayerId, TypeSystem.Tuples.GetDynamicEncoder()) + : base(location, relativeLocation, prefix, new FdbDirectoryLayer(KeySubspace.CreateDynamic(prefix + FdbKey.Directory, TypeSystem.Tuples), KeySubspace.CreateDynamic(prefix, TypeSystem.Tuples), location), LayerId, TypeSystem.Tuples.GetDynamicEncoder()) { m_parentDirectoryLayer = directoryLayer; } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index 5edb70f5f..8a1122f84 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -40,7 +40,7 @@ namespace FoundationDB.Layers.Directories /// A Directory Subspace represents the contents of a directory, but it also remembers the path with which it was opened and offers convenience methods to operate on the directory at that path. /// An instance of DirectorySubspace can be used for all the usual subspace operations. It can also be used to operate on the directory with which it was opened. [DebuggerDisplay("Path={this.FullName}, Prefix={InternalKey}, Layer={Layer}")] - public class FdbDirectorySubspace : FdbDynamicSubspace, IFdbDirectory + public class FdbDirectorySubspace : DynamicKeySubspace, IFdbDirectory { internal FdbDirectorySubspace(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IDynamicKeyEncoder encoder) diff --git a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs index 7fbe97ffc..07918ff8d 100644 --- a/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs +++ b/FoundationDB.Client/Layers/Directories/FdbHighContentionAllocator.cs @@ -46,7 +46,7 @@ public sealed class FdbHighContentionAllocator /// Create an allocator operating under a specific location /// - public FdbHighContentionAllocator(IFdbDynamicSubspace subspace) + public FdbHighContentionAllocator(IDynamicKeySubspace subspace) { if (subspace == null) throw new ArgumentException("subspace"); @@ -56,13 +56,13 @@ public FdbHighContentionAllocator(IFdbDynamicSubspace subspace) } /// Location of the allocator - public IFdbDynamicSubspace Subspace { [NotNull] get; private set; } + public IDynamicKeySubspace Subspace { [NotNull] get; private set; } /// Subspace used to store the allocation count for the current window - private IFdbDynamicSubspace Counters { [NotNull] get; set; } + private IDynamicKeySubspace Counters { [NotNull] get; set; } /// Subspace used to store the prefixes allocated in the current window - private IFdbDynamicSubspace Recent { [NotNull] get; set; } + private IDynamicKeySubspace Recent { [NotNull] get; set; } /// Returns a 64-bit integer that /// 1) has never and will never be returned by another call to this @@ -91,10 +91,10 @@ public async Task AllocateAsync([NotNull] IFdbTransaction trans) if ((count + 1) * 2 >= window) { // advance the window if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Advance allocator window size to {0} starting at {1}", window, start + window); - trans.ClearRange(this.Counters.Key, this.Counters.Keys.Encode(start) + FdbKey.MinValue); + trans.ClearRange(this.Counters.GetPrefix(), this.Counters.Keys.Encode(start) + FdbKey.MinValue); start += window; count = 0; - trans.ClearRange(this.Recent.Key, this.Recent.Keys.Encode(start)); + trans.ClearRange(this.Recent.GetPrefix(), this.Recent.Keys.Encode(start)); } // Increment the allocation count for the current window diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index add745128..48123c07e 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -34,7 +34,7 @@ namespace FoundationDB.Layers.Tuples /// Type codec that uses the Tuple Encoding format /// Type of the values encoded by this codec - public sealed class TupleCodec : FdbTypeCodec, IValueEncoder + public sealed class TupleCodec : TypeCodec, IValueEncoder { private static volatile TupleCodec s_defaultSerializer; diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs index 59b731477..277d88e11 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs @@ -39,7 +39,7 @@ public sealed class TupleKeyEncoder : IDynamicKeyEncoder private TupleKeyEncoder() { } - public IFdbKeyEncoding Encoding + public IKeyEncoding Encoding { get { return TypeSystem.Tuples; } } diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs index ca4b0baf0..9a29e0fee 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs @@ -32,7 +32,7 @@ namespace FoundationDB.Layers.Tuples using System; using FoundationDB.Client; - public sealed class TupleKeyEncoding : IFdbKeyEncoding + public sealed class TupleKeyEncoding : IKeyEncoding { public IDynamicKeyEncoder GetDynamicEncoder() { diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs similarity index 71% rename from FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs rename to FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 8dec069fe..750e14874 100644 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspaceKeys.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -1,4 +1,4 @@ -#region BSD Licence +#region BSD Licence /* Copyright (c) 2013-2018, Doxense SAS All rights reserved. @@ -26,82 +26,65 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using FoundationDB.Client.Utils; + namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Linq; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client.Utils; using FoundationDB.Layers.Tuples; using JetBrains.Annotations; - internal static class Batched + public class DynamicKeySubspace : KeySubspace, IDynamicKeySubspace { - - public delegate void Handler(ref SliceWriter writer, TValue item, TState state); - - [NotNull] - public static Slice[] Convert(SliceWriter writer, [NotNull, ItemNotNull] IEnumerable values, Handler handler, TState state) + /// Create a new subspace from a binary prefix + /// Prefix of the new subspace + /// If true, take a copy of the prefix + /// Type System used to encode keys in this subspace (optional, will use Tuple Encoding by default) + internal DynamicKeySubspace(Slice rawPrefix, bool copy, IDynamicKeyEncoder encoder) + : base (rawPrefix, copy) { - Contract.Requires(values != null && handler != null); - - //Note on performance: - // - we will reuse the same buffer for each temp key, and copy them into a slice buffer - // - doing it this way adds a memory copy (writer => buffer) but reduce the number of byte[] allocations (and reduce the GC overhead) - - int start = writer.Position; + this.Encoder = encoder ?? TypeSystem.Default.GetDynamicEncoder(); + this.Keys = new DynamicKeys(this, this.Encoder); + this.Partition = new DynamicPartition(this, this.Encoder); + } - var buffer = new SliceBuffer(); + public DynamicKeySubspace(Slice rawPrefix, IDynamicKeyEncoder encoder) + : this(rawPrefix, true, encoder) + { } - var coll = values as ICollection; - if (coll != null) - { // pre-allocate the final array with the correct size - var res = new Slice[coll.Count]; - int p = 0; - foreach (var tuple in coll) - { - // reset position to just after the subspace prefix - writer.Position = start; + protected override IKeySubspace CreateChildren(Slice suffix) + { + return new DynamicKeySubspace(ConcatKey(suffix), this.Encoder); + } - handler(ref writer, tuple, state); + /// Encoder for the keys of this subspace + public IDynamicKeyEncoder Encoder { get; } - // copy full key in the buffer - res[p++] = buffer.Intern(writer.ToSlice()); - } - Contract.Assert(p == res.Length); - return res; - } - else - { // we won't now the array size until the end... - var res = new List(); - foreach (var tuple in values) - { - // reset position to just after the subspace prefix - writer.Position = start; + /// Return a view of all the possible binary keys of this subspace + public DynamicKeys Keys { get; } - handler(ref writer, tuple, state); + /// Returns an helper object that knows how to create sub-partitions of this subspace + public DynamicPartition Partition { get; } - // copy full key in the buffer - res.Add(buffer.Intern(writer.ToSlice())); - } - return res.ToArray(); - } - } } /// Key helper for a dynamic TypeSystem - public struct FdbDynamicSubspaceKeys + public /*readonly*/ struct DynamicKeys { //NOTE: everytime an ITuple is used here, it is as a container (vector of objects), and NOT as the Tuple Encoding scheme ! (separate concept) /// Parent subspace - [NotNull] public readonly IFdbSubspace Subspace; + [NotNull] + public readonly IKeySubspace Subspace; /// Encoder used to format keys in this subspace - [NotNull] public readonly IDynamicKeyEncoder Encoder; + [NotNull] + public readonly IDynamicKeyEncoder Encoder; - public FdbDynamicSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + public DynamicKeys([NotNull] IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { Contract.Requires(subspace != null && encoder != null); this.Subspace = subspace; @@ -111,7 +94,7 @@ public FdbDynamicSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] IDynami /// Return a key range that encompass all the keys inside this subspace, according to the current key encoder public KeyRange ToRange() { - return this.Encoder.ToRange(this.Subspace.Key); + return this.Encoder.ToRange(this.Subspace.GetPrefix()); } /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder @@ -130,19 +113,13 @@ public KeyRange ToRange([NotNull] ITupleFormattable item) /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix - /// This is a shortcut for - public Slice this[[NotNull] ITuple tuple] - { - get { return Pack(tuple); } - } + /// This is a shortcut for + public Slice this[[NotNull] ITuple tuple] => Pack(tuple); /// Convert an item into a key of this subspace /// Convertible item that will be packed and appended to the subspace prefix /// This is a shortcut for - public Slice this[[NotNull] ITupleFormattable item] - { - get { return Pack(item); } - } + public Slice this[[NotNull] ITupleFormattable item] => Pack(item); /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix @@ -159,7 +136,7 @@ public Slice Pack([NotNull] ITuple tuple) /// Sequence of tuple that will be packed and appended to the subspace prefix public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable tuples) { - if (tuples == null) throw new ArgumentNullException("tuples"); + if (tuples == null) throw new ArgumentNullException(nameof(tuples)); return Batched.Convert( this.Subspace.GetWriter(), @@ -173,7 +150,7 @@ public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable tuples) /// Convertible item that will be packed and appended to the subspace prefix public Slice Pack([NotNull] ITupleFormattable item) { - if (item == null) throw new ArgumentNullException("item"); + if (item == null) throw new ArgumentNullException(nameof(item)); return Pack(item.ToTuple()); } @@ -182,7 +159,7 @@ public Slice Pack([NotNull] ITupleFormattable item) /// Sequence of convertible items that will be packed and appended to the subspace prefix public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable items) { - if (items == null) throw new ArgumentNullException("items"); + if (items == null) throw new ArgumentNullException(nameof(items)); return Batched.Convert( this.Subspace.GetWriter(), @@ -356,17 +333,16 @@ public Slice[] EncodeMany(IEnumerableUnpack a key of this subspace, back into a tuple - /// Key that was produced by a previous call to + /// Key that was produced by a previous call to /// Original tuple public ITuple Unpack(Slice packed) { return this.Encoder.UnpackKey(this.Subspace.ExtractKey(packed)); } - private static T[] BatchDecode(IEnumerable packed, IFdbSubspace subspace, IDynamicKeyEncoder encoder, Func decode) + private static T[] BatchDecode(IEnumerable packed, IKeySubspace subspace, IDynamicKeyEncoder encoder, Func decode) { - var coll = packed as ICollection; - if (coll != null) + if (packed is ICollection coll) { var res = new T[coll.Count]; int p = 0; @@ -389,7 +365,7 @@ private static T[] BatchDecode(IEnumerable packed, IFdbSubspace subspa } /// Unpack a batch of keys of this subspace, back into an array of tuples - /// Sequence of keys that were produced by a previous call to or + /// Sequence of keys that were produced by a previous call to or /// Array containing the original tuples public ITuple[] UnpackMany(IEnumerable packed) { @@ -491,7 +467,7 @@ public IEnumerable DecodeLastMany(Slice[] packed) [NotNull] public ITuple ToTuple() { - return new PrefixedTuple(this.Subspace.Key, STuple.Empty); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Empty); } /// Attach a tuple to an existing subspace. @@ -500,7 +476,7 @@ public ITuple ToTuple() [NotNull] public ITuple Concat([NotNull] ITuple tuple) { - return new PrefixedTuple(this.Subspace.Key, tuple); + return new PrefixedTuple(this.Subspace.GetPrefix(), tuple); } /// Convert a formattable item into a tuple that is attached to this subspace. @@ -510,10 +486,10 @@ public ITuple Concat([NotNull] ITuple tuple) [NotNull] public ITuple Concat([NotNull] ITupleFormattable formattable) { - if (formattable == null) throw new ArgumentNullException("formattable"); + if (formattable == null) throw new ArgumentNullException(nameof(formattable)); var tuple = formattable.ToTuple(); if (tuple == null) throw new InvalidOperationException("Formattable item cannot return an empty tuple"); - return new PrefixedTuple(this.Subspace.Key, tuple); + return new PrefixedTuple(this.Subspace.GetPrefix(), tuple); } /// Create a new 1-tuple that is attached to this subspace @@ -524,7 +500,7 @@ public ITuple Concat([NotNull] ITupleFormattable formattable) [NotNull] public ITuple Append(T value) { - return new PrefixedTuple(this.Subspace.Key, STuple.Create(value)); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(value)); } /// Create a new 2-tuple that is attached to this subspace @@ -537,7 +513,7 @@ public ITuple Append(T value) [NotNull] public ITuple Append(T1 item1, T2 item2) { - return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2)); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2)); } /// Create a new 3-tuple that is attached to this subspace @@ -552,7 +528,7 @@ public ITuple Append(T1 item1, T2 item2) [NotNull] public ITuple Append(T1 item1, T2 item2, T3 item3) { - return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3)); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3)); } /// Create a new 4-tuple that is attached to this subspace @@ -569,7 +545,7 @@ public ITuple Append(T1 item1, T2 item2, T3 item3) [NotNull] public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4) { - return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3, item4)); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3, item4)); } /// Create a new 5-tuple that is attached to this subspace @@ -588,10 +564,160 @@ public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4) [NotNull] public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return new PrefixedTuple(this.Subspace.Key, STuple.Create(item1, item2, item3, item4, item5)); + return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3, item4, item5)); } #endregion } + + public /*readonly*/ struct DynamicPartition + { + + [NotNull] + public readonly IDynamicKeySubspace Subspace; + + [NotNull] + public readonly IDynamicKeyEncoder Encoder; + + public DynamicPartition([NotNull] IDynamicKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + /// Returns the same view but using a different Type System + /// Type System that will code keys in this new view + /// Review that will partition this subspace using a different Type System + /// + /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. + /// If you are calling this in a loop, consider creating a new subspace using that encoding. + /// + [Pure] + public DynamicPartition Using([NotNull] IKeyEncoding encoding) + { + Contract.NotNull(encoding, nameof(encoding)); + var encoder = encoding.GetDynamicEncoder(); + return UsingEncoder(encoder); + } + + /// Returns the same view but using a different Type System + /// Type System that will code keys in this new view + /// Review that will partition this subspace using a different Type System + /// + /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. + /// If you are calling this in a loop, consider creating a new subspace using that encoder. + /// + [Pure] + public DynamicPartition UsingEncoder([NotNull] IDynamicKeyEncoder encoder) + { + return new DynamicPartition(this.Subspace, encoder); + } + + /// Create a new subspace by appdending a suffix to the current subspace + /// Suffix of the new subspace + /// New subspace with prefix equal to the current subspace's prefix, followed by + public IDynamicKeySubspace this[Slice suffix] + { + [Pure, NotNull] + get + { + if (suffix.IsNull) throw new ArgumentException("Partition suffix cannot be null", nameof(suffix)); + //TODO: find a way to limit the number of copies of the key? + return new DynamicKeySubspace(this.Subspace.ConcatKey(suffix), false, this.Encoder); + } + } + + public IDynamicKeySubspace this[ITuple tuple] + { + [Pure, ContractAnnotation("null => halt; notnull => notnull")] + get + { + if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + //TODO: find a way to limit the number of copies of the packed tuple? + return new DynamicKeySubspace(this.Subspace.Keys.Pack(tuple), false, this.Encoder); + } + } + + public IDynamicKeySubspace this[ITupleFormattable item] + { + [Pure, ContractAnnotation("null => halt; notnull => notnull")] + get + { + Contract.NotNull(item, nameof(item)); + var tuple = item.ToTuple(); + if (tuple == null) throw new InvalidOperationException("Formattable item returned an empty tuple"); + return this[tuple]; + } + } + + /// Partition this subspace into a child subspace + /// Type of the child subspace key + /// Value of the child subspace + /// New subspace that is logically contained by the current subspace + /// Subspace([Foo, ]).Partition(Bar) is equivalent to Subspace([Foo, Bar, ]) + /// + /// new FdbSubspace(["Users", ]).Partition("Contacts") == new FdbSubspace(["Users", "Contacts", ]) + /// + [Pure, NotNull] + public IDynamicKeySubspace ByKey(T value) + { + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value), false, this.Encoder); + } + + /// Partition this subspace into a child subspace + /// Type of the first subspace key + /// Type of the second subspace key + /// Value of the first subspace key + /// Value of the second subspace key + /// New subspace that is logically contained by the current subspace + /// Subspace([Foo, ]).Partition(Bar, Baz) is equivalent to Subspace([Foo, Bar, Baz]) + /// + /// new FdbSubspace(["Users", ]).Partition("Contacts", "Friends") == new FdbSubspace(["Users", "Contacts", "Friends", ]) + /// + [Pure, NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2) + { + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2), false, this.Encoder); + } + + /// Partition this subspace into a child subspace + /// Type of the first subspace key + /// Type of the second subspace key + /// Type of the third subspace key + /// Value of the first subspace key + /// Value of the second subspace key + /// Value of the third subspace key + /// New subspace that is logically contained by the current subspace + /// + /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", ]) + /// + [Pure, NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3) + { + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3), false, this.Encoder); + } + + /// Partition this subspace into a child subspace + /// Type of the first subspace key + /// Type of the second subspace key + /// Type of the third subspace key + /// Type of the fourth subspace key + /// Value of the first subspace key + /// Value of the second subspace key + /// Value of the third subspace key + /// Value of the fourth subspace key + /// New subspace that is logically contained by the current subspace + /// + /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends", "Messages") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", "Messages", ]) + /// + [Pure, NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) + { + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3, value4), false, this.Encoder); + } + + } + } diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index 921fa72da..6ab4ac552 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 // By convention, all named databases will be under the "/Databases" folder FdbDatabase db = null; - var rootSpace = FdbSubspace.Empty; + var rootSpace = KeySubspace.Empty; try { db = await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, cancellationToken: cancellationToken).ConfigureAwait(false); @@ -87,7 +87,7 @@ public static async Task OpenNamedPartitionAsync(string clusterFil if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", String.Format("Found named partition '{0}' at prefix {1}", descriptor.FullName, descriptor)); // we have to chroot the database to the new prefix, and create a new DirectoryLayer with a new '/' - rootSpace = FdbSubspace.Copy(descriptor); //note: create a copy of the key + rootSpace = KeySubspace.Copy(descriptor); //note: create a copy of the key //TODO: find a nicer way to do that! db.ChangeRoot(rootSpace, FdbDirectoryLayer.Create(rootSpace, partitionPath), readOnly); diff --git a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs index 25d811dcc..3b85fc27b 100644 --- a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs @@ -40,77 +40,51 @@ namespace FoundationDB.Client [DebuggerDisplay("Database={Database.Name}, Partition={Directory.FullName}, Prefix={Database.GlobalSpace}")] public sealed class FdbDatabasePartition : IFdbDirectory { - private readonly IFdbDatabase m_database; - private readonly IFdbDirectory m_directory; /// Wrap an existing database with a root directory public FdbDatabasePartition(IFdbDatabase database, IFdbDirectory directory) { - if (database == null) throw new ArgumentNullException("database"); - if (directory == null) throw new ArgumentNullException("directory"); - - m_database = database; - m_directory = directory; + this.Database = database ?? throw new ArgumentNullException(nameof(database)); + this.Directory = directory ?? throw new ArgumentNullException(nameof(directory)); } /// Wrapped Directory instance - public IFdbDirectory Directory - { - [NotNull] - get { return m_directory; } - } + [NotNull] + public IFdbDirectory Directory { get; } /// Wrapped Database instance - public IFdbDatabase Database - { - [NotNull] - get { return m_database; } - } + [NotNull] + public IFdbDatabase Database { get; } /// Name of this Partition. /// This returns the last part of the path - public string Name - { - get { return m_directory.Name; } - } + public string Name => this.Directory.Name; /// Formatted path of this Partition /// This returns the formatted path, using '/' as the separator - public string FullName - { - [NotNull] - get { return m_directory.FullName; } - } + [NotNull] + public string FullName => this.Directory.FullName; /// Gets the path represented by this Partition. /// Returns an empty list for the root partition of the database, or a non empty list for a sub-partition - public IReadOnlyList Path - { - [NotNull] - get { return m_directory.Path; } - } + [NotNull] + public IReadOnlyList Path => this.Directory.Path; /// Get the DirectoryLayer that was used to create this partition. - public FdbDirectoryLayer DirectoryLayer - { - [NotNull] - get { return m_directory.DirectoryLayer; } - } + [NotNull] + public FdbDirectoryLayer DirectoryLayer => this.Directory.DirectoryLayer; #region Layer... /// Returns "partition" (ASCII) /// This should be equal to - public Slice Layer - { - get { return m_directory.Layer; } - } + public Slice Layer => this.Directory.Layer; void IFdbDirectory.CheckLayer(Slice layer) { if (layer.IsPresent && layer != this.Layer) { - throw new InvalidOperationException(String.Format("The directory {0} is a partition which is not compatible with layer {1}.", this.FullName, layer.ToAsciiOrHexaString())); + throw new InvalidOperationException($"The directory {this.FullName} is a partition which is not compatible with layer {layer.ToAsciiOrHexaString()}."); } } @@ -129,7 +103,7 @@ Task IFdbDirectory.ChangeLayerAsync(IFdbTransaction trans, /// public Task CreateOrOpenAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given path. @@ -138,7 +112,7 @@ public Task CreateOrOpenAsync([NotNull] string name, Cance /// public Task CreateOrOpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateOrOpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new[] { name }, layer), cancellationToken); } /// Opens a subdirectory with the given path. @@ -147,7 +121,7 @@ public Task CreateOrOpenAsync([NotNull] string name, Slice /// public Task CreateOrOpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateOrOpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given path. @@ -156,12 +130,12 @@ public Task CreateOrOpenAsync([NotNull] IEnumerable public Task CreateOrOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateOrOpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, layer), cancellationToken); } Task IFdbDirectory.CreateOrOpenAsync(IFdbTransaction trans, IEnumerable subPath, Slice layer) { - return m_directory.CreateOrOpenAsync(trans, subPath, layer); + return this.Directory.CreateOrOpenAsync(trans, subPath, layer); } #endregion @@ -174,7 +148,7 @@ Task IFdbDirectory.CreateOrOpenAsync(IFdbTransaction trans /// Name of the subdirectory to open public Task OpenAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.OpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given . @@ -184,7 +158,7 @@ public Task OpenAsync([NotNull] string name, CancellationT /// Expected layer id for the subdirectory (optional) public Task OpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.OpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new[] { name }, layer), cancellationToken); } /// Opens a subdirectory with the given . @@ -193,7 +167,7 @@ public Task OpenAsync([NotNull] string name, Slice layer, /// Relative path of the subdirectory to open public Task OpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.OpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given . @@ -203,12 +177,12 @@ public Task OpenAsync([NotNull] IEnumerable path, /// Expected layer id for the subdirectory (optional) public Task OpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.OpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, layer), cancellationToken); } Task IFdbDirectory.OpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer) { - return m_directory.OpenAsync(trans, path, layer); + return this.Directory.OpenAsync(trans, path, layer); } #endregion @@ -222,7 +196,7 @@ Task IFdbDirectory.OpenAsync(IFdbReadOnlyTransaction trans /// Returns the directory if it exists, or null if it was not found public Task TryOpenAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.TryOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given . @@ -233,7 +207,7 @@ public Task TryOpenAsync([NotNull] string name, Cancellati /// Returns the directory if it exists, or null if it was not found public Task TryOpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.TryOpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new[] { name }, layer), cancellationToken); } /// Opens a subdirectory with the given . @@ -243,7 +217,7 @@ public Task TryOpenAsync([NotNull] string name, Slice laye /// Returns the directory if it exists, or null if it was not found public Task TryOpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.TryOpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, Slice.Nil), cancellationToken); } /// Opens a subdirectory with the given . @@ -254,12 +228,12 @@ public Task TryOpenAsync([NotNull] IEnumerable pat /// Returns the directory if it exists, or null if it was not found public Task TryOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadAsync((tr) => m_directory.TryOpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, layer), cancellationToken); } Task IFdbDirectory.TryOpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer) { - return m_directory.TryOpenAsync(trans, path, layer); + return this.Directory.TryOpenAsync(trans, path, layer); } #endregion @@ -268,27 +242,27 @@ Task IFdbDirectory.TryOpenAsync(IFdbReadOnlyTransaction tr public Task CreateAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateAsync(tr, new[] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new[] { name }, Slice.Nil), cancellationToken); } public Task CreateAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateAsync(tr, new [] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new [] { name }, layer), cancellationToken); } public Task CreateAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, Slice.Nil), cancellationToken); } public Task CreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.CreateAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, layer), cancellationToken); } Task IFdbDirectory.CreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer) { - return m_directory.CreateAsync(trans, path, layer); + return this.Directory.CreateAsync(trans, path, layer); } #endregion @@ -297,27 +271,27 @@ Task IFdbDirectory.CreateAsync(IFdbTransaction trans, IEnu public Task TryCreateAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), cancellationToken); } public Task TryCreateAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryCreateAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new[] { name }, layer), cancellationToken); } public Task TryCreateAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryCreateAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, Slice.Nil), cancellationToken); } public Task TryCreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryCreateAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, layer), cancellationToken); } Task IFdbDirectory.TryCreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer) { - return m_directory.TryCreateAsync(trans, path, layer); + return this.Directory.TryCreateAsync(trans, path, layer); } #endregion @@ -330,7 +304,7 @@ Task IFdbDirectory.TryCreateAsync(IFdbTransaction trans, I /// The directory will be created with the given physical prefix; otherwise a prefix is allocated automatically. public Task RegisterAsync([NotNull] string name, Slice layer, Slice prefix, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.RegisterAsync(tr, new[] { name }, layer, prefix), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, new[] { name }, layer, prefix), cancellationToken); } /// Registers an existing prefix as a directory with the given (creating parent directories if necessary). This method is only indented for advanced use cases. @@ -339,12 +313,12 @@ public Task RegisterAsync([NotNull] string name, Slice lay /// The directory will be created with the given physical prefix; otherwise a prefix is allocated automatically. public Task RegisterAsync([NotNull] IEnumerable path, Slice layer, Slice prefix, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.RegisterAsync(tr, path, layer, prefix), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, path, layer, prefix), cancellationToken); } Task IFdbDirectory.RegisterAsync(IFdbTransaction trans, IEnumerable path, Slice layer, Slice prefix) { - return m_directory.RegisterAsync(trans, path, layer, prefix); + return this.Directory.RegisterAsync(trans, path, layer, prefix); } #endregion @@ -353,12 +327,12 @@ Task IFdbDirectory.RegisterAsync(IFdbTransaction trans, IE public Task MoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.MoveAsync(tr, oldPath, newPath), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.MoveAsync(tr, oldPath, newPath), cancellationToken); } Task IFdbDirectory.MoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) { - return m_directory.MoveAsync(trans, oldPath, newPath); + return this.Directory.MoveAsync(trans, oldPath, newPath); } #endregion @@ -367,12 +341,12 @@ Task IFdbDirectory.MoveAsync(IFdbTransaction trans, IEnume public Task TryMoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryMoveAsync(tr, oldPath, newPath), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryMoveAsync(tr, oldPath, newPath), cancellationToken); } Task IFdbDirectory.TryMoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) { - return m_directory.TryMoveAsync(trans, oldPath, newPath); + return this.Directory.TryMoveAsync(trans, oldPath, newPath); } #endregion @@ -399,17 +373,17 @@ public Task TryMoveToAsync(IFdbTransaction trans, IEnumera public Task RemoveAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.RemoveAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, new string[] { name }), cancellationToken); } public Task RemoveAsync(IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.RemoveAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, path), cancellationToken); } Task IFdbDirectory.RemoveAsync(IFdbTransaction trans, IEnumerable path) { - return m_directory.RemoveAsync(trans, path); + return this.Directory.RemoveAsync(trans, path); } #endregion @@ -418,17 +392,17 @@ Task IFdbDirectory.RemoveAsync(IFdbTransaction trans, IEnumerable path) public Task TryRemoveAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryRemoveAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, new [] { name }), cancellationToken); } public Task TryRemoveAsync(IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.TryRemoveAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, path), cancellationToken); } Task IFdbDirectory.TryRemoveAsync(IFdbTransaction trans, IEnumerable path) { - return m_directory.TryRemoveAsync(trans, path); + return this.Directory.TryRemoveAsync(trans, path); } #endregion @@ -437,17 +411,17 @@ Task IFdbDirectory.TryRemoveAsync(IFdbTransaction trans, IEnumerable ExistsAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.ExistsAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, new [] { name }), cancellationToken); } public Task ExistsAsync(IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.ExistsAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, path), cancellationToken); } Task IFdbDirectory.ExistsAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - return m_directory.ExistsAsync(trans, path); + return this.Directory.ExistsAsync(trans, path); } #endregion @@ -457,24 +431,24 @@ Task IFdbDirectory.ExistsAsync(IFdbReadOnlyTransaction trans, IEnumerable< /// Returns the list of all the top level directories of this database instance. public Task> ListAsync(CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.ListAsync(tr), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr), cancellationToken); } /// Returns the list of all the top level directories of this database instance. public Task> ListAsync([NotNull] string name, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.ListAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, new string[] { name }), cancellationToken); } /// Returns the list of all the top level directories of this database instance. public Task> ListAsync(IEnumerable path, CancellationToken cancellationToken) { - return m_database.ReadWriteAsync((tr) => m_directory.ListAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, path), cancellationToken); } Task> IFdbDirectory.ListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - return m_directory.ListAsync(trans, path); + return this.Directory.ListAsync(trans, path); } #endregion @@ -486,7 +460,7 @@ public Task> TryListAsync(CancellationToken cancellationToken) { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return m_database.ReadWriteAsync((tr) => m_directory.TryListAsync(tr), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr), cancellationToken); } /// Returns the list of all the top level directories of this database instance. @@ -494,7 +468,7 @@ public Task> TryListAsync([NotNull] string name, CancellationToken { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return m_database.ReadWriteAsync((tr) => m_directory.TryListAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, new string[] { name }), cancellationToken); } /// Returns the list of all the top level directories of this database instance. @@ -502,12 +476,12 @@ public Task> TryListAsync(IEnumerable path, CancellationTok { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return m_database.ReadWriteAsync((tr) => m_directory.TryListAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, path), cancellationToken); } Task> IFdbDirectory.TryListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - return m_directory.TryListAsync(trans, path); + return this.Directory.TryListAsync(trans, path); } #endregion diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs deleted file mode 100644 index 1a5f4e229..000000000 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspace.cs +++ /dev/null @@ -1,80 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using System.Diagnostics; - using JetBrains.Annotations; - - public class FdbDynamicSubspace : FdbSubspace, IFdbDynamicSubspace - { - /// Encoder for the keys of this subspace - private readonly IDynamicKeyEncoder m_encoder; - - /// Create a new subspace from a binary prefix - /// Prefix of the new subspace - /// If true, take a copy of the prefix - /// Type System used to encode keys in this subspace (optional, will use Tuple Encoding by default) - internal FdbDynamicSubspace(Slice rawPrefix, bool copy, IDynamicKeyEncoder encoder) - : base (rawPrefix, copy) - { - this.m_encoder = encoder ?? TypeSystem.Default.GetDynamicEncoder(); - } - - public FdbDynamicSubspace(Slice rawPrefix, IDynamicKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } - - protected override IFdbSubspace CreateChildren(Slice suffix) - { - return new FdbDynamicSubspace(ConcatKey(suffix), m_encoder); - } - - public IDynamicKeyEncoder Encoder - { - get { return m_encoder; } - } - - /// Return a view of all the possible binary keys of this subspace - public FdbDynamicSubspaceKeys Keys - { - [DebuggerStepThrough] - get { return new FdbDynamicSubspaceKeys(this, m_encoder); } - } - - /// Returns an helper object that knows how to create sub-partitions of this subspace - public FdbDynamicSubspacePartition Partition - { - //note: not cached, because this is probably not be called frequently (except in the init path) - [DebuggerStepThrough] - get { return new FdbDynamicSubspacePartition(this, m_encoder); } - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs b/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs deleted file mode 100644 index 865656f68..000000000 --- a/FoundationDB.Client/Subspaces/FdbDynamicSubspacePartition.cs +++ /dev/null @@ -1,184 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; - - public struct FdbDynamicSubspacePartition - { - [NotNull] - public readonly IFdbDynamicSubspace Subspace; - - [NotNull] - public readonly IDynamicKeyEncoder Encoder; - - public FdbDynamicSubspacePartition([NotNull] IFdbDynamicSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) - { - Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoder, nameof(encoder)); - this.Subspace = subspace; - this.Encoder = encoder; - } - - /// Returns the same view but using a different Type System - /// Type System that will code keys in this new view - /// Review that will partition this subspace using a different Type System - /// - /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. - /// If you are calling this in a loop, consider creating a new subspace using that encoding. - /// - [Pure] - public FdbDynamicSubspacePartition Using([NotNull] IFdbKeyEncoding encoding) - { - Contract.NotNull(encoding, nameof(encoding)); - var encoder = encoding.GetDynamicEncoder(); - return UsingEncoder(encoder); - } - - /// Returns the same view but using a different Type System - /// Type System that will code keys in this new view - /// Review that will partition this subspace using a different Type System - /// - /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. - /// If you are calling this in a loop, consider creating a new subspace using that encoder. - /// - [Pure] - public FdbDynamicSubspacePartition UsingEncoder([NotNull] IDynamicKeyEncoder encoder) - { - return new FdbDynamicSubspacePartition(this.Subspace, encoder); - } - - /// Create a new subspace by appdending a suffix to the current subspace - /// Suffix of the new subspace - /// New subspace with prefix equal to the current subspace's prefix, followed by - public IFdbDynamicSubspace this[Slice suffix] - { - [NotNull] - get - { - if (suffix.IsNull) throw new ArgumentException("Partition suffix cannot be null", nameof(suffix)); - //TODO: find a way to limit the number of copies of the key? - return new FdbDynamicSubspace(this.Subspace.ConcatKey(suffix), false, this.Encoder); - } - } - - public IFdbDynamicSubspace this[ITuple tuple] - { - [ContractAnnotation("null => halt; notnull => notnull")] - get - { - if (tuple == null) throw new ArgumentNullException("tuple"); - //TODO: find a way to limit the number of copies of the packed tuple? - return new FdbDynamicSubspace(this.Subspace.Keys.Pack(tuple), false, this.Encoder); - } - } - - public IFdbDynamicSubspace this[ITupleFormattable item] - { - [ContractAnnotation("null => halt; notnull => notnull")] - get - { - Contract.NotNull(item, nameof(item)); - var tuple = item.ToTuple(); - if (tuple == null) throw new InvalidOperationException("Formattable item returned an empty tuple"); - return this[tuple]; - } - } - - /// Partition this subspace into a child subspace - /// Type of the child subspace key - /// Value of the child subspace - /// New subspace that is logically contained by the current subspace - /// Subspace([Foo, ]).Partition(Bar) is equivalent to Subspace([Foo, Bar, ]) - /// - /// new FdbSubspace(["Users", ]).Partition("Contacts") == new FdbSubspace(["Users", "Contacts", ]) - /// - [Pure, NotNull] - public IFdbDynamicSubspace ByKey(T value) - { - return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value), false, this.Encoder); - } - - /// Partition this subspace into a child subspace - /// Type of the first subspace key - /// Type of the second subspace key - /// Value of the first subspace key - /// Value of the second subspace key - /// New subspace that is logically contained by the current subspace - /// Subspace([Foo, ]).Partition(Bar, Baz) is equivalent to Subspace([Foo, Bar, Baz]) - /// - /// new FdbSubspace(["Users", ]).Partition("Contacts", "Friends") == new FdbSubspace(["Users", "Contacts", "Friends", ]) - /// - [Pure, NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2) - { - return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2), false, this.Encoder); - } - - /// Partition this subspace into a child subspace - /// Type of the first subspace key - /// Type of the second subspace key - /// Type of the third subspace key - /// Value of the first subspace key - /// Value of the second subspace key - /// Value of the third subspace key - /// New subspace that is logically contained by the current subspace - /// - /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", ]) - /// - [Pure, NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3) - { - return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2, value3), false, this.Encoder); - } - - /// Partition this subspace into a child subspace - /// Type of the first subspace key - /// Type of the second subspace key - /// Type of the third subspace key - /// Type of the fourth subspace key - /// Value of the first subspace key - /// Value of the second subspace key - /// Value of the third subspace key - /// Value of the fourth subspace key - /// New subspace that is logically contained by the current subspace - /// - /// new FdbSubspace(["Users", ]).Partition("John Smith", "Contacts", "Friends", "Messages") == new FdbSubspace(["Users", "John Smith", "Contacts", "Friends", "Messages", ]) - /// - [Pure, NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) - { - return new FdbDynamicSubspace(this.Subspace.Keys.Encode(value1, value2, value3, value4), false, this.Encoder); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs deleted file mode 100644 index 549a92705..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`1.cs +++ /dev/null @@ -1,86 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; - - public struct FdbEncoderSubspaceKeys - { - - [NotNull] - public readonly IFdbSubspace Subspace; - - [NotNull] - public readonly IKeyEncoder Encoder; - - public FdbEncoderSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) - { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; - } - - public Slice this[T value] - { - get { return Encode(value); } - } - - public Slice Encode(T value) - { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)); - } - - public Slice[] Encode([NotNull] IEnumerable values) - { - if (values == null) throw new ArgumentNullException("values"); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, T value, IKeyEncoder encoder) => { writer.WriteBytes(encoder.EncodeKey(value)); }, - this.Encoder - ); - } - - public T Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - public KeyRange ToRange(T value) - { - //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value)); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs deleted file mode 100644 index 01f1742e1..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`3.cs +++ /dev/null @@ -1,81 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using System.Collections.Generic; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; - - public struct FdbEncoderSubspaceKeys - { - - public readonly IFdbSubspace Subspace; - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) - { - this.Subspace = subspace; - this.Encoder = encoder; - } - - public Slice this[T1 value1, T2 value2, T3 value3] - { - get { return Encode(value1, value2, value3); } - } - - public Slice Encode(T1 value1, T2 value2, T3 value3) - { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)); - } - - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3) - { - if (values == null) throw new ArgumentNullException("values"); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value))), - this.Encoder - ); - } - - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - public KeyRange ToRange(T1 value1, T2 value2, T3 value3) - { - //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2, value3)); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs deleted file mode 100644 index 49e322355..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`4.cs +++ /dev/null @@ -1,87 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; - - public struct FdbEncoderSubspaceKeys - { - - [NotNull] - public readonly IFdbSubspace Subspace; - - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; - } - - public Slice this[T1 value1, T2 value2, T3 value3, T4 value4] - { - get { return Encode(value1, value2, value3, value4); } - } - - public Slice Encode(T1 value1, T2 value2, T3 value3, T4 value4) - { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)); - } - - [NotNull] - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3, [NotNull] Func selector4) - { - if (values == null) throw new ArgumentNullException("values"); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value), selector4(value))), - this.Encoder - ); - } - - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) - { - //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2, value3, value4)); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs deleted file mode 100644 index b33e6c3d9..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`1.cs +++ /dev/null @@ -1,82 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -using System; -using JetBrains.Annotations; - -namespace FoundationDB.Client -{ - public struct FdbEncoderSubspacePartition - { - public readonly IFdbSubspace Subspace; - public readonly IKeyEncoder Encoder; - - public FdbEncoderSubspacePartition([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) - { - this.Subspace = subspace; - this.Encoder = encoder; - } - - public IFdbSubspace this[T value] - { - [NotNull] - get { return ByKey(value); } - } - - [NotNull] - public IFdbSubspace ByKey(T value) - { - return this.Subspace[this.Encoder.EncodeKey(value)]; - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T value, [NotNull] IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T value, [NotNull] IDynamicKeyEncoder encoder) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T value, [NotNull] IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T value, [NotNull] IKeyEncoder encoder) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); - } - - } -} \ No newline at end of file diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs deleted file mode 100644 index d56fe24c1..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`2.cs +++ /dev/null @@ -1,86 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using JetBrains.Annotations; - - public struct FdbEncoderSubspacePartition - { - [NotNull] - public readonly IFdbSubspace Subspace; - - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspacePartition([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) - { - this.Subspace = subspace; - this.Encoder = encoder; - } - - public IFdbSubspace this[T1 value1, T2 value2] - { - [NotNull] - get - { return ByKey(value1, value2); } - } - - [NotNull] - public IFdbSubspace ByKey(T1 value1, T2 value2) - { - return this.Subspace[this.Encoder.EncodeKey(value1, value2)]; - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, IDynamicKeyEncoder encoder) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, IKeyEncoder encoder) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs deleted file mode 100644 index cc90d8bff..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`3.cs +++ /dev/null @@ -1,87 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using System.Diagnostics.Contracts; - using JetBrains.Annotations; - - public struct FdbEncoderSubspacePartition - { - [NotNull] - public readonly IFdbSubspace Subspace; - - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspacePartition([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; - } - - public IFdbSubspace this[T1 value1, T2 value2, T3 value3] - { - [NotNull] - get { return ByKey(value1, value2, value3); } - } - - [NotNull] - public IFdbSubspace ByKey(T1 value1, T2 value2, T3 value3) - { - return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3)]; - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, IDynamicKeyEncoder encoder) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoder encoder) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs deleted file mode 100644 index 7bcf0d137..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspacePartition`4.cs +++ /dev/null @@ -1,88 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using Doxense.Diagnostics.Contracts; - using JetBrains.Annotations; - using FoundationDB.Client.Utils; - - public struct FdbEncoderSubspacePartition - { - [NotNull] - public readonly IFdbSubspace Subspace; - - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspacePartition([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; - } - - public IFdbSubspace this[T1 value1, T2 value2, T3 value3, T4 value4] - { - [NotNull] - get { return ByKey(value1, value2, value3, value4); } - } - - [NotNull] - public IFdbSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) - { - return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3, value4)]; - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); - } - - [NotNull] - public IFdbDynamicSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IDynamicKeyEncoder encoder) - { - return FdbSubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IFdbKeyEncoding encoding) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); - } - - [NotNull] - public IFdbEncoderSubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoder encoder) - { - return FdbSubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); - } - - } -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs deleted file mode 100644 index 7099dcfa4..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`1.cs +++ /dev/null @@ -1,72 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using JetBrains.Annotations; - - /// Subspace that knows how to encode and decode its key - /// Type of the key handled by this subspace - public class FdbEncoderSubspace : FdbSubspace, IFdbEncoderSubspace - { - private readonly IKeyEncoder m_encoder; - - // ReSharper disable once FieldCanBeMadeReadOnly.Local - private /*readonly*/ FdbEncoderSubspaceKeys m_keys; - - public FdbEncoderSubspace(Slice rawPrefix, [NotNull] IKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } - - internal FdbEncoderSubspace(Slice rawPrefix, bool copy, [NotNull] IKeyEncoder encoder) - : base(rawPrefix, copy) - { - if (encoder == null) throw new ArgumentNullException("encoder"); - m_encoder = encoder; - m_keys = new FdbEncoderSubspaceKeys(this, encoder); - } - - public IKeyEncoder Encoder - { - get { return m_encoder; } - } - - public FdbEncoderSubspaceKeys Keys - { - get { return m_keys; } - } - - public FdbEncoderSubspacePartition Partition - { - get { return new FdbEncoderSubspacePartition(this, m_encoder); } - } - - } - -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs deleted file mode 100644 index d3c698a84..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`2.cs +++ /dev/null @@ -1,81 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using Doxense.Diagnostics.Contracts; - using JetBrains.Annotations; - - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - public class FdbEncoderSubspace : FdbSubspace, IFdbEncoderSubspace - { - private readonly ICompositeKeyEncoder m_encoder; - - // ReSharper disable once FieldCanBeMadeReadOnly.Local - private /*readonly*/ FdbEncoderSubspaceKeys m_keys; - - public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } - - internal FdbEncoderSubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) - { - Contract.NotNull(encoder, nameof(encoder)); - m_encoder = encoder; - m_keys = new FdbEncoderSubspaceKeys(this, encoder); - } - - private FdbEncoderSubspace m_partial; - - public IFdbEncoderSubspace Partial - { - get { return m_partial ?? (m_partial = new FdbEncoderSubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(m_encoder))); } - } - - public ICompositeKeyEncoder Encoder - { - get { return m_encoder; } - } - - public FdbEncoderSubspaceKeys Keys - { - get { return m_keys; } - } - - public FdbEncoderSubspacePartition Partition - { - get { return new FdbEncoderSubspacePartition(this, m_encoder); } - } - - } - -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs deleted file mode 100644 index 5a642c0ab..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`3.cs +++ /dev/null @@ -1,87 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using Doxense.Diagnostics.Contracts; - using JetBrains.Annotations; - - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace - public class FdbEncoderSubspace : FdbSubspace, IFdbEncoderSubspace - { - private readonly ICompositeKeyEncoder m_encoder; - - // ReSharper disable once FieldCanBeMadeReadOnly.Local - private /*readonly*/ FdbEncoderSubspaceKeys m_keys; - private FdbEncoderSubspace m_head; - private FdbEncoderSubspace m_partial; - - public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } - - internal FdbEncoderSubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) - { - Contract.NotNull(encoder, nameof(encoder)); - m_encoder = encoder; - m_keys = new FdbEncoderSubspaceKeys(this, encoder); - } - - public IFdbEncoderSubspace Head - { - get { return m_head ?? (m_head = new FdbEncoderSubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(m_encoder))); } - } - - public IFdbEncoderSubspace Partial - { - get { return m_partial ?? (m_partial = new FdbEncoderSubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(m_encoder))); } - } - - public ICompositeKeyEncoder Encoder - { - get { return m_encoder; } - } - - public FdbEncoderSubspaceKeys Keys - { - get { return m_keys; } - } - - public FdbEncoderSubspacePartition Partition - { - get { return new FdbEncoderSubspacePartition(this, m_encoder); } - } - - } - -} diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs b/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs deleted file mode 100644 index ff5c33e54..000000000 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspace`4.cs +++ /dev/null @@ -1,88 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using JetBrains.Annotations; - using Doxense.Diagnostics.Contracts; - - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace - /// Type of the fourth item of the keys handled by this subspace - public class FdbEncoderSubspace : FdbSubspace, IFdbEncoderSubspace - { - private readonly ICompositeKeyEncoder m_encoder; - - // ReSharper disable once FieldCanBeMadeReadOnly.Local - private /*readonly*/ FdbEncoderSubspaceKeys m_keys; - private FdbEncoderSubspace m_head; - private FdbEncoderSubspace m_partial; - - public FdbEncoderSubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } - - internal FdbEncoderSubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) - { - Contract.NotNull(encoder, nameof(encoder)); - m_encoder = encoder; - m_keys = new FdbEncoderSubspaceKeys(this, encoder); - } - - public IFdbEncoderSubspace Head - { - get { return m_head ?? (m_head = new FdbEncoderSubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(m_encoder))); } - } - - public IFdbEncoderSubspace Partial - { - get { return m_partial ?? (m_partial = new FdbEncoderSubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(m_encoder))); } - } - - public ICompositeKeyEncoder Encoder - { - get { return m_encoder; } - } - - public FdbEncoderSubspaceKeys Keys - { - get { return m_keys; } - } - - public FdbEncoderSubspacePartition Partition - { - get { return new FdbEncoderSubspacePartition(this, m_encoder); } - } - - } - -} diff --git a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs b/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs similarity index 58% rename from FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs rename to FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs index 2d2adc923..82e0e56f4 100644 --- a/FoundationDB.Client/Subspaces/IFdbDynamicSubspace.cs +++ b/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs @@ -31,18 +31,30 @@ namespace FoundationDB.Client using System; using JetBrains.Annotations; + /// Represents a Key Subspace which can encode and decode keys of arbitrary size. + /// This is usefull when dealing with subspaces that store keys of different types and shapes. + /// In pseudo code, we obtain a dynamic subspace that wraps a prefix, and uses the Tuple Encoder Format to encode variable-size tuples into binary: + /// + /// subspace = {...}.OpenOrCreate(..., "/some/path/to/data", TypeSystem.Tuples) + /// subspace.GetPrefix() => {prefix} + /// subspace.Keys.Pack(("Hello", "World")) => (PREFIX, 'Hello', 'World') => {prefix}.'\x02Hello\x00\x02World\x00' + /// subspace.Keys.Encode("Hello", "World") => (PREFIX, 'Hello', 'World') => {prefix}.'\x02Hello\x00\x02World\x00' + /// subspace.Keys.Decode({prefix}'\x02Hello\x00\x15\x42') => ('Hello', 0x42) + /// + /// [PublicAPI] - public interface IFdbDynamicSubspace : IFdbSubspace + public interface IDynamicKeySubspace : IKeySubspace { - /// Encoding used to convert keys of this subspace into Slice - IDynamicKeyEncoder Encoder {[NotNull] get; } + /// Codec used by this subspace to convert keys into/from binary + [NotNull] + IDynamicKeyEncoder Encoder {get; } - /// Returns a view of the keys of this subspace - FdbDynamicSubspaceKeys Keys { get; } + /// View of the keys of this subspace + DynamicKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace - FdbDynamicSubspacePartition Partition { get; } + DynamicPartition Partition { get; } } diff --git a/FoundationDB.Client/Subspaces/IFdbSubspace.cs b/FoundationDB.Client/Subspaces/IKeySubspace.cs similarity index 84% rename from FoundationDB.Client/Subspaces/IFdbSubspace.cs rename to FoundationDB.Client/Subspaces/IKeySubspace.cs index f32964658..c436796d7 100644 --- a/FoundationDB.Client/Subspaces/IFdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/IKeySubspace.cs @@ -28,17 +28,31 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using JetBrains.Annotations; using System; using System.Collections.Generic; + using JetBrains.Annotations; + /// Represents a sub-partition of the global key space. + /// + /// A subspace is the logical equivalent of a key prefix that is implicitly prepended to all keys generated from it. + /// A "vanilla" data subspace does not imply any encoding scheme by default, but can be wrapped into a more complex subspace which includes Key Codec. + /// + /// + /// In pseudo code, and given a 'MySubspaceImpl' that implement : + /// + /// subspace = new MySubspaceImpl({ABC}) + /// subspace.ConcatKey({123}) => {ABC123} + /// subspace.ExtractKey({ABC123}) => {123} + /// subspace.ExtractKey({DEF123}) => ERROR + /// + /// [PublicAPI] - public interface IFdbSubspace + public interface IKeySubspace { // This interface helps solve some type resolution ambiguities at compile time between types that all implement IFdbKey but have different semantics for partitionning and concatenation /// Returns the prefix of this subspace - Slice Key { [Pure] get; } + Slice GetPrefix(); /// Return a key range that contains all the keys in this subspace, including the prefix itself /// Return the range: Key <= x <= Increment(Key) @@ -54,7 +68,11 @@ public interface IFdbSubspace /// Create a new subspace by adding a suffix to the key of the current subspace. /// Binary suffix that will be appended to the current prefix /// New subspace whose prefix is the concatenation of the parent prefix, and - IFdbSubspace this[Slice suffix] { [Pure, NotNull] get; } + IKeySubspace this[Slice suffix] + { + [Pure, NotNull] get; + } + //REVIEW this should probably be renamed into GetSubspace(suffix) or Partition(suffix) in order to make it explicit that it is for creating subspaces instances that can be reused multiple times, and not single-use to generate a single key! /// Test if a key is inside the range of keys logically contained by this subspace /// Key to test @@ -80,6 +98,7 @@ public interface IFdbSubspace /// Array of which is equivalent to calling on each entry in [Pure, NotNull] Slice[] ConcatKeys([NotNull] IEnumerable suffixes); + //REVIEW: could this be done via an extension method? /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace /// Complete key that contains the current subspace prefix, and a binary suffix @@ -97,12 +116,14 @@ public interface IFdbSubspace /// If is true and at least one key in is outside the current subspace. [Pure, NotNull] Slice[] ExtractKeys([NotNull] IEnumerable keys, bool boundCheck = false); + //REVIEW: could this be done via an extension method? /// Return a new slice buffer, initialized with the subspace prefix, that can be used for custom key serialization /// If non-zero, the expected buffer capacity. The size of the subspace prefix will be added to this value. /// Instance of a SliceWriter with the prefix of this subspace already copied. [Pure] SliceWriter GetWriter(int capacity = 0); + //REVIEW: this is an internal implementation detail that may be moved to a different interface? } diff --git a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs b/FoundationDB.Client/Subspaces/ITypedKeySubspace.cs similarity index 73% rename from FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs rename to FoundationDB.Client/Subspaces/ITypedKeySubspace.cs index 6e650f6c5..558f5c057 100644 --- a/FoundationDB.Client/Subspaces/IFdbEncoderSubspace.cs +++ b/FoundationDB.Client/Subspaces/ITypedKeySubspace.cs @@ -32,64 +32,64 @@ namespace FoundationDB.Client using JetBrains.Annotations; [PublicAPI] - public interface IFdbEncoderSubspace : IFdbSubspace + public interface ITypedKeySubspace : IKeySubspace { IKeyEncoder Encoder { [NotNull] get; } /// Return a view of all the possible keys of this subspace - FdbEncoderSubspaceKeys Keys { get; } + TypedKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace - FdbEncoderSubspacePartition Partition { get; } + TypedSubspacePartition Partition { get; } } - public interface IFdbEncoderSubspace : IFdbSubspace + public interface ITypedKeySubspace : IKeySubspace { ICompositeKeyEncoder Encoder {[NotNull] get; } /// Return a view of all the possible keys of this subspace - FdbEncoderSubspaceKeys Keys { get; } + TypedKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace - FdbEncoderSubspacePartition Partition { get; } + TypedSubspacePartition Partition { get; } - IFdbEncoderSubspace Partial {[NotNull] get; } + ITypedKeySubspace Partial {[NotNull] get; } } - public interface IFdbEncoderSubspace : IFdbSubspace + public interface ITypedKeySubspace : IKeySubspace { ICompositeKeyEncoder Encoder {[NotNull] get; } /// Return a view of all the possible keys of this subspace - FdbEncoderSubspaceKeys Keys { get; } + TypedKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace - FdbEncoderSubspacePartition Partition { get; } + TypedSubspacePartition Partition { get; } - IFdbEncoderSubspace Head { [NotNull] get; } + ITypedKeySubspace Head { [NotNull] get; } - IFdbEncoderSubspace Partial {[NotNull] get; } + ITypedKeySubspace Partial {[NotNull] get; } } - public interface IFdbEncoderSubspace : IFdbSubspace + public interface ITypedKeySubspace : IKeySubspace { ICompositeKeyEncoder Encoder {[NotNull] get; } /// Return a view of all the possible keys of this subspace - FdbEncoderSubspaceKeys Keys { get; } + TypedKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace - FdbEncoderSubspacePartition Partition { get; } + TypedSubspacePartition Partition { get; } - IFdbEncoderSubspace Head {[NotNull] get; } + ITypedKeySubspace Head {[NotNull] get; } - IFdbEncoderSubspace Partial {[NotNull] get; } + ITypedKeySubspace Partial {[NotNull] get; } } diff --git a/FoundationDB.Client/Subspaces/FdbSubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs similarity index 75% rename from FoundationDB.Client/Subspaces/FdbSubspace.cs rename to FoundationDB.Client/Subspaces/KeySubspace.cs index dd8aab308..6ae622ebd 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -31,36 +31,31 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; - using System.Linq; using Doxense.Diagnostics.Contracts; using FoundationDB.Layers.Tuples; using JetBrains.Annotations; /// Adds a prefix on every keys, to group them inside a common subspace [PublicAPI] - public class FdbSubspace : IFdbSubspace, IEquatable, IComparable + public class KeySubspace : IKeySubspace, IEquatable, IComparable { /// Empty subspace, that does not add any prefix to the keys - public static readonly IFdbSubspace Empty = new FdbSubspace(Slice.Empty); + public static readonly IKeySubspace Empty = new KeySubspace(Slice.Empty); /// Binary prefix of this subspace private Slice m_rawPrefix; //PERF: readonly struct /// Returns the key of this directory subspace /// This should only be used by methods that can use the key internally, even if it is not supposed to be exposed (as is the case for directory partitions) - protected Slice InternalKey - { - get { return m_rawPrefix; } - } + protected Slice InternalKey => m_rawPrefix; #region Constructors... /// Wraps an existing subspace, without copying the prefix (if possible) - protected FdbSubspace([NotNull] IFdbSubspace copy) + protected KeySubspace([NotNull] IKeySubspace copy) { Contract.NotNull(copy, nameof(copy)); - var sub = copy as FdbSubspace; - Slice key = sub != null ? sub.m_rawPrefix : copy.Key; + Slice key = copy is KeySubspace sub ? sub.m_rawPrefix : copy.GetPrefix(); if (key.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(copy)); m_rawPrefix = key; } @@ -68,7 +63,7 @@ protected FdbSubspace([NotNull] IFdbSubspace copy) /// Create a new subspace from a binary prefix /// Prefix of the new subspace /// If true, take a copy of the prefix - internal FdbSubspace(Slice rawPrefix, bool copy) + internal KeySubspace(Slice rawPrefix, bool copy) { if (rawPrefix.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(rawPrefix)); if (copy) rawPrefix = rawPrefix.Memoize(); @@ -77,7 +72,7 @@ internal FdbSubspace(Slice rawPrefix, bool copy) /// Create a new subspace from a binary prefix /// Prefix of the new subspace - public FdbSubspace(Slice rawPrefix) + public KeySubspace(Slice rawPrefix) : this(rawPrefix, true) { } @@ -89,9 +84,9 @@ public FdbSubspace(Slice rawPrefix) /// Prefix of the new subspace /// New subspace that will use a copy of as its prefix [Pure, NotNull] - public static IFdbSubspace Create(Slice slice) + public static IKeySubspace Create(Slice slice) { - return new FdbDynamicSubspace(slice, TypeSystem.Default.GetDynamicEncoder()); + return new DynamicKeySubspace(slice, TypeSystem.Default.GetDynamicEncoder()); } /// Create a new Subspace using a binary key as the prefix @@ -99,10 +94,10 @@ public static IFdbSubspace Create(Slice slice) /// Type System used to encode the keys of this subspace /// New subspace that will use a copy of as its prefix [Pure, NotNull] - public static IFdbDynamicSubspace CreateDynamic(Slice slice, IFdbKeyEncoding encoding = null) + public static IDynamicKeySubspace CreateDynamic(Slice slice, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new FdbDynamicSubspace(slice, encoder); + return new DynamicKeySubspace(slice, encoder); } /// Create a new Subspace using a binary key as the prefix @@ -110,10 +105,10 @@ public static IFdbDynamicSubspace CreateDynamic(Slice slice, IFdbKeyEncoding enc /// Type System used to encode the keys of this subspace /// New subspace that will use a copy of as its prefix [Pure, NotNull] - public static IFdbDynamicSubspace CreateDynamic(Slice slice, [NotNull] IDynamicKeyEncoder encoder) + public static IDynamicKeySubspace CreateDynamic(Slice slice, [NotNull] IDynamicKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbDynamicSubspace(slice, encoder); + return new DynamicKeySubspace(slice, encoder); } /// Create a new Subspace using a tuples as the prefix @@ -121,179 +116,177 @@ public static IFdbDynamicSubspace CreateDynamic(Slice slice, [NotNull] IDynamicK /// Optional type encoding used by this subspace. /// New subspace instance that will use the packed representation of as its prefix [Pure, NotNull] - public static IFdbDynamicSubspace CreateDynamic([NotNull] ITuple tuple, IFdbKeyEncoding encoding = null) + public static IDynamicKeySubspace CreateDynamic([NotNull] ITuple tuple, IKeyEncoding encoding = null) { Contract.NotNull(tuple, nameof(tuple)); var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new FdbDynamicSubspace(tuple.ToSlice(), true, encoder); + return new DynamicKeySubspace(tuple.ToSlice(), true, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, IKeyEncoder encoder) + public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } [Pure, NotNull] - public static IFdbEncoderSubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(slice, encoder); + return new TypedKeySubspace(slice, encoder); } /// Clone this subspace /// New Subspace that uses the same prefix key /// Hint: Cloning a special Subspace like a or will not keep all the "special abilities" of the parent. [Pure, NotNull] - public static IFdbSubspace Copy([NotNull] IFdbSubspace subspace) + public static IKeySubspace Copy([NotNull] IKeySubspace subspace) { - var dyn = subspace as FdbDynamicSubspace; - if (dyn != null) + if (subspace is DynamicKeySubspace dyn) { - return new FdbDynamicSubspace(dyn.InternalKey, true, dyn.Encoder); + return new DynamicKeySubspace(dyn.InternalKey, copy: true, encoder: dyn.Encoder); } - var sub = subspace as FdbSubspace; - if (sub != null) + if (subspace is KeySubspace sub) { //SPOILER WARNING: You didn't hear it from me, but some say that you can use this to bypass the fact that FdbDirectoryPartition.get_Key and ToRange() throws in v2.x ... If you bypass this protection and bork your database, don't come crying! - return new FdbSubspace(sub.InternalKey, true); + return new KeySubspace(sub.InternalKey, copy: true); } - return new FdbSubspace(subspace.Key, true); + return new KeySubspace(subspace.GetPrefix(), copy: true); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbDynamicSubspace CopyDynamic([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) + public static IDynamicKeySubspace CopyDynamic([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new FdbDynamicSubspace(subspace.Key, true, encoder); + return new DynamicKeySubspace(subspace.GetPrefix(), true, encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbDynamicSubspace CopyDynamic([NotNull] IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + public static IDynamicKeySubspace CopyDynamic([NotNull] IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbDynamicSubspace(subspace.Key, true, encoder); + return new DynamicKeySubspace(subspace.GetPrefix(), true, encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), true, encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, IFdbKeyEncoding encoding = null) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) { var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } /// Create a copy of a subspace, using a specific Type System /// New Subspace that uses the same prefix key, and the provided Type System [Pure, NotNull] - public static IFdbEncoderSubspace CopyEncoder([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); - return new FdbEncoderSubspace(subspace.Key, true, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); } #endregion @@ -302,9 +295,9 @@ public static IFdbEncoderSubspace CopyEncoder([N /// Returns the raw prefix of this subspace /// Will throw if the prefix is not publicly visible, as is the case for Directory Partitions - public Slice Key + public Slice GetPrefix() { - get { return GetKeyPrefix(); } + return GetKeyPrefix(); } /// Returns the key to use when creating direct keys that are inside this subspace @@ -316,9 +309,9 @@ protected virtual Slice GetKeyPrefix() return m_rawPrefix; } - protected virtual IFdbSubspace CreateChildren(Slice suffix) + protected virtual IKeySubspace CreateChildren(Slice suffix) { - return new FdbSubspace(ConcatKey(suffix)); + return new KeySubspace(ConcatKey(suffix)); } public KeyRange ToRange() @@ -331,10 +324,7 @@ public virtual KeyRange ToRange(Slice suffix) return KeyRange.StartsWith(ConcatKey(suffix)); } - public IFdbSubspace this[Slice suffix] - { - get { return CreateChildren(suffix); } - } + public IKeySubspace this[Slice suffix] => CreateChildren(suffix); /// Tests whether the specified starts with this Subspace's prefix, indicating that the Subspace logically contains . /// The key to be tested @@ -393,8 +383,7 @@ public Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) var prefix = GetKeyPrefix(); - var arr = keys as Slice[]; - if (arr != null) + if (keys is Slice[] arr) { // fast-path for Sice[] (frequent for range reads) var res = new Slice[arr.Length]; @@ -413,8 +402,7 @@ public Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) } else { // slow path for the rest - var coll = keys as ICollection; - var res = coll != null ? new List(coll.Count) : new List(); + var res = keys is ICollection coll ? new List(coll.Count) : new List(); foreach(var key in keys) { if (key.StartsWith(prefix)) @@ -448,33 +436,31 @@ public SliceWriter GetWriter(int capacity = 0) #region IEquatable / IComparable... /// Compare this subspace with another subspace - public int CompareTo(IFdbSubspace other) + public int CompareTo(IKeySubspace other) { if (other == null) return +1; if (object.ReferenceEquals(this, other)) return 0; - var sub = other as FdbSubspace; - if (sub != null) + if (other is KeySubspace sub) return this.InternalKey.CompareTo(sub.InternalKey); else - return this.InternalKey.CompareTo(other.Key); + return this.InternalKey.CompareTo(other.GetPrefix()); } /// Test if both subspaces have the same prefix - public bool Equals(IFdbSubspace other) + public bool Equals(IKeySubspace other) { if (other == null) return false; if (object.ReferenceEquals(this, other)) return true; - var sub = other as FdbSubspace; - if (sub != null) + if (other is KeySubspace sub) return this.InternalKey.Equals(sub.InternalKey); else - return this.InternalKey.Equals(other.Key); + return this.InternalKey.Equals(other.GetPrefix()); } /// Test if an object is a subspace with the same prefix public override bool Equals(object obj) { - return Equals(obj as FdbSubspace); + return Equals(obj as KeySubspace); } /// Compute a hashcode based on the prefix of this subspace diff --git a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs similarity index 76% rename from FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs rename to FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index af706bb87..da7aaa336 100644 --- a/FoundationDB.Client/Subspaces/FdbSubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -35,8 +35,8 @@ namespace FoundationDB.Client using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; - /// Extensions methods to add FdbSubspace overrides to various types - public static class FdbSubspaceExtensions + /// Extensions methods and helpers to work with Key Subspaces + public static class KeySubspaceExtensions { /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -44,11 +44,11 @@ public static class FdbSubspaceExtensions /// If non-null, uses this specific instance of the TypeSystem. If null, uses the default instance for this particular TypeSystem /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbDynamicSubspace Using([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) + public static IDynamicKeySubspace Using([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return FdbSubspace.CopyDynamic(subspace, encoding); + return KeySubspace.CopyDynamic(subspace, encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -56,11 +56,11 @@ public static IFdbDynamicSubspace Using([NotNull] this IFdbSubspace subspace, [N /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbDynamicSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + public static IDynamicKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return FdbSubspace.CopyDynamic(subspace, encoder); + return KeySubspace.CopyDynamic(subspace, encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -68,11 +68,11 @@ public static IFdbDynamicSubspace UsingEncoder([NotNull] this IFdbSubspace subsp /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return FdbSubspace.CopyEncoder(subspace, encoding); + return KeySubspace.CopyEncoder(subspace, encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -80,11 +80,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IKeyEncoder encoder) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return FdbSubspace.CopyEncoder(subspace, encoder); + return KeySubspace.CopyEncoder(subspace, encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -92,11 +92,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return FdbSubspace.CopyEncoder(subspace, encoding); + return KeySubspace.CopyEncoder(subspace, encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -104,11 +104,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IF /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return FdbSubspace.CopyEncoder(subspace, encoder); + return KeySubspace.CopyEncoder(subspace, encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -116,11 +116,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] this IF /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return FdbSubspace.CopyEncoder(subspace, encoding); + return KeySubspace.CopyEncoder(subspace, encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -128,11 +128,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return FdbSubspace.CopyEncoder(subspace, encoder); + return KeySubspace.CopyEncoder(subspace, encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -140,11 +140,11 @@ public static IFdbEncoderSubspace UsingEncoder([NotNull] /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] IFdbKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return FdbSubspace.CopyEncoder(subspace, encoding); + return KeySubspace.CopyEncoder(subspace, encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -152,15 +152,15 @@ public static IFdbEncoderSubspace UsingEncoder([ /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IFdbEncoderSubspace UsingEncoder([NotNull] this IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return FdbSubspace.CopyEncoder(subspace, encoder); + return KeySubspace.CopyEncoder(subspace, encoder); } /// Clear the entire content of a subspace - public static void ClearRange(this IFdbTransaction trans, [NotNull] IFdbSubspace subspace) + public static void ClearRange(this IFdbTransaction trans, [NotNull] IKeySubspace subspace) { Contract.Requires(trans != null && subspace != null); @@ -169,7 +169,7 @@ public static void ClearRange(this IFdbTransaction trans, [NotNull] IFdbSubspace } /// Clear the entire content of a subspace - public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IFdbSubspace subspace, CancellationToken cancellationToken) + public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IKeySubspace subspace, CancellationToken cancellationToken) { Contract.NotNull(db, nameof(db)); Contract.NotNull(subspace, nameof(subspace)); @@ -179,7 +179,7 @@ public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IFdbSubspace /// Returns all the keys inside of a subspace [Pure, NotNull] - public static FdbRangeQuery> GetRangeStartsWith(this IFdbReadOnlyTransaction trans, [NotNull] IFdbSubspace subspace, FdbRangeOptions options = null) + public static FdbRangeQuery> GetRangeStartsWith(this IFdbReadOnlyTransaction trans, [NotNull] IKeySubspace subspace, FdbRangeOptions options = null) { //REVIEW: should we remove this method? Contract.Requires(trans != null && subspace != null); diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs new file mode 100644 index 000000000..010265f25 --- /dev/null +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs @@ -0,0 +1,162 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using FoundationDB.Client.Utils; + +namespace FoundationDB.Client +{ + using System; + using System.Collections.Generic; + using System.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + + /// Subspace that knows how to encode and decode its key + /// Type of the key handled by this subspace + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + public TypedKeySubspace(Slice rawPrefix, [NotNull] IKeyEncoder encoder) + : this(rawPrefix, true, encoder) + { } + + internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] IKeyEncoder encoder) + : base(rawPrefix, copy) + { + this.Encoder = encoder ?? throw new ArgumentNullException(nameof(encoder)); + this.Keys = new TypedKeys(this, encoder); + } + + [NotNull] + public IKeyEncoder Encoder { get; } + + public TypedKeys Keys { get; } + + public TypedSubspacePartition Partition => new TypedSubspacePartition(this, Encoder); + } + + /// Encodes and Decodes keys composed of a single element + /// Type of the key handled by this subspace + public /*readonly*/ struct TypedKeys + { + + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly IKeyEncoder Encoder; + + public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + public Slice this[T value] => Encode(value); + + public Slice Encode(T value) + { + return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)); + } + + public Slice[] Encode([NotNull] IEnumerable values) + { + if (values == null) throw new ArgumentNullException(nameof(values)); + return Batched>.Convert( + this.Subspace.GetWriter(), + values, + (ref SliceWriter writer, T value, IKeyEncoder encoder) => { writer.WriteBytes(encoder.EncodeKey(value)); }, + this.Encoder + ); + } + + public T Decode(Slice packed) + { + return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + } + + public KeyRange ToRange(T value) + { + //REVIEW: which semantic for ToRange() should we use? + return STuple.ToRange(Encode(value)); + } + + } + + public /*readonly*/ struct TypedSubspacePartition + { + + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly IKeyEncoder Encoder; + + public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + [NotNull] + public IKeySubspace this[T value] => ByKey(value); + + [NotNull] + public IKeySubspace ByKey(T value) + { + return this.Subspace[this.Encoder.EncodeKey(value)]; + } + + [NotNull] + public IDynamicKeySubspace ByKey(T value, [NotNull] IKeyEncoding encoding) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); + } + + [NotNull] + public IDynamicKeySubspace ByKey(T value, [NotNull] IDynamicKeyEncoder encoder) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); + } + + [NotNull] + public ITypedKeySubspace ByKey(T value, [NotNull] IKeyEncoding encoding) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); + } + + [NotNull] + public ITypedKeySubspace ByKey(T value, [NotNull] IKeyEncoder encoder) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); + } + + } + +} diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs new file mode 100644 index 000000000..6a7f0a472 --- /dev/null +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -0,0 +1,166 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using FoundationDB.Client.Utils; + +namespace FoundationDB.Client +{ + using System; + using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + using FoundationDB.Layers.Tuples; + + /// Subspace that knows how to encode and decode its key + /// Type of the first item of the keys handled by this subspace + /// Type of the second item of the keys handled by this subspace + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + // ReSharper disable once FieldCanBeMadeReadOnly.Local + + public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) + : this(rawPrefix, true, encoder) + { } + + internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) + : base(rawPrefix, copy) + { + this.Encoder = encoder ?? throw new ArgumentNullException(nameof(encoder)); + this.Keys = new TypedKeys(this, encoder); + this.Partition = new TypedSubspacePartition(this, Encoder); + } + + public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); + private TypedKeySubspace m_partial; + + public ICompositeKeyEncoder Encoder { get; } + + public TypedKeys Keys { get; } + + public TypedSubspacePartition Partition { get; } + + } + + public /*readonly*/ struct TypedKeys + { + + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly ICompositeKeyEncoder Encoder; + + public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + public Slice this[T1 value1, T2 value2] => Encode(value1, value2); + + public Slice Encode(T1 value1, T2 value2) + { + return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)); + } + + public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2) + { + Contract.NotNull(values, nameof(values)); + return Batched>.Convert( + this.Subspace.GetWriter(), + values, + (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value))), + this.Encoder + ); + } + + public STuple Decode(Slice packed) + { + return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + } + + public KeyRange ToRange(T1 value1, T2 value2) + { + //REVIEW: which semantic for ToRange() should we use? + return STuple.ToRange(Encode(value1, value2)); + } + + } + + public /*readonly*/ struct TypedSubspacePartition + { + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly ICompositeKeyEncoder Encoder; + + public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + [NotNull] + public IKeySubspace this[T1 value1, T2 value2] => ByKey(value1, value2); + + [NotNull] + public IKeySubspace ByKey(T1 value1, T2 value2) + { + return this.Subspace[this.Encoder.EncodeKey(value1, value2)]; + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoding encoding) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, IDynamicKeyEncoder encoder) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoding encoding) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoder encoder) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); + } + + } + +} diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs new file mode 100644 index 000000000..1b658f547 --- /dev/null +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -0,0 +1,169 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using FoundationDB.Client.Utils; + +namespace FoundationDB.Client +{ + using System; + using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + using FoundationDB.Layers.Tuples; + + /// Subspace that knows how to encode and decode its key + /// Type of the first item of the keys handled by this subspace + /// Type of the second item of the keys handled by this subspace + /// Type of the third item of the keys handled by this subspace + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) + : this(rawPrefix, true, encoder) + { } + + internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) + : base(rawPrefix, copy) + { + Contract.Requires(encoder != null); + this.Encoder = encoder; + this.Keys = new TypedKeys(this, encoder); + this.Partition = new TypedSubspacePartition(this, Encoder); + } + + public ITypedKeySubspace Head => m_head ?? (m_head = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); + private TypedKeySubspace m_head; + + public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(Encoder))); + private TypedKeySubspace m_partial; + + public ICompositeKeyEncoder Encoder { get; } + + public TypedKeys Keys { get; } + + public TypedSubspacePartition Partition { get; } + + } + + /// Encodes and Decodes keys composed of three elements + /// Type of the first item of the keys handled by this subspace + /// Type of the second item of the keys handled by this subspace + /// Type of the third item of the keys handled by this subspace + public /*readonly*/ struct TypedKeys + { + + public readonly IKeySubspace Subspace; + public readonly ICompositeKeyEncoder Encoder; + + public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + this.Subspace = subspace; + this.Encoder = encoder; + } + + public Slice this[T1 value1, T2 value2, T3 value3] => Encode(value1, value2, value3); + + public Slice Encode(T1 value1, T2 value2, T3 value3) + { + return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)); + } + + public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3) + { + if (values == null) throw new ArgumentNullException(nameof(values)); + return Batched>.Convert( + this.Subspace.GetWriter(), + values, + (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value))), + this.Encoder + ); + } + + public STuple Decode(Slice packed) + { + return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + } + + public KeyRange ToRange(T1 value1, T2 value2, T3 value3) + { + //REVIEW: which semantic for ToRange() should we use? + return STuple.ToRange(Encode(value1, value2, value3)); + } + + } + + public /*readonly*/ struct TypedSubspacePartition + { + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly ICompositeKeyEncoder Encoder; + + public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + [NotNull] + public IKeySubspace this[T1 value1, T2 value2, T3 value3] => ByKey(value1, value2, value3); + + [NotNull] + public IKeySubspace ByKey(T1 value1, T2 value2, T3 value3) + { + return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3)]; + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoding encoding) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IDynamicKeyEncoder encoder) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoding encoding) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoder encoder) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); + } + + } + +} diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs new file mode 100644 index 000000000..5a13c2ec2 --- /dev/null +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -0,0 +1,178 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using FoundationDB.Client.Utils; + +namespace FoundationDB.Client +{ + using System; + using System.Collections.Generic; + using Doxense.Diagnostics.Contracts; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; + + /// Subspace that knows how to encode and decode its key + /// Type of the first item of the keys handled by this subspace + /// Type of the second item of the keys handled by this subspace + /// Type of the third item of the keys handled by this subspace + /// Type of the fourth item of the keys handled by this subspace + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + // ReSharper disable once FieldCanBeMadeReadOnly.Local + private TypedKeySubspace m_head; + private TypedKeySubspace m_partial; + + public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) + : this(rawPrefix, true, encoder) + { } + + internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) + : base(rawPrefix, copy) + { + Contract.Requires(encoder != null); + this.Encoder = encoder; + this.Keys = new TypedKeys(this, encoder); + this.Partition = new TypedSubspacePartition(this, Encoder); + } + + public ITypedKeySubspace Head => m_head ?? (m_head = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); + + public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(Encoder))); + + public ICompositeKeyEncoder Encoder { get; } + + public TypedKeys Keys { get; } + + public TypedSubspacePartition Partition { get; } + + } + + /// Encodes and Decodes keys composed of four elements + /// Type of the first item of the keys handled by this subspace + /// Type of the second item of the keys handled by this subspace + /// Type of the third item of the keys handled by this subspace + /// Type of the fourth item of the keys handled by this subspace + public /*readonly*/ struct TypedKeys + { + + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly ICompositeKeyEncoder Encoder; + + public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + System.Diagnostics.Contracts.Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + public Slice this[T1 value1, T2 value2, T3 value3, T4 value4] => Encode(value1, value2, value3, value4); + + public Slice Encode(T1 value1, T2 value2, T3 value3, T4 value4) + { + return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)); + } + + [NotNull] + public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3, [NotNull] Func selector4) + { + if (values == null) throw new ArgumentNullException(nameof(values)); + return Batched>.Convert( + this.Subspace.GetWriter(), + values, + (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value), selector4(value))), + this.Encoder + ); + } + + public STuple Decode(Slice packed) + { + return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + } + + public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) + { + //REVIEW: which semantic for ToRange() should we use? + return STuple.ToRange(Encode(value1, value2, value3, value4)); + } + + } + + public /*readonly*/ struct TypedSubspacePartition + { + [NotNull] + public readonly IKeySubspace Subspace; + + [NotNull] + public readonly ICompositeKeyEncoder Encoder; + + public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.Requires(subspace != null && encoder != null); + this.Subspace = subspace; + this.Encoder = encoder; + } + + [NotNull] + public IKeySubspace this[T1 value1, T2 value2, T3 value3, T4 value4] => ByKey(value1, value2, value3, value4); + + [NotNull] + public IKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) + { + return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3, value4)]; + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoding encoding) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); + } + + [NotNull] + public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IDynamicKeyEncoder encoder) + { + return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoding encoding) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); + } + + [NotNull] + public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoder encoder) + { + return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); + } + + } + +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index de76de445..96ba147cb 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -6,7 +6,7 @@ namespace FoundationDB.Client public abstract class DynamicKeyEncoderBase : IDynamicKeyEncoder { - public abstract IFdbKeyEncoding Encoding { get; } + public abstract IKeyEncoding Encoding { get; } public virtual KeyRange ToRange(Slice prefix) { diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs index 3929d8ee2..8312636bd 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs @@ -38,7 +38,7 @@ public interface IDynamicKeyEncoder { /// Return the parent key encoding - IFdbKeyEncoding Encoding {[NotNull] get; } + IKeyEncoding Encoding {[NotNull] get; } /// Return a range that contains all the keys under a subspace of the encoder subspace, using the semantic of the encoding /// Optional binary prefix diff --git a/FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs b/FoundationDB.Client/TypeSystem/IKeyEncoder`1.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/IKeyValueEncoder.cs rename to FoundationDB.Client/TypeSystem/IKeyEncoder`1.cs diff --git a/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs b/FoundationDB.Client/TypeSystem/IKeyEncoding.cs similarity index 89% rename from FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs rename to FoundationDB.Client/TypeSystem/IKeyEncoding.cs index 3417b27a4..8d6a2807d 100644 --- a/FoundationDB.Client/TypeSystem/IFdbKeyEncoding.cs +++ b/FoundationDB.Client/TypeSystem/IKeyEncoding.cs @@ -31,8 +31,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client //REVIEW: what namespace? { - /// Type system that handles values of arbitrary sizes and types - public interface IFdbKeyEncoding + /// Type system that handles encoding and decoding of differnt types of keys + /// + /// An implementation of this interface knows to create different types of Key Encoders that will all use the same "binary format" to encode and decode keys of various shapes. + /// A good analogy for values would be a 'JSON' encoding, or 'XML' encoding. + /// + public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTypeSystem"? { /// Returns an encoder which can process keys of any size and types diff --git a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs index d26d6136d..bc3326eea 100644 --- a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs @@ -33,9 +33,11 @@ namespace FoundationDB.Client public interface IOrderedTypeCodec { void EncodeOrderedSelfTerm(ref SliceWriter output, T value); + T DecodeOrderedSelfTerm(ref SliceReader input); Slice EncodeOrdered(T value); + T DecodeOrdered(Slice input); } diff --git a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs index 75112738f..fb7bdd866 100644 --- a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs @@ -33,9 +33,11 @@ namespace FoundationDB.Client public interface IUnorderedTypeCodec { void EncodeUnorderedSelfTerm(ref SliceWriter output, T value); + T DecodeUnorderedSelfTerm(ref SliceReader input); Slice EncodeUnordered(T value); + T DecodeUnordered(Slice input); } diff --git a/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs similarity index 94% rename from FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs rename to FoundationDB.Client/TypeSystem/TypeCodec`1.cs index 6bdbc589b..5424672fa 100644 --- a/FoundationDB.Client/TypeSystem/FdbTypeCodec`1.cs +++ b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs @@ -26,14 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System.ComponentModel; -using System.Runtime.CompilerServices; - namespace FoundationDB.Client { using System; - public abstract class FdbTypeCodec : IOrderedTypeCodec, IUnorderedTypeCodec + public abstract class TypeCodec : IOrderedTypeCodec, IUnorderedTypeCodec { public abstract void EncodeOrderedSelfTerm(ref SliceWriter output, T value); diff --git a/FoundationDB.Client/TypeSystem/TypeSystem.cs b/FoundationDB.Client/TypeSystem/TypeSystem.cs index 0fe33ef11..71239cc25 100644 --- a/FoundationDB.Client/TypeSystem/TypeSystem.cs +++ b/FoundationDB.Client/TypeSystem/TypeSystem.cs @@ -35,10 +35,10 @@ namespace FoundationDB.Client //REVIEW: what namespace? public static class TypeSystem { [NotNull] - public static readonly IFdbKeyEncoding Default; + public static readonly IKeyEncoding Default; [NotNull] - public static readonly IFdbKeyEncoding Tuples; + public static readonly IKeyEncoding Tuples; static TypeSystem() { diff --git a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs b/FoundationDB.Client/Utils/Batched.cs similarity index 51% rename from FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs rename to FoundationDB.Client/Utils/Batched.cs index ce97fc2a4..76cb4de9a 100644 --- a/FoundationDB.Client/Subspaces/FdbEncoderSubspaceKeys`2.cs +++ b/FoundationDB.Client/Utils/Batched.cs @@ -1,4 +1,4 @@ -#region BSD Licence +#region BSD Licence /* Copyright (c) 2013-2018, Doxense SAS All rights reserved. @@ -26,61 +26,64 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB.Client.Utils { using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; - public struct FdbEncoderSubspaceKeys + internal static class Batched { - [NotNull] - public readonly IFdbSubspace Subspace; + public delegate void Handler(ref SliceWriter writer, TValue item, TState state); [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public FdbEncoderSubspaceKeys([NotNull] IFdbSubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static Slice[] Convert(SliceWriter writer, [NotNull, ItemNotNull] IEnumerable values, Handler handler, TState state) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; - } + Contract.Requires(values != null && handler != null); - public Slice this[T1 value1, T2 value2] - { - get { return Encode(value1, value2); } - } + //Note on performance: + // - we will reuse the same buffer for each temp key, and copy them into a slice buffer + // - doing it this way adds a memory copy (writer => buffer) but reduce the number of byte[] allocations (and reduce the GC overhead) - public Slice Encode(T1 value1, T2 value2) - { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)); - } + int start = writer.Position; - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2) - { - Contract.NotNull(values, nameof(values)); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value))), - this.Encoder - ); - } + var buffer = new SliceBuffer(); - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } + if (values is ICollection coll) + { // pre-allocate the final array with the correct size + var res = new Slice[coll.Count]; + int p = 0; + foreach (var tuple in coll) + { + // reset position to just after the subspace prefix + writer.Position = start; - public KeyRange ToRange(T1 value1, T2 value2) - { - //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2)); - } + handler(ref writer, tuple, state); + // copy full key in the buffer + res[p++] = buffer.Intern(writer.ToSlice()); + } + Contract.Assert(p == res.Length); + return res; + } + else + { // we won't now the array size until the end... + var res = new List(); + foreach (var tuple in values) + { + // reset position to just after the subspace prefix + writer.Position = start; + + handler(ref writer, tuple, state); + + // copy full key in the buffer + res.Add(buffer.Intern(writer.ToSlice())); + } + return res.ToArray(); + } + } } + } diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 4bc98a833..2cf43d9be 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -52,7 +52,7 @@ public class FdbBlob /// Only keys within the subspace will be used by the object. /// Other clients of the database should refrain from modifying the subspace. /// Subspace to be used for storing the blob data and metadata - public FdbBlob([NotNull] IFdbSubspace subspace) + public FdbBlob([NotNull] IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException("subspace"); @@ -60,7 +60,7 @@ public FdbBlob([NotNull] IFdbSubspace subspace) } /// Subspace used as a prefix for all items in this table - public IFdbDynamicSubspace Subspace {[NotNull] get; private set; } + public IDynamicKeySubspace Subspace {[NotNull] get; private set; } /// Returns the key for data chunk at the specified offset /// diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index e5dd2c532..a1a79521a 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -43,11 +43,11 @@ namespace FoundationDB.Layers.Collections public class FdbMap { - public FdbMap([NotNull] string name, [NotNull] IFdbSubspace subspace, [NotNull] IValueEncoder valueEncoder) + public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] IValueEncoder valueEncoder) : this(name, subspace, KeyValueEncoders.Tuples.Key(), valueEncoder) { } - public FdbMap([NotNull] string name, [NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) + public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) { if (name == null) throw new ArgumentNullException("name"); if (subspace == null) throw new ArgumentNullException("subspace"); @@ -67,10 +67,10 @@ public FdbMap([NotNull] string name, [NotNull] IFdbSubspace subspace, [NotNull] public string Name { [NotNull] get; private set; } /// Subspace used as a prefix for all items in this map - public IFdbSubspace Subspace { [NotNull] get; private set; } + public IKeySubspace Subspace { [NotNull] get; private set; } /// Subspace used to encoded the keys for the items - protected IFdbEncoderSubspace Location { [NotNull] get; private set; } + protected ITypedKeySubspace Location { [NotNull] get; private set; } /// Class that can serialize/deserialize values into/from slices public IValueEncoder ValueEncoder { [NotNull] get; private set; } diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index 2891409bf..a54e6a2b7 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -54,7 +54,7 @@ public class FdbMultiMap /// Create a new multimap /// Location where the map will be stored in the database /// If true, allow negative or zero values to stay in the map. - public FdbMultiMap(IFdbSubspace subspace, bool allowNegativeValues) + public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues) : this(subspace, allowNegativeValues, KeyValueEncoders.Tuples.CompositeKey()) { } @@ -62,7 +62,7 @@ public FdbMultiMap(IFdbSubspace subspace, bool allowNegativeValues) /// Location where the map will be stored in the database /// If true, allow negative or zero values to stay in the map. /// Encoder for the key/value pairs - public FdbMultiMap(IFdbSubspace subspace, bool allowNegativeValues, ICompositeKeyEncoder encoder) + public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues, ICompositeKeyEncoder encoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (encoder == null) throw new ArgumentNullException("encoder"); @@ -75,13 +75,13 @@ public FdbMultiMap(IFdbSubspace subspace, bool allowNegativeValues, ICompositeKe #region Public Properties... /// Subspace used as a prefix for all items in this map - public IFdbSubspace Subspace { [NotNull] get; } + public IKeySubspace Subspace { [NotNull] get; } /// If true, allow negative or zero values to stay in the map. public bool AllowNegativeValues { get; } /// Subspace used to encoded the keys for the items - protected IFdbEncoderSubspace Location { [NotNull] get; } + protected ITypedKeySubspace Location { [NotNull] get; } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index e05f531b7..d5301ce17 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -52,7 +52,7 @@ public class FdbQueue /// Create a new High Contention Queue /// Subspace where the queue will be stored /// Uses the default Tuple serializer - public FdbQueue([NotNull] FdbSubspace subspace) + public FdbQueue([NotNull] KeySubspace subspace) : this(subspace, highContention: true, encoder: KeyValueEncoders.Tuples.Value()) { } @@ -60,14 +60,14 @@ public FdbQueue([NotNull] FdbSubspace subspace) /// Subspace where the queue will be stored /// If true, uses High Contention Mode (lots of popping clients). If true, uses the Simple Mode (a few popping clients). /// Uses the default Tuple serializer - public FdbQueue([NotNull] FdbSubspace subspace, bool highContention) + public FdbQueue([NotNull] KeySubspace subspace, bool highContention) : this(subspace, highContention: highContention, encoder: KeyValueEncoders.Tuples.Value()) { } /// Create a new queue using either High Contention mode or Simple mode /// Subspace where the queue will be stored /// If true, uses High Contention Mode (lots of popping clients). If true, uses the Simple Mode (a few popping clients). - public FdbQueue([NotNull] IFdbSubspace subspace, bool highContention, [NotNull] IValueEncoder encoder) + public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] IValueEncoder encoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (encoder == null) throw new ArgumentNullException("encoder"); @@ -83,7 +83,7 @@ public FdbQueue([NotNull] IFdbSubspace subspace, bool highContention, [NotNull] } /// Subspace used as a prefix for all items in this table - public IFdbDynamicSubspace Subspace { [NotNull] get; private set; } + public IDynamicKeySubspace Subspace { [NotNull] get; private set; } /// If true, the queue is operating in High Contention mode that will scale better with a lot of popping clients. public bool HighContention { get; private set; } @@ -91,11 +91,11 @@ public FdbQueue([NotNull] IFdbSubspace subspace, bool highContention, [NotNull] /// Serializer for the elements of the queue public IValueEncoder Encoder { [NotNull] get; private set; } - internal IFdbDynamicSubspace ConflictedPop { get; private set; } + internal IDynamicKeySubspace ConflictedPop { get; private set; } - internal IFdbDynamicSubspace ConflictedItem { get; private set; } + internal IDynamicKeySubspace ConflictedItem { get; private set; } - internal IFdbDynamicSubspace QueueItem { get; private set; } + internal IDynamicKeySubspace QueueItem { get; private set; } /// Remove all items from the queue. public void Clear([NotNull] IFdbTransaction trans) @@ -277,7 +277,7 @@ private async Task PushAtAsync([NotNull] IFdbTransaction tr, T value, long index tr.Set(key, this.Encoder.EncodeValue(value)); } - private async Task GetNextIndexAsync([NotNull] IFdbReadOnlyTransaction tr, IFdbDynamicSubspace subspace) + private async Task GetNextIndexAsync([NotNull] IFdbReadOnlyTransaction tr, IDynamicKeySubspace subspace) { var range = subspace.Keys.ToRange(); diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index db54e1923..1c104865b 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -51,7 +51,7 @@ public class FdbRankedSet /// Initializes a new ranked set at a given location /// Subspace where the set will be stored - public FdbRankedSet([NotNull] IFdbSubspace subspace) + public FdbRankedSet([NotNull] IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException("subspace"); @@ -65,7 +65,7 @@ public Task OpenAsync([NotNull] IFdbTransaction trans) } /// Subspace used as a prefix for all items in this table - public IFdbDynamicSubspace Subspace { [NotNull] get; private set; } + public IDynamicKeySubspace Subspace { [NotNull] get; private set; } /// Returns the number of items in the set. /// diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index 9db163605..3f8c60fb7 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -60,16 +60,16 @@ public class FdbVector /// Create a new sparse Vector /// Subspace where the vector will be stored /// Sparse entries will be assigned the value Slice.Empty - public FdbVector([NotNull] FdbSubspace subspace) + public FdbVector([NotNull] KeySubspace subspace) : this(subspace, default(T)) { } /// Create a new sparse Vector /// Subspace where the vector will be stored /// Default value for sparse entries - public FdbVector([NotNull] IFdbSubspace subspace, T defaultValue) + public FdbVector([NotNull] IKeySubspace subspace, T defaultValue) : this(subspace, defaultValue, KeyValueEncoders.Tuples.Value()) { } - public FdbVector([NotNull] IFdbSubspace subspace, T defaultValue, [NotNull] IValueEncoder encoder) + public FdbVector([NotNull] IKeySubspace subspace, T defaultValue, [NotNull] IValueEncoder encoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (encoder == null) throw new ArgumentNullException("encoder"); @@ -81,7 +81,7 @@ public FdbVector([NotNull] IFdbSubspace subspace, T defaultValue, [NotNull] IVal /// Subspace used as a prefix for all items in this vector - public IFdbDynamicSubspace Subspace { [NotNull] get; private set; } + public IDynamicKeySubspace Subspace { [NotNull] get; private set; } /// Default value for sparse entries public T DefaultValue { get; private set; } diff --git a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs index 32869753e..254502bbd 100644 --- a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs +++ b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs @@ -41,12 +41,12 @@ public sealed class FdbCounterMap private static readonly Slice MinusOne = Slice.FromFixed64(-1); /// Create a new counter map. - public FdbCounterMap([NotNull] IFdbSubspace subspace) + public FdbCounterMap([NotNull] IKeySubspace subspace) : this(subspace, KeyValueEncoders.Tuples.Key()) { } /// Create a new counter map, using a specific key encoder. - public FdbCounterMap([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoder keyEncoder) + public FdbCounterMap([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder keyEncoder) { if (subspace == null) throw new ArgumentNullException("subspace"); if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); @@ -57,12 +57,12 @@ public FdbCounterMap([NotNull] IFdbSubspace subspace, [NotNull] IKeyEncoderSubspace used as a prefix for all items in this counter list - public IFdbSubspace Subspace { [NotNull] get; private set; } + public IKeySubspace Subspace { [NotNull] get; private set; } /// Encoder for the keys of the counter map public IKeyEncoder KeyEncoder { [NotNull] get; private set; } - internal IFdbEncoderSubspace Location { [NotNull] get; private set; } + internal ITypedKeySubspace Location { [NotNull] get; private set; } /// Add a value to a counter in one atomic operation /// diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index 397837e27..e39e430cd 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -54,7 +54,7 @@ public class FdbHighContentionCounter /// Create a new High Contention counter. /// Database used by this layer /// Subspace to be used for storing the counter - public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IFdbSubspace subspace) + public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace) : this(db, subspace, KeyValueEncoders.Tuples.Value()) { } @@ -62,7 +62,7 @@ public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IFdbSubspac /// Database used by this layer /// Subspace to be used for storing the counter /// Encoder for the counter values - public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IFdbSubspace subspace, [NotNull] IValueEncoder encoder) + public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace, [NotNull] IValueEncoder encoder) { if (db == null) throw new ArgumentNullException("db"); if (subspace == null) throw new ArgumentNullException("subspace"); @@ -74,7 +74,7 @@ public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IFdbSubspac } /// Subspace used as a prefix for all items in this table - public IFdbDynamicSubspace Subspace {[NotNull] get; private set; } + public IDynamicKeySubspace Subspace {[NotNull] get; private set; } /// Database instance that is used to perform background coalescing of the counter public IFdbDatabase Database {[NotNull] get; private set; } diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index 6bad9e106..fa76eb686 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -44,11 +44,11 @@ namespace FoundationDB.Layers.Indexing public class FdbIndex { - public FdbIndex([NotNull] string name, [NotNull] IFdbSubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) + public FdbIndex([NotNull] string name, [NotNull] IKeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) : this(name, subspace, valueComparer, indexNullValues, KeyValueEncoders.Tuples.CompositeKey()) { } - public FdbIndex([NotNull] string name, [NotNull] IFdbSubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] ICompositeKeyEncoder encoder) + public FdbIndex([NotNull] string name, [NotNull] IKeySubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] ICompositeKeyEncoder encoder) { if (name == null) throw new ArgumentNullException("name"); if (subspace == null) throw new ArgumentNullException("subspace"); @@ -63,9 +63,9 @@ public FdbIndex([NotNull] string name, [NotNull] IFdbSubspace subspace, IEqualit public string Name { [NotNull] get; private set; } - public IFdbSubspace Subspace { [NotNull] get; private set; } + public IKeySubspace Subspace { [NotNull] get; private set; } - protected IFdbEncoderSubspace Location { [NotNull] get; private set; } + protected ITypedKeySubspace Location { [NotNull] get; private set; } public IEqualityComparer ValueComparer { [NotNull] get; private set; } diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index 769959163..a112db88b 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -91,7 +91,7 @@ public override int GetHashCode() private readonly RandomNumberGenerator m_prng = RandomNumberGenerator.Create(); private readonly ReaderWriterLockSlim m_lock = new ReaderWriterLockSlim(); - public FdbStringIntern(IFdbSubspace subspace) + public FdbStringIntern(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException("subspace"); @@ -99,7 +99,7 @@ public FdbStringIntern(IFdbSubspace subspace) } - public IFdbDynamicSubspace Subspace { get; private set; } + public IDynamicKeySubspace Subspace { get; private set; } #region Private Helpers... diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index 3ddc9c587..6125688d5 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -41,11 +41,11 @@ public class FdbDocumentCollection public const int DefaultChunkSize = 1 << 20; // 1 MB - public FdbDocumentCollection(FdbSubspace subspace, Func selector, IValueEncoder valueEncoder) + public FdbDocumentCollection(KeySubspace subspace, Func selector, IValueEncoder valueEncoder) : this(subspace, selector, KeyValueEncoders.Tuples.CompositeKey(), valueEncoder) { } - public FdbDocumentCollection(FdbSubspace subspace, Func selector, ICompositeKeyEncoder keyEncoder, IValueEncoder valueEncoder) + public FdbDocumentCollection(KeySubspace subspace, Func selector, ICompositeKeyEncoder keyEncoder, IValueEncoder valueEncoder) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); if (selector == null) throw new ArgumentNullException(nameof(selector)); @@ -75,9 +75,9 @@ protected virtual TDocument DecodeParts(List parts) } /// Subspace used as a prefix for all hashsets in this collection - public FdbSubspace Subspace { get; } + public KeySubspace Subspace { get; } - protected IFdbEncoderSubspace Location { get; } + protected ITypedKeySubspace Location { get; } /// Encoder that packs/unpacks the documents public IValueEncoder ValueEncoder { get; } diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index d5fadbaf6..0074d51bd 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -44,7 +44,7 @@ namespace FoundationDB.Layers.Blobs public class FdbHashSetCollection { - public FdbHashSetCollection(IFdbSubspace subspace) + public FdbHashSetCollection(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); @@ -52,7 +52,7 @@ public FdbHashSetCollection(IFdbSubspace subspace) } /// Subspace used as a prefix for all hashsets in this collection - public IFdbDynamicSubspace Subspace { get; } + public IDynamicKeySubspace Subspace { get; } /// Returns the key prefix of an HashSet: (subspace, id, ) /// diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index 681d3ff99..955c5e32d 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -46,11 +46,11 @@ namespace FoundationDB.Layers.Experimental.Indexing public class FdbCompressedBitmapIndex { - public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) + public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] KeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) : this(name, subspace, valueComparer, indexNullValues, KeyValueEncoders.Tuples.Key()) { } - public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] IKeyEncoder encoder) + public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] KeySubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] IKeyEncoder encoder) { Contract.NotNull(name, nameof(name)); Contract.NotNull(subspace, nameof(subspace)); @@ -65,9 +65,9 @@ public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] FdbSubspace sub public string Name { [NotNull] get; } - public FdbSubspace Subspace { [NotNull] get; } + public KeySubspace Subspace { [NotNull] get; } - protected IFdbEncoderSubspace Location { [NotNull] get; } + protected ITypedKeySubspace Location { [NotNull] get; } public IEqualityComparer ValueComparer { [NotNull] get; } diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 937fb81ed..f12e4ae0b 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -62,15 +62,15 @@ public class FdbWorkerPool private readonly RandomNumberGenerator m_rng = RandomNumberGenerator.Create(); - public IFdbDynamicSubspace Subspace { get; } + public IDynamicKeySubspace Subspace { get; } - internal IFdbDynamicSubspace TaskStore { get; } + internal IDynamicKeySubspace TaskStore { get; } - internal IFdbDynamicSubspace IdleRing { get; } + internal IDynamicKeySubspace IdleRing { get; } - internal IFdbDynamicSubspace BusyRing { get; } + internal IDynamicKeySubspace BusyRing { get; } - internal IFdbDynamicSubspace UnassignedTaskRing { get; } + internal IDynamicKeySubspace UnassignedTaskRing { get; } internal FdbCounterMap Counters { get; } @@ -106,7 +106,7 @@ public class FdbWorkerPool #endregion - public FdbWorkerPool(IFdbSubspace subspace) + public FdbWorkerPool(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); @@ -120,7 +120,7 @@ public FdbWorkerPool(IFdbSubspace subspace) this.Counters = new FdbCounterMap(this.Subspace.Partition.ByKey(Slice.FromChar('C'))); } - private async Task> FindRandomItem(IFdbTransaction tr, IFdbDynamicSubspace ring) + private async Task> FindRandomItem(IFdbTransaction tr, IDynamicKeySubspace ring) { var range = ring.Keys.ToRange(); @@ -152,7 +152,7 @@ private Slice GetRandomId() } } - private async Task PushQueueAsync(IFdbTransaction tr, IFdbDynamicSubspace queue, Slice taskId) + private async Task PushQueueAsync(IFdbTransaction tr, IDynamicKeySubspace queue, Slice taskId) { //TODO: use a high contention algo ? // - must support Push and Pop @@ -174,7 +174,7 @@ private void StoreTask(IFdbTransaction tr, Slice taskId, DateTime scheduledUtc, var prefix = this.TaskStore.Partition.ByKey(taskId); // store task body and timestamp - tr.Set(prefix.Key, taskBody); + tr.Set(prefix.GetPrefix(), taskBody); tr.Set(prefix.Keys.Encode(TASK_META_SCHEDULED), Slice.FromInt64(scheduledUtc.Ticks)); // increment total and pending number of tasks this.Counters.Increment(tr, COUNTER_TOTAL_TASKS); @@ -320,7 +320,7 @@ await db.ReadWriteAsync( var prefix = this.TaskStore.Partition.ByKey(msg.Id); //TODO: replace this with a get_range ? var data = await tr.GetValuesAsync(new [] { - prefix.Key, + prefix.GetPrefix(), prefix.Keys.Encode(TASK_META_SCHEDULED) }).ConfigureAwait(false); diff --git a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs index 2cbeb5658..b72761018 100644 --- a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs +++ b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs @@ -58,7 +58,7 @@ public void Main() } } - private async Task RunAsync(IFdbDatabase db, IFdbDynamicSubspace location, CancellationToken ct, Action done, int N, int K, int W) + private async Task RunAsync(IFdbDatabase db, IDynamicKeySubspace location, CancellationToken ct, Action done, int N, int K, int W) { if (db == null) throw new ArgumentNullException(nameof(db)); @@ -73,7 +73,7 @@ private async Task RunAsync(IFdbDatabase db, IFdbDynamicSubspace location, Cance { var workerPool = new FdbWorkerPool(location); - Console.WriteLine("workerPool at " + location.Key.ToAsciiOrHexaString()); + Console.WriteLine("workerPool at " + location.GetPrefix().ToAsciiOrHexaString()); var workerSignal = new AsyncCancelableMutex(ct); var clientSignal = new AsyncCancelableMutex(ct); @@ -134,11 +134,11 @@ private async Task RunAsync(IFdbDatabase db, IFdbDynamicSubspace location, Cance Func dump = async (label) => { - Console.WriteLine(""); + Console.WriteLine(""); using (var tr = db.BeginTransaction(ct)) { await tr.Snapshot - .GetRange(KeyRange.StartsWith(location.Key)) + .GetRange(KeyRange.StartsWith(location.GetPrefix())) .ForEachAsync((kvp) => { Console.WriteLine(" - " + location.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); diff --git a/FoundationDB.Samples/Benchmarks/BenchRunner.cs b/FoundationDB.Samples/Benchmarks/BenchRunner.cs index 1f5e660e3..88b30901e 100644 --- a/FoundationDB.Samples/Benchmarks/BenchRunner.cs +++ b/FoundationDB.Samples/Benchmarks/BenchRunner.cs @@ -40,7 +40,7 @@ public BenchRunner(BenchMode mode, int value = 1) public BenchMode Mode { get; } - public IFdbDynamicSubspace Subspace { get; private set; } + public IDynamicKeySubspace Subspace { get; private set; } public RobustHistogram Histo { get; } diff --git a/FoundationDB.Samples/Benchmarks/LeakTest.cs b/FoundationDB.Samples/Benchmarks/LeakTest.cs index 1cc3cdec9..5cf2863b1 100644 --- a/FoundationDB.Samples/Benchmarks/LeakTest.cs +++ b/FoundationDB.Samples/Benchmarks/LeakTest.cs @@ -29,7 +29,7 @@ public LeakTest(int k, int m, int n, TimeSpan delay) public TimeSpan Delay { get; } - public IFdbDynamicSubspace Subspace { get; private set; } + public IDynamicKeySubspace Subspace { get; private set; } /// /// Setup the initial state of the database @@ -45,8 +45,8 @@ public async Task Init(IFdbDatabase db, CancellationToken ct) // insert all the classes await db.WriteAsync((tr) => { - tr.Set(this.Subspace.Key + FdbKey.MinValue, Slice.FromString("BEGIN")); - tr.Set(this.Subspace.Key + FdbKey.MaxValue, Slice.FromString("END")); + tr.Set(this.Subspace.GetPrefix() + FdbKey.MinValue, Slice.FromString("BEGIN")); + tr.Set(this.Subspace.GetPrefix() + FdbKey.MaxValue, Slice.FromString("END")); }, ct); } diff --git a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs index a0c1ae9f3..b4c1a9904 100644 --- a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs +++ b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs @@ -54,7 +54,7 @@ public MessageQueueRunner(string id, AgentRole role, TimeSpan delayMin, TimeSpan public TimeSpan DelayMax { get; } - public FdbSubspace Subspace { get; private set; } + public KeySubspace Subspace { get; private set; } public FdbWorkerPool WorkerPool { get; private set; } diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index 01fc43301..6b010c967 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -30,7 +30,7 @@ public ClassScheduling() public string[] ClassNames { get; } - public IFdbDynamicSubspace Subspace { get; private set; } + public IDynamicKeySubspace Subspace { get; private set; } protected Slice ClassKey(string c) { diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index e757d32e9..d1b4229b5 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -180,7 +180,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, FdbSubspace.Create(STuple.EncodeKey(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.Create(STuple.EncodeKey(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index c4cee30ca..745b09ebc 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -52,7 +52,7 @@ public async Task Test_Can_Open_Database() Assert.That(cluster, Is.Not.Null); Assert.That(cluster.Path, Is.Null); - using (var db = await cluster.OpenDatabaseAsync("DB", FdbSubspace.Empty, false, this.Cancellation)) + using (var db = await cluster.OpenDatabaseAsync("DB", KeySubspace.Empty, false, this.Cancellation)) { Assert.That(db, Is.Not.Null, "Should return a valid object"); Assert.That(db.Name, Is.EqualTo("DB"), "FdbDatabase.Name should match"); @@ -78,7 +78,7 @@ public async Task Test_Open_Database_With_Cancelled_Token_Should_Fail() using (var cluster = await Fdb.CreateClusterAsync(cts.Token)) { cts.Cancel(); - Assert.Throws(() => cluster.OpenDatabaseAsync("DB", FdbSubspace.Empty, false, cts.Token).GetAwaiter().GetResult()); + Assert.Throws(() => cluster.OpenDatabaseAsync("DB", KeySubspace.Empty, false, cts.Token).GetAwaiter().GetResult()); } } } @@ -93,12 +93,12 @@ public async Task Test_Open_Database_With_Invalid_Name_Should_Fail() using (var cluster = await Fdb.CreateClusterAsync(this.Cancellation)) { - await TestHelpers.AssertThrowsFdbErrorAsync(() => cluster.OpenDatabaseAsync("SomeOtherName", FdbSubspace.Empty, false, this.Cancellation), FdbError.InvalidDatabaseName, "Passing anything other then 'DB' should fail"); + await TestHelpers.AssertThrowsFdbErrorAsync(() => cluster.OpenDatabaseAsync("SomeOtherName", KeySubspace.Empty, false, this.Cancellation), FdbError.InvalidDatabaseName, "Passing anything other then 'DB' should fail"); } await TestHelpers.AssertThrowsFdbErrorAsync(() => Fdb.OpenAsync(null, "SomeOtherName"), FdbError.InvalidDatabaseName, "Passing anything other then 'DB' should fail"); - await TestHelpers.AssertThrowsFdbErrorAsync(() => Fdb.OpenAsync(null, "SomeOtherName", FdbSubspace.Empty), FdbError.InvalidDatabaseName, "Passing anything other then 'DB' should fail"); + await TestHelpers.AssertThrowsFdbErrorAsync(() => Fdb.OpenAsync(null, "SomeOtherName", KeySubspace.Empty), FdbError.InvalidDatabaseName, "Passing anything other then 'DB' should fail"); } [Test] @@ -270,14 +270,14 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", FdbSubspace.Create(STuple.EncodeKey("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.Create(STuple.EncodeKey("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); - Assert.That(db.GlobalSpace.Key.ToString(), Is.EqualTo("<02>test<00>")); + Assert.That(db.GlobalSpace.GetPrefix().ToString(), Is.EqualTo("<02>test<00>")); var subspace = db.Partition.ByKey("hello"); - Assert.That(subspace.Key.ToString(), Is.EqualTo("<02>test<00><02>hello<00>")); + Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>test<00><02>hello<00>")); // keys inside the global space are valid Assert.That(db.IsKeyValid(STuple.EncodeKey("test", 123)), Is.True); @@ -287,14 +287,14 @@ public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() } // using a random binary prefix - using (var db = await Fdb.OpenAsync(null, "DB", new FdbSubspace(Slice.Create(new byte[] { 42, 255, 0, 90 })), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", new KeySubspace(Slice.Create(new byte[] { 42, 255, 0, 90 })), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); - Assert.That(db.GlobalSpace.Key.ToString(), Is.EqualTo("*<00>Z")); + Assert.That(db.GlobalSpace.GetPrefix().ToString(), Is.EqualTo("*<00>Z")); var subspace = db.Partition.ByKey("hello"); - Assert.That(subspace.Key.ToString(), Is.EqualTo("*<00>Z<02>hello<00>")); + Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00>Z<02>hello<00>")); // keys inside the global space are valid Assert.That(db.IsKeyValid(Slice.Unescape("*<00>Z123")), Is.True); @@ -343,11 +343,11 @@ public async Task Test_Database_Instance_Should_Have_Default_Root_Directory() var dl = directory.DirectoryLayer; Assert.That(dl, Is.Not.Null); Assert.That(dl.ContentSubspace, Is.Not.Null); - Assert.That(dl.ContentSubspace.Key, Is.EqualTo(db.GlobalSpace.Key)); + Assert.That(dl.ContentSubspace.GetPrefix(), Is.EqualTo(db.GlobalSpace.GetPrefix())); Assert.That(dl.NodeSubspace, Is.Not.Null); - Assert.That(dl.NodeSubspace.Key, Is.EqualTo(db.GlobalSpace.ConcatKey(Slice.FromByte(254)))); - Assert.That(db.GlobalSpace.Contains(dl.ContentSubspace.Key), Is.True); - Assert.That(db.GlobalSpace.Contains(dl.NodeSubspace.Key), Is.True); + Assert.That(dl.NodeSubspace.GetPrefix(), Is.EqualTo(db.GlobalSpace.ConcatKey(Slice.FromByte(254)))); + Assert.That(db.GlobalSpace.Contains(dl.ContentSubspace.GetPrefix()), Is.True); + Assert.That(db.GlobalSpace.Contains(dl.NodeSubspace.GetPrefix()), Is.True); } } @@ -369,7 +369,7 @@ public async Task Test_Check_Timeout_On_Non_Existing_Database() { tr.Timeout = 250; // ms Log("check ..."); - await tr.GetAsync(db.GlobalSpace.Key); + await tr.GetAsync(db.GlobalSpace.GetPrefix()); Log("Uhoh ...?"); exists = true; } diff --git a/FoundationDB.Tests/FdbTest.cs b/FoundationDB.Tests/FdbTest.cs index bc6ba321c..1de8262b3 100644 --- a/FoundationDB.Tests/FdbTest.cs +++ b/FoundationDB.Tests/FdbTest.cs @@ -130,13 +130,13 @@ protected Task GetCleanDirectory(IFdbDatabase db, params s } [DebuggerStepThrough] - protected Task DumpSubspace(IFdbDatabase db, IFdbSubspace subspace) + protected Task DumpSubspace(IFdbDatabase db, IKeySubspace subspace) { return TestHelpers.DumpSubspace(db, subspace, this.Cancellation); } [DebuggerStepThrough] - protected async Task DeleteSubspace(IFdbDatabase db, IFdbSubspace subspace) + protected async Task DeleteSubspace(IFdbDatabase db, IKeySubspace subspace) { using (var tr = db.BeginTransaction(this.Cancellation)) { diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 16a925943..c1e3dfff9 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -124,9 +124,9 @@ public async Task Test_CreateOrOpen_Simple() var directory = FdbDirectoryLayer.Create(location); Assert.That(directory.ContentSubspace, Is.Not.Null); - Assert.That(directory.ContentSubspace.Key, Is.EqualTo(location.Key)); + Assert.That(directory.ContentSubspace.GetPrefix(), Is.EqualTo(location.GetPrefix())); Assert.That(directory.NodeSubspace, Is.Not.Null); - Assert.That(directory.NodeSubspace.Key, Is.EqualTo(location.Key + Slice.FromByte(254))); + Assert.That(directory.NodeSubspace.GetPrefix(), Is.EqualTo(location.GetPrefix() + Slice.FromByte(254))); // first call should create a new subspace (with a random prefix) FdbDirectorySubspace foo; @@ -160,7 +160,7 @@ public async Task Test_CreateOrOpen_Simple() Assert.That(foo2.Path, Is.EqualTo(new[] { "Foo" })); Assert.That(foo2.Layer, Is.EqualTo(Slice.Empty)); Assert.That(foo2.DirectoryLayer, Is.SameAs(directory)); - Assert.That(foo2.Key, Is.EqualTo(foo.Key), "Second call to CreateOrOpen should return the same subspace"); + Assert.That(foo2.GetPrefix(), Is.EqualTo(foo.GetPrefix()), "Second call to CreateOrOpen should return the same subspace"); #if ENABLE_LOGGING foreach (var log in list) @@ -193,7 +193,7 @@ public async Task Test_CreateOrOpen_With_Layer() Assert.That(directory.ContentSubspace, Is.Not.Null); Assert.That(directory.ContentSubspace, Is.EqualTo(location)); Assert.That(directory.NodeSubspace, Is.Not.Null); - Assert.That(directory.NodeSubspace.Key, Is.EqualTo(location.Key + Slice.FromByte(254))); + Assert.That(directory.NodeSubspace.GetPrefix(), Is.EqualTo(location.GetPrefix() + Slice.FromByte(254))); // first call should create a new subspace (with a random prefix) var foo = await directory.CreateOrOpenAsync(logged, new[] { "Foo" }, Slice.FromString("AcmeLayer"), this.Cancellation); @@ -214,7 +214,7 @@ public async Task Test_CreateOrOpen_With_Layer() Assert.That(foo2.Path, Is.EqualTo(new[] { "Foo" })); Assert.That(foo2.Layer.ToUnicode(), Is.EqualTo("AcmeLayer")); Assert.That(foo2.DirectoryLayer, Is.SameAs(directory)); - Assert.That(foo2.Key, Is.EqualTo(foo.Key), "Second call to CreateOrOpen should return the same subspace"); + Assert.That(foo2.GetPrefix(), Is.EqualTo(foo.GetPrefix()), "Second call to CreateOrOpen should return the same subspace"); // opening it with wrong layer id should fail Assert.Throws(async () => await directory.OpenAsync(logged, new[] { "Foo" }, Slice.FromString("OtherLayer"), this.Cancellation), "Opening with invalid layer id should fail"); @@ -423,7 +423,7 @@ public async Task Test_Move_Folder() Assert.That(renamed, Is.Not.Null); Assert.That(renamed.FullName, Is.EqualTo("Bar")); Assert.That(renamed.Path, Is.EqualTo(new[] { "Bar" })); - Assert.That(renamed.Key, Is.EqualTo(original.Key)); + Assert.That(renamed.GetPrefix(), Is.EqualTo(original.GetPrefix())); // opening the old path should fail Assert.Throws(async () => await directory.OpenAsync(logged, "Foo", this.Cancellation)); @@ -433,7 +433,7 @@ public async Task Test_Move_Folder() Assert.That(folder, Is.Not.Null); Assert.That(folder.FullName, Is.EqualTo(renamed.FullName)); Assert.That(folder.Path, Is.EqualTo(renamed.Path)); - Assert.That(folder.Key, Is.EqualTo(renamed.Key)); + Assert.That(folder.GetPrefix(), Is.EqualTo(renamed.GetPrefix())); // moving the folder under itself should fail Assert.Throws(async () => await folder.MoveToAsync(logged, new[] { "Bar", "Baz" }, this.Cancellation)); @@ -538,7 +538,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() Assert.That(folder2.Layer.ToUnicode(), Is.EqualTo("bar")); Assert.That(folder2.FullName, Is.EqualTo("Test")); Assert.That(folder2.Path, Is.EqualTo(STuple.Create("Test"))); - Assert.That(folder2.Key, Is.EqualTo(folder.Key)); + Assert.That(folder2.GetPrefix(), Is.EqualTo(folder.GetPrefix())); // opening the directory with the new layer should succeed var folder3 = await directory.OpenAsync(logged, "Test", layer: Slice.FromString("bar"), cancellationToken: this.Cancellation); @@ -569,37 +569,37 @@ public async Task Test_Directory_Partitions() var partition = await directory.CreateAsync(db, "Foo", Slice.FromAscii("partition"), this.Cancellation); // we can't get the partition key directory (because it's a root directory) so we need to cheat a little bit - var partitionKey = FdbSubspace.Copy(partition).Key; + var partitionKey = KeySubspace.Copy(partition).GetPrefix(); Console.WriteLine(partition); Assert.That(partition, Is.InstanceOf()); Assert.That(partition.Layer, Is.EqualTo(Slice.FromAscii("partition"))); Assert.That(partition.FullName, Is.EqualTo("Foo")); Assert.That(partition.Path, Is.EqualTo(new[] { "Foo" }), "Partition's path should be absolute"); Assert.That(partition.DirectoryLayer, Is.Not.SameAs(directory), "Partitions should have their own DL"); - Assert.That(partition.DirectoryLayer.ContentSubspace.Key, Is.EqualTo(partitionKey), "Partition's content should be under the partition's prefix"); - Assert.That(partition.DirectoryLayer.NodeSubspace.Key, Is.EqualTo(partitionKey + FdbKey.Directory), "Partition's nodes should be under the partition's prefix"); + Assert.That(partition.DirectoryLayer.ContentSubspace.GetPrefix(), Is.EqualTo(partitionKey), "Partition's content should be under the partition's prefix"); + Assert.That(partition.DirectoryLayer.NodeSubspace.GetPrefix(), Is.EqualTo(partitionKey + FdbKey.Directory), "Partition's nodes should be under the partition's prefix"); var bar = await partition.CreateAsync(db, "Bar", this.Cancellation); Console.WriteLine(bar); Assert.That(bar, Is.InstanceOf()); Assert.That(bar.Path, Is.EqualTo(new [] { "Foo", "Bar" }), "Path of directories under a partition should be absolute"); - Assert.That(bar.Key, Is.Not.EqualTo(partitionKey), "{0} should be located under {1}", bar, partition); - Assert.That(bar.Key.StartsWith(partitionKey), Is.True, "{0} should be located under {1}", bar, partition); + Assert.That(bar.GetPrefix(), Is.Not.EqualTo(partitionKey), "{0} should be located under {1}", bar, partition); + Assert.That(bar.GetPrefix().StartsWith(partitionKey), Is.True, "{0} should be located under {1}", bar, partition); var baz = await partition.CreateAsync(db, "Baz", this.Cancellation); Console.WriteLine(baz); Assert.That(baz, Is.InstanceOf()); Assert.That(baz.FullName, Is.EqualTo("Foo/Baz")); Assert.That(baz.Path, Is.EqualTo(new[] { "Foo", "Baz" }), "Path of directories under a partition should be absolute"); - Assert.That(baz.Key, Is.Not.EqualTo(partitionKey), "{0} should be located under {1}", baz, partition); - Assert.That(baz.Key.StartsWith(partitionKey), Is.True, "{0} should be located under {1}", baz, partition); + Assert.That(baz.GetPrefix(), Is.Not.EqualTo(partitionKey), "{0} should be located under {1}", baz, partition); + Assert.That(baz.GetPrefix().StartsWith(partitionKey), Is.True, "{0} should be located under {1}", baz, partition); // Rename 'Bar' to 'BarBar' var bar2 = await bar.MoveToAsync(db, new[] { "Foo", "BarBar" }, this.Cancellation); Console.WriteLine(bar2); Assert.That(bar2, Is.InstanceOf()); Assert.That(bar2, Is.Not.SameAs(bar)); - Assert.That(bar2.Key, Is.EqualTo(bar.Key)); + Assert.That(bar2.GetPrefix(), Is.EqualTo(bar.GetPrefix())); Assert.That(bar2.FullName, Is.EqualTo("Foo/BarBar")); Assert.That(bar2.Path, Is.EqualTo(new[] { "Foo", "BarBar" })); Assert.That(bar2.DirectoryLayer, Is.SameAs(bar.DirectoryLayer)); @@ -679,14 +679,14 @@ public async Task Test_Directory_Cannot_Move_To_A_Sub_Partition() var foo2 = await directory.MoveAsync(db, new[] { "Outer", "Foo" }, new[] { "Outer", "SubFolder", "Foo" }, this.Cancellation); Assert.That(foo2.FullName, Is.EqualTo("Outer/SubFolder/Foo")); Assert.That(foo2.Path, Is.EqualTo(new[] { "Outer", "SubFolder", "Foo" })); - Assert.That(foo2.Key, Is.EqualTo(foo.Key)); + Assert.That(foo2.GetPrefix(), Is.EqualTo(foo.GetPrefix())); // Moving 'Bar' inside the Inner partition itself should work await directory.CreateAsync(db, new[] { "Outer", "Inner", "SubFolder" }, this.Cancellation); // parent of destination folder must already exist when moving... var bar2 = await directory.MoveAsync(db, new[] { "Outer", "Inner", "Bar" }, new[] { "Outer", "Inner", "SubFolder", "Bar" }, this.Cancellation); Assert.That(bar2.FullName, Is.EqualTo("Outer/Inner/SubFolder/Bar")); Assert.That(bar2.Path, Is.EqualTo(new[] { "Outer", "Inner", "SubFolder", "Bar" })); - Assert.That(bar2.Key, Is.EqualTo(bar.Key)); + Assert.That(bar2.GetPrefix(), Is.EqualTo(bar.GetPrefix())); } } @@ -729,7 +729,7 @@ public async Task Test_Renaming_Partition_Uses_Parent_DirectoryLayer() // should have kept the same prefix //note: we need to cheat to get the key of the partition - Assert.That(FdbSubspace.Copy(bar).Key, Is.EqualTo(FdbSubspace.Copy(foo).Key)); + Assert.That(KeySubspace.Copy(bar).GetPrefix(), Is.EqualTo(KeySubspace.Copy(foo).GetPrefix())); // verify list again folders = await directory.ListAsync(tr); @@ -849,7 +849,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K var partition = await directory.CreateAsync(db, "Foo", Slice.FromAscii("partition"), this.Cancellation); //note: if we want a testable key INSIDE the partition, we have to get it from a sub-directory var subdir = await partition.CreateOrOpenAsync(db, "Bar", this.Cancellation); - var barKey = subdir.Key; + var barKey = subdir.GetPrefix(); // the constraint will always be the same for all the checks Action shouldFail = (del) => @@ -864,7 +864,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K // === PASS === // these methods are allowed to succeed on directory partitions, because we need them for the rest to work - shouldPass(() => { var _ = FdbSubspace.Copy(partition).Key; }); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! + shouldPass(() => { var _ = KeySubspace.Copy(partition).GetPrefix(); }); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! shouldPass(() => partition.ToString()); // EXCEPTION: this should never fail! shouldPass(() => partition.DumpKey(barKey)); // EXCEPTION: this should always work, because this can be used for debugging and logging... shouldPass(() => partition.BoundCheck(barKey, true)); // EXCEPTION: needs to work because it is used by GetRange() and GetKey() @@ -872,7 +872,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K // === FAIL ==== // Key - shouldFail(() => { var _ = partition.Key; }); + shouldFail(() => { var _ = partition.GetPrefix(); }); // Contains shouldFail(() => partition.Contains(barKey)); @@ -891,11 +891,11 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K // Keys shouldFail(() => partition.ConcatKey(Slice.FromString("hello"))); - shouldFail(() => partition.ConcatKey(location.Key)); + shouldFail(() => partition.ConcatKey(location.GetPrefix())); shouldFail(() => partition.ConcatKeys(new[] { Slice.FromString("hello"), Slice.FromString("world"), Slice.FromString("!") })); shouldFail(() => { var _ = partition[Slice.FromString("hello")]; }); - shouldFail(() => { var _ = partition[location.Key]; }); + shouldFail(() => { var _ = partition[location.GetPrefix()]; }); shouldFail(() => partition.ToRange()); shouldFail(() => partition.ToRange(Slice.FromString("hello"))); @@ -973,10 +973,10 @@ await Task.WhenAll( ); var first = await directory.CreateAsync(tr1, new[] { "First" }, Slice.Nil); - tr1.Set(first.Key, Slice.FromString("This belongs to the first directory")); + tr1.Set(first.GetPrefix(), Slice.FromString("This belongs to the first directory")); var second = await directory.CreateAsync(tr2, new[] { "Second" }, Slice.Nil); - tr2.Set(second.Key, Slice.FromString("This belongs to the second directory")); + tr2.Set(second.GetPrefix(), Slice.FromString("This belongs to the second directory")); Console.WriteLine("Committing T1..."); await tr1.CommitAsync(); @@ -1032,10 +1032,10 @@ await Task.WhenAll( ); var first = await directory.RegisterAsync(tr1, new[] { "First" }, Slice.Nil, location.Keys.Encode("abc")); - tr1.Set(first.Key, Slice.FromString("This belongs to the first directory")); + tr1.Set(first.GetPrefix(), Slice.FromString("This belongs to the first directory")); var second = await directory.RegisterAsync(tr2, new[] { "Second" }, Slice.Nil, location.Keys.Encode("def")); - tr2.Set(second.Key, Slice.FromString("This belongs to the second directory")); + tr2.Set(second.GetPrefix(), Slice.FromString("This belongs to the second directory")); Console.WriteLine("Committing T1..."); await tr1.CommitAsync(); diff --git a/FoundationDB.Tests/Layers/QueuesFacts.cs b/FoundationDB.Tests/Layers/QueuesFacts.cs index ab39ff736..a620fba64 100644 --- a/FoundationDB.Tests/Layers/QueuesFacts.cs +++ b/FoundationDB.Tests/Layers/QueuesFacts.cs @@ -150,7 +150,7 @@ public async Task Test_Single_Client() } - private static async Task RunMultiClientTest(IFdbDatabase db, FdbSubspace location, bool highContention, string desc, int K, int NUM, CancellationToken ct) + private static async Task RunMultiClientTest(IFdbDatabase db, KeySubspace location, bool highContention, string desc, int K, int NUM, CancellationToken ct) { Console.WriteLine("Starting {0} test with {1} threads and {2} iterations", desc, K, NUM); diff --git a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs b/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs index 5fdf3a057..65d550649 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs +++ b/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs @@ -62,7 +62,7 @@ await db.WriteAsync((tr) => tr.Set(location.Keys.Encode("Narf"), Slice.FromString("Zort")); }, this.Cancellation); - var range = db.Query().RangeStartsWith(location.Key); + var range = db.Query().RangeStartsWith(location.GetPrefix()); Assert.That(range, Is.InstanceOf>>()); Assert.That(range.Expression, Is.InstanceOf()); Console.WriteLine(range.Expression.DebugView); diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index e69c99ae5..f52bdfbd4 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -40,8 +40,8 @@ namespace FoundationDB.Linq.Expressions.Tests public class FdbQueryExpressionFacts { - private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", FdbSubspace.Create(STuple.EncodeKey("Foos", 1))); - private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", FdbSubspace.Create(STuple.EncodeKey("Foos", 2))); + private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(STuple.EncodeKey("Foos", 1))); + private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(STuple.EncodeKey("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 5dd064b14..8bf627551 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -296,8 +296,8 @@ await db.WriteAsync((tr) => tr.Set(a.Keys.Encode(i), Slice.FromInt32(i)); } // add guard keys - tr.Set(location.Key, Slice.FromInt32(-1)); - tr.Set(location.Key + (byte)255, Slice.FromInt32(-1)); + tr.Set(location.GetPrefix(), Slice.FromInt32(-1)); + tr.Set(location.GetPrefix() + (byte)255, Slice.FromInt32(-1)); }, this.Cancellation); // Take(5) should return the first 5 items diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 8bd26db3a..84b36f0c6 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -40,25 +40,25 @@ public class SubspaceFacts [Test] public void Test_Empty_Subspace_Is_Empty() { - var subspace = FdbSubspace.Empty; + var subspace = KeySubspace.Empty; Assert.That(subspace, Is.Not.Null, "FdbSubspace.Empty should not return null"); - Assert.That(FdbSubspace.Empty, Is.SameAs(subspace), "FdbSubspace.Empty is a singleton"); + Assert.That(KeySubspace.Empty, Is.SameAs(subspace), "FdbSubspace.Empty is a singleton"); - Assert.That(subspace.Key.Count, Is.EqualTo(0), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); - Assert.That(subspace.Key.HasValue, Is.True, "FdbSubspace.Empty.Key should be equal to Slice.Empty"); + Assert.That(subspace.GetPrefix().Count, Is.EqualTo(0), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); + Assert.That(subspace.GetPrefix().HasValue, Is.True, "FdbSubspace.Empty.Key should be equal to Slice.Empty"); - Assert.That(FdbSubspace.Copy(subspace), Is.Not.SameAs(subspace)); + Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); } [Test] [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = FdbSubspace.CreateDynamic(Slice.Create(new byte[] { 42, 255, 0, 127 })); + var subspace = KeySubspace.CreateDynamic(Slice.Create(new byte[] { 42, 255, 0, 127 })); - Assert.That(subspace.Key.ToString(), Is.EqualTo("*<00><7F>")); - Assert.That(FdbSubspace.Copy(subspace), Is.Not.SameAs(subspace)); - Assert.That(FdbSubspace.Copy(subspace).Key, Is.EqualTo(subspace.Key)); + Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); + Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); + Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the binary prefix directly Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("*<00><7F><04><03><02><01>")); @@ -92,12 +92,12 @@ public void Test_Subspace_With_Binary_Prefix() [Test] public void Test_Subspace_Copy_Does_Not_Share_Key_Buffer() { - var original = FdbSubspace.Create(Slice.FromString("Hello")); - var copy = FdbSubspace.Copy(original); + var original = KeySubspace.Create(Slice.FromString("Hello")); + var copy = KeySubspace.Copy(original); Assert.That(copy, Is.Not.Null); Assert.That(copy, Is.Not.SameAs(original), "Copy should be a new instance"); - Assert.That(copy.Key, Is.EqualTo(original.Key), "Key should be equal"); - Assert.That(copy.Key.Array, Is.Not.SameAs(original.Key.Array), "Key should be a copy of the original"); + Assert.That(copy.GetPrefix(), Is.EqualTo(original.GetPrefix()), "Key should be equal"); + Assert.That(copy.GetPrefix().Array, Is.Not.SameAs(original.GetPrefix().Array), "Key should be a copy of the original"); Assert.That(copy, Is.EqualTo(original), "Copy and original should be considered equal"); Assert.That(copy.ToString(), Is.EqualTo(original.ToString()), "Copy and original should have the same string representation"); @@ -107,8 +107,8 @@ public void Test_Subspace_Copy_Does_Not_Share_Key_Buffer() [Test] public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() { - Assert.That(() => new FdbSubspace(Slice.Nil), Throws.ArgumentException); - Assert.That(() => FdbSubspace.Create(Slice.Nil), Throws.ArgumentException); + Assert.That(() => new KeySubspace(Slice.Nil), Throws.ArgumentException); + Assert.That(() => KeySubspace.Create(Slice.Nil), Throws.ArgumentException); //FIXME: typed subspaces refactoring ! //Assert.That(() => FdbSubspace.Empty.Partition[Slice.Nil], Throws.ArgumentException); //Assert.That(() => FdbSubspace.Create(FdbKey.Directory).Partition[Slice.Nil], Throws.ArgumentException); @@ -118,11 +118,11 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = FdbSubspace.CreateDynamic(STuple.Create("hello")); + var subspace = KeySubspace.CreateDynamic(STuple.Create("hello")); - Assert.That(subspace.Key.ToString(), Is.EqualTo("<02>hello<00>")); - Assert.That(FdbSubspace.Copy(subspace), Is.Not.SameAs(subspace)); - Assert.That(FdbSubspace.Copy(subspace).Key, Is.EqualTo(subspace.Key)); + Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); + Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); + Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the tuple prefix directly Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("<02>hello<00><04><03><02><01>")); @@ -157,13 +157,13 @@ public void Test_Subspace_With_Tuple_Prefix() public void Test_Subspace_Partitioning_With_Binary_Suffix() { // start from a parent subspace - var parent = FdbSubspace.Empty.Using(TypeSystem.Tuples); - Assert.That(parent.Key.ToString(), Is.EqualTo("")); + var parent = KeySubspace.Empty.Using(TypeSystem.Tuples); + Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple var child = parent.Partition[FdbKey.Directory]; Assert.That(child, Is.Not.Null); - Assert.That(child.Key.ToString(), Is.EqualTo("")); + Assert.That(child.GetPrefix().ToString(), Is.EqualTo("")); // create a key from this child subspace var key = child.ConcatKey(Slice.FromFixed32(0x01020304)); @@ -172,13 +172,13 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() // create another child var grandChild = child.Partition[Slice.FromAscii("hello")]; Assert.That(grandChild, Is.Not.Null); - Assert.That(grandChild.Key.ToString(), Is.EqualTo("hello")); + Assert.That(grandChild.GetPrefix().ToString(), Is.EqualTo("hello")); key = grandChild.ConcatKey(Slice.FromFixed32(0x01020304)); Assert.That(key.ToString(), Is.EqualTo("hello<04><03><02><01>")); // cornercase - Assert.That(child.Partition[Slice.Empty].Key, Is.EqualTo(child.Key)); + Assert.That(child.Partition[Slice.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); } [Test] @@ -186,13 +186,13 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() public void Test_Subspace_Partitioning_With_Tuple_Suffix() { // start from a parent subspace - var parent = FdbSubspace.CreateDynamic(Slice.FromByte(254), TypeSystem.Tuples); - Assert.That(parent.Key.ToString(), Is.EqualTo("")); + var parent = KeySubspace.CreateDynamic(Slice.FromByte(254), TypeSystem.Tuples); + Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple var child = parent.Partition.ByKey("hca"); Assert.That(child, Is.Not.Null); - Assert.That(child.Key.ToString(), Is.EqualTo("<02>hca<00>")); + Assert.That(child.GetPrefix().ToString(), Is.EqualTo("<02>hca<00>")); // create a tuple from this child subspace var tuple = child.Keys.Append(123); @@ -208,7 +208,7 @@ public void Test_Subspace_Partitioning_With_Tuple_Suffix() Assert.That(t2.ToSlice(), Is.EqualTo(t1.ToSlice())); // cornercase - Assert.That(child.Partition[STuple.Empty].Key, Is.EqualTo(child.Key)); + Assert.That(child.Partition[STuple.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); } diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index fd4844a32..75fa63bf8 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -52,7 +52,7 @@ internal static class TestHelpers /// Connect to the local test database public static Task OpenTestDatabaseAsync(CancellationToken ct) { - var subspace = new FdbSubspace(TestGlobalPrefix.Memoize()); + var subspace = new KeySubspace(TestGlobalPrefix.Memoize()); return Fdb.OpenAsync(TestClusterFile, TestDbName, subspace, false, ct); } @@ -78,14 +78,14 @@ public static async Task GetCleanDirectory([NotNull] IFdbD // create new var subspace = await db.Directory.CreateAsync(path, ct); Assert.That(subspace, Is.Not.Null); - Assert.That(db.GlobalSpace.Contains(subspace.Key), Is.True); + Assert.That(db.GlobalSpace.Contains(subspace.GetPrefix()), Is.True); return subspace; } - public static async Task DumpSubspace([NotNull] IFdbDatabase db, [NotNull] IFdbSubspace subspace, CancellationToken ct) + public static async Task DumpSubspace([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace, CancellationToken ct) { Assert.That(db, Is.Not.Null); - Assert.That(db.GlobalSpace.Contains(subspace.Key), Is.True, "Using a location outside of the test database partition!!! This is probably a bug in the test..."); + Assert.That(db.GlobalSpace.Contains(subspace.GetPrefix()), Is.True, "Using a location outside of the test database partition!!! This is probably a bug in the test..."); // do not log db = db.WithoutLogging(); @@ -96,14 +96,14 @@ public static async Task DumpSubspace([NotNull] IFdbDatabase db, [NotNull] IFdbS } } - public static async Task DumpSubspace([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IFdbSubspace subspace) + public static async Task DumpSubspace([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IKeySubspace subspace) { Assert.That(tr, Is.Not.Null); Console.WriteLine("Dumping content of subspace " + subspace.ToString() + " :"); int count = 0; await tr - .GetRange(KeyRange.StartsWith(subspace.Key)) + .GetRange(KeyRange.StartsWith(subspace.GetPrefix())) .ForEachAsync((kvp) => { var key = subspace.ExtractKey(kvp.Key, boundCheck: true); diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 94f585d5d..7ccf4f345 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -394,8 +394,8 @@ public async Task Test_Can_Resolve_Key_Selector() var location = db.Partition.ByKey("keys"); await db.ClearRangeAsync(location, this.Cancellation); - var minKey = location.Key + FdbKey.MinValue; - var maxKey = location.Key + FdbKey.MaxValue; + var minKey = location.GetPrefix() + FdbKey.MinValue; + var maxKey = location.GetPrefix() + FdbKey.MaxValue; #region Insert a bunch of keys ... using (var tr = db.BeginTransaction(this.Cancellation)) @@ -483,7 +483,7 @@ public async Task Test_Can_Resolve_Key_Selector_Outside_Boundaries() Slice key; // note: we can't have any prefix on the keys, so open the test database in read-only mode - using (var db = await Fdb.OpenAsync(TestHelpers.TestClusterFile, TestHelpers.TestDbName, FdbSubspace.Empty, readOnly: true, cancellationToken: this.Cancellation)) + using (var db = await Fdb.OpenAsync(TestHelpers.TestClusterFile, TestHelpers.TestDbName, KeySubspace.Empty, readOnly: true, cancellationToken: this.Cancellation)) { using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) { @@ -586,8 +586,8 @@ public async Task Test_Get_Multiple_Keys() var location = db.Partition.ByKey("keys"); await db.ClearRangeAsync(location, this.Cancellation); - var minKey = location.Key + FdbKey.MinValue; - var maxKey = location.Key + FdbKey.MaxValue; + var minKey = location.GetPrefix() + FdbKey.MinValue; + var maxKey = location.GetPrefix() + FdbKey.MaxValue; #region Insert a bunch of keys ... using (var tr = db.BeginTransaction(this.Cancellation)) From ed9e7e65da69f37ef95900ab8d725fe2f3c8973d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 13:21:47 +0200 Subject: [PATCH 065/153] Shorten all "cancellationToken" parameters into "ct" --- FdbShell/Commands/BasicCommands.cs | 20 +- FoundationDB.Client/Async/AsyncBuffer.cs | 8 +- FoundationDB.Client/Async/AsyncHelpers.cs | 28 +- .../Async/AsyncProducerConsumerQueue.cs | 2 +- FoundationDB.Client/Async/AsyncPump.cs | 10 +- FoundationDB.Client/Async/AsyncTaskBuffer.cs | 8 +- FoundationDB.Client/Async/AsyncTransform.cs | 12 +- .../Async/AsyncTransformQueue.cs | 24 +- FoundationDB.Client/Async/IAsyncEnumerator.cs | 4 +- FoundationDB.Client/Async/IAsyncPump.cs | 4 +- FoundationDB.Client/Async/IAsyncSource.cs | 4 +- FoundationDB.Client/Async/IAsyncTarget.cs | 4 +- FoundationDB.Client/Async/TaskHelpers.cs | 16 +- .../Core/IFdbClusterHandler.cs | 2 +- .../Core/IFdbTransactionHandler.cs | 38 +- FoundationDB.Client/Fdb.Bulk.cs | 312 ++++++------ FoundationDB.Client/Fdb.System.cs | 86 ++-- FoundationDB.Client/Fdb.cs | 60 +-- FoundationDB.Client/FdbCluster.cs | 18 +- FoundationDB.Client/FdbDatabase.cs | 68 +-- FoundationDB.Client/FdbDatabaseExtensions.cs | 110 ++-- FoundationDB.Client/FdbExceptIterator.cs | 6 +- FoundationDB.Client/FdbIntersectIterator.cs | 6 +- FoundationDB.Client/FdbMergeSortIterator.cs | 4 +- FoundationDB.Client/FdbOperationContext.cs | 60 +-- FoundationDB.Client/FdbQueryMergeIterator.cs | 14 +- .../FdbRangeQuery.PagingIterator.cs | 11 +- .../FdbRangeQuery.ResultIterator.cs | 10 +- .../FdbTransaction.Snapshot.cs | 12 +- FoundationDB.Client/FdbTransaction.cs | 26 +- .../FdbTransactionExtensions.cs | 30 +- .../Filters/FdbDatabaseFilter.cs | 54 +- .../Filters/FdbTransactionFilter.cs | 4 +- .../Filters/Logging/FdbLoggedDatabase.cs | 5 +- .../Filters/Logging/FdbLoggedTransaction.cs | 4 +- .../Filters/PrefixRewriterTransaction.cs | 4 +- FoundationDB.Client/IFdbCluster.cs | 4 +- FoundationDB.Client/IFdbDatabase.cs | 4 +- FoundationDB.Client/IFdbReadOnlyRetryable.cs | 12 +- FoundationDB.Client/IFdbRetryable.cs | 24 +- FoundationDB.Client/IFdbTransaction.cs | 4 +- .../Directories/FdbDirectoryExtensions.cs | 474 +++++++++--------- .../Linq/FdbAsyncEnumerable.EmptySequence.cs | 10 +- .../FdbAsyncEnumerable.EnumerableIterator.cs | 4 +- .../FdbAsyncEnumerable.OrderedSequence.cs | 4 +- .../Linq/FdbAsyncEnumerable.cs | 280 +++++------ .../Linq/Iterators/FdbAsyncIterator.cs | 4 +- .../Iterators/FdbDistinctAsyncIterator.cs | 10 +- .../Linq/Iterators/FdbObserverIterator.cs | 12 +- .../FdbParallelSelectAsyncIterator.cs | 4 +- .../Iterators/FdbSelectManyAsyncIterator.cs | 24 +- .../Linq/Iterators/FdbWhereAsyncIterator.cs | 12 +- .../Iterators/FdbWhereSelectAsyncIterator.cs | 14 +- FoundationDB.Client/Native/FdbFuture.cs | 28 +- FoundationDB.Client/Native/FdbFutureArray.cs | 14 +- FoundationDB.Client/Native/FdbFutureSingle.cs | 12 +- .../Native/FdbNativeCluster.cs | 10 +- .../Native/FdbNativeTransaction.cs | 40 +- .../Subspaces/Fdb.Directory.cs | 30 +- .../Subspaces/FdbDatabasePartition.cs | 144 +++--- .../Subspaces/KeySubspaceExtensions.cs | 4 +- FoundationDB.Client/Utils/Slice.cs | 98 ++-- FoundationDB.Client/Utils/SliceListStream.cs | 22 +- FoundationDB.Client/Utils/SliceStream.cs | 26 +- .../Collections/FdbMap`2.cs | 138 ++--- .../Collections/FdbQueue`1.cs | 64 +-- .../Counters/FdbHighContentionCounter.cs | 40 +- .../Documents/FdbHashSetCollection.cs | 5 +- .../Messaging/FdbWorkerPool.cs | 4 +- .../Providers/FdbAsyncQuery.cs | 4 +- .../Benchmarks/BenchRunner.cs | 2 +- FoundationDB.Samples/Benchmarks/LeakTest.cs | 2 +- .../MessageQueue/MessageQueueRunner.cs | 2 +- .../Tutorials/ClassScheduling.cs | 2 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 8 +- FoundationDB.Tests/TransactionFacts.cs | 2 +- 76 files changed, 1345 insertions(+), 1344 deletions(-) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 262f2ae99..50b9fbe8b 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -29,7 +29,7 @@ public static async Task TryOpenCurrentDirectoryAsync(string[] pa { if (path != null && path.Length > 0) { - return await db.Directory.TryOpenAsync(path, cancellationToken: ct); + return await db.Directory.TryOpenAsync(path, ct: ct); } else { @@ -104,18 +104,18 @@ public static async Task CreateDirectory(string[] path, ITuple extras, IFdbDatab log.WriteLine("# Creating directory {0} with layer '{1}'", String.Join("/", path), layer); - var folder = await db.Directory.TryOpenAsync(path, cancellationToken: ct); + var folder = await db.Directory.TryOpenAsync(path, ct: ct); if (folder != null) { log.WriteLine("- Directory {0} already exists!", string.Join("/", path)); return; } - folder = await db.Directory.TryCreateAsync(path, Slice.FromString(layer), cancellationToken: ct); + folder = await db.Directory.TryCreateAsync(path, Slice.FromString(layer), ct: ct); log.WriteLine("- Created under {0} [{1}]", FdbKey.Dump(folder.GetPrefix()), folder.GetPrefix().ToHexaString(' ')); // look if there is already stuff under there - var stuff = await db.ReadAsync((tr) => tr.GetRange(folder.Keys.ToRange()).FirstOrDefaultAsync(), cancellationToken: ct); + var stuff = await db.ReadAsync((tr) => tr.GetRange(folder.Keys.ToRange()).FirstOrDefaultAsync(), ct: ct); if (stuff.Key.IsPresent) { log.WriteLine("CAUTION: There is already some data under {0} !"); @@ -130,7 +130,7 @@ public static async Task RemoveDirectory(string[] path, ITuple extras, IFdbDatab string layer = extras.Count > 0 ? extras.Get(0) : null; - var folder = await db.Directory.TryOpenAsync(path, cancellationToken: ct); + var folder = await db.Directory.TryOpenAsync(path, ct: ct); if (folder == null) { log.WriteLine("# Directory {0} does not exist", string.Join("/", path)); @@ -155,14 +155,14 @@ public static async Task RemoveDirectory(string[] path, ITuple extras, IFdbDatab /// Move/Rename a directory public static async Task MoveDirectory(string[] srcPath, string[] dstPath, ITuple extras, IFdbDatabase db, TextWriter log, CancellationToken ct) { - var folder = await db.Directory.TryOpenAsync(srcPath, cancellationToken: ct); + var folder = await db.Directory.TryOpenAsync(srcPath, ct: ct); if (folder == null) { log.WriteLine("# Source directory {0} does not exist!", string.Join("/", srcPath)); return; } - folder = await db.Directory.TryOpenAsync(dstPath, cancellationToken: ct); + folder = await db.Directory.TryOpenAsync(dstPath, ct: ct); if (folder != null) { log.WriteLine("# Destination directory {0} already exists!", string.Join("/", dstPath)); @@ -239,7 +239,7 @@ public static async Task Show(string[] path, ITuple extras, bool reverse, IFdbDa } // look if there is something under there - var folder = await db.Directory.TryOpenAsync(path, cancellationToken: ct); + var folder = await db.Directory.TryOpenAsync(path, ct: ct); if (folder != null) { log.WriteLine("# Content of {0} [{1}]", FdbKey.Dump(folder.GetPrefix()), folder.GetPrefix().ToHexaString(' ')); @@ -249,7 +249,7 @@ public static async Task Show(string[] path, ITuple extras, bool reverse, IFdbDa return reverse ? query.Reverse().Take(count) : query.Take(count + 1); - }, cancellationToken: ct); + }, ct: ct); if (keys.Count > 0) { if (reverse) keys.Reverse(); @@ -277,7 +277,7 @@ public static async Task Tree(string[] path, ITuple extras, IFdbDatabase db, Tex log.WriteLine("# Tree of {0}:", String.Join("/", path)); FdbDirectorySubspace root = null; - if (path.Length > 0) root = await db.Directory.TryOpenAsync(path, cancellationToken: ct); + if (path.Length > 0) root = await db.Directory.TryOpenAsync(path, ct: ct); await TreeDirectoryWalk(root, new List(), db, log, ct); diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Async/AsyncBuffer.cs index 107e2b83e..f1c3eb128 100644 --- a/FoundationDB.Client/Async/AsyncBuffer.cs +++ b/FoundationDB.Client/Async/AsyncBuffer.cs @@ -68,9 +68,9 @@ public AsyncBuffer([NotNull] Func transform, int capacity) #region IFdbAsyncTarget... - public override Task OnNextAsync(T value, CancellationToken cancellationToken) + public override Task OnNextAsync(T value, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); LogProducer("Received new value"); @@ -86,11 +86,11 @@ public override Task OnNextAsync(T value, CancellationToken cancellationToken) } // we are blocked, we will need to wait ! - wait = MarkProducerAsBlocked_NeedsLocking(cancellationToken); + wait = MarkProducerAsBlocked_NeedsLocking(ct); } // slow path - return WaitForNextFreeSlotThenEnqueueAsync(value, wait, cancellationToken); + return WaitForNextFreeSlotThenEnqueueAsync(value, wait, ct); } public override void OnCompleted() diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Async/AsyncHelpers.cs index 1008deb1b..418986bed 100644 --- a/FoundationDB.Client/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Async/AsyncHelpers.cs @@ -133,9 +133,9 @@ Action onError m_onError = onError; } - public Task OnNextAsync(T value, CancellationToken cancellationToken) + public Task OnNextAsync(T value, CancellationToken ct) { - return m_onNextAsync(value, cancellationToken); + return m_onNextAsync(value, ct); } public void OnCompleted() @@ -184,9 +184,9 @@ Action onError m_onError = onError; } - public Task OnNextAsync(T value, CancellationToken cancellationToken) + public Task OnNextAsync(T value, CancellationToken ct) { - return TaskHelpers.Inline(m_onNext, value, cancellationToken, cancellationToken); + return TaskHelpers.Inline(m_onNext, value, ct, ct); } public void OnCompleted() @@ -220,20 +220,20 @@ public void OnError(ExceptionDispatchInfo error) #region Pumps... - public static async Task PumpToAsync(this IAsyncSource source, IAsyncTarget target, CancellationToken cancellationToken) + public static async Task PumpToAsync(this IAsyncSource source, IAsyncTarget target, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); using (var pump = new AsyncPump(source, target)) { - await pump.PumpAsync(stopOnFirstError: true, cancellationToken: cancellationToken).ConfigureAwait(false); + await pump.PumpAsync(stopOnFirstError: true, ct: ct).ConfigureAwait(false); } } /// Pump the content of a source into a list - public static async Task> PumpToListAsync(this IAsyncSource source, CancellationToken cancellationToken) + public static async Task> PumpToListAsync(this IAsyncSource source, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); var buffer = new FoundationDB.Linq.FdbAsyncEnumerable.Buffer(); @@ -241,7 +241,7 @@ public static async Task> PumpToListAsync(this IAsyncSource source (x, _) => buffer.Add(x) ); - await PumpToAsync(source, target, cancellationToken).ConfigureAwait(false); + await PumpToAsync(source, target, ct).ConfigureAwait(false); return buffer.ToList(); } @@ -270,19 +270,19 @@ public static AsyncTransform CreateAsyncTransform(Func(transform, target, scheduler); } - public static async Task> TransformToListAsync(IAsyncSource source, Func> transform, CancellationToken cancellationToken, int? maxConcurrency = null, TaskScheduler scheduler = null) + public static async Task> TransformToListAsync(IAsyncSource source, Func> transform, CancellationToken ct, int? maxConcurrency = null, TaskScheduler scheduler = null) { - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); using (var queue = CreateOrderPreservingAsyncBuffer(maxConcurrency ?? 32)) { using (var pipe = CreateAsyncTransform(transform, queue, scheduler)) { // start the output pump - var output = PumpToListAsync(queue, cancellationToken); + var output = PumpToListAsync(queue, ct); // start the intput pump - var input = PumpToAsync(source, pipe, cancellationToken); + var input = PumpToAsync(source, pipe, ct); await Task.WhenAll(input, output).ConfigureAwait(false); diff --git a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs index 68c10f003..81ec68d41 100644 --- a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs +++ b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs @@ -65,7 +65,7 @@ protected AsyncProducerConsumerQueue(int capacity) m_capacity = capacity; } - public abstract Task OnNextAsync(T value, CancellationToken cancellationToken); + public abstract Task OnNextAsync(T value, CancellationToken ct); public abstract void OnCompleted(); diff --git a/FoundationDB.Client/Async/AsyncPump.cs b/FoundationDB.Client/Async/AsyncPump.cs index 98b93cf6f..601fb7b30 100644 --- a/FoundationDB.Client/Async/AsyncPump.cs +++ b/FoundationDB.Client/Async/AsyncPump.cs @@ -79,7 +79,7 @@ internal int State public IAsyncTarget Target { [NotNull] get { return m_target; } } /// Run the pump until the inner iterator is done, an error occurs, or the cancellation token is fired - public async Task PumpAsync(bool stopOnFirstError, CancellationToken cancellationToken) + public async Task PumpAsync(bool stopOnFirstError, CancellationToken ct) { if (m_state != STATE_IDLE) { @@ -115,16 +115,16 @@ public async Task PumpAsync(bool stopOnFirstError, CancellationToken cancellatio { LogPump("Starting pump"); - while (!cancellationToken.IsCancellationRequested && m_state != STATE_DISPOSED) + while (!ct.IsCancellationRequested && m_state != STATE_DISPOSED) { LogPump("Waiting for next"); m_state = STATE_WAITING_FOR_NEXT; - var current = await m_source.ReceiveAsync(cancellationToken).ConfigureAwait(false); + var current = await m_source.ReceiveAsync(ct).ConfigureAwait(false); LogPump("Received " + (current.HasValue ? "value" : current.HasFailed ? "error" : "completion") + ", publishing..."); m_state = STATE_PUBLISHING_TO_TARGET; - await m_target.Publish(current, cancellationToken).ConfigureAwait(false); + await m_target.Publish(current, ct).ConfigureAwait(false); if (current.HasFailed && stopOnFirstError) { @@ -141,7 +141,7 @@ public async Task PumpAsync(bool stopOnFirstError, CancellationToken cancellatio } // push the cancellation on the queue, and throw - throw new OperationCanceledException(cancellationToken); + throw new OperationCanceledException(ct); } catch (Exception e) { diff --git a/FoundationDB.Client/Async/AsyncTaskBuffer.cs b/FoundationDB.Client/Async/AsyncTaskBuffer.cs index 6e89e2097..421840df4 100644 --- a/FoundationDB.Client/Async/AsyncTaskBuffer.cs +++ b/FoundationDB.Client/Async/AsyncTaskBuffer.cs @@ -71,9 +71,9 @@ public AsyncTaskBuffer(AsyncOrderingMode mode, int capacity) #region IFdbAsyncTarget... - public override Task OnNextAsync(Task task, CancellationToken cancellationToken) + public override Task OnNextAsync(Task task, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); LogProducer("Received task #" + task.Id + " (" + task.Status + ")"); @@ -104,11 +104,11 @@ public override Task OnNextAsync(Task task, CancellationToken cancellationTok } // we are blocked, we will need to wait ! - wait = MarkProducerAsBlocked_NeedsLocking(cancellationToken); + wait = MarkProducerAsBlocked_NeedsLocking(ct); } // slow path - return WaitForNextFreeSlotThenEnqueueAsync(task, wait, cancellationToken); + return WaitForNextFreeSlotThenEnqueueAsync(task, wait, ct); } private void NotifyConsumerOfTaskCompletion_NeedsLocking() diff --git a/FoundationDB.Client/Async/AsyncTransform.cs b/FoundationDB.Client/Async/AsyncTransform.cs index 1a76ab222..c9d0886d9 100644 --- a/FoundationDB.Client/Async/AsyncTransform.cs +++ b/FoundationDB.Client/Async/AsyncTransform.cs @@ -62,9 +62,9 @@ public AsyncTransform([NotNull] Func> transform, [ #region IAsyncTarget... - public Task OnNextAsync(T value, CancellationToken cancellationToken) + public Task OnNextAsync(T value, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) return TaskHelpers.CompletedTask; + if (ct.IsCancellationRequested) return TaskHelpers.CompletedTask; if (m_done) throw new InvalidOperationException("Cannot send any more values because this transform has already completed"); @@ -76,7 +76,7 @@ public Task OnNextAsync(T value, CancellationToken cancellationToken) Task task; if (m_scheduler == null) { // execute inline - task = m_transform(value, cancellationToken); + task = m_transform(value, ct); } else { // execute in a scheduler @@ -86,14 +86,14 @@ public Task OnNextAsync(T value, CancellationToken cancellationToken) var prms = (Tuple, T, CancellationToken>)state; return prms.Item1.m_transform(prms.Item2, prms.Item3); }, - Tuple.Create(this, value, cancellationToken), - cancellationToken, + Tuple.Create(this, value, ct), + ct, TaskCreationOptions.PreferFairness, m_scheduler ).Unwrap(); } - return m_target.OnNextAsync(task, cancellationToken); + return m_target.OnNextAsync(task, ct); } catch(Exception e) { diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Async/AsyncTransformQueue.cs index 9a303aaaa..b5bb6c3ff 100644 --- a/FoundationDB.Client/Async/AsyncTransformQueue.cs +++ b/FoundationDB.Client/Async/AsyncTransformQueue.cs @@ -53,8 +53,8 @@ internal class AsyncTransformQueue : IAsyncBuffer> transform, int capacity, TaskScheduler scheduler) { - if (transform == null) throw new ArgumentNullException("transform"); - if (capacity <= 0) throw new ArgumentOutOfRangeException("capacity", "Capacity must be greater than zero"); + if (transform == null) throw new ArgumentNullException(nameof(transform)); + if (capacity <= 0) throw new ArgumentOutOfRangeException(nameof(capacity), "Capacity must be greater than zero"); m_transform = transform; m_capacity = capacity; @@ -147,9 +147,9 @@ private static async Task> ProcessItemHandler(object state) private static readonly Func>> s_processItemHandler = ProcessItemHandler; - public async Task OnNextAsync(TInput value, CancellationToken cancellationToken) + public async Task OnNextAsync(TInput value, CancellationToken ct) { - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { AsyncCancelableMutex waiter; lock (m_lock) @@ -160,8 +160,8 @@ public async Task OnNextAsync(TInput value, CancellationToken cancellationToken) { var t = Task.Factory.StartNew( s_processItemHandler, - Tuple.Create(this, value, cancellationToken), - cancellationToken, + Tuple.Create(this, value, ct), + ct, TaskCreationOptions.PreferFairness, m_scheduler ).Unwrap(); @@ -184,14 +184,14 @@ public async Task OnNextAsync(TInput value, CancellationToken cancellationToken) } // no luck, we need to wait for the queue to become non-full - waiter = new AsyncCancelableMutex(cancellationToken); + waiter = new AsyncCancelableMutex(ct); m_blockedProducer = waiter; } await waiter.Task.ConfigureAwait(false); } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); } public void OnCompleted() @@ -232,18 +232,18 @@ ExceptionDispatchInfo error #region IFdbAsyncBatchTarget... - public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken cancellationToken) + public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken ct) { - if (batch == null) throw new ArgumentNullException("batch"); + if (batch == null) throw new ArgumentNullException(nameof(batch)); if (batch.Length == 0) return; - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); //TODO: optimized version ! foreach (var item in batch) { - await OnNextAsync(item, cancellationToken).ConfigureAwait(false); + await OnNextAsync(item, ct).ConfigureAwait(false); } } diff --git a/FoundationDB.Client/Async/IAsyncEnumerator.cs b/FoundationDB.Client/Async/IAsyncEnumerator.cs index 40823eefa..580201bb0 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerator.cs +++ b/FoundationDB.Client/Async/IAsyncEnumerator.cs @@ -40,12 +40,12 @@ namespace FoundationDB.Async public interface IAsyncEnumerator : IDisposable { /// Advances the enumerator to the next element in the sequence, returning the result asynchronously. - /// Cancellation token that can be used to cancel the operation. + /// Cancellation token that can be used to cancel the operation. /// /// Task containing the result of the operation: true if the enumerator was successfully advanced /// to the next element; false if the enumerator has passed the end of the sequence. /// - Task MoveNextAsync(CancellationToken cancellationToken); + Task MoveNextAsync(CancellationToken ct); /// Gets the current element in the iteration. T Current { get; } diff --git a/FoundationDB.Client/Async/IAsyncPump.cs b/FoundationDB.Client/Async/IAsyncPump.cs index 192ed854d..12011eca1 100644 --- a/FoundationDB.Client/Async/IAsyncPump.cs +++ b/FoundationDB.Client/Async/IAsyncPump.cs @@ -47,9 +47,9 @@ public interface IAsyncPump : IDisposable /// Consume all the items of the source by passing them to the Target /// If true, aborts on the first error. If false, continue processing items until the source has finished. - /// Cancellation token that can be used to abort the pump at any time. Any unprocessed items will be lost. + /// Cancellation token that can be used to abort the pump at any time. Any unprocessed items will be lost. /// Task that will complete successfully if all the items from the source have been processed by the target, or fails if an error occurred or the pump was cancelled. - Task PumpAsync(bool stopOnFirstError, CancellationToken cancellationToken); + Task PumpAsync(bool stopOnFirstError, CancellationToken ct); } } diff --git a/FoundationDB.Client/Async/IAsyncSource.cs b/FoundationDB.Client/Async/IAsyncSource.cs index 514b03273..e98c488a4 100644 --- a/FoundationDB.Client/Async/IAsyncSource.cs +++ b/FoundationDB.Client/Async/IAsyncSource.cs @@ -39,9 +39,9 @@ public interface IAsyncSource //note: T cannot be covariant because Task<..> is not covariant :( /// Consume a new value from the source - /// Token used to cancel the operation + /// Token used to cancel the operation /// Task that will return a new value, nothing (if it has completed) or on exception - Task> ReceiveAsync(CancellationToken cancellationToken); + Task> ReceiveAsync(CancellationToken ct); } } diff --git a/FoundationDB.Client/Async/IAsyncTarget.cs b/FoundationDB.Client/Async/IAsyncTarget.cs index 9142ac5c0..df46a1cf0 100644 --- a/FoundationDB.Client/Async/IAsyncTarget.cs +++ b/FoundationDB.Client/Async/IAsyncTarget.cs @@ -42,9 +42,9 @@ public interface IAsyncTarget /// Push a new item onto the target, if it can accept one /// New value that is being published - /// Cancellation token that is used to abort the call if the target is blocked + /// Cancellation token that is used to abort the call if the target is blocked /// Task that completes once the target has accepted the new value (or fails if the cancellation token fires) - Task OnNextAsync(T value, CancellationToken cancellationToken); + Task OnNextAsync(T value, CancellationToken ct); /// Notifies the target that the producer is done and that no more values will be published void OnCompleted(); diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Async/TaskHelpers.cs index d1a5fe6ac..4d43e3999 100644 --- a/FoundationDB.Client/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Async/TaskHelpers.cs @@ -300,18 +300,18 @@ public static Func> WithCancellationReturns a cancelled Task that is linked with a specific token /// Type of the result of the task - /// Cancellation token that should already be cancelled + /// Cancellation token that should already be cancelled /// Task in the cancelled state that is linked with this cancellation token - public static Task FromCancellation(CancellationToken cancellationToken) + public static Task FromCancellation(CancellationToken ct) { // There is a Task.FromCancellation() method in the BCL, but unfortunately it is internal :( // The "best" way I've seen to emulate the same behavior, is creating a fake task (with a dummy action) with the same alread-cancelled CancellationToken // This should throw the correct TaskCanceledException that is linked with this token // ensure that it is actually cancelled, so that we don't deadlock - if (!cancellationToken.IsCancellationRequested) throw new InvalidOperationException(); + if (!ct.IsCancellationRequested) throw new InvalidOperationException(); - return new Task(Cache.Nop, cancellationToken); + return new Task(Cache.Nop, ct); } /// Returns a cancelled Task that is not linked to any particular token @@ -351,14 +351,14 @@ public static Task FromException(Exception e) /// Returns a failed Task that wraps an exception /// Type of the result of the task /// Exception that will be wrapped in the task - /// Original cancellation token that may have triggered + /// Original cancellation token that may have triggered /// Task that is already completed, and that will rethrow the exception once observed - public static Task FromFailure(Exception e, CancellationToken cancellationToken) + public static Task FromFailure(Exception e, CancellationToken ct) { if (e is OperationCanceledException) { - if (cancellationToken.IsCancellationRequested) - return FromCancellation(cancellationToken); + if (ct.IsCancellationRequested) + return FromCancellation(ct); else return Canceled(); } diff --git a/FoundationDB.Client/Core/IFdbClusterHandler.cs b/FoundationDB.Client/Core/IFdbClusterHandler.cs index 2d88d4608..2eac8ab30 100644 --- a/FoundationDB.Client/Core/IFdbClusterHandler.cs +++ b/FoundationDB.Client/Core/IFdbClusterHandler.cs @@ -43,7 +43,7 @@ public interface IFdbClusterHandler : IDisposable void SetOption(FdbClusterOption option, Slice data); [ItemNotNull] - Task OpenDatabaseAsync(string databaseName, CancellationToken cancellationToken); + Task OpenDatabaseAsync(string databaseName, CancellationToken ct); } } diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 84503c585..4c59c4bc0 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.cs @@ -52,7 +52,7 @@ public interface IFdbTransactionHandler : IDisposable void SetOption(FdbTransactionOption option, Slice data); /// Returns this transaction snapshot read version. - Task GetReadVersionAsync(CancellationToken cancellationToken); + Task GetReadVersionAsync(CancellationToken ct); /// 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. @@ -72,32 +72,32 @@ public interface IFdbTransactionHandler : IDisposable /// Reads a get from the database /// Key to read /// Set to true for snapshot reads - /// + /// /// - Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken); + Task GetAsync(Slice key, bool snapshot, CancellationToken ct); /// 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 /// 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. [ItemNotNull] - Task GetValuesAsync([NotNull] Slice[] keys, bool snapshot, CancellationToken cancellationToken); + Task GetValuesAsync([NotNull] Slice[] keys, bool snapshot, CancellationToken ct); /// 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 /// Task that will return the key matching the selector, or an exception - Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken cancellationToken); + Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken ct); /// 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 /// Task that will return an array of keys matching the selectors, or an exception [ItemNotNull] - Task GetKeysAsync([NotNull] KeySelector[] selectors, bool snapshot, CancellationToken cancellationToken); + Task GetKeysAsync([NotNull] KeySelector[] selectors, bool snapshot, CancellationToken ct); /// 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 @@ -105,16 +105,16 @@ 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 /// - Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken); + Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, [NotNull] FdbRangeOptions options, int iteration, bool snapshot, CancellationToken ct); /// 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 /// Task that will return an array of strings, or an exception [ItemNotNull] - Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken); + Task GetAddressesForKeyAsync(Slice key, CancellationToken ct); /// 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(). @@ -148,28 +148,28 @@ public interface IFdbTransactionHandler : IDisposable /// Watch a key for any change in the database. /// Key to watch - /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance + /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance /// FdbWatch that can be awaited and will complete when the key has changed in the database, or cancellation occurs. You can call Cancel() at any time if you are not interested in watching the key anymore. You MUST always call Dispose() if the watch completes or is cancelled, to ensure that resources are released properly. /// You can directly await an FdbWatch, or obtain a Task<Slice> by reading the property. - FdbWatch Watch(Slice key, CancellationToken cancellationToken); + FdbWatch Watch(Slice key, CancellationToken ct); /// Attempts to commit the sets and clears previously applied to the database snapshot represented by this transaction to the actual database. /// 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 /// 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 ct); /// 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 /// 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 ct); /// 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.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index 553638344..be530b06e 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -86,21 +86,21 @@ public WriteOptions() /// Writes a potentially large sequence of key/value pairs into the database, by using as many transactions as necessary, and automatically scaling the size of each batch. /// Database used for the operation /// Sequence of key/value pairs - /// Token used to cancel the operation + /// Token used to cancel the operation /// Total number of values inserted in the database /// In case of a non-retryable error, some of the keys may remain in the database. Other transactions running at the same time may observe only a fraction of the keys until the operation completes. - public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, CancellationToken cancellationToken) + public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (data == null) throw new ArgumentNullException("data"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (data == null) throw new ArgumentNullException(nameof(data)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunWriteOperationAsync( db, data, new WriteOptions(), - cancellationToken + ct ); } @@ -108,29 +108,29 @@ public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumer /// Database used for the operation /// Sequence of key/value pairs /// Custom options used to configure the behaviour of the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Total number of values inserted in the database /// In case of a non-retryable error, some of the keys may remain in the database. Other transactions running at the same time may observe only a fraction of the keys until the operation completes. - public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, WriteOptions options, CancellationToken cancellationToken) + public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, WriteOptions options, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (data == null) throw new ArgumentNullException("data"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (data == null) throw new ArgumentNullException(nameof(data)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunWriteOperationAsync( db, data, options ?? new WriteOptions(), - cancellationToken + ct ); } - internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, WriteOptions options, CancellationToken cancellationToken) + internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, WriteOptions options, CancellationToken ct) { Contract.Requires(db != null && data != null && options != null); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); // We will batch keys into chunks (bounding by count and bytes), then attempt to insert that batch in the database. // Each transaction should try to never exceed ~1MB of size @@ -152,7 +152,7 @@ internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase d { if (progress != null) progress.Report(0); - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { chunk.Clear(); int bytes = 0; @@ -180,7 +180,7 @@ await db.WriteAsync((tr) => { tr.Set(pair.Key, pair.Value); } - }, cancellationToken).ConfigureAwait(false); + }, ct).ConfigureAwait(false); items += chunk.Count; @@ -188,7 +188,7 @@ await db.WriteAsync((tr) => } } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return items; } @@ -214,23 +214,23 @@ await db.WriteAsync((tr) => /// Database used for the operation /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, CancellationToken cancellationToken) + public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunInsertOperationAsync( db, source, handler, new WriteOptions(), - cancellationToken + ct ); } @@ -240,23 +240,23 @@ public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEn /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. /// Custom options used to configure the behaviour of the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, WriteOptions options, CancellationToken cancellationToken) + public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, WriteOptions options, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunInsertOperationAsync( db, source, handler, options ?? new WriteOptions(), - cancellationToken + ct ); } @@ -265,23 +265,23 @@ public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEn /// Database used for the operation /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, CancellationToken cancellationToken) + public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunInsertOperationAsync( db, source, handler, new WriteOptions(), - cancellationToken + ct ); } @@ -291,23 +291,23 @@ public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEn /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. /// Custom options used to configure the behaviour of the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, WriteOptions options, CancellationToken cancellationToken) + public static Task InsertAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, WriteOptions options, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunInsertOperationAsync( db, source, handler, options ?? new WriteOptions(), - cancellationToken + ct ); } @@ -317,7 +317,7 @@ internal static async Task RunInsertOperationAsync( [NotNull] IEnumerable source, [NotNull] Delegate body, [NotNull] WriteOptions options, - CancellationToken cancellationToken + CancellationToken ct ) { Contract.Requires(db != null && source != null && body != null && options != null); @@ -337,13 +337,13 @@ CancellationToken cancellationToken var bodyBlocking = body as Action; if (bodyAsync == null && bodyBlocking == null) { - throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), "body"); + throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), nameof(body)); } var batch = new List(batchCount); long itemCount = 0; - using (var trans = db.BeginTransaction(cancellationToken)) + using (var trans = db.BeginTransaction(ct)) { var timer = Stopwatch.StartNew(); @@ -398,7 +398,7 @@ CancellationToken cancellationToken foreach(var item in source) { - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; // store it (in case we need to retry) batch.Add(item); @@ -424,7 +424,7 @@ CancellationToken cancellationToken } } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); // handle the last (or only) batch if (batch.Count > 0) @@ -530,23 +530,23 @@ private static async Task RetryChunk([NotNull] IFdbTransaction trans, [ /// Database used for the operation /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, CancellationToken cancellationToken) + public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunBatchedInsertOperationAsync( db, source, handler, new WriteOptions(), - cancellationToken + ct ); } @@ -556,23 +556,23 @@ public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNu /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. /// Custom options used to configure the behaviour of the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, WriteOptions options, CancellationToken cancellationToken) + public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action handler, WriteOptions options, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunBatchedInsertOperationAsync( db, source, handler, options ?? new WriteOptions(), - cancellationToken + ct ); } @@ -581,23 +581,23 @@ public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNu /// Database used for the operation /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, CancellationToken cancellationToken) + public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunBatchedInsertOperationAsync( db, source, handler, new WriteOptions(), - cancellationToken + ct ); } @@ -607,23 +607,23 @@ public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNu /// Sequence of items to be processed /// Lambda called at least once for each item in the source. The method may not have any side effect outside of the passed transaction. /// Custom options used to configure the behaviour of the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of items that have been inserted /// In case of a non-retryable error, some of the items may remain in the database. Other transactions running at the same time may observe only a fraction of the items until the operation completes. - public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, WriteOptions options, CancellationToken cancellationToken) + public static Task InsertBatchedAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func handler, WriteOptions options, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return RunBatchedInsertOperationAsync( db, source, handler, options ?? new WriteOptions(), - cancellationToken + ct ); } @@ -633,7 +633,7 @@ internal static async Task RunBatchedInsertOperationAsync( [NotNull] IEnumerable source, [NotNull] Delegate body, [NotNull] WriteOptions options, - CancellationToken cancellationToken + CancellationToken ct ) { Contract.Requires(db != null && source != null && body != null && options != null); @@ -653,14 +653,14 @@ CancellationToken cancellationToken var bodyBlocking = body as Action; if (bodyAsync == null && bodyBlocking == null) { - throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), "body"); + throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), nameof(body)); } var chunk = new List(batchCount); // holds all the items processed in the current transaction cycle long itemCount = 0; // total number of items processed - using (var trans = db.BeginTransaction(cancellationToken)) + using (var trans = db.BeginTransaction(ct)) { var timer = Stopwatch.StartNew(); @@ -725,7 +725,7 @@ CancellationToken cancellationToken foreach (var item in source) { - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; // store it (in case we need to retry) chunk.Add(item); @@ -758,7 +758,7 @@ CancellationToken cancellationToken } } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); // handle the last (or only) batch if (chunk.Count > 0) @@ -931,23 +931,23 @@ public void Stop() /// Lambda function that is called once, and returns the initial state that will be passed to the first batch /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. /// Lambda function that will be called after the last batch, and will be passed the last known state. - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered public static Task ForEachAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func localInit, [NotNull, InstantHandle] Func> body, [NotNull, InstantHandle] Action localFinally, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (localInit == null) throw new ArgumentNullException("localInit"); - if (body == null) throw new ArgumentNullException("body"); - if (localFinally == null) throw new ArgumentNullException("localFinally"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (localInit == null) throw new ArgumentNullException(nameof(localInit)); + if (body == null) throw new ArgumentNullException(nameof(body)); + if (localFinally == null) throw new ArgumentNullException(nameof(localFinally)); - return RunBatchedReadOperationAsync(db, source, localInit, body, localFinally, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, localInit, body, localFinally, DefaultInitialBatchSize, ct); } /// Execute a potentially long read-only operation on batches of elements from a source sequence, using as many transactions as necessary, and automatically scaling the size of each batch to maximize the throughput. @@ -958,8 +958,8 @@ public static Task ForEachAsync( /// Lambda function that is called once, and returns the initial state that will be passed to the first batch /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. /// Lambda function that will be called after the last batch, and will be passed the last known state. - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered [Obsolete("EXPERIMENTAL: do not use yet!")] public static Task ForEachAsync( [NotNull] IFdbDatabase db, @@ -967,20 +967,20 @@ public static Task ForEachAsync( [NotNull, InstantHandle] Func localInit, [NotNull, InstantHandle] Func body, [NotNull, InstantHandle] Action localFinally, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (localInit == null) throw new ArgumentNullException("localInit"); - if (body == null) throw new ArgumentNullException("body"); - if (localFinally == null) throw new ArgumentNullException("localFinally"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (localInit == null) throw new ArgumentNullException(nameof(localInit)); + if (body == null) throw new ArgumentNullException(nameof(body)); + if (localFinally == null) throw new ArgumentNullException(nameof(localFinally)); //REVIEW: what is the point if the body is not async ? // > either is can read and generate past_version errors then it needs to be async // > either it's not async, then it could only Write/Clear, and in which case we need a writeable transaction ... ? (and who will commit and when ??) // It could maybe make sense if the source was an IAsyncEnumerable because you could not use Parallel.ForEach(...) for that - return RunBatchedReadOperationAsync(db, source, localInit, body, localFinally, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, localInit, body, localFinally, DefaultInitialBatchSize, ct); } /// Execute a potentially long read-only operation on batches of elements from a source sequence, using as many transactions as necessary, and automatically scaling the size of each batch to maximize the throughput. @@ -988,19 +988,19 @@ public static Task ForEachAsync( /// Source database /// Source sequence that will be split into batch. The size of each batch will scale up and down automatically depending on the speed of execution /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered public static Task ForEachAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func body, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (body == null) throw new ArgumentNullException("body"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (body == null) throw new ArgumentNullException(nameof(body)); - return RunBatchedReadOperationAsync(db, source, null, body, null, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, null, body, null, DefaultInitialBatchSize, ct); } /// Execute a potentially long read-only operation on batches of elements from a source sequence, using as many transactions as necessary, and automatically scaling the size of each batch to maximize the throughput. @@ -1008,25 +1008,25 @@ public static Task ForEachAsync( /// Source database /// Source sequence that will be split into batch. The size of each batch will scale up and down automatically depending on the speed of execution /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered [Obsolete("EXPERIMENTAL: do not use yet!")] public static Task ForEachAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Action body, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (body == null) throw new ArgumentNullException("body"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (body == null) throw new ArgumentNullException(nameof(body)); //REVIEW: what is the point if the body is not async ? // > either is can read and generate past_version errors then it needs to be async // > either it's not async, then it could only Write/Clear, and in which case we need a writeable transaction ... ? (and who will commit and when ??) // It could maybe make sense if the source was an IAsyncEnumerable because you could not use Parallel.ForEach(...) for that - return RunBatchedReadOperationAsync(db, source, null, body, null, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, null, body, null, DefaultInitialBatchSize, ct); } #endregion @@ -1040,22 +1040,22 @@ public static Task ForEachAsync( /// Source sequence that will be split into batch. The size of each batch will scale up and down automatically depending on the speed of execution /// Lambda function that is called once, and returns the initial state that will be passed to the first batch /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered public static Task AggregateAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func localInit, [NotNull, InstantHandle] Func> body, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (localInit == null) throw new ArgumentNullException("localInit"); - if (body == null) throw new ArgumentNullException("body"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (localInit == null) throw new ArgumentNullException(nameof(localInit)); + if (body == null) throw new ArgumentNullException(nameof(body)); Func identity = (x) => x; - return RunBatchedReadOperationAsync(db, source, localInit, body, identity, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, localInit, body, identity, DefaultInitialBatchSize, ct); } /// Execute a potentially long aggregation on batches of elements from a source sequence, using as many transactions as necessary, and automatically scaling the size of each batch to maximize the throughput. @@ -1067,23 +1067,23 @@ public static Task AggregateAsync( /// Lambda function that is called once, and returns the initial state that will be passed to the first batch /// Retryable lambda function that receives a batch of elements from the source sequence, the current context, and the previous state. If the transaction expires while this lambda function is running, it will be automatically retried with a new transaction, and a smaller batch. /// Lambda function called with the aggregate returned by the last batch, and which will compute the final result of the operation - /// Token used to cancel the operation - /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered + /// Token used to cancel the operation + /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered public static Task AggregateAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, [NotNull, InstantHandle] Func init, [NotNull, InstantHandle] Func> body, [NotNull, InstantHandle] Func transform, - CancellationToken cancellationToken) + CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (source == null) throw new ArgumentNullException("source"); - if (init == null) throw new ArgumentNullException("init"); - if (body == null) throw new ArgumentNullException("body"); - if (transform == null) throw new ArgumentNullException("transform"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (init == null) throw new ArgumentNullException(nameof(init)); + if (body == null) throw new ArgumentNullException(nameof(body)); + if (transform == null) throw new ArgumentNullException(nameof(transform)); - return RunBatchedReadOperationAsync(db, source, init, body, transform, DefaultInitialBatchSize, cancellationToken); + return RunBatchedReadOperationAsync(db, source, init, body, transform, DefaultInitialBatchSize, ct); } #endregion @@ -1096,7 +1096,7 @@ internal static async Task RunBatchedReadOperationAsync 0 && body != null); @@ -1111,7 +1111,7 @@ CancellationToken cancellationToken bodyWithContextAndState == null && bodyWithContext == null) { - throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), "body"); + throw new ArgumentException(String.Format("Unsupported delegate type {0} for body", body.GetType().FullName), nameof(body)); } var localFinallyVoid = localFinally as Action; @@ -1120,7 +1120,7 @@ CancellationToken cancellationToken localFinallyVoid == null && localFinallyWithResult == null) { - throw new ArgumentException(String.Format("Unsupported delegate type {0} for local finally", body.GetType().FullName), "localFinally"); + throw new ArgumentException(String.Format("Unsupported delegate type {0} for local finally", body.GetType().FullName), nameof(localFinally)); } int batchSize = initialBatchSize; @@ -1135,7 +1135,7 @@ CancellationToken cancellationToken { var totalTimer = Stopwatch.StartNew(); - using (var trans = db.BeginReadOnlyTransaction(cancellationToken)) + using (var trans = db.BeginReadOnlyTransaction(ct)) { var ctx = new BatchOperationContext() { @@ -1154,7 +1154,7 @@ CancellationToken cancellationToken localInitialized = true; } - while (!ctx.Abort && !cancellationToken.IsCancellationRequested) + while (!ctx.Abort && !ct.IsCancellationRequested) { FillNextBatch(iterator, batch, ctx.Step); @@ -1164,7 +1164,7 @@ CancellationToken cancellationToken } int offsetInCurrentBatch = 0; - while (!ctx.Abort && !cancellationToken.IsCancellationRequested) + while (!ctx.Abort && !ct.IsCancellationRequested) { var r = Math.Min(ctx.Step, batch.Count - offsetInCurrentBatch); if (items == null || items.Length != r) @@ -1249,12 +1249,12 @@ CancellationToken cancellationToken } } } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); ctx.Position += offsetInCurrentBatch; batch.Clear(); } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); } catch(Exception) { @@ -1309,24 +1309,24 @@ private static int FillNextBatch(IEnumerator iterator, List batch, int /// Key defining the start range (included) /// Key defining the end of the range (excluded) /// Lambda that will be called for each batch of data read from the database. The first argument is the array of ordered key/value pairs in the batch, taken from the same database snapshot. The second argument is the offset of the first item in the array, from the start of the range. The third argument is a token should be used by any async i/o performed by the lambda. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys exported /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. - public static Task ExportAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) + public static Task ExportAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken ct) { - return ExportAsync(db, KeySelector.FirstGreaterOrEqual(beginInclusive), KeySelector.FirstGreaterOrEqual(endExclusive), handler, cancellationToken); + return ExportAsync(db, KeySelector.FirstGreaterOrEqual(beginInclusive), KeySelector.FirstGreaterOrEqual(endExclusive), handler, ct); } /// Export the content of a potentially large range of keys defined by a pair of begin and end keys. /// Database used for the operation /// Pair of keys defining the start range /// Lambda that will be called for each batch of data read from the database. The first argument is the array of ordered key/value pairs in the batch, taken from the same database snapshot. The second argument is the offset of the first item in the array, from the start of the range. The third argument is a token should be used by any async i/o performed by the lambda. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys exported /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. - public static Task ExportAsync([NotNull] IFdbDatabase db, KeyRange range, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) + public static Task ExportAsync([NotNull] IFdbDatabase db, KeyRange range, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken ct) { - return ExportAsync(db, KeySelector.FirstGreaterOrEqual(range.Begin), KeySelector.FirstGreaterOrEqual(range.End), handler, cancellationToken); + return ExportAsync(db, KeySelector.FirstGreaterOrEqual(range.Begin), KeySelector.FirstGreaterOrEqual(range.End), handler, ct); } /// Export the content of a potentially large range of keys defined by a pair of selectors. @@ -1334,14 +1334,14 @@ public static Task ExportAsync([NotNull] IFdbDatabase db, KeyRange range, /// Selector defining the start of the range (included) /// Selector defining the end of the range (excluded) /// Lambda that will be called for each batch of data read from the database. The first argument is the array of ordered key/value pairs in the batch, taken from the same database snapshot. The second argument is the offset of the first item in the array, from the start of the range. The third argument is a token should be used by any async i/o performed by the lambda. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys exported /// This method cannot guarantee that all data will be read from the same snapshot of the database, which means that writes committed while the export is running may be seen partially. Only the items inside a single batch are guaranteed to be from the same snapshot of the database. - public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelector begin, KeySelector end, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken cancellationToken) + public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelector begin, KeySelector end, [NotNull, InstantHandle] Func[], long, CancellationToken, Task> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); - cancellationToken.ThrowIfCancellationRequested(); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); + ct.ThrowIfCancellationRequested(); // to maximize throughput, we want to read as much as possible per transaction, so that means that we should prefetch the next batch while the current batch is processing @@ -1369,7 +1369,7 @@ public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelecto tr.WithPriorityBatch(); }; - using (var tr = db.BeginReadOnlyTransaction(cancellationToken)) + using (var tr = db.BeginReadOnlyTransaction(ct)) { reset(tr); @@ -1396,8 +1396,8 @@ public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelecto // process the current one if (page.Count > 0) { - cancellationToken.ThrowIfCancellationRequested(); - await handler(page.Chunk, count, cancellationToken); + ct.ThrowIfCancellationRequested(); + await handler(page.Chunk, count, ct); ++chunks; count += page.Count; } @@ -1409,8 +1409,8 @@ public static async Task ExportAsync([NotNull] IFdbDatabase db, KeySelecto // process the last page, if any if (page.Count > 0) { - cancellationToken.ThrowIfCancellationRequested(); - await handler(page.Chunk, count, cancellationToken); + ct.ThrowIfCancellationRequested(); + await handler(page.Chunk, count, ct); ++chunks; count += page.Count; } diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index c45e9659b..9a07152aa 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -90,7 +90,7 @@ public static class System [ItemCanBeNull] public static async Task GetStatusAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); Slice data = await trans.GetAsync(StatusJsonKey).ConfigureAwait(false); @@ -113,7 +113,7 @@ public static async Task GetStatusAsync([NotNull] IFdbReadOnlyT [ItemCanBeNull] public static async Task GetStatusAsync([NotNull] IFdbDatabase db, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); // we should not retry the read to the status key! using (var trans = db.BeginReadOnlyTransaction(ct)) @@ -130,12 +130,12 @@ public static async Task GetStatusAsync([NotNull] IFdbDatabase /// Returns an object describing the list of the current coordinators for the cluster /// Database to use for the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Since the list of coordinators may change at anytime, the results may already be obsolete once this method completes! [ItemNotNull] - public static async Task GetCoordinatorsAsync([NotNull] IFdbDatabase db, CancellationToken cancellationToken) + public static async Task GetCoordinatorsAsync([NotNull] IFdbDatabase db, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); var coordinators = await db.ReadAsync((tr) => { @@ -144,7 +144,7 @@ public static async Task GetCoordinatorsAsync([NotNull] IFdbData //note: we ask for high priotity, because this method maybe called by a monitoring system than has to run when the cluster is clogged up in requests return tr.GetAsync(Fdb.System.Coordinators); - }, cancellationToken).ConfigureAwait(false); + }, ct).ConfigureAwait(false); if (coordinators.IsNull) throw new InvalidOperationException("Failed to read the list of coordinators from the cluster's system keyspace."); @@ -154,12 +154,12 @@ public static async Task GetCoordinatorsAsync([NotNull] IFdbData /// Return the value of a configuration parameter (located under '\xFF/conf/') /// Database to use for the operation /// Name of the configuration key (ex: "storage_engine") - /// Token used to cancel the operation + /// Token used to cancel the operation /// Value of '\xFF/conf/storage_engine' - public static Task GetConfigParameterAsync([NotNull] IFdbDatabase db, [NotNull] string name, CancellationToken cancellationToken) + public static Task GetConfigParameterAsync([NotNull] IFdbDatabase db, [NotNull] string name, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (string.IsNullOrEmpty(name)) throw new ArgumentException("Configuration parameter name cannot be null or empty", "name"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (string.IsNullOrEmpty(name)) throw new ArgumentException("Configuration parameter name cannot be null or empty", nameof(name)); return db.ReadAsync((tr) => { @@ -168,7 +168,7 @@ public static Task GetConfigParameterAsync([NotNull] IFdbDatabase db, [No //note: we ask for high priotity, because this method maybe called by a monitoring system than has to run when the cluster is clogged up in requests return tr.GetAsync(Fdb.System.ConfigKey(name)); - }, cancellationToken); + }, ct); } /// Return the corresponding key for a config attribute @@ -176,7 +176,7 @@ public static Task GetConfigParameterAsync([NotNull] IFdbDatabase db, [No /// "\xFF/conf/foo" public static Slice ConfigKey([NotNull] string name) { - if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", "name"); + if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); return ConfigPrefix + Slice.FromAscii(name); } @@ -185,7 +185,7 @@ public static Slice ConfigKey([NotNull] string name) /// "\xFF/globals/foo" public static Slice GlobalsKey([NotNull] string name) { - if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", "name"); + if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); return GlobalsPrefix + Slice.FromAscii(name); } @@ -195,22 +195,22 @@ public static Slice GlobalsKey([NotNull] string name) /// "\xFF/workers/ABC123/foo" public static Slice WorkersKey([NotNull] string id, [NotNull] string name) { - if (string.IsNullOrEmpty(id)) throw new ArgumentException("Id cannot be null or empty", "id"); - if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", "name"); + if (string.IsNullOrEmpty(id)) throw new ArgumentException("Id cannot be null or empty", nameof(id)); + if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); return WorkersPrefix + Slice.FromAscii(id) + Slice.FromChar('/') + Slice.FromAscii(name); } /// Returns the current storage engine mode of the cluster /// Database to use for the operation - /// Token used to cancel the operation + /// Token used to cancel the operation /// Returns either "memory" or "ssd" /// Will return a string starting with "unknown" if the storage engine mode is not recognized [ItemNotNull] - public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabase db, CancellationToken cancellationToken) + public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabase db, CancellationToken ct) { // The '\xFF/conf/storage_engine' keys has value "0" (ASCII) for ssd engine, and "1" (ASCII) for memory engine - var value = await GetConfigParameterAsync(db, "storage_engine", cancellationToken).ConfigureAwait(false); + var value = await GetConfigParameterAsync(db, "storage_engine", ct).ConfigureAwait(false); if (value.IsNull) throw new InvalidOperationException("Failed to read the storage engine mode from the cluster's system keyspace"); @@ -235,7 +235,7 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas [ItemNotNull] public static async Task> GetBoundaryKeysAsync([NotNull] IFdbReadOnlyTransaction trans, Slice beginInclusive, Slice endExclusive) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); Contract.Assert(trans.Context != null && trans.Context.Database != null); using (var shadow = trans.Context.Database.BeginReadOnlyTransaction(trans.Cancellation)) @@ -254,15 +254,15 @@ public static async Task> GetBoundaryKeysAsync([NotNull] IFdbReadOnl /// Database to use for the operation /// First key (inclusive) of the range to inspect /// End key (exclusive) of the range to inspect - /// Token used to cancel the operation + /// Token used to cancel the operation /// List of keys that mark the start of a new chunk /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned boundaries are an estimate and may not represent the exact boundary locations at any database version. [ItemNotNull] - public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) + public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadAsync((trans) => GetBoundaryKeysInternalAsync(trans, beginInclusive, endExclusive), cancellationToken); + return db.ReadAsync((trans) => GetBoundaryKeysInternalAsync(trans, beginInclusive, endExclusive), ct); } //REVIEW: should we call this chunks? shard? fragments? contigous ranges? @@ -270,32 +270,32 @@ public static Task> GetBoundaryKeysAsync([NotNull] IFdbDatabase db, /// Split a range of keys into smaller chunks where each chunk represents a contiguous range stored on a single server /// Database to use for the operation /// Range of keys to split up into smaller chunks - /// Token used to cancel the operation + /// Token used to cancel the operation /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. [ItemNotNull] - public static Task> GetChunksAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken cancellationToken) + public static Task> GetChunksAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken ct) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? - return GetChunksAsync(db, range.Begin, range.End, cancellationToken); + return GetChunksAsync(db, range.Begin, range.End, ct); } /// Split a range of keys into chunks representing a contiguous range stored on a single server /// Database to use for the operation /// First key (inclusive) of the range to inspect /// End key (exclusive) of the range to inspect - /// Token used to cancel the operation + /// Token used to cancel the operation /// List of one or more chunks that constitutes the range, where each chunk represents a contiguous range stored on a single server. If the list contains a single range, that means that the range is small enough to fit inside a single chunk. /// This method is not transactional. It will return an answer no older than the Database object it is passed, but the returned ranges are an estimate and may not represent the exact boundary locations at any database version. [ItemNotNull] - public static async Task> GetChunksAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken cancellationToken) + public static async Task> GetChunksAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, CancellationToken ct) { //REVIEW: maybe rename this to SplitIntoChunksAsync or SplitIntoShardsAsync or GetFragmentsAsync ? - if (db == null) throw new ArgumentNullException("db"); - if (endExclusive < beginInclusive) throw new ArgumentException("The end key cannot be less than the begin key", "endExclusive"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (endExclusive < beginInclusive) throw new ArgumentException("The end key cannot be less than the begin key", nameof(endExclusive)); - var boundaries = await GetBoundaryKeysAsync(db, beginInclusive, endExclusive, cancellationToken).ConfigureAwait(false); + var boundaries = await GetBoundaryKeysAsync(db, beginInclusive, endExclusive, ct).ConfigureAwait(false); int count = boundaries.Count; var chunks = new List(count + 2); @@ -392,12 +392,12 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF /// Estimate the number of keys in the specified range. /// Database used for the operation /// Range defining the keys to count - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken cancellationToken) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, CancellationToken ct) { - return EstimateCountAsync(db, range.Begin, range.End, null, cancellationToken); + return EstimateCountAsync(db, range.Begin, range.End, null, ct); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! } @@ -405,12 +405,12 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Database used for the operation /// Range defining the keys to count /// Optional callback called everytime the count is updated. The first argument is the current count, and the second argument is the last key that was found. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken cancellationToken) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken ct) { - return EstimateCountAsync(db, range.Begin, range.End, onProgress, cancellationToken); + return EstimateCountAsync(db, range.Begin, range.End, onProgress, ct); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! } @@ -419,19 +419,19 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Key defining the beginning of the range /// Key defining the end of the range /// Optional callback called everytime the count is updated. The first argument is the current count, and the second argument is the last key that was found. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Number of keys k such that <= k > /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress> onProgress, CancellationToken cancellationToken) + public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress> onProgress, CancellationToken ct) { const int INIT_WINDOW_SIZE = 1 << 8; // start at 256 //1024 const int MAX_WINDOW_SIZE = 1 << 13; // never use more than 4096 const int MIN_WINDOW_SIZE = 64; // use range reads when the windows size is smaller than 64 - if (db == null) throw new ArgumentNullException("db"); - if (endExclusive < beginInclusive) throw new ArgumentException("The end key cannot be less than the begin key", "endExclusive"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (endExclusive < beginInclusive) throw new ArgumentException("The end key cannot be less than the begin key", nameof(endExclusive)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); // To count the number of items in the range, we will scan it using a key selector with an offset equal to our window size // > if the returned key is still inside the range, we add the window size to the counter, and start again from the current key @@ -451,7 +451,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli var cursor = beginInclusive.Memoize(); var end = endExclusive.Memoize(); - using (var tr = db.BeginReadOnlyTransaction(cancellationToken)) + using (var tr = db.BeginReadOnlyTransaction(ct)) { #if TRACE_COUNTING tr.Annotate("Estimating number of keys in range {0}", KeyRange.Create(beginInclusive, endExclusive)); diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index fab98017e..2fa06b8fe 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -440,40 +440,40 @@ private static void FailCannotExecuteOnNetworkThread() #region Cluster... /// Opens a connection to an existing FoundationDB cluster using the default cluster file - /// Token used to abort the operation + /// Token used to abort the operation /// Task that will return an FdbCluster, or an exception [ItemNotNull] - public static Task CreateClusterAsync(CancellationToken cancellationToken) + public static Task CreateClusterAsync(CancellationToken ct) { - return CreateClusterAsync(null, cancellationToken); + return CreateClusterAsync(null, ct); } /// Opens a connection to an existing FDB Cluster /// Path to the 'fdb.cluster' file to use, or null for the default cluster file - /// Token used to abort the operation + /// Token used to abort the operation /// Task that will return an FdbCluster, or an exception [ItemNotNull] - public static async Task CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) + public static async Task CreateClusterAsync(string clusterFile, CancellationToken ct) { - return await CreateClusterInternalAsync(clusterFile, cancellationToken).ConfigureAwait(false); + return await CreateClusterInternalAsync(clusterFile, ct).ConfigureAwait(false); } [ItemNotNull] - private static async Task CreateClusterInternalAsync(string clusterFile, CancellationToken cancellationToken) + private static async Task CreateClusterInternalAsync(string clusterFile, CancellationToken ct) { EnsureIsStarted(); // "" should also be considered to mean "default cluster file" if (string.IsNullOrEmpty(clusterFile)) clusterFile = null; - if (Logging.On) Logging.Info(typeof(Fdb), "CreateClusterAsync", clusterFile == null ? "Connecting to default cluster..." : String.Format("Connecting to cluster using '{0}' ...", clusterFile)); + if (Logging.On) Logging.Info(typeof(Fdb), "CreateClusterAsync", clusterFile == null ? "Connecting to default cluster..." : $"Connecting to cluster using '{clusterFile}' ..."); - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); //TODO: check the path ? (exists, readable, ...) //TODO: have a way to configure the default IFdbClusterHander ! - var handler = await FdbNativeCluster.CreateClusterAsync(clusterFile, cancellationToken).ConfigureAwait(false); + var handler = await FdbNativeCluster.CreateClusterAsync(clusterFile, ct).ConfigureAwait(false); return new FdbCluster(handler, clusterFile); } @@ -482,41 +482,41 @@ private static async Task CreateClusterInternalAsync(string clusterF #region Database... /// Create a new connection with the "DB" database on the cluster specified by the default cluster file. - /// Token used to abort the operation + /// Token used to abort the operation /// Task that will return an FdbDatabase, or an exception - /// If the token is cancelled + /// 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. [ItemNotNull] - public static Task OpenAsync(CancellationToken cancellationToken = default(CancellationToken)) + public static Task OpenAsync(CancellationToken ct = default(CancellationToken)) { - return OpenAsync(clusterFile: null, dbName: null, globalSpace: KeySubspace.Empty, cancellationToken: cancellationToken); + return OpenAsync(clusterFile: null, dbName: null, globalSpace: KeySubspace.Empty, ct: ct); } /// Create a new connection with the "DB" database on the cluster specified by the default cluster file, and with the specified global subspace /// Global subspace used as a prefix for all keys and layers - /// Token used to abort the operation + /// Token used to abort the operation /// Task that will return an FdbDatabase, or an exception - /// If the token is cancelled + /// 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. [ItemNotNull] - public static Task OpenAsync(IKeySubspace globalSpace, CancellationToken cancellationToken = default(CancellationToken)) + public static Task OpenAsync(IKeySubspace globalSpace, CancellationToken ct = default(CancellationToken)) { - return OpenAsync(clusterFile: null, dbName: null, globalSpace: globalSpace, cancellationToken: cancellationToken); + return OpenAsync(clusterFile: null, dbName: null, globalSpace: globalSpace, ct: ct); } /// Create a new connection with a database on the specified cluster /// Path to the 'fdb.cluster' file to use, or null for the default cluster file /// Name of the database, or "DB" if not specified. - /// Cancellation Token + /// Cancellation Token /// Task that will return an FdbDatabase, or an exception /// As of 1.0, the only supported database name is "DB" /// If is anything other than "DB" - /// If the token is cancelled + /// 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. [ItemNotNull] - public static Task OpenAsync(string clusterFile, string dbName, CancellationToken cancellationToken = default(CancellationToken)) + public static Task OpenAsync(string clusterFile, string dbName, CancellationToken ct = default(CancellationToken)) { - return OpenAsync(clusterFile, dbName, KeySubspace.Empty, readOnly: false, cancellationToken: cancellationToken); + return OpenAsync(clusterFile, dbName, KeySubspace.Empty, readOnly: false, ct: ct); } /// Create a new connection with a database on the specified cluster @@ -524,23 +524,23 @@ private static async Task CreateClusterInternalAsync(string clusterF /// Name of the database. Must be 'DB' /// Global subspace used as a prefix for all keys and layers /// If true, the database instance will only allow read operations - /// Token used to abort the operation + /// Token used to abort the operation /// Task that will return an FdbDatabase, or an exception /// As of 1.0, the only supported database name is 'DB' /// If is anything other than 'DB' - /// If the token is cancelled + /// 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. [ItemNotNull] - public static async Task OpenAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly = false, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task OpenAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly = false, CancellationToken ct = default(CancellationToken)) { - return await OpenInternalAsync(clusterFile, dbName, globalSpace, readOnly, cancellationToken); + return await OpenInternalAsync(clusterFile, dbName, globalSpace, readOnly, ct); } /// Create a new database handler instance using the specificied cluster file, database name, global subspace and read only settings [ItemNotNull] - internal static async Task OpenInternalAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly, CancellationToken cancellationToken) + internal static async Task OpenInternalAsync(string clusterFile, string dbName, IKeySubspace globalSpace, bool readOnly, CancellationToken ct) { - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); dbName = dbName ?? "DB"; globalSpace = globalSpace ?? KeySubspace.Empty; @@ -552,9 +552,9 @@ internal static async Task OpenInternalAsync(string clusterFile, st bool success = false; try { - cluster = await CreateClusterInternalAsync(clusterFile, cancellationToken).ConfigureAwait(false); + cluster = await CreateClusterInternalAsync(clusterFile, ct).ConfigureAwait(false); //note: since the cluster is not provided by the caller, link it with the database's Dispose() - db = await cluster.OpenDatabaseInternalAsync(dbName, globalSpace, readOnly: readOnly, ownsCluster: true, cancellationToken: cancellationToken).ConfigureAwait(false); + db = await cluster.OpenDatabaseInternalAsync(dbName, globalSpace, readOnly: readOnly, ownsCluster: true, ct: ct).ConfigureAwait(false); success = true; return db; } diff --git a/FoundationDB.Client/FdbCluster.cs b/FoundationDB.Client/FdbCluster.cs index 37f0da3f4..7d609a905 100644 --- a/FoundationDB.Client/FdbCluster.cs +++ b/FoundationDB.Client/FdbCluster.cs @@ -106,16 +106,16 @@ protected virtual void Dispose(bool disposing) /// Name of the database. Must be 'DB' (as of Beta 2) /// Subspace of keys that will be accessed. /// If true, the database will only allow read operations. - /// Cancellation Token (optionnal) for the connect operation + /// Cancellation Token (optionnal) for the connect operation /// Task that will return an FdbDatabase, or an exception /// If is anything other than 'DB' - /// If the token is cancelled + /// If the token is cancelled /// Any attempt to use a key outside the specified subspace will throw an exception [ItemNotNull] - public async Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken cancellationToken) + public async Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken ct) { if (subspace == null) throw new ArgumentNullException("subspace"); - return await OpenDatabaseInternalAsync(databaseName, subspace, readOnly: readOnly, ownsCluster: false, cancellationToken: cancellationToken).ConfigureAwait(false); + return await OpenDatabaseInternalAsync(databaseName, subspace, readOnly: readOnly, ownsCluster: false, ct: ct).ConfigureAwait(false); } /// Opens a database on this cluster @@ -123,13 +123,13 @@ public async Task OpenDatabaseAsync(string databaseName, IKeySubsp /// Subspace of keys that will be accessed. /// If true, the database will only allow read operations. /// If true, the database will dispose this cluster when it is disposed. - /// Cancellation Token + /// Cancellation Token /// Task that will return an FdbDatabase, or an exception /// If is anything other than 'DB' - /// If the token is cancelled + /// If the token is cancelled /// As of Beta2, the only supported database name is 'DB' [ItemNotNull] - internal async Task OpenDatabaseInternalAsync(string databaseName, IKeySubspace subspace, bool readOnly, bool ownsCluster, CancellationToken cancellationToken) + internal async Task OpenDatabaseInternalAsync(string databaseName, IKeySubspace subspace, bool readOnly, bool ownsCluster, CancellationToken ct) { ThrowIfDisposed(); if (string.IsNullOrEmpty(databaseName)) throw new ArgumentNullException("databaseName"); @@ -137,9 +137,9 @@ internal async Task OpenDatabaseInternalAsync(string databaseName, if (Logging.On) Logging.Info(typeof(FdbCluster), "OpenDatabaseAsync", String.Format("Connecting to database '{0}' ...", databaseName)); - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); - var handler = await m_handler.OpenDatabaseAsync(databaseName, cancellationToken).ConfigureAwait(false); + var handler = await m_handler.OpenDatabaseAsync(databaseName, ct).ConfigureAwait(false); if (Logging.On && Logging.IsVerbose) Logging.Verbose(typeof(FdbCluster), "OpenDatabaseAsync", String.Format("Connected to database '{0}'", databaseName)); diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index ada696fd2..6816d6dd7 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -195,7 +195,7 @@ protected virtual FdbDatabasePartition GetRootDirectory() /// Start a new transaction on this database /// Mode of the new transaction (read-only, read-write, ...) - /// Optional cancellation token that can abort all pending async operations started by this transaction. + /// Optional cancellation token that can abort all pending async operations started by this transaction. /// If not null, attach the new transaction to an existing context. /// New transaction instance that can read from or write to the database. /// You MUST call Dispose() on the transaction when you are done with it. You SHOULD wrap it in a 'using' statement to ensure that it is disposed in all cases. @@ -206,10 +206,10 @@ protected virtual FdbDatabasePartition GetRootDirectory() /// tr.Clear(Slice.FromString("OldValue")); /// await tr.CommitAsync(); /// } - public IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken cancellationToken, FdbOperationContext context = null) + public IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken ct, FdbOperationContext context = null) { - cancellationToken.ThrowIfCancellationRequested(); - if (context == null) context = new FdbOperationContext(this, mode, cancellationToken); + ct.ThrowIfCancellationRequested(); + if (context == null) context = new FdbOperationContext(this, mode, ct); return CreateNewTransaction(context); } @@ -305,30 +305,30 @@ internal void UnregisterTransaction(FdbTransaction transaction) /// Runs a transactional lambda function against this database, inside a read-only transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-only transaction on each retry. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadAsync([InstantHandle] Func asyncHandler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task ReadAsync([InstantHandle] Func asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunReadAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunReadAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task ReadAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunReadAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunReadAsync(this, asyncHandler, onDone, ct); } /// Runs a transactional lambda function against this database, inside a read-only transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-only transaction on each retry. The result of the task will also be the result of the transactional. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadAsync(Func> asyncHandler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task ReadAsync(Func> asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task ReadAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, ct); } #endregion @@ -337,65 +337,65 @@ public Task ReadAsync([InstantHandle] FuncRuns a transactional lambda function against this database, inside a write-only transaction context, with retry logic. /// Lambda function that is passed a new read-write transaction on each retry. It should only call non-async methods, such as Set, Clear or any atomic operation. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task WriteAsync([InstantHandle] Action handler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task WriteAsync([InstantHandle] Action handler, CancellationToken ct) { - return FdbOperationContext.RunWriteAsync(this, handler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, null, ct); } /// EXPERIMENTAL - public Task WriteAsync([InstantHandle] Action handler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task WriteAsync([InstantHandle] Action handler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunWriteAsync(this, handler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, onDone, ct); } /// Runs a transactional lambda function against this database, inside a write-only transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-write transaction on each retry. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task WriteAsync([InstantHandle] Func asyncHandler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task WriteAsync([InstantHandle] Func asyncHandler, CancellationToken ct) { //REVIEW: right now, nothing prevents the lambda from calling read methods on the transaction, making this equivalent to calling ReadWriteAsync() // => this version of WriteAsync is only there to catch mistakes when someones passes in an async lambda, instead of an Action //TODO: have a "WriteOnly" mode on transaction to forbid doing any reads ? - return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task WriteAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task WriteAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { //REVIEW: right now, nothing prevents the lambda from calling read methods on the transaction, making this equivalent to calling ReadWriteAsync() // => this version of WriteAsync is only there to catch mistakes when someones passes in an async lambda, instead of an Action //TODO: have a "WriteOnly" mode on transaction to forbid doing any reads ? - return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, ct); } /// Runs a transactional lambda function against this database, inside a read-write transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-write transaction on each retry. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadWriteAsync([InstantHandle] Func asyncHandler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task ReadWriteAsync([InstantHandle] Func asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadWriteAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task ReadWriteAsync([InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, ct); } /// Runs a transactional lambda function against this database, inside a read-write transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-write transaction on each retry. The result of the task will also be the result of the transactional. - /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, CancellationToken cancellationToken) + /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. + public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken) + public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, ct); } #endregion diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index a51fb2bfb..d94f642a4 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -43,7 +43,7 @@ public static class FdbDatabaseExtensions /// Start a new read-only transaction on this database /// Database instance - /// Optional cancellation token that can abort all pending async operations started by this transaction. + /// Optional cancellation token that can abort all pending async operations started by this transaction. /// New transaction instance that can read from the database. /// You MUST call Dispose() on the transaction when you are done with it. You SHOULD wrap it in a 'using' statement to ensure that it is disposed in all cases. /// @@ -56,15 +56,15 @@ public static class FdbDatabaseExtensions /// /// [Pure, NotNull] - public static IFdbReadOnlyTransaction BeginReadOnlyTransaction([NotNull] this IFdbDatabase db, CancellationToken cancellationToken) + public static IFdbReadOnlyTransaction BeginReadOnlyTransaction([NotNull] this IFdbDatabase db, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.BeginTransaction(FdbTransactionMode.ReadOnly, cancellationToken, default(FdbOperationContext)); + return db.BeginTransaction(FdbTransactionMode.ReadOnly, ct, default(FdbOperationContext)); } /// Start a new transaction on this database /// Database instance - /// Optional cancellation token that can abort all pending async operations started by this transaction. + /// Optional cancellation token that can abort all pending async operations started by this transaction. /// New transaction instance that can read from or write to the database. /// You MUST call Dispose() on the transaction when you are done with it. You SHOULD wrap it in a 'using' statement to ensure that it is disposed in all cases. /// @@ -75,10 +75,10 @@ public static IFdbReadOnlyTransaction BeginReadOnlyTransaction([NotNull] this IF /// await tr.CommitAsync(); /// } [Pure, NotNull] - public static IFdbTransaction BeginTransaction([NotNull] this IFdbDatabase db, CancellationToken cancellationToken) + public static IFdbTransaction BeginTransaction([NotNull] this IFdbDatabase db, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.BeginTransaction(FdbTransactionMode.Default, cancellationToken, default(FdbOperationContext)); + return db.BeginTransaction(FdbTransactionMode.Default, ct, default(FdbOperationContext)); } #endregion @@ -212,33 +212,33 @@ public static Slice Extract([NotNull] this IFdbDatabase db, Slice keyAbsolute) /// Read a single key from the database, using a dedicated transaction. /// Database instance /// - /// + /// /// /// /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to read several keys at once, use a version of . /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetAsync([NotNull] this IFdbReadOnlyRetryable db, Slice key, CancellationToken cancellationToken) + public static Task GetAsync([NotNull] this IFdbReadOnlyRetryable db, Slice key, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.ReadAsync((tr) => tr.GetAsync(key), cancellationToken); + return db.ReadAsync((tr) => tr.GetAsync(key), ct); } /// Read a list of keys from the database, using a dedicated transaction. /// Database instance /// - /// + /// /// /// /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] Slice[] keys, CancellationToken cancellationToken) + public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] Slice[] keys, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.ReadAsync((tr) => tr.GetValuesAsync(keys), cancellationToken); + return db.ReadAsync((tr) => tr.GetValuesAsync(keys), ct); } /// Read a sequence of keys from the database, using a dedicated transaction. @@ -248,10 +248,10 @@ public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keys, CancellationToken cancellationToken) + public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keys, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.ReadAsync((tr) => tr.GetValuesAsync(keys), cancellationToken); + return db.ReadAsync((tr) => tr.GetValuesAsync(keys), ct); } /// Resolve a single key selector from the database, using a dedicated transaction. @@ -260,10 +260,10 @@ public static Task GetValuesAsync([NotNull] this IFdbReadOnlyRetryable /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector keySelector, CancellationToken cancellationToken) + public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector keySelector, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.ReadAsync((tr) => tr.GetKeyAsync(keySelector), cancellationToken); + return db.ReadAsync((tr) => tr.GetKeyAsync(keySelector), ct); } /// Resolve a list of key selectors from the database, using a dedicated transaction. @@ -273,11 +273,11 @@ public static Task GetKeyAsync([NotNull] this IFdbReadOnlyRetryable db, K /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] KeySelector[] keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] KeySelector[] keySelectors, CancellationToken ct) { Contract.NotNull(db, nameof(db)); Contract.NotNull(keySelectors, nameof(keySelectors)); - return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), cancellationToken); + return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), ct); } /// Resolve a sequence of key selectors from the database, using a dedicated transaction. @@ -287,11 +287,11 @@ public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// [ItemNotNull] - public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken cancellationToken) + public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull] IEnumerable keySelectors, CancellationToken ct) { Contract.NotNull(db, nameof(db)); Contract.NotNull(keySelectors, nameof(keySelectors)); - return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), cancellationToken); + return db.ReadAsync((tr) => tr.GetKeysAsync(keySelectors), ct); } /// Read a single page of a range query from the database, using a dedicated transaction. @@ -300,10 +300,10 @@ public static Task GetKeysAsync([NotNull] this IFdbReadOnlyRetryable db /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken cancellationToken) + public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetryable db, KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options, int iteration, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.ReadAsync((tr) => tr.GetRangeAsync(beginInclusive, endExclusive, options, iteration), cancellationToken); + return db.ReadAsync((tr) => tr.GetRangeAsync(beginInclusive, endExclusive, options, iteration), ct); } /// Set the value of a single key in the database, using a dedicated transaction. @@ -312,10 +312,10 @@ public static Task GetRangeAsync([NotNull] this IFdbReadOnlyRetry /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetAsync([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task SetAsync([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Set(key, value), cancellationToken); + return db.WriteAsync((tr) => tr.Set(key, value), ct); } /// Set the values of a list of keys in the database, using a dedicated transaction. @@ -324,7 +324,7 @@ public static Task SetAsync([NotNull] this IFdbRetryable db, Slice key, Slice va /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair[] keyValuePairs, CancellationToken cancellationToken) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair[] keyValuePairs, CancellationToken ct) { Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => @@ -333,7 +333,7 @@ public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair< { tr.Set(kv.Key, kv.Value); } - }, cancellationToken); + }, ct); } /// Set the values of a sequence of keys in the database, using a dedicated transaction. @@ -342,7 +342,7 @@ public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair< /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable> keyValuePairs, CancellationToken cancellationToken) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable> keyValuePairs, CancellationToken ct) { Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => @@ -351,7 +351,7 @@ public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerableClear a single key in the database, using a dedicated transaction. @@ -360,10 +360,10 @@ public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable or overrides. /// - public static Task ClearAsync([NotNull] this IFdbRetryable db, Slice key, CancellationToken cancellationToken) + public static Task ClearAsync([NotNull] this IFdbRetryable db, Slice key, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Clear(key), cancellationToken); + return db.WriteAsync((tr) => tr.Clear(key), ct); } /// Clear a single range in the database, using a dedicated transaction. @@ -371,10 +371,10 @@ public static Task ClearAsync([NotNull] this IFdbRetryable db, Slice key, Cancel /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, Slice beginKeyInclusive, Slice endKeyExclusive, CancellationToken cancellationToken) + public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, Slice beginKeyInclusive, Slice endKeyExclusive, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.ClearRange(beginKeyInclusive, endKeyExclusive), cancellationToken); + return db.WriteAsync((tr) => tr.ClearRange(beginKeyInclusive, endKeyExclusive), ct); } /// Clear a single range in the database, using a dedicated transaction. @@ -382,26 +382,26 @@ public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, Slice beginK /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, KeyRange range, CancellationToken cancellationToken) + public static Task ClearRangeAsync([NotNull] this IFdbRetryable db, KeyRange range, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.ClearRange(range), cancellationToken); + return db.WriteAsync((tr) => tr.ClearRange(range), ct); } /// Atomically add to the value of a single key in the database, using a dedicated transaction. /// /// /// - /// + /// /// /// /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicAdd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicAdd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Add), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Add), ct); } /// Atomically perform a bitwise AND to the value of a single key in the database, using a dedicated transaction. @@ -409,10 +409,10 @@ public static Task AtomicAdd([NotNull] this IFdbRetryable db, Slice key, Slice v /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitAnd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitAnd([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitAnd), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitAnd), ct); } /// Atomically perform a bitwise OR to the value of a single key in the database, using a dedicated transaction. @@ -420,10 +420,10 @@ public static Task AtomicBitAnd([NotNull] this IFdbRetryable db, Slice key, Slic /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitOr([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitOr([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitOr), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitOr), ct); } /// Atomically perform a bitwise XOR to the value of a single key in the database, using a dedicated transaction. @@ -431,10 +431,10 @@ public static Task AtomicBitOr([NotNull] this IFdbRetryable db, Slice key, Slice /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicBitXor([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicBitXor([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitXor), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.BitXor), ct); } /// Atomically update a value if it is larger than the value in the database, using a dedicated transaction. @@ -442,10 +442,10 @@ public static Task AtomicBitXor([NotNull] this IFdbRetryable db, Slice key, Slic /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicMax([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicMax([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Max), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Max), ct); } /// Atomically update a value if it is smaller than the value in the database, using a dedicated transaction. @@ -453,10 +453,10 @@ public static Task AtomicMax([NotNull] this IFdbRetryable db, Slice key, Slice v /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task AtomicMin([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static Task AtomicMin([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); - return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Min), cancellationToken); + return db.WriteAsync((tr) => tr.Atomic(key, value, FdbMutationType.Min), ct); } #endregion @@ -466,28 +466,28 @@ public static Task AtomicMin([NotNull] this IFdbRetryable db, Slice key, Slice v /// Reads the value associated with , and returns a Watch that will complete after a subsequent change to key in the database. /// Database instance. /// Key to be looked up in the database - /// Token that can be used to cancel the Watch from the outside. + /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property contains the current value of the key. - public static Task GetAndWatch([NotNull] this IFdbRetryable db, Slice key, CancellationToken cancellationToken) + public static Task GetAndWatch([NotNull] this IFdbRetryable db, Slice key, CancellationToken ct) { Contract.NotNull(db, nameof(db)); return db.ReadWriteAsync(async (tr) => { var result = await tr.GetAsync(key).ConfigureAwait(false); - var watch = tr.Watch(key, cancellationToken); + var watch = tr.Watch(key, ct); watch.Value = result.Memoize(); return watch; - }, cancellationToken); + }, ct); } /// Sets to and returns a Watch that will complete after a subsequent change to the key in the database. /// Database instance. /// Name of the key to be inserted into the database. /// Value to be inserted into the database. - /// Token that can be used to cancel the Watch from the outside. + /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static async Task SetAndWatch([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken cancellationToken) + public static async Task SetAndWatch([NotNull] this IFdbRetryable db, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(db, nameof(db)); @@ -496,8 +496,8 @@ public static async Task SetAndWatch([NotNull] this IFdbRetryable db, await db.WriteAsync((tr) => { tr.Set(key, value); - watch = tr.Watch(key, cancellationToken); - }, cancellationToken).ConfigureAwait(false); + watch = tr.Watch(key, ct); + }, ct).ConfigureAwait(false); watch.Value = value.Memoize(); return watch; diff --git a/FoundationDB.Client/FdbExceptIterator.cs b/FoundationDB.Client/FdbExceptIterator.cs index 77e280671..53e916f5b 100644 --- a/FoundationDB.Client/FdbExceptIterator.cs +++ b/FoundationDB.Client/FdbExceptIterator.cs @@ -48,7 +48,7 @@ protected override FdbAsyncIterator Clone() return new FdbExceptIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken cancellationToken, out int index, out TSource current) + protected override bool FindNext(CancellationToken ct, out int index, out TSource current) { index = -1; current = default(TSource); @@ -83,7 +83,7 @@ protected override bool FindNext(CancellationToken cancellationToken, out int in { output = false; if (cmp == 0) discard = true; - AdvanceIterator(i, cancellationToken); + AdvanceIterator(i, ct); } } @@ -95,7 +95,7 @@ protected override bool FindNext(CancellationToken cancellationToken, out int in if (output || discard) { - AdvanceIterator(0, cancellationToken); + AdvanceIterator(0, ct); } return true; diff --git a/FoundationDB.Client/FdbIntersectIterator.cs b/FoundationDB.Client/FdbIntersectIterator.cs index f7f3445ba..85c260bbc 100644 --- a/FoundationDB.Client/FdbIntersectIterator.cs +++ b/FoundationDB.Client/FdbIntersectIterator.cs @@ -48,7 +48,7 @@ protected override FdbAsyncIterator Clone() return new FdbIntersectIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken cancellationToken, out int index, out TSource current) + protected override bool FindNext(CancellationToken ct, out int index, out TSource current) { index = -1; current = default(TSource); @@ -92,7 +92,7 @@ protected override bool FindNext(CancellationToken cancellationToken, out int in // advance everyone ! for (int i = 0; i < m_iterators.Length;i++) { - if (m_iterators[i].Active) AdvanceIterator(i, cancellationToken); + if (m_iterators[i].Active) AdvanceIterator(i, ct); } return true; } @@ -102,7 +102,7 @@ protected override bool FindNext(CancellationToken cancellationToken, out int in { if (m_iterators[i].Active && m_keyComparer.Compare(m_iterators[i].Current, max) < 0) { - AdvanceIterator(i, cancellationToken); + AdvanceIterator(i, ct); } } diff --git a/FoundationDB.Client/FdbMergeSortIterator.cs b/FoundationDB.Client/FdbMergeSortIterator.cs index 87b3555a4..05ca95465 100644 --- a/FoundationDB.Client/FdbMergeSortIterator.cs +++ b/FoundationDB.Client/FdbMergeSortIterator.cs @@ -49,7 +49,7 @@ protected override FdbAsyncIterator Clone() return new FdbMergeSortIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken cancellationToken, out int index, out TSource current) + protected override bool FindNext(CancellationToken ct, out int index, out TSource current) { index = -1; current = default(TSource); @@ -71,7 +71,7 @@ protected override bool FindNext(CancellationToken cancellationToken, out int in current = m_iterators[index].Iterator.Current; if (m_remaining == null || m_remaining.Value > 1) { // start getting the next value on this iterator - AdvanceIterator(index, cancellationToken); + AdvanceIterator(index, ct); } } diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index 3d1d6979d..ad3a208d3 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -94,10 +94,10 @@ public sealed class FdbOperationContext : IDisposable /// Create a new retry loop operation context /// Database that will be used by the retry loop /// Operation mode of the retry loop - /// Optional cancellation token that will abort the retry loop if triggered. - public FdbOperationContext([NotNull] IFdbDatabase db, FdbTransactionMode mode, CancellationToken cancellationToken) + /// Optional cancellation token that will abort the retry loop if triggered. + public FdbOperationContext([NotNull] IFdbDatabase db, FdbTransactionMode mode, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); this.Database = db; this.Mode = mode; @@ -107,9 +107,9 @@ public FdbOperationContext([NotNull] IFdbDatabase db, FdbTransactionMode mode, C // by default, we hook ourselves to the db's CancellationToken, but we may need to also // hook with a different, caller-provided, token and respond to cancellation from both sites. var token = db.Cancellation; - if (cancellationToken.CanBeCanceled && !cancellationToken.Equals(token)) + if (ct.CanBeCanceled && !ct.Equals(token)) { - this.TokenSource = CancellationTokenSource.CreateLinkedTokenSource(token, cancellationToken); + this.TokenSource = CancellationTokenSource.CreateLinkedTokenSource(token, ct); token = this.TokenSource.Token; } this.Cancellation = token; @@ -248,22 +248,22 @@ public void Dispose() #region Read-Only operations... /// Run a read-only operation until it suceeds, timeouts, or fail with non-retryable error - public static Task RunReadAsync([NotNull] IFdbDatabase db, [NotNull] Func asyncHandler, Action onDone, CancellationToken cancellationToken) + public static Task RunReadAsync([NotNull] IFdbDatabase db, [NotNull] Func asyncHandler, Action onDone, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); - var context = new FdbOperationContext(db, FdbTransactionMode.ReadOnly | FdbTransactionMode.InsideRetryLoop, cancellationToken); + var context = new FdbOperationContext(db, FdbTransactionMode.ReadOnly | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, asyncHandler, onDone); } /// Run a read-only operation until it suceeds, timeouts, or fail with non-retryable error - public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken cancellationToken) + public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); - cancellationToken.ThrowIfCancellationRequested(); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); + ct.ThrowIfCancellationRequested(); R result = default(R); Func handler = async (tr) => @@ -271,7 +271,7 @@ public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, result = await asyncHandler(tr).ConfigureAwait(false); }; - var context = new FdbOperationContext(db, FdbTransactionMode.ReadOnly | FdbTransactionMode.InsideRetryLoop, cancellationToken); + var context = new FdbOperationContext(db, FdbTransactionMode.ReadOnly | FdbTransactionMode.InsideRetryLoop, ct); await ExecuteInternal(db, context, handler, onDone).ConfigureAwait(false); return result; } @@ -281,40 +281,40 @@ public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, #region Read/Write operations... /// Run a read/write operation until it suceeds, timeouts, or fail with non-retryable error - public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Func asyncHandler, Action onDone, CancellationToken cancellationToken) + public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Func asyncHandler, Action onDone, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); - var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, cancellationToken); + var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, asyncHandler, onDone); } /// Run a write operation until it suceeds, timeouts, or fail with non-retryable error - public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Action handler, Action onDone, CancellationToken cancellationToken) + public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Action handler, Action onDone, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); - var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, cancellationToken); + var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, handler, onDone); } /// Run a read/write operation until it suceeds, timeouts, or fail with non-retryable error - public static async Task RunWriteWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken cancellationToken) + public static async Task RunWriteWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); - cancellationToken.ThrowIfCancellationRequested(); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); + ct.ThrowIfCancellationRequested(); Func handler = async (tr) => { tr.Context.Result = await asyncHandler(tr).ConfigureAwait(false); }; - var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, cancellationToken); + var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); await ExecuteInternal(db, context, handler, onDone).ConfigureAwait(false); return (R)context.Result; } diff --git a/FoundationDB.Client/FdbQueryMergeIterator.cs b/FoundationDB.Client/FdbQueryMergeIterator.cs index 5eaed639e..661b232c7 100644 --- a/FoundationDB.Client/FdbQueryMergeIterator.cs +++ b/FoundationDB.Client/FdbQueryMergeIterator.cs @@ -78,7 +78,7 @@ protected FdbQueryMergeIterator(IEnumerable> source m_resultSelector = resultSelector; } - protected override Task OnFirstAsync(CancellationToken cancellationToken) + protected override Task OnFirstAsync(CancellationToken ct) { if (m_remaining != null && m_remaining.Value < 0) { // empty list ?? @@ -99,7 +99,7 @@ protected override Task OnFirstAsync(CancellationToken cancellationToken) var state = new IteratorState(); state.Active = true; state.Iterator = sources[i].GetEnumerator(mode); - state.Next = state.Iterator.MoveNextAsync(cancellationToken); + state.Next = state.Iterator.MoveNextAsync(ct); iterators[i] = state; } @@ -122,7 +122,7 @@ protected override Task OnFirstAsync(CancellationToken cancellationToken) } /// Finds the next smallest item from all the active iterators - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { if (m_remaining != null && m_remaining.Value <= 0) { @@ -155,7 +155,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } // find the next value to advance - if (!FindNext(cancellationToken, out index, out current)) + if (!FindNext(ct, out index, out current)) { // nothing left anymore ? return Completed(); } @@ -179,13 +179,13 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo return true; } - protected abstract bool FindNext(CancellationToken cancellationToken, out int index, out TSource current); + protected abstract bool FindNext(CancellationToken ct, out int index, out TSource current); - protected void AdvanceIterator(int index, CancellationToken cancellationToken) + protected void AdvanceIterator(int index, CancellationToken ct) { m_iterators[index].HasCurrent = false; m_iterators[index].Current = default(TKey); - m_iterators[index].Next = m_iterators[index].Iterator.MoveNextAsync(cancellationToken); + m_iterators[index].Next = m_iterators[index].Iterator.MoveNextAsync(ct); } private static void Cleanup(IteratorState[] iterators) diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 2dbb8da9a..92ac13c6b 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -107,7 +107,7 @@ protected override FdbAsyncIterator[]> Clone() #region IFdbAsyncEnumerator... - protected override async Task OnFirstAsync(CancellationToken cancellationToken) + protected override async Task OnFirstAsync(CancellationToken ct) { this.RemainingCount = this.Query.Limit; this.RemainingSize = this.Query.TargetBytes; @@ -141,7 +141,7 @@ protected override async Task OnFirstAsync(CancellationToken cancellationT return true; } - protected override Task OnNextAsync(CancellationToken cancellationToken) + protected override Task OnNextAsync(CancellationToken ct) { // Make sure that we are not called while the previous fetch is still running if (this.PendingReadTask != null && !this.PendingReadTask.IsCompleted) @@ -155,18 +155,17 @@ protected override Task OnNextAsync(CancellationToken cancellationToken) } // slower path, we need to actually read the first batch... - return FetchNextPageAsync(cancellationToken); + return FetchNextPageAsync(ct); } /// Asynchronously fetch a new page of results - /// /// True if Chunk contains a new page of results. False if all results have been read. - private Task FetchNextPageAsync(CancellationToken cancellationToken) + private Task FetchNextPageAsync(CancellationToken ct) { Contract.Requires(!this.AtEnd); Contract.Requires(this.Iteration >= 0); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); this.Transaction.EnsureCanRead(); this.Iteration++; diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 415ce3dcb..eaa84ed29 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -87,7 +87,7 @@ protected override FdbAsyncIterator Clone() return new ResultIterator(m_query, m_transaction, m_resultTransform); } - protected override Task OnFirstAsync(CancellationToken cancellationToken) + protected override Task OnFirstAsync(CancellationToken ct) { // on first call, setup the page iterator if (m_chunkIterator == null) @@ -97,7 +97,7 @@ protected override Task OnFirstAsync(CancellationToken cancellationToken) return TaskHelpers.TrueTask; } - protected override Task OnNextAsync(CancellationToken cancellationToken) + protected override Task OnNextAsync(CancellationToken ct) { if (m_itemsRemainingInChunk > 0) { // we need can get another one from the batch @@ -116,17 +116,17 @@ protected override Task OnNextAsync(CancellationToken cancellationToken) // slower path, we need to actually read the first batch... m_chunk = null; m_currentOffsetInChunk = -1; - return ReadAnotherBatchAsync(cancellationToken); + return ReadAnotherBatchAsync(ct); } - private async Task ReadAnotherBatchAsync(CancellationToken cancellationToken) + private async Task ReadAnotherBatchAsync(CancellationToken ct) { Contract.Requires(m_itemsRemainingInChunk == 0 && m_currentOffsetInChunk == -1 && !m_outOfChunks); var iterator = m_chunkIterator; // start reading the next batch - if (await iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // we got a new chunk ! //note: Dispose() or Cleanup() maybe have been called concurrently! diff --git a/FoundationDB.Client/FdbTransaction.Snapshot.cs b/FoundationDB.Client/FdbTransaction.Snapshot.cs index d114e6459..dcdbc89f9 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, ct: m_parent.m_cancellation); } 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, ct: m_parent.m_cancellation); } public async Task GetKeyAsync(KeySelector selector) @@ -147,7 +147,7 @@ public async Task GetKeyAsync(KeySelector 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, ct: m_parent.m_cancellation).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(KeySelector[] 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, ct: m_parent.m_cancellation); } public Task GetRangeAsync(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options, int iteration) @@ -183,7 +183,7 @@ public Task GetRangeAsync(KeySelector beginInclusive, KeySelector // 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, ct: m_parent.m_cancellation); } public FdbRangeQuery> GetRange(KeySelector beginInclusive, KeySelector endExclusive, FdbRangeOptions options) @@ -196,7 +196,7 @@ public FdbRangeQuery> GetRange(KeySelector beginInclu 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, ct: m_parent.m_cancellation); } void IFdbReadOnlyTransaction.Cancel() diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index e0fa7d1df..d9f4dcbd8 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -311,7 +311,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, ct: m_cancellation); } #endregion @@ -334,7 +334,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, ct: m_cancellation); } #endregion @@ -364,7 +364,7 @@ public Task GetRangeAsync(KeySelector beginInclusive, KeySelector // 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, ct: m_cancellation); } #endregion @@ -417,7 +417,7 @@ public async Task GetKeyAsync(KeySelector 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, ct: m_cancellation).ConfigureAwait(false); // don't forget to truncate keys that would fall outside of the database's globalspace ! return m_database.BoundCheck(key); @@ -445,7 +445,7 @@ public Task GetKeysAsync(KeySelector[] 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, ct: m_cancellation); } #endregion @@ -635,7 +635,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, ct: m_cancellation); } #endregion @@ -683,12 +683,12 @@ public async Task CommitAsync() /// Watch a key for any change in the database. /// /// Key to watch - /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance + /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance /// FdbWatch that can be awaited and will complete when the key has changed in the database, or cancellation occurs. You can call Cancel() at any time if you are not interested in watching the key anymore. You MUST always call Dispose() if the watch completes or is cancelled, to ensure that resources are released properly. - /// You can directly await an FdbWatch, or obtain a Task<Slice> by reading the property - public FdbWatch Watch(Slice key, CancellationToken cancellationToken) + /// You can directly await an FdbWatch, or obtain a Task<Slice> by reading the property + public FdbWatch Watch(Slice key, CancellationToken ct) { - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); EnsureCanRead(); m_database.EnsureKeyIsValid(ref key); @@ -699,14 +699,14 @@ public FdbWatch Watch(Slice key, CancellationToken cancellationToken) key = key.Memoize(); #if DEBUG - if (Logging.On) Logging.Verbose(this, "WatchAsync", String.Format("Watching key '{0}'", key.ToString())); + if (Logging.On) Logging.Verbose(this, "WatchAsync", $"Watching key '{key.ToString()}'"); #endif // Note: the FDBFuture returned by 'fdb_transaction_watch()' outlives the transaction, and can only be cancelled with 'fdb_future_cancel()' or 'fdb_future_destroy()' // Since Task does not expose any cancellation mechanism by itself (and we don't want to force the caller to create a CancellationTokenSource everytime), // we will return the FdbWatch that wraps the FdbFuture directly, since it knows how to cancel itself. - return m_handler.Watch(key, cancellationToken); + return m_handler.Watch(key, ct); } #endregion @@ -725,7 +725,7 @@ public async Task OnErrorAsync(FdbError code) { EnsureCanRetry(); - await m_handler.OnErrorAsync(code, cancellationToken: m_cancellation).ConfigureAwait(false); + await m_handler.OnErrorAsync(code, ct: m_cancellation).ConfigureAwait(false); // If fdb_transaction_on_error succeeds, that means that the transaction has been reset and is usable again var state = this.State; diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index c668661de..6c3e245d8 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -626,15 +626,15 @@ public static void AddWriteConflictKey([NotNull] this IFdbTransaction trans, Sli /// Reads the value associated with , and returns a Watch that will complete after a subsequent change to key in the database. /// Transaction to use for the operation /// Key to be looked up in the database - /// Token that can be used to cancel the Watch from the outside. + /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property contains the current value of the key. - public static async Task GetAndWatchAsync([NotNull] this IFdbTransaction trans, Slice key, CancellationToken cancellationToken) + public static async Task GetAndWatchAsync([NotNull] this IFdbTransaction trans, Slice key, CancellationToken ct) { Contract.NotNull(trans, nameof(trans)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); var value = await trans.GetAsync(key); - var watch = trans.Watch(key, cancellationToken); + var watch = trans.Watch(key, ct); watch.Value = value; return watch; @@ -644,15 +644,15 @@ public static async Task GetAndWatchAsync([NotNull] this IFdbTransacti /// Transaction to use for the operation /// Name of the key to be inserted into the database. /// Value to be inserted into the database. - /// Token that can be used to cancel the Watch from the outside. + /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, Slice value, CancellationToken cancellationToken) + public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, Slice value, CancellationToken ct) { Contract.NotNull(trans, nameof(trans)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); trans.Set(key, value); - var watch = trans.Watch(key, cancellationToken); + var watch = trans.Watch(key, ct); watch.Value = value; return watch; @@ -664,13 +664,13 @@ public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice k /// Name of the key to be inserted into the database. /// Value to be inserted into the database. /// Encoder use to convert into a slice - /// Token that can be used to cancel the Watch from the outside. + /// Token that can be used to cancel the Watch from the outside. /// A new Watch that will track any changes to in the database, and whose Value property will be a copy of argument - public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder, CancellationToken cancellationToken) + public static FdbWatch SetAndWatch([NotNull] this IFdbTransaction trans, Slice key, TValue value, [NotNull] IValueEncoder encoder, CancellationToken ct) { Contract.NotNull(encoder, nameof(encoder)); - cancellationToken.ThrowIfCancellationRequested(); - return SetAndWatch(trans, key, encoder.EncodeValue(value), cancellationToken); + ct.ThrowIfCancellationRequested(); + return SetAndWatch(trans, key, encoder.EncodeValue(value), ct); } #endregion @@ -819,10 +819,10 @@ public static async Task[]> GetBatchAsync([N /// Runs a query inside a read-only transaction context, with retry-logic. /// Database used for the operation /// Lambda function that returns an async enumerable. The function may be called multiple times if the transaction conflicts. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Task returning the list of all the elements of the async enumerable returned by the last successfull call to . [ItemNotNull] - public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken cancellationToken) + public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken ct) { Contract.NotNull(db, nameof(db)); Contract.NotNull(handler, nameof(handler)); @@ -834,7 +834,7 @@ public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable d if (query == null) throw new InvalidOperationException("The query handler returned a null sequence"); return query.ToListAsync(); }, - cancellationToken + ct ); } diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index 76510d82e..b1935d73c 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -209,7 +209,7 @@ public virtual IDynamicKeyEncoder Encoder #region Transactionals... - public virtual IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken cancellationToken = default(CancellationToken), FdbOperationContext context = null) + public virtual IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken ct = default(CancellationToken), FdbOperationContext context = null) { ThrowIfDisposed(); @@ -218,82 +218,82 @@ public virtual IDynamicKeyEncoder Encoder if (context == null) { - context = new FdbOperationContext(this, mode, cancellationToken); + context = new FdbOperationContext(this, mode, ct); } - return m_database.BeginTransaction(mode, cancellationToken, context); + return m_database.BeginTransaction(mode, ct, context); } - public Task ReadAsync(Func asyncHandler, CancellationToken cancellationToken) + public Task ReadAsync(Func asyncHandler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunReadAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunReadAsync(this, asyncHandler, null, ct); } - public Task ReadAsync(Func asyncHandler, Action onDone, CancellationToken cancellationToken) + public Task ReadAsync(Func asyncHandler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunReadAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunReadAsync(this, asyncHandler, onDone, ct); } - public Task ReadAsync(Func> asyncHandler, CancellationToken cancellationToken) + public Task ReadAsync(Func> asyncHandler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, ct); } - public Task ReadAsync(Func> asyncHandler, Action onDone, CancellationToken cancellationToken) + public Task ReadAsync(Func> asyncHandler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, ct); } - public Task WriteAsync(Action handler, CancellationToken cancellationToken) + public Task WriteAsync(Action handler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, handler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, null, ct); } - public Task WriteAsync(Action handler, Action onDone, CancellationToken cancellationToken) + public Task WriteAsync(Action handler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, handler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, onDone, ct); } - public Task WriteAsync(Func handler, CancellationToken cancellationToken) + public Task WriteAsync(Func handler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, handler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, null, ct); } - public Task WriteAsync(Func handler, Action onDone, CancellationToken cancellationToken) + public Task WriteAsync(Func handler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, handler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, handler, onDone, ct); } - public Task ReadWriteAsync(Func asyncHandler, CancellationToken cancellationToken) + public Task ReadWriteAsync(Func asyncHandler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, null, ct); } - public Task ReadWriteAsync(Func asyncHandler, Action onDone, CancellationToken cancellationToken) + public Task ReadWriteAsync(Func asyncHandler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunWriteAsync(this, asyncHandler, onDone, ct); } - public Task ReadWriteAsync(Func> asyncHandler, CancellationToken cancellationToken) + public Task ReadWriteAsync(Func> asyncHandler, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, cancellationToken); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, ct); } - public Task ReadWriteAsync(Func> asyncHandler, Action onDone, CancellationToken cancellationToken) + public Task ReadWriteAsync(Func> asyncHandler, Action onDone, CancellationToken ct) { ThrowIfDisposed(); - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, cancellationToken); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, ct); } #endregion diff --git a/FoundationDB.Client/Filters/FdbTransactionFilter.cs b/FoundationDB.Client/Filters/FdbTransactionFilter.cs index 6f6567a7f..688f3e6e9 100644 --- a/FoundationDB.Client/Filters/FdbTransactionFilter.cs +++ b/FoundationDB.Client/Filters/FdbTransactionFilter.cs @@ -263,10 +263,10 @@ public virtual Task OnErrorAsync(FdbError code) return m_transaction.OnErrorAsync(code); } - public virtual FdbWatch Watch(Slice key, CancellationToken cancellationToken) + public virtual FdbWatch Watch(Slice key, CancellationToken ct) { ThrowIfDisposed(); - return m_transaction.Watch(key, cancellationToken); + return m_transaction.Watch(key, ct); } public virtual void SetOption(FdbTransactionOption option) diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs index 87d0d516a..bf413d693 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs @@ -46,6 +46,7 @@ public sealed class FdbLoggedDatabase : FdbDatabaseFilter /// If true, deny all write operations. /// If true, also dispose the wrapped database if this instance is disposed. /// Handler that will be called when a transaction is either committed succesfully, or disposed. The log can be accessed via the property. + /// public FdbLoggedDatabase(IFdbDatabase database, bool forceReadOnly, bool ownsDatabase, Action onCommitted, FdbLoggingOptions defaultOptions = FdbLoggingOptions.Default) : base(database, forceReadOnly, ownsDatabase) { @@ -54,10 +55,10 @@ public FdbLoggedDatabase(IFdbDatabase database, bool forceReadOnly, bool ownsDat } /// Create a new logged transaction - public override IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken cancellationToken = default(CancellationToken), FdbOperationContext context = null) + public override IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken ct = default(CancellationToken), FdbOperationContext context = null) { return new FdbLoggedTransaction( - base.BeginTransaction(mode, cancellationToken, context), + base.BeginTransaction(mode, ct, context), true, this.OnCommitted, this.LoggingOptions diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index b0c127c74..0415b5c72 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -274,11 +274,11 @@ public override void Reset() ); } - public override FdbWatch Watch(Slice key, CancellationToken cancellationToken) + public override FdbWatch Watch(Slice key, CancellationToken ct) { var cmd = new FdbTransactionLog.WatchCommand(Grab(key)); this.Log.AddOperation(cmd); - return m_transaction.Watch(cmd.Key, cancellationToken); + return m_transaction.Watch(cmd.Key, ct); } #endregion diff --git a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs index 0feb35de1..a658b67d9 100644 --- a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs +++ b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs @@ -165,10 +165,10 @@ public override void AddConflictRange(Slice beginKeyInclusive, Slice endKeyExclu base.AddConflictRange(Encode(beginKeyInclusive), Encode(endKeyExclusive), type); } - public override FdbWatch Watch(Slice key, CancellationToken cancellationToken) + public override FdbWatch Watch(Slice key, CancellationToken ct) { //BUGBUG: the watch returns the key, that would need to be decoded! - return base.Watch(Encode(key), cancellationToken); + return base.Watch(Encode(key), ct); } } diff --git a/FoundationDB.Client/IFdbCluster.cs b/FoundationDB.Client/IFdbCluster.cs index 02b580cf5..4a9db7589 100644 --- a/FoundationDB.Client/IFdbCluster.cs +++ b/FoundationDB.Client/IFdbCluster.cs @@ -58,10 +58,10 @@ public interface IFdbCluster : IDisposable /// Name of the database. Must be 'DB' (as of Beta 2) /// Subspace of keys that will be accessed. /// If true, the database will only allow read operations. - /// Cancellation Token (optionnal) for the connect operation + /// Cancellation Token (optionnal) for the connect operation /// Task that will return an FdbDatabase, or an exception [ItemNotNull] - Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken cancellationToken); + Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken ct); } } diff --git a/FoundationDB.Client/IFdbDatabase.cs b/FoundationDB.Client/IFdbDatabase.cs index 69471c02b..d2d3ba190 100644 --- a/FoundationDB.Client/IFdbDatabase.cs +++ b/FoundationDB.Client/IFdbDatabase.cs @@ -82,7 +82,7 @@ public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IDynamicKe /// Start a new transaction on this database, with the specified mode /// Mode of the transaction (read-only, read-write, ....) - /// Optional cancellation token that can abort all pending async operations started by this transaction. + /// Optional cancellation token that can abort all pending async operations started by this transaction. /// Existing parent context, if the transaction needs to be linked with a retry loop, or a parent transaction. If null, will create a new standalone context valid only for this transaction /// New transaction instance that can read from or write to the database. /// You MUST call Dispose() on the transaction when you are done with it. You SHOULD wrap it in a 'using' statement to ensure that it is disposed in all cases. @@ -94,7 +94,7 @@ public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IDynamicKe /// await tr.CommitAsync(); /// } [NotNull] - IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken cancellationToken, FdbOperationContext context = null); + IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationToken ct, FdbOperationContext context = null); } diff --git a/FoundationDB.Client/IFdbReadOnlyRetryable.cs b/FoundationDB.Client/IFdbReadOnlyRetryable.cs index fc5b332f5..32e1be41e 100644 --- a/FoundationDB.Client/IFdbReadOnlyRetryable.cs +++ b/FoundationDB.Client/IFdbReadOnlyRetryable.cs @@ -65,29 +65,29 @@ public interface IFdbReadOnlyRetryable /// Runs a transactional lambda function inside a read-only transaction, which can be executed more than once if any retryable error occurs. /// Asynchronous handler that will be retried until it succeeds, or a non-recoverable error occurs. - /// Token used to cancel the operation + /// Token used to cancel the operation /// /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task ReadAsync([NotNull, InstantHandle] Func asyncHandler, CancellationToken cancellationToken); + Task ReadAsync([NotNull, InstantHandle] Func asyncHandler, CancellationToken ct); /// Runs a transactional lambda function inside a read-only transaction, which can be executed more than once if any retryable error occurs. /// Asynchronous handler that will be retried until it succeeds, or a non-recoverable error occurs. - /// Token used to cancel the operation + /// Token used to cancel the operation /// /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task ReadAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken cancellationToken); + Task ReadAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken ct); //REVIEW: should we keep these ? /// [EXPERIMENTAL] do not use yet!. - Task ReadAsync([NotNull, InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken); + Task ReadAsync([NotNull, InstantHandle] Func asyncHandler, [InstantHandle] Action onDone, CancellationToken ct); /// [EXPERIMENTAL] do not use yet!. - Task ReadAsync([NotNull, InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken cancellationToken); + Task ReadAsync([NotNull, InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct); } diff --git a/FoundationDB.Client/IFdbRetryable.cs b/FoundationDB.Client/IFdbRetryable.cs index a67396ac9..a2a87989b 100644 --- a/FoundationDB.Client/IFdbRetryable.cs +++ b/FoundationDB.Client/IFdbRetryable.cs @@ -41,58 +41,58 @@ public interface IFdbRetryable : IFdbReadOnlyRetryable /// Run an idempotent transaction block inside a write-only transaction, which can be executed more than once if any retryable error occurs. /// Idempotent handler that should only call write methods on the transation, and may be retried until the transaction commits, or a non-recoverable error occurs. - /// Token used to cancel the operation + /// Token used to cancel the operation /// /// You do not need to commit the transaction inside the handler, it will be done automatically. /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task WriteAsync([NotNull, InstantHandle] Action handler, CancellationToken cancellationToken); + Task WriteAsync([NotNull, InstantHandle] Action handler, CancellationToken ct); /// Run an idempotent transactional block inside a write-only transaction, which can be executed more than once if any retryable error occurs. /// Idempotent async handler that will be retried until the transaction commits, or a non-recoverable error occurs. - /// Token used to cancel the operation + /// Token used to cancel the operation /// /// You do not need to commit the transaction inside the handler, it will be done automatically. /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task WriteAsync([NotNull, InstantHandle] Func handler, CancellationToken cancellationToken); + Task WriteAsync([NotNull, InstantHandle] Func handler, CancellationToken ct); /// Run an idempotent transactional block inside a read-write transaction, which can be executed more than once if any retryable error occurs. /// Idempotent asynchronous handler that will be retried until the transaction commits, or a non-recoverable error occurs. - /// Token used to cancel the operation + /// Token used to cancel the operation /// /// You do not need to commit the transaction inside the handler, it will be done automatically. /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task ReadWriteAsync([NotNull, InstantHandle] Func asyncHandler, CancellationToken cancellationToken); + Task ReadWriteAsync([NotNull, InstantHandle] Func asyncHandler, CancellationToken ct); /// Run an idempotent transactional block that returns a value, inside a read-write transaction, which can be executed more than once if any retryable error occurs. /// Idempotent asynchronous lambda function that will be retried until the transaction commits, or a non-recoverable error occurs. The returned value of the last call will be the result of the operation. - /// Token used to cancel the operation + /// Token used to cancel the operation /// Result of the lambda function if the transaction committed successfully. /// /// You do not need to commit the transaction inside the handler, it will be done automatically. /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken cancellationToken); + Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken ct); //REVIEW: should we keep these ? /// [EXPERIMENTAL] do not use yet!. - Task WriteAsync([NotNull, InstantHandle] Action handler, [NotNull, InstantHandle] Action onDone, CancellationToken cancellationToken); + Task WriteAsync([NotNull, InstantHandle] Action handler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); /// [EXPERIMENTAL] do not use yet!. - Task WriteAsync([NotNull, InstantHandle] Func handler, [NotNull, InstantHandle] Action onDone, CancellationToken cancellationToken); + Task WriteAsync([NotNull, InstantHandle] Func handler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); /// [EXPERIMENTAL] do not use yet!. - Task ReadWriteAsync([NotNull, InstantHandle] Func asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken cancellationToken); + Task ReadWriteAsync([NotNull, InstantHandle] Func asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); /// [EXPERIMENTAL] do not use yet!. - Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken cancellationToken); + Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); } } diff --git a/FoundationDB.Client/IFdbTransaction.cs b/FoundationDB.Client/IFdbTransaction.cs index 911403fab..3d6c4ccda 100644 --- a/FoundationDB.Client/IFdbTransaction.cs +++ b/FoundationDB.Client/IFdbTransaction.cs @@ -116,10 +116,10 @@ public interface IFdbTransaction : IFdbReadOnlyTransaction /// Watch a key for any change in the database. /// /// Key to watch - /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance + /// CancellationToken used to abort the watch if the caller doesn't want to wait anymore. Note that you can manually cancel the watch by calling Cancel() on the returned FdbWatch instance /// FdbWatch that can be awaited and will complete when the key has changed in the database, or cancellation occurs. You can call Cancel() at any time if you are not interested in watching the key anymore. You MUST always call Dispose() if the watch completes or is cancelled, to ensure that resources are released properly. /// You can directly await an FdbWatch, or obtain a Task<Slice> by reading the property. - FdbWatch Watch(Slice key, CancellationToken cancellationToken); + FdbWatch Watch(Slice key, CancellationToken ct); } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs index 06bba208a..5d08ffb13 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryExtensions.cs @@ -45,52 +45,52 @@ public static class FdbDirectoryExtensions /// If the directory does not exist, it is created (creating parent directories if necessary). /// If layer is specified, it is checked against the layer of an existing directory or set as the layer of a new directory. /// - public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, path, Slice.Nil), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, path, Slice.Nil), ct); } /// Opens the directory with the given . /// If the directory does not exist, it is created (creating parent directories if necessary). /// If layer is specified, it is checked against the layer of an existing directory or set as the layer of a new directory. /// - public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, path, layer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, path, layer), ct); } /// Opens the directory with the given . /// If the directory does not exist, it is created (creating parent directories if necessary). /// If layer is specified, it is checked against the layer of an existing directory or set as the layer of a new directory. /// - public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), ct); } /// Opens the directory with the given . /// If the directory does not exist, it is created (creating parent directories if necessary). /// If layer is specified, it is checked against the layer of an existing directory or set as the layer of a new directory. /// - public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken cancellationToken) + public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, new[] { name }, layer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateOrOpenAsync(tr, new[] { name }, layer), ct); } /// Opens the directory with the given . @@ -99,9 +99,9 @@ public static Task CreateOrOpenAsync([NotNull] this IFdbDi /// public static Task CreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name, Slice layer = default(Slice)) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.CreateOrOpenAsync(trans, new[] { name }, layer); } @@ -118,9 +118,9 @@ public static Task CreateOrOpenAsync([NotNull] this IFdbDi /// public static Task TryCreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, bool readOnly, Slice layer = default(Slice)) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); if (readOnly) return directory.TryOpenAsync(trans, path, layer); @@ -140,7 +140,7 @@ public static Task CreateOrOpenAsync([NotNull] this IFdbDi /// public static Task TryCreateOrOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name, bool readOnly, Slice layer = default(Slice)) { - if (name == null) throw new ArgumentNullException("name"); + if (name == null) throw new ArgumentNullException(nameof(name)); return TryCreateOrOpenAsync(directory, trans, new[] { name }, readOnly, layer); } @@ -152,51 +152,51 @@ public static Task CreateOrOpenAsync([NotNull] this IFdbDi /// Creates a directory with the given (creating parent directories if necessary). /// An error is raised if the given directory already exists. /// - public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, path, Slice.Nil), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, path, Slice.Nil), ct); } /// Creates a directory with the given (creating parent directories if necessary). /// An error is raised if the given directory already exists. /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// - public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, path, layer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, path, layer), ct); } /// Creates a directory with the given . /// An error is raised if the given directory already exists. /// - public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, new [] { name }, Slice.Nil), ct); } /// Creates a directory with the given . /// An error is raised if the given directory already exists. /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// - public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken cancellationToken) + public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, new[] { name }, layer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.CreateAsync(tr, new[] { name }, layer), ct); } /// Creates a directory with the given . @@ -205,53 +205,53 @@ public static Task CreateAsync([NotNull] this IFdbDirector /// public static Task CreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name, Slice layer = default(Slice)) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.CreateAsync(trans, new[] { name }, layer); } /// Attempts to create a directory with the given (creating parent directories if necessary). - public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, path, Slice.Nil), cancellationToken); + return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, path, Slice.Nil), ct); } /// Attempts to create a directory with the given (creating parent directories if necessary). /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// - public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, path, layer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, path, layer), ct); } /// Attempts to create a directory with the given . - public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); - return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); + return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), ct); } /// Attempts to create a directory with the given . /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// - public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken cancellationToken) + public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); - return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, new[] { name }, layer), cancellationToken); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); + return db.ReadWriteAsync((tr) => directory.TryCreateAsync(tr, new[] { name }, layer), ct); } /// Attempts to create a directory with the given . @@ -259,9 +259,9 @@ public static Task TryCreateAsync([NotNull] this IFdbDirec /// public static Task TryCreateAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name, Slice layer = default(Slice)) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.TryCreateAsync(trans, new[] { name }, layer); } @@ -273,49 +273,49 @@ public static Task TryCreateAsync([NotNull] this IFdbDirec /// Opens the directory with the given . /// An error is raised if the directory does not exist. /// - public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadAsync((tr) => directory.OpenAsync(tr, path, Slice.Nil), cancellationToken); + return db.ReadAsync((tr) => directory.OpenAsync(tr, path, Slice.Nil), ct); } /// Opens the directory with the given . /// An error is raised if the directory does not exist, or if a layer is specified and a different layer was specified when the directory was created. /// - public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadAsync((tr) => directory.OpenAsync(tr, path, layer), cancellationToken); + return db.ReadAsync((tr) => directory.OpenAsync(tr, path, layer), ct); } /// Opens the sub-directory with the given . /// An error is raised if the directory does not exist. /// - public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.OpenAsync(tr, new[] { name }, Slice.Nil), cancellationToken); + return db.ReadAsync((tr) => directory.OpenAsync(tr, new[] { name }, Slice.Nil), ct); } /// Opens the sub-directory with the given . /// An error is raised if the directory does not exist, or if a layer is specified and a different layer was specified when the directory was created. /// - public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken cancellationToken) + public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.OpenAsync(tr, new[] { name }, layer), cancellationToken); + return db.ReadAsync((tr) => directory.OpenAsync(tr, new[] { name }, layer), ct); } /// Opens the sub-directory with the given . @@ -323,59 +323,59 @@ public static Task OpenAsync([NotNull] this IFdbDirectory /// public static Task OpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name, Slice layer = default(Slice)) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.OpenAsync(trans, new[] { name }, layer); } /// Attempts to open the directory with the given . - public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadAsync((tr) => directory.TryOpenAsync(tr, path, Slice.Nil), cancellationToken); + return db.ReadAsync((tr) => directory.TryOpenAsync(tr, path, Slice.Nil), ct); } /// Attempts to open the directory with the given . - public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadAsync((tr) => directory.TryOpenAsync(tr, path, layer), cancellationToken); + return db.ReadAsync((tr) => directory.TryOpenAsync(tr, path, layer), ct); } /// Attempts to open the directory with the given . - public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.TryOpenAsync(tr, new[] { name }, Slice.Nil), cancellationToken); + return db.ReadAsync((tr) => directory.TryOpenAsync(tr, new[] { name }, Slice.Nil), ct); } /// Attempts to open the directory with the given . - public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, Slice layer, CancellationToken cancellationToken) + public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, Slice layer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.TryOpenAsync(tr, new[] { name }, layer), cancellationToken); + return db.ReadAsync((tr) => directory.TryOpenAsync(tr, new[] { name }, layer), ct); } /// Attempts to open the directory with the given . public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.TryOpenAsync(trans, new[] { name }, Slice.Nil); } @@ -383,9 +383,9 @@ public static Task TryOpenAsync([NotNull] this IFdbDirecto /// Attempts to open the directory with the given . public static Task TryOpenAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name, Slice layer) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.TryOpenAsync(trans, new[] { name }, layer); } @@ -398,27 +398,27 @@ public static Task TryOpenAsync([NotNull] this IFdbDirecto /// There is no effect on the physical prefix of the given directory, or on clients that already have the directory open. /// An error is raised if the old directory does not exist, a directory already exists at `new_path`, or the parent directory of `new_path` does not exist. /// - public static Task MoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public static Task MoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (oldPath == null) throw new ArgumentNullException("oldPath"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (oldPath == null) throw new ArgumentNullException(nameof(oldPath)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); - return db.ReadWriteAsync((tr) => directory.MoveAsync(tr, oldPath, newPath), cancellationToken); + return db.ReadWriteAsync((tr) => directory.MoveAsync(tr, oldPath, newPath), ct); } /// Attempts to move the directory found at to . /// There is no effect on the physical prefix of the given directory, or on clients that already have the directory open. /// - public static Task TryMoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public static Task TryMoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (oldPath == null) throw new ArgumentNullException("oldPath"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (oldPath == null) throw new ArgumentNullException(nameof(oldPath)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); - return db.ReadWriteAsync((tr) => directory.TryMoveAsync(tr, oldPath, newPath), cancellationToken); + return db.ReadWriteAsync((tr) => directory.TryMoveAsync(tr, oldPath, newPath), ct); } #endregion @@ -429,25 +429,25 @@ public static Task TryMoveAsync([NotNull] this IFdbDirecto /// There is no effect on the physical prefix of the given directory, or on clients that already have the directory open. /// An error is raised if a directory already exists at `new_path`, or if the new path points to a child of the current directory. /// - public static Task MoveToAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public static Task MoveToAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, [NotNull] IEnumerable newPath, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); - return db.ReadWriteAsync((tr) => subspace.MoveToAsync(tr, newPath), cancellationToken); + return db.ReadWriteAsync((tr) => subspace.MoveToAsync(tr, newPath), ct); } /// Attempts to move the current directory to . /// There is no effect on the physical prefix of the given directory, or on clients that already have the directory open. /// - public static Task TryMoveToAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public static Task TryMoveToAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, [NotNull] IEnumerable newPath, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); - return db.ReadWriteAsync((tr) => subspace.TryMoveToAsync(tr, newPath), cancellationToken); + return db.ReadWriteAsync((tr) => subspace.TryMoveToAsync(tr, newPath), ct); } #endregion @@ -457,34 +457,34 @@ public static Task TryMoveToAsync([NotNull] this FdbDirect /// Removes the directory, its contents, and all subdirectories. /// Warning: Clients that have already opened the directory might still insert data into its contents after it is removed. /// - public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, IEnumerable path, CancellationToken cancellationToken) + public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr, path), cancellationToken); + return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr, path), ct); } /// Removes the directory, its contents, and all subdirectories. /// Warning: Clients that have already opened the directory might still insert data into its contents after it is removed. /// - public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); - return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr, new [] { name }), cancellationToken); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); + return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr, new [] { name }), ct); } /// Removes the directory, its contents, and all subdirectories. /// Warning: Clients that have already opened the directory might still insert data into its contents after it is removed. /// - public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, CancellationToken cancellationToken) + public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr), cancellationToken); + return db.ReadWriteAsync((tr) => directory.RemoveAsync(tr), ct); } /// Removes the directory, its contents, and all subdirectories. @@ -492,9 +492,9 @@ public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] /// public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.RemoveAsync(trans, new[] { name }); } @@ -502,24 +502,24 @@ public static Task RemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] /// Removes the directory, its contents, and all subdirectories. /// Warning: Clients that have already opened the directory might still insert data into its contents after it is removed. /// - public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, IEnumerable path, CancellationToken cancellationToken) + public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadWriteAsync((tr) => directory.TryRemoveAsync(tr, path), cancellationToken); + return db.ReadWriteAsync((tr) => directory.TryRemoveAsync(tr, path), ct); } /// Removes the directory, its contents, and all subdirectories. /// Warning: Clients that have already opened the directory might still insert data into its contents after it is removed. /// - public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadWriteAsync((tr) => directory.TryRemoveAsync(tr, new [] { name }), cancellationToken); + return db.ReadWriteAsync((tr) => directory.TryRemoveAsync(tr, new [] { name }), ct); } /// Removes the directory, its contents, and all subdirectories. @@ -527,9 +527,9 @@ public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, /// public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.TryRemoveAsync(trans, new[] { name }); } @@ -540,44 +540,44 @@ public static Task TryRemoveAsync([NotNull] this IFdbDirectory directory, /// Checks if a directory already exists /// Returns true if the directory exists, otherwise false. - public static Task ExistsAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, IEnumerable path, CancellationToken cancellationToken) + public static Task ExistsAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadAsync((tr) => directory.ExistsAsync(tr, path), cancellationToken); + return db.ReadAsync((tr) => directory.ExistsAsync(tr, path), ct); } /// Checks if a directory already exists /// Returns true if the directory exists, otherwise false. - public static Task ExistsAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task ExistsAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.ExistsAsync(tr, new[] { name }), cancellationToken); + return db.ReadAsync((tr) => directory.ExistsAsync(tr, new[] { name }), ct); } /// Checks if a directory already exists /// Returns true if the directory exists, otherwise false. public static Task ExistsAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.ExistsAsync(trans, new[] { name }); } /// Checks if this directory exists /// Returns true if the directory exists, otherwise false. - public static Task ExistsAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken cancellationToken) + public static Task ExistsAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadAsync((tr) => subspace.ExistsAsync(tr), cancellationToken); + return db.ReadAsync((tr) => subspace.ExistsAsync(tr), ct); } #endregion @@ -585,95 +585,95 @@ public static Task ExistsAsync([NotNull] this FdbDirectorySubspace subspac #region List / TryList... /// Returns the list of subdirectories of directory at . - public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); - return db.ReadAsync((tr) => directory.ListAsync(tr, path), cancellationToken); + return db.ReadAsync((tr) => directory.ListAsync(tr, path), ct); } /// Returns the list of subdirectories of the sub-directory with the given . - public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); - return db.ReadAsync((tr) => directory.ListAsync(tr, new [] { name }), cancellationToken); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); + return db.ReadAsync((tr) => directory.ListAsync(tr, new [] { name }), ct); } /// Returns the list of subdirectories of the current directory. - public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, CancellationToken cancellationToken) + public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - return db.ReadAsync((tr) => directory.ListAsync(tr), cancellationToken); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + return db.ReadAsync((tr) => directory.ListAsync(tr), ct); } /// Returns the list of subdirectories of the current directory. public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return directory.ListAsync(trans); } /// Returns the list of subdirectories of the sub-directory with the given . public static Task> ListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.ListAsync(trans, new[] { name }); } /// Returns the list of subdirectories of directory at , if it exists - public static Task> TryListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, IEnumerable path, CancellationToken cancellationToken) + public static Task> TryListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, IEnumerable path, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); //REVIEW: or not? + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); //REVIEW: or not? - return db.ReadAsync((tr) => directory.TryListAsync(tr, path), cancellationToken); + return db.ReadAsync((tr) => directory.TryListAsync(tr, path), ct); } /// Returns the list of subdirectories of the sub-directory with the given , if it exists - public static Task> TryListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken cancellationToken) + public static Task> TryListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyRetryable db, [NotNull] string name, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (name == null) throw new ArgumentNullException(nameof(name)); - return db.ReadAsync((tr) => directory.TryListAsync(tr, new [] { name }), cancellationToken); + return db.ReadAsync((tr) => directory.TryListAsync(tr, new [] { name }), ct); } /// Returns the list of subdirectories of the sub-directory with the given , if it exists public static Task> TryListAsync([NotNull] this IFdbDirectory directory, [NotNull] IFdbReadOnlyTransaction trans, [NotNull] string name) { - if (directory == null) throw new ArgumentNullException("directory"); - if (trans == null) throw new ArgumentNullException("trans"); - if (name == null) throw new ArgumentNullException("name"); + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (name == null) throw new ArgumentNullException(nameof(name)); return directory.TryListAsync(trans, new[] { name }); } /// Returns the list of all the subdirectories of the current directory. - public static Task> ListAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken cancellationToken) + public static Task> ListAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadAsync((tr) => subspace.ListAsync(tr), cancellationToken); + return db.ReadAsync((tr) => subspace.ListAsync(tr), ct); } /// Returns the list of all the subdirectories of the current directory, it it exists. - public static Task> TryListAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken cancellationToken) + public static Task> TryListAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbReadOnlyRetryable db, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadAsync((tr) => subspace.TryListAsync(tr), cancellationToken); + return db.ReadAsync((tr) => subspace.TryListAsync(tr), ct); } #endregion @@ -681,22 +681,22 @@ public static Task> TryListAsync([NotNull] this FdbDirectorySubspac #region Metadata /// Change the layer id of the directory at - public static Task ChangeLayerAsync([NotNull] this FdbDirectoryLayer directory, [NotNull] IFdbRetryable db, IEnumerable path, Slice newLayer, CancellationToken cancellationToken) + public static Task ChangeLayerAsync([NotNull] this FdbDirectoryLayer directory, [NotNull] IFdbRetryable db, IEnumerable path, Slice newLayer, CancellationToken ct) { - if (directory == null) throw new ArgumentNullException("directory"); - if (db == null) throw new ArgumentNullException("db"); - if (path == null) throw new ArgumentNullException("path"); //REVIEW: or not? + if (directory == null) throw new ArgumentNullException(nameof(directory)); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (path == null) throw new ArgumentNullException(nameof(path)); //REVIEW: or not? - return db.ReadWriteAsync((tr) => directory.ChangeLayerAsync(tr, path, newLayer), cancellationToken); + return db.ReadWriteAsync((tr) => directory.ChangeLayerAsync(tr, path, newLayer), ct); } /// Change the layer id of this directory - public static Task ChangeLayerAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, Slice newLayer, CancellationToken cancellationToken) + public static Task ChangeLayerAsync([NotNull] this FdbDirectorySubspace subspace, [NotNull] IFdbRetryable db, Slice newLayer, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (db == null) throw new ArgumentNullException("db"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (db == null) throw new ArgumentNullException(nameof(db)); - return db.ReadWriteAsync((tr) => subspace.ChangeLayerAsync(tr, newLayer), cancellationToken); + return db.ReadWriteAsync((tr) => subspace.ChangeLayerAsync(tr, newLayer), ct); } #endregion diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs index 3655ebe52..a2142cc2b 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs @@ -45,9 +45,9 @@ private sealed class EmptySequence : IFdbAsyncEnumerable, IFdb private EmptySequence() { } - Task IAsyncEnumerator.MoveNextAsync(CancellationToken cancellationToken) + Task IAsyncEnumerator.MoveNextAsync(CancellationToken ct) { - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return TaskHelpers.FalseTask; } @@ -107,9 +107,9 @@ IAsyncEnumerator IAsyncEnumerable.GetEnumerator() return this.GetEnumerator(); } - async Task IAsyncEnumerator.MoveNextAsync(CancellationToken cancellationToken) + async Task IAsyncEnumerator.MoveNextAsync(CancellationToken ct) { - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); if (m_called) return false; //note: avoid using local variables as much as possible! @@ -129,7 +129,7 @@ async Task IAsyncEnumerator.MoveNextAsync(CancellationToken canc if (lambda is Func>) { - m_current = await ((Func>)lambda)(cancellationToken).ConfigureAwait(false); + m_current = await ((Func>)lambda)(ct).ConfigureAwait(false); return true; } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs index bc2b71b1b..e948279fa 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs @@ -56,7 +56,7 @@ public EnumerableIterator(IEnumerator iterator, Func MoveNextAsync(CancellationToken cancellationToken) + public async Task MoveNextAsync(CancellationToken ct) { if (m_disposed) { @@ -64,7 +64,7 @@ public async Task MoveNextAsync(CancellationToken cancellationToken) return false; } - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); if (m_iterator.MoveNext()) { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs index ee02eedce..63888b502 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs @@ -193,13 +193,13 @@ private async Task ReadAllThenSort(CancellationToken ct) return true; } - public Task MoveNextAsync(CancellationToken cancellationToken) + public Task MoveNextAsync(CancellationToken ct) { // Firt call will be slow (and async), but the rest of the calls will use the results already sorted in memory, and should be as fast as possible! if (m_map == null) { - return ReadAllThenSort(cancellationToken); + return ReadAllThenSort(ct); } int pos = checked(m_offset + 1); diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs index c0ac7006a..df90f67b5 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs @@ -65,7 +65,7 @@ public static IFdbAsyncEnumerable Singleton(T value) [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Single([NotNull] Func lambda) { - if (lambda == null) throw new ArgumentNullException("lambda"); + if (lambda == null) throw new ArgumentNullException(nameof(lambda)); return new SingletonSequence(lambda); } @@ -75,7 +75,7 @@ public static IFdbAsyncEnumerable Single([NotNull] Func lambda) [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Single([NotNull] Func> asyncLambda) { - if (asyncLambda == null) throw new ArgumentNullException("asyncLambda"); + if (asyncLambda == null) throw new ArgumentNullException(nameof(asyncLambda)); return new SingletonSequence(asyncLambda); } @@ -85,7 +85,7 @@ public static IFdbAsyncEnumerable Single([NotNull] Func> asyncLamb [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Single([NotNull] Func> asyncLambda) { - if (asyncLambda == null) throw new ArgumentNullException("asyncLambda"); + if (asyncLambda == null) throw new ArgumentNullException(nameof(asyncLambda)); return new SingletonSequence(asyncLambda); } @@ -93,8 +93,8 @@ public static IFdbAsyncEnumerable Single([NotNull] Func ToAsyncEnumerable([NotNull] this IEnumerable source, [NotNull] Func> lambda) { - if (source == null) throw new ArgumentNullException("source"); - if (lambda == null) throw new ArgumentNullException("lambda"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (lambda == null) throw new ArgumentNullException(nameof(lambda)); return Create(source, (iterator) => new EnumerableIterator(iterator, lambda)); } @@ -103,7 +103,7 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnu [NotNull, LinqTunnel] public static IFdbAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnumerable source) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return Create(source, (iterator) => new EnumerableIterator(iterator, x => Task.FromResult(x))); } @@ -124,8 +124,8 @@ public static IFdbAsyncEnumerable FromTask([NotNull] Func> asyncLa [NotNull, LinqTunnel] public static IEnumerable> Buffered([NotNull] this IEnumerable source, int batchSize) { - if (source == null) throw new ArgumentNullException("source"); - if (batchSize <= 0) throw new ArgumentException("Batch size must be greater than zero.", "batchSize"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (batchSize <= 0) throw new ArgumentException("Batch size must be greater than zero.", nameof(batchSize)); var list = new List(batchSize); foreach (var item in source) @@ -149,8 +149,8 @@ public static IEnumerable> Buffered([NotNull] this IEnumerable sou [NotNull, LinqTunnel] public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> selector) { - if (source == null) throw new ArgumentNullException("source"); - if (selector == null) throw new ArgumentNullException("selector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -165,8 +165,8 @@ public static IFdbAsyncEnumerable SelectMany([NotNull [NotNull, LinqTunnel] public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); return SelectMany(source, TaskHelpers.WithCancellation(asyncSelector)); } @@ -175,8 +175,8 @@ public static IFdbAsyncEnumerable SelectMany([NotNull [NotNull, LinqTunnel] public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -191,9 +191,9 @@ public static IFdbAsyncEnumerable SelectMany([NotNull [NotNull, LinqTunnel] public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> collectionSelector, [NotNull] Func resultSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (collectionSelector == null) throw new ArgumentNullException("collectionSelector"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (collectionSelector == null) throw new ArgumentNullException(nameof(collectionSelector)); + if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -208,9 +208,9 @@ public static IFdbAsyncEnumerable SelectMany SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncCollectionSelector == null) throw new ArgumentNullException("asyncCollectionSelector"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncCollectionSelector == null) throw new ArgumentNullException(nameof(asyncCollectionSelector)); + if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); return SelectMany(source, TaskHelpers.WithCancellation(asyncCollectionSelector), resultSelector); } @@ -219,9 +219,9 @@ public static IFdbAsyncEnumerable SelectMany SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncCollectionSelector == null) throw new ArgumentNullException("asyncCollectionSelector"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncCollectionSelector == null) throw new ArgumentNullException(nameof(asyncCollectionSelector)); + if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -240,8 +240,8 @@ public static IFdbAsyncEnumerable SelectMany Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func selector) { - if (source == null) throw new ArgumentNullException("source"); - if (selector == null) throw new ArgumentNullException("selector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -256,8 +256,8 @@ public static IFdbAsyncEnumerable Select([NotNull] th [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); return Select(source, TaskHelpers.WithCancellation(asyncSelector)); } @@ -266,8 +266,8 @@ public static IFdbAsyncEnumerable Select([NotNull] th [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -286,8 +286,8 @@ public static IFdbAsyncEnumerable Select([NotNull] th [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func predicate) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -302,8 +302,8 @@ public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsy [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncPredicate) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncPredicate == null) throw new ArgumentNullException("asyncPredicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncPredicate == null) throw new ArgumentNullException(nameof(asyncPredicate)); return Where(source, TaskHelpers.WithCancellation(asyncPredicate)); } @@ -312,8 +312,8 @@ public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncPredicate) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncPredicate == null) throw new ArgumentNullException("asyncPredicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncPredicate == null) throw new ArgumentNullException(nameof(asyncPredicate)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -332,8 +332,8 @@ public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsy [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Take([NotNull] this IFdbAsyncEnumerable source, int count) { - if (source == null) throw new ArgumentNullException("source"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count cannot be less than zero"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -352,8 +352,8 @@ public static IFdbAsyncEnumerable Take([NotNull] this IFdbAsyn [NotNull, LinqTunnel] public static IFdbAsyncEnumerable TakeWhile([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func condition) { - if (source == null) throw new ArgumentNullException("source"); - if (condition == null) throw new ArgumentNullException("condition"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (condition == null) throw new ArgumentNullException(nameof(condition)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -389,8 +389,8 @@ public static IFdbAsyncEnumerable TakeWhile(this IFdbAsyncEnum [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Skip([NotNull] this IFdbAsyncEnumerable source, int count) { - if (source == null) throw new ArgumentNullException("source"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count cannot be less than zero"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -409,8 +409,8 @@ public static IFdbAsyncEnumerable Skip([NotNull] this IFdbAsyn [NotNull, LinqTunnel] public static IFdbAsyncEnumerable SelectAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector, FdbParallelQueryOptions options = null) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); return new FdbParallelSelectAsyncIterator(source, asyncSelector, options ?? new FdbParallelQueryOptions()); } @@ -426,7 +426,7 @@ public static IFdbAsyncEnumerable SelectAsync([NotNul [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdbAsyncEnumerable source) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return new FdbPrefetchingAsyncIterator(source, 1); } @@ -443,8 +443,8 @@ public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdb [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdbAsyncEnumerable source, int prefetchCount) { - if (source == null) throw new ArgumentNullException("source"); - if (prefetchCount <= 0) throw new ArgumentOutOfRangeException("prefetchCount", prefetchCount, "Prefetch count must be at least one."); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (prefetchCount <= 0) throw new ArgumentOutOfRangeException(nameof(prefetchCount), prefetchCount, "Prefetch count must be at least one."); return new FdbPrefetchingAsyncIterator(source, prefetchCount); } @@ -461,8 +461,8 @@ public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdb [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Window([NotNull] this IFdbAsyncEnumerable source, int maxWindowSize) { - if (source == null) throw new ArgumentNullException("source"); - if (maxWindowSize <= 0) throw new ArgumentOutOfRangeException("maxWindowSize", maxWindowSize, "Window size must be at least one."); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (maxWindowSize <= 0) throw new ArgumentOutOfRangeException(nameof(maxWindowSize), maxWindowSize, "Window size must be at least one."); return new FdbWindowingAsyncIterator(source, maxWindowSize); } @@ -479,8 +479,8 @@ public static IFdbAsyncEnumerable Window([NotNull] this IFdb [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Batch([NotNull] this IFdbAsyncEnumerable source, int batchSize) { - if (source == null) throw new ArgumentNullException("source"); - if (batchSize <= 0) throw new ArgumentOutOfRangeException("batchSize", batchSize, "Batch size must be at least one."); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (batchSize <= 0) throw new ArgumentOutOfRangeException(nameof(batchSize), batchSize, "Batch size must be at least one."); return new FdbBatchingAsyncIterator(source, batchSize); } @@ -492,7 +492,7 @@ public static IFdbAsyncEnumerable Batch([NotNull] this IFdbA [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Distinct([NotNull] this IFdbAsyncEnumerable source, IEqualityComparer comparer = null) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); comparer = comparer ?? EqualityComparer.Default; return new FdbDistinctAsyncIterator(source, comparer); @@ -505,8 +505,8 @@ public static IFdbAsyncEnumerable Distinct([NotNull] this IFdb [NotNull] public static IFdbAsyncOrderedEnumerable OrderBy([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) { - if (source == null) throw new ArgumentNullException("source"); - if (keySelector == null) throw new ArgumentNullException("keySelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); comparer = comparer ?? Comparer.Default; return new OrderedSequence(source, keySelector, comparer, descending: false, parent: null); @@ -515,8 +515,8 @@ public static IFdbAsyncOrderedEnumerable OrderBy([NotNul [NotNull] public static IFdbAsyncOrderedEnumerable OrderByDescending([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) { - if (source == null) throw new ArgumentNullException("source"); - if (keySelector == null) throw new ArgumentNullException("keySelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); comparer = comparer ?? Comparer.Default; return new OrderedSequence(source, keySelector, comparer, descending: true, parent: null); @@ -525,14 +525,14 @@ public static IFdbAsyncOrderedEnumerable OrderByDescending ThenBy([NotNull] this IFdbAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) { - if (source == null) throw new ArgumentNullException("keySelector"); + if (source == null) throw new ArgumentNullException(nameof(keySelector)); return source.CreateOrderedEnumerable(keySelector, comparer, descending: false); } [NotNull] public static IFdbAsyncOrderedEnumerable ThenByDescending([NotNull] this IFdbAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) { - if (source == null) throw new ArgumentNullException("keySelector"); + if (source == null) throw new ArgumentNullException(nameof(keySelector)); return source.CreateOrderedEnumerable(keySelector, comparer, descending: true); } @@ -551,8 +551,8 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingExecute an action for each element of an async sequence public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Action action, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (action == null) throw new ArgumentNullException("action"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (action == null) throw new ArgumentNullException(nameof(action)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -568,7 +568,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingExecute an async action for each element of an async sequence public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) { - if (asyncAction == null) throw new ArgumentNullException("asyncAction"); + if (asyncAction == null) throw new ArgumentNullException(nameof(asyncAction)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -584,8 +584,8 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingExecute an async action for each element of an async sequence public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncAction == null) throw new ArgumentNullException("asyncAction"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncAction == null) throw new ArgumentNullException(nameof(asyncAction)); var iterator = source as FdbAsyncIterator; if (iterator != null) @@ -602,7 +602,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescending> ToHashsetAsync([NotNull] this IFdbAsyncEnumerable source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return AggregateAsync( source, @@ -617,7 +617,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescending> ToListAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return AggregateAsync( source, @@ -630,22 +630,22 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingCreate an array from an async sequence. [ItemNotNull] - public static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken cancellationToken = default(CancellationToken)) + public static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return AggregateAsync( source, new Buffer(), (buffer, x) => buffer.Add(x), (buffer) => buffer.ToArray(), - cancellationToken + ct ); } /// Create an array from an async sequence, knowing a rough estimation of the number of elements. [ItemNotNull] - internal static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, int estimatedSize, CancellationToken cancellationToken = default(CancellationToken)) + internal static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, int estimatedSize, CancellationToken ct = default(CancellationToken)) { Contract.Requires(source != null && estimatedSize >= 0); @@ -654,74 +654,74 @@ public static IFdbAsyncOrderedEnumerable ThenByDescending(estimatedSize), (buffer, x) => buffer.Add(x), (buffer) => buffer.ToArray(), - cancellationToken + ct ); } /// Creates a Dictionary from an async sequence according to a specified key selector function and key comparer. [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, IEqualityComparer comparer = null, CancellationToken cancellationToken = default(CancellationToken)) + public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (keySelector == null) throw new ArgumentNullException("keySelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); return AggregateAsync( source, new Dictionary(comparer ?? EqualityComparer.Default), (results, x) => { results[keySelector(x)] = x; }, - cancellationToken + ct ); } /// Creates a Dictionary from an async sequence according to a specified key selector function, a comparer, and an element selector function. [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, [NotNull, InstantHandle] Func elementSelector, IEqualityComparer comparer = null, CancellationToken cancellationToken = default(CancellationToken)) + public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, [NotNull, InstantHandle] Func elementSelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (keySelector == null) throw new ArgumentNullException("keySelector"); - if (elementSelector == null) throw new ArgumentNullException("elementSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); + if (elementSelector == null) throw new ArgumentNullException(nameof(elementSelector)); return AggregateAsync( source, new Dictionary(comparer ?? EqualityComparer.Default), (results, x) => { results[keySelector(x)] = elementSelector(x); }, - cancellationToken + ct ); } /// Creates a Dictionary from an async sequence of pairs of keys and values. [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable> source, IEqualityComparer comparer = null, CancellationToken cancellationToken = default(CancellationToken)) + public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable> source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - cancellationToken.ThrowIfCancellationRequested(); + if (source == null) throw new ArgumentNullException(nameof(source)); + ct.ThrowIfCancellationRequested(); return AggregateAsync( source, new Dictionary(comparer ?? EqualityComparer.Default), (results, x) => { results[x.Key] = x.Value; }, - cancellationToken + ct ); } /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func aggregator, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (aggregator == null) throw new ArgumentNullException("aggregator"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); using (var iterator = source.GetEnumerator(FdbAsyncMode.All)) { Contract.Assert(iterator != null, "The sequence returned a null async iterator"); - if (!(await iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false))) + if (!(await iterator.MoveNextAsync(ct).ConfigureAwait(false))) { throw new InvalidOperationException("The sequence was empty"); } var item = iterator.Current; - while (await iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) { item = aggregator(item, iterator.Current); } @@ -731,55 +731,55 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingApplies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (aggregator == null) throw new ArgumentNullException("aggregator"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); var accumulate = seed; - await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, cancellationToken).ConfigureAwait(false); + await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); return accumulate; } /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (aggregator == null) throw new ArgumentNullException("aggregator"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); var accumulate = seed; - await ForEachAsync(source, (x) => { aggregator(accumulate, x); }, cancellationToken).ConfigureAwait(false); + await ForEachAsync(source, (x) => { aggregator(accumulate, x); }, ct).ConfigureAwait(false); return accumulate; } /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (aggregator == null) throw new ArgumentNullException("aggregator"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); + if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); var accumulate = seed; - await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, cancellationToken).ConfigureAwait(false); + await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); return resultSelector(accumulate); } /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken cancellationToken = default(CancellationToken)) + public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (aggregator == null) throw new ArgumentNullException("aggregator"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); + if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); var accumulate = seed; - await ForEachAsync(source, (x) => aggregator(accumulate, x), cancellationToken); + await ForEachAsync(source, (x) => aggregator(accumulate, x), ct); return resultSelector(accumulate); } /// Returns the first element of an async sequence, or an exception if it is empty public static Task FirstAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -791,7 +791,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingReturns the first element of an async sequence, or the default value for the type if it is empty public static Task FirstOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -804,7 +804,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingWill need to call MoveNext at least twice to ensure that there is no second element. public static Task SingleAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -817,7 +817,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingWill need to call MoveNext at least twice to ensure that there is no second element. public static Task SingleOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -829,7 +829,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingReturns the last element of an async sequence, or an exception if it is empty public static async Task LastAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -847,7 +847,7 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingReturns the last element of an async sequence, or the default value for the type if it is empty public static async Task LastOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -864,8 +864,8 @@ public static IFdbAsyncOrderedEnumerable ThenByDescendingReturns the element at a specific location of an async sequence, or an exception if there are not enough elements public static async Task ElementAtAsync([NotNull] this IFdbAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (index < 0) throw new ArgumentOutOfRangeException("index"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -891,8 +891,8 @@ await Run( /// Returns the element at a specific location of an async sequence, or the default value for the type if it there are not enough elements public static async Task ElementAtOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (index < 0) throw new ArgumentOutOfRangeException("index"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); ct.ThrowIfCancellationRequested(); var rq = source as FdbRangeQuery; @@ -920,7 +920,7 @@ await Run( /// Returns the number of elements in an async sequence. public static async Task CountAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); int count = 0; @@ -933,8 +933,8 @@ await Run( /// Returns a number that represents how many elements in the specified async sequence satisfy a condition. public static async Task CountAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); int count = 0; @@ -946,7 +946,7 @@ await Run( /// Returns the sum of all elements in the specified async sequence. public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ulong sum = 0; @@ -958,8 +958,8 @@ await Run( /// Returns the sum of all elements in the specified async sequence that satisfy a condition. public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); ulong sum = 0; @@ -971,7 +971,7 @@ await Run( /// Returns the sum of all elements in the specified async sequence. public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); long sum = 0; @@ -983,8 +983,8 @@ await Run( /// Returns the sum of all elements in the specified async sequence that satisfy a condition. public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); long sum = 0; @@ -996,7 +996,7 @@ await Run( /// Returns the smallest value in the specified async sequence public static async Task MinAsync([NotNull] this IFdbAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); comparer = comparer ?? Comparer.Default; bool found = false; @@ -1022,7 +1022,7 @@ await ForEachAsync( /// Returns the largest value in the specified async sequence public static async Task MaxAsync([NotNull] this IFdbAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); comparer = comparer ?? Comparer.Default; bool found = false; @@ -1049,7 +1049,7 @@ await ForEachAsync( /// This is the logical equivalent to "source.Count() > 0" but can be better optimized by some providers public static async Task AnyAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) @@ -1061,8 +1061,8 @@ await ForEachAsync( /// Determines whether any element of an async sequence satisfies a condition. public static async Task AnyAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); ct.ThrowIfCancellationRequested(); using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) @@ -1079,7 +1079,7 @@ await ForEachAsync( /// This is the logical equivalent to "source.Count() == 0" or "!source.Any()" but can be better optimized by some providers public static async Task NoneAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); ct.ThrowIfCancellationRequested(); using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) @@ -1091,8 +1091,8 @@ await ForEachAsync( /// Determines whether none of the elements of an async sequence satisfies a condition. public static async Task NoneAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) { - if (source == null) throw new ArgumentNullException("source"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); ct.ThrowIfCancellationRequested(); using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) @@ -1171,7 +1171,7 @@ public void Add(int size) [NotNull, LinqTunnel] public static IFdbAsyncEnumerable WithCountStatistics([NotNull] this IFdbAsyncEnumerable source, out QueryStatistics counter) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); var signal = new QueryStatistics(0); counter = signal; @@ -1191,7 +1191,7 @@ public static IFdbAsyncEnumerable WithCountStatistics([NotNull [NotNull, LinqTunnel] public static IFdbAsyncEnumerable> WithSizeStatistics([NotNull] this IFdbAsyncEnumerable> source, out QueryStatistics statistics) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); var data = new KeyValueSize(); statistics = new QueryStatistics(data); @@ -1211,7 +1211,7 @@ public static IFdbAsyncEnumerable> WithSizeStatistics [NotNull, LinqTunnel] public static IFdbAsyncEnumerable WithSizeStatistics([NotNull] this IFdbAsyncEnumerable source, out QueryStatistics statistics) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); var data = new DataSize(); statistics = new QueryStatistics(data); @@ -1231,8 +1231,8 @@ public static IFdbAsyncEnumerable WithSizeStatistics([NotNull] this IFdbA [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Observe([NotNull] this IFdbAsyncEnumerable source, [NotNull] Action handler) { - if (source == null) throw new ArgumentNullException("source"); - if (handler == null) throw new ArgumentNullException("handler"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); return new FdbObserverIterator(source, new AsyncObserverExpression(handler)); } @@ -1242,8 +1242,8 @@ public static IFdbAsyncEnumerable Observe([NotNull] this IFdbA [NotNull, LinqTunnel] public static IFdbAsyncEnumerable Observe([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func asyncHandler) { - if (source == null) throw new ArgumentNullException("source"); - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); return new FdbObserverIterator(source, new AsyncObserverExpression(asyncHandler)); } diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs index 5ecd89b4f..e0ba6344a 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs @@ -271,11 +271,11 @@ protected void MarkAsFailed() this.Dispose(); } - protected bool Canceled(CancellationToken cancellationToken) + protected bool Canceled(CancellationToken ct) { //TODO: store the state "canceled" somewhere? this.Dispose(); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); return false; } diff --git a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs index 767baf138..fb19d9ec4 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs @@ -64,17 +64,17 @@ protected override Task OnFirstAsync(CancellationToken ct) return base.OnFirstAsync(ct); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed m_set = null; return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_set.Add(current)) @@ -86,7 +86,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } m_set = null; - return Canceled(cancellationToken); + return Canceled(ct); } public override async Task ExecuteAsync(Action handler, CancellationToken ct) diff --git a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs index b080e9672..cbf33dfcb 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs @@ -52,16 +52,16 @@ protected override FdbAsyncIterator Clone() return new FdbObserverIterator(m_source, m_observer); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_observer.Async) @@ -70,13 +70,13 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - await m_observer.InvokeAsync(current, cancellationToken).ConfigureAwait(false); + await m_observer.InvokeAsync(current, ct).ConfigureAwait(false); } return Publish(current); } - return Canceled(cancellationToken); + return Canceled(ct); } } diff --git a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs index 473d7d5d2..53ebc2b59 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs @@ -115,7 +115,7 @@ protected override async Task OnFirstAsync(CancellationToken ct) return true; } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { try { @@ -123,7 +123,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo if (m_done) return false; - var next = await m_processingQueue.ReceiveAsync(cancellationToken).ConfigureAwait(false); + var next = await m_processingQueue.ReceiveAsync(ct).ConfigureAwait(false); LogDebug("[OnNextAsync] got result from queue"); if (!next.HasValue) diff --git a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs index f565ff394..e30442dfe 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs @@ -57,23 +57,23 @@ protected override FdbAsyncIterator Clone() return new FdbSelectManyAsyncIterator(m_source, m_selector); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { // if we are in a batch, iterate over it // if not, wait for the next batch - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { if (m_batch == null) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // inner completed return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; IEnumerable sequence; if (!m_selector.Async) @@ -82,7 +82,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - sequence = await m_selector.InvokeAsync(m_iterator.Current, cancellationToken).ConfigureAwait(false); + sequence = await m_selector.InvokeAsync(m_iterator.Current, ct).ConfigureAwait(false); } if (sequence == null) throw new InvalidOperationException("The inner sequence returned a null collection"); @@ -100,7 +100,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo return Publish(m_batch.Current); } - return Canceled(cancellationToken); + return Canceled(ct); } protected override void Cleanup() @@ -149,23 +149,23 @@ protected override FdbAsyncIterator Clone() return new FdbSelectManyAsyncIterator(m_source, m_collectionSelector, m_resultSelector); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { // if we are in a batch, iterate over it // if not, wait for the next batch - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { if (m_batch == null) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // inner completed return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; m_sourceCurrent = m_iterator.Current; @@ -177,7 +177,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - sequence = await m_collectionSelector.InvokeAsync(m_sourceCurrent, cancellationToken).ConfigureAwait(false); + sequence = await m_collectionSelector.InvokeAsync(m_sourceCurrent, ct).ConfigureAwait(false); } if (sequence == null) throw new InvalidOperationException("The inner sequence returned a null collection"); @@ -196,7 +196,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo return Publish(m_resultSelector(m_sourceCurrent, m_batch.Current)); } - return Canceled(cancellationToken); + return Canceled(ct); } protected override void Cleanup() diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs index 82f18ebd0..34a2ddf41 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs @@ -54,16 +54,16 @@ protected override FdbAsyncIterator Clone() return new FdbWhereAsyncIterator(m_source, m_filter); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_filter.Async) @@ -75,7 +75,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - if (!await m_filter.InvokeAsync(current, cancellationToken).ConfigureAwait(false)) + if (!await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) { continue; } @@ -84,7 +84,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo return Publish(current); } - return Canceled(cancellationToken); + return Canceled(ct); } public override FdbAsyncIterator Where(Func predicate) diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs index 11ffbc1e9..018576670 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs @@ -80,20 +80,20 @@ protected override Task OnFirstAsync(CancellationToken ct) return base.OnFirstAsync(ct); } - protected override async Task OnNextAsync(CancellationToken cancellationToken) + protected override async Task OnNextAsync(CancellationToken ct) { if (m_remaining != null && m_remaining.Value <= 0) { // reached limit! return Completed(); } - while (!cancellationToken.IsCancellationRequested) + while (!ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(cancellationToken).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) { // completed return Completed(); } - if (cancellationToken.IsCancellationRequested) break; + if (ct.IsCancellationRequested) break; #region Filtering... @@ -106,7 +106,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - if (!await m_filter.InvokeAsync(current, cancellationToken).ConfigureAwait(false)) continue; + if (!await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) continue; } } @@ -136,7 +136,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo } else { - result = await m_transform.InvokeAsync(current, cancellationToken).ConfigureAwait(false); + result = await m_transform.InvokeAsync(current, ct).ConfigureAwait(false); } #endregion @@ -153,7 +153,7 @@ protected override async Task OnNextAsync(CancellationToken cancellationTo #endregion } - return Canceled(cancellationToken); + return Canceled(ct); } public override FdbAsyncIterator Select(Func selector) diff --git a/FoundationDB.Client/Native/FdbFuture.cs b/FoundationDB.Client/Native/FdbFuture.cs index f54c925fb..b896503ef 100644 --- a/FoundationDB.Client/Native/FdbFuture.cs +++ b/FoundationDB.Client/Native/FdbFuture.cs @@ -71,52 +71,52 @@ public static class Flags /// 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 + /// 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) + public static FdbFutureSingle FromHandle([NotNull] FutureHandle handle, [NotNull] Func selector, CancellationToken ct) { - return new FdbFutureSingle(handle, selector, cancellationToken); + return new FdbFutureSingle(handle, selector, ct); } /// 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 + /// 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) + public static FdbFutureArray FromHandleArray([NotNull] FutureHandle[] handles, [NotNull] Func selector, CancellationToken ct) { - return new FdbFutureArray(handles, selector, cancellationToken); + return new FdbFutureArray(handles, selector, ct); } /// Wrap a FDBFuture* pointer into a /// Type of the result of the task /// FDBFuture* pointer /// Lambda that will be called once the future completes successfully, to extract the result from the future handle. - /// Optional cancellation token that can be used to cancel the future + /// 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) + public static Task CreateTaskFromHandle([NotNull] FutureHandle handle, [NotNull] Func continuation, CancellationToken ct) { - return new FdbFutureSingle(handle, continuation, cancellationToken).Task; + return new FdbFutureSingle(handle, continuation, ct).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 successfully, to extract the result from the future handle. - /// Optional cancellation token that can be used to cancel the future + /// 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. [ItemNotNull] - public static Task CreateTaskFromHandleArray([NotNull] FutureHandle[] handles, [NotNull] Func continuation, CancellationToken cancellationToken) + public static Task CreateTaskFromHandleArray([NotNull] FutureHandle[] handles, [NotNull] Func continuation, CancellationToken ct) { // 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; + return new FdbFutureArray(handles, continuation, ct).Task; } } @@ -409,11 +409,11 @@ internal static FdbFuture GetFutureFromCallbackParameter(IntPtr parameter) #region Cancellation... - protected void RegisterForCancellation(CancellationToken cancellationToken) + protected void RegisterForCancellation(CancellationToken ct) { //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( + m_ctr = ct.Register( (_state) => { CancellationHandler(_state); }, this, false diff --git a/FoundationDB.Client/Native/FdbFutureArray.cs b/FoundationDB.Client/Native/FdbFutureArray.cs index 3f6e3296e..5546179be 100644 --- a/FoundationDB.Client/Native/FdbFutureArray.cs +++ b/FoundationDB.Client/Native/FdbFutureArray.cs @@ -55,11 +55,11 @@ internal sealed class FdbFutureArray : FdbFuture #region Constructors... - internal FdbFutureArray([NotNull] FutureHandle[] handles, [NotNull] Func selector, CancellationToken cancellationToken) + internal FdbFutureArray([NotNull] FutureHandle[] handles, [NotNull] Func selector, CancellationToken ct) { - 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"); + if (handles == null) throw new ArgumentNullException(nameof(handles)); + if (handles.Length == 0) throw new ArgumentException("Handle array cannot be empty", nameof(handles)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); m_handles = handles; m_resultSelector = selector; @@ -68,7 +68,7 @@ internal FdbFutureArray([NotNull] FutureHandle[] handles, [NotNull] Func : FdbFuture #region Constructors... - internal FdbFutureSingle([NotNull] FutureHandle handle, [NotNull] Func selector, CancellationToken cancellationToken) + internal FdbFutureSingle([NotNull] FutureHandle handle, [NotNull] Func selector, CancellationToken ct) { - if (handle == null) throw new ArgumentNullException("handle"); - if (selector == null) throw new ArgumentNullException("selector"); + if (handle == null) throw new ArgumentNullException(nameof(handle)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); m_handle = handle; m_resultSelector = selector; @@ -82,9 +82,9 @@ internal FdbFutureSingle([NotNull] FutureHandle handle, [NotNull] Func CreateClusterAsync(string clusterFile, CancellationToken cancellationToken) + public static Task CreateClusterAsync(string clusterFile, CancellationToken ct) { var future = FdbNative.CreateCluster(clusterFile); return FdbFuture.CreateTaskFromHandle(future, @@ -62,7 +62,7 @@ public static Task CreateClusterAsync(string clusterFile, Ca var handler = new FdbNativeCluster(cluster); return (IFdbClusterHandler) handler; }, - cancellationToken + ct ); } @@ -92,9 +92,9 @@ public void SetOption(FdbClusterOption option, Slice data) } } - public Task OpenDatabaseAsync(string databaseName, CancellationToken cancellationToken) + public Task OpenDatabaseAsync(string databaseName, CancellationToken ct) { - if (cancellationToken.IsCancellationRequested) return Task.FromCanceled(cancellationToken); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var future = FdbNative.ClusterCreateDatabase(m_handle, databaseName); return FdbFuture.CreateTaskFromHandle( @@ -111,7 +111,7 @@ public Task OpenDatabaseAsync(string databaseName, Cancella var handler = new FdbNativeDatabase(database); return (IFdbDatabaseHandler) handler; }, - cancellationToken + ct ); } diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index 74174e25d..dabb06068 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -131,7 +131,7 @@ public void SetOption(FdbTransactionOption option, Slice data) #region Reading... - public Task GetReadVersionAsync(CancellationToken cancellationToken) + public Task GetReadVersionAsync(CancellationToken ct) { var future = FdbNative.TransactionGetReadVersion(m_handle); return FdbFuture.CreateTaskFromHandle(future, @@ -145,7 +145,7 @@ public Task GetReadVersionAsync(CancellationToken cancellationToken) Fdb.DieOnError(err); return version; }, - cancellationToken + ct ); } @@ -179,13 +179,13 @@ private static Slice GetValueResultBytes(FutureHandle h) return result; } - public Task GetAsync(Slice key, bool snapshot, CancellationToken cancellationToken) + public Task GetAsync(Slice key, bool snapshot, CancellationToken ct) { var future = FdbNative.TransactionGet(m_handle, key, snapshot); - return FdbFuture.CreateTaskFromHandle(future, (h) => GetValueResultBytes(h), cancellationToken); + return FdbFuture.CreateTaskFromHandle(future, (h) => GetValueResultBytes(h), ct); } - public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationToken cancellationToken) + public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationToken ct) { Contract.Requires(keys != null); @@ -208,7 +208,7 @@ public Task GetValuesAsync(Slice[] keys, bool snapshot, CancellationTok } throw; } - return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetValueResultBytes(h), cancellationToken); + return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetValueResultBytes(h), ct); } /// Extract a chunk of result from a completed Future @@ -228,7 +228,7 @@ private static KeyValuePair[] GetKeyValueArrayResult(FutureHandle /// Asynchronously fetch a new page of results /// True if Chunk contains a new page of results. False if all results have been read. - public Task GetRangeAsync(KeySelector begin, KeySelector end, FdbRangeOptions options, int iteration, bool snapshot, CancellationToken cancellationToken) + public Task GetRangeAsync(KeySelector begin, KeySelector end, FdbRangeOptions options, int iteration, bool snapshot, CancellationToken ct) { Contract.Requires(options != null); @@ -245,7 +245,7 @@ public Task GetRangeAsync(KeySelector begin, KeySelector end, Fdb return new FdbRangeChunk(hasMore, chunk, iteration, reversed); }, - cancellationToken + ct ); } @@ -262,17 +262,17 @@ private static Slice GetKeyResult(FutureHandle h) return result; } - public Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken cancellationToken) + public Task GetKeyAsync(KeySelector selector, bool snapshot, CancellationToken ct) { var future = FdbNative.TransactionGetKey(m_handle, selector, snapshot); return FdbFuture.CreateTaskFromHandle( future, (h) => GetKeyResult(h), - cancellationToken + ct ); } - public Task GetKeysAsync(KeySelector[] selectors, bool snapshot, CancellationToken cancellationToken) + public Task GetKeysAsync(KeySelector[] selectors, bool snapshot, CancellationToken ct) { Contract.Requires(selectors != null); @@ -293,7 +293,7 @@ public Task GetKeysAsync(KeySelector[] selectors, bool snapshot, Cancel } throw; } - return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetKeyResult(h), cancellationToken); + return FdbFuture.CreateTaskFromHandleArray(futures, (h) => GetKeyResult(h), ct); } @@ -354,13 +354,13 @@ private static string[] GetStringArrayResult(FutureHandle h) return result; } - public Task GetAddressesForKeyAsync(Slice key, CancellationToken cancellationToken) + public Task GetAddressesForKeyAsync(Slice key, CancellationToken ct) { var future = FdbNative.TransactionGetAddressesForKey(m_handle, key); return FdbFuture.CreateTaskFromHandle( future, (h) => GetStringArrayResult(h), - cancellationToken + ct ); } @@ -368,11 +368,11 @@ public Task GetAddressesForKeyAsync(Slice key, CancellationToken cance #region Watches... - public FdbWatch Watch(Slice key, CancellationToken cancellationToken) + public FdbWatch Watch(Slice key, CancellationToken ct) { var future = FdbNative.TransactionWatch(m_handle, key); return new FdbWatch( - FdbFuture.FromHandle(future, (h) => key, cancellationToken), + FdbFuture.FromHandle(future, (h) => key, ct), key, Slice.Nil ); @@ -400,16 +400,16 @@ public long GetCommittedVersion() /// /// 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. - public Task CommitAsync(CancellationToken cancellationToken) + public Task CommitAsync(CancellationToken ct) { var future = FdbNative.TransactionCommit(m_handle); - return FdbFuture.CreateTaskFromHandle(future, (h) => null, cancellationToken); + return FdbFuture.CreateTaskFromHandle(future, (h) => null, ct); } - public Task OnErrorAsync(FdbError code, CancellationToken cancellationToken) + public Task OnErrorAsync(FdbError code, CancellationToken ct) { var future = FdbNative.TransactionOnError(m_handle, code); - return FdbFuture.CreateTaskFromHandle(future, (h) => { ResetInternal(); return null; }, cancellationToken); + return FdbFuture.CreateTaskFromHandle(future, (h) => { ResetInternal(); return null; }, ct); } public void Reset() diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index 6ab4ac552..7163f009c 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -49,12 +49,12 @@ public static class Directory { /// Open a named partition of the default cluster /// Path of the named partition to open - /// Token used to cancel this operation + /// Token used to cancel this operation /// Returns a new database instance that will only be able to read and write inside the specified partition. If the partition does not exist, it will be automatically created [ItemNotNull] - public static Task OpenNamedPartitionAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public static Task OpenNamedPartitionAsync([NotNull] IEnumerable path, CancellationToken ct) { - return OpenNamedPartitionAsync(clusterFile: null, dbName: null, path: path, readOnly: false, cancellationToken: cancellationToken); + return OpenNamedPartitionAsync(clusterFile: null, dbName: null, path: path, readOnly: false, ct: ct); } /// Open a named partition of a specific cluster @@ -62,14 +62,14 @@ public static Task OpenNamedPartitionAsync([NotNull] IEnumerableName of the database, or "DB" if not specified. /// Path of the named partition to open /// If true, the database instance will only allow read operations - /// Token used to cancel this operation + /// Token used to cancel this operation /// Returns a new database instance that will only be able to read and write inside the specified partition. If the partition does not exist, it will be automatically created [ItemNotNull] - public static async Task OpenNamedPartitionAsync(string clusterFile, string dbName, [NotNull] IEnumerable path, bool readOnly, CancellationToken cancellationToken) + public static async Task OpenNamedPartitionAsync(string clusterFile, string dbName, [NotNull] IEnumerable path, bool readOnly, CancellationToken ct) { - if (path == null) throw new ArgumentNullException("path"); + if (path == null) throw new ArgumentNullException(nameof(path)); var partitionPath = path.ToList(); - if (partitionPath.Count == 0) throw new ArgumentException("The path to the named partition cannot be empty", "path"); + if (partitionPath.Count == 0) throw new ArgumentException("The path to the named partition cannot be empty", nameof(path)); // looks at the global partition table for the specified named partition @@ -78,12 +78,12 @@ public static async Task OpenNamedPartitionAsync(string clusterFil var rootSpace = KeySubspace.Empty; try { - db = await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, cancellationToken: cancellationToken).ConfigureAwait(false); + db = await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, ct: ct).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)); // look up in the root layer for the named partition - var descriptor = await rootLayer.CreateOrOpenAsync(db, partitionPath, layer: FdbDirectoryPartition.LayerId, cancellationToken: cancellationToken).ConfigureAwait(false); + var descriptor = await rootLayer.CreateOrOpenAsync(db, partitionPath, layer: FdbDirectoryPartition.LayerId, ct: ct).ConfigureAwait(false); if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", String.Format("Found named partition '{0}' at prefix {1}", descriptor.FullName, descriptor)); // we have to chroot the database to the new prefix, and create a new DirectoryLayer with a new '/' @@ -106,13 +106,13 @@ public static async Task OpenNamedPartitionAsync(string clusterFil /// List and open the sub-directories of the given directory /// Database used for the operation /// Parent directory - /// Token used to cancel this operation + /// Token used to cancel this operation /// Dictionary of all the sub directories of the directory. [ItemNotNull] - public static async Task> BrowseAsync([NotNull] IFdbDatabase db, [NotNull] IFdbDirectory parent, CancellationToken cancellationToken) + public static async Task> BrowseAsync([NotNull] IFdbDatabase db, [NotNull] IFdbDirectory parent, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (parent == null) throw new ArgumentNullException("parent"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (parent == null) throw new ArgumentNullException(nameof(parent)); return await db.ReadAsync(async (tr) => { @@ -122,13 +122,13 @@ public static async Task> BrowseAsync([ // open all the subdirectories var folders = await names .ToAsyncEnumerable() - .SelectAsync((name, ct) => parent.OpenAsync(tr, name)) + .SelectAsync((name, _) => parent.OpenAsync(tr, name)) .ToListAsync(); // map the result return folders.ToDictionary(ds => ds.Name); - }, cancellationToken).ConfigureAwait(false); + }, ct).ConfigureAwait(false); } } diff --git a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs index 3b85fc27b..e7d863ee8 100644 --- a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs @@ -101,36 +101,36 @@ Task IFdbDirectory.ChangeLayerAsync(IFdbTransaction trans, /// If the subdirectory does not exist, it is created (creating intermediate subdirectories if necessary). /// If layer is specified, it is checked against the layer of an existing subdirectory or set as the layer of a new subdirectory. /// - public Task CreateOrOpenAsync([NotNull] string name, CancellationToken cancellationToken) + public Task CreateOrOpenAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new [] { name }, Slice.Nil), ct); } /// Opens a subdirectory with the given path. /// If the subdirectory does not exist, it is created (creating intermediate subdirectories if necessary). /// If layer is specified, it is checked against the layer of an existing subdirectory or set as the layer of a new subdirectory. /// - public Task CreateOrOpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) + public Task CreateOrOpenAsync([NotNull] string name, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, new[] { name }, layer), ct); } /// Opens a subdirectory with the given path. /// If the subdirectory does not exist, it is created (creating intermediate subdirectories if necessary). /// If layer is specified, it is checked against the layer of an existing subdirectory or set as the layer of a new subdirectory. /// - public Task CreateOrOpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public Task CreateOrOpenAsync([NotNull] IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, Slice.Nil), ct); } /// Opens a subdirectory with the given path. /// If the subdirectory does not exist, it is created (creating intermediate subdirectories if necessary). /// If layer is specified, it is checked against the layer of an existing subdirectory or set as the layer of a new subdirectory. /// - public Task CreateOrOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public Task CreateOrOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateOrOpenAsync(tr, path, layer), ct); } Task IFdbDirectory.CreateOrOpenAsync(IFdbTransaction trans, IEnumerable subPath, Slice layer) @@ -146,9 +146,9 @@ Task IFdbDirectory.CreateOrOpenAsync(IFdbTransaction trans /// An exception is thrown if the subdirectory does not exist, or if a layer is specified and a different layer was specified when the subdirectory was created. /// /// Name of the subdirectory to open - public Task OpenAsync([NotNull] string name, CancellationToken cancellationToken) + public Task OpenAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new [] { name }, Slice.Nil), ct); } /// Opens a subdirectory with the given . @@ -156,18 +156,18 @@ public Task OpenAsync([NotNull] string name, CancellationT /// /// Name of the subdirectory to open /// Expected layer id for the subdirectory (optional) - public Task OpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) + public Task OpenAsync([NotNull] string name, Slice layer, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, new[] { name }, layer), ct); } /// Opens a subdirectory with the given . /// An exception is thrown if the subdirectory does not exist, or if a layer is specified and a different layer was specified when the subdirectory was created. /// /// Relative path of the subdirectory to open - public Task OpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public Task OpenAsync([NotNull] IEnumerable path, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, Slice.Nil), ct); } /// Opens a subdirectory with the given . @@ -175,9 +175,9 @@ public Task OpenAsync([NotNull] IEnumerable path, /// /// Relative path of the subdirectory to open /// Expected layer id for the subdirectory (optional) - public Task OpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public Task OpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.OpenAsync(tr, path, layer), ct); } Task IFdbDirectory.OpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer) @@ -194,9 +194,9 @@ Task IFdbDirectory.OpenAsync(IFdbReadOnlyTransaction trans /// /// Name of the subdirectory to open /// Returns the directory if it exists, or null if it was not found - public Task TryOpenAsync([NotNull] string name, CancellationToken cancellationToken) + public Task TryOpenAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new [] { name }, Slice.Nil), ct); } /// Opens a subdirectory with the given . @@ -205,9 +205,9 @@ public Task TryOpenAsync([NotNull] string name, Cancellati /// Name of the subdirectory to open /// Expected layer id for the subdirectory (optional) /// Returns the directory if it exists, or null if it was not found - public Task TryOpenAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) + public Task TryOpenAsync([NotNull] string name, Slice layer, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, new[] { name }, layer), ct); } /// Opens a subdirectory with the given . @@ -215,9 +215,9 @@ public Task TryOpenAsync([NotNull] string name, Slice laye /// /// Relative path of the subdirectory to open /// Returns the directory if it exists, or null if it was not found - public Task TryOpenAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public Task TryOpenAsync([NotNull] IEnumerable path, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, Slice.Nil), ct); } /// Opens a subdirectory with the given . @@ -226,9 +226,9 @@ public Task TryOpenAsync([NotNull] IEnumerable pat /// Relative path of the subdirectory to open /// Expected layer id for the subdirectory (optional) /// Returns the directory if it exists, or null if it was not found - public Task TryOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public Task TryOpenAsync([NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, layer), cancellationToken); + return this.Database.ReadAsync((tr) => this.Directory.TryOpenAsync(tr, path, layer), ct); } Task IFdbDirectory.TryOpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer) @@ -240,24 +240,24 @@ Task IFdbDirectory.TryOpenAsync(IFdbReadOnlyTransaction tr #region Create... - public Task CreateAsync([NotNull] string name, CancellationToken cancellationToken) + public Task CreateAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new[] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new[] { name }, Slice.Nil), ct); } - public Task CreateAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) + public Task CreateAsync([NotNull] string name, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new [] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, new [] { name }, layer), ct); } - public Task CreateAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public Task CreateAsync([NotNull] IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, Slice.Nil), ct); } - public Task CreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public Task CreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.CreateAsync(tr, path, layer), ct); } Task IFdbDirectory.CreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer) @@ -269,24 +269,24 @@ Task IFdbDirectory.CreateAsync(IFdbTransaction trans, IEnu #region TryCreate... - public Task TryCreateAsync([NotNull] string name, CancellationToken cancellationToken) + public Task TryCreateAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new [] { name }, Slice.Nil), ct); } - public Task TryCreateAsync([NotNull] string name, Slice layer, CancellationToken cancellationToken) + public Task TryCreateAsync([NotNull] string name, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new[] { name }, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, new[] { name }, layer), ct); } - public Task TryCreateAsync([NotNull] IEnumerable path, CancellationToken cancellationToken) + public Task TryCreateAsync([NotNull] IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, Slice.Nil), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, Slice.Nil), ct); } - public Task TryCreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken cancellationToken) + public Task TryCreateAsync([NotNull] IEnumerable path, Slice layer, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, layer), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryCreateAsync(tr, path, layer), ct); } Task IFdbDirectory.TryCreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer) @@ -302,18 +302,18 @@ Task IFdbDirectory.TryCreateAsync(IFdbTransaction trans, I /// Name of the directory to create /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// The directory will be created with the given physical prefix; otherwise a prefix is allocated automatically. - public Task RegisterAsync([NotNull] string name, Slice layer, Slice prefix, CancellationToken cancellationToken) + public Task RegisterAsync([NotNull] string name, Slice layer, Slice prefix, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, new[] { name }, layer, prefix), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, new[] { name }, layer, prefix), ct); } /// Registers an existing prefix as a directory with the given (creating parent directories if necessary). This method is only indented for advanced use cases. /// Path of the directory to create /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// The directory will be created with the given physical prefix; otherwise a prefix is allocated automatically. - public Task RegisterAsync([NotNull] IEnumerable path, Slice layer, Slice prefix, CancellationToken cancellationToken) + public Task RegisterAsync([NotNull] IEnumerable path, Slice layer, Slice prefix, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, path, layer, prefix), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RegisterAsync(tr, path, layer, prefix), ct); } Task IFdbDirectory.RegisterAsync(IFdbTransaction trans, IEnumerable path, Slice layer, Slice prefix) @@ -325,9 +325,9 @@ Task IFdbDirectory.RegisterAsync(IFdbTransaction trans, IE #region Move... - public Task MoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public Task MoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.MoveAsync(tr, oldPath, newPath), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.MoveAsync(tr, oldPath, newPath), ct); } Task IFdbDirectory.MoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) @@ -339,9 +339,9 @@ Task IFdbDirectory.MoveAsync(IFdbTransaction trans, IEnume #region TryMove... - public Task TryMoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken cancellationToken) + public Task TryMoveAsync([NotNull] IEnumerable oldPath, [NotNull] IEnumerable newPath, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryMoveAsync(tr, oldPath, newPath), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryMoveAsync(tr, oldPath, newPath), ct); } Task IFdbDirectory.TryMoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) @@ -371,14 +371,14 @@ public Task TryMoveToAsync(IFdbTransaction trans, IEnumera #region Remove... - public Task RemoveAsync([NotNull] string name, CancellationToken cancellationToken) + public Task RemoveAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, new string[] { name }), ct); } - public Task RemoveAsync(IEnumerable path, CancellationToken cancellationToken) + public Task RemoveAsync(IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.RemoveAsync(tr, path), ct); } Task IFdbDirectory.RemoveAsync(IFdbTransaction trans, IEnumerable path) @@ -390,14 +390,14 @@ Task IFdbDirectory.RemoveAsync(IFdbTransaction trans, IEnumerable path) #region TryRemove... - public Task TryRemoveAsync([NotNull] string name, CancellationToken cancellationToken) + public Task TryRemoveAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, new [] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, new [] { name }), ct); } - public Task TryRemoveAsync(IEnumerable path, CancellationToken cancellationToken) + public Task TryRemoveAsync(IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryRemoveAsync(tr, path), ct); } Task IFdbDirectory.TryRemoveAsync(IFdbTransaction trans, IEnumerable path) @@ -409,14 +409,14 @@ Task IFdbDirectory.TryRemoveAsync(IFdbTransaction trans, IEnumerable ExistsAsync([NotNull] string name, CancellationToken cancellationToken) + public Task ExistsAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, new [] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, new [] { name }), ct); } - public Task ExistsAsync(IEnumerable path, CancellationToken cancellationToken) + public Task ExistsAsync(IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ExistsAsync(tr, path), ct); } Task IFdbDirectory.ExistsAsync(IFdbReadOnlyTransaction trans, IEnumerable path) @@ -429,21 +429,21 @@ Task IFdbDirectory.ExistsAsync(IFdbReadOnlyTransaction trans, IEnumerable< #region List... /// Returns the list of all the top level directories of this database instance. - public Task> ListAsync(CancellationToken cancellationToken) + public Task> ListAsync(CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr), ct); } /// Returns the list of all the top level directories of this database instance. - public Task> ListAsync([NotNull] string name, CancellationToken cancellationToken) + public Task> ListAsync([NotNull] string name, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, new string[] { name }), ct); } /// Returns the list of all the top level directories of this database instance. - public Task> ListAsync(IEnumerable path, CancellationToken cancellationToken) + public Task> ListAsync(IEnumerable path, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.ListAsync(tr, path), ct); } Task> IFdbDirectory.ListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) @@ -456,27 +456,27 @@ Task> IFdbDirectory.ListAsync(IFdbReadOnlyTransaction trans, IEnume #region TryList... /// Returns the list of all the top level directories of this database instance. - public Task> TryListAsync(CancellationToken cancellationToken) + public Task> TryListAsync(CancellationToken ct) { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr), ct); } /// Returns the list of all the top level directories of this database instance. - public Task> TryListAsync([NotNull] string name, CancellationToken cancellationToken) + public Task> TryListAsync([NotNull] string name, CancellationToken ct) { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, new string[] { name }), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, new string[] { name }), ct); } /// Returns the list of all the top level directories of this database instance. - public Task> TryListAsync(IEnumerable path, CancellationToken cancellationToken) + public Task> TryListAsync(IEnumerable path, CancellationToken ct) { //REVIEW: is it possible for this method to fail on a top-level db partition? // => it not, should be removed because it is a duplicate of ListAsync(..) - return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, path), cancellationToken); + return this.Database.ReadWriteAsync((tr) => this.Directory.TryListAsync(tr, path), ct); } Task> IFdbDirectory.TryListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) diff --git a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index da7aaa336..bd2bba752 100644 --- a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -169,12 +169,12 @@ public static void ClearRange(this IFdbTransaction trans, [NotNull] IKeySubspace } /// Clear the entire content of a subspace - public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IKeySubspace subspace, CancellationToken cancellationToken) + public static Task ClearRangeAsync(this IFdbRetryable db, [NotNull] IKeySubspace subspace, CancellationToken ct) { Contract.NotNull(db, nameof(db)); Contract.NotNull(subspace, nameof(subspace)); - return db.WriteAsync((tr) => ClearRange(tr, subspace), cancellationToken); + return db.WriteAsync((tr) => ClearRange(tr, subspace), ct); } /// Returns all the keys inside of a subspace diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index 2122271ee..d2a46f38e 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -169,8 +169,8 @@ internal static unsafe Slice Create(byte* ptr, int count) { return ptr == null ? Slice.Nil : Slice.Empty; } - if (ptr == null) throw new ArgumentNullException("ptr"); - if (count < 0) throw new ArgumentOutOfRangeException("count"); + if (ptr == null) throw new ArgumentNullException(nameof(ptr)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count)); if (count == 1) { @@ -203,8 +203,8 @@ public static Slice Repeat(byte value, int count) /// Warning: is not thread-safe ! If the instance is shared between threads, then it needs to be locked before calling this method. public static Slice Random([NotNull] Random prng, int count) { - if (prng == null) throw new ArgumentNullException("prng"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count cannot be negative"); + if (prng == null) throw new ArgumentNullException(nameof(prng)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); if (count == 0) return Slice.Empty; var bytes = new byte[count]; @@ -220,8 +220,8 @@ public static Slice Random([NotNull] Random prng, int count) /// Warning: All RNG implementations may not be thread-safe ! If the instance is shared between threads, then it may need to be locked before calling this method. public static Slice Random([NotNull] System.Security.Cryptography.RandomNumberGenerator rng, int count, bool nonZeroBytes = false) { - if (rng == null) throw new ArgumentNullException("rng"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count cannot be negative"); + if (rng == null) throw new ArgumentNullException(nameof(rng)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); if (count == 0) return Slice.Empty; var bytes = new byte[count]; @@ -281,7 +281,7 @@ public static int Find(Slice source, Slice value) /// If is null. public static Slice Join(Slice separator, [NotNull] Slice[] values) { - if (values == null) throw new ArgumentNullException("values"); + if (values == null) throw new ArgumentNullException(nameof(values)); int count = values.Length; if (count == 0) return Slice.Empty; @@ -302,11 +302,11 @@ public static Slice Join(Slice separator, [NotNull] Slice[] values, int startInd // Note: this method is modeled after String.Join() and should behave the same // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) - if (values == null) throw new ArgumentNullException("values"); + if (values == null) throw new ArgumentNullException(nameof(values)); //REVIEW: support negative indexing ? - if (startIndex < 0) throw new ArgumentOutOfRangeException("startIndex", startIndex, "Start index must be a positive integer"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count must be a positive integer"); - if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException("startIndex", startIndex, "Start index must fit within the array"); + if (startIndex < 0) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); + if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); if (count == 0) return Slice.Empty; if (count == 1) return values[startIndex]; @@ -337,7 +337,7 @@ public static Slice Join(Slice separator, [NotNull] Slice[] values, int startInd /// If is null. public static Slice Join(Slice separator, [NotNull] IEnumerable values) { - if (values == null) throw new ArgumentNullException("values"); + if (values == null) throw new ArgumentNullException(nameof(values)); var array = (values as Slice[]) ?? values.ToArray(); return Join(separator, array, 0, array.Length); } @@ -356,11 +356,11 @@ public static byte[] JoinBytes(Slice separator, [NotNull] Slice[] values, int st // Note: this method is modeled after String.Join() and should behave the same // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) - if (values == null) throw new ArgumentNullException("values"); + if (values == null) throw new ArgumentNullException(nameof(values)); //REVIEW: support negative indexing ? - if (startIndex < 0) throw new ArgumentOutOfRangeException("startIndex", startIndex, "Start index must be a positive integer"); - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Count must be a positive integer"); - if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException("startIndex", startIndex, "Start index must fit within the array"); + if (startIndex < 0) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); + if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); if (count == 0) return Slice.EmptyArray; if (count == 1) return values[startIndex].GetBytes() ?? Slice.EmptyArray; @@ -392,7 +392,7 @@ public static byte[] JoinBytes(Slice separator, [NotNull] Slice[] values, int st [NotNull] public static byte[] JoinBytes(Slice separator, [NotNull] IEnumerable values) { - if (values == null) throw new ArgumentNullException("values"); + if (values == null) throw new ArgumentNullException(nameof(values)); var array = (values as Slice[]) ?? values.ToArray(); return JoinBytes(separator, array, 0, array.Length); } @@ -417,7 +417,7 @@ public static Slice[] Split(Slice input, Slice separator, StringSplitOptions opt var list = new List(); - if (separator.Count <= 0) throw new ArgumentException("Separator must have at least one byte", "separator"); + if (separator.Count <= 0) throw new ArgumentException("Separator must have at least one byte", nameof(separator)); if (options < StringSplitOptions.None || options > StringSplitOptions.RemoveEmptyEntries) throw new ArgumentException("options"); bool skipEmpty = options.HasFlag(StringSplitOptions.RemoveEmptyEntries); @@ -893,7 +893,7 @@ public static Slice FromHexa(string hexaString) hexaString = hexaString.Replace(" ", ""); } - if ((hexaString.Length & 1) != 0) throw new ArgumentException("Hexadecimal string must be of even length", "hexaString"); + if ((hexaString.Length & 1) != 0) throw new ArgumentException("Hexadecimal string must be of even length", nameof(hexaString)); var buffer = new byte[hexaString.Length >> 1]; for (int i = 0; i < hexaString.Length; i += 2) @@ -941,8 +941,8 @@ public byte[] GetBytes(int offset, int count) { //TODO: throw if this.Array == null ? (what does "Slice.Nil.GetBytes(..., 0)" mean ?) - if (offset < 0) throw new ArgumentOutOfRangeException("offset"); - if (count < 0 || offset + count > this.Count) throw new ArgumentOutOfRangeException("count"); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset)); + if (count < 0 || offset + count > this.Count) throw new ArgumentOutOfRangeException(nameof(count)); if (count == 0) return this.Array == null ? null : Slice.EmptyArray; SliceHelpers.EnsureSliceIsValid(ref this); @@ -1226,7 +1226,7 @@ public ushort ToUInt16BE() [Pure] public ushort ReadUInt16(int offset, int bytes) { - if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException(nameof(bytes)); var buffer = this.Array; int p = UnsafeMapToOffset(offset); @@ -1246,7 +1246,7 @@ public ushort ReadUInt16(int offset, int bytes) [Pure] public ushort ReadUInt16BE(int offset, int bytes) { - if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException(nameof(bytes)); var buffer = this.Array; int p = UnsafeMapToOffset(offset); @@ -1361,7 +1361,7 @@ public uint ToUInt32BE() [Pure] public uint ReadUInt32(int offset, int bytes) { - if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException(nameof(bytes)); if (bytes == 0) return 0; var buffer = this.Array; @@ -1383,7 +1383,7 @@ public uint ReadUInt32(int offset, int bytes) [Pure] public uint ReadUInt32BE(int offset, int bytes) { - if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException(nameof(bytes)); if (bytes == 0) return 0; var buffer = this.Array; @@ -1498,7 +1498,7 @@ public ulong ToUInt64BE() [Pure] public ulong ReadUInt64(int offset, int bytes) { - if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException(nameof(bytes)); if (bytes == 0) return 0UL; var buffer = this.Array; @@ -1520,7 +1520,7 @@ public ulong ReadUInt64(int offset, int bytes) [Pure] public ulong ReadUInt64BE(int offset, int bytes) { - if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException("bytes"); + if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException(nameof(bytes)); if (bytes == 0) return 0UL; var buffer = this.Array; @@ -1734,8 +1734,8 @@ public Slice Substring(int offset) // negative values means from the end if (offset < 0) offset = this.Count + offset; - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less then start of the slice"); - if (offset > this.Count) throw new ArgumentOutOfRangeException("offset", "Offset cannot be larger than end of slice"); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less then start of the slice"); + if (offset > this.Count) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be larger than end of slice"); return this.Count == offset ? Slice.Empty : new Slice(this.Array, this.Offset + offset, this.Count - offset); } @@ -1760,9 +1760,9 @@ public Slice Substring(int offset, int count) // negative values means from the end if (offset < 0) offset = this.Count + offset; - if (offset < 0 || offset >= this.Count) throw new ArgumentOutOfRangeException("offset", "Offset must be inside the slice"); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count must be a positive integer"); - if (offset > this.Count - count) throw new ArgumentOutOfRangeException("count", "Offset and count must refer to a location within the slice"); + if (offset < 0 || offset >= this.Count) throw new ArgumentOutOfRangeException(nameof(offset), "Offset must be inside the slice"); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count must be a positive integer"); + if (offset > this.Count - count) throw new ArgumentOutOfRangeException(nameof(count), "Offset and count must refer to a location within the slice"); return new Slice(this.Array, this.Offset + offset, count); } @@ -1794,7 +1794,7 @@ public int IndexOf(Slice value) public int IndexOf(Slice value, int startIndex) { //REVIEW: support negative indexing ? - if (startIndex < 0 || startIndex > this.Count) throw new ArgumentOutOfRangeException("startIndex", startIndex, "Start index must be inside the buffer"); + if (startIndex < 0 || startIndex > this.Count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be inside the buffer"); if (this.Count == 0) { return value.Count == 0 ? startIndex : -1; @@ -1809,7 +1809,7 @@ public int IndexOf(Slice value, int startIndex) [Pure] public bool StartsWith(Slice value) { - if (!value.HasValue) throw new ArgumentNullException("value"); + if (!value.HasValue) throw new ArgumentNullException(nameof(value)); // any strings starts with the empty string if (value.Count == 0) return true; @@ -1826,7 +1826,7 @@ public bool StartsWith(Slice value) [Pure] public bool EndsWith(Slice value) { - if (!value.HasValue) throw new ArgumentNullException("value"); + if (!value.HasValue) throw new ArgumentNullException(nameof(value)); // any strings ends with the empty string if (value.Count == 0) return true; @@ -1896,7 +1896,7 @@ public Slice Concat(Slice tail) [Pure, NotNull] public Slice[] ConcatRange([NotNull] Slice[] slices) { - if (slices == null) throw new ArgumentNullException("slices"); + if (slices == null) throw new ArgumentNullException(nameof(slices)); SliceHelpers.EnsureSliceIsValid(ref this); // pre-allocate by computing final buffer capacity @@ -1923,7 +1923,7 @@ public Slice[] ConcatRange([NotNull] Slice[] slices) [Pure, NotNull] public Slice[] ConcatRange([NotNull] IEnumerable slices) { - if (slices == null) throw new ArgumentNullException("slices"); + if (slices == null) throw new ArgumentNullException(nameof(slices)); // use optimized version for arrays var array = slices as Slice[]; @@ -1981,7 +1981,7 @@ public static Slice Concat(params Slice[] args) /// This method is optmized to reduce the amount of memory allocated public static Slice[] ConcatRange(Slice prefix, IEnumerable slices) { - if (slices == null) throw new ArgumentNullException("slices"); + if (slices == null) throw new ArgumentNullException(nameof(slices)); if (prefix.IsNullOrEmpty) { // nothing to do, but we still need to copy the array @@ -2118,8 +2118,8 @@ public static implicit operator Slice(ArraySegment value) /// Smaller slice public static Slice operator -(Slice s, int n) { - if (n < 0) throw new ArgumentOutOfRangeException("n", "Cannot subtract a negative number from a slice"); - if (n > s.Count) throw new ArgumentOutOfRangeException("n", "Cannout substract more bytes than the slice contains"); + if (n < 0) throw new ArgumentOutOfRangeException(nameof(n), "Cannot subtract a negative number from a slice"); + if (n > s.Count) throw new ArgumentOutOfRangeException(nameof(n), "Cannout substract more bytes than the slice contains"); if (n == 0) return s; if (n == s.Count) return Slice.Empty; @@ -2247,7 +2247,7 @@ public static Slice Unescape(string value) /// If the size of the stream exceeds or if it does not support reading. public static Slice FromStream([NotNull] Stream data) { - if (data == null) throw new ArgumentNullException("data"); + if (data == null) throw new ArgumentNullException(nameof(data)); // special case for empty values if (data == Stream.Null) return Slice.Nil; @@ -2271,13 +2271,13 @@ public static Slice FromStream([NotNull] Stream data) /// Asynchronously read the content of a stream into a slice /// Source stream, that must be in a readable state - /// Optional cancellation token for this operation + /// Optional cancellation token for this operation /// Slice containing the stream content (or if the stream is ) /// If is null. /// If the size of the stream exceeds or if it does not support reading. - public static Task FromStreamAsync([NotNull] Stream data, CancellationToken cancellationToken) + public static Task FromStreamAsync([NotNull] Stream data, CancellationToken ct) { - if (data == null) throw new ArgumentNullException("data"); + if (data == null) throw new ArgumentNullException(nameof(data)); // special case for empty values if (data == Stream.Null) return Task.FromResult(Slice.Nil); @@ -2287,7 +2287,7 @@ public static Task FromStreamAsync([NotNull] Stream data, CancellationTok if (data.Length > int.MaxValue) throw new InvalidOperationException("Streams of more than 2GB are not supported"); //TODO: other checks? - if (cancellationToken.IsCancellationRequested) return TaskHelpers.FromCancellation(cancellationToken); + if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); int length; checked { length = (int)data.Length; } @@ -2298,7 +2298,7 @@ public static Task FromStreamAsync([NotNull] Stream data, CancellationTok } // read asynchronoulsy - return LoadFromBlockingStreamAsync(data, length, 0, cancellationToken); + return LoadFromBlockingStreamAsync(data, length, 0, ct); } /// Read from a non-blocking stream that already contains all the data in memory (MemoryStream, UnmanagedStream, ...) @@ -2369,9 +2369,9 @@ private static Slice LoadFromBlockingStream([NotNull] Stream source, int length, /// Source stream /// Number of bytes to read from the stream /// If non zero, max amount of bytes to read in one chunk. If zero, tries to read everything at once - /// Optional cancellation token for this operation + /// Optional cancellation token for this operation /// Slice containing the loaded data - private static async Task LoadFromBlockingStreamAsync([NotNull] Stream source, int length, int chunkSize, CancellationToken cancellationToken) + private static async Task LoadFromBlockingStreamAsync([NotNull] Stream source, int length, int chunkSize, CancellationToken ct) { Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue && chunkSize >= 0); @@ -2385,7 +2385,7 @@ private static async Task LoadFromBlockingStreamAsync([NotNull] Stream so while (r > 0) { int c = Math.Min(r, chunkSize); - int n = await source.ReadAsync(buffer, p, c, cancellationToken); + int n = await source.ReadAsync(buffer, p, c, ct); if (n <= 0) throw new InvalidOperationException(String.Format("Unexpected end of stream at {0} / {1} bytes", p, length)); p += n; r -= n; diff --git a/FoundationDB.Client/Utils/SliceListStream.cs b/FoundationDB.Client/Utils/SliceListStream.cs index 7f748de17..b49313e38 100644 --- a/FoundationDB.Client/Utils/SliceListStream.cs +++ b/FoundationDB.Client/Utils/SliceListStream.cs @@ -49,13 +49,13 @@ public sealed class SliceListStream : Stream internal SliceListStream([NotNull] Slice[] slices) { - if (slices == null) throw new ArgumentNullException("slices"); + if (slices == null) throw new ArgumentNullException(nameof(slices)); Init(slices); } public SliceListStream([NotNull] IEnumerable slices) { - if (slices == null) throw new ArgumentNullException("slices"); + if (slices == null) throw new ArgumentNullException(nameof(slices)); Init(slices.ToArray()); } @@ -97,7 +97,7 @@ public override long Length public override long Seek(long offset, SeekOrigin origin) { if (m_slices == null) StreamIsClosed(); - if (offset > int.MaxValue) throw new ArgumentOutOfRangeException("offset"); + if (offset > int.MaxValue) throw new ArgumentOutOfRangeException(nameof(offset)); switch (origin) { @@ -247,13 +247,13 @@ public override int Read(byte[] buffer, int offset, int count) #if !NET_4_0 - public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken cancellationToken) + public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { ValidateBuffer(buffer, offset, count); - if (cancellationToken.IsCancellationRequested) + if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(cancellationToken); + return TaskHelpers.FromCancellation(ct); } try @@ -288,7 +288,7 @@ public override void Write(byte[] buffer, int offset, int count) #if !NET_4_0 - public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken cancellationToken) + public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { return TaskHelpers.FromException(new NotSupportedException()); } @@ -300,7 +300,7 @@ public override void Flush() // Not supported, but don't throw here } - public override Task FlushAsync(System.Threading.CancellationToken cancellationToken) + public override Task FlushAsync(System.Threading.CancellationToken ct) { // Not supported, but don't throw here return TaskHelpers.CompletedTask; @@ -310,9 +310,9 @@ public override Task FlushAsync(System.Threading.CancellationToken cancellationT private static void ValidateBuffer(byte[] buffer, int offset, int count) { - if (buffer == null) throw new ArgumentNullException("buffer"); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); + if (buffer == null) throw new ArgumentNullException(nameof(buffer)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); if (offset > buffer.Length - count) throw new ArgumentException("Offset and count must fit inside the buffer"); } diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/SliceStream.cs index afdb376d4..a4dbe887d 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/SliceStream.cs @@ -79,7 +79,7 @@ public override long Length public override long Seek(long offset, SeekOrigin origin) { if (!m_slice.HasValue) StreamIsClosed(); - if (offset > int.MaxValue) throw new ArgumentOutOfRangeException("offset"); + if (offset > int.MaxValue) throw new ArgumentOutOfRangeException(nameof(offset)); switch (origin) { @@ -180,13 +180,13 @@ public override int Read(byte[] buffer, int offset, int count) #if !NET_4_0 /// Asynchronously reads a sequence of bytes from the underlying slice and advances the position within the slice by the number of bytes read. - public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken cancellationToken) + public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { ValidateBuffer(buffer, offset, count); - if (cancellationToken.IsCancellationRequested) + if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(cancellationToken); + return TaskHelpers.FromCancellation(ct); } try @@ -204,12 +204,12 @@ public override Task ReadAsync(byte[] buffer, int offset, int count, System } /// Asynchronously reads the bytes from the underlying slice and writes them to another stream, using a specified buffer size and cancellation token. - public override Task CopyToAsync(Stream destination, int bufferSize, System.Threading.CancellationToken cancellationToken) + public override Task CopyToAsync(Stream destination, int bufferSize, System.Threading.CancellationToken ct) { Contract.Ensures(m_position >= 0 && m_position <= m_slice.Count); - if (destination == null) throw new ArgumentNullException("destination"); - if (!destination.CanWrite) throw new ArgumentException("The destination stream cannot be written to", "destination"); + if (destination == null) throw new ArgumentNullException(nameof(destination)); + if (!destination.CanWrite) throw new ArgumentException("The destination stream cannot be written to", nameof(destination)); int remaining = m_slice.Count - m_position; if (remaining <= 0) return TaskHelpers.CompletedTask; @@ -218,7 +218,7 @@ public override Task CopyToAsync(Stream destination, int bufferSize, System.Thre m_position += remaining; // we can write everyting in one go, so just call WriteAsync and return that - return destination.WriteAsync(m_slice.Array, m_slice.Offset, remaining, cancellationToken); + return destination.WriteAsync(m_slice.Array, m_slice.Offset, remaining, ct); } #endif @@ -242,7 +242,7 @@ public override void Write(byte[] buffer, int offset, int count) #if !NET_4_0 /// This methods is not supported - public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken cancellationToken) + public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { return TaskHelpers.FromException(new NotSupportedException()); } @@ -256,7 +256,7 @@ public override void Flush() } /// This methods does nothing. - public override Task FlushAsync(System.Threading.CancellationToken cancellationToken) + public override Task FlushAsync(System.Threading.CancellationToken ct) { // Not supported, but don't throw here return TaskHelpers.CompletedTask; @@ -266,9 +266,9 @@ public override Task FlushAsync(System.Threading.CancellationToken cancellationT private static void ValidateBuffer(byte[] buffer, int offset, int count) { - if (buffer == null) throw new ArgumentNullException("buffer"); - if (count < 0) throw new ArgumentOutOfRangeException("count", "Count cannot be less than zero"); - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); + if (buffer == null) throw new ArgumentNullException(nameof(buffer)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); if (offset > buffer.Length - count) throw new ArgumentException("Offset and count must fit inside the buffer"); } diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index a1a79521a..fee380684 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -49,10 +49,10 @@ public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) { - if (name == null) throw new ArgumentNullException("name"); - if (subspace == null) throw new ArgumentNullException("subspace"); - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); - if (valueEncoder == null) throw new ArgumentNullException("valueEncoder"); + if (name == null) throw new ArgumentNullException(nameof(name)); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); + if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); this.Name = name; this.Subspace = subspace; @@ -87,8 +87,8 @@ public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] /// If the map does not contain an entry with this key. public async Task GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); var data = await trans.GetAsync(this.Location.Keys.Encode(id)).ConfigureAwait(false); @@ -102,8 +102,8 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey /// Optional with the value of the entry it it exists, or an empty result if it is not present in the map. public async Task> TryGetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); var data = await trans.GetAsync(this.Location.Keys.Encode(id)).ConfigureAwait(false); @@ -118,8 +118,8 @@ public async Task> TryGetAsync([NotNull] IFdbReadOnlyTransactio /// If the entry did not exist, it will be created. If not, its value will be replace with . public void Set([NotNull] IFdbTransaction trans, TKey id, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); trans.Set(this.Location.Keys.Encode(id), this.ValueEncoder.EncodeValue(value)); } @@ -130,8 +130,8 @@ public void Set([NotNull] IFdbTransaction trans, TKey id, TValue value) /// If the entry did not exist, the operation will not do anything. public void Remove([NotNull] IFdbTransaction trans, TKey id) { - if (trans == null) throw new ArgumentNullException("trans"); - if (id == null) throw new ArgumentNullException("id"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (id == null) throw new ArgumentNullException(nameof(id)); trans.Clear(this.Location.Keys.Encode(id)); } @@ -143,7 +143,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey id) [NotNull] public IFdbAsyncEnumerable> All([NotNull] IFdbReadOnlyTransaction trans, FdbRangeOptions options = null) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return trans .GetRange(this.Location.ToRange(), options) @@ -156,8 +156,8 @@ public IFdbAsyncEnumerable> All([NotNull] IFdbReadOnl /// Array of results, in the same order as specified in . public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ids) { - if (trans == null) throw new ArgumentNullException("trans"); - if (ids == null) throw new ArgumentNullException("ids"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (ids == null) throw new ArgumentNullException(nameof(ids)); var results = await trans.GetValuesAsync(this.Location.Keys.Encode(ids)).ConfigureAwait(false); @@ -200,7 +200,7 @@ private KeyValuePair[] DecodeItems(KeyValuePair[] ba /// This will delete EVERYTHING in the map! public void Clear([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.ClearRange(this.Location.ToRange()); } @@ -210,18 +210,18 @@ public void Clear([NotNull] IFdbTransaction trans) /// Exports the content of this map out of the database, by using as many transactions as necessary. /// Database used for the operation /// Handler called for each entry in the map. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map. Any change made to the map while the export is running may be partially exported. - public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action> handler, CancellationToken cancellationToken) + public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); return Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - (batch, _, ct) => + (batch, _, __) => { foreach (var item in batch) { @@ -229,50 +229,50 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] ActionExports the content of this map out of the database, by using as many transactions as necessary. /// Database used for the operation /// Handler called for each entry in the map. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map. Any change made to the map while the export is running may be partially exported. - public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func, CancellationToken, Task> handler, CancellationToken cancellationToken) + public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func, CancellationToken, Task> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); return Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - async (batch, _, ct) => + async (batch, _, __) => { foreach (var item in batch) { - await handler(DecodeItem(item), cancellationToken); + await handler(DecodeItem(item), ct); } }, - cancellationToken + ct ); } /// Exports the content of this map out of the database, by using as many transactions as necessary. /// Database used for the operation /// Handler called for each batch of items in the map. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map, except that all the items in a single batch are from the same snapshot. Any change made to the map while the export is running may be partially exported. - public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action[]> handler, CancellationToken cancellationToken) + public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action[]> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); return Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - (batch, _, ct) => + (batch, _, __) => { if (batch.Length > 0) { @@ -280,26 +280,26 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] ActionExports the content of this map out of the database, by using as many transactions as necessary. /// Database used for the operation /// Handler called for each batch of items in the map. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map, except that all the items in a single batch are from the same snapshot. Any change made to the map while the export is running may be partially exported. - public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func[], CancellationToken, Task> handler, CancellationToken cancellationToken) + public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func[], CancellationToken, Task> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); return Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - (batch, _, ct) => handler(DecodeItems(batch), ct), - cancellationToken + (batch, _, tok) => handler(DecodeItems(batch), tok), + ct ); } @@ -307,13 +307,13 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] FuncDatabase used for the operation /// Handler that is called once before the first batch, to produce the initial state. /// Handler called for each batch of items in the map. It is given the previous state, and should return the updated state. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed and return the result of the last call to if there was at least one batch, or the result of if the map was empty. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map, except that all the items in a single batch are from the same snapshot. Any change made to the map while the export is running may be partially exported. - public async Task AggregateAsync([NotNull] IFdbDatabase db, Func init, [NotNull] Func[], TResult> handler, CancellationToken cancellationToken) + public async Task AggregateAsync([NotNull] IFdbDatabase db, Func init, [NotNull] Func[], TResult> handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); var state = default(TResult); if (init != null) @@ -324,12 +324,12 @@ public async Task AggregateAsync([NotNull] IFdbDatabase db, Fu await Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - (batch, _, ct) => + (batch, _, __) => { state = handler(state, DecodeItems(batch)); return TaskHelpers.CompletedTask; }, - cancellationToken + ct ); return state; @@ -340,13 +340,13 @@ await Fdb.Bulk.ExportAsync( /// Handler that is called once before the first batch, to produce the initial state. /// Handler called for each batch of items in the map. It is given the previous state, and should return the updated state. Calls to the handler are serialized, so it does not need to take locks. Any exception will abort the export and be thrown to the caller /// Handler that is called one after the last batch, to produce the final result out of the last state. - /// Token used to cancel the operation. + /// Token used to cancel the operation. /// Task that completes once all the entries have been processed and return the result of calling with the state return by the last call to if there was at least one batch, or the result of if the map was empty. /// This method does not guarantee that the export will be a complete and coherent snapshot of the map, except that all the items in a single batch are from the same snapshot. Any change made to the map while the export is running may be partially exported. - public async Task AggregateAsync([NotNull] IFdbDatabase db, Func init, [NotNull] Func[], TState> handler, Func finish, CancellationToken cancellationToken) + public async Task AggregateAsync([NotNull] IFdbDatabase db, Func init, [NotNull] Func[], TState> handler, Func finish, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); var state = default(TState); if (init != null) @@ -357,15 +357,15 @@ public async Task AggregateAsync([NotNull] IFdbDatabas await Fdb.Bulk.ExportAsync( db, this.Location.ToRange(), - (batch, _, ct) => + (batch, _, __) => { state = handler(state, DecodeItems(batch)); return TaskHelpers.CompletedTask; }, - cancellationToken + ct ); - cancellationToken.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); var result = default(TResult); if (finish != null) @@ -383,22 +383,22 @@ await Fdb.Bulk.ExportAsync( /// Imports a potentially large sequence of items into the map. /// Database used for the operation /// Sequence of items to import. If the item already exists in the map, its value will be overwritten. - /// Token used to cancel the operation + /// Token used to cancel the operation /// ///

Any previously existing items in the map will remain. If you want to get from the previous content, you need to clear the map before hand.

///

Other transactions may see a partial view of the map while the sequence is being imported. If this is a problem, you may need to import the map into a temporary subspace, and then 'publish' the final result using an indirection layer (like the Directory Layer)

///

If the import operation fails midway, all items that have already been successfully imported will be kept in the database.

///
- public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> items, CancellationToken cancellationToken) + public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> items, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (items == null) throw new ArgumentNullException("items"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (items == null) throw new ArgumentNullException(nameof(items)); return Fdb.Bulk.InsertAsync( db, items, (item, tr) => this.Set(tr, item.Key, item.Value), - cancellationToken + ct ); } @@ -406,23 +406,23 @@ public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerableDatabase used for the operation /// Sequence of elements to import. If an item with the same key already exists in the map, its value will be overwritten. /// Lambda that will extract the key of an element - /// Token used to cancel the operation + /// Token used to cancel the operation /// ///

Any previously existing items in the map will remain. If you want to get from the previous content, you need to clear the map before hand.

///

Other transactions may see a partial view of the map while the sequence is being imported. If this is a problem, you may need to import the map into a temporary subspace, and then 'publish' the final result using an indirection layer (like the Directory Layer)

///

If the import operation fails midway, all items that have already been successfully imported will be kept in the database.

///
- public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable items, [NotNull] Func keySelector, CancellationToken cancellationToken) + public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable items, [NotNull] Func keySelector, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (items == null) throw new ArgumentNullException("items"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (items == null) throw new ArgumentNullException(nameof(items)); if (keySelector == null) throw new ArgumentException("keySelector"); return Fdb.Bulk.InsertAsync( db, items, (item, tr) => this.Set(tr, keySelector(item), item), - cancellationToken + ct ); } @@ -431,16 +431,16 @@ public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable /// Sequence of elements to import. If an item with the same key already exists in the map, its value will be overwritten. /// Lambda that will return the key of an element /// Lambda that will return the value of an element - /// Token used to cancel the operation + /// Token used to cancel the operation /// ///

Any previously existing items in the map will remain. If you want to get from the previous content, you need to clear the map before hand.

///

Other transactions may see a partial view of the map while the sequence is being imported. If this is a problem, you may need to import the map into a temporary subspace, and then 'publish' the final result using an indirection layer (like the Directory Layer)

///

If the import operation fails midway, all items that have already been successfully imported will be kept in the database.

///
- public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable items, [NotNull] Func keySelector, [NotNull] Func valueSelector, CancellationToken cancellationToken) + public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable items, [NotNull] Func keySelector, [NotNull] Func valueSelector, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (items == null) throw new ArgumentNullException("items"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (items == null) throw new ArgumentNullException(nameof(items)); if (keySelector == null) throw new ArgumentException("keySelector"); if (valueSelector == null) throw new ArgumentException("valueSelector"); @@ -448,7 +448,7 @@ public Task ImportAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerab db, items, (item, tr) => this.Set(tr, keySelector(item), valueSelector(item)), - cancellationToken + ct ); } diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index d5301ce17..8d2acf328 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -69,8 +69,8 @@ public FdbQueue([NotNull] KeySubspace subspace, bool highContention) /// If true, uses High Contention Mode (lots of popping clients). If true, uses the Simple Mode (a few popping clients). public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] IValueEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); this.Subspace = subspace.Using(TypeSystem.Tuples); this.HighContention = highContention; @@ -100,7 +100,7 @@ public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] /// Remove all items from the queue. public void Clear([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.ClearRange(this.Subspace); } @@ -108,7 +108,7 @@ public void Clear([NotNull] IFdbTransaction trans) /// Push a single item onto the queue. public async Task PushAsync([NotNull] IFdbTransaction trans, T value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); #if DEBUG trans.Annotate("Push({0})", value); @@ -124,22 +124,22 @@ public async Task PushAsync([NotNull] IFdbTransaction trans, T value) } /// Pop the next item from the queue. Cannot be composed with other functions in a single transaction. - public Task> PopAsync([NotNull] IFdbDatabase db, CancellationToken cancellationToken) + public Task> PopAsync([NotNull] IFdbDatabase db, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); - if (cancellationToken.IsCancellationRequested) + if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation>(cancellationToken); + return TaskHelpers.FromCancellation>(ct); } if (this.HighContention) { - return PopHighContentionAsync(db, cancellationToken); + return PopHighContentionAsync(db, ct); } else { - return db.ReadWriteAsync((tr) => this.PopSimpleAsync(tr), cancellationToken); + return db.ReadWriteAsync((tr) => PopSimpleAsync(tr), ct); } } @@ -165,86 +165,86 @@ public async Task> PeekAsync([NotNull] IFdbReadOnlyTransaction tr) #region Bulk Operations - public Task ExportAsync(IFdbDatabase db, Action handler, CancellationToken cancellationToken) + public Task ExportAsync(IFdbDatabase db, Action handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); //REVIEW: is this approach correct ? return Fdb.Bulk.ExportAsync( db, this.QueueItem.Keys.ToRange(), - (kvs, offset, ct) => + (kvs, offset, _) => { foreach(var kv in kvs) { - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); handler(this.Encoder.DecodeValue(kv.Value), offset); ++offset; } return TaskHelpers.CompletedTask; }, - cancellationToken + ct ); } - public Task ExportAsync(IFdbDatabase db, Func handler, CancellationToken cancellationToken) + public Task ExportAsync(IFdbDatabase db, Func handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); //REVIEW: is this approach correct ? return Fdb.Bulk.ExportAsync( db, this.QueueItem.Keys.ToRange(), - async (kvs, offset, ct) => + async (kvs, offset, _) => { foreach (var kv in kvs) { - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); await handler(this.Encoder.DecodeValue(kv.Value), offset); ++offset; } }, - cancellationToken + ct ); } - public Task ExportAsync(IFdbDatabase db, Action handler, CancellationToken cancellationToken) + public Task ExportAsync(IFdbDatabase db, Action handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); //REVIEW: is this approach correct ? return Fdb.Bulk.ExportAsync( db, this.QueueItem.Keys.ToRange(), - (kvs, offset, ct) => + (kvs, offset, _) => { handler(this.Encoder.DecodeValues(kvs), offset); return TaskHelpers.CompletedTask; }, - cancellationToken + ct ); } - public Task ExportAsync(IFdbDatabase db, Func handler, CancellationToken cancellationToken) + public Task ExportAsync(IFdbDatabase db, Func handler, CancellationToken ct) { - if (db == null) throw new ArgumentNullException("db"); - if (handler == null) throw new ArgumentNullException("handler"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (handler == null) throw new ArgumentNullException(nameof(handler)); //REVIEW: is this approach correct ? return Fdb.Bulk.ExportAsync( db, this.QueueItem.Keys.ToRange(), - (kvs, offset, ct) => handler(this.Encoder.DecodeValues(kvs), offset), - cancellationToken + (kvs, offset, _) => handler(this.Encoder.DecodeValues(kvs), offset), + ct ); } diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index e39e430cd..494833c02 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -64,9 +64,9 @@ public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspac /// Encoder for the counter values public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace, [NotNull] IValueEncoder encoder) { - if (db == null) throw new ArgumentNullException("db"); - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + if (db == null) throw new ArgumentNullException(nameof(db)); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); this.Database = db; this.Subspace = subspace.Using(TypeSystem.Tuples); @@ -74,13 +74,13 @@ public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspac } /// Subspace used as a prefix for all items in this table - public IDynamicKeySubspace Subspace {[NotNull] get; private set; } + public IDynamicKeySubspace Subspace {[NotNull] get; } /// Database instance that is used to perform background coalescing of the counter - public IFdbDatabase Database {[NotNull] get; private set; } + public IFdbDatabase Database {[NotNull] get; } /// Encoder for the integer values of the counter - public IValueEncoder Encoder {[NotNull] get; private set; } + public IValueEncoder Encoder {[NotNull] get; } /// Generate a new random slice protected virtual Slice RandomId() @@ -129,7 +129,7 @@ private async Task Coalesce(int N, CancellationToken ct) catch (FdbException x) { //TODO: logging ? - System.Diagnostics.Debug.WriteLine("Coalesce error: " + x.Message); + System.Diagnostics.Debug.WriteLine($"Coalesce error: {x.Message}"); return; } } @@ -155,7 +155,7 @@ private void BackgroundCoalesce(int n, CancellationToken ct) { var x = t.Exception; //TODO: logging ? - System.Diagnostics.Debug.WriteLine("Background Coalesce error: " + x.ToString()); + System.Diagnostics.Debug.WriteLine($"Background Coalesce error: {x}"); } }); } @@ -172,7 +172,7 @@ private void BackgroundCoalesce(int n, CancellationToken ct) /// public async Task GetTransactional([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); long total = 0; await trans @@ -186,7 +186,7 @@ await trans /// Get the value of the counter with snapshot isolation (no transaction conflicts). public Task GetSnapshot([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return GetTransactional(trans.Snapshot); } @@ -194,7 +194,7 @@ public Task GetSnapshot([NotNull] IFdbReadOnlyTransaction trans) /// Add the value x to the counter. public void Add([NotNull] IFdbTransaction trans, long x) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.Set(this.Subspace.Keys.Encode(RandomId()), this.Encoder.EncodeValue(x)); @@ -212,7 +212,7 @@ public void Add([NotNull] IFdbTransaction trans, long x) /// Set the counter to value x. public async Task SetTotal([NotNull] IFdbTransaction trans, long x) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); long value = await GetSnapshot(trans).ConfigureAwait(false); Add(trans, x - value); @@ -223,27 +223,27 @@ public async Task SetTotal([NotNull] IFdbTransaction trans, long x) /// Get the value of the counter. /// Not recommended for use with read/write transactions when the counter is being frequently updated (conflicts will be very likely). /// - public Task GetTransactionalAsync(CancellationToken cancellationToken) + public Task GetTransactionalAsync(CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.GetTransactional(tr), cancellationToken); + return this.Database.ReadAsync((tr) => GetTransactional(tr), ct); } /// Get the value of the counter with snapshot isolation (no transaction conflicts). - public Task GetSnapshotAsync(CancellationToken cancellationToken) + public Task GetSnapshotAsync(CancellationToken ct) { - return this.Database.ReadAsync((tr) => this.GetSnapshot(tr), cancellationToken); + return this.Database.ReadAsync((tr) => GetSnapshot(tr), ct); } /// Add the value x to the counter. - public Task AddAsync(long x, CancellationToken cancellationToken) + public Task AddAsync(long x, CancellationToken ct) { - return this.Database.WriteAsync((tr) => this.Add(tr, x), cancellationToken); + return this.Database.WriteAsync((tr) => Add(tr, x), ct); } /// Set the counter to value x. - public Task SetTotalAsync(long x, CancellationToken cancellationToken) + public Task SetTotalAsync(long x, CancellationToken ct) { - return this.Database.ReadWriteAsync((tr) => this.SetTotal(tr, x), cancellationToken); + return this.Database.ReadWriteAsync((tr) => SetTotal(tr, x), ct); } #endregion diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 0074d51bd..b185acac1 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -220,8 +220,9 @@ public void Delete(IFdbTransaction trans, ITuple id, params string[] fields) /// Return the list the names of all fields of an hashset /// Transaction that will be used for this request /// Unique identifier of the hashset + /// /// List of all fields. If the list is empty, the hashset does not exist - public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id, CancellationToken cancellationToken = default(CancellationToken)) + public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id, CancellationToken ct = default(CancellationToken)) { //note: As of Beta2, FDB does not have a fdb_get_range that only return the keys. That means that we will have to also read the values from the db, in order to just get the names of the fields :( //TODO: find a way to optimize this ? @@ -235,7 +236,7 @@ public void Delete(IFdbTransaction trans, ITuple id, params string[] fields) return trans .GetRange(KeyRange.StartsWith(prefix)) .Select((kvp) => ParseFieldKey(STuple.Unpack(kvp.Key))) - .ToListAsync(cancellationToken); + .ToListAsync(ct); } #endregion diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index f12e4ae0b..3660bdb13 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -241,7 +241,7 @@ await db.ReadWriteAsync(async (tr) => { Interlocked.Increment(ref m_schedulingMessages); }, - cancellationToken: ct).ConfigureAwait(false); + ct: ct).ConfigureAwait(false); } static int counter = 0; @@ -356,7 +356,7 @@ await db.ReadWriteAsync( previousTaskId = Slice.Nil; workerId = myId; }, - cancellationToken: ct + ct: ct ).ConfigureAwait(false); if (msg.Id.IsNullOrEmpty) diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs index d2848a200..e59007947 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs @@ -245,9 +245,9 @@ public TransactionIterator(IFdbTransaction transaction, IFdbAsyncEnumerator i m_iterator = iterator; } - public Task MoveNextAsync(CancellationToken cancellationToken) + public Task MoveNextAsync(CancellationToken ct) { - return m_iterator.MoveNextAsync(cancellationToken); + return m_iterator.MoveNextAsync(ct); } public T Current diff --git a/FoundationDB.Samples/Benchmarks/BenchRunner.cs b/FoundationDB.Samples/Benchmarks/BenchRunner.cs index 88b30901e..3b4ec443d 100644 --- a/FoundationDB.Samples/Benchmarks/BenchRunner.cs +++ b/FoundationDB.Samples/Benchmarks/BenchRunner.cs @@ -51,7 +51,7 @@ public BenchRunner(BenchMode mode, int value = 1) public async Task Init(IFdbDatabase db, CancellationToken ct) { // open the folder where we will store everything - this.Subspace = await db.Directory.CreateOrOpenAsync("Benchmarks", cancellationToken: ct); + this.Subspace = await db.Directory.CreateOrOpenAsync("Benchmarks", ct: ct); } diff --git a/FoundationDB.Samples/Benchmarks/LeakTest.cs b/FoundationDB.Samples/Benchmarks/LeakTest.cs index 5cf2863b1..4a3468b28 100644 --- a/FoundationDB.Samples/Benchmarks/LeakTest.cs +++ b/FoundationDB.Samples/Benchmarks/LeakTest.cs @@ -37,7 +37,7 @@ public LeakTest(int k, int m, int n, TimeSpan delay) public async Task Init(IFdbDatabase db, CancellationToken ct) { // open the folder where we will store everything - this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Benchmarks", "LeakTest" }, cancellationToken: ct); + this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Benchmarks", "LeakTest" }, ct: ct); // clear all previous values await db.ClearRangeAsync(this.Subspace, ct); diff --git a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs index b4c1a9904..261a83ba5 100644 --- a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs +++ b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs @@ -66,7 +66,7 @@ public MessageQueueRunner(string id, AgentRole role, TimeSpan delayMin, TimeSpan public async Task Init(IFdbDatabase db, CancellationToken ct) { // open the folder where we will store everything - this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Samples", "MessageQueueTest" }, cancellationToken: ct); + this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Samples", "MessageQueueTest" }, ct: ct); this.WorkerPool = new FdbWorkerPool(this.Subspace); diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index 6b010c967..cc14886c9 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -53,7 +53,7 @@ protected KeyRange AttendsKeys(string s) public async Task Init(IFdbDatabase db, CancellationToken ct) { // open the folder where we will store everything - this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Tutorials", "ClassScheduling" }, cancellationToken: ct); + this.Subspace = await db.Directory.CreateOrOpenAsync(new [] { "Tutorials", "ClassScheduling" }, ct: ct); // clear all previous values await db.ClearRangeAsync(this.Subspace, ct); diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index c1e3dfff9..bf2dc887e 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -220,7 +220,7 @@ public async Task Test_CreateOrOpen_With_Layer() Assert.Throws(async () => await directory.OpenAsync(logged, new[] { "Foo" }, Slice.FromString("OtherLayer"), this.Cancellation), "Opening with invalid layer id should fail"); // opening without specifying a layer should disable the layer check - var foo3 = await directory.OpenAsync(logged, "Foo", layer: Slice.Nil, cancellationToken: this.Cancellation); + var foo3 = await directory.OpenAsync(logged, "Foo", layer: Slice.Nil, ct: this.Cancellation); Assert.That(foo3, Is.Not.Null); Assert.That(foo3.Layer.ToUnicode(), Is.EqualTo("AcmeLayer")); @@ -522,7 +522,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() var logged = db; #endif - var folder = await directory.CreateAsync(logged, "Test", layer: Slice.FromString("foo"), cancellationToken: this.Cancellation); + var folder = await directory.CreateAsync(logged, "Test", layer: Slice.FromString("foo"), ct: this.Cancellation); #if DEBUG await DumpSubspace(db, location); #endif @@ -541,11 +541,11 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() Assert.That(folder2.GetPrefix(), Is.EqualTo(folder.GetPrefix())); // opening the directory with the new layer should succeed - var folder3 = await directory.OpenAsync(logged, "Test", layer: Slice.FromString("bar"), cancellationToken: this.Cancellation); + var folder3 = await directory.OpenAsync(logged, "Test", layer: Slice.FromString("bar"), ct: this.Cancellation); Assert.That(folder3, Is.Not.Null); // opening the directory with the old layer should fail - Assert.Throws(async () => await directory.OpenAsync(logged, "Test", layer: Slice.FromString("foo"), cancellationToken: this.Cancellation)); + Assert.Throws(async () => await directory.OpenAsync(logged, "Test", layer: Slice.FromString("foo"), ct: this.Cancellation)); #if ENABLE_LOGGING foreach (var log in list) diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 7ccf4f345..41af6be75 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -483,7 +483,7 @@ public async Task Test_Can_Resolve_Key_Selector_Outside_Boundaries() Slice key; // note: we can't have any prefix on the keys, so open the test database in read-only mode - using (var db = await Fdb.OpenAsync(TestHelpers.TestClusterFile, TestHelpers.TestDbName, KeySubspace.Empty, readOnly: true, cancellationToken: this.Cancellation)) + using (var db = await Fdb.OpenAsync(TestHelpers.TestClusterFile, TestHelpers.TestDbName, KeySubspace.Empty, readOnly: true, ct: this.Cancellation)) { using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) { From c1f523fb020c0e26ee4892f4451d306267cbdd71 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 13:48:20 +0200 Subject: [PATCH 066/153] Changed the namespace of "core" types into "FoundationDB" (from "FoundationDB.Client") - FoundationDB: contains all generic types used to deal with keys, values, ... - FoundationDB.Client: contains all implementation for the client itself - FoundationDB.Layers.*: all layers build on top of the client --- FdbShell/Commands/BasicCommands.cs | 30 +++-- FoundationDB.Client/FdbKey.cs | 14 +- .../Logging/FdbTransactionLog.Commands.cs | 8 +- .../FoundationDB.Client.csproj | 1 + FoundationDB.Client/KeyRange.cs | 8 +- FoundationDB.Client/KeyRangeComparer.cs | 2 +- FoundationDB.Client/KeySelector.cs | 5 +- FoundationDB.Client/KeySelectorPair.cs | 3 +- FoundationDB.Client/Utils/Batched.cs | 2 +- FoundationDB.Client/Utils/Contract.cs | 1 - .../Utils/ExceptionExtensions.cs | 122 ++++++++++++++++++ FoundationDB.Client/Utils/Logging.cs | 2 +- FoundationDB.Client/Utils/Slice.cs | 2 +- FoundationDB.Client/Utils/SliceBuffer.cs | 4 +- FoundationDB.Client/Utils/SliceComparer.cs | 2 +- FoundationDB.Client/Utils/SliceHelpers.cs | 2 +- FoundationDB.Client/Utils/SliceListStream.cs | 2 +- .../Utils/SlicePairComparer.cs | 2 +- FoundationDB.Client/Utils/SliceReader.cs | 2 +- FoundationDB.Client/Utils/SliceStream.cs | 2 +- FoundationDB.Client/Utils/SliceWriter.cs | 2 +- FoundationDB.Client/Utils/ThrowHelper.cs | 2 +- FoundationDB.Client/Utils/Uuid128.cs | 2 +- FoundationDB.Client/Utils/Uuid64.cs | 2 +- 24 files changed, 177 insertions(+), 47 deletions(-) create mode 100644 FoundationDB.Client/Utils/ExceptionExtensions.cs diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 50b9fbe8b..9c97113a6 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -1,19 +1,21 @@ -using FoundationDB.Client; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.IO; -using System.Linq; -using System.Net; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - + namespace FdbShell { + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Globalization; + using System.IO; + using System.Linq; + using System.Net; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using FoundationDB; + using FoundationDB.Client; + using FoundationDB.Layers.Directories; + using FoundationDB.Layers.Tuples; + public static class BasicCommands { diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index d47c577da..0de5c5298 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -26,7 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +using FoundationDB.Client; + +namespace FoundationDB { using System; using System.Collections.Generic; @@ -63,7 +65,7 @@ public static class FdbKey /// public static Slice Increment(Slice slice) { - if (slice.IsNull) throw new ArgumentException("Cannot increment null buffer", "slice"); + if (slice.IsNull) throw new ArgumentException("Cannot increment null buffer", nameof(slice)); int lastNonFFByte; var tmp = slice.GetBytes(); @@ -91,8 +93,8 @@ public static Slice Increment(Slice slice) [NotNull] public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) { - if (prefix.IsNull) throw new ArgumentNullException("prefix"); - if (keys == null) throw new ArgumentNullException("keys"); + if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); + if (keys == null) throw new ArgumentNullException(nameof(keys)); //REVIEW: merge this code with Slice.ConcatRange! @@ -120,8 +122,8 @@ public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) [NotNull] public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) { - if (prefix.IsNull) throw new ArgumentNullException("prefix"); - if (keys == null) throw new ArgumentNullException("keys"); + if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); + if (keys == null) throw new ArgumentNullException(nameof(keys)); //REVIEW: merge this code with Slice.ConcatRange! diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index f9b32bb4b..1c91daab2 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -28,22 +28,22 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Filters.Logging { - using FoundationDB.Async; - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.Text; using System.Threading.Tasks; + using FoundationDB.Async; + using FoundationDB.Client; using FoundationDB.Layers.Directories; + using JetBrains.Annotations; public partial class FdbTransactionLog { /// Base class of all types of operations performed on a transaction - [DebuggerDisplay("{ToString()}")] + [DebuggerDisplay("{ToString(),nq}")] public abstract class Command { diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 8ee988e68..847f6d805 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -223,6 +223,7 @@ + diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index a6c133751..1efb20cd3 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -26,21 +26,23 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client + +namespace FoundationDB { using System; using System.Diagnostics; using Doxense.Diagnostics.Contracts; + using FoundationDB.Client; /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] public struct KeyRange : IEquatable, IComparable { /// Returns an empty pair of keys - public static KeyRange Empty { get { return default(KeyRange); } } + public static KeyRange Empty => default(KeyRange); /// Returns a range that contains all the keys in the database - public static KeyRange All { get { return new KeyRange(FdbKey.MinValue, FdbKey.MaxValue); } } + public static KeyRange All => new KeyRange(FdbKey.MinValue, FdbKey.MaxValue); /// Start of the range public Slice Begin { get { return m_begin; } } diff --git a/FoundationDB.Client/KeyRangeComparer.cs b/FoundationDB.Client/KeyRangeComparer.cs index 94aaabdb9..27d091727 100644 --- a/FoundationDB.Client/KeyRangeComparer.cs +++ b/FoundationDB.Client/KeyRangeComparer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index d670ebfba..b0248d46f 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -26,7 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client + +namespace FoundationDB { using System; using System.Diagnostics; @@ -34,7 +35,7 @@ namespace FoundationDB.Client using JetBrains.Annotations; /// Defines a selector for a key in the database - [DebuggerDisplay("{ToString()}")] + [DebuggerDisplay("{ToString(),nq}")] public struct KeySelector : IEquatable { /// Empty key selector diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index d72e32b96..3891ee1f5 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -26,7 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client + +namespace FoundationDB { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/Utils/Batched.cs b/FoundationDB.Client/Utils/Batched.cs index 76cb4de9a..fa6f329b1 100644 --- a/FoundationDB.Client/Utils/Batched.cs +++ b/FoundationDB.Client/Utils/Batched.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Utils +namespace FoundationDB { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/Utils/Contract.cs b/FoundationDB.Client/Utils/Contract.cs index f07b1dfdd..22b602fb0 100644 --- a/FoundationDB.Client/Utils/Contract.cs +++ b/FoundationDB.Client/Utils/Contract.cs @@ -8,7 +8,6 @@ namespace Doxense.Diagnostics.Contracts using System; using System.Collections.Generic; using System.Diagnostics; - using System.Globalization; using System.Reflection; using System.Runtime.CompilerServices; using System.Runtime.ConstrainedExecution; diff --git a/FoundationDB.Client/Utils/ExceptionExtensions.cs b/FoundationDB.Client/Utils/ExceptionExtensions.cs new file mode 100644 index 000000000..76902e7da --- /dev/null +++ b/FoundationDB.Client/Utils/ExceptionExtensions.cs @@ -0,0 +1,122 @@ +#region Copyright Doxense 2005-2015 +// +// All rights are reserved. Reproduction or transmission in whole or in part, in +// any form or by any means, electronic, mechanical or otherwise, is prohibited +// without the prior written consent of the copyright owner. +// +#endregion + +namespace FoundationDB +{ + using Doxense.Diagnostics.Contracts; + using System; + using System.Collections.Generic; + using System.Reflection; + using JetBrains.Annotations; + + internal static class ExceptionExtensions + { + private static readonly MethodInfo s_preserveStackTrace; + private static readonly MethodInfo s_prepForRemoting; + + static ExceptionExtensions() + { + try + { + s_preserveStackTrace = typeof(Exception).GetMethod("InternalPreserveStackTrace", BindingFlags.Instance | BindingFlags.NonPublic); + s_prepForRemoting = typeof(Exception).GetMethod("PrepForRemoting", BindingFlags.Instance | BindingFlags.NonPublic); + } + catch { } + Contract.Ensures(s_preserveStackTrace != null, "Exception.InternalPreserveStackTrace not found?"); + Contract.Ensures(s_prepForRemoting != null, "Exception.PrepForRemoting not found?"); + } + + /// Dtermine s'il s'agit d'une erreur fatale (qu'il faudrait bouncer) + /// Exception tester + /// True s'il s'agit d'une ThreadAbortException, OutOfMemoryException ou StackOverflowException, ou une AggregateException qui contient une de ces erreurs + [Pure] + public static bool IsFatalError([CanBeNull] this Exception self) + { + return self is System.Threading.ThreadAbortException || self is OutOfMemoryException || self is StackOverflowException || (self is AggregateException && IsFatalError(self.InnerException)); + } + + /// Prserve la stacktrace lorsqu'on cre une exception, qui sera re-throw plus haut + /// Exception qui a t catche + /// La mme exception, mais avec la StackTrace prserve + [NotNull] + public static Exception PreserveStackTrace([NotNull] this Exception self) + { + self = UnwrapIfAggregate(self); + if (s_preserveStackTrace != null) s_preserveStackTrace.Invoke(self, null); + return self; + } + + /// Prserve la stacktrace lorsqu'on veut re-thrower une exception catche + /// Exception qui a t catche + /// La mme exception, mais avec la StackTrace prserve + /// Similaire l'extension mthode PrepareForRethrow prsente dans System.CoreEx.dll du Reactive Framework + [NotNull] + public static Exception PrepForRemoting([NotNull] this Exception self) + { + //TODO: cette extensions mthode est galement prsente dans System.CoreEx.dll du Reactive Framework! + // il faudra peut etre a terme rerouter vers cette version (si un jour Sioux refrence Rx directement...) + self = UnwrapIfAggregate(self); + if (s_prepForRemoting != null) s_prepForRemoting.Invoke(self, null); + return self; + } + + /// Retourne la premire exeception non-aggregate trouve dans l'arbre des InnerExceptions + /// AggregateException racine + /// Premire exception dans l'arbre des InnerExceptions qui ne soit pas de type AggregateException + [NotNull] + public static Exception GetFirstConcreteException([NotNull] this AggregateException self) + { + // dans la majorit des cas, on a une branche avec potentiellement plusieurs couches de AggEx mais une seule InnerException + var e = self.GetBaseException(); + if (!(e is AggregateException)) return e; + + // Sinon c'est qu'on a un arbre a plusieures branches, qu'on va devoir parcourir... + var list = new Queue(); + list.Enqueue(self); + while (list.Count > 0) + { + foreach (var e2 in list.Dequeue().InnerExceptions) + { + if (e2 == null) continue; + if (!(e2 is AggregateException x)) return e2; // on a trouv une exception concrte ! + list.Enqueue(x); + } + } + // uhoh ? + return self; + } + + /// Retourne la premire exception non-aggregate si c'est une AggregateException, ou l'exception elle mme dans les autres cas + /// + /// + [NotNull] + public static Exception UnwrapIfAggregate([NotNull] this Exception self) + { + return self is AggregateException aggEx ? GetFirstConcreteException(aggEx) : self; + } + + /// Rethrow la premire exception non-aggregate trouve, en jettant les autres s'il y en a + /// AggregateException racine + [ContractAnnotation("self:null => null")] + public static Exception Unwrap(this AggregateException self) + { + return self != null ? GetFirstConcreteException(self).PrepForRemoting() : null; + } + + /// Unwrap generic exceptions like or to return the inner exceptions + [NotNull] + public static Exception Unwrap([NotNull] this Exception self) + { + if (self is AggregateException aggEx) return GetFirstConcreteException(aggEx); + if (self is TargetInvocationException tiEx) return tiEx.InnerException ?? self; + //add other type of "container" exceptions as required + return self; + } + } + +} diff --git a/FoundationDB.Client/Utils/Logging.cs b/FoundationDB.Client/Utils/Logging.cs index 3e1968410..52b316255 100644 --- a/FoundationDB.Client/Utils/Logging.cs +++ b/FoundationDB.Client/Utils/Logging.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Utils +namespace FoundationDB.Client { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index d2a46f38e..a8bec27f9 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using JetBrains.Annotations; using System; diff --git a/FoundationDB.Client/Utils/SliceBuffer.cs b/FoundationDB.Client/Utils/SliceBuffer.cs index 48bd07807..35c1a8d6a 100644 --- a/FoundationDB.Client/Utils/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/SliceBuffer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Utils +namespace FoundationDB { using System; using System.Collections.Generic; @@ -133,7 +133,7 @@ public KeyRange InternRange(Slice begin, Slice end) public KeyRange InternRangeFromKey(Slice key) { // Since the end key only adds \0 to the begin key, we can reuse the same bytes by making both overlap - var tmp = Intern(key, FdbKey.MinValue, aligned: true); + var tmp = Intern(key, Slice.FromByte(0), aligned: true); return new KeyRange( tmp.Substring(0, key.Count), diff --git a/FoundationDB.Client/Utils/SliceComparer.cs b/FoundationDB.Client/Utils/SliceComparer.cs index 2971bfe33..d640bc6cb 100644 --- a/FoundationDB.Client/Utils/SliceComparer.cs +++ b/FoundationDB.Client/Utils/SliceComparer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs index 7576d37e7..8148c001d 100644 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ b/FoundationDB.Client/Utils/SliceHelpers.cs @@ -28,7 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #define USE_NATIVE_MEMORY_OPERATORS -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Runtime.CompilerServices; diff --git a/FoundationDB.Client/Utils/SliceListStream.cs b/FoundationDB.Client/Utils/SliceListStream.cs index b49313e38..1503ab61e 100644 --- a/FoundationDB.Client/Utils/SliceListStream.cs +++ b/FoundationDB.Client/Utils/SliceListStream.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/Utils/SlicePairComparer.cs b/FoundationDB.Client/Utils/SlicePairComparer.cs index b7f1081c8..209f52fad 100644 --- a/FoundationDB.Client/Utils/SlicePairComparer.cs +++ b/FoundationDB.Client/Utils/SlicePairComparer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/Utils/SliceReader.cs b/FoundationDB.Client/Utils/SliceReader.cs index 6637554e8..057056895 100644 --- a/FoundationDB.Client/Utils/SliceReader.cs +++ b/FoundationDB.Client/Utils/SliceReader.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using JetBrains.Annotations; using System; diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/SliceStream.cs index a4dbe887d..7b20c10cc 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/SliceStream.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.IO; diff --git a/FoundationDB.Client/Utils/SliceWriter.cs b/FoundationDB.Client/Utils/SliceWriter.cs index 01b7f4528..4b35a8fe0 100644 --- a/FoundationDB.Client/Utils/SliceWriter.cs +++ b/FoundationDB.Client/Utils/SliceWriter.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/Utils/ThrowHelper.cs b/FoundationDB.Client/Utils/ThrowHelper.cs index 5c0c0de8f..5a5e99610 100644 --- a/FoundationDB.Client/Utils/ThrowHelper.cs +++ b/FoundationDB.Client/Utils/ThrowHelper.cs @@ -12,7 +12,7 @@ namespace Doxense.Diagnostics.Contracts using System.Runtime.CompilerServices; [DebuggerNonUserCode] - public static class ThrowHelper + internal static class ThrowHelper { #region ArgumentNullException... diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Utils/Uuid128.cs index 1bd9a60cc..ab273b5d3 100644 --- a/FoundationDB.Client/Utils/Uuid128.cs +++ b/FoundationDB.Client/Utils/Uuid128.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using JetBrains.Annotations; using System; diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index e507c78e6..5e08d1b44 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace FoundationDB { using System; using System.ComponentModel; From 370ddea818b68faede18a45c7407414f7aa5428d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 15:55:09 +0200 Subject: [PATCH 067/153] AsyncLinq refactored and updated to the latest version of the upstream Doxense.Linq library - based on temporary interfaces IAsyncEnumerable/IAsyncIterator (until they are in the BCL) - Dropped all the "Fdb" prefix since this is not tied to FDB per se - Update all utility classes (Maybe, TaskHelpers, ...) --- FdbBurner/Program.cs | 21 +- FdbShell/Program.cs | 74 +- FdbShell/Utils/PerfCounters.cs | 18 +- FoundationDB.Client/Async/AsyncBuffer.cs | 72 +- .../Async/AsyncCancellableMutex.cs | 10 +- FoundationDB.Client/Async/AsyncHelpers.cs | 175 +- .../Async/AsyncOrderingMode.cs | 2 +- .../Async/AsyncProducerConsumerQueue.cs | 18 +- FoundationDB.Client/Async/AsyncPump.cs | 6 +- FoundationDB.Client/Async/AsyncTaskBuffer.cs | 79 +- FoundationDB.Client/Async/AsyncTransform.cs | 43 +- .../Async/AsyncTransformQueue.cs | 69 +- FoundationDB.Client/Async/IAsyncBuffer.cs | 4 +- FoundationDB.Client/Async/IAsyncPump.cs | 2 +- FoundationDB.Client/Async/IAsyncSource.cs | 3 +- FoundationDB.Client/Async/IAsyncTarget.cs | 6 +- FoundationDB.Client/Async/Maybe.cs | 599 +++++-- FoundationDB.Client/Async/TaskHelpers.cs | 110 +- FoundationDB.Client/FdbDatabase.cs | 3 +- .../FdbMergeQueryExtensions.cs | 65 +- FoundationDB.Client/FdbOperationContext.cs | 10 +- FoundationDB.Client/FdbRangeChunk.cs | 30 +- .../FdbRangeQuery.PagingIterator.cs | 26 +- .../FdbRangeQuery.ResultIterator.cs | 32 +- FoundationDB.Client/FdbRangeQuery.cs | 31 +- FoundationDB.Client/FdbTransaction.cs | 4 +- .../FdbTransactionExtensions.cs | 5 +- .../Filters/Logging/FdbLoggedTransaction.cs | 7 +- .../Logging/FdbTransactionLog.Commands.cs | 5 +- .../FoundationDB.Client.csproj | 72 +- .../Layers/Directories/FdbDirectoryLayer.cs | 4 +- .../AsyncSequence.cs} | 26 +- .../Expressions/AsyncFilterExpression.cs | 26 +- .../Expressions/AsyncObserverExpression.cs | 13 +- .../Expressions/AsyncTransformExpression.cs | 24 +- .../Iterators/AnonymousAsyncGenerator.cs} | 46 +- .../Iterators/AsyncFilterIterator.cs} | 37 +- .../Iterators/AsyncIterator.cs} | 125 +- .../Iterators/AsyncIteratorPump.cs} | 16 +- .../Iterators/BatchingAsyncIterator.cs} | 23 +- .../Iterators/DistinctAsyncIterator.cs} | 40 +- .../Async/Iterators/ExceptAsyncIterator.cs} | 28 +- .../Iterators/IntersectAsyncIterator.cs} | 28 +- .../Async/Iterators/MergeAsyncIterator.cs} | 51 +- .../Iterators/MergeSortAsyncIterator.cs} | 26 +- .../Iterators/ObserverAsyncIterator.cs} | 24 +- .../Iterators/ParallelSelectAsyncIterator.cs} | 37 +- .../Iterators/PrefetchingAsyncIterator.cs} | 46 +- .../Iterators/SelectManyAsyncIterator.cs} | 72 +- .../Iterators/TakeWhileAsyncIterator.cs} | 21 +- .../Iterators/WhereAsyncIterator.cs} | 65 +- .../Iterators/WhereSelectAsyncIterator.cs} | 116 +- .../Iterators/WindowingAsyncIterator.cs} | 39 +- ...ce.cs => AsyncEnumerable.EmptySequence.cs} | 116 +- ... => AsyncEnumerable.EnumerableIterator.cs} | 21 +- ... => AsyncEnumerable.EnumerableSequence.cs} | 24 +- ...rators.cs => AsyncEnumerable.Iterators.cs} | 537 +++--- ....cs => AsyncEnumerable.OrderedSequence.cs} | 103 +- ....Sorters.cs => AsyncEnumerable.Sorters.cs} | 9 +- FoundationDB.Client/Linq/AsyncEnumerable.cs | 1445 +++++++++++++++++ ...{FdbAsyncMode.cs => AsyncIterationHint.cs} | 7 +- .../Linq/EnumerableExtensions.cs | 142 ++ .../Linq/FdbAsyncEnumerable.cs | 1255 -------------- .../{Async => Linq}/IAsyncEnumerable.cs | 9 +- .../{Async => Linq}/IAsyncEnumerator.cs | 8 +- ...umerable.cs => IAsyncOrderedEnumerable.cs} | 15 +- .../Linq/IFdbAsyncEnumerable.cs | 48 - .../Linq/IFdbAsyncEnumerator.cs | 38 - ...ptions.cs => ParallelAsyncQueryOptions.cs} | 4 +- FoundationDB.Client/Linq/QueryStatistics.cs | 89 + .../Subspaces/Fdb.Directory.cs | 5 +- .../Utils/ExceptionExtensions.cs | 2 +- FoundationDB.Client/Utils/Slice.cs | 4 +- FoundationDB.Client/Utils/SliceListStream.cs | 6 +- FoundationDB.Client/Utils/SliceStream.cs | 11 +- .../Collections/FdbMap`2.cs | 15 +- .../Collections/FdbMultimap`2.cs | 33 +- .../Collections/FdbQueue`1.cs | 15 +- .../Collections/FdbRankedSet.cs | 20 +- .../Collections/FdbVector`1.cs | 4 +- .../Documents/FdbHashSetCollection.cs | 7 +- .../Messaging/WorkerPoolTest.cs | 21 +- .../Expressions/FdbExpressionHelpers.cs | 22 +- .../FdbQueryAsyncEnumerableExpression.cs | 11 +- .../Expressions/FdbQueryExpression.cs | 11 +- .../Expressions/FdbQueryExpressions.cs | 40 +- .../Expressions/FdbQueryFilterExpression.cs | 7 +- .../FdbQueryIndexLookupExpression.cs | 11 +- .../FdbQueryIntersectExpression.cs | 11 +- .../Expressions/FdbQueryRangeExpression.cs | 13 +- .../Expressions/FdbQuerySequenceExpression.cs | 11 +- .../Expressions/FdbQuerySingleExpression.cs | 38 +- .../FdbQueryTransformExpression.cs | 7 +- .../FdbAsyncQueryable.cs | 53 +- .../Providers/FdbAsyncQuery.cs | 54 +- .../Providers/FdbAsyncSequenceQuery.cs | 6 +- FoundationDB.Samples/Program.cs | 2 +- .../Tutorials/ClassScheduling.cs | 2 +- FoundationDB.Tests.Sandbox/Program.cs | 8 +- FoundationDB.Tests/Async/AsyncBufferFacts.cs | 9 +- FoundationDB.Tests/FdbTest.cs | 4 +- FoundationDB.Tests/KeyFacts.cs | 7 +- FoundationDB.Tests/Layers/IndexingFacts.cs | 11 +- FoundationDB.Tests/Layers/QueuesFacts.cs | 9 +- .../Linq/FdbAsyncEnumerableFacts.cs | 130 +- .../Linq/FdbQueryExpressionFacts.cs | 22 +- FoundationDB.Tests/RangeQueryFacts.cs | 17 +- 107 files changed, 3848 insertions(+), 3215 deletions(-) rename FoundationDB.Client/Linq/{FdbAsyncSequence.cs => Async/AsyncSequence.cs} (75%) rename FoundationDB.Client/Linq/{ => Async}/Expressions/AsyncFilterExpression.cs (90%) rename FoundationDB.Client/Linq/{ => Async}/Expressions/AsyncObserverExpression.cs (91%) rename FoundationDB.Client/Linq/{ => Async}/Expressions/AsyncTransformExpression.cs (91%) rename FoundationDB.Client/Linq/{Iterators/FdbAnonymousAsyncGenerator.cs => Async/Iterators/AnonymousAsyncGenerator.cs} (68%) rename FoundationDB.Client/Linq/{Iterators/FdbAsyncFilterIterator.cs => Async/Iterators/AsyncFilterIterator.cs} (77%) rename FoundationDB.Client/Linq/{Iterators/FdbAsyncIterator.cs => Async/Iterators/AsyncIterator.cs} (53%) rename FoundationDB.Client/Linq/{Iterators/FdbAsyncIteratorPump.cs => Async/Iterators/AsyncIteratorPump.cs} (93%) rename FoundationDB.Client/Linq/{Iterators/FdbBatchingAsyncIterator.cs => Async/Iterators/BatchingAsyncIterator.cs} (82%) rename FoundationDB.Client/Linq/{Iterators/FdbDistinctAsyncIterator.cs => Async/Iterators/DistinctAsyncIterator.cs} (75%) rename FoundationDB.Client/{FdbExceptIterator.cs => Linq/Async/Iterators/ExceptAsyncIterator.cs} (78%) rename FoundationDB.Client/{FdbIntersectIterator.cs => Linq/Async/Iterators/IntersectAsyncIterator.cs} (77%) rename FoundationDB.Client/{FdbQueryMergeIterator.cs => Linq/Async/Iterators/MergeAsyncIterator.cs} (81%) rename FoundationDB.Client/{FdbMergeSortIterator.cs => Linq/Async/Iterators/MergeSortAsyncIterator.cs} (75%) rename FoundationDB.Client/Linq/{Iterators/FdbObserverIterator.cs => Async/Iterators/ObserverAsyncIterator.cs} (75%) rename FoundationDB.Client/Linq/{Iterators/FdbParallelSelectAsyncIterator.cs => Async/Iterators/ParallelSelectAsyncIterator.cs} (84%) rename FoundationDB.Client/Linq/{Iterators/FdbPrefetchingAsyncIterator.cs => Async/Iterators/PrefetchingAsyncIterator.cs} (78%) rename FoundationDB.Client/Linq/{Iterators/FdbSelectManyAsyncIterator.cs => Async/Iterators/SelectManyAsyncIterator.cs} (73%) rename FoundationDB.Client/Linq/{Iterators/FdbTakeWhileAsyncIterator.cs => Async/Iterators/TakeWhileAsyncIterator.cs} (77%) rename FoundationDB.Client/Linq/{Iterators/FdbWhereAsyncIterator.cs => Async/Iterators/WhereAsyncIterator.cs} (70%) rename FoundationDB.Client/Linq/{Iterators/FdbWhereSelectAsyncIterator.cs => Async/Iterators/WhereSelectAsyncIterator.cs} (71%) rename FoundationDB.Client/Linq/{Iterators/FdbWindowingAsyncIterator.cs => Async/Iterators/WindowingAsyncIterator.cs} (87%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.EmptySequence.cs => AsyncEnumerable.EmptySequence.cs} (56%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.EnumerableIterator.cs => AsyncEnumerable.EnumerableIterator.cs} (88%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.EnumerableSequence.cs => AsyncEnumerable.EnumerableSequence.cs} (82%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.Iterators.cs => AsyncEnumerable.Iterators.cs} (54%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.OrderedSequence.cs => AsyncEnumerable.OrderedSequence.cs} (56%) rename FoundationDB.Client/Linq/{FdbAsyncEnumerable.Sorters.cs => AsyncEnumerable.Sorters.cs} (97%) create mode 100644 FoundationDB.Client/Linq/AsyncEnumerable.cs rename FoundationDB.Client/Linq/{FdbAsyncMode.cs => AsyncIterationHint.cs} (97%) create mode 100644 FoundationDB.Client/Linq/EnumerableExtensions.cs delete mode 100644 FoundationDB.Client/Linq/FdbAsyncEnumerable.cs rename FoundationDB.Client/{Async => Linq}/IAsyncEnumerable.cs (85%) rename FoundationDB.Client/{Async => Linq}/IAsyncEnumerator.cs (91%) rename FoundationDB.Client/Linq/{IFdbAsyncOrderedEnumerable.cs => IAsyncOrderedEnumerable.cs} (68%) delete mode 100644 FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs delete mode 100644 FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs rename FoundationDB.Client/Linq/{FdbParallelQueryOptions.cs => ParallelAsyncQueryOptions.cs} (96%) create mode 100644 FoundationDB.Client/Linq/QueryStatistics.cs diff --git a/FdbBurner/Program.cs b/FdbBurner/Program.cs index 8e898300f..dc962817d 100644 --- a/FdbBurner/Program.cs +++ b/FdbBurner/Program.cs @@ -1,15 +1,16 @@ -using FoundationDB.Client; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - + namespace FdbBurner { + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Globalization; + using System.Linq; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using FoundationDB; + using FoundationDB.Client; public class Program { diff --git a/FdbShell/Program.cs b/FdbShell/Program.cs index 5fbbc26de..d7719ec29 100644 --- a/FdbShell/Program.cs +++ b/FdbShell/Program.cs @@ -1,22 +1,49 @@ -using FoundationDB.Async; -using FoundationDB.Client; -using FoundationDB.Filters.Logging; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using Mono.Options; -using Mono.Terminal; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Linq; -using System.Net; -using System.Text; -using System.Threading; -using System.Threading.Tasks; +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace FdbShell { + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.IO; + using System.Linq; + using System.Net; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using Doxense; + using FoundationDB.Client; + using FoundationDB.Filters.Logging; + using FoundationDB.Layers.Directories; + using FoundationDB.Layers.Tuples; + using Mono.Options; + using Mono.Terminal; public static class Program { @@ -382,7 +409,12 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) { var newPath = CombinePath(CurrentDirectoryPath, prm); var res = await RunAsyncCommand((db, log, ct) => BasicCommands.TryOpenCurrentDirectoryAsync(ParsePath(newPath), db, ct), cancel); - if (res == null) + if (res.Failed) + { + Console.Error.WriteLine("# Failed to open Directory {0}: {1}", newPath, res.Error.Message); + Console.Beep(); + } + else if (res.Value == null) { Console.WriteLine("# Directory {0} does not exist!", newPath); Console.Beep(); @@ -396,7 +428,11 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) else { var res = await RunAsyncCommand((db, log, ct) => BasicCommands.TryOpenCurrentDirectoryAsync(ParsePath(CurrentDirectoryPath), db, ct), cancel); - if (res.GetValueOrDefault() == null) + if (res.Failed) + { + Console.Error.WriteLine("# Failed to query Directory {0}: {1}", Program.CurrentDirectoryPath, res.Error.Message); + } + else if (res.Value == null) { Console.WriteLine("# Directory {0} does not exist anymore", CurrentDirectoryPath); } @@ -579,7 +615,7 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) case "wtf": { var result = await RunAsyncCommand((_, log, ct) => FdbCliCommands.RunFdbCliCommand("status details", null, clusterFile, log, ct), cancel); - if (result.HasFailed) break; + if (result.Failed) break; if (result.Value.ExitCode != 0) { Console.WriteLine("# fdbcli exited with code {0}", result.Value.ExitCode); diff --git a/FdbShell/Utils/PerfCounters.cs b/FdbShell/Utils/PerfCounters.cs index 70e4f1056..2b85b3255 100644 --- a/FdbShell/Utils/PerfCounters.cs +++ b/FdbShell/Utils/PerfCounters.cs @@ -1,20 +1,8 @@ -using FoundationDB.Async; -using FoundationDB.Client; -using FoundationDB.Filters.Logging; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Linq; -using System.Net; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - + namespace FdbShell { + using System; + using System.Diagnostics; public static class PerfCounters { diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Async/AsyncBuffer.cs index f1c3eb128..97b5a6cfc 100644 --- a/FoundationDB.Client/Async/AsyncBuffer.cs +++ b/FoundationDB.Client/Async/AsyncBuffer.cs @@ -28,7 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Collections.Generic; @@ -36,53 +36,54 @@ namespace FoundationDB.Async using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using FoundationDB; using JetBrains.Annotations; /// Buffer that holds a fixed number of items and can rate-limit the producer - /// - /// - public class AsyncBuffer : AsyncProducerConsumerQueue, IAsyncSource + /// + /// + public class AsyncBuffer : AsyncProducerConsumerQueue, IAsyncSource { #region Private Members... /// Transformation applied on the values - private readonly Func m_transform; + private readonly Func m_transform; /// Queue that holds items produced but not yet consumed /// The queue can sometime go over the limit because the Complete/Error message are added without locking - private readonly Queue> m_queue = new Queue>(); + private readonly Queue> m_queue = new Queue>(); #endregion #region Constructors... - public AsyncBuffer([NotNull] Func transform, int capacity) + public AsyncBuffer([NotNull] Func transform, int capacity) : base(capacity) { - if (transform == null) throw new ArgumentNullException("transform"); + Contract.NotNull(transform, nameof(transform)); m_transform = transform; } #endregion - #region IFdbAsyncTarget... + #region IAsyncTarget... - public override Task OnNextAsync(T value, CancellationToken ct) + public override Task OnNextAsync(TInput value, CancellationToken ct) { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); LogProducer("Received new value"); Task wait; lock (m_lock) { - if (m_done) return TaskHelpers.FromException(new InvalidOperationException("Cannot send any more values because this buffer has already completed")); + if (m_done) return Task.FromException(new InvalidOperationException("Cannot send any more values because this buffer has already completed")); if (m_queue.Count < m_capacity) { // quick path Enqueue_NeedsLocking(Maybe.Return(value)); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } // we are blocked, we will need to wait ! @@ -101,26 +102,12 @@ public override void OnCompleted() { LogProducer("Completion received"); m_done = true; - m_queue.Enqueue(Maybe.Nothing()); + m_queue.Enqueue(Maybe.Nothing()); WakeUpBlockedConsumer_NeedsLocking(); } } } -#if NET_4_0 - public override void OnError(Exception error) - { - lock (m_lock) - { - if (!m_done) - { - LogProducer("Error received: " + error.Message); - m_queue.Enqueue(Maybe.Error(error)); - WakeUpBlockedConsumer_NeedsLocking(); - } - } - } -#else public override void OnError(ExceptionDispatchInfo error) { lock (m_lock) @@ -128,14 +115,13 @@ public override void OnError(ExceptionDispatchInfo error) if (!m_done) { LogProducer("Error received: " + error.SourceException.Message); - m_queue.Enqueue(Maybe.Error(error)); + m_queue.Enqueue(Maybe.Error(error)); WakeUpBlockedConsumer_NeedsLocking(); } } } -#endif - private void Enqueue_NeedsLocking(Maybe value) + private void Enqueue_NeedsLocking(Maybe value) { m_queue.Enqueue(value); @@ -145,7 +131,7 @@ private void Enqueue_NeedsLocking(Maybe value) } } - private async Task WaitForNextFreeSlotThenEnqueueAsync(T value, Task wait, CancellationToken ct) + private async Task WaitForNextFreeSlotThenEnqueueAsync(TInput value, Task wait, CancellationToken ct) { ct.ThrowIfCancellationRequested(); @@ -162,16 +148,16 @@ private async Task WaitForNextFreeSlotThenEnqueueAsync(T value, Task wait, Cance #endregion - #region IFdbAsyncSource... + #region IAsyncSource... - public Task> ReceiveAsync(CancellationToken ct) + public Task> ReceiveAsync(CancellationToken ct) { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation>(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled>(ct); LogConsumer("Looking for next value..."); Task wait = null; - Maybe item; + Maybe item; lock (m_lock) { if (m_queue.Count > 0) @@ -183,12 +169,12 @@ public Task> ReceiveAsync(CancellationToken ct) else if (m_done) { LogConsumer("The queue was complete"); - item = Maybe.Nothing(); + item = Maybe.Nothing(); } else { wait = MarkConsumerAsBlocked_NeedsLocking(ct); - item = default(Maybe); // needed to please the compiler + item = default(Maybe); // needed to please the compiler } } @@ -200,26 +186,26 @@ public Task> ReceiveAsync(CancellationToken ct) return Task.FromResult(ProcessResult(item)); } - private Maybe ProcessResult(Maybe item) + private Maybe ProcessResult(Maybe item) { if (item.IsEmpty) { // that was the last one ! m_receivedLast = true; LogConsumer("Received last item"); - return Maybe.Nothing(); + return Maybe.Nothing(); } LogConsumer("Applying transform on item"); - return Maybe.Apply(item, m_transform); + return Maybe.Apply(item, m_transform); } - private async Task> WaitForNextItemAsync(Task wait, CancellationToken ct) + private async Task> WaitForNextItemAsync(Task wait, CancellationToken ct) { await wait.ConfigureAwait(false); LogConsumer("Wake up because one item arrived"); - Maybe item; + Maybe item; lock(m_lock) { ct.ThrowIfCancellationRequested(); diff --git a/FoundationDB.Client/Async/AsyncCancellableMutex.cs b/FoundationDB.Client/Async/AsyncCancellableMutex.cs index 8fdd48d95..d3e2d0f55 100644 --- a/FoundationDB.Client/Async/AsyncCancellableMutex.cs +++ b/FoundationDB.Client/Async/AsyncCancellableMutex.cs @@ -26,12 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using JetBrains.Annotations; /// Implements a async mutex that supports cancellation [DebuggerDisplay("Status={this.Task.Status}, CancellationState=({m_state}, {m_ct.IsCancellationRequested?\"alive\":\"cancelled\"})")] @@ -43,7 +44,6 @@ public class AsyncCancelableMutex : TaskCompletionSource // note: this is not really a mutex because there is no "Reset()" method (not possible to reset a TCS)... private static readonly Action s_cancellationCallback = CancellationHandler; - private static readonly AsyncCancelableMutex s_alreadyCompleted = CreateAlreadyDone(); /// Returns an already completed, new mutex instance private static AsyncCancelableMutex CreateAlreadyDone() @@ -53,10 +53,8 @@ private static AsyncCancelableMutex CreateAlreadyDone() return mtx; } - public static AsyncCancelableMutex AlreadyDone - { - get { return s_alreadyCompleted; } - } + /// Mutex that has already completed + public static AsyncCancelableMutex AlreadyDone { [NotNull] get; } = CreateAlreadyDone(); private const int STATE_NONE = 0; private const int STATE_SET = 1; diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Async/AsyncHelpers.cs index 418986bed..5216bb8c5 100644 --- a/FoundationDB.Client/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Async/AsyncHelpers.cs @@ -26,7 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +using Doxense.Linq; + +namespace Doxense.Async { using System; using System.Collections.Generic; @@ -39,13 +41,8 @@ namespace FoundationDB.Async public static class AsyncHelpers { internal static readonly Action NoOpCompletion = () => { }; -#if NET_4_0 - internal static readonly Action NoOpError = (e) => { }; - internal static readonly Action RethrowError = (e) => { throw e; }; -#else internal static readonly Action NoOpError = (e) => { }; internal static readonly Action RethrowError = (e) => { e.Throw(); }; -#endif #region Targets... @@ -53,11 +50,7 @@ public static class AsyncHelpers public static IAsyncTarget CreateTarget( Func onNextAsync, Action onCompleted = null, -#if NET_4_0 - Action onError = null -#else Action onError = null -#endif ) { return new AnonymousAsyncTarget(onNextAsync, onCompleted, onError); @@ -67,11 +60,7 @@ public static IAsyncTarget CreateTarget( public static IAsyncTarget CreateTarget( Action onNext, Action onCompleted = null, -#if NET_4_0 - Action onError = null -#else Action onError = null -#endif ) { return new AnonymousTarget(onNext, onCompleted, onError); @@ -82,26 +71,22 @@ public static Task Publish(this IAsyncTarget target, Maybe result, Canc { Contract.Requires(target != null); - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); if (result.HasValue) - { + { // we have the next value return target.OnNextAsync(result.Value, ct); } - else if (result.HasFailed) - { -#if NET_4_0 - target.OnError(result.Error); -#else + + if (result.Failed) + { // we have failed target.OnError(result.CapturedError); -#endif - return TaskHelpers.CompletedTask; - } - else - { - target.OnCompleted(); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } + + // this is the end of the stream + target.OnCompleted(); + return Task.CompletedTask; } /// Wrapper class for use with async lambda callbacks @@ -112,20 +97,12 @@ internal sealed class AnonymousAsyncTarget : IAsyncTarget private readonly Action m_onCompleted; -#if NET_4_0 - private readonly Action m_onError; -#else private readonly Action m_onError; -#endif public AnonymousAsyncTarget( Func onNextAsync, Action onCompleted, -#if NET_4_0 - Action onError -#else Action onError -#endif ) { m_onNextAsync = onNextAsync; @@ -143,11 +120,7 @@ public void OnCompleted() m_onCompleted(); } -#if NET_4_0 - public void OnError(Exception error) -#else public void OnError(ExceptionDispatchInfo error) -#endif { m_onError(error); } @@ -161,23 +134,15 @@ internal sealed class AnonymousTarget : IAsyncTarget private readonly Action m_onCompleted; -#if NET_4_0 - private readonly Action m_onError; -#else private readonly Action m_onError; -#endif public AnonymousTarget( Action onNext, Action onCompleted, -#if NET_4_0 - Action onError -#else Action onError -#endif ) { - if (onNext == null) throw new ArgumentNullException("onNext"); + Contract.NotNull(onNext, nameof(onNext)); m_onNext = onNext; m_onCompleted = onCompleted; @@ -191,21 +156,9 @@ public Task OnNextAsync(T value, CancellationToken ct) public void OnCompleted() { - if (m_onCompleted != null) - { - m_onCompleted(); - } + m_onCompleted?.Invoke(); } -#if NET_4_0 - public void OnError(Exception error) - { - if (m_onError != null) - m_onError(error); - else - throw error; - } -#else public void OnError(ExceptionDispatchInfo error) { if (m_onError != null) @@ -213,20 +166,98 @@ public void OnError(ExceptionDispatchInfo error) else error.Throw(); } -#endif } #endregion #region Pumps... + /// Consumes all the elements of the source, and publish them to the target, one by one and in order + /// Source that produces elements asynchronously + /// Target that consumes elements asynchronously + /// Cancellation token + /// Task that completes when all the elements of the source have been published to the target, or fails if on the first error, or the token is cancelled unexpectedly + /// The pump will only read one element at a time, and wait for it to be published to the target, before reading the next element. public static async Task PumpToAsync(this IAsyncSource source, IAsyncTarget target, CancellationToken ct) { - if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); + ct.ThrowIfCancellationRequested(); + + bool notifiedCompletion = false; + bool notifiedError = false; - using (var pump = new AsyncPump(source, target)) + try { - await pump.PumpAsync(stopOnFirstError: true, ct: ct).ConfigureAwait(false); + //LogPump("Starting pump"); + + while (!ct.IsCancellationRequested) + { + //LogPump("Waiting for next"); + + var current = await source.ReceiveAsync(ct).ConfigureAwait(false); + + //LogPump("Received " + (current.HasValue ? "value" : current.Failed ? "error" : "completion") + ", publishing... " + current); + if (ct.IsCancellationRequested) + { + // REVIEW: should we notify the target? + // REVIEW: if the item is IDisposble, who will clean up? + break; + } + + // push the data/error/completion on to the target, which will triage and update its state accordingly + await target.Publish(current, ct).ConfigureAwait(false); + + if (current.Failed) + { // bounce the error back to the caller + //REVIEW: SHOULD WE? We poush the error to the target, and the SAME error to the caller... who should be responsible for handling it? + // => target should know about the error (to cancel something) + // => caller should maybe also know that the pump failed unexpectedly.... + notifiedError = true; + current.ThrowForNonSuccess(); // throws an exception right here + return; // should not be reached + } + else if (current.IsEmpty) + { // the source has completed, stop the pump + //LogPump("Completed"); + notifiedCompletion = true; + return; + } + } + + // notify cancellation if it happend while we were pumping + if (ct.IsCancellationRequested) + { + //LogPump("We were cancelled!"); + throw new OperationCanceledException(ct); + } + } + catch (Exception e) + { + //LogPump("Failed: " + e); + + if (!notifiedCompletion && !notifiedError) + { // notify the target that we crashed while fetching the next + try + { + //LogPump("Push error down to target: " + e.Message); + target.OnError(ExceptionDispatchInfo.Capture(e)); + notifiedError = true; + } + catch (Exception x) when (!x.IsFatalError()) + { + //LogPump("Failed to notify target of error: " + x.Message); + } + } + + throw; + } + finally + { + if (!notifiedCompletion) + { // we must be sure to complete the target if we haven't done so yet! + //LogPump("Notify target of completion due to unexpected conditions"); + target.OnCompleted(); + } + //LogPump("Stopped pump"); } } @@ -235,7 +266,7 @@ public static async Task> PumpToListAsync(this IAsyncSource source { if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); - var buffer = new FoundationDB.Linq.FdbAsyncEnumerable.Buffer(); + var buffer = new Buffer(); var target = CreateTarget( (x, _) => buffer.Add(x) @@ -265,18 +296,18 @@ public static AsyncTaskBuffer CreateUnorderedAsyncBuffer(int capacity) #region Transforms... - public static AsyncTransform CreateAsyncTransform(Func> transform, IAsyncTarget> target, TaskScheduler scheduler = null) + public static AsyncTransform CreateAsyncTransform(Func> transform, IAsyncTarget> target, TaskScheduler scheduler = null) { - return new AsyncTransform(transform, target, scheduler); + return new AsyncTransform(transform, target, scheduler); } - public static async Task> TransformToListAsync(IAsyncSource source, Func> transform, CancellationToken ct, int? maxConcurrency = null, TaskScheduler scheduler = null) + public static async Task> TransformToListAsync(IAsyncSource source, Func> transform, CancellationToken ct, int? maxConcurrency = null, TaskScheduler scheduler = null) { ct.ThrowIfCancellationRequested(); - using (var queue = CreateOrderPreservingAsyncBuffer(maxConcurrency ?? 32)) + using (var queue = CreateOrderPreservingAsyncBuffer(maxConcurrency ?? 32)) { - using (var pipe = CreateAsyncTransform(transform, queue, scheduler)) + using (var pipe = CreateAsyncTransform(transform, queue, scheduler)) { // start the output pump var output = PumpToListAsync(queue, ct); diff --git a/FoundationDB.Client/Async/AsyncOrderingMode.cs b/FoundationDB.Client/Async/AsyncOrderingMode.cs index 52c5ef327..b7106d96c 100644 --- a/FoundationDB.Client/Async/AsyncOrderingMode.cs +++ b/FoundationDB.Client/Async/AsyncOrderingMode.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; diff --git a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs index 81ec68d41..cbcaeea3f 100644 --- a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs +++ b/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs @@ -28,7 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Diagnostics; @@ -60,7 +60,7 @@ public abstract class AsyncProducerConsumerQueue : IAsyncTarget, IDisposab protected AsyncProducerConsumerQueue(int capacity) { - if (capacity <= 0) throw new ArgumentOutOfRangeException("capacity", "Capacity must be greater than zero"); + if (capacity <= 0) throw new ArgumentOutOfRangeException(nameof(capacity), "Capacity must be greater than zero"); m_capacity = capacity; } @@ -69,11 +69,7 @@ protected AsyncProducerConsumerQueue(int capacity) public abstract void OnCompleted(); -#if NET_4_0 - public abstract void OnError(Exception error); -#else public abstract void OnError(ExceptionDispatchInfo error); -#endif /// Delcare the producer as beeing blocked on a full queue /// @@ -82,7 +78,7 @@ protected Task MarkProducerAsBlocked_NeedsLocking(CancellationToken ct) { if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(ct); + return Task.FromCanceled(ct); } if (m_producerLock.IsCompleted) { @@ -108,7 +104,7 @@ protected Task MarkConsumerAsBlocked_NeedsLocking(CancellationToken ct) { if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(ct); + return Task.FromCanceled(ct); } if (m_consumerLock.IsCompleted) { @@ -123,7 +119,11 @@ protected void WakeUpBlockedConsumer_NeedsLocking() { if (m_consumerLock.Set(async: true)) { - LogProducer("Woke up blocked consumer"); + LogProducer("Woke up blocked consumer"); + } + else + { + LogProducer("Consumer was already unblocked?"); } } diff --git a/FoundationDB.Client/Async/AsyncPump.cs b/FoundationDB.Client/Async/AsyncPump.cs index 601fb7b30..05f2c0041 100644 --- a/FoundationDB.Client/Async/AsyncPump.cs +++ b/FoundationDB.Client/Async/AsyncPump.cs @@ -28,7 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Async +namespace Doxense.Async { using JetBrains.Annotations; using System; @@ -121,12 +121,12 @@ public async Task PumpAsync(bool stopOnFirstError, CancellationToken ct) m_state = STATE_WAITING_FOR_NEXT; var current = await m_source.ReceiveAsync(ct).ConfigureAwait(false); - LogPump("Received " + (current.HasValue ? "value" : current.HasFailed ? "error" : "completion") + ", publishing..."); + LogPump("Received " + (current.HasValue ? "value" : current.Failed ? "error" : "completion") + ", publishing..."); m_state = STATE_PUBLISHING_TO_TARGET; await m_target.Publish(current, ct).ConfigureAwait(false); - if (current.HasFailed && stopOnFirstError) + if (current.Failed && stopOnFirstError) { m_state = STATE_FAILED; LogPump("Stopping after this error"); diff --git a/FoundationDB.Client/Async/AsyncTaskBuffer.cs b/FoundationDB.Client/Async/AsyncTaskBuffer.cs index 421840df4..5f86164bf 100644 --- a/FoundationDB.Client/Async/AsyncTaskBuffer.cs +++ b/FoundationDB.Client/Async/AsyncTaskBuffer.cs @@ -28,7 +28,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Collections.Generic; @@ -62,25 +62,25 @@ public class AsyncTaskBuffer : AsyncProducerConsumerQueue>, IAsyncSou public AsyncTaskBuffer(AsyncOrderingMode mode, int capacity) : base(capacity) { - if (mode != AsyncOrderingMode.ArrivalOrder && mode != AsyncOrderingMode.CompletionOrder) throw new ArgumentOutOfRangeException("mode", "Unsupported ordering mode"); + if (mode != AsyncOrderingMode.ArrivalOrder && mode != AsyncOrderingMode.CompletionOrder) throw new ArgumentOutOfRangeException(nameof(mode), "Unsupported ordering mode"); m_mode = mode; } #endregion - #region IFdbAsyncTarget... + #region IAsyncTarget... public override Task OnNextAsync(Task task, CancellationToken ct) { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); LogProducer("Received task #" + task.Id + " (" + task.Status + ")"); Task wait; lock (m_lock) { - if (m_done) return TaskHelpers.FromException(new InvalidOperationException("Cannot send any more values because this buffer has already completed")); + if (m_done) return Task.FromException(new InvalidOperationException("Cannot send any more values because this buffer has already completed")); if (m_queue.Count < m_capacity) { // quick path @@ -100,7 +100,7 @@ public override Task OnNextAsync(Task task, CancellationToken ct) } } - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } // we are blocked, we will need to wait ! @@ -124,7 +124,7 @@ private void NotifyConsumerOfTaskCompletion_NeedsLocking() /// Observe the completion of a task to wake up the consumer private void ObserveTaskCompletion([NotNull] Task task) { - var _ = task.ContinueWith( + task.ContinueWith( (t, state) => { LogProducer("Task #" + t.Id + " " + t.Status); @@ -154,21 +154,6 @@ public override void OnCompleted() } } -#if NET_4_0 - public override void OnError(Exception error) - { - lock (m_lock) - { - if (!m_done) - { - LogProducer("Error received: " + error.Message); - m_queue.AddLast(new LinkedListNode>(TaskHelpers.FromException(error))); - WakeUpBlockedConsumer_NeedsLocking(); - if (m_mode == AsyncOrderingMode.CompletionOrder) NotifyConsumerOfTaskCompletion_NeedsLocking(); - } - } - } -#else public override void OnError(ExceptionDispatchInfo error) { lock (m_lock) @@ -176,13 +161,12 @@ public override void OnError(ExceptionDispatchInfo error) if (!m_done) { LogProducer("Error received: " + error.SourceException.Message); - m_queue.AddLast(new LinkedListNode>(TaskHelpers.FromException(error.SourceException))); + m_queue.AddLast(new LinkedListNode>(Task.FromException(error.SourceException))); WakeUpBlockedConsumer_NeedsLocking(); if (m_mode == AsyncOrderingMode.CompletionOrder) NotifyConsumerOfTaskCompletion_NeedsLocking(); } } } -#endif private void Enqueue_NeedsLocking(Task task) { @@ -196,22 +180,29 @@ private async Task WaitForNextFreeSlotThenEnqueueAsync(Task task, [NotNull] T await wait.ConfigureAwait(false); - LogProducer("Wake up because one slot got freed"); + LogProducer("Woke up because one slot got freed"); lock (m_lock) { Contract.Assert(m_queue.Count < m_capacity); Enqueue_NeedsLocking(task); + + if (m_mode == AsyncOrderingMode.CompletionOrder) + { // we need to observe task completion to wake up the consumer as soon as one is ready ! + LogConsumer("Task still pending after wait, and must be observed"); + ObserveTaskCompletion(task); + } + } } #endregion - #region IFdbAsyncSource... + #region IAsyncSource... public Task> ReceiveAsync(CancellationToken ct) { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation>(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled>(ct); LogConsumer("Looking for next value..."); @@ -228,7 +219,8 @@ public Task> ReceiveAsync(CancellationToken ct) throw new InvalidOperationException("Last item has already been received"); } - var current = m_queue.First; + var queue = m_queue; + var current = queue.First; if (current != null) { if (m_mode == AsyncOrderingMode.ArrivalOrder) @@ -236,7 +228,7 @@ public Task> ReceiveAsync(CancellationToken ct) if (current.Value == null || current.Value.IsCompleted) { // it's ready - m_queue.RemoveFirst(); + queue.RemoveFirst(); LogConsumer("First task #" + current.Value.Id + " was already " + current.Value.Status); return CompleteTask(current.Value); } @@ -247,21 +239,22 @@ public Task> ReceiveAsync(CancellationToken ct) else { // note: if one is already completed, it will be return immediately ! - while(current != null) + while (current != null) { - if (current.Value != null && current.Value.IsCompleted) + var t = current.Value; + if (t != null && t.IsCompleted) { - m_queue.Remove(current); - LogConsumer("Found task #" + current.Value.Id + " that was already " + current.Value.Status); - return CompleteTask(current.Value); + queue.Remove(current); + LogConsumer("Found task #" + t.Id + " that was already " + t.Status); + return CompleteTask(t); } current = current.Next; } // in case of completion, it would be the last - if (m_queue.First == m_queue.Last && m_queue.First.Value == null) + if (queue.First == queue.Last && queue.First.Value == null) { // last one - m_queue.Clear(); + queue.Clear(); m_receivedLast = true; LogConsumer("Received completion notification"); return CompleteTask(null); @@ -314,11 +307,7 @@ private async Task> WaitForTaskToCompleteAsync([NotNull] Task task, catch(Exception e) { LogConsumer("Notified that task #" + task + " failed"); -#if NET_4_0 - return Maybe.Error(e); -#else return Maybe.Error(ExceptionDispatchInfo.Capture(e)); -#endif } } @@ -336,13 +325,15 @@ protected Task MarkConsumerAsAwaitingCompletion_NeedsLocking(CancellationToken c { Contract.Requires(m_mode == AsyncOrderingMode.CompletionOrder); - if (m_completionLock.IsCompleted) + var cl = m_completionLock; + if (cl.IsCompleted) { LogConsumer("Creating new task completion lock"); - m_completionLock = new AsyncCancelableMutex(ct); + cl = new AsyncCancelableMutex(ct); + m_completionLock = cl; } LogConsumer("marked as waiting for task completion"); - return m_completionLock.Task; + return cl.Task; } #endregion @@ -353,6 +344,7 @@ protected override void Dispose(bool disposing) { if (disposing) { + LogConsumer("Disposing consumer!"); lock (m_lock) { m_done = true; @@ -360,6 +352,7 @@ protected override void Dispose(bool disposing) m_consumerLock.Abort(); m_completionLock.Abort(); m_queue.Clear(); + LogConsumer("Consumer has been disposed"); } } } diff --git a/FoundationDB.Client/Async/AsyncTransform.cs b/FoundationDB.Client/Async/AsyncTransform.cs index c9d0886d9..36fa36241 100644 --- a/FoundationDB.Client/Async/AsyncTransform.cs +++ b/FoundationDB.Client/Async/AsyncTransform.cs @@ -26,28 +26,27 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using JetBrains.Annotations; using System; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; /// Pump that takes items from a source, transform them, and outputs them - /// - /// - public sealed class AsyncTransform : IAsyncTarget, IDisposable + public sealed class AsyncTransform : IAsyncTarget, IDisposable { - private readonly IAsyncTarget> m_target; - private readonly Func> m_transform; + private readonly IAsyncTarget> m_target; + private readonly Func> m_transform; private readonly TaskScheduler m_scheduler; private bool m_done; - public AsyncTransform([NotNull] Func> transform, [NotNull] IAsyncTarget> target, TaskScheduler scheduler = null) + public AsyncTransform([NotNull] Func> transform, [NotNull] IAsyncTarget> target, TaskScheduler scheduler = null) { - if (transform == null) throw new ArgumentNullException("transform"); - if (target == null) throw new ArgumentNullException("target"); + Contract.NotNull(transform, nameof(transform)); + Contract.NotNull(target, nameof(target)); m_transform = transform; m_target = target; @@ -55,16 +54,16 @@ public AsyncTransform([NotNull] Func> transform, [ } /// Target of the transform - public IAsyncTarget> Target { get { return m_target; } } + public IAsyncTarget> Target { get { return m_target; } } /// Optional scheduler used to run the tasks public TaskScheduler Scheduler { get { return m_scheduler; } } #region IAsyncTarget... - public Task OnNextAsync(T value, CancellationToken ct) + public Task OnNextAsync(TInput value, CancellationToken ct) { - if (ct.IsCancellationRequested) return TaskHelpers.CompletedTask; + if (ct.IsCancellationRequested) return Task.CompletedTask; if (m_done) throw new InvalidOperationException("Cannot send any more values because this transform has already completed"); @@ -73,7 +72,7 @@ public Task OnNextAsync(T value, CancellationToken ct) // we start the task here, but do NOT wait for its completion! // It is the job of the target to handle that (and ordering) - Task task; + Task task; if (m_scheduler == null) { // execute inline task = m_transform(value, ct); @@ -83,7 +82,7 @@ public Task OnNextAsync(T value, CancellationToken ct) task = Task.Factory.StartNew( (state) => { - var prms = (Tuple, T, CancellationToken>)state; + var prms = (Tuple, TInput, CancellationToken>)state; return prms.Item1.m_transform(prms.Item2, prms.Item3); }, Tuple.Create(this, value, ct), @@ -97,12 +96,8 @@ public Task OnNextAsync(T value, CancellationToken ct) } catch(Exception e) { -#if NET_4_0 - m_target.OnError(e); -#else m_target.OnError(ExceptionDispatchInfo.Capture(e)); -#endif - return TaskHelpers.FromException(e); + return Task.FromException(e); } } @@ -111,15 +106,6 @@ public void OnCompleted() Dispose(); } -#if NET_4_0 - public void OnError(Exception e) - { - if (!m_done) - { - m_target.OnError(e); - } - } -#else public void OnError(ExceptionDispatchInfo e) { if (!m_done) @@ -127,7 +113,6 @@ public void OnError(ExceptionDispatchInfo e) m_target.OnError(e); } } -#endif #endregion diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Async/AsyncTransformQueue.cs index b5bb6c3ff..38c57a174 100644 --- a/FoundationDB.Client/Async/AsyncTransformQueue.cs +++ b/FoundationDB.Client/Async/AsyncTransformQueue.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Collections.Generic; @@ -40,7 +40,7 @@ namespace FoundationDB.Async /// Implements an async queue that asynchronously transform items, outputing them in arrival order, while throttling the producer /// Type of the input elements (from the inner async iterator) /// Type of the output elements (produced by an async lambda) - internal class AsyncTransformQueue : IAsyncBuffer + public class AsyncTransformQueue : IAsyncBuffer { private readonly Func> m_transform; private readonly Queue>> m_queue = new Queue>>(); @@ -53,7 +53,7 @@ internal class AsyncTransformQueue : IAsyncBuffer> transform, int capacity, TaskScheduler scheduler) { - if (transform == null) throw new ArgumentNullException(nameof(transform)); + Contract.NotNull(transform, nameof(transform)); if (capacity <= 0) throw new ArgumentOutOfRangeException(nameof(capacity), "Capacity must be greater than zero"); m_transform = transform; @@ -61,16 +61,14 @@ public AsyncTransformQueue([NotNull] Func... + #region IAsyncBuffer... /// Returns the current number of items in the queue public int Count { get { -#if !CORE_CLR Debugger.NotifyOfCrossThreadDependency(); -#endif lock (m_lock) { return m_queue.Count; @@ -79,22 +77,17 @@ public int Count } /// Returns the maximum capacity of the queue - public int Capacity - { - get { return m_capacity; } - } + public int Capacity => m_capacity; /// Returns true if the producer is blocked (queue is full) public bool IsConsumerBlocked { get { -#if !CORE_CLR Debugger.NotifyOfCrossThreadDependency(); -#endif lock (m_lock) { - return m_blockedConsumer != null && m_blockedConsumer.Task.IsCompleted; + return m_blockedConsumer?.Task.IsCompleted == true; } } } @@ -104,12 +97,10 @@ public bool IsProducerBlocked { get { -#if !CORE_CLR Debugger.NotifyOfCrossThreadDependency(); -#endif lock (m_lock) { - return m_blockedProducer != null && m_blockedProducer.Task.IsCompleted; + return m_blockedProducer?.Task.IsCompleted == true; } } } @@ -121,7 +112,7 @@ public Task DrainAsync() #endregion - #region IFdbAsyncTarget... + #region IAsyncTarget... private static async Task> ProcessItemHandler(object state) { @@ -137,11 +128,7 @@ private static async Task> ProcessItemHandler(object state) } catch (Exception e) { -#if NET_4_0 - return Maybe.Error(e); -#else return Maybe.Error(ExceptionDispatchInfo.Capture(e)); -#endif } } @@ -208,13 +195,7 @@ public void OnCompleted() } } - public void OnError( -#if NET_4_0 - Exception error -#else - ExceptionDispatchInfo error -#endif - ) + public void OnError(ExceptionDispatchInfo error) { lock(m_lock) { @@ -230,26 +211,26 @@ ExceptionDispatchInfo error #endregion - #region IFdbAsyncBatchTarget... + #region IAsyncBatchTarget... - public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken ct) + public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken cancellationToken) { - if (batch == null) throw new ArgumentNullException(nameof(batch)); + Contract.NotNull(batch, nameof(batch)); if (batch.Length == 0) return; - if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); + if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); //TODO: optimized version ! foreach (var item in batch) { - await OnNextAsync(item, ct).ConfigureAwait(false); + await OnNextAsync(item, cancellationToken).ConfigureAwait(false); } } #endregion - #region IFdbAsyncSource... + #region IAsyncSource... public Task> ReceiveAsync(CancellationToken ct) { @@ -308,11 +289,7 @@ private async Task> ReceiveWhenDoneAsync(Task> tas } catch(Exception e) { -#if NET_4_0 - return Maybe.Error(e); -#else return Maybe.Error(ExceptionDispatchInfo.Capture(e)); -#endif } finally { @@ -366,7 +343,7 @@ private async Task> ReceiveSlowAsync(Task waiter, CancellationTok #endregion - #region IFdbAsyncBatchSource... + #region IAsyncBatchSource... public Task[]> ReceiveBatchAsync(int count, CancellationToken ct) { @@ -450,7 +427,7 @@ private bool DrainItems_NeedsLocking([NotNull] List> buffer, int private Task WaitForNextItem_NeedsLocking(CancellationToken ct) { - if (m_done) return TaskHelpers.CompletedTask; + if (m_done) return Task.CompletedTask; Contract.Requires(m_blockedConsumer == null || m_blockedConsumer.Task.IsCompleted); @@ -462,21 +439,15 @@ private Task WaitForNextItem_NeedsLocking(CancellationToken ct) private void WakeUpProducer_NeedsLocking() { var waiter = Interlocked.Exchange(ref m_blockedProducer, null); - if (waiter != null) - { - waiter.Set(async: true); - } + waiter?.Set(async: true); } private void WakeUpConsumer_NeedLocking() { var waiter = Interlocked.Exchange(ref m_blockedConsumer, null); - if (waiter != null) - { - waiter.Set(async: true); - } + waiter?.Set(async: true); } - + } } diff --git a/FoundationDB.Client/Async/IAsyncBuffer.cs b/FoundationDB.Client/Async/IAsyncBuffer.cs index 4aa5e345f..3f304f89e 100644 --- a/FoundationDB.Client/Async/IAsyncBuffer.cs +++ b/FoundationDB.Client/Async/IAsyncBuffer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Threading.Tasks; @@ -34,7 +34,7 @@ namespace FoundationDB.Async /// Defines a producer/consumer buffer queue that can hold several items before blocking the producer /// Type of elements entering the buffer /// Type of elements exiting the buffer. Can be different from if the buffer also transforms the elements. - interface IAsyncBuffer : IAsyncTarget, IAsyncSource + public interface IAsyncBuffer : IAsyncTarget, IAsyncSource { /// Returns the current number of items in the buffer int Count { get; } diff --git a/FoundationDB.Client/Async/IAsyncPump.cs b/FoundationDB.Client/Async/IAsyncPump.cs index 12011eca1..748c1506a 100644 --- a/FoundationDB.Client/Async/IAsyncPump.cs +++ b/FoundationDB.Client/Async/IAsyncPump.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using JetBrains.Annotations; using System; diff --git a/FoundationDB.Client/Async/IAsyncSource.cs b/FoundationDB.Client/Async/IAsyncSource.cs index e98c488a4..b3c5e71e7 100644 --- a/FoundationDB.Client/Async/IAsyncSource.cs +++ b/FoundationDB.Client/Async/IAsyncSource.cs @@ -26,7 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async + +namespace Doxense.Async { using System; using System.Threading; diff --git a/FoundationDB.Client/Async/IAsyncTarget.cs b/FoundationDB.Client/Async/IAsyncTarget.cs index df46a1cf0..cfdbb65e6 100644 --- a/FoundationDB.Client/Async/IAsyncTarget.cs +++ b/FoundationDB.Client/Async/IAsyncTarget.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Runtime.ExceptionServices; @@ -51,11 +51,7 @@ public interface IAsyncTarget /// Notifies the target that tere was an exception, and that no more values will be published /// The error that occurred -#if NET_4_0 - void OnError(Exception error); -#else void OnError(ExceptionDispatchInfo error); -#endif } } diff --git a/FoundationDB.Client/Async/Maybe.cs b/FoundationDB.Client/Async/Maybe.cs index 3846dc03c..2151c49be 100644 --- a/FoundationDB.Client/Async/Maybe.cs +++ b/FoundationDB.Client/Async/Maybe.cs @@ -26,238 +26,556 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense { using System; using System.Collections.Generic; + using System.Runtime; + using System.Runtime.CompilerServices; using System.Runtime.ExceptionServices; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; - /// Either has a value, nothing, or an exception - /// Type of the value - public struct Maybe : IEquatable>, IEquatable + public /*readonly*/ struct Maybe : IEquatable>, IEquatable, IComparable>, IComparable, IFormattable { + /// Réprésente un résultat vide (no computation) + public static readonly Maybe Nothing = new Maybe(); + + /// Représente un résultat correspondant à la valeur par défaut du type (0, false, null) + public static readonly Maybe Default = new Maybe(default(T)); + + /// Cached completed Task that always return an empty value + public static readonly Task> EmptyTask = Task.FromResult(default(Maybe)); + + #region Private Fields... + + // ================================================================================== + // m_hasValue | m_value | m_error | description + // ================================================================================== + // True | Resultat | null | Le calcul a produit un résultat (qui peut etre le défaut du type, mais qui n'est pas "vide") + // False | - | null | Le calcul n'a pas produit de résultat + // False | - | Exception | Le calcul a provoqué une exception /// If true, there is a value. If false, either no value or an exception - public readonly bool HasValue; + private readonly T m_value; /// If HasValue is true, holds the value. Else, contains default(T) - public readonly T Value; + private readonly bool m_hasValue; /// If HasValue is false optinally holds an error that was captured private readonly object m_errorContainer; // either an Exception, or an ExceptionDispatchInfo + #endregion + + public Maybe(T value) + { + m_hasValue = true; + m_value = value; + m_errorContainer = null; + } + internal Maybe(bool hasValue, T value, object errorContainer) { -#if NET_4_0 - Contract.Requires(errorContainer == null || (errorContainer is Exception)); -#else Contract.Requires(errorContainer == null || (errorContainer is Exception) || (errorContainer is ExceptionDispatchInfo)); -#endif - this.HasValue = hasValue; - this.Value = value; + m_hasValue = hasValue; + m_value = value; m_errorContainer = errorContainer; } - /// Returns an empty value - public static Maybe Empty { get { return default(Maybe); } } + /// There is a value + /// !(IsEmpty || HasFailed) + public bool HasValue + { + [TargetedPatchingOptOut("Performance critical to inline this type of method across NGen image boundaries")] + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => m_hasValue; + } - /// Cached completed Task that always return an empty value - public static readonly Task> EmptyTask = Task.FromResult(default(Maybe)); + /// Returns the value if the computation succeeded + /// If the value is empty + /// If the value has failed to compute + public T Value + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => m_hasValue ? m_value : ThrowInvalidState(); + } - /// Returns the stored value, of the default value for the type if it was empty - /// + /// Returns the value if the computation succeeded, or default() in all other cases + [TargetedPatchingOptOut("Performance critical to inline this type of method across NGen image boundaries")] + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public T GetValueOrDefault() { - ThrowForNonSuccess(); - return this.Value; + return m_value; } - /// If true, then there is no value and no error - public bool IsEmpty { get { return !this.HasValue && m_errorContainer == null; } } - - /// If true then there was an error captured - public bool HasFailed { get { return m_errorContainer != null; } } - - /// Return the captured Error, or null if there wasn't any public Exception Error { + [TargetedPatchingOptOut("Performance critical to inline this type of method across NGen image boundaries")] + [Pure] get { -#if !NET_4_0 - var edi = m_errorContainer as ExceptionDispatchInfo; - if (edi != null) return edi.SourceException; -#endif - return m_errorContainer as Exception; + return m_errorContainer is ExceptionDispatchInfo edi + ? edi.SourceException + : m_errorContainer as Exception; } } -#if !NET_4_0 /// Return the captured error context, or null if there wasn't any - public ExceptionDispatchInfo CapturedError - { - get - { - var exception = m_errorContainer as Exception; - if (exception != null) return ExceptionDispatchInfo.Capture(exception); - - var edi = m_errorContainer as ExceptionDispatchInfo; - if (edi != null) return edi; + public ExceptionDispatchInfo CapturedError => m_errorContainer is Exception exception ? ExceptionDispatchInfo.Capture(exception) : m_errorContainer as ExceptionDispatchInfo; - return null; - } + /// The value failed to compute + /// !(HasValue || IsEmpty) + public bool Failed + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return m_errorContainer != null; } } -#endif /// Rethrows any captured error, if there was one. public void ThrowForNonSuccess() { if (m_errorContainer != null) { - var exception = m_errorContainer as Exception; -#if !NET_4_0 - if (exception == null) + if (!(m_errorContainer is Exception exception)) { - ((ExceptionDispatchInfo)m_errorContainer).Throw(); + ((ExceptionDispatchInfo) m_errorContainer).Throw(); return; // never reached, but helps with code analysis } -#endif throw exception; } } internal object ErrorContainer { - get { return m_errorContainer; } + [Pure] + get => m_errorContainer; } - public override bool Equals(object obj) + /// No value was returned + /// !(HasValue || Failed) + public bool IsEmpty { - System.Diagnostics.Trace.WriteLine("Maybe[" + this + "].Equals(object " + obj + ")"); - if (obj == null) return IsEmpty; - if (obj is Maybe) return Equals((Maybe)obj); - if (obj is T) return Equals((T)obj); - return false; + [Pure] + get => !m_hasValue && m_errorContainer == null; } - public bool Equals(Maybe other) + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + private T ThrowInvalidState() + { + if (m_errorContainer != null) throw new AggregateException("A computation has triggered an exception.", this.Error); + if (!m_hasValue) throw new InvalidOperationException("This computation has no value."); + throw new InvalidOperationException("This computation already has a value."); + } + + [Pure, NotNull] + public static Func, Maybe> Return([NotNull] Func computation) + { + return Bind(x => new Maybe(computation(x))); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Failure(Exception error) + { + return new Maybe(false, default(T), error); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Failure(ExceptionDispatchInfo error) + { + return new Maybe(false, default(T), error); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static explicit operator T(Maybe m) + { + return m.Value; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator Maybe(T value) + { + return new Maybe(value); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator Maybe(Exception error) { - System.Diagnostics.Trace.WriteLine("Maybe[" + this + "].Equals(Maybe " + other + ")"); - return this.HasValue == other.HasValue && object.ReferenceEquals(this.ErrorContainer, other.ErrorContainer) && EqualityComparer.Default.Equals(this.Value, other.Value); + return Failure(error); } + public bool Equals(Maybe other) + { + if (m_hasValue) return other.m_hasValue && EqualityComparer.Default.Equals(m_value, other.m_value); + if (m_errorContainer != null) return !m_hasValue && m_errorContainer.Equals(other.m_errorContainer); + return !other.m_hasValue & other.m_errorContainer == null; + } public bool Equals(T other) { - System.Diagnostics.Trace.WriteLine("Maybe[" + this + "].Equals(T " + other + ")"); - return this.HasValue && this.ErrorContainer == null && EqualityComparer.Default.Equals(this.Value, other); + return m_hasValue && EqualityComparer.Default.Equals(m_value, other); + } + + public override bool Equals(object obj) + { + if (obj == null) return !m_hasValue; + if (obj is T value) return Equals(value); + if (obj is Maybe maybe) return Equals(maybe); + if (obj is Exception err) return !m_hasValue && err.Equals(m_errorContainer); + return false; } public override int GetHashCode() { - if (this.ErrorContainer != null) return this.ErrorContainer.GetHashCode(); - if (!this.HasValue) return 0; - return EqualityComparer.Default.GetHashCode(this.Value); + return m_hasValue ? EqualityComparer.Default.GetHashCode(m_value) : m_errorContainer?.GetHashCode() ?? -1; } - public static bool operator ==(Maybe left, Maybe right) + public int CompareTo(Maybe other) { - return left.Equals(right); + // in order: "nothing", then values, then errors + + if (m_hasValue) + { // Some + if (other.m_hasValue) return Comparer.Default.Compare(m_value, other.m_value); + if (other.m_errorContainer != null) return -1; // values come before errors + return +1; // values come after nothing + } + + if (m_errorContainer != null) + { // Error + if (other.m_hasValue | other.m_errorContainer == null) return +1; // errors come after everything except errors + //note: this is tricky, because we cannot realy sort Exceptions, so this sort may not be stable :( + // => the "only" way would be to compare their hash codes! + return ReferenceEquals(m_errorContainer, other.m_errorContainer) ? 0 : m_errorContainer.GetHashCode().CompareTo(other.m_errorContainer.GetHashCode()); + } + + // Nothing comes before everything except nothing + return other.m_hasValue | other.m_errorContainer != null ? -1 : 0; } + public int CompareTo(T other) + { + // in order: "nothing", then values, then errors + if (!m_hasValue) + { + return m_errorContainer != null ? +1 : -1; + } + return Comparer.Default.Compare(m_value, other); + } + + public string ToString(string format, IFormatProvider formatProvider) + { + if (this.Failed) return ""; + if (!this.HasValue) return ""; + if (this.Value == null) return ""; //REVIEW: => "" ? + if (this.Value is IFormattable fmt) return fmt.ToString(format, formatProvider); + return this.Value.ToString(); + } + + public override string ToString() + { + return ToString(null, null); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator ==(Maybe left, T right) { return left.Equals(right); } - public static bool operator ==(Nullable> left, Nullable> right) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator !=(Maybe left, T right) + { + return !left.Equals(right); + } + + public static bool operator >(Maybe left, T right) + { + return left.CompareTo(right) > 0; + } + + public static bool operator >=(Maybe left, T right) + { + return left.CompareTo(right) >= 0; + } + + public static bool operator <(Maybe left, T right) + { + return left.CompareTo(right) < 0; + } + + public static bool operator <=(Maybe left, T right) { - if (!right.HasValue) return !left.HasValue || !left.Value.HasValue; - if (!left.HasValue) return !right.Value.HasValue; - return left.Value.Equals(right.Value); + return left.CompareTo(right) <= 0; } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator ==(Maybe left, Maybe right) + { + return left.Equals(right); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator !=(Maybe left, Maybe right) { return !left.Equals(right); } - public static bool operator !=(Maybe left, T right) + public static bool operator >(Maybe left, Maybe right) { - return !left.Equals(right); + return left.CompareTo(right) > 0; } - public static bool operator !=(Nullable> left, Nullable> right) + public static bool operator >=(Maybe left, Maybe right) { - if (!right.HasValue) return left.HasValue && left.Value.HasValue; - if (!left.HasValue) return right.Value.HasValue; - return !left.Value.Equals(right.Value); + return left.CompareTo(right) >= 0; } - public override string ToString() + public static bool operator <(Maybe left, Maybe right) { - if (this.ErrorContainer != null) return ""; - if (!this.HasValue) return ""; - if (this.Value == null) return ""; - //TODO: consider adding '['/']' around the value, to distinguish a Maybe between a T in the console and the debugger ? - return this.Value.ToString(); + return left.CompareTo(right) < 0; } + + public static bool operator <=(Maybe left, Maybe right) + { + return left.CompareTo(right) <= 0; + } + + #region Function Binding... + + public static Func, Maybe> Bind(Func> computation) + { + return (x) => + { + if (x.m_errorContainer != null) return new Maybe(false, default(TResult), x.m_errorContainer); + if (!x.m_hasValue) return Maybe.Nothing; + + try + { + return computation(x.m_value); + } + catch (Exception e) + { + return Maybe.Failure(e); + } + }; + } + + public static Func, Maybe, Maybe> Bind(Func> computation) + { + return (x, y) => + { + if (x.m_errorContainer != null || y.m_errorContainer != null) return Maybe.Error(default(TResult), x.Error, y.Error); + if (x.m_hasValue && y.m_hasValue) + { + try + { + + return computation(x.m_value, y.m_value); + } + catch (Exception e) + { + return Maybe.Failure(e); + } + } + return Maybe.Nothing; + }; + } + + #endregion + } - /// - /// Helper methods for creating instances - /// + /// Helper class to deal with Maybe<T> monads public static class Maybe { - /// Wraps a value into a + /// Crée un Maybe<T> représentant une valeur connue + /// Type de la valeur + /// Valeur à convertir + /// Maybe<T> contenant la valeur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Maybe Return(T value) { - return new Maybe(true, value, null); + // ENTER THE MONAD ! + return new Maybe(value); + } + + /// Retourne un Maybe<T> correspondant à cette valeur + /// Type de la valeur + /// Valeur à convertir + /// Maybe<T> contenant cette valeur + /// Note: si T est un ReferenceType et que value est null, le Maybe retourné n'est pas vide (il a une valeur, qui est null). Il faut utiliser .IfNotNull() pour protéger contre les nullref + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe ToMaybe(this T value) + { + // ENTER THE MONAD ! + return new Maybe(value); + } + + /// Retourne un Maybe<T> correspondant à cette valeur + /// Type de la valeur + /// Valeur à convertir (ou null) + /// Maybe<T> contenant cette valeur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe ToMaybe(this T? value) + where T : struct + { + return value.HasValue ? new Maybe(value.Value) : Maybe.Nothing; } - /// Returns an empty + /// Convertit les référence null en Maybe.Nothing + /// Reference Type + /// Instance à protéger (peut être null) + /// Maybe.Nothing si l'instance est null, sinon un Maybe encapsulant cette instance + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe IfNotNull(this T value) + where T : class + { + // ENTER THE MONAD + return value == null ? Maybe.Nothing : new Maybe(value); + } + + /// Helper pour créer un Maybe<T>.Nothing + /// Type de la valeur + /// Maybe vide + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Maybe Nothing() { - return Maybe.Empty; + // ENTER THE MONAD ! + return default(Maybe); + } + + /// Helper pour créer un Maybe<T>.Nothing en utilisant le compilateur pour inférer le type de la valeur + /// Type de la valeur + /// Paramètre dont la valeur est ignorée, et qui sert juste à aider le compilateur à inférer le type + /// Maybe vide + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Nothing(T _) + { + // ENTER THE MONAD ! + return default(Maybe); + } + + /// Helper pour créer un Maybe<T> représentant une Exception + /// Type de la valeur + /// Exception à enrober + /// Maybe encapsulant l'erreur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Error(Exception error) + { + // ENTER THE MONAD ! + return Maybe.Failure(error); + } + + /// Helper pour créer un Maybe<T> représentant une Exception + /// Type de la valeur + /// Exception à enrober + /// Maybe encapsulant l'erreur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Error(ExceptionDispatchInfo error) + { + // ENTER THE MONAD ! + return Maybe.Failure(error); } - /// Capture an exception into a - public static Maybe Error(Exception e) + /// Helper pour créer un Maybe<T> représentant une Exception, en utilisant le compilateur pour inférer le type de la valeur + /// Type de la valeur + /// Paramètre dont la valeur est ignorée, et qui sert juste à aider le compilateur à inférer le type + /// Exception à enrober + /// Maybe encapsulant l'erreur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Error(T _, Exception error) { - return new Maybe(false, default(T), e); + // ENTER THE MONAD ! + return Maybe.Failure(error); } -#if !NET_4_0 - /// Capture an exception into a - public static Maybe Error(ExceptionDispatchInfo e) + /// Helper pour créer un Maybe<T> représentant une Exception, en utilisant le compilateur pour inférer le type de la valeur + /// Type de la valeur + /// Paramètre dont la valeur est ignorée, et qui sert juste à aider le compilateur à inférer le type + /// Exception à enrober + /// Maybe encapsulant l'erreur + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe Error(T _, ExceptionDispatchInfo error) { - return new Maybe(false, default(T), e); + // ENTER THE MONAD ! + return Maybe.Failure(error); + } + + /// Helper pour combiner des erreurs, en utilisant le compilateur pour inférer le type de la valeur + /// + /// Paramètre dont la valeur est ignorée, et qui sert juste à aider le compilateur à inférer le type + /// Première exception (peut être null) + /// Deuxième exception (peut être null) + /// Maybe encapsulant la ou les erreur. Si les deux erreurs sont présentes, elles sont combinées dans une AggregateException + [Pure] + public static Maybe Error(T _, Exception error0, Exception error1) + { + // Il faut au moins une des deux ! + Contract.Assert(error0 != null || error1 != null); + + if (error1 == null) + { + return Maybe.Failure(error0); + } + if (error0 == null) + { + return Maybe.Failure(error1); + } + return Maybe.Failure(new AggregateException(error0, error1)); + } + + /// Convertit un Maybe<T&;t en T? (lorsque T est un ValueType) + /// ValueType + /// Maybe à convertir + /// Version nullable du maybe, qui vaut default(T?) si le Maybe est Nothing, ou la valeur elle même s'il contient un résultat. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static T? ToNullable(this Maybe m) + where T : struct + { + // EXIT THE MONAD + //TODO: propager l'exception ? + return m.HasValue ? m.Value : default(T?); + } + + /// Convertit un T? en Maybe<T&;t (lorsque T est un ValueType) + /// ValueType + /// Nullable à convertir + /// Version maybe du nullable, qui vaut Nothing si le nullable est default(T?), ou la valeur elle même s'il contient un résultat. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Maybe FromNullable(T? value) + where T : struct + { + return value.HasValue ? new Maybe(value.Value) : default(Maybe); + } + + /// Retourne le résultat d'un Maybe, ou une valeur par défaut s'il est vide. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static T OrDefault(this Maybe m, T @default = default(T)) + { + // EXIT THE MONAD + return m.HasValue ? m.Value : @default; } -#endif /// Immediately apply a function to a value, and capture the result into a - public static Maybe Apply(T value, Func lambda) + [Pure] + public static Maybe Apply(T value, [NotNull, InstantHandle] Func lambda) { Contract.Requires(lambda != null); try { - return Return(lambda(value)); + return Return(lambda(value)); } catch (Exception e) { -#if NET_4_0 - return Error(e); -#else - return Error(ExceptionDispatchInfo.Capture(e)); -#endif + return Error(ExceptionDispatchInfo.Capture(e)); } } /// Immediately apply a function to a value, and capture the result into a - public static Maybe Apply(T value, [NotNull] Func> lambda) + [Pure] + public static Maybe Apply(T value, [NotNull, InstantHandle] Func> lambda) { Contract.Requires(lambda != null); try @@ -266,49 +584,47 @@ public static Maybe Apply(T value, [NotNull] Func> lambda) } catch (Exception e) { -#if NET_4_0 - return Error(e); -#else - return Error(ExceptionDispatchInfo.Capture(e)); -#endif + return Error(ExceptionDispatchInfo.Capture(e)); } } /// Immediately apply a function to a value, and capture the result into a - public static Maybe Apply(Maybe value, [NotNull] Func lambda) + [Pure] + public static Maybe Apply(Maybe value, [NotNull, InstantHandle] Func lambda) { Contract.Requires(lambda != null); if (!value.HasValue) { - if (value.HasFailed) + if (value.Failed) { // keep the original error untouched - return new Maybe(false, default(R), value.ErrorContainer); + return new Maybe(false, default(TResult), value.ErrorContainer); } - return Nothing(); + return Nothing(); } try { - return Return(lambda(value.Value)); + return Return(lambda(value.Value)); } catch (Exception e) { - return Error(e); + return Error(e); } } /// Immediately apply a function to a value, and capture the result into a - public static Maybe Apply(Maybe value, [NotNull] Func> lambda) + [Pure] + public static Maybe Apply(Maybe value, [NotNull, InstantHandle] Func> lambda) { Contract.Requires(lambda != null); if (!value.HasValue) { - if (value.HasFailed) + if (value.Failed) { // keep the original error untouched - return new Maybe(false, default(R), value.ErrorContainer); + return new Maybe(false, default(TResult), value.ErrorContainer); } - return Nothing(); + return Nothing(); } try { @@ -316,11 +632,12 @@ public static Maybe Apply(Maybe value, [NotNull] Func> l } catch (Exception e) { - return Error(e); + return Error(e); } } /// Convert a completed into an equivalent + [Pure] public static Maybe FromTask([NotNull] Task task) { //REVIEW: should we return Maybe.Empty if task == null ? @@ -334,16 +651,16 @@ public static Maybe FromTask([NotNull] Task task) case TaskStatus.Faulted: { //TODO: pass the failed task itself as the error container? (we would keep the original callstack that way...) - var aggEx = task.Exception.Flatten(); - if (aggEx.InnerExceptions.Count == 1) + var aggEx = task.Exception?.Flatten(); + if (aggEx?.InnerExceptions.Count == 1) { - return Maybe.Error(aggEx.InnerException); + return Error(aggEx.InnerException); } - return Maybe.Error(aggEx); + return Error(aggEx); } case TaskStatus.Canceled: { - return Maybe.Error(new OperationCanceledException()); + return Error(new OperationCanceledException()); } default: { @@ -352,7 +669,8 @@ public static Maybe FromTask([NotNull] Task task) } } - /// Convert a completed into an equivalent + /// Convert a completed with being a , into an equivalent + [Pure] public static Maybe FromTask([NotNull] Task> task) { Contract.Requires(task != null); @@ -368,9 +686,9 @@ public static Maybe FromTask([NotNull] Task> task) var aggEx = task.Exception.Flatten(); if (aggEx.InnerExceptions.Count == 1) { - return Maybe.Error(aggEx.InnerException); + return Error(aggEx.InnerException); } - return Maybe.Error(aggEx); + return Error(aggEx); } case TaskStatus.Canceled: { @@ -384,10 +702,11 @@ public static Maybe FromTask([NotNull] Task> task) } /// Streamline a potentially failed Task<Maybe<T>> into a version that capture the error into the itself + [Pure] public static Task> Unwrap([NotNull] Task> task) { Contract.Requires(task != null); - switch(task.Status) + switch (task.Status) { case TaskStatus.RanToCompletion: { @@ -414,6 +733,24 @@ public static Task> Unwrap([NotNull] Task> task) } } + [Pure, NotNull] + private static Func, Maybe> Combine([NotNull] Func, Maybe> f, Func, Maybe> g) + { + return (mt) => g(f(mt)); + } + + [Pure, NotNull] + public static Func, Maybe> Bind([NotNull] Func> f, [NotNull] Func> g) + { + return Combine(Maybe.Bind(f), Maybe.Bind(g)); + } + + [Pure, NotNull] + public static Func, Maybe> Bind([NotNull] Func> f, [NotNull] Func, Maybe> g) + { + return Combine(Maybe.Bind(f), g); + } + } } diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Async/TaskHelpers.cs index 4d43e3999..995762043 100644 --- a/FoundationDB.Client/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Async/TaskHelpers.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Async { using System; using System.Threading; @@ -39,7 +39,7 @@ internal static class TaskHelpers { /// Helper type cache class - public static class Cache + public static class CachedTasks { public static readonly Task Default = Task.FromResult(default(T)); @@ -66,22 +66,18 @@ public static Func Identity } - /// Return a task that is already completed - // README: There is a Task.CompletedTask object in the BCL that is internal, and one 'easy' way to get access to it is via Task.Delay(0) that returns it if param is equal to 0... - public static readonly Task CompletedTask = Task.Delay(0); - /// Already completed task that returns false - public static readonly Task FalseTask = Task.FromResult(false); + public static readonly Task False = Task.FromResult(false); /// Already completed task that returns true - public static readonly Task TrueTask = Task.FromResult(true); + public static readonly Task True = Task.FromResult(true); /// Returns an already completed boolean task that is either true of false /// Value of the task /// Already completed task the returns public static Task FromResult(bool value) { - return value ? TrueTask : FalseTask; + return value ? TaskHelpers.True : TaskHelpers.False; } /// Returns a cached completed task that returns the default value of type @@ -89,7 +85,47 @@ public static Task FromResult(bool value) /// Task that is already completed, and returns default() public static Task Default() { - return Cache.Default; + return CachedTasks.Default; + } + + /// Fait en sorte que toute exception non gérée soit observée + /// Tâche, qui peut potentiellement déclencher une exception + /// La même task, mais avec une continuation qui viendra observer toute erreur + /// Cette méthode a pour unique but dans la vie de faire taire les warning du compilateur sur les tasks non awaitées (ou variable non utilisées) + public static void Observed(this TTask task) + where TTask : Task + { + if (task == null) return; + + // A la base en .NET 4.0, le destructeur des task rethrow les errors non observées sur le TP ce qui pouvait killer le process + // => il faut que quelqu'un "touche" a la propriété "Exception" de la task, pour empecher cela. + switch (task.Status) + { + case TaskStatus.Faulted: + case TaskStatus.Canceled: + TouchFaultedTask(task); + return; + + case TaskStatus.RanToCompletion: + return; + + default: + task.ContinueWith((t) => TouchFaultedTask(t), TaskContinuationOptions.OnlyOnFaulted); + return; + } + } + + private static void TouchFaultedTask(Task t) + { + // ReSharper disable once UnusedVariable + var error = t.Exception; +#if DEBUG + if (t.IsFaulted) + { + // C'est une mauvaise pratique, donc râle quand même dans les logs en mode debug! + System.Diagnostics.Debug.WriteLine($"### muted unobserved failed Task[{t.Id}]: [{error?.InnerException?.GetType().Name}] {error?.InnerException?.Message}"); + } +#endif } /// Continue processing a task, if it succeeded @@ -118,9 +154,9 @@ public static async Task Then(this Task task, [NotNull] Func i /// If is null public static Task Inline([NotNull] Func lambda, CancellationToken ct = default(CancellationToken)) { - if (lambda == null) throw new ArgumentNullException("lambda"); + if (lambda == null) throw new ArgumentNullException(nameof(lambda)); - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { var res = lambda(); @@ -142,13 +178,13 @@ public static async Task Then(this Task task, [NotNull] Func i public static Task Inline([NotNull] Action action, T1 arg1, CancellationToken ct = default(CancellationToken)) { // note: if action is null, then there is a bug in the caller, and it should blow up instantly (will help preserving the call stack) - if (action == null) throw new ArgumentNullException("action"); + if (action == null) throw new ArgumentNullException(nameof(action)); // for all other exceptions, they will be wrapped in the returned task - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { action(arg1); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } catch (Exception e) { @@ -168,13 +204,13 @@ public static async Task Then(this Task task, [NotNull] Func i public static Task Inline([NotNull] Action action, T1 arg1, T2 arg2, CancellationToken ct = default(CancellationToken)) { // note: if action is null, then there is a bug in the caller, and it should blow up instantly (will help preserving the call stack) - if (action == null) throw new ArgumentNullException("action"); + if (action == null) throw new ArgumentNullException(nameof(action)); // for all other exceptions, they will be wrapped in the returned task - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { action(arg1, arg2); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } catch (Exception e) { @@ -196,13 +232,13 @@ public static async Task Then(this Task task, [NotNull] Func i public static Task Inline([NotNull] Action action, T1 arg1, T2 arg2, T3 arg3, CancellationToken ct = default(CancellationToken)) { // note: if action is null, then there is a bug in the caller, and it should blow up instantly (will help preserving the call stack) - if (action == null) throw new ArgumentNullException("action"); + if (action == null) throw new ArgumentNullException(nameof(action)); // for all other exceptions, they will be wrapped in the returned task - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { action(arg1, arg2, arg3); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } catch (Exception e) { @@ -226,13 +262,13 @@ public static async Task Then(this Task task, [NotNull] Func i public static Task Inline([NotNull] Action action, T1 arg1, T2 arg2, T3 arg3, T4 arg4, CancellationToken ct = default(CancellationToken)) { // note: if action is null, then there is a bug in the caller, and it should blow up instantly (will help preserving the call stack) - if (action == null) throw new ArgumentNullException("action"); + if (action == null) throw new ArgumentNullException(nameof(action)); // for all other exceptions, they will be wrapped in the returned task - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { action(arg1, arg2, arg3, arg4); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } catch (Exception e) { @@ -258,13 +294,13 @@ public static async Task Then(this Task task, [NotNull] Func i public static Task Inline([NotNull] Action action, T1 arg1, T2 arg2, T3 arg3, T4 arg4, T5 arg5, CancellationToken ct = default(CancellationToken)) { // note: if action is null, then there is a bug in the caller, and it should blow up instantly (will help preserving the call stack) - if (action == null) throw new ArgumentNullException("action"); + if (action == null) throw new ArgumentNullException(nameof(action)); // for all other exceptions, they will be wrapped in the returned task - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); try { action(arg1, arg2, arg3, arg4, arg5); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } catch (Exception e) { @@ -293,27 +329,11 @@ public static Func> WithCancellation { - if (ct.IsCancellationRequested) return FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); return lambda(value); }; } - /// Returns a cancelled Task that is linked with a specific token - /// Type of the result of the task - /// Cancellation token that should already be cancelled - /// Task in the cancelled state that is linked with this cancellation token - public static Task FromCancellation(CancellationToken ct) - { - // There is a Task.FromCancellation() method in the BCL, but unfortunately it is internal :( - // The "best" way I've seen to emulate the same behavior, is creating a fake task (with a dummy action) with the same alread-cancelled CancellationToken - // This should throw the correct TaskCanceledException that is linked with this token - - // ensure that it is actually cancelled, so that we don't deadlock - if (!ct.IsCancellationRequested) throw new InvalidOperationException(); - - return new Task(Cache.Nop, ct); - } - /// Returns a cancelled Task that is not linked to any particular token /// Type of the result of the task /// Task in the cancelled state @@ -358,7 +378,7 @@ public static Task FromFailure(Exception e, CancellationToken ct) if (e is OperationCanceledException) { if (ct.IsCancellationRequested) - return FromCancellation(ct); + return Task.FromCanceled(ct); else return Canceled(); } diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 6816d6dd7..c7f56bab1 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -35,11 +35,10 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using FoundationDB.Client.Core; using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; using FoundationDB.Layers.Directories; /// FoundationDB database session handle diff --git a/FoundationDB.Client/FdbMergeQueryExtensions.cs b/FoundationDB.Client/FdbMergeQueryExtensions.cs index a0926b66a..4c3a765ba 100644 --- a/FoundationDB.Client/FdbMergeQueryExtensions.cs +++ b/FoundationDB.Client/FdbMergeQueryExtensions.cs @@ -26,15 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Linq; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; + using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; - using FoundationDB.Async; - using FoundationDB.Linq; public static class FdbMergeQueryExtensions { @@ -42,7 +43,7 @@ public static class FdbMergeQueryExtensions #region MergeSort (x OR y) [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> MergeSort([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> MergeSort([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -50,7 +51,7 @@ public static IFdbAsyncEnumerable> MergeSort([N Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); - return new FdbMergeSortIterator, TKey, KeyValuePair>( + return new MergeSortAsyncIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -60,7 +61,7 @@ public static IFdbAsyncEnumerable> MergeSort([N } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -69,7 +70,7 @@ public static IFdbAsyncEnumerable MergeSort([NotNull] th Contract.NotNull(resultSelector, nameof(resultSelector)); trans.EnsureCanRead(); - return new FdbMergeSortIterator, TKey, TResult>( + return new MergeSortAsyncIterator, TKey, TResult>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -79,11 +80,11 @@ public static IFdbAsyncEnumerable MergeSort([NotNull] th } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Union([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Union([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); - return new FdbMergeSortIterator( + return new MergeSortAsyncIterator( sources, null, keySelector, @@ -93,10 +94,10 @@ public static IFdbAsyncEnumerable Union([NotNull] IEnume } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Union([NotNull] IEnumerable> sources, IComparer keyComparer = null) + public static IAsyncEnumerable Union([NotNull] IEnumerable> sources, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); - return new FdbMergeSortIterator( + return new MergeSortAsyncIterator( sources, null, (x) => x, @@ -110,7 +111,7 @@ public static IFdbAsyncEnumerable Union([NotNull] IEnumerable< #region Intersect (x AND y) [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -118,7 +119,7 @@ public static IFdbAsyncEnumerable> Intersect([N Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); - return new FdbIntersectIterator, TKey, KeyValuePair>( + return new IntersectAsyncIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -128,12 +129,12 @@ public static IFdbAsyncEnumerable> Intersect([N } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? trans.EnsureCanRead(); - return new FdbIntersectIterator, TKey, TResult>( + return new IntersectAsyncIterator, TKey, TResult>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -143,11 +144,11 @@ public static IFdbAsyncEnumerable Intersect([NotNull] th } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, IAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); - return new FdbIntersectIterator( + return new IntersectAsyncIterator( new[] { first, second }, null, keySelector, @@ -157,11 +158,11 @@ public static IFdbAsyncEnumerable Intersect([NotNull] th } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect([NotNull] this IFdbAsyncEnumerable first, IFdbAsyncEnumerable second, IComparer comparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, IAsyncEnumerable second, IComparer comparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); - return new FdbIntersectIterator( + return new IntersectAsyncIterator( new [] { first, second }, null, (x) => x, @@ -171,11 +172,11 @@ public static IFdbAsyncEnumerable Intersect([NotNull] this IFd } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); - return new FdbIntersectIterator( + return new IntersectAsyncIterator( sources, null, keySelector, @@ -185,10 +186,10 @@ public static IFdbAsyncEnumerable Intersect([NotNull] IE } [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Intersect([NotNull] IEnumerable> sources, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] IEnumerable> sources, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); - return new FdbIntersectIterator( + return new IntersectAsyncIterator( sources, null, (x) => x, @@ -209,7 +210,7 @@ public static IFdbAsyncEnumerable Intersect([NotNull] IEnumera /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -217,7 +218,7 @@ public static IFdbAsyncEnumerable> Except([NotN Contract.NotNull(keySelector, nameof(keySelector)); trans.EnsureCanRead(); - return new FdbExceptIterator, TKey, KeyValuePair>( + return new ExceptAsyncIterator, TKey, KeyValuePair>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -234,7 +235,7 @@ public static IFdbAsyncEnumerable> Except([NotN /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) { Contract.NotNull(ranges, nameof(ranges)); return Except(trans, ranges.Select(r => KeySelectorPair.Create(r)), keySelector, keyComparer); @@ -250,12 +251,12 @@ public static IFdbAsyncEnumerable> Except([NotN /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? trans.EnsureCanRead(); - return new FdbExceptIterator, TKey, TResult>( + return new ExceptAsyncIterator, TKey, TResult>( ranges.Select(range => trans.GetRange(range, new FdbRangeOptions { Mode = FdbStreamingMode.Iterator })), default(int?), keySelector, @@ -274,7 +275,7 @@ public static IFdbAsyncEnumerable Except([NotNull] this /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { Contract.NotNull(ranges, nameof(ranges)); return Except(trans, ranges.Select(r => KeySelectorPair.Create(r)), keySelector, resultSelector, keyComparer); @@ -289,12 +290,12 @@ public static IFdbAsyncEnumerable Except([NotNull] this /// Instance used to compare keys /// Async query that returns only the elements that are in , and not in [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbAsyncEnumerable first, [NotNull] IFdbAsyncEnumerable second, [NotNull] Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Except([NotNull] this IAsyncEnumerable first, [NotNull] IAsyncEnumerable second, [NotNull] Func keySelector, IComparer keyComparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); Contract.NotNull(keySelector, nameof(keySelector)); - return new FdbExceptIterator( + return new ExceptAsyncIterator( new[] { first, second }, null, keySelector, @@ -310,11 +311,11 @@ public static IFdbAsyncEnumerable Except([NotNull] this /// Instance used to compare elements /// Async query that returns only the elements that are in , and not in [Pure, NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Except([NotNull] this IFdbAsyncEnumerable first, [NotNull] IFdbAsyncEnumerable second, IComparer comparer = null) + public static IAsyncEnumerable Except([NotNull] this IAsyncEnumerable first, [NotNull] IAsyncEnumerable second, IComparer comparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); - return new FdbExceptIterator( + return new ExceptAsyncIterator( new[] { first, second }, null, (x) => x, diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index ad3a208d3..088a12ed9 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense.Async; + namespace FoundationDB.Client { using System; @@ -34,8 +36,6 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Client.Utils; using JetBrains.Annotations; /// @@ -252,7 +252,7 @@ public static Task RunReadAsync([NotNull] IFdbDatabase db, [NotNull] Func(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var context = new FdbOperationContext(db, FdbTransactionMode.ReadOnly | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, asyncHandler, onDone); @@ -285,7 +285,7 @@ public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Func(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, asyncHandler, onDone); @@ -296,7 +296,7 @@ public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] Action(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); return ExecuteInternal(db, context, handler, onDone); diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index 4ed43c223..286c53f37 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -58,10 +58,10 @@ public FdbRangeChunk(bool hasMore, KeyValuePair[] chunk, int itera } /// Returns the number of results in this chunk - public int Count { get { return this.Chunk != null ? this.Chunk.Length : 0; } } + public int Count => Chunk?.Length ?? 0; /// Returns true if the chunk does not contain any item. - public bool IsEmpty { get { return this.Chunk == null || this.Chunk.Length == 0; } } + public bool IsEmpty => this.Chunk == null || this.Chunk.Length == 0; /// Returns the first item in the chunk /// Note that if the range is reversed, then the first item will be GREATER than the last ! @@ -111,8 +111,8 @@ public int GetSize() [NotNull] public KeyValuePair[] Decode([NotNull] Func keyHandler, [NotNull] Func valueHandler) { - if (keyHandler == null) throw new ArgumentNullException("keyHandler"); - if (valueHandler == null) throw new ArgumentNullException("valueHandler"); + if (keyHandler == null) throw new ArgumentNullException(nameof(keyHandler)); + if (valueHandler == null) throw new ArgumentNullException(nameof(valueHandler)); var chunk = this.Chunk; var results = new KeyValuePair[chunk.Length]; @@ -140,9 +140,9 @@ public KeyValuePair[] Decode([NotNull] Func[] Decode([NotNull] KeySubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); - if (valueEncoder == null) throw new ArgumentNullException("valueEncoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); + if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); var chunk = this.Chunk; var results = new KeyValuePair[chunk.Length]; @@ -168,8 +168,8 @@ public KeyValuePair[] Decode([NotNull] KeySubspace s [NotNull] public KeyValuePair[] Decode([NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) { - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); - if (valueEncoder == null) throw new ArgumentNullException("valueEncoder"); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); + if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); var chunk = this.Chunk; var results = new KeyValuePair[chunk.Length]; @@ -192,7 +192,7 @@ public KeyValuePair[] Decode([NotNull] IKeyEncoder([NotNull] Func handler) { - if (handler == null) throw new ArgumentNullException("handler"); + if (handler == null) throw new ArgumentNullException(nameof(handler)); var results = new T[this.Count]; for (int i = 0; i < results.Length; i++) @@ -209,8 +209,8 @@ public T[] DecodeKeys([NotNull] Func handler) [NotNull] public T[] DecodeKeys([NotNull] KeySubspace subspace, [NotNull] IKeyEncoder keyEncoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); var results = new T[this.Count]; for(int i = 0; i< results.Length;i++) @@ -227,7 +227,7 @@ public T[] DecodeKeys([NotNull] KeySubspace subspace, [NotNull] IKeyEncoder([NotNull] IKeyEncoder keyEncoder) { - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); + if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); var results = new T[this.Count]; for (int i = 0; i < results.Length; i++) @@ -244,7 +244,7 @@ public T[] DecodeKeys([NotNull] IKeyEncoder keyEncoder) [NotNull] public T[] DecodeValues([NotNull] Func handler) { - if (handler == null) throw new ArgumentNullException("handler"); + if (handler == null) throw new ArgumentNullException(nameof(handler)); var results = new T[this.Count]; for (int i = 0; i < results.Length; i++) @@ -261,7 +261,7 @@ public T[] DecodeValues([NotNull] Func handler) [NotNull] public T[] DecodeValues([NotNull] IValueEncoder valueEncoder) { - if (valueEncoder == null) throw new ArgumentNullException("valueEncoder"); + if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); var results = new T[this.Count]; for (int i = 0; i < results.Length; i++) diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 92ac13c6b..7b9d3afe9 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -29,6 +29,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //enable this to enable verbose traces when doing paging //#define DEBUG_RANGE_PAGING +using Doxense.Linq; + namespace FoundationDB.Client { using System; @@ -36,10 +38,9 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Client.Utils; - using FoundationDB.Linq; + using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; public partial class FdbRangeQuery @@ -47,7 +48,7 @@ public partial class FdbRangeQuery /// Async iterator that fetches the results by batch, but return them one by one [DebuggerDisplay("State={m_state}, Current={m_current}, Iteration={Iteration}, AtEnd={AtEnd}, HasMore={HasMore}")] - private sealed class PagingIterator : FdbAsyncIterator[]> + private sealed class PagingIterator : AsyncIterator[]> { #region Iterable Properties... @@ -100,14 +101,14 @@ public PagingIterator([NotNull] FdbRangeQuery query, IFdbReadOnlyTransaction this.Transaction = transaction ?? query.Transaction; } - protected override FdbAsyncIterator[]> Clone() + protected override AsyncIterator[]> Clone() { return new PagingIterator(this.Query, this.Transaction); } #region IFdbAsyncEnumerator... - protected override async Task OnFirstAsync(CancellationToken ct) + protected override async Task OnFirstAsync() { this.RemainingCount = this.Query.Limit; this.RemainingSize = this.Query.TargetBytes; @@ -141,7 +142,7 @@ protected override async Task OnFirstAsync(CancellationToken ct) return true; } - protected override Task OnNextAsync(CancellationToken ct) + protected override Task OnNextAsync() { // Make sure that we are not called while the previous fetch is still running if (this.PendingReadTask != null && !this.PendingReadTask.IsCompleted) @@ -155,17 +156,16 @@ protected override Task OnNextAsync(CancellationToken ct) } // slower path, we need to actually read the first batch... - return FetchNextPageAsync(ct); + return FetchNextPageAsync(); } /// Asynchronously fetch a new page of results /// True if Chunk contains a new page of results. False if all results have been read. - private Task FetchNextPageAsync(CancellationToken ct) + private Task FetchNextPageAsync() { Contract.Requires(!this.AtEnd); Contract.Requires(this.Iteration >= 0); - ct.ThrowIfCancellationRequested(); this.Transaction.EnsureCanRead(); this.Iteration++; @@ -185,19 +185,19 @@ private Task FetchNextPageAsync(CancellationToken ct) // select the appropriate streaming mode if purpose is not default switch(m_mode) { - case FdbAsyncMode.Iterator: + case AsyncIterationHint.Iterator: { // the caller is responsible for calling MoveNext(..) and deciding if it wants to continue or not.. options.Mode = FdbStreamingMode.Iterator; break; } - case FdbAsyncMode.All: + case AsyncIterationHint.All: { // we are in a ToList or ForEach, we want to read everything in as few chunks as possible options.Mode = FdbStreamingMode.WantAll; break; } - case FdbAsyncMode.Head: + case AsyncIterationHint.Head: { // the caller only expect one (or zero) values options.Mode = FdbStreamingMode.Iterator; diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index eaa84ed29..83db267ab 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //enable this to enable verbose traces when doing paging //#define DEBUG_RANGE_ITERATOR + namespace FoundationDB.Client { using System; @@ -36,9 +37,10 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Linq; + using Doxense.Linq; + using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; public partial class FdbRangeQuery @@ -46,7 +48,7 @@ public partial class FdbRangeQuery /// Async iterator that fetches the results by batch, but return them one by one [DebuggerDisplay("State={m_state}, Current={m_current}, RemainingInChunk={m_itemsRemainingInChunk}, OutOfChunks={m_outOfChunks}")] - private sealed class ResultIterator : FdbAsyncIterator + private sealed class ResultIterator : AsyncIterator { private readonly FdbRangeQuery m_query; @@ -57,7 +59,7 @@ private sealed class ResultIterator : FdbAsyncIterator private readonly Func, T> m_resultTransform; /// Iterator used to read chunks from the database - private IFdbAsyncEnumerator[]> m_chunkIterator; + private IAsyncEnumerator[]> m_chunkIterator; /// True if we have reached the last page private bool m_outOfChunks; @@ -82,22 +84,22 @@ public ResultIterator([NotNull] FdbRangeQuery query, IFdbReadOnlyTransaction m_resultTransform = transform; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { return new ResultIterator(m_query, m_transaction, m_resultTransform); } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { // on first call, setup the page iterator if (m_chunkIterator == null) { - m_chunkIterator = new PagingIterator(m_query, m_transaction).GetEnumerator(m_mode); + m_chunkIterator = new PagingIterator(m_query, m_transaction).GetEnumerator(m_transaction.Cancellation, m_mode); } - return TaskHelpers.TrueTask; + return TaskHelpers.True; } - protected override Task OnNextAsync(CancellationToken ct) + protected override Task OnNextAsync() { if (m_itemsRemainingInChunk > 0) { // we need can get another one from the batch @@ -110,23 +112,23 @@ protected override Task OnNextAsync(CancellationToken ct) #if DEBUG_RANGE_ITERATOR Debug.WriteLine("No more items and it was the last batch"); #endif - return TaskHelpers.FalseTask; + return TaskHelpers.False; } // slower path, we need to actually read the first batch... m_chunk = null; m_currentOffsetInChunk = -1; - return ReadAnotherBatchAsync(ct); + return ReadAnotherBatchAsync(); } - private async Task ReadAnotherBatchAsync(CancellationToken ct) + private async Task ReadAnotherBatchAsync() { Contract.Requires(m_itemsRemainingInChunk == 0 && m_currentOffsetInChunk == -1 && !m_outOfChunks); var iterator = m_chunkIterator; // start reading the next batch - if (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (await iterator.MoveNextAsync().ConfigureAwait(false)) { // we got a new chunk ! //note: Dispose() or Cleanup() maybe have been called concurrently! @@ -170,7 +172,7 @@ private bool ProcessNextItem() #region LINQ - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { var query = new FdbRangeQuery( m_transaction, @@ -184,7 +186,7 @@ public override FdbAsyncIterator Select(Func selector) return new FdbRangeQuery.ResultIterator(query, m_transaction, query.Transform); } - public override FdbAsyncIterator Take(int limit) + public override AsyncIterator Take(int limit) { return new ResultIterator(m_query.Take(limit), m_transaction, m_resultTransform); } diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index 07b950c23..8703e8e28 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using System.Threading; + namespace FoundationDB.Client { using System; @@ -34,13 +36,12 @@ namespace FoundationDB.Client using System.Globalization; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Linq; + using Doxense.Linq; using JetBrains.Annotations; /// Query describing an ongoing GetRange operation [DebuggerDisplay("Begin={Begin}, End={End}, Limit={Limit}, Mode={Mode}, Reverse={Reverse}, Snapshot={Snapshot}")] - public sealed partial class FdbRangeQuery : IFdbAsyncEnumerable + public sealed partial class FdbRangeQuery : IAsyncEnumerable { /// Construct a query with a set of initial settings @@ -268,12 +269,12 @@ public FdbRangeQuery UseTransaction([NotNull] IFdbReadOnlyTransaction transac public IAsyncEnumerator GetEnumerator() { - return this.GetEnumerator(FdbAsyncMode.Default); + return this.GetEnumerator(this.Transaction.Cancellation, AsyncIterationHint.Default); } - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { - return new ResultIterator(this, this.Transaction, this.Transform).GetEnumerator(mode); + return new ResultIterator(this, this.Transaction, this.Transform).GetEnumerator(ct, mode); } /// Return a list of all the elements of the range results @@ -281,7 +282,7 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) public Task> ToListAsync() { // ReSharper disable once InvokeAsExtensionMethod - return FdbAsyncEnumerable.ToListAsync(this, this.Transaction.Cancellation); + return AsyncEnumerable.ToListAsync(this, this.Transaction.Cancellation); } /// Return an array with all the elements of the range results @@ -289,7 +290,7 @@ public Task> ToListAsync() public Task ToArrayAsync() { // ReSharper disable once InvokeAsExtensionMethod - return FdbAsyncEnumerable.ToArrayAsync(this, this.Transaction.Cancellation); + return AsyncEnumerable.ToArrayAsync(this, this.Transaction.Cancellation); } /// Return the number of elements in the range, by reading them @@ -297,7 +298,7 @@ public Task ToArrayAsync() public Task CountAsync() { // ReSharper disable once InvokeAsExtensionMethod - return FdbAsyncEnumerable.CountAsync(this, this.Transaction.Cancellation); + return AsyncEnumerable.CountAsync(this, this.Transaction.Cancellation); } [NotNull] @@ -328,9 +329,9 @@ public FdbRangeQuery Select([NotNull] Func lambda) /// Filters the range results based on a predicate. /// Caution: filtering occurs on the client side ! [NotNull] - public IFdbAsyncEnumerable Where([NotNull] Func predicate) + public IAsyncEnumerable Where([NotNull] Func predicate) { - return FdbAsyncEnumerable.Where(this, predicate); + return AsyncEnumerable.Where(this, predicate); } public Task FirstOrDefaultAsync() @@ -392,7 +393,7 @@ public Task NoneAsync() public Task ForEachAsync([NotNull] Action action) { // ReSharper disable once InvokeAsExtensionMethod - return FdbAsyncEnumerable.ForEachAsync(this, action, this.Transaction.Cancellation); + return AsyncEnumerable.ForEachAsync(this, action, this.Transaction.Cancellation); } internal async Task HeadAsync(bool single, bool orDefault) @@ -402,7 +403,7 @@ internal async Task HeadAsync(bool single, bool orDefault) // we can use the EXACT streaming mode with Limit = 1|2, and it will work if TargetBytes is 0 if ((this.TargetBytes ?? 0) != 0 || (this.Mode != FdbStreamingMode.Iterator && this.Mode != FdbStreamingMode.Exact)) { // fallback to the default implementation - return await FdbAsyncEnumerable.Head(this, single, orDefault, this.Transaction.Cancellation).ConfigureAwait(false); + return await AsyncEnumerable.Head(this, single, orDefault, this.Transaction.Cancellation).ConfigureAwait(false); } var options = new FdbRangeOptions() @@ -442,9 +443,9 @@ internal async Task AnyOrNoneAsync(bool any) { // fallback to the default implementation // ReSharper disable InvokeAsExtensionMethod if (any) - return await FdbAsyncEnumerable.AnyAsync(this, this.Transaction.Cancellation); + return await AsyncEnumerable.AnyAsync(this, this.Transaction.Cancellation); else - return await FdbAsyncEnumerable.NoneAsync(this, this.Transaction.Cancellation); + return await AsyncEnumerable.NoneAsync(this, this.Transaction.Cancellation); // ReSharper restore InvokeAsExtensionMethod } diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index d9f4dcbd8..b39e7792f 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -29,6 +29,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY // enable this to help debug Transactions //#define DEBUG_TRANSACTIONS +using Doxense.Async; + namespace FoundationDB.Client { using System; @@ -37,10 +39,8 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using FoundationDB.Client.Core; using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; using JetBrains.Annotations; /// FounrationDB transaction handle. diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 6c3e245d8..07cf4394b 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; @@ -35,7 +36,7 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Linq; + using Doxense.Linq; using JetBrains.Annotations; /// Provides a set of extensions methods shared by all FoundationDB transaction implementations. @@ -822,7 +823,7 @@ public static async Task[]> GetBatchAsync([N /// Token used to cancel the operation /// Task returning the list of all the elements of the async enumerable returned by the last successfull call to . [ItemNotNull] - public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken ct) + public static Task> QueryAsync([NotNull] this IFdbReadOnlyRetryable db, [NotNull, InstantHandle] Func> handler, CancellationToken ct) { Contract.NotNull(db, nameof(db)); Contract.NotNull(handler, nameof(handler)); diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index 0415b5c72..d9a29aac2 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -26,15 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Filters.Logging { - using FoundationDB.Async; - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense; + using FoundationDB.Client; + using JetBrains.Annotations; [Flags] public enum FdbLoggingOptions diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index 1c91daab2..8b67d1fad 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense; + namespace FoundationDB.Filters.Logging { using System; @@ -34,7 +36,6 @@ namespace FoundationDB.Filters.Logging using System.Globalization; using System.Text; using System.Threading.Tasks; - using FoundationDB.Async; using FoundationDB.Client; using FoundationDB.Layers.Directories; using JetBrains.Annotations; @@ -236,7 +237,7 @@ public override string GetResult(KeyResolver resolver) { if (this.Error != null) return base.GetResult(resolver); - if (this.Result.HasFailed) return ""; + if (this.Result.Failed) return ""; if (!this.Result.HasValue) return ""; if (this.Result.Value == null) return ""; diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 847f6d805..be9de267d 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -65,15 +65,17 @@ + + - - - - + + + + @@ -82,14 +84,14 @@ - - - - - - - - + + + + + + + + @@ -128,11 +130,11 @@ - - - + + + - + @@ -178,12 +180,12 @@ - - - - - - + + + + + + @@ -191,21 +193,19 @@ - - - - - - - - - - + + + + + + + + + + - - - - + + diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 35b85c315..c7040f497 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -35,9 +35,9 @@ namespace FoundationDB.Layers.Directories using System.Threading.Tasks; using JetBrains.Annotations; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using FoundationDB.Filters.Logging; /// Provides a FdbDirectoryLayer class for managing directories in FoundationDB. @@ -991,7 +991,7 @@ private async Task FindAsync([NotNull] IFdbReadOnlyTransaction tr, [NotNul /// Returns the list of names and nodes of all children of the specified node [NotNull] - private IFdbAsyncEnumerable> SubdirNamesAndNodes([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IDynamicKeySubspace node) + private IAsyncEnumerable> SubdirNamesAndNodes([NotNull] IFdbReadOnlyTransaction tr, [NotNull] IDynamicKeySubspace node) { Contract.Requires(tr != null && node != null); diff --git a/FoundationDB.Client/Linq/FdbAsyncSequence.cs b/FoundationDB.Client/Linq/Async/AsyncSequence.cs similarity index 75% rename from FoundationDB.Client/Linq/FdbAsyncSequence.cs rename to FoundationDB.Client/Linq/Async/AsyncSequence.cs index 87ff3095b..0bf9480a4 100644 --- a/FoundationDB.Client/Linq/FdbAsyncSequence.cs +++ b/FoundationDB.Client/Linq/Async/AsyncSequence.cs @@ -26,39 +26,35 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq + namespace Doxense.Linq.Async { using System; + using System.Threading; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using JetBrains.Annotations; /// Wraps an async sequence of items into another async sequence of items /// Type of elements of the inner async sequence /// Type of elements of the outer async sequence - internal sealed class FdbAsyncSequence : IFdbAsyncEnumerable + internal sealed class AsyncSequence : IAsyncEnumerable { - public readonly IFdbAsyncEnumerable Source; - public readonly Func, IFdbAsyncEnumerator> Factory; + public readonly IAsyncEnumerable Source; + public readonly Func, IAsyncEnumerator> Factory; - public FdbAsyncSequence([NotNull] IFdbAsyncEnumerable source, [NotNull] Func, IFdbAsyncEnumerator> factory) + public AsyncSequence([NotNull] IAsyncEnumerable source, [NotNull] Func, IAsyncEnumerator> factory) { Contract.Requires(source != null && factory != null); this.Source = source; this.Factory = factory; } - public IAsyncEnumerator GetEnumerator() + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { - return this.GetEnumerator(FdbAsyncMode.Default); - } - - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) - { - IFdbAsyncEnumerator inner = null; + ct.ThrowIfCancellationRequested(); + IAsyncEnumerator inner = null; try { - inner = this.Source.GetEnumerator(mode); + inner = this.Source.GetEnumerator(ct, mode); Contract.Requires(inner != null, "The underlying async sequence returned an empty enumerator"); var outer = this.Factory(inner); @@ -69,7 +65,7 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) catch (Exception) { //make sure that the inner iterator gets disposed if something went wrong - if (inner != null) inner.Dispose(); + inner?.Dispose(); throw; } } diff --git a/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs b/FoundationDB.Client/Linq/Async/Expressions/AsyncFilterExpression.cs similarity index 90% rename from FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs rename to FoundationDB.Client/Linq/Async/Expressions/AsyncFilterExpression.cs index 61f6089cb..0e9e67de7 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncFilterExpression.cs +++ b/FoundationDB.Client/Linq/Async/Expressions/AsyncFilterExpression.cs @@ -26,30 +26,32 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq + +namespace Doxense.Linq.Async.Expressions { - using FoundationDB.Async; using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; + using Doxense.Diagnostics.Contracts; /// Expression that evalute a condition on each item /// Type of the filtered elements - internal sealed class AsyncFilterExpression + public sealed class AsyncFilterExpression { private readonly Func m_filter; private readonly Func> m_asyncFilter; public AsyncFilterExpression(Func filter) { - if (filter == null) throw new ArgumentNullException("filter"); + Contract.NotNull(filter, nameof(filter)); m_filter = filter; } public AsyncFilterExpression(Func> asyncFilter) { - if (asyncFilter == null) throw new ArgumentNullException("asyncFilter"); + Contract.NotNull(asyncFilter, nameof(asyncFilter)); m_asyncFilter = asyncFilter; } @@ -94,8 +96,8 @@ public AsyncFilterExpression OrElse([NotNull] AsyncFilterExpression AndAlso([NotNull] AsyncFilterExpression left, [NotNull] AsyncFilterExpression right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); // combine two expressions into a logical AND expression. // Note: if the first expression returns false, the second one will NOT be evaluated @@ -111,7 +113,7 @@ public static AsyncFilterExpression AndAlso([NotNull] AsyncFilterExpres else { // next one is async var g = right.m_asyncFilter; - return new AsyncFilterExpression((x, ct) => f(x) ? g(x, ct) : TaskHelpers.FalseTask); + return new AsyncFilterExpression((x, ct) => f(x) ? g(x, ct) : TaskHelpers.False); } } else @@ -133,8 +135,8 @@ public static AsyncFilterExpression AndAlso([NotNull] AsyncFilterExpres [NotNull] public static AsyncFilterExpression OrElse([NotNull] AsyncFilterExpression left, [NotNull] AsyncFilterExpression right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); // combine two expressions into a logical OR expression. // Note: if the first expression returns true, the second one will NOT be evaluated @@ -150,7 +152,7 @@ public static AsyncFilterExpression OrElse([NotNull] AsyncFilterExpress else { // next one is async var g = right.m_asyncFilter; - return new AsyncFilterExpression((x, ct) => f(x) ? TaskHelpers.TrueTask : g(x, ct)); + return new AsyncFilterExpression((x, ct) => f(x) ? TaskHelpers.True : g(x, ct)); } } else @@ -159,11 +161,13 @@ public static AsyncFilterExpression OrElse([NotNull] AsyncFilterExpress if (right.m_asyncFilter != null) { // so is the next one var g = left.m_asyncFilter; + Contract.Assert(g != null); return new AsyncFilterExpression(async (x, ct) => (await f(x, ct).ConfigureAwait(false)) || (await g(x, ct).ConfigureAwait(false))); } else { var g = left.m_filter; + Contract.Assert(g != null); return new AsyncFilterExpression(async (x, ct) => (await f(x, ct).ConfigureAwait(false)) || g(x)); } } diff --git a/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs b/FoundationDB.Client/Linq/Async/Expressions/AsyncObserverExpression.cs similarity index 91% rename from FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs rename to FoundationDB.Client/Linq/Async/Expressions/AsyncObserverExpression.cs index 8920e87b5..4168fc1b6 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncObserverExpression.cs +++ b/FoundationDB.Client/Linq/Async/Expressions/AsyncObserverExpression.cs @@ -26,29 +26,30 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Expressions { using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; /// Expression that execute an action on each item, but does not change the source expression in anyway /// Type of observed items - internal sealed class AsyncObserverExpression + public sealed class AsyncObserverExpression { private readonly Action m_handler; private readonly Func m_asyncHandler; public AsyncObserverExpression(Action handler) { - if (handler == null) throw new ArgumentNullException("handler"); + Contract.NotNull(handler, nameof(handler)); m_handler = handler; } public AsyncObserverExpression(Func asyncHandler) { - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); + Contract.NotNull(asyncHandler, nameof(asyncHandler)); m_asyncHandler = asyncHandler; } @@ -93,8 +94,8 @@ public AsyncObserverExpression Then([NotNull] AsyncObserverExpression Then([NotNull] AsyncObserverExpression left, [NotNull] AsyncObserverExpression right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); if (left.m_handler != null) { diff --git a/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs b/FoundationDB.Client/Linq/Async/Expressions/AsyncTransformExpression.cs similarity index 91% rename from FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs rename to FoundationDB.Client/Linq/Async/Expressions/AsyncTransformExpression.cs index c0207c4e9..694d43053 100644 --- a/FoundationDB.Client/Linq/Expressions/AsyncTransformExpression.cs +++ b/FoundationDB.Client/Linq/Async/Expressions/AsyncTransformExpression.cs @@ -26,43 +26,41 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Expressions { - using FoundationDB.Async; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Expression that applies a transformation on each item /// Type of the source items /// Type of the transformed items - internal sealed class AsyncTransformExpression + public sealed class AsyncTransformExpression { private readonly Func m_transform; private readonly Func> m_asyncTransform; public AsyncTransformExpression(Func transform) { - if (transform == null) throw new ArgumentNullException("transform"); + Contract.NotNull(transform, nameof(transform)); m_transform = transform; } public AsyncTransformExpression(Func> asyncTransform) { - if (asyncTransform == null) throw new ArgumentNullException("asyncTransform"); + Contract.NotNull(asyncTransform, nameof(asyncTransform)); m_asyncTransform = asyncTransform; } - public bool Async - { - get { return m_asyncTransform != null; } - } + public bool Async => m_asyncTransform != null; public bool IsIdentity() { //note: Identity Function is not async, and is only possible if TSource == TResult, so we can skip checking the types ourselves... - return m_transform != null && object.ReferenceEquals(m_transform, TaskHelpers.Cache.Identity); + return m_transform != null && object.ReferenceEquals(m_transform, TaskHelpers.CachedTasks.Identity); } public TResult Invoke(TSource item) @@ -121,8 +119,8 @@ public AsyncTransformExpression Then([NotNull] AsyncTra [NotNull] public static AsyncTransformExpression Then([NotNull] AsyncTransformExpression left, [NotNull] AsyncTransformExpression right) { - if (left == null) throw new ArgumentNullException("left"); - if (right == null) throw new ArgumentNullException("right"); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); if (left.IsIdentity()) { // we can optimize the left expression away, since we know that TSource == TResult ! diff --git a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs b/FoundationDB.Client/Linq/Async/Iterators/AnonymousAsyncGenerator.cs similarity index 68% rename from FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs rename to FoundationDB.Client/Linq/Async/Iterators/AnonymousAsyncGenerator.cs index 752f5c5d7..89153e9a2 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAnonymousAsyncGenerator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/AnonymousAsyncGenerator.cs @@ -26,17 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +using Doxense.Async; + +namespace Doxense.Linq.Async.Iterators { using System; using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; /// Generate items asynchronously, using a user-provided lambda /// Type of the items produced by this generator - internal class FdbAnonymousAsyncGenerator : FdbAsyncIterator + public class AnonymousAsyncGenerator : AsyncIterator { // use a custom lambda that returns Maybe results, asynchronously // => as long as the result has a value, continue iterating @@ -45,39 +46,58 @@ internal class FdbAnonymousAsyncGenerator : FdbAsyncIterator // ITERABLE - private readonly Func>> m_generator; + private readonly Delegate m_generator; + // can be either one of: + // - Func>> + // - Func>> // ITERATOR private long m_index; - public FdbAnonymousAsyncGenerator(Func>> generator) + public AnonymousAsyncGenerator(Func>> generator) + : this((Delegate) generator) + { } + + private AnonymousAsyncGenerator(Delegate generator) { Contract.Requires(generator != null); m_generator = generator; m_index = -1; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbAnonymousAsyncGenerator(m_generator); + return new AnonymousAsyncGenerator(m_generator); } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { m_index = 0; - return TaskHelpers.TrueTask; + return TaskHelpers.True; } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { - ct.ThrowIfCancellationRequested(); + m_ct.ThrowIfCancellationRequested(); if (m_index < 0) return false; long index = m_index; - var res = await m_generator(index, ct); + Maybe res; + if (m_generator is Func>> genT) + { + res = await genT(index, m_ct); + } + else if (m_generator is Func>> genV) + { + res = await genV(index, m_ct); + } + else + { + throw new InvalidOperationException(); + } - if (res.HasFailed) res.ThrowForNonSuccess(); + if (res.Failed) res.ThrowForNonSuccess(); if (res.IsEmpty) return Completed(); m_index = checked(index + 1); return Publish(res.Value); diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/AsyncFilterIterator.cs similarity index 77% rename from FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/AsyncFilterIterator.cs index f84155d62..356c46be1 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncFilterIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/AsyncFilterIterator.cs @@ -26,39 +26,40 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using JetBrains.Annotations; - internal abstract class FdbAsyncFilterIterator : FdbAsyncIterator + public abstract class AsyncFilterIterator : AsyncIterator { /// Source sequence (when in iterable mode) - protected IFdbAsyncEnumerable m_source; + protected IAsyncEnumerable m_source; /// Active iterator on the source (when in iterator mode) - protected IFdbAsyncEnumerator m_iterator; + protected IAsyncEnumerator m_iterator; protected bool m_innerHasCompleted; - protected FdbAsyncFilterIterator([NotNull] IFdbAsyncEnumerable source) + protected AsyncFilterIterator([NotNull] IAsyncEnumerable source) { Contract.Requires(source != null); m_source = source; } /// Start the inner iterator - protected virtual IFdbAsyncEnumerator StartInner() + protected virtual IAsyncEnumerator StartInner(CancellationToken ct) { + ct.ThrowIfCancellationRequested(); // filtering changes the number of items, so that means that, even if the underlying caller wants one item, we may need to read more. // => change all "Head" requests into "Iterator" to prevent any wrong optimizations by the underlying source (ex: using a too small batch size) var mode = m_mode; - if (mode == FdbAsyncMode.Head) mode = FdbAsyncMode.Iterator; + if (mode == AsyncIterationHint.Head) mode = AsyncIterationHint.Iterator; - return m_source.GetEnumerator(mode); + return m_source.GetEnumerator(m_ct, mode); } protected void MarkInnerAsCompleted() @@ -66,24 +67,20 @@ protected void MarkInnerAsCompleted() m_innerHasCompleted = true; // we don't need the inerator, so we can dispose of it immediately - var iterator = Interlocked.Exchange(ref m_iterator, null); - if (iterator != null) - { - iterator.Dispose(); - } + Interlocked.Exchange(ref m_iterator, null)?.Dispose(); } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { // on the first call to MoveNext, we have to hook up with the source iterator - IFdbAsyncEnumerator iterator = null; + IAsyncEnumerator iterator = null; try { - iterator = StartInner(); - if (iterator == null) return TaskHelpers.FalseTask; + iterator = StartInner(m_ct); + if (iterator == null) return TaskHelpers.False; OnStarted(iterator); - return TaskHelpers.TrueTask; + return TaskHelpers.True; } catch (Exception) { @@ -101,7 +98,7 @@ protected override Task OnFirstAsync(CancellationToken ct) } } - protected virtual void OnStarted(IFdbAsyncEnumerator iterator) + protected virtual void OnStarted(IAsyncEnumerator iterator) { //override this to add custom starting logic once we know that the inner iterator is ready } diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/AsyncIterator.cs similarity index 53% rename from FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/AsyncIterator.cs index e0ba6344a..0cf8545fe 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/AsyncIterator.cs @@ -26,21 +26,22 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { - using FoundationDB.Async; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Expressions; + using JetBrains.Annotations; /// Base class for all async iterators /// Type of elements of the outer async sequence - internal abstract class FdbAsyncIterator : IFdbAsyncEnumerable, IFdbAsyncEnumerator + public abstract class AsyncIterator : IAsyncEnumerable, IAsyncEnumerator { - //REVIEW: we could need an IFdbAsyncIterator interface that holds all the Select(),Where(),Take(),... so that it can be used by FdbAsyncEnumerable to either call them directly (if the query supports it) or use a generic implementation - // => this would be implemented by FdbAsyncIterator as well as FdbRangeQuery (and ony other 'self optimizing' class) + //REVIEW: we could need an IAsyncIterator interface that holds all the Select(),Where(),Take(),... so that it can be used by AsyncEnumerable to either call them directly (if the query supports it) or use a generic implementation + // => this would be implemented by AsyncIterator as well as FdbRangeQuery (and ony other 'self optimizing' class) private const int STATE_SEQ = 0; private const int STATE_INIT = 1; @@ -50,35 +51,35 @@ internal abstract class FdbAsyncIterator : IFdbAsyncEnumerable protected TResult m_current; protected int m_state; - protected FdbAsyncMode m_mode; + protected AsyncIterationHint m_mode; + protected CancellationToken m_ct; - #region IFdbAsyncEnumerable... + #region IAsyncEnumerable... - public IAsyncEnumerator GetEnumerator() + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { - return this.GetEnumerator(FdbAsyncMode.Default); - } + ct.ThrowIfCancellationRequested(); - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) - { // reuse the same instance the first time if (Interlocked.CompareExchange(ref m_state, STATE_INIT, STATE_SEQ) == STATE_SEQ) { m_mode = mode; + m_ct = ct; return this; } // create a new one var iter = Clone(); iter.m_mode = mode; + iter.m_ct = ct; Volatile.Write(ref iter.m_state, STATE_INIT); return iter; } - protected abstract FdbAsyncIterator Clone(); + protected abstract AsyncIterator Clone(); #endregion - #region IFdbAsyncEnumerator... + #region IAsyncEnumerator... public TResult Current { @@ -89,7 +90,7 @@ public TResult Current } } - public async Task MoveNextAsync(CancellationToken ct) + public async Task MoveNextAsync() { var state = Volatile.Read(ref m_state); @@ -103,16 +104,16 @@ public async Task MoveNextAsync(CancellationToken ct) return false; } - if (ct.IsCancellationRequested) + if (m_ct.IsCancellationRequested) { - return Canceled(ct); + return Canceled(); } try { if (state == STATE_INIT) { - if (!await OnFirstAsync(ct).ConfigureAwait(false)) + if (!await OnFirstAsync().ConfigureAwait(false)) { // did not start at all ? return Completed(); } @@ -123,7 +124,7 @@ public async Task MoveNextAsync(CancellationToken ct) } } - return await OnNextAsync(ct).ConfigureAwait(false); + return await OnNextAsync().ConfigureAwait(false); } catch (Exception) { @@ -137,109 +138,109 @@ public async Task MoveNextAsync(CancellationToken ct) #region LINQ... [NotNull] - public virtual FdbAsyncIterator Where([NotNull] Func predicate) + public virtual AsyncIterator Where([NotNull] Func predicate) { - if (predicate == null) throw new ArgumentNullException("predicate"); + Contract.NotNull(predicate, nameof(predicate)); - return FdbAsyncEnumerable.Filter(this, new AsyncFilterExpression(predicate)); + return AsyncEnumerable.Filter(this, new AsyncFilterExpression(predicate)); } [NotNull] - public virtual FdbAsyncIterator Where([NotNull] Func> asyncPredicate) + public virtual AsyncIterator Where([NotNull] Func> asyncPredicate) { - if (asyncPredicate == null) throw new ArgumentNullException("asyncPredicate"); + Contract.NotNull(asyncPredicate, nameof(asyncPredicate)); - return FdbAsyncEnumerable.Filter(this, new AsyncFilterExpression(asyncPredicate)); + return AsyncEnumerable.Filter(this, new AsyncFilterExpression(asyncPredicate)); } [NotNull] - public virtual FdbAsyncIterator Select([NotNull] Func selector) + public virtual AsyncIterator Select([NotNull] Func selector) { - if (selector == null) throw new ArgumentNullException("selector"); + Contract.NotNull(selector, nameof(selector)); - return FdbAsyncEnumerable.Map(this, new AsyncTransformExpression(selector)); + return AsyncEnumerable.Map(this, new AsyncTransformExpression(selector)); } [NotNull] - public virtual FdbAsyncIterator Select([NotNull] Func> asyncSelector) + public virtual AsyncIterator Select([NotNull] Func> asyncSelector) { - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); - return FdbAsyncEnumerable.Map(this, new AsyncTransformExpression(asyncSelector)); + return AsyncEnumerable.Map(this, new AsyncTransformExpression(asyncSelector)); } [NotNull] - public virtual FdbAsyncIterator SelectMany([NotNull] Func> selector) + public virtual AsyncIterator SelectMany([NotNull] Func> selector) { - if (selector == null) throw new ArgumentNullException("selector"); + Contract.NotNull(selector, nameof(selector)); - return FdbAsyncEnumerable.Flatten(this, new AsyncTransformExpression>(selector)); + return AsyncEnumerable.Flatten(this, new AsyncTransformExpression>(selector)); } [NotNull] - public virtual FdbAsyncIterator SelectMany([NotNull] Func>> asyncSelector) + public virtual AsyncIterator SelectMany([NotNull] Func>> asyncSelector) { - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); - return FdbAsyncEnumerable.Flatten(this, new AsyncTransformExpression>(asyncSelector)); + return AsyncEnumerable.Flatten(this, new AsyncTransformExpression>(asyncSelector)); } [NotNull] - public virtual FdbAsyncIterator SelectMany([NotNull] Func> collectionSelector, [NotNull] Func resultSelector) + public virtual AsyncIterator SelectMany([NotNull] Func> collectionSelector, [NotNull] Func resultSelector) { - if (collectionSelector == null) throw new ArgumentNullException("collectionSelector"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + Contract.NotNull(collectionSelector, nameof(collectionSelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); - return FdbAsyncEnumerable.Flatten(this, new AsyncTransformExpression>(collectionSelector), resultSelector); + return AsyncEnumerable.Flatten(this, new AsyncTransformExpression>(collectionSelector), resultSelector); } [NotNull] - public virtual FdbAsyncIterator SelectMany([NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) + public virtual AsyncIterator SelectMany([NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) { - if (asyncCollectionSelector == null) throw new ArgumentNullException("asyncCollectionSelector"); - if (resultSelector == null) throw new ArgumentNullException("resultSelector"); + Contract.NotNull(asyncCollectionSelector, nameof(asyncCollectionSelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); - return FdbAsyncEnumerable.Flatten(this, new AsyncTransformExpression>(asyncCollectionSelector), resultSelector); + return AsyncEnumerable.Flatten(this, new AsyncTransformExpression>(asyncCollectionSelector), resultSelector); } [NotNull] - public virtual FdbAsyncIterator Take(int count) + public virtual AsyncIterator Take(int count) { - return FdbAsyncEnumerable.Limit(this, count); + return AsyncEnumerable.Limit(this, count); } [NotNull] - public virtual FdbAsyncIterator TakeWhile([NotNull] Func condition) + public virtual AsyncIterator TakeWhile([NotNull] Func condition) { - return FdbAsyncEnumerable.Limit(this, condition); + return AsyncEnumerable.Limit(this, condition); } [NotNull] - public virtual FdbAsyncIterator Skip(int count) + public virtual AsyncIterator Skip(int count) { - return FdbAsyncEnumerable.Offset(this, count); + return AsyncEnumerable.Offset(this, count); } /// Execute an action on the result of this async sequence [NotNull] public virtual Task ExecuteAsync([NotNull] Action action, CancellationToken ct) { - return FdbAsyncEnumerable.Run(this, FdbAsyncMode.All, action, ct); + return AsyncEnumerable.Run(this, AsyncIterationHint.All, action, ct); } [NotNull] public virtual Task ExecuteAsync([NotNull] Func asyncAction, CancellationToken ct) { - return FdbAsyncEnumerable.Run(this, FdbAsyncMode.All, asyncAction, ct); + return AsyncEnumerable.Run(this, AsyncIterationHint.All, asyncAction, ct); } #endregion #region Iterator Impl... - protected abstract Task OnFirstAsync(CancellationToken ct); + protected abstract Task OnFirstAsync(); - protected abstract Task OnNextAsync(CancellationToken ct); + protected abstract Task OnNextAsync(); protected bool Publish(TResult current) { @@ -259,7 +260,7 @@ protected bool Completed() } else if (Interlocked.CompareExchange(ref m_state, STATE_COMPLETED, STATE_ITERATING) == STATE_ITERATING) { // the iterator has done at least something, so we can clean it up - this.Cleanup(); + Cleanup(); } return false; } @@ -268,15 +269,15 @@ protected bool Completed() protected void MarkAsFailed() { //TODO: store the state "failed" somewhere? - this.Dispose(); + Dispose(); } - protected bool Canceled(CancellationToken ct) + protected bool Canceled() { //TODO: store the state "canceled" somewhere? - this.Dispose(); - ct.ThrowIfCancellationRequested(); - return false; + Dispose(); + m_ct.ThrowIfCancellationRequested(); // should throw here! + return false; //note: should not be reached } protected void ThrowInvalidState() diff --git a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs b/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs similarity index 93% rename from FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs rename to FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs index 2cdae90c0..6d040ba0e 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbAsyncIteratorPump.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs @@ -28,20 +28,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Diagnostics; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; /// Pump that repeatedly calls MoveNext on an iterator and tries to publish the values in a Producer/Consumer queue /// [DebuggerDisplay("State={m_state}")] - internal sealed class FdbAsyncIteratorPump + public sealed class AsyncIteratorPump { private const int STATE_IDLE = 0; private const int STATE_WAITING_FOR_NEXT = 1; @@ -50,11 +50,11 @@ internal sealed class FdbAsyncIteratorPump private const int STATE_DONE = 4; private volatile int m_state; - private readonly IFdbAsyncEnumerator m_iterator; + private readonly IAsyncEnumerator m_iterator; private readonly IAsyncTarget m_target; - public FdbAsyncIteratorPump( - IFdbAsyncEnumerator iterator, + public AsyncIteratorPump( + IAsyncEnumerator iterator, IAsyncTarget target ) { @@ -96,12 +96,12 @@ public async Task PumpAsync(CancellationToken ct) } try - { + { while (!ct.IsCancellationRequested) { LogDebug("waiting for next"); m_state = STATE_WAITING_FOR_NEXT; - if (!(await m_iterator.MoveNextAsync(ct).ConfigureAwait(false))) + if (!(await m_iterator.MoveNextAsync().ConfigureAwait(false))) { LogDebug("completed"); m_state = STATE_DONE; diff --git a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs similarity index 82% rename from FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs index c490eb2ad..fb4d54475 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbBatchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs @@ -26,17 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; /// Packs items from an inner sequence, into a sequence of fixed-size arrays. /// Type the the items from the source sequence - internal class FdbBatchingAsyncIterator : FdbAsyncFilterIterator + public class BatchingAsyncIterator : AsyncFilterIterator { // Typical use cas: to merge incoming streams of items into a sequence of arrays. This is basically the inverse of the SelectMany() operator. // This iterator should mostly be used on sequence that have either no latency (reading from an in-memory buffer) or where the latency is the same for each items. @@ -54,7 +53,7 @@ internal class FdbBatchingAsyncIterator : FdbAsyncFilterIteratorCreate a new batching iterator /// Source sequence of items that must be batched by waves /// Maximum size of a batch to return down the line - public FdbBatchingAsyncIterator(IFdbAsyncEnumerable source, int batchSize) + public BatchingAsyncIterator(IAsyncEnumerable source, int batchSize) : base(source) { Contract.Requires(batchSize > 0); @@ -63,18 +62,18 @@ public FdbBatchingAsyncIterator(IFdbAsyncEnumerable source, int batchSiz m_batchSize = batchSize; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbBatchingAsyncIterator(m_source, m_batchSize); + return new BatchingAsyncIterator(m_source, m_batchSize); } - protected override void OnStarted(IFdbAsyncEnumerator iterator) + protected override void OnStarted(IAsyncEnumerator iterator) { // pre-allocate the inner buffer, if it is not too big m_buffer = new List(Math.Min(m_batchSize, 1024)); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { // read items from the source until the buffer is full, or the source has completed @@ -86,16 +85,16 @@ protected override async Task OnNextAsync(CancellationToken ct) var iterator = m_iterator; var buffer = m_buffer; - bool hasMore = await iterator.MoveNextAsync(ct).ConfigureAwait(false); + bool hasMore = await iterator.MoveNextAsync().ConfigureAwait(false); - while(hasMore && !ct.IsCancellationRequested) + while(hasMore && !m_ct.IsCancellationRequested) { buffer.Add(iterator.Current); if (buffer.Count >= m_batchSize) break; - hasMore = await iterator.MoveNextAsync(ct).ConfigureAwait(false); + hasMore = await iterator.MoveNextAsync().ConfigureAwait(false); } - ct.ThrowIfCancellationRequested(); + m_ct.ThrowIfCancellationRequested(); if (!hasMore) { diff --git a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/DistinctAsyncIterator.cs similarity index 75% rename from FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/DistinctAsyncIterator.cs index fb19d9ec4..4d932cfe5 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbDistinctAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/DistinctAsyncIterator.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; @@ -37,13 +37,13 @@ namespace FoundationDB.Linq /// Filters duplicate items from an async sequence /// Type of elements of the async sequence - internal sealed class FdbDistinctAsyncIterator : FdbAsyncFilterIterator + public sealed class DistinctAsyncIterator : AsyncFilterIterator { private readonly IEqualityComparer m_comparer; private HashSet m_set; - public FdbDistinctAsyncIterator([NotNull] IFdbAsyncEnumerable source, IEqualityComparer comparer) + public DistinctAsyncIterator([NotNull] IAsyncEnumerable source, IEqualityComparer comparer) : base(source) { Contract.Requires(comparer != null); @@ -51,30 +51,30 @@ public FdbDistinctAsyncIterator([NotNull] IFdbAsyncEnumerable source, I m_comparer = comparer; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbDistinctAsyncIterator(m_source, m_comparer); + return new DistinctAsyncIterator(m_source, m_comparer); } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { // we start with an empty set... m_set = new HashSet(m_comparer); - return base.OnFirstAsync(ct); + return base.OnFirstAsync(); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // completed m_set = null; return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_set.Add(current)) @@ -86,23 +86,23 @@ protected override async Task OnNextAsync(CancellationToken ct) } m_set = null; - return Canceled(ct); + return Canceled(); } public override async Task ExecuteAsync(Action handler, CancellationToken ct) { - if (handler == null) throw new ArgumentNullException("handler"); + Contract.NotNull(handler, nameof(handler)); if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); var mode = m_mode; - if (mode == FdbAsyncMode.Head) mode = FdbAsyncMode.Iterator; + if (mode == AsyncIterationHint.Head) mode = AsyncIterationHint.Iterator; - using (var iter = m_source.GetEnumerator(mode)) + using (var iter = m_source.GetEnumerator(ct, mode)) { var set = new HashSet(m_comparer); - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (set.Add(current)) @@ -118,18 +118,18 @@ public override async Task ExecuteAsync(Action handler, CancellationTok public override async Task ExecuteAsync(Func asyncHandler, CancellationToken ct) { - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); + Contract.NotNull(asyncHandler, nameof(asyncHandler)); if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); var mode = m_mode; - if (mode == FdbAsyncMode.Head) mode = FdbAsyncMode.Iterator; + if (mode == AsyncIterationHint.Head) mode = AsyncIterationHint.Iterator; - using (var iter = m_source.GetEnumerator(mode)) + using (var iter = m_source.GetEnumerator(ct, mode)) { var set = new HashSet(m_comparer); - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (set.Add(current)) diff --git a/FoundationDB.Client/FdbExceptIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs similarity index 78% rename from FoundationDB.Client/FdbExceptIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs index 53e916f5b..274df25c2 100644 --- a/FoundationDB.Client/FdbExceptIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs @@ -26,29 +26,27 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Linq.Async.Iterators { - using FoundationDB.Linq; using System; using System.Collections.Generic; - using System.Threading; /// Returns only the values for the keys that are in the first sub query, but not in the others /// Type of the elements from the source async sequences /// Type of the keys extracted from the source elements /// Type of the elements of resulting async sequence - internal sealed class FdbExceptIterator : FdbQueryMergeIterator + public sealed class ExceptAsyncIterator : MergeAsyncIterator { - public FdbExceptIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) + public ExceptAsyncIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) : base(sources, limit, keySelector, resultSelector, comparer) { } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbExceptIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); + return new ExceptAsyncIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken ct, out int index, out TSource current) + protected override bool FindNext(out int index, out TSource current) { index = -1; current = default(TSource); @@ -83,7 +81,7 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc { output = false; if (cmp == 0) discard = true; - AdvanceIterator(i, ct); + AdvanceIterator(i); } } @@ -95,16 +93,16 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc if (output || discard) { - AdvanceIterator(0, ct); + AdvanceIterator(0); } return true; } /// Apply a transformation on the results of the intersection - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - return new FdbExceptIterator( + return new ExceptAsyncIterator( m_sources, m_limit, m_keySelector, @@ -116,13 +114,13 @@ public override FdbAsyncIterator Select(Func selector /// Limit the number of elements returned by the intersection /// Maximum number of results to return /// New Intersect that will only return the specified number of results - public override FdbAsyncIterator Take(int limit) + public override AsyncIterator Take(int limit) { - if (limit < 0) throw new ArgumentOutOfRangeException("limit", "Value cannot be less than zero"); + if (limit < 0) throw new ArgumentOutOfRangeException(nameof(limit), "Value cannot be less than zero"); if (m_limit != null && m_limit < limit) return this; - return new FdbExceptIterator( + return new ExceptAsyncIterator( m_sources, limit, m_keySelector, diff --git a/FoundationDB.Client/FdbIntersectIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs similarity index 77% rename from FoundationDB.Client/FdbIntersectIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs index 85c260bbc..c558eef4d 100644 --- a/FoundationDB.Client/FdbIntersectIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs @@ -26,29 +26,27 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Linq.Async.Iterators { - using FoundationDB.Linq; using System; using System.Collections.Generic; - using System.Threading; /// Returns only the values for the keys that are in all the sub queries /// Type of the elements from the source async sequences /// Type of the keys extracted from the source elements /// Type of the elements of resulting async sequence - internal sealed class FdbIntersectIterator : FdbQueryMergeIterator + public sealed class IntersectAsyncIterator : MergeAsyncIterator { - public FdbIntersectIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) + public IntersectAsyncIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) : base(sources, limit, keySelector, resultSelector, comparer) { } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbIntersectIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); + return new IntersectAsyncIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken ct, out int index, out TSource current) + protected override bool FindNext(out int index, out TSource current) { index = -1; current = default(TSource); @@ -92,7 +90,7 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc // advance everyone ! for (int i = 0; i < m_iterators.Length;i++) { - if (m_iterators[i].Active) AdvanceIterator(i, ct); + if (m_iterators[i].Active) AdvanceIterator(i); } return true; } @@ -102,7 +100,7 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc { if (m_iterators[i].Active && m_keyComparer.Compare(m_iterators[i].Current, max) < 0) { - AdvanceIterator(i, ct); + AdvanceIterator(i); } } @@ -112,9 +110,9 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc } /// Apply a transformation on the results of the intersection - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - return new FdbIntersectIterator( + return new IntersectAsyncIterator( m_sources, m_limit, m_keySelector, @@ -126,13 +124,13 @@ public override FdbAsyncIterator Select(Func selector /// Limit the number of elements returned by the intersection /// Maximum number of results to return /// New Intersect that will only return the specified number of results - public override FdbAsyncIterator Take(int limit) + public override AsyncIterator Take(int limit) { - if (limit < 0) throw new ArgumentOutOfRangeException("limit", "Value cannot be less than zero"); + if (limit < 0) throw new ArgumentOutOfRangeException(nameof(limit), "Value cannot be less than zero"); if (m_limit != null && m_limit < limit) return this; - return new FdbIntersectIterator( + return new IntersectAsyncIterator( m_sources, limit, m_keySelector, diff --git a/FoundationDB.Client/FdbQueryMergeIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs similarity index 81% rename from FoundationDB.Client/FdbQueryMergeIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs index 661b232c7..d35c9d27c 100644 --- a/FoundationDB.Client/FdbQueryMergeIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs @@ -26,31 +26,29 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; using System.Linq; - using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Linq; /// Performs a Merge Sort on several concurrent range queries /// Type of the elements in the source queries /// Type of values extracted from the keys, that will be used for sorting /// Type of results returned - internal abstract class FdbQueryMergeIterator : FdbAsyncIterator + public abstract class MergeAsyncIterator : AsyncIterator { // Takes several range queries that return **SORTED** lists of items // - Make all querie's iterators run concurrently // - At each step, finds the "smallest" value from all remaining iterators, transform it into a TResult and expose it as the current element - // - Extract a TKey value from the keys and compare them with the provided comparer + // - Extract a TKey value from the keys and compare them with the provided comparer // The order of the extracted keys MUST be the same as the order of the binary keys ! This algorithm will NOT work if extracted keys are not in the same order as there binary representation ! - protected IEnumerable> m_sources; + protected IEnumerable> m_sources; protected Func m_keySelector; protected IComparer m_keyComparer; protected Func m_resultSelector; @@ -62,13 +60,13 @@ internal abstract class FdbQueryMergeIterator : FdbAsync protected struct IteratorState { public bool Active; - public IFdbAsyncEnumerator Iterator; + public IAsyncEnumerator Iterator; public Task Next; public bool HasCurrent; public TKey Current; } - protected FdbQueryMergeIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) + protected MergeAsyncIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) { Contract.Requires(sources != null && (limit == null || limit >= 0) && keySelector != null && resultSelector != null); m_sources = sources; @@ -78,16 +76,16 @@ protected FdbQueryMergeIterator(IEnumerable> source m_resultSelector = resultSelector; } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { if (m_remaining != null && m_remaining.Value < 0) - { // empty list ?? + { // empty list ?? return TaskHelpers.FromResult(Completed()); } // even if the caller only wants the first, we will probably need to read more than that... var mode = m_mode; - if (mode == FdbAsyncMode.Head) mode = FdbAsyncMode.Iterator; + if (mode == AsyncIterationHint.Head) mode = AsyncIterationHint.Iterator; var sources = m_sources.ToArray(); var iterators = new IteratorState[sources.Length]; @@ -96,10 +94,12 @@ protected override Task OnFirstAsync(CancellationToken ct) // start all the iterators for (int i = 0; i < sources.Length;i++) { - var state = new IteratorState(); - state.Active = true; - state.Iterator = sources[i].GetEnumerator(mode); - state.Next = state.Iterator.MoveNextAsync(ct); + var state = new IteratorState + { + Active = true, + Iterator = sources[i].GetEnumerator(m_ct, mode) + }; + state.Next = state.Iterator.MoveNextAsync(); iterators[i] = state; } @@ -112,8 +112,8 @@ protected override Task OnFirstAsync(CancellationToken ct) // dispose already opened iterators var tmp = iterators; iterators = null; - try { Cleanup(tmp); } catch (Exception) { } - return TaskHelpers.FromException(e); + try { Cleanup(tmp); } catch { } + return Task.FromException(e); } finally { @@ -122,7 +122,7 @@ protected override Task OnFirstAsync(CancellationToken ct) } /// Finds the next smallest item from all the active iterators - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { if (m_remaining != null && m_remaining.Value <= 0) { @@ -155,7 +155,7 @@ protected override async Task OnNextAsync(CancellationToken ct) } // find the next value to advance - if (!FindNext(ct, out index, out current)) + if (!FindNext(out index, out current)) { // nothing left anymore ? return Completed(); } @@ -171,21 +171,18 @@ protected override async Task OnNextAsync(CancellationToken ct) } // advance the current iterator - if (m_remaining != null) - { - m_remaining = m_remaining.Value - 1; - } + m_remaining = m_remaining - 1; return true; } - protected abstract bool FindNext(CancellationToken ct, out int index, out TSource current); + protected abstract bool FindNext(out int index, out TSource current); - protected void AdvanceIterator(int index, CancellationToken ct) + protected void AdvanceIterator(int index) { m_iterators[index].HasCurrent = false; m_iterators[index].Current = default(TKey); - m_iterators[index].Next = m_iterators[index].Iterator.MoveNextAsync(ct); + m_iterators[index].Next = m_iterators[index].Iterator.MoveNextAsync(); } private static void Cleanup(IteratorState[] iterators) diff --git a/FoundationDB.Client/FdbMergeSortIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs similarity index 75% rename from FoundationDB.Client/FdbMergeSortIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs index 05ca95465..264cd6154 100644 --- a/FoundationDB.Client/FdbMergeSortIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs @@ -26,30 +26,28 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Linq.Async.Iterators { - using FoundationDB.Linq; using System; using System.Collections.Generic; - using System.Threading; /// Merge all the elements of several ordered queries into one single async sequence /// Type of the elements from the source async sequences /// Type of the keys extracted from the source elements /// Type of the elements of resulting async sequence - internal sealed class FdbMergeSortIterator : FdbQueryMergeIterator + public sealed class MergeSortAsyncIterator : MergeAsyncIterator { - public FdbMergeSortIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) + public MergeSortAsyncIterator(IEnumerable> sources, int? limit, Func keySelector, Func resultSelector, IComparer comparer) : base(sources, limit, keySelector, resultSelector, comparer) { } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbMergeSortIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); + return new MergeSortAsyncIterator(m_sources, m_limit, m_keySelector, m_resultSelector, m_keyComparer); } - protected override bool FindNext(CancellationToken ct, out int index, out TSource current) + protected override bool FindNext(out int index, out TSource current) { index = -1; current = default(TSource); @@ -71,7 +69,7 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc current = m_iterators[index].Iterator.Current; if (m_remaining == null || m_remaining.Value > 1) { // start getting the next value on this iterator - AdvanceIterator(index, ct); + AdvanceIterator(index); } } @@ -80,9 +78,9 @@ protected override bool FindNext(CancellationToken ct, out int index, out TSourc /// Apply a transformation on the results of the merge sort - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - return new FdbMergeSortIterator( + return new MergeSortAsyncIterator( m_sources, m_limit, m_keySelector, @@ -94,13 +92,13 @@ public override FdbAsyncIterator Select(Func selector /// Limit the number of elements returned by the MergeSort /// Maximum number of results to return /// New MergeSort that will only return the specified number of results - public override FdbAsyncIterator Take(int limit) + public override AsyncIterator Take(int limit) { - if (limit < 0) throw new ArgumentOutOfRangeException("limit", "Value cannot be less than zero"); + if (limit < 0) throw new ArgumentOutOfRangeException(nameof(limit), "Value cannot be less than zero"); if (m_limit != null && m_limit < limit) return this; - return new FdbMergeSortIterator( + return new MergeSortAsyncIterator( m_sources, limit, m_keySelector, diff --git a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/ObserverAsyncIterator.cs similarity index 75% rename from FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/ObserverAsyncIterator.cs index cbf33dfcb..1a06eb70f 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbObserverIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/ObserverAsyncIterator.cs @@ -26,42 +26,42 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Expressions; /// Observe the items of an async sequence /// Type of the observed elements - internal sealed class FdbObserverIterator : FdbAsyncFilterIterator + public sealed class ObserverAsyncIterator : AsyncFilterIterator { private readonly AsyncObserverExpression m_observer; - public FdbObserverIterator(IFdbAsyncEnumerable source, AsyncObserverExpression observer) + public ObserverAsyncIterator(IAsyncEnumerable source, AsyncObserverExpression observer) : base(source) { Contract.Requires(observer != null); m_observer = observer; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbObserverIterator(m_source, m_observer); + return new ObserverAsyncIterator(m_source, m_observer); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_observer.Async) @@ -70,13 +70,13 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - await m_observer.InvokeAsync(current, ct).ConfigureAwait(false); + await m_observer.InvokeAsync(current, m_ct).ConfigureAwait(false); } return Publish(current); } - return Canceled(ct); + return Canceled(); } } diff --git a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs similarity index 84% rename from FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs index 53ebc2b59..fe29ad7b0 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs @@ -28,20 +28,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define FULL_DEBUG -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using JetBrains.Annotations; /// [EXPERIMENTAL] Iterates over an async sequence of items, kick off an async task in parallel, and returning the results in order /// Type of elements of the inner async sequence /// Type of elements of the outer async sequence - internal sealed class FdbParallelSelectAsyncIterator : FdbAsyncFilterIterator + public sealed class ParallelSelectAsyncIterator : AsyncFilterIterator { /// Default max concurrency when doing batch queries /// TODO: this is a placeholder value ! @@ -52,23 +52,23 @@ internal sealed class FdbParallelSelectAsyncIterator : FdbAsyn // Since we can't spin out too many tasks, we also want to be able to put a cap no the max number of pending tasks private readonly Func> m_taskSelector; - private readonly FdbParallelQueryOptions m_options; + private readonly ParallelAsyncQueryOptions m_options; private CancellationTokenSource m_cts; private CancellationToken m_token; private volatile bool m_done; /// Pump that reads values from the inner iterator - private FdbAsyncIteratorPump m_pump; + private AsyncIteratorPump m_pump; /// Inner pump task private Task m_pumpTask; /// Queue that holds items that are being processed private AsyncTransformQueue m_processingQueue; - public FdbParallelSelectAsyncIterator( - [NotNull] IFdbAsyncEnumerable source, + public ParallelSelectAsyncIterator( + [NotNull] IAsyncEnumerable source, [NotNull] Func> taskSelector, - [NotNull] FdbParallelQueryOptions options + [NotNull] ParallelAsyncQueryOptions options ) : base(source) { @@ -78,13 +78,13 @@ [NotNull] FdbParallelQueryOptions options m_options = options; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbParallelSelectAsyncIterator(m_source, m_taskSelector, m_options); + return new ParallelSelectAsyncIterator(m_source, m_taskSelector, m_options); } - protected override async Task OnFirstAsync(CancellationToken ct) + protected override async Task OnFirstAsync() { - if (!await base.OnFirstAsync(ct)) + if (!await base.OnFirstAsync()) { return false; } @@ -99,12 +99,13 @@ protected override async Task OnFirstAsync(CancellationToken ct) m_processingQueue = new AsyncTransformQueue(m_taskSelector, m_options.MaxConcurrency ?? DefaultMaxConcurrency, m_options.Scheduler); // we also need a pump that will work on the inner sequence - m_pump = new FdbAsyncIteratorPump(m_iterator, m_processingQueue); + m_pump = new AsyncIteratorPump(m_iterator, m_processingQueue); // start pumping m_pumpTask = m_pump.PumpAsync(m_token).ContinueWith((t) => - { - var e = t.Exception; + { + // ReSharper disable once RedundantAssignment + var e = t.Exception; // observe the exception LogDebug("Pump stopped with error: " + e.Message); }, TaskContinuationOptions.OnlyOnFaulted); @@ -115,7 +116,7 @@ protected override async Task OnFirstAsync(CancellationToken ct) return true; } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { try { @@ -123,13 +124,13 @@ protected override async Task OnNextAsync(CancellationToken ct) if (m_done) return false; - var next = await m_processingQueue.ReceiveAsync(ct).ConfigureAwait(false); + var next = await m_processingQueue.ReceiveAsync(m_ct).ConfigureAwait(false); LogDebug("[OnNextAsync] got result from queue"); if (!next.HasValue) { m_done = true; - if (next.HasFailed) + if (next.Failed) { LogDebug("[OnNextAsync] received failure"); // we want to make sure that the exception callstack is as clean as possible, diff --git a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs similarity index 78% rename from FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs index 0d303db18..f8fc010c6 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbPrefetchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs @@ -26,18 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; /// Prefetches items from the inner sequence, before outputing them down the line. /// Type the the items from the source sequence - internal class FdbPrefetchingAsyncIterator : FdbAsyncFilterIterator + public class PrefetchingAsyncIterator : AsyncFilterIterator { // This iterator can be used to already ask for the next few items, while they are being processed somewhere down the line of the query. // This can be usefull, when combined with Batching or Windowing, to maximize the throughput of db queries that read pages of results at a time. @@ -57,25 +57,25 @@ internal class FdbPrefetchingAsyncIterator : FdbAsyncFilterIteratorCreate a new batching iterator /// Source sequence of items that must be batched by waves /// Maximum size of a batch to return down the line - public FdbPrefetchingAsyncIterator(IFdbAsyncEnumerable source, int prefetchCount) + public PrefetchingAsyncIterator(IAsyncEnumerable source, int prefetchCount) : base(source) { Contract.Requires(prefetchCount > 0); m_prefetchCount = prefetchCount; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbPrefetchingAsyncIterator(m_source, m_prefetchCount); + return new PrefetchingAsyncIterator(m_source, m_prefetchCount); } - protected override void OnStarted(IFdbAsyncEnumerator iterator) + protected override void OnStarted(IAsyncEnumerator iterator) { // pre-allocate the buffer with the number of slot we expect to use m_buffer = new Queue(m_prefetchCount); } - protected override Task OnNextAsync(CancellationToken ct) + protected override Task OnNextAsync() { var buffer = m_buffer; if (buffer != null && buffer.Count > 0) @@ -83,48 +83,47 @@ protected override Task OnNextAsync(CancellationToken ct) var nextTask = m_nextTask; if (nextTask == null || !m_nextTask.IsCompleted) { - var current = buffer.Dequeue(); - return Publish(current) ? TaskHelpers.TrueTask : TaskHelpers.FalseTask; + return TaskHelpers.FromResult(Publish(buffer.Dequeue())); } } - return PrefetchNextItemsAsync(ct); + return PrefetchNextItemsAsync(); } - protected virtual async Task PrefetchNextItemsAsync(CancellationToken ct) + protected virtual async Task PrefetchNextItemsAsync() { // read items from the source until the next call to Inner.MoveNext() is not already complete, or we have filled our prefetch buffer, then returns the first item in the buffer. - var t = Interlocked.Exchange(ref m_nextTask, null); - if (t == null) + var ft = Interlocked.Exchange(ref m_nextTask, null); + if (ft == null) { // read the next item from the inner iterator if (m_innerHasCompleted) return Completed(); - t = m_iterator.MoveNextAsync(ct); + ft = m_iterator.MoveNextAsync(); } // always wait for the first item (so that we have at least something in the batch) - bool hasMore = await t.ConfigureAwait(false); + bool hasMore = await ft.ConfigureAwait(false); // most db queries will read items by chunks, so there is a high chance the the next following calls to MoveNext() will already be completed // as long as this is the case, and that our buffer is not full, continue eating items. Stop only when we end up with a pending task. - while (hasMore && !ct.IsCancellationRequested) + while (hasMore && !m_ct.IsCancellationRequested) { if (m_buffer == null) m_buffer = new Queue(m_prefetchCount); m_buffer.Enqueue(m_iterator.Current); - t = m_iterator.MoveNextAsync(ct); - if (m_buffer.Count >= m_prefetchCount || !t.IsCompleted) + var vt = m_iterator.MoveNextAsync(); + if (m_buffer.Count >= m_prefetchCount || !vt.IsCompleted) { // save it for next time - m_nextTask = t; + m_nextTask = vt; break; } // we know the task is already completed, so we will immediately get the next result, or blow up if the inner iterator failed - hasMore = t.GetAwaiter().GetResult(); + hasMore = vt.Result; //note: if inner blows up, we won't send any previously read items down the line. This may change the behavior of queries with a .Take(N) that would have stopped before reading the (N+1)th item that would have failed. } - ct.ThrowIfCancellationRequested(); + m_ct.ThrowIfCancellationRequested(); if (!hasMore) { @@ -144,8 +143,7 @@ protected override void OnStopped() m_buffer = null; // defuse the task, which should fail once we dispose the inner iterator below... - var nextTask = Interlocked.Exchange(ref m_nextTask, null); - if (nextTask != null) TaskHelpers.Observe(nextTask); + Interlocked.Exchange(ref m_nextTask, null)?.Observed(); } } diff --git a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/SelectManyAsyncIterator.cs similarity index 73% rename from FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/SelectManyAsyncIterator.cs index e30442dfe..96d3e7c63 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbSelectManyAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/SelectManyAsyncIterator.cs @@ -26,24 +26,24 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Expressions; using JetBrains.Annotations; /// Iterates over an async sequence of items /// Type of elements of the inner async sequence /// Type of elements of the outer async sequence - internal sealed class FdbSelectManyAsyncIterator : FdbAsyncFilterIterator + public sealed class SelectManyAsyncIterator : AsyncFilterIterator { private readonly AsyncTransformExpression> m_selector; private IEnumerator m_batch; - public FdbSelectManyAsyncIterator([NotNull] IFdbAsyncEnumerable source, AsyncTransformExpression> selector) + public SelectManyAsyncIterator([NotNull] IAsyncEnumerable source, AsyncTransformExpression> selector) : base(source) { // Must have at least one, but not both @@ -52,28 +52,28 @@ public FdbSelectManyAsyncIterator([NotNull] IFdbAsyncEnumerable source, m_selector = selector; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbSelectManyAsyncIterator(m_source, m_selector); + return new SelectManyAsyncIterator(m_source, m_selector); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { // if we are in a batch, iterate over it // if not, wait for the next batch - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { if (m_batch == null) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // inner completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; IEnumerable sequence; if (!m_selector.Async) @@ -82,7 +82,7 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - sequence = await m_selector.InvokeAsync(m_iterator.Current, ct).ConfigureAwait(false); + sequence = await m_selector.InvokeAsync(m_iterator.Current, m_ct).ConfigureAwait(false); } if (sequence == null) throw new InvalidOperationException("The inner sequence returned a null collection"); @@ -100,17 +100,14 @@ protected override async Task OnNextAsync(CancellationToken ct) return Publish(m_batch.Current); } - return Canceled(ct); + return Canceled(); } protected override void Cleanup() { try { - if (m_batch != null) - { - m_batch.Dispose(); - } + m_batch?.Dispose(); } finally { @@ -124,15 +121,15 @@ protected override void Cleanup() /// Type of elements of the inner async sequence /// Type of the elements of the sequences produced from each elements /// Type of elements of the outer async sequence - internal sealed class FdbSelectManyAsyncIterator : FdbAsyncFilterIterator + internal sealed class SelectManyAsyncIterator : AsyncFilterIterator { private readonly AsyncTransformExpression> m_collectionSelector; private readonly Func m_resultSelector; private TSource m_sourceCurrent; private IEnumerator m_batch; - public FdbSelectManyAsyncIterator( - [NotNull] IFdbAsyncEnumerable source, + public SelectManyAsyncIterator( + [NotNull] IAsyncEnumerable source, AsyncTransformExpression> collectionSelector, [NotNull] Func resultSelector ) @@ -144,28 +141,28 @@ [NotNull] Func resultSelector m_resultSelector = resultSelector; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbSelectManyAsyncIterator(m_source, m_collectionSelector, m_resultSelector); + return new SelectManyAsyncIterator(m_source, m_collectionSelector, m_resultSelector); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { // if we are in a batch, iterate over it // if not, wait for the next batch - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - - if (m_batch == null) + var batch = m_batch; + if (batch == null) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // inner completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; m_sourceCurrent = m_iterator.Current; @@ -177,36 +174,33 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - sequence = await m_collectionSelector.InvokeAsync(m_sourceCurrent, ct).ConfigureAwait(false); + sequence = await m_collectionSelector.InvokeAsync(m_sourceCurrent, m_ct).ConfigureAwait(false); } if (sequence == null) throw new InvalidOperationException("The inner sequence returned a null collection"); - m_batch = sequence.GetEnumerator(); - Contract.Requires(m_batch != null); + m_batch = batch = sequence.GetEnumerator(); + Contract.Requires(batch != null); } - if (!m_batch.MoveNext()) + if (!batch.MoveNext()) { // the current batch is exhausted, move to the next - m_batch.Dispose(); + batch.Dispose(); m_batch = null; m_sourceCurrent = default(TSource); continue; } - return Publish(m_resultSelector(m_sourceCurrent, m_batch.Current)); + return Publish(m_resultSelector(m_sourceCurrent, batch.Current)); } - return Canceled(ct); + return Canceled(); } protected override void Cleanup() { try - { - if (m_batch != null) - { - m_batch.Dispose(); - } + { // cleanup any pending batch + m_batch?.Dispose(); } finally { diff --git a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/TakeWhileAsyncIterator.cs similarity index 77% rename from FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/TakeWhileAsyncIterator.cs index 17af0110a..c63b143b6 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbTakeWhileAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/TakeWhileAsyncIterator.cs @@ -26,22 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; /// Reads an async sequence of items until a condition becomes false /// Type of elements of the async sequence - internal sealed class FdbTakeWhileAsyncIterator : FdbAsyncFilterIterator + public sealed class TakeWhileAsyncIterator : AsyncFilterIterator { private readonly Func m_condition; //TODO: also accept a Func> ? - public FdbTakeWhileAsyncIterator([NotNull] IFdbAsyncEnumerable source, [NotNull] Func condition) + public TakeWhileAsyncIterator([NotNull] IAsyncEnumerable source, [NotNull] Func condition) : base(source) { Contract.Requires(condition != null); @@ -49,21 +48,21 @@ public FdbTakeWhileAsyncIterator([NotNull] IFdbAsyncEnumerable source, m_condition = condition; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbTakeWhileAsyncIterator(m_source, m_condition); + return new TakeWhileAsyncIterator(m_source, m_condition); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_condition(current)) @@ -73,7 +72,7 @@ protected override async Task OnNextAsync(CancellationToken ct) return Publish(current); } - return Canceled(ct); + return Canceled(); } } diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/WhereAsyncIterator.cs similarity index 70% rename from FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/WhereAsyncIterator.cs index 34a2ddf41..104f69272 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/WhereAsyncIterator.cs @@ -26,22 +26,23 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; + using Doxense.Linq.Async.Expressions; using JetBrains.Annotations; /// Filters an async sequence of items /// Type of elements of the async sequence - internal sealed class FdbWhereAsyncIterator : FdbAsyncFilterIterator + public sealed class WhereAsyncIterator : AsyncFilterIterator { private readonly AsyncFilterExpression m_filter; - public FdbWhereAsyncIterator([NotNull] IFdbAsyncEnumerable source, AsyncFilterExpression filter) + public WhereAsyncIterator([NotNull] IAsyncEnumerable source, AsyncFilterExpression filter) : base(source) { Contract.Requires(filter != null, "there can be only one kind of filter specified"); @@ -49,21 +50,21 @@ public FdbWhereAsyncIterator([NotNull] IFdbAsyncEnumerable source, Asyn m_filter = filter; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbWhereAsyncIterator(m_source, m_filter); + return new WhereAsyncIterator(m_source, m_filter); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; TSource current = m_iterator.Current; if (!m_filter.Async) @@ -75,7 +76,7 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - if (!await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) + if (!await m_filter.InvokeAsync(current, m_ct).ConfigureAwait(false)) { continue; } @@ -84,28 +85,28 @@ protected override async Task OnNextAsync(CancellationToken ct) return Publish(current); } - return Canceled(ct); + return Canceled(); } - public override FdbAsyncIterator Where(Func predicate) + public override AsyncIterator Where(Func predicate) { - return FdbAsyncEnumerable.Filter( + return AsyncEnumerable.Filter( m_source, m_filter.AndAlso(new AsyncFilterExpression(predicate)) ); } - public override FdbAsyncIterator Where(Func> asyncPredicate) + public override AsyncIterator Where(Func> asyncPredicate) { - return FdbAsyncEnumerable.Filter( + return AsyncEnumerable.Filter( m_source, m_filter.AndAlso(new AsyncFilterExpression(asyncPredicate)) ); } - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, new AsyncTransformExpression(selector), @@ -114,9 +115,9 @@ public override FdbAsyncIterator Select(Func selector ); } - public override FdbAsyncIterator Select(Func> asyncSelector) + public override AsyncIterator Select(Func> asyncSelector) { - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, new AsyncTransformExpression(asyncSelector), @@ -125,14 +126,14 @@ public override FdbAsyncIterator Select(Func Take(int limit) + public override AsyncIterator Take(int limit) { - if (limit < 0) throw new ArgumentOutOfRangeException("limit", "Limit cannot be less than zero"); + if (limit < 0) throw new ArgumentOutOfRangeException(nameof(limit), "Limit cannot be less than zero"); - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, - new AsyncTransformExpression(TaskHelpers.Cache.Identity), + new AsyncTransformExpression(TaskHelpers.CachedTasks.Identity), limit: limit, offset: null ); @@ -140,15 +141,15 @@ public override FdbAsyncIterator Take(int limit) public override async Task ExecuteAsync(Action handler, CancellationToken ct) { - if (handler == null) throw new ArgumentNullException("handler"); + Contract.NotNull(handler, nameof(handler)); if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); - using (var iter = StartInner()) + using (var iter = StartInner(ct)) { if (!m_filter.Async) { - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (m_filter.Invoke(current)) @@ -159,7 +160,7 @@ public override async Task ExecuteAsync(Action handler, CancellationTok } else { - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) @@ -175,15 +176,15 @@ public override async Task ExecuteAsync(Action handler, CancellationTok public override async Task ExecuteAsync(Func asyncHandler, CancellationToken ct) { - if (asyncHandler == null) throw new ArgumentNullException("asyncHandler"); + Contract.NotNull(asyncHandler, nameof(asyncHandler)); if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); - using (var iter = StartInner()) + using (var iter = StartInner(ct)) { if (!m_filter.Async) { - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (m_filter.Invoke(current)) @@ -194,7 +195,7 @@ public override async Task ExecuteAsync(Func a } else { - while (!ct.IsCancellationRequested && (await iter.MoveNextAsync(ct).ConfigureAwait(false))) + while (!ct.IsCancellationRequested && (await iter.MoveNextAsync().ConfigureAwait(false))) { var current = iter.Current; if (await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) diff --git a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/WhereSelectAsyncIterator.cs similarity index 71% rename from FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/WhereSelectAsyncIterator.cs index 018576670..6ae309a58 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWhereSelectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/WhereSelectAsyncIterator.cs @@ -26,19 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Expressions; using JetBrains.Annotations; /// Iterates over an async sequence of items /// Type of elements of the inner async sequence /// Type of elements of the outer async sequence - internal sealed class FdbWhereSelectAsyncIterator : FdbAsyncFilterIterator + public sealed class WhereSelectAsyncIterator : AsyncFilterIterator { private readonly AsyncFilterExpression m_filter; private readonly AsyncTransformExpression m_transform; @@ -50,8 +51,8 @@ internal sealed class FdbWhereSelectAsyncIterator : FdbAsyncFi private int? m_remaining; private int? m_skipped; - public FdbWhereSelectAsyncIterator( - [NotNull] IFdbAsyncEnumerable source, + public WhereSelectAsyncIterator( + [NotNull] IAsyncEnumerable source, AsyncFilterExpression filter, AsyncTransformExpression transform, int? limit, @@ -68,32 +69,32 @@ public FdbWhereSelectAsyncIterator( m_offset = offset; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbWhereSelectAsyncIterator(m_source, m_filter, m_transform, m_limit, m_offset); + return new WhereSelectAsyncIterator(m_source, m_filter, m_transform, m_limit, m_offset); } - protected override Task OnFirstAsync(CancellationToken ct) + protected override Task OnFirstAsync() { m_remaining = m_limit; m_skipped = m_offset; - return base.OnFirstAsync(ct); + return base.OnFirstAsync(); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { if (m_remaining != null && m_remaining.Value <= 0) { // reached limit! return Completed(); } - while (!ct.IsCancellationRequested) + while (!m_ct.IsCancellationRequested) { - if (!await m_iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await m_iterator.MoveNextAsync().ConfigureAwait(false)) { // completed return Completed(); } - if (ct.IsCancellationRequested) break; + if (m_ct.IsCancellationRequested) break; #region Filtering... @@ -106,7 +107,7 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - if (!await m_filter.InvokeAsync(current, ct).ConfigureAwait(false)) continue; + if (!await m_filter.InvokeAsync(current, m_ct).ConfigureAwait(false)) continue; } } @@ -136,31 +137,29 @@ protected override async Task OnNextAsync(CancellationToken ct) } else { - result = await m_transform.InvokeAsync(current, ct).ConfigureAwait(false); + result = await m_transform.InvokeAsync(current, m_ct).ConfigureAwait(false); } #endregion #region Publishing... - if (m_remaining != null) - { // decrement remaining quota - m_remaining = m_remaining.Value - 1; - } + // decrement remaining quota + m_remaining = m_remaining - 1; return Publish(result); #endregion } - return Canceled(ct); + return Canceled(); } - public override FdbAsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - if (selector == null) throw new ArgumentNullException("selector"); + Contract.NotNull(selector, nameof(selector)); - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, m_transform.Then(new AsyncTransformExpression(selector)), @@ -169,11 +168,11 @@ public override FdbAsyncIterator Select(Func selector ); } - public override FdbAsyncIterator Select(Func> asyncSelector) + public override AsyncIterator Select(Func> asyncSelector) { - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, m_transform.Then(new AsyncTransformExpression(asyncSelector)), @@ -182,13 +181,13 @@ public override FdbAsyncIterator Select(Func SelectMany(Func> selector) + public override AsyncIterator SelectMany(Func> selector) { - if (selector == null) throw new ArgumentNullException("selector"); + Contract.NotNull(selector, nameof(selector)); if (m_filter == null && m_limit == null && m_offset == null) { - return new FdbSelectManyAsyncIterator( + return new SelectManyAsyncIterator( m_source, m_transform.Then(new AsyncTransformExpression>(selector)) ); @@ -198,13 +197,13 @@ public override FdbAsyncIterator SelectMany(Func(selector); } - public override FdbAsyncIterator SelectMany(Func>> asyncSelector) + public override AsyncIterator SelectMany(Func>> asyncSelector) { - if (asyncSelector == null) throw new ArgumentNullException("asyncSelector"); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); if (m_filter == null && m_limit == null && m_offset == null) { - return new FdbSelectManyAsyncIterator( + return new SelectManyAsyncIterator( m_source, m_transform.Then(new AsyncTransformExpression>(asyncSelector)) ); @@ -214,9 +213,9 @@ public override FdbAsyncIterator SelectMany(Func(asyncSelector); } - public override FdbAsyncIterator Take(int limit) + public override AsyncIterator Take(int limit) { - if (limit < 0) throw new ArgumentOutOfRangeException("limit", "Limit cannot be less than zero"); + if (limit < 0) throw new ArgumentOutOfRangeException(nameof(limit), "Limit cannot be less than zero"); if (m_limit != null && m_limit.Value <= limit) { @@ -224,7 +223,7 @@ public override FdbAsyncIterator Take(int limit) return this; } - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, m_transform, @@ -233,15 +232,15 @@ public override FdbAsyncIterator Take(int limit) ); } - public override FdbAsyncIterator Skip(int offset) + public override AsyncIterator Skip(int offset) { - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); if (offset == 0) return this; if (m_offset != null) offset += m_offset.Value; - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, m_filter, m_transform, @@ -250,9 +249,9 @@ public override FdbAsyncIterator Skip(int offset) ); } - public override FdbAsyncIterator Where(Func predicate) + public override AsyncIterator Where(Func predicate) { - if (predicate == null) throw new ArgumentNullException("predicate"); + Contract.NotNull(predicate, nameof(predicate)); // note: the only possible optimization here is if TSource == TResult, then we can combine both predicates // remember: limit/offset are applied AFTER the filtering, so can only combine if they are null @@ -266,9 +265,9 @@ public override FdbAsyncIterator Where(Func predicate) if (m_filter != null) filter = m_filter.AndAlso(filter); //BUGBUG: if the query already has a select, it should be evaluated BEFORE the new filter, - // but currently FdbWhereSelectAsyncIterator<> filters before transformations ! + // but currently WhereSelectAsyncIterator<> filters before transformations ! - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, filter, m_transform, @@ -281,9 +280,9 @@ public override FdbAsyncIterator Where(Func predicate) return base.Where(predicate); } - public override FdbAsyncIterator Where(Func> asyncPredicate) + public override AsyncIterator Where(Func> asyncPredicate) { - if (asyncPredicate == null) throw new ArgumentNullException("asyncPredicate"); + Contract.NotNull(asyncPredicate, nameof(asyncPredicate)); // note: the only possible optimization here is if TSource == TResult, then we can combine both predicates // remember: limit/offset are applied AFTER the filtering, so can only combine if they are null @@ -293,9 +292,9 @@ public override FdbAsyncIterator Where(Func filters before transformations ! + // but currently WhereSelectAsyncIterator<> filters before transformations ! - return new FdbWhereSelectAsyncIterator( + return new WhereSelectAsyncIterator( m_source, asyncFilter, m_transform, @@ -310,16 +309,16 @@ public override FdbAsyncIterator Where(Func action, CancellationToken ct) { - if (action == null) throw new ArgumentNullException("action"); + Contract.NotNull(action, nameof(action)); int? remaining = m_limit; int? skipped = m_offset; - using (var iterator = StartInner()) + using (var iterator = StartInner(ct)) { while (remaining == null || remaining.Value > 0) { - if (!await iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await iterator.MoveNextAsync().ConfigureAwait(false)) { // completed break; } @@ -366,10 +365,8 @@ public override async Task ExecuteAsync(Action action, CancellationToke // Publish... - if (remaining != null) - { // decrement remaining quota - remaining = remaining.Value - 1; - } + // decrement remaining quota + remaining = remaining - 1; action(result); } @@ -379,16 +376,16 @@ public override async Task ExecuteAsync(Action action, CancellationToke public override async Task ExecuteAsync(Func asyncAction, CancellationToken ct) { - if (asyncAction == null) throw new ArgumentNullException("asyncAction"); + Contract.NotNull(asyncAction, nameof(asyncAction)); int? remaining = m_limit; int? skipped = m_offset; - using (var iterator = StartInner()) + using (var iterator = StartInner(ct)) { while (remaining == null || remaining.Value > 0) { - if (!await iterator.MoveNextAsync(ct).ConfigureAwait(false)) + if (!await iterator.MoveNextAsync().ConfigureAwait(false)) { // completed break; } @@ -435,10 +432,9 @@ public override async Task ExecuteAsync(Func a // Publish... - if (remaining != null) - { // decrement remaining quota - remaining = remaining.Value - 1; - } + // decrement remaining quota + remaining = remaining - 1; + await asyncAction(result, ct).ConfigureAwait(false); } @@ -447,4 +443,4 @@ public override async Task ExecuteAsync(Func a } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs similarity index 87% rename from FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs rename to FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs index 7e357e396..14844e84c 100644 --- a/FoundationDB.Client/Linq/Iterators/FdbWindowingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs @@ -26,18 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq.Async.Iterators { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; /// Merges bursts of already-completed items from a source async sequence, into a sequence of batches. /// Type the the items from the source sequence - internal class FdbWindowingAsyncIterator : FdbAsyncFilterIterator + public class WindowingAsyncIterator : AsyncFilterIterator { // Typical use cas: to merge back into arrays the result of readers that read one page at a time from the database, but return each item individually. // This iterator will attempt to reconstruct full batches from sequences of items that where all produced at the same time, so that asynchronous operations @@ -99,64 +99,64 @@ internal class FdbWindowingAsyncIterator : FdbAsyncFilterIteratorCreate a new batching iterator /// Source sequence of items that must be batched by waves /// Maximum size of a batch to return down the line - public FdbWindowingAsyncIterator(IFdbAsyncEnumerable source, int maxWindowSize) + public WindowingAsyncIterator(IAsyncEnumerable source, int maxWindowSize) : base(source) { Contract.Requires(maxWindowSize > 0); m_maxWindowSize = maxWindowSize; } - protected override FdbAsyncIterator Clone() + protected override AsyncIterator Clone() { - return new FdbWindowingAsyncIterator(m_source, m_maxWindowSize); + return new WindowingAsyncIterator(m_source, m_maxWindowSize); } - protected override void OnStarted(IFdbAsyncEnumerator iterator) + protected override void OnStarted(IAsyncEnumerator iterator) { // pre-allocate the inner buffer, if it is not too big m_buffer = new List(Math.Min(m_maxWindowSize, 1024)); } - protected override async Task OnNextAsync(CancellationToken ct) + protected override async Task OnNextAsync() { // read items from the source until the next call to Inner.MoveNext() is not already complete, or we have filled our buffer var iterator = m_iterator; var buffer = m_buffer; - var t = Interlocked.Exchange(ref m_nextTask, null); - if (t == null) + var ft = Interlocked.Exchange(ref m_nextTask, null); + if (ft == null) { // read the next item from the inner iterator if (m_innerHasCompleted) return Completed(); - t = iterator.MoveNextAsync(ct); + ft = iterator.MoveNextAsync(); } // always wait for the first item (so that we have at least something in the batch) - bool hasMore = await t.ConfigureAwait(false); + bool hasMore = await ft.ConfigureAwait(false); // most db queries will read items by chunks, so there is a high chance the the next following calls to MoveNext() will already be completed // as long as this is the case, and that our buffer is not full, continue eating items. Stop only when we end up with a pending task. - while (hasMore && !ct.IsCancellationRequested) + while (hasMore && !m_ct.IsCancellationRequested) { buffer.Add(iterator.Current); - t = iterator.MoveNextAsync(ct); - if (buffer.Count >= m_maxWindowSize || !t.IsCompleted) + var vt = iterator.MoveNextAsync(); + if (buffer.Count >= m_maxWindowSize || !vt.IsCompleted) { // save it for next time //TODO: add heuristics to check if the batch is large enough to stop there, or if we should eat the latency and wait for the next wave of items to arrive! // ex: we batch by 10, inner return 11 consecutive items. We will transform the first 10, then only fill the next batch with the 11th item because the 12th item is still not ready. - m_nextTask = t; + m_nextTask = vt; break; } // we know the task is already completed, so we will immediately get the next result, or blow up if the inner iterator failed - hasMore = t.GetAwaiter().GetResult(); + hasMore = vt.Result; //note: if inner blows up, we won't send any previously read items down the line. This may change the behavior of queries with a .Take(N) that would have stopped before reading the (N+1)th item that would have failed. } - ct.ThrowIfCancellationRequested(); + m_ct.ThrowIfCancellationRequested(); if (!hasMore) { @@ -180,8 +180,7 @@ protected override void OnStopped() m_buffer = null; // defuse the task, which should fail once we dispose the inner iterator below... - var nextTask = Interlocked.Exchange(ref m_nextTask, null); - if (nextTask != null) TaskHelpers.Observe(nextTask); + Interlocked.Exchange(ref m_nextTask, null)?.Observed(); } } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs similarity index 56% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs index a2142cc2b..3d0b6282c 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs @@ -26,58 +26,47 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { /// An empty sequence - private sealed class EmptySequence : IFdbAsyncEnumerable, IFdbAsyncEnumerator + private sealed class EmptySequence : IAsyncEnumerable, IAsyncEnumerator { public static readonly EmptySequence Default = new EmptySequence(); private EmptySequence() { } - Task IAsyncEnumerator.MoveNextAsync(CancellationToken ct) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { ct.ThrowIfCancellationRequested(); - return TaskHelpers.FalseTask; + return this; } - TSource IAsyncEnumerator.Current + Task IAsyncEnumerator.MoveNextAsync() { - get { throw new InvalidOperationException("This sequence is emty"); } + return TaskHelpers.False; } - void IDisposable.Dispose() - { - // NOOP! - } + TSource IAsyncEnumerator.Current => default(TSource); - public IAsyncEnumerator GetEnumerator() - { - return this; - } + void IDisposable.Dispose() + { } - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode) - { - return this; - } } - private sealed class SingletonSequence : IFdbAsyncEnumerable, IFdbAsyncEnumerator + private sealed class SingletonSequence : IAsyncEnumerable { private readonly Delegate m_lambda; - private TElement m_current; - private bool m_called; private SingletonSequence(Delegate lambda) { @@ -97,54 +86,67 @@ public SingletonSequence(Func> lambda) : this((Delegate)lambda) { } - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode = FdbAsyncMode.Default) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { - return new SingletonSequence(m_lambda); + ct.ThrowIfCancellationRequested(); + return new Enumerator(m_lambda, ct); } - IAsyncEnumerator IAsyncEnumerable.GetEnumerator() + private sealed class Enumerator : IAsyncEnumerator { - return this.GetEnumerator(); - } + //REVIEW: we could have specialized version for Task returning vs non-Task returning lambdas - async Task IAsyncEnumerator.MoveNextAsync(CancellationToken ct) - { - ct.ThrowIfCancellationRequested(); - if (m_called) return false; + private CancellationToken m_ct; + private readonly Delegate m_lambda; + private bool m_called; + private TElement m_current; - //note: avoid using local variables as much as possible! - m_called = true; - var lambda = m_lambda; - if (lambda is Func) + public Enumerator(Delegate lambda, CancellationToken ct) { - m_current = ((Func)lambda)(); - return true; + m_ct = ct; + m_lambda = lambda; } - if (lambda is Func>) + public async Task MoveNextAsync() { - m_current = await ((Func>)lambda)().ConfigureAwait(false); - return true; + m_ct.ThrowIfCancellationRequested(); + if (m_called) + { + m_current = default(TElement); + return false; + } + + //note: avoid using local variables as much as possible! + m_called = true; + var lambda = m_lambda; + if (lambda is Func f) + { + m_current = f(); + return true; + } + + if (lambda is Func> ft) + { + m_current = await ft().ConfigureAwait(false); + return true; + } + + if (lambda is Func> fct) + { + m_current = await fct(m_ct).ConfigureAwait(false); + return true; + } + + throw new InvalidOperationException("Unsupported delegate type"); } - if (lambda is Func>) + public TElement Current => m_current; + + public void Dispose() { - m_current = await ((Func>)lambda)(ct).ConfigureAwait(false); - return true; + m_called = true; + m_current = default(TElement); } - - throw new InvalidOperationException("Unsupported delegate type"); - } - - TElement IAsyncEnumerator.Current - { - get { return m_current; } - } - - void IDisposable.Dispose() - { - m_called = true; - m_current = default(TElement); } } } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs similarity index 88% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs index e948279fa..a5d7f162e 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableIterator.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs @@ -26,37 +26,40 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { /// Iterates over a sequence of items /// Type of elements of the inner sequence /// Type of elements of the outer async sequence - internal sealed class EnumerableIterator : IFdbAsyncEnumerator + internal sealed class EnumerableIterator : IAsyncEnumerator { private IEnumerator m_iterator; private Func> m_transform; private bool m_disposed; private TResult m_current; + private CancellationToken m_ct; - public EnumerableIterator(IEnumerator iterator, Func> transform) + public EnumerableIterator(IEnumerator iterator, Func> transform, CancellationToken ct) { Contract.Requires(iterator != null && transform != null); m_iterator = iterator; m_transform = transform; + m_ct = ct; } - public async Task MoveNextAsync(CancellationToken ct) + public async Task MoveNextAsync() { if (m_disposed) { @@ -64,7 +67,7 @@ public async Task MoveNextAsync(CancellationToken ct) return false; } - ct.ThrowIfCancellationRequested(); + m_ct.ThrowIfCancellationRequested(); if (m_iterator.MoveNext()) { @@ -88,14 +91,12 @@ public TResult Current public void Dispose() { - if (m_iterator != null) - { - m_iterator.Dispose(); - } + m_iterator?.Dispose(); m_iterator = null; m_transform = null; m_disposed = true; m_current = default(TResult); + m_ct = default(CancellationToken); } } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs similarity index 82% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs index 1a78ff6a1..ab6a9b6d3 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.EnumerableSequence.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs @@ -26,36 +26,32 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Collections.Generic; + using System.Threading; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; + using Doxense.Async; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { /// Wraps a sequence of items into an async sequence of items /// Type of elements of the inner sequence /// Type of elements of the outer async sequence - internal sealed class EnumerableSequence : IFdbAsyncEnumerable + internal sealed class EnumerableSequence : IAsyncEnumerable { public readonly IEnumerable Source; - public readonly Func, IFdbAsyncEnumerator> Factory; + public readonly Func, CancellationToken, IAsyncEnumerator> Factory; - public EnumerableSequence(IEnumerable source, Func, IFdbAsyncEnumerator> factory) + public EnumerableSequence(IEnumerable source, Func, CancellationToken, IAsyncEnumerator> factory) { this.Source = source; this.Factory = factory; } - public IAsyncEnumerator GetEnumerator() - { - return this.GetEnumerator(FdbAsyncMode.Default); - } - - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode _) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint _) { IEnumerator inner = null; try @@ -63,7 +59,7 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode _) inner = this.Source.GetEnumerator(); Contract.Assert(inner != null, "The underlying sequence returned an empty enumerator"); - var outer = this.Factory(inner); + var outer = this.Factory(inner, ct); if (outer == null) throw new InvalidOperationException("The async factory returned en empty enumerator"); return outer; @@ -71,7 +67,7 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode _) catch (Exception) { //make sure that the inner iterator gets disposed if something went wrong - if (inner != null) inner.Dispose(); + inner?.Dispose(); throw; } } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs b/FoundationDB.Client/Linq/AsyncEnumerable.Iterators.cs similarity index 54% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.Iterators.cs index 8df8c4673..adf835a6d 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.Iterators.cs @@ -26,18 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; + using Doxense.Linq.Async; + using Doxense.Linq.Async.Expressions; + using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { #region Create... @@ -48,12 +51,12 @@ public static partial class FdbAsyncEnumerable /// Source async sequence that will be wrapped /// Factory method called when the outer sequence starts iterating. Must return an async enumerator /// New async sequence - internal static FdbAsyncSequence Create( - IFdbAsyncEnumerable source, - Func, - IFdbAsyncEnumerator> factory) + internal static AsyncSequence Create( + IAsyncEnumerable source, + Func, + IAsyncEnumerator> factory) { - return new FdbAsyncSequence(source, factory); + return new AsyncSequence(source, factory); } /// Create a new async sequence that will transform an inner sequence @@ -64,41 +67,36 @@ internal static FdbAsyncSequence Create( /// New async sequence internal static EnumerableSequence Create( IEnumerable source, - Func, - IFdbAsyncEnumerator> factory) + Func, CancellationToken, IAsyncEnumerator> factory) { return new EnumerableSequence(source, factory); } /// Create a new async sequence from a factory method - public static IFdbAsyncEnumerable Create( - Func> factory, + public static IAsyncEnumerable Create( + Func> factory, object state = null) { return new AnonymousIterable(factory, state); } - internal sealed class AnonymousIterable : IFdbAsyncEnumerable + internal sealed class AnonymousIterable : IAsyncEnumerable { - private readonly Func> m_factory; + private readonly Func> m_factory; private readonly object m_state; - public AnonymousIterable(Func> factory, object state) + public AnonymousIterable(Func> factory, object state) { Contract.Requires(factory != null); m_factory = factory; m_state = state; } - public IAsyncEnumerator GetEnumerator() + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint _) { - return this.GetEnumerator(FdbAsyncMode.Default); - } - - public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode _) - { - return m_factory(m_state); + ct.ThrowIfCancellationRequested(); + return m_factory(m_state, ct); } } @@ -107,20 +105,20 @@ public IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode _) #region Helpers... [NotNull] - internal static FdbSelectManyAsyncIterator Flatten( - [NotNull] IFdbAsyncEnumerable source, + internal static SelectManyAsyncIterator Flatten( + [NotNull] IAsyncEnumerable source, [NotNull] AsyncTransformExpression> selector) { - return new FdbSelectManyAsyncIterator(source, selector); + return new SelectManyAsyncIterator(source, selector); } [NotNull] - internal static FdbSelectManyAsyncIterator Flatten( - [NotNull] IFdbAsyncEnumerable source, + internal static SelectManyAsyncIterator Flatten( + [NotNull] IAsyncEnumerable source, [NotNull] AsyncTransformExpression> collectionSelector, [NotNull] Func resultSelector) { - return new FdbSelectManyAsyncIterator( + return new SelectManyAsyncIterator( source, collectionSelector, resultSelector @@ -128,250 +126,51 @@ internal static FdbSelectManyAsyncIterator Flatte } [NotNull] - internal static FdbWhereSelectAsyncIterator Map( - [NotNull] IFdbAsyncEnumerable source, + internal static WhereSelectAsyncIterator Map( + [NotNull] IAsyncEnumerable source, [NotNull] AsyncTransformExpression selector, int? limit = null, int? offset = null) { - return new FdbWhereSelectAsyncIterator(source, filter: null, transform: selector, limit: limit, offset: offset); + return new WhereSelectAsyncIterator(source, filter: null, transform: selector, limit: limit, offset: offset); } [NotNull] - internal static FdbWhereAsyncIterator Filter( - [NotNull] IFdbAsyncEnumerable source, + internal static WhereAsyncIterator Filter( + [NotNull] IAsyncEnumerable source, [NotNull] AsyncFilterExpression filter) { - return new FdbWhereAsyncIterator(source, filter); + return new WhereAsyncIterator(source, filter); } [NotNull] - internal static FdbWhereSelectAsyncIterator Offset( - [NotNull] IFdbAsyncEnumerable source, + internal static WhereSelectAsyncIterator Offset( + [NotNull] IAsyncEnumerable source, int offset) { - return new FdbWhereSelectAsyncIterator(source, filter: null, transform: new AsyncTransformExpression(TaskHelpers.Cache.Identity), limit: null, offset: offset); + return new WhereSelectAsyncIterator(source, filter: null, transform: new AsyncTransformExpression(TaskHelpers.CachedTasks.Identity), limit: null, offset: offset); } [NotNull] - internal static FdbWhereSelectAsyncIterator Limit( - [NotNull] IFdbAsyncEnumerable source, + internal static WhereSelectAsyncIterator Limit( + [NotNull] IAsyncEnumerable source, int limit) { - return new FdbWhereSelectAsyncIterator(source, filter: null, transform: new AsyncTransformExpression(TaskHelpers.Cache.Identity), limit: limit, offset: null); + return new WhereSelectAsyncIterator(source, filter: null, transform: new AsyncTransformExpression(TaskHelpers.CachedTasks.Identity), limit: limit, offset: null); } [NotNull] - internal static FdbTakeWhileAsyncIterator Limit( - [NotNull] IFdbAsyncEnumerable source, + internal static TakeWhileAsyncIterator Limit( + [NotNull] IAsyncEnumerable source, [NotNull] Func condition) { - return new FdbTakeWhileAsyncIterator(source, condition); + return new TakeWhileAsyncIterator(source, condition); } #endregion #region Run... - /// Small buffer that keeps a list of chunks that are larger and larger - /// Type of elements stored in the buffer - [DebuggerDisplay("Count={Count}, Chunks={this.Chunks.Length}, Current={Index}/{Current.Length}")] - internal class Buffer - { - // We want to avoid growing the same array again and again ! - // Instead, we grow list of chunks, that grow in size (until a max), and concatenate all the chunks together at the end, once we know the final size - - /// Default intial capacity, if not specified - const int DefaultCapacity = 16; - //REVIEW: should we use a power of 2 or of 10 for initial capacity? - // Since humans prefer the decimal system, it is more likely that query limit count be set to something like 10, 50, 100 or 1000 - // but most "human friendly" limits are close to the next power of 2, like 10 ~= 16, 50 ~= 64, 100 ~= 128, 500 ~= 512, 1000 ~= 1024, so we don't waste that much space... - - /// Maximum size of a chunk - const int MaxChunkSize = 4096; - - /// Number of items in the buffer - public int Count; - /// Index in the current chunk - public int Index; - /// List of chunks - public T[][] Chunks; - /// Current (and last) chunk - public T[] Current; - - public Buffer(int capacity = 0) - { - if (capacity <= 0) capacity = DefaultCapacity; - - this.Count = 0; - this.Index = 0; - this.Chunks = new T[1][]; - this.Current = new T[capacity]; - this.Chunks[0] = this.Current; - } - - public void Add(T item) - { - if (this.Index == this.Current.Length) - { - Grow(); - } - - checked { ++this.Count; } - this.Current[this.Index++] = item; - } - - private void Grow() - { - // Growth rate: - // - newly created chunk is always half the total size - // - except the first chunk who is set to the inital capacity - - Array.Resize(ref this.Chunks, this.Chunks.Length + 1); - this.Current = new T[Math.Min(this.Count, MaxChunkSize)]; - this.Chunks[this.Chunks.Length - 1] = this.Current; - this.Index = 0; - } - - [NotNull] - private T[] MergeChunks() - { - var tmp = new T[this.Count]; - int count = this.Count; - int index = 0; - for (int i = 0; i < this.Chunks.Length - 1; i++) - { - var chunk = this.Chunks[i]; - Array.Copy(chunk, 0, tmp, index, chunk.Length); - index += chunk.Length; - count -= chunk.Length; - } - Array.Copy(this.Current, 0, tmp, index, count); - return tmp; - } - - /// Return a buffer containing all of the items - /// Buffer that contains all the items, and may be larger than required - /// This is equivalent to calling ToArray(), except that if the buffer is empty, or if it consists of a single page, then no new allocations will be performed. - [NotNull] - public T[] GetBuffer() - { - //note: this is called by internal operator like OrderBy - // In this case we want to reduce the copying as much as possible, - // and we can suppose that the buffer won't be exposed to the application - - if (this.Count == 0) - { // empty - return new T[0]; - } - else if (this.Chunks.Length == 1) - { // everything fits in a single chunk - return this.Current; - } - else - { // we need to stitch all the buffers together - return MergeChunks(); - } - } - - /// Return the content of the buffer - /// Array of size containing all the items in this buffer - [NotNull] - public T[] ToArray() - { - if (this.Count == 0) - { // empty sequence - return new T[0]; - } - else if (this.Chunks.Length == 1 && this.Current.Length == this.Count) - { // a single buffer page was used - return this.Current; - } - else - { // concatenate all the buffer pages into one big array - return MergeChunks(); - } - } - - /// Return the content of the buffer - /// List of size containing all the items in this buffer - [NotNull] - public List ToList() - { - int count = this.Count; - if (count == 0) - { // empty sequence - return new List(); - } - - var list = new List(count); - var chunks = this.Chunks; - for (int i = 0; i < chunks.Length - 1; i++) - { - list.AddRange(chunks[i]); - count -= chunks[i].Length; - } - - var current = this.Current; - if (count == current.Length) - { // the last chunk fits perfectly - list.AddRange(current); - } - else - { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer - // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum - for (int i = 0; i < count; i++) - { - list.Add(current[i]); - } - } - - return list; - } - - /// Return the content of the buffer - /// List of size containing all the items in this buffer - [NotNull] - public HashSet ToHashSet(IEqualityComparer comparer = null) - { - int count = this.Count; - var hashset = new HashSet(comparer); - if (count == 0) - { - return hashset; - } - - var chunks = this.Chunks; - - for (int i = 0; i < chunks.Length - 1; i++) - { - foreach (var item in chunks[i]) - { - hashset.Add(item); - } - count -= chunks[i].Length; - } - - var current = this.Current; - if (count == current.Length) - { // the last chunk fits perfectly - foreach (var item in current) - { - hashset.Add(item); - } - } - else - { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer - // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum - for (int i = 0; i < count; i++) - { - hashset.Add(current[i]); - } - } - return hashset; - } - } - /// Immediately execute an action on each element of an async sequence /// Type of elements of the async sequence /// Source async sequence @@ -380,24 +179,24 @@ public HashSet ToHashSet(IEqualityComparer comparer = null) /// Cancellation token that can be used to cancel the operation /// Number of items that have been processed internal static async Task Run( - [NotNull] IFdbAsyncEnumerable source, - FdbAsyncMode mode, + [NotNull] IAsyncEnumerable source, + AsyncIterationHint mode, [NotNull, InstantHandle] Action action, CancellationToken ct) { - if (source == null) throw new ArgumentNullException("source"); - if (action == null) throw new ArgumentNullException("action"); + Contract.NotNull(source, nameof(source)); + Contract.NotNull(action, nameof(action)); ct.ThrowIfCancellationRequested(); //note: we should not use "ConfigureAwait(false)" here because we would like to execute the action in the original synchronization context if possible... long count = 0; - using (var iterator = source.GetEnumerator(mode)) + using (var iterator = source.GetEnumerator(ct, mode)) { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNextAsync(ct)) + while (await iterator.MoveNextAsync()) { action(iterator.Current); ++count; @@ -414,24 +213,24 @@ internal static async Task Run( /// Cancellation token that can be used to cancel the operation /// Number of items that have been processed successfully internal static async Task Run( - [NotNull] IFdbAsyncEnumerable source, - FdbAsyncMode mode, + [NotNull] IAsyncEnumerable source, + AsyncIterationHint mode, [NotNull] Func action, CancellationToken ct) { - if (source == null) throw new ArgumentNullException("source"); - if (action == null) throw new ArgumentNullException("action"); + Contract.NotNull(source, nameof(source)); + Contract.NotNull(action, nameof(action)); ct.ThrowIfCancellationRequested(); //note: we should not use "ConfigureAwait(false)" here because we would like to execute the action in the original synchronization context if possible... long count = 0; - using (var iterator = source.GetEnumerator(mode)) + using (var iterator = source.GetEnumerator(ct, mode)) { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNextAsync(ct)) + while (await iterator.MoveNextAsync()) { if (!action(iterator.Current)) { @@ -451,8 +250,8 @@ internal static async Task Run( /// Cancellation token that can be used to cancel the operation /// Number of items that have been processed internal static async Task Run( - [NotNull] IFdbAsyncEnumerable source, - FdbAsyncMode mode, + [NotNull] IAsyncEnumerable source, + AsyncIterationHint mode, [NotNull] Func action, CancellationToken ct) { @@ -461,11 +260,11 @@ internal static async Task Run( //note: we should not use "ConfigureAwait(false)" here because we would like to execute the action in the original synchronization context if possible... long count = 0; - using (var iterator = source.GetEnumerator(mode)) + using (var iterator = source.GetEnumerator(ct, mode)) { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNextAsync(ct)) + while (await iterator.MoveNextAsync()) { await action(iterator.Current, ct); ++count; @@ -482,8 +281,8 @@ internal static async Task Run( /// Cancellation token that can be used to cancel the operation /// Number of items that have been processed internal static async Task Run( - [NotNull] IFdbAsyncEnumerable source, - FdbAsyncMode mode, + [NotNull] IAsyncEnumerable source, + AsyncIterationHint mode, [NotNull] Func action, CancellationToken ct) { @@ -492,11 +291,11 @@ internal static async Task Run( //note: we should not use "ConfigureAwait(false)" here because we would like to execute the action in the original synchronization context if possible... long count = 0; - using (var iterator = source.GetEnumerator(mode)) + using (var iterator = source.GetEnumerator(ct, mode)) { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - while (await iterator.MoveNextAsync(ct)) + while (await iterator.MoveNextAsync()) { ct.ThrowIfCancellationRequested(); await action(iterator.Current); @@ -513,8 +312,8 @@ internal static async Task Run( /// When the sequence is empty: If true then returns the default value for the type. Otherwise, throws an exception /// Cancellation token that can be used to cancel the operation /// Value of the first element of the sequence, or the default value, or an exception (depending on and - internal static async Task Head( - [NotNull] IFdbAsyncEnumerable source, + public static async Task Head( + [NotNull] IAsyncEnumerable source, bool single, bool orDefault, CancellationToken ct) @@ -523,16 +322,16 @@ internal static async Task Head( //note: we should not use "ConfigureAwait(false)" here because we would like to execute the action in the original synchronization context if possible... - using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.Head)) { Contract.Assert(iterator != null, "The underlying sequence returned a null async iterator"); - if (await iterator.MoveNextAsync(ct)) + if (await iterator.MoveNextAsync()) { TSource first = iterator.Current; if (single) { - if (await iterator.MoveNextAsync(ct)) throw new InvalidOperationException("The sequence contained more than one element"); + if (await iterator.MoveNextAsync()) throw new InvalidOperationException("The sequence contained more than one element"); } return first; } @@ -544,4 +343,208 @@ internal static async Task Head( #endregion } + + /// Small buffer that keeps a list of chunks that are larger and larger + /// Type of elements stored in the buffer + [DebuggerDisplay("Count={Count}, Chunks={this.Chunks.Length}, Current={Index}/{Current.Length}")] + public sealed class Buffer + { + // We want to avoid growing the same array again and again ! + // Instead, we grow list of chunks, that grow in size (until a max), and concatenate all the chunks together at the end, once we know the final size + + /// Default intial capacity, if not specified + const int DefaultCapacity = 16; + //REVIEW: should we use a power of 2 or of 10 for initial capacity? + // Since humans prefer the decimal system, it is more likely that query limit count be set to something like 10, 50, 100 or 1000 + // but most "human friendly" limits are close to the next power of 2, like 10 ~= 16, 50 ~= 64, 100 ~= 128, 500 ~= 512, 1000 ~= 1024, so we don't waste that much space... + + /// Maximum size of a chunk + const int MaxChunkSize = 4096; + + /// Number of items in the buffer + public int Count; + /// Index in the current chunk + public int Index; + /// List of chunks + public T[][] Chunks; + /// Current (and last) chunk + public T[] Current; + + public Buffer(int capacity = 0) + { + if (capacity <= 0) capacity = DefaultCapacity; + + this.Count = 0; + this.Index = 0; + this.Chunks = new T[1][]; + this.Current = new T[capacity]; + this.Chunks[0] = this.Current; + } + + public void Add(T item) + { + if (this.Index == this.Current.Length) + { + Grow(); + } + + checked { ++this.Count; } + this.Current[this.Index++] = item; + } + + private void Grow() + { + // Growth rate: + // - newly created chunk is always half the total size + // - except the first chunk who is set to the inital capacity + + Array.Resize(ref this.Chunks, this.Chunks.Length + 1); + this.Current = new T[Math.Min(this.Count, MaxChunkSize)]; + this.Chunks[this.Chunks.Length - 1] = this.Current; + this.Index = 0; + } + + [NotNull] + private T[] MergeChunks() + { + var tmp = new T[this.Count]; + int count = this.Count; + int index = 0; + for (int i = 0; i < this.Chunks.Length - 1; i++) + { + var chunk = this.Chunks[i]; + Array.Copy(chunk, 0, tmp, index, chunk.Length); + index += chunk.Length; + count -= chunk.Length; + } + Array.Copy(this.Current, 0, tmp, index, count); + return tmp; + } + + /// Return a buffer containing all of the items + /// Buffer that contains all the items, and may be larger than required + /// This is equivalent to calling ToArray(), except that if the buffer is empty, or if it consists of a single page, then no new allocations will be performed. + [NotNull] + public T[] GetBuffer() + { + //note: this is called by internal operator like OrderBy + // In this case we want to reduce the copying as much as possible, + // and we can suppose that the buffer won't be exposed to the application + + if (this.Count == 0) + { // empty + return new T[0]; + } + else if (this.Chunks.Length == 1) + { // everything fits in a single chunk + return this.Current; + } + else + { // we need to stitch all the buffers together + return MergeChunks(); + } + } + + /// Return the content of the buffer + /// Array of size containing all the items in this buffer + [NotNull] + public T[] ToArray() + { + if (this.Count == 0) + { // empty sequence + return new T[0]; + } + else if (this.Chunks.Length == 1 && this.Current.Length == this.Count) + { // a single buffer page was used + return this.Current; + } + else + { // concatenate all the buffer pages into one big array + return MergeChunks(); + } + } + + /// Return the content of the buffer + /// List of size containing all the items in this buffer + [NotNull] + public List ToList() + { + int count = this.Count; + if (count == 0) + { // empty sequence + return new List(); + } + + var list = new List(count); + if (count > 0) + { + var chunks = this.Chunks; + for (int i = 0; i < chunks.Length - 1; i++) + { + list.AddRange(chunks[i]); + count -= chunks[i].Length; + } + + var current = this.Current; + if (count == current.Length) + { // the last chunk fits perfectly + list.AddRange(current); + } + else + { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer + // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum + for (int i = 0; i < count; i++) + { + list.Add(current[i]); + } + } + } + + return list; + } + + /// Return the content of the buffer + /// List of size containing all the items in this buffer + [NotNull] + public HashSet ToHashSet(IEqualityComparer comparer = null) + { + int count = this.Count; + var hashset = new HashSet(comparer); + if (count == 0) + { + return hashset; + } + + var chunks = this.Chunks; + + for (int i = 0; i < chunks.Length - 1; i++) + { + foreach (var item in chunks[i]) + { + hashset.Add(item); + } + count -= chunks[i].Length; + } + + var current = this.Current; + if (count == current.Length) + { // the last chunk fits perfectly + foreach (var item in current) + { + hashset.Add(item); + } + } + else + { // there is no List.AddRange(buffer, offset, count), and copying in a tmp buffer would waste the memory we tried to save with the buffer + // also, for most of the small queries, like FirstOrDefault()/SingleOrDefault(), count will be 1 (or very small) so calling Add(T) will still be optimum + for (int i = 0; i < count; i++) + { + hashset.Add(current[i]); + } + } + return hashset; + } + + } + } diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Linq/AsyncEnumerable.OrderedSequence.cs similarity index 56% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.OrderedSequence.cs index 63888b502..6ba01898d 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.OrderedSequence.cs @@ -1,57 +1,39 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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. - */ +#region Copyright Doxense SAS 2013-2016 +// +// All rights are reserved. Reproduction or transmission in whole or in part, in +// any form or by any means, electronic, mechanical or otherwise, is prohibited +// without the prior written consent of the copyright owner. +// #endregion -namespace FoundationDB.Linq +using Doxense.Async; + +namespace Doxense.Linq { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; + using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { /// Represent an async sequence that returns its elements according to a specific sort order /// Type of the elements of the sequence - internal class OrderedSequence : IFdbAsyncOrderedEnumerable + internal class OrderedSequence : IAsyncOrderedEnumerable { // If an instance of the base class is constructed, it will sort by the items themselves (using a Comparer) // If an instance of the derived class is constructed, then it will sort the a key extracted the each item (sing a Comparer) - protected readonly IFdbAsyncEnumerable m_source; + protected readonly IAsyncEnumerable m_source; private readonly IComparer m_comparer; // null if comparing using keys protected readonly bool m_descending; protected readonly OrderedSequence m_parent;// null if primary sort key - public OrderedSequence(IFdbAsyncEnumerable source, IComparer comparer, bool descending, OrderedSequence parent) + public OrderedSequence(IAsyncEnumerable source, IComparer comparer, bool descending, OrderedSequence parent) { Contract.Requires(source != null); @@ -61,7 +43,7 @@ public OrderedSequence(IFdbAsyncEnumerable source, IComparer c m_parent = parent; } - protected OrderedSequence(IFdbAsyncEnumerable source, bool descending, OrderedSequence parent) + protected OrderedSequence(IAsyncEnumerable source, bool descending, OrderedSequence parent) { Contract.Requires(source != null); @@ -78,31 +60,26 @@ internal virtual SequenceSorter GetEnumerableSorter(SequenceSorter GetEnumerator(FdbAsyncMode mode = FdbAsyncMode.Default) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) { + ct.ThrowIfCancellationRequested(); var sorter = GetEnumerableSorter(null); - var enumerator = default(IFdbAsyncEnumerator); + var enumerator = default(IAsyncEnumerator); try { - enumerator = m_source.GetEnumerator(mode); - return new OrderedEnumerator(enumerator, sorter); + enumerator = m_source.GetEnumerator(ct, mode); + return new OrderedEnumerator(enumerator, sorter, ct); } catch (Exception) { - if (enumerator != null) enumerator.Dispose(); + enumerator?.Dispose(); throw; } } - IAsyncEnumerator IAsyncEnumerable.GetEnumerator() + public IAsyncOrderedEnumerable CreateOrderedEnumerable(Func keySelector, IComparer comparer, bool descending) { - return GetEnumerator(FdbAsyncMode.All); - } - - [NotNull] - public IFdbAsyncOrderedEnumerable CreateOrderedEnumerable([NotNull] Func keySelector, IComparer comparer, bool descending) - { - if (keySelector == null) throw new ArgumentNullException("keySelector"); + Contract.NotNull(keySelector, nameof(keySelector)); return new OrderedSequence(this, keySelector, comparer, descending, this); } @@ -117,7 +94,7 @@ internal sealed class OrderedSequence : OrderedSequence private readonly Func m_keySelector; private readonly IComparer m_keyComparer; - public OrderedSequence(IFdbAsyncEnumerable source, Func keySelector, IComparer comparer, bool descending, OrderedSequence parent) + public OrderedSequence(IAsyncEnumerable source, Func keySelector, IComparer comparer, bool descending, OrderedSequence parent) : base(source, descending, parent) { Contract.Requires(keySelector != null); @@ -135,43 +112,44 @@ internal override SequenceSorter GetEnumerableSorter(SequenceSorterIterator that will sort all the items produced by an inner iterator, before outputting the results all at once - internal sealed class OrderedEnumerator : IFdbAsyncEnumerator + internal sealed class OrderedEnumerator : IAsyncEnumerator { // This iterator must first before EVERY items of the source in memory, before being able to sort them. // The first MoveNext() will return only once the inner sequence has finished (succesfully), which can take some time! // Ordering is done in-memory using QuickSort - private readonly IFdbAsyncEnumerator m_inner; + private readonly IAsyncEnumerator m_inner; private readonly SequenceSorter m_sorter; private TSource[] m_items; private int[] m_map; private int m_offset; private TSource m_current; + private readonly CancellationToken m_ct; - public OrderedEnumerator(IFdbAsyncEnumerator enumerator, SequenceSorter sorter) + public OrderedEnumerator(IAsyncEnumerator enumerator, SequenceSorter sorter, CancellationToken ct) { Contract.Requires(enumerator != null && sorter != null); m_inner = enumerator; m_sorter = sorter; + m_ct = ct; } - private async Task ReadAllThenSort(CancellationToken ct) + private async Task ReadAllThenSort() { if (m_offset == -1) return false; // already EOF or Disposed // first we need to spool everything from the inner iterator into memory - var buffer = new FdbAsyncEnumerable.Buffer(); + var buffer = new Buffer(); var inner = m_inner; - var iterator = inner as FdbAsyncIterator; - if (iterator != null) + if (inner is AsyncIterator iterator) { - await iterator.ExecuteAsync((x) => buffer.Add(x), ct).ConfigureAwait(false); + await iterator.ExecuteAsync((x) => buffer.Add(x), m_ct).ConfigureAwait(false); } else { - while (await inner.MoveNextAsync(ct).ConfigureAwait(false)) + while (await inner.MoveNextAsync().ConfigureAwait(false)) { buffer.Add(inner.Current); } @@ -193,25 +171,25 @@ private async Task ReadAllThenSort(CancellationToken ct) return true; } - public Task MoveNextAsync(CancellationToken ct) + public Task MoveNextAsync() { // Firt call will be slow (and async), but the rest of the calls will use the results already sorted in memory, and should be as fast as possible! if (m_map == null) { - return ReadAllThenSort(ct); + return ReadAllThenSort(); } int pos = checked(m_offset + 1); if (pos < m_map.Length) { Publish(pos); - return TaskHelpers.TrueTask; + return TaskHelpers.True; } else { Completed(); - return TaskHelpers.FalseTask; + return TaskHelpers.False; } } @@ -231,10 +209,7 @@ private void Completed() } - public TSource Current - { - get { return m_current; } - } + public TSource Current => m_current; public void Dispose() { diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs b/FoundationDB.Client/Linq/AsyncEnumerable.Sorters.cs similarity index 97% rename from FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs rename to FoundationDB.Client/Linq/AsyncEnumerable.Sorters.cs index a79f810b7..58d79beb9 100644 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.Sorters.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.Sorters.cs @@ -26,14 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; - public static partial class FdbAsyncEnumerable + public static partial class AsyncEnumerable { // These classes contain the logic to sort items (by themselves or by keys) // They are single-use and constructed at runtime, when an ordered sequence starts enumerating. @@ -171,10 +171,7 @@ internal override void ComputeKeys(TSource[] items, int count) keys[i] = selector(items[i]); } m_keys = keys; - if (m_next != null) - { - m_next.ComputeKeys(items, count); - } + m_next?.ComputeKeys(items, count); } internal override int CompareKeys(int index1, int index2) diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.cs b/FoundationDB.Client/Linq/AsyncEnumerable.cs new file mode 100644 index 000000000..738724e6c --- /dev/null +++ b/FoundationDB.Client/Linq/AsyncEnumerable.cs @@ -0,0 +1,1445 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using Doxense.Async; +using FoundationDB; + +namespace Doxense.Linq +{ + using System; + using System.Collections.Generic; + using System.Threading; + using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Expressions; + using Doxense.Linq.Async.Iterators; + using FoundationDB.Client; + using JetBrains.Annotations; + + /// Provides a set of static methods for querying objects that implement . + public static partial class AsyncEnumerable + { + // Welcome to the wonderful world of the Monads! + + #region Entering the Monad... + + /// Returns an empty async sequence + [Pure, NotNull] + public static IAsyncEnumerable Empty() + { + return EmptySequence.Default; + } + + /// Returns an async sequence with a single element, which is a constant + [Pure, NotNull] + public static IAsyncEnumerable Singleton(T value) + { + //note: we can't call this method Single(T), because then Single(Func) would be ambigous with Single>(T) + return new SingletonSequence(() => value); + } + + /// Returns an async sequence which will produce a single element, using the specified lambda + /// Lambda that will be called once per iteration, to produce the single element of this sequene + /// If the sequence is iterated multiple times, then will be called once for each iteration. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Single([NotNull] Func lambda) + { + Contract.NotNull(lambda, nameof(lambda)); + return new SingletonSequence(lambda); + } + + /// Returns an async sequence which will produce a single element, using the specified lambda + /// Lambda that will be called once per iteration, to produce the single element of this sequene + /// If the sequence is iterated multiple times, then will be called once for each iteration. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Single([NotNull] Func> asyncLambda) + { + Contract.NotNull(asyncLambda, nameof(asyncLambda)); + return new SingletonSequence(asyncLambda); + } + + /// Returns an async sequence which will produce a single element, using the specified lambda + /// Lambda that will be called once per iteration, to produce the single element of this sequene + /// If the sequence is iterated multiple times, then will be called once for each iteration. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Single([NotNull] Func> asyncLambda) + { + Contract.NotNull(asyncLambda, nameof(asyncLambda)); + return new SingletonSequence(asyncLambda); + } + + /// Apply an async lambda to a sequence of elements to transform it into an async sequence + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnumerable source, [NotNull] Func> lambda) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(lambda, nameof(lambda)); + + return Create(source, (iterator, ct) => new EnumerableIterator(iterator, lambda, ct)); + } + + /// Apply an async lambda to a sequence of elements to transform it into an async sequence + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnumerable source) + { + Contract.NotNull(source, nameof(source)); + + return Create(source, (iterator, ct) => new EnumerableIterator(iterator, x => Task.FromResult(x), ct)); + } + + /// Wraps an async lambda into an async sequence that will return the result of the lambda + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable FromTask([NotNull] Func> asyncLambda) + { + //TODO: create a custom iterator for this ? + return ToAsyncEnumerable(new [] { asyncLambda }).Select(x => x()); + } + + /// Split a sequence of items into several batches + /// Type of the elemenst in + /// Source sequence + /// Maximum size of each batch + /// Sequence of batches, whose size will always we , except for the last batch that will only hold the remaning items. If the source is empty, an empty sequence is returned. + [Pure, NotNull, LinqTunnel] + public static IEnumerable> Buffered([NotNull] this IEnumerable source, int batchSize) + { + Contract.NotNull(source, nameof(source)); + if (batchSize <= 0) throw new ArgumentException("Batch size must be greater than zero.", nameof(batchSize)); + + var list = new List(batchSize); + foreach (var item in source) + { + list.Add(item); + if (list.Count >= batchSize) + { + yield return list; + list.Clear(); + } + } + if (list.Count > 0) + { + yield return list; + } + } + + #endregion + + #region Staying in the Monad... + + #region SelectMany... + + /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func> selector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + if (source is AsyncIterator iterator) + { + return iterator.SelectMany(selector); + } + + return Flatten(source, new AsyncTransformExpression>(selector)); + } + + /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func>> asyncSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); + + return SelectMany(source, TaskHelpers.WithCancellation(asyncSelector)); + } + + /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func>> asyncSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); + + if (source is AsyncIterator iterator) + { + return iterator.SelectMany(asyncSelector); + } + + return Flatten(source, new AsyncTransformExpression>(asyncSelector)); + } + + /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func> collectionSelector, [NotNull] Func resultSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(collectionSelector, nameof(collectionSelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); + + if (source is AsyncIterator iterator) + { + return iterator.SelectMany(collectionSelector, resultSelector); + } + + return Flatten(source, new AsyncTransformExpression>(collectionSelector), resultSelector); + } + + /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncCollectionSelector, nameof(asyncCollectionSelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); + + return SelectMany(source, TaskHelpers.WithCancellation(asyncCollectionSelector), resultSelector); + } + + /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectMany([NotNull] this IAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncCollectionSelector, nameof(asyncCollectionSelector)); + Contract.NotNull(resultSelector, nameof(resultSelector)); + + if (source is AsyncIterator iterator) + { + return iterator.SelectMany(asyncCollectionSelector, resultSelector); + } + + return Flatten(source, new AsyncTransformExpression>(asyncCollectionSelector), resultSelector); + } + + #endregion + + #region Select... + + /// Projects each element of an async sequence into a new form. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Select([NotNull] this IAsyncEnumerable source, [NotNull] Func selector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + if (source is AsyncIterator iterator) + { + return iterator.Select(selector); + } + + return Map(source, new AsyncTransformExpression(selector)); + } + + /// Projects each element of an async sequence into a new form. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Select([NotNull] this IAsyncEnumerable source, [NotNull] Func> asyncSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); + + return Select(source, TaskHelpers.WithCancellation(asyncSelector)); + } + + /// Projects each element of an async sequence into a new form. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Select([NotNull] this IAsyncEnumerable source, [NotNull] Func> asyncSelector) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); + + if (source is AsyncIterator iterator) + { + return iterator.Select(asyncSelector); + } + + return Map(source, new AsyncTransformExpression(asyncSelector)); + } + + #endregion + + #region Where... + + /// Filters an async sequence of values based on a predicate. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Where([NotNull] this IAsyncEnumerable source, [NotNull] Func predicate) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + + if (source is AsyncIterator iterator) + { + return iterator.Where(predicate); + } + + return Filter(source, new AsyncFilterExpression(predicate)); + } + + /// Filters an async sequence of values based on a predicate. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Where([NotNull] this IAsyncEnumerable source, [NotNull] Func> asyncPredicate) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncPredicate, nameof(asyncPredicate)); + + return Where(source, TaskHelpers.WithCancellation(asyncPredicate)); + } + + /// Filters an async sequence of values based on a predicate. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Where([NotNull] this IAsyncEnumerable source, [NotNull] Func> asyncPredicate) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncPredicate, nameof(asyncPredicate)); + + if (source is AsyncIterator iterator) + { + return iterator.Where(asyncPredicate); + } + + return Filter(source, new AsyncFilterExpression(asyncPredicate)); + } + + #endregion + + #region Take... + + /// Returns a specified number of contiguous elements from the start of an async sequence. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Take([NotNull] this IAsyncEnumerable source, int count) + { + Contract.NotNull(source, nameof(source)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); + + if (source is AsyncIterator iterator) + { + return iterator.Take(count); + } + + return Limit(source, count); + } + + #endregion + + #region TakeWhile... + + /// Returns elements from an async sequence as long as a specified condition is true, and then skips the remaining elements. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable TakeWhile([NotNull] this IAsyncEnumerable source, [NotNull] Func condition) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(condition, nameof(condition)); + + if (source is AsyncIterator iterator) + { + return iterator.TakeWhile(condition); + } + + return Limit(source, condition); + } + + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable TakeWhile([NotNull] this IAsyncEnumerable source, [NotNull] Func condition, out QueryStatistics stopped) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(condition, nameof(condition)); + + var signal = new QueryStatistics(false); + stopped = signal; + + // to trigger the signal, we just intercept the condition returning false (which only happen once!) + bool Wrapped(TSource x) + { + if (condition(x)) return true; + signal.Update(true); + return false; + } + + return TakeWhile(source, Wrapped); + } + + #endregion + + #region Skip... + + /// Skips the first elements of an async sequence. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Skip([NotNull] this IAsyncEnumerable source, int count) + { + Contract.NotNull(source, nameof(source)); + if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); + + if (source is AsyncIterator iterator) + { + return iterator.Skip(count); + } + + return Offset(source, count); + } + + #endregion + + #region SelectAsync + + /// Projects each element of an async sequence into a new form. + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable SelectAsync([NotNull] this IAsyncEnumerable source, [NotNull] Func> asyncSelector, ParallelAsyncQueryOptions options = null) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncSelector, nameof(asyncSelector)); + + return new ParallelSelectAsyncIterator(source, asyncSelector, options ?? new ParallelAsyncQueryOptions()); + } + + /// Always prefetch the next item from the inner sequence. + /// Type of the items in the source sequence + /// Source sequence that has a high latency, and from which we want to prefetch a set number of items. + /// Sequence that prefetch the next item, when outputing the current item. + /// + /// This iterator can help smooth out the query pipeline when every call to the inner sequence has a somewhat high latency (ex: reading the next page of results from the database). + /// Avoid prefetching from a source that is already reading from a buffer of results. + /// + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Prefetch([NotNull] this IAsyncEnumerable source) + { + Contract.NotNull(source, nameof(source)); + + return new PrefetchingAsyncIterator(source, 1); + } + + /// Prefetch a certain number of items from the inner sequence, before outputing the results one by one. + /// Type of the items in the source sequence + /// Source sequence that has a high latency, and from which we want to prefetch a set number of items. + /// Maximum number of items to buffer from the source before they are consumed by the rest of the query. + /// Sequence that returns items from a buffer of prefetched list. + /// + /// This iterator can help smooth out the query pipeline when every call to the inner sequence has a somewhat high latency (ex: reading the next page of results from the database). + /// Avoid prefetching from a source that is already reading from a buffer of results. + /// + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Prefetch([NotNull] this IAsyncEnumerable source, int prefetchCount) + { + Contract.NotNull(source, nameof(source)); + if (prefetchCount <= 0) throw new ArgumentOutOfRangeException(nameof(prefetchCount), prefetchCount, "Prefetch count must be at least one."); + + return new PrefetchingAsyncIterator(source, prefetchCount); + } + + /// Buffers the items of a bursty sequence, into a sequence of variable-sized arrays made up of items that where produced in a very short timespan. + /// Type of the items in the source sequence + /// Source sequence, that produces bursts of items, produced from the same page of results, before reading the next page. + /// Maximum number of items to return in a single window. If more items arrive at the same time, a new window will be opened with the rest of the items. + /// Sequence of batches, where all the items of a single batch arrived at the same time. A batch is closed once the next call to MoveNext() on the inner sequence does not complete immediately. Batches can be smaller than . + /// + /// This should only be called on bursty asynchronous sequences, and when you want to process items in batches, without incurring the cost of latency between two pages of results. + /// You should avoid using this operator on sequences where each call to MoveNext() is asynchronous, since it would only produce batchs with only a single item. + /// + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Window([NotNull] this IAsyncEnumerable source, int maxWindowSize) + { + Contract.NotNull(source, nameof(source)); + if (maxWindowSize <= 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(maxWindowSize), maxWindowSize, "Window size must be at least one."); + + return new WindowingAsyncIterator(source, maxWindowSize); + } + + /// Buffers the items of a source sequence, and outputs a sequence of fixed-sized arrays. + /// Type of the items in the source sequence + /// Source sequence that will be cut into chunks containing at most items. + /// Number of items per batch. The last batch may contain less items, but should never be empty. + /// Sequence of arrays of size , except the last batch which can have less items. + /// + /// This operator does not care about the latency of each item, and will always try to fill each batch completely, before outputing a result. + /// If you are working on an inner sequence that is bursty in nature, where items arrives in waves, you should use which attempts to minimize the latency by outputing incomplete batches if needed. + /// + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Batch([NotNull] this IAsyncEnumerable source, int batchSize) + { + Contract.NotNull(source, nameof(source)); + if (batchSize <= 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(batchSize), batchSize, "Batch size must be at least one."); + + return new BatchingAsyncIterator(source, batchSize); + } + + #endregion + + #region Distinct... + + [Pure, NotNull, LinqTunnel] + public static IAsyncEnumerable Distinct([NotNull] this IAsyncEnumerable source, IEqualityComparer comparer = null) + { + Contract.NotNull(source, nameof(source)); + comparer = comparer ?? EqualityComparer.Default; + + return new DistinctAsyncIterator(source, comparer); + } + + #endregion + + #region OrderBy... + + [Pure, NotNull, LinqTunnel] + public static IAsyncOrderedEnumerable OrderBy([NotNull] this IAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(keySelector, nameof(keySelector)); + comparer = comparer ?? Comparer.Default; + + return new OrderedSequence(source, keySelector, comparer, descending: false, parent: null); + } + + [Pure, NotNull, LinqTunnel] + public static IAsyncOrderedEnumerable OrderByDescending([NotNull] this IAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(keySelector, nameof(keySelector)); + comparer = comparer ?? Comparer.Default; + + return new OrderedSequence(source, keySelector, comparer, descending: true, parent: null); + } + + [Pure, NotNull, LinqTunnel] + public static IAsyncOrderedEnumerable ThenBy([NotNull] this IAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) + { + Contract.NotNull(source, nameof(source)); + return source.CreateOrderedEnumerable(keySelector, comparer, descending: false); + } + + [Pure, NotNull, LinqTunnel] + public static IAsyncOrderedEnumerable ThenByDescending([NotNull] this IAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) + { + Contract.NotNull(source, nameof(source)); + return source.CreateOrderedEnumerable(keySelector, comparer, descending: true); + } + + #endregion + + // If you are bored, maybe consider adding: + // - DefaultIfEmpty + // - Zip + // - OrderBy and OrderBy + // - GroupBy + + #endregion + + #region Leaving the Monad... + + /// Execute an action for each element of an async sequence + public static Task ForEachAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Action action, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(action, nameof(action)); + + if (source is AsyncIterator iterator) + { + return iterator.ExecuteAsync(action, ct); + } + return Run(source, AsyncIterationHint.All, action, ct); + } + + /// Execute an async action for each element of an async sequence + public static Task ForEachAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(asyncAction, nameof(asyncAction)); + + if (source is AsyncIterator iterator) + { + return iterator.ExecuteAsync(TaskHelpers.WithCancellation(asyncAction), ct); + } + + return ForEachAsync(source, TaskHelpers.WithCancellation(asyncAction), ct); + } + + /// Execute an async action for each element of an async sequence + public static Task ForEachAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncAction, nameof(asyncAction)); + + if (source is AsyncIterator iterator) + { + return iterator.ExecuteAsync(asyncAction, ct); + } + + return Run(source, AsyncIterationHint.All, asyncAction, ct); + } + + #region ToList/Array/Dictionary/HashSet... + + /// Create a list from an async sequence. + [ItemNotNull] + public static Task> ToListAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync( + source, + new Buffer(), + (buffer, x) => buffer.Add(x), + (buffer) => buffer.ToList(), + ct + ); + } + + /// Create an array from an async sequence. + [ItemNotNull] + public static Task ToArrayAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync( + source, + new Buffer(), + (buffer, x) => buffer.Add(x), + (buffer) => buffer.ToArray(), + ct + ); + } + + /// Create an array from an async sequence, knowing a rough estimation of the number of elements. + [ItemNotNull] + internal static Task ToArrayAsync([NotNull] this IAsyncEnumerable source, int estimatedSize, CancellationToken ct = default(CancellationToken)) + { + Contract.Requires(source != null && estimatedSize >= 0); + + return AggregateAsync( + source, + new List(estimatedSize), + (buffer, x) => buffer.Add(x), + (buffer) => buffer.ToArray(), + ct + ); + } + + /// Creates a Dictionary from an async sequence according to a specified key selector function and key comparer. + [ItemNotNull] + public static Task> ToDictionaryAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(keySelector, nameof(keySelector)); + + return AggregateAsync( + source, + new Dictionary(comparer ?? EqualityComparer.Default), + (results, x) => { results[keySelector(x)] = x; }, + ct + ); + } + + /// Creates a Dictionary from an async sequence according to a specified key selector function, a comparer, and an element selector function. + [ItemNotNull] + public static Task> ToDictionaryAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, [NotNull, InstantHandle] Func elementSelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(keySelector, nameof(keySelector)); + Contract.NotNull(elementSelector, nameof(elementSelector)); + + return AggregateAsync( + source, + new Dictionary(comparer ?? EqualityComparer.Default), + (results, x) => { results[keySelector(x)] = elementSelector(x); }, + ct + ); + } + + /// Creates a Dictionary from an async sequence of pairs of keys and values. + [ItemNotNull] + public static Task> ToDictionaryAsync([NotNull] this IAsyncEnumerable> source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + return AggregateAsync( + source, + new Dictionary(comparer ?? EqualityComparer.Default), + (results, x) => { results[x.Key] = x.Value; }, + ct + ); + } + + /// Create an Hashset from an async sequence. + [ItemNotNull] + public static Task> ToHashSetAsync([NotNull] this IAsyncEnumerable source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + return AggregateAsync( + source, + new Buffer(), + (buffer, x) => buffer.Add(x), + (buffer) => buffer.ToHashSet(comparer), + ct + ); + } + + #endregion + + #region Aggregate... + + /// Applies an accumulator function over an async sequence. + public static async Task AggregateAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(aggregator, nameof(aggregator)); + + ct.ThrowIfCancellationRequested(); + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.All)) + { + Contract.Assert(iterator != null, "The sequence returned a null async iterator"); + + if (!(await iterator.MoveNextAsync().ConfigureAwait(false))) + { + throw new InvalidOperationException("The sequence was empty"); + } + + var item = iterator.Current; + while (await iterator.MoveNextAsync().ConfigureAwait(false)) + { + item = aggregator(item, iterator.Current); + } + + return item; + } + } + + /// Applies an accumulator function over an async sequence. + public static async Task AggregateAsync([NotNull] this IAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(aggregator, nameof(aggregator)); + + //TODO: opitmize this to not have to allocate lambdas! + var accumulate = seed; + await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); + return accumulate; + } + + /// Applies an accumulator function over an async sequence. + public static async Task AggregateAsync([NotNull] this IAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(aggregator, nameof(aggregator)); + + var accumulate = seed; + await ForEachAsync(source, (x) => { aggregator(accumulate, x); }, ct).ConfigureAwait(false); + return accumulate; + } + + /// Applies an accumulator function over an async sequence. + public static async Task AggregateAsync([NotNull] this IAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(aggregator, nameof(aggregator)); + Contract.NotNull(resultSelector, nameof(resultSelector)); + + var accumulate = seed; + await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); + return resultSelector(accumulate); + } + + /// Applies an accumulator function over an async sequence. + public static async Task AggregateAsync([NotNull] this IAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(aggregator, nameof(aggregator)); + Contract.NotNull(resultSelector, nameof(resultSelector)); + + var accumulate = seed; + await ForEachAsync(source, (x) => aggregator(accumulate, x), ct); + return resultSelector(accumulate); + } + + #endregion + + #region First/Last/Single... + + /// Returns the first element of an async sequence, or an exception if it is empty + public static Task FirstAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.FirstAsync(); + + return Head(source, single: false, orDefault: false, ct: ct); + } + + /// Returns the first element of an async sequence, or an exception if it is empty + public static Task FirstAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.FirstAsync(); + + //TODO: PERF: custom implementation for this? + return Head(source.Where(predicate), single: false, orDefault: false, ct: ct); + } + + /// Returns the first element of an async sequence, or the default value for the type if it is empty + public static Task FirstOrDefaultAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.FirstOrDefaultAsync(); + + return Head(source, single: false, orDefault: true, ct: ct); + } + + /// Returns the first element of an async sequence, or the default value for the type if it is empty + public static Task FirstOrDefaultAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.FirstOrDefaultAsync(); + + //TODO: PERF: custom implementation for this? + return Head(source.Where(predicate), single: false, orDefault: true, ct: ct); + } + + /// Returns the first and only element of an async sequence, or an exception if it is empty or have two or more elements + /// Will need to call MoveNext at least twice to ensure that there is no second element. + public static Task SingleAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.SingleAsync(); + + return Head(source, single: true, orDefault: false, ct: ct); + } + + /// Returns the first and only element of an async sequence, or an exception if it is empty or have two or more elements + /// Will need to call MoveNext at least twice to ensure that there is no second element. + public static Task SingleAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.SingleAsync(); + + //TODO: PERF: custom implementation for this? + return Head(source.Where(predicate), single: true, orDefault: false, ct: ct); + } + + /// Returns the first and only element of an async sequence, the default value for the type if it is empty, or an exception if it has two or more elements + /// Will need to call MoveNext at least twice to ensure that there is no second element. + public static Task SingleOrDefaultAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.SingleOrDefaultAsync(); + + return Head(source, single: true, orDefault: true, ct: ct); + } + + /// Returns the first and only element of an async sequence, the default value for the type if it is empty, or an exception if it has two or more elements + /// Will need to call MoveNext at least twice to ensure that there is no second element. + public static Task SingleOrDefaultAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return rq.SingleOrDefaultAsync(); + + return Head(source.Where(predicate), single: true, orDefault: true, ct: ct); + } + + /// Returns the last element of an async sequence, or an exception if it is empty + public static async Task LastAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.LastAsync(); + + bool found = false; + T last = default(T); + + await ForEachAsync(source, (x) => { found = true; last = x; }, ct).ConfigureAwait(false); + + if (!found) throw new InvalidOperationException("The sequence was empty"); + return last; + } + + /// Returns the last element of an async sequence, or an exception if it is empty + public static async Task LastAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.LastAsync(); + + bool found = false; + T last = default(T); + + await ForEachAsync(source, (x) => { if (predicate(x)) { found = true; last = x; } }, ct).ConfigureAwait(false); + + if (!found) throw new InvalidOperationException("The sequence was empty"); + return last; + } + + /// Returns the last element of an async sequence, or the default value for the type if it is empty + public static async Task LastOrDefaultAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.LastOrDefaultAsync(); + + bool found = false; + T last = default(T); + + await ForEachAsync(source, (x) => { found = true; last = x; }, ct).ConfigureAwait(false); + + return found ? last : default(T); + } + + /// Returns the last element of an async sequence, or the default value for the type if it is empty + public static async Task LastOrDefaultAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.LastOrDefaultAsync(); + + bool found = false; + T last = default(T); + + await ForEachAsync(source, (x) => { if (predicate(x)) { found = true; last = x; } }, ct).ConfigureAwait(false); + + return found ? last : default(T); + } + + /// Returns the element at a specific location of an async sequence, or an exception if there are not enough elements + public static async Task ElementAtAsync([NotNull] this IAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.Skip(index).SingleAsync(); + + int counter = index; + T item = default(T); + await Run( + source, + AsyncIterationHint.All, + (x) => + { + if (counter-- == 0) { item = x; return false; } + return true; + }, + ct + ).ConfigureAwait(false); + + if (counter >= 0) throw new InvalidOperationException("The sequence was too small"); + return item; + } + + /// Returns the element at a specific location of an async sequence, or the default value for the type if it there are not enough elements + public static async Task ElementAtOrDefaultAsync([NotNull] this IAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); + ct.ThrowIfCancellationRequested(); + + //TODO:REFACTORING: create some interface or base class for this? + //var rq = source as FdbRangeQuery; + //if (rq != null) return await rq.Skip(index).SingleAsync(); + + int counter = index; + T item = default(T); + + //TODO: use ExecuteAsync() if the source is an Iterator! + await Run( + source, + AsyncIterationHint.All, + (x) => + { + if (counter-- == 0) { item = x; return false; } + return true; + }, + ct + ).ConfigureAwait(false); + + if (counter >= 0) return default(T); + return item; + } + + #endregion + + #region Count/Sum... + + /// Returns the number of elements in an async sequence. + public static async Task CountAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + int count = 0; + + await ForEachAsync(source, (_) => { ++count; }, ct).ConfigureAwait(false); + + return count; + } + + /// Returns a number that represents how many elements in the specified async sequence satisfy a condition. + public static async Task CountAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + int count = 0; + + await ForEachAsync(source, (x) => { if (predicate(x)) ++count; }, ct).ConfigureAwait(false); + + return count; + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0U, (sum, x) => checked(sum + x), ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0U, (sum, x) => checked(sum + selector(x)), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0UL, (sum, x) => checked(sum + x), ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0UL, (sum, x) => checked(sum + selector(x)), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0, (sum, x) => checked(sum + x), ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0, (sum, x) => checked(sum + selector(x)), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0L, (sum, x) => checked(sum + x), ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0L, (sum, x) => checked(sum + selector(x)), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0.0f, (sum, x) => sum + x, ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0.0f, (sum, x) => sum + selector(x), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0.0, (sum, x) => sum + x, ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0.0, (sum, x) => sum + selector(x), ct); + } + + /// Returns the sum of all elements in the specified async sequence. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + + return AggregateAsync(source, 0m, (sum, x) => sum + x, ct); + } + + /// Returns the sum of all elements in the specified async sequence that satisfy a condition. + public static Task SumAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func selector, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(selector, nameof(selector)); + + return AggregateAsync(source, 0m, (sum, x) => sum + selector(x), ct); + } + + #endregion + + #region Min/Max... + + /// Returns the smallest value in the specified async sequence + public static async Task MinAsync([NotNull] this IAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + comparer = comparer ?? Comparer.Default; + + //REVIEW: use C#7 tuples + bool found = false; + T min = default(T); + + await ForEachAsync( + source, + (x) => + { + if (!found || comparer.Compare(x, min) < 0) + { + min = x; + found = true; + } + }, + ct + ).ConfigureAwait(false); + + if (!found) throw ThrowHelper.InvalidOperationException("The sequence was empty"); + return min; + } + + /// Returns the smallest value in the specified async sequence + public static async Task MinAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + comparer = comparer ?? Comparer.Default; + + //REVIEW: use C#7 tuples + bool found = false; + T min = default(T); + + await ForEachAsync( + source, + (x) => + { + if (predicate(x) && (!found || comparer.Compare(x, min) < 0)) + { + min = x; + found = true; + } + }, + ct + ).ConfigureAwait(false); + + if (!found) throw ThrowHelper.InvalidOperationException("The sequence was empty"); + return min; + } + + /// Returns the largest value in the specified async sequence + public static async Task MaxAsync([NotNull] this IAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + comparer = comparer ?? Comparer.Default; + + //REVIEW: use C#7 tuples + bool found = false; + T max = default(T); + + await ForEachAsync( + source, + (x) => + { + if (!found || comparer.Compare(x, max) > 0) + { + max = x; + found = true; + } + }, + ct + ).ConfigureAwait(false); + + if (!found) throw ThrowHelper.InvalidOperationException("The sequence was empty"); + return max; + } + + /// Returns the largest value in the specified async sequence + public static async Task MaxAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + comparer = comparer ?? Comparer.Default; + + //REVIEW: use C#7 tuples + bool found = false; + T max = default(T); + + await ForEachAsync( + source, + (x) => + { + if (predicate(x) && (!found || comparer.Compare(x, max) > 0)) + { + max = x; + found = true; + } + }, + ct + ).ConfigureAwait(false); + + if (!found) throw ThrowHelper.InvalidOperationException("The sequence was empty"); + return max; + } + + #endregion + + #region Any/None... + + /// Determines whether an async sequence contains any elements. + /// This is the logical equivalent to "source.Count() > 0" but can be better optimized by some providers + public static async Task AnyAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.Head)) + { + return await iterator.MoveNextAsync().ConfigureAwait(false); + } + } + + /// Determines whether any element of an async sequence satisfies a condition. + public static async Task AnyAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.Head)) + { + while (await iterator.MoveNextAsync().ConfigureAwait(false)) + { + if (predicate(iterator.Current)) return true; + } + } + return false; + } + + /// Determines wether an async sequence contains no elements at all. + /// This is the logical equivalent to "source.Count() == 0" or "!source.Any()" but can be better optimized by some providers + public static async Task NoneAsync([NotNull] this IAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + ct.ThrowIfCancellationRequested(); + + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.Head)) + { + return !(await iterator.MoveNextAsync().ConfigureAwait(false)); + } + } + + /// Determines whether none of the elements of an async sequence satisfies a condition. + public static async Task NoneAsync([NotNull] this IAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + ct.ThrowIfCancellationRequested(); + + using (var iterator = source.GetEnumerator(ct, AsyncIterationHint.Head)) + { + while (await iterator.MoveNextAsync().ConfigureAwait(false)) + { + if (predicate(iterator.Current)) return false; + } + } + return true; + } + + #endregion + + #endregion + + #region Query Statistics... + + //TODO: move this somewhere else? + + /// Measure the number of items that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IAsyncEnumerable WithCountStatistics([NotNull] this IAsyncEnumerable source, out QueryStatistics counter) + { + Contract.NotNull(source, nameof(source)); + + var signal = new QueryStatistics(0); + counter = signal; + + // to count, we just increment the signal each type a value flows through here + return Select(source, (x) => + { + signal.Update(checked(signal.Value + 1)); + return x; + }); + } + + /// Measure the number and size of slices that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IAsyncEnumerable> WithSizeStatistics([NotNull] this IAsyncEnumerable> source, out QueryStatistics statistics) + { + Contract.NotNull(source, nameof(source)); + + var data = new KeyValueSizeStatistics(); + statistics = new QueryStatistics(data); + + // to count, we just increment the signal each type a value flows through here + return Select(source,(kvp) => + { + data.Add(kvp.Key.Count, kvp.Value.Count); + return kvp; + }); + } + + /// Measure the number and sizes of the keys and values that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IAsyncEnumerable WithSizeStatistics([NotNull] this IAsyncEnumerable source, out QueryStatistics statistics) + { + Contract.NotNull(source, nameof(source)); + + var data = new DataSizeStatistics(); + statistics = new QueryStatistics(data); + + // to count, we just increment the signal each type a value flows through here + return Select(source, (x) => + { + data.Add(x.Count); + return x; + }); + } + + /// Execute an action on each item passing through the sequence, without modifying the original sequence + /// The is execute inline before passing the item down the line, and should not block + [NotNull, LinqTunnel] + public static IAsyncEnumerable Observe([NotNull] this IAsyncEnumerable source, [NotNull] Action handler) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(handler, nameof(handler)); + + return new ObserverAsyncIterator(source, new AsyncObserverExpression(handler)); + } + + /// Execute an action on each item passing through the sequence, without modifying the original sequence + /// The is execute inline before passing the item down the line, and should not block + [NotNull, LinqTunnel] + public static IAsyncEnumerable Observe([NotNull] this IAsyncEnumerable source, [NotNull] Func asyncHandler) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(asyncHandler, nameof(asyncHandler)); + + return new ObserverAsyncIterator(source, new AsyncObserverExpression(asyncHandler)); + } + + #endregion + + } +} diff --git a/FoundationDB.Client/Linq/FdbAsyncMode.cs b/FoundationDB.Client/Linq/AsyncIterationHint.cs similarity index 97% rename from FoundationDB.Client/Linq/FdbAsyncMode.cs rename to FoundationDB.Client/Linq/AsyncIterationHint.cs index 69765f3b9..555269b8f 100644 --- a/FoundationDB.Client/Linq/FdbAsyncMode.cs +++ b/FoundationDB.Client/Linq/AsyncIterationHint.cs @@ -26,13 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { /// /// Defines the intent of a consumer of an async iterator /// - public enum FdbAsyncMode + public enum AsyncIterationHint { /// /// Use the default settings. The provider will make no attempt at optimizing the query. @@ -42,7 +42,6 @@ public enum FdbAsyncMode /// /// The query will be consumed by chunks and may be aborted at any point. The provider will produce small chunks of data for the first few reads but should still be efficient if the caller consume all the sequence. /// - /// Iterator, /// @@ -62,4 +61,4 @@ public enum FdbAsyncMode } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Linq/EnumerableExtensions.cs b/FoundationDB.Client/Linq/EnumerableExtensions.cs new file mode 100644 index 000000000..63faab43b --- /dev/null +++ b/FoundationDB.Client/Linq/EnumerableExtensions.cs @@ -0,0 +1,142 @@ +#region BSD Licence +/* Copyright (c) 2013, Doxense SARL +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 + + namespace Doxense.Linq +{ + using System; + using System.Collections.Generic; + using System.Linq; + using Doxense.Diagnostics.Contracts; + using Doxense.Linq.Async.Iterators; + using FoundationDB; + using JetBrains.Annotations; + + /// Provides a set of static methods for querying objects that implement . + public static class EnumerableExtensions + { + + //TODO: ajouter Batch(...) ? + //TODO: peut-être merger avec les CollectionExtensions.cs? + + /// Determines wether a sequence contains no elements at all. + /// This is the logical equivalent to "source.Count() == 0" or "!source.Any()" but can be better optimized by some providers + public static bool None([NotNull, InstantHandle] this IEnumerable source) + { + Contract.NotNull(source, nameof(source)); + + using (var iterator = source.GetEnumerator()) + { + return !iterator.MoveNext(); + } + } + + /// Determines whether none of the elements of a sequence satisfies a condition. + public static bool None([NotNull, InstantHandle] this IEnumerable source, [NotNull, InstantHandle] Func predicate) + { + Contract.NotNull(source, nameof(source)); + Contract.NotNull(predicate, nameof(predicate)); + + using (var iterator = source.GetEnumerator()) + { + while (iterator.MoveNext()) + { + if (predicate(iterator.Current)) return false; + } + } + return true; + } + + #region Query Statistics... + + //TODO: move this somewhere else? + + /// Measure the number of items that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IEnumerable WithCountStatistics([NotNull] this IEnumerable source, out QueryStatistics counter) + { + Contract.NotNull(source, nameof(source)); + + var signal = new QueryStatistics(0); + counter = signal; + + // to count, we just increment the signal each type a value flows through here + Func wrapped = (x) => + { + signal.Update(checked(signal.Value + 1)); + return x; + }; + + return source.Select(wrapped); + } + + /// Measure the number and size of slices that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IEnumerable> WithSizeStatistics([NotNull] this IEnumerable> source, out QueryStatistics statistics) + { + Contract.NotNull(source, nameof(source)); + + var data = new KeyValueSizeStatistics(); + statistics = new QueryStatistics(data); + + // to count, we just increment the signal each type a value flows through here + Func, KeyValuePair> wrapped = (kvp) => + { + data.Add(kvp.Key.Count, kvp.Value.Count); + return kvp; + }; + + return source.Select(wrapped); + } + + /// Measure the number and sizes of the keys and values that pass through this point of the query + /// The values returned in are only safe to read once the query has ended + [NotNull, LinqTunnel] + public static IEnumerable WithSizeStatistics([NotNull] this IEnumerable source, out QueryStatistics statistics) + { + Contract.NotNull(source, nameof(source)); + + var data = new DataSizeStatistics(); + statistics = new QueryStatistics(data); + + // to count, we just increment the signal each type a value flows through here + Func wrapped = (x) => + { + data.Add(x.Count); + return x; + }; + + return source.Select(wrapped); + } + + #endregion + + } + +} diff --git a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs deleted file mode 100644 index df90f67b5..000000000 --- a/FoundationDB.Client/Linq/FdbAsyncEnumerable.cs +++ /dev/null @@ -1,1255 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Linq -{ - using System; - using System.Collections.Generic; - using System.Threading; - using System.Threading.Tasks; - using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Client; - using JetBrains.Annotations; - - /// Provides a set of static methods for querying objects that implement . - public static partial class FdbAsyncEnumerable - { - // Welcome to the wonderful world of the Monads! - - #region Entering the Monad... - - /// Returns an empty async sequence - [NotNull] - public static IFdbAsyncEnumerable Empty() - { - return EmptySequence.Default; - } - - /// Returns an async sequence with a single element, which is a constant - [NotNull] - public static IFdbAsyncEnumerable Singleton(T value) - { - //note: we can't call this method Single(T), because then Single(Func) would be ambigous with Single>(T) - return new SingletonSequence(() => value); - } - - /// Returns an async sequence which will produce a single element, using the specified lambda - /// Lambda that will be called once per iteration, to produce the single element of this sequene - /// If the sequence is iterated multiple times, then will be called once for each iteration. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Single([NotNull] Func lambda) - { - if (lambda == null) throw new ArgumentNullException(nameof(lambda)); - return new SingletonSequence(lambda); - } - - /// Returns an async sequence which will produce a single element, using the specified lambda - /// Lambda that will be called once per iteration, to produce the single element of this sequene - /// If the sequence is iterated multiple times, then will be called once for each iteration. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Single([NotNull] Func> asyncLambda) - { - if (asyncLambda == null) throw new ArgumentNullException(nameof(asyncLambda)); - return new SingletonSequence(asyncLambda); - } - - /// Returns an async sequence which will produce a single element, using the specified lambda - /// Lambda that will be called once per iteration, to produce the single element of this sequene - /// If the sequence is iterated multiple times, then will be called once for each iteration. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Single([NotNull] Func> asyncLambda) - { - if (asyncLambda == null) throw new ArgumentNullException(nameof(asyncLambda)); - return new SingletonSequence(asyncLambda); - } - - /// Apply an async lambda to a sequence of elements to transform it into an async sequence - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnumerable source, [NotNull] Func> lambda) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (lambda == null) throw new ArgumentNullException(nameof(lambda)); - - return Create(source, (iterator) => new EnumerableIterator(iterator, lambda)); - } - - /// Apply an async lambda to a sequence of elements to transform it into an async sequence - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable ToAsyncEnumerable([NotNull] this IEnumerable source) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - return Create(source, (iterator) => new EnumerableIterator(iterator, x => Task.FromResult(x))); - } - - /// Wraps an async lambda into an async sequence that will return the result of the lambda - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable FromTask([NotNull] Func> asyncLambda) - { - //TODO: create a custom iterator for this ? - return ToAsyncEnumerable(new [] { asyncLambda }).Select(x => x()); - } - - /// Split a sequence of items into several batches - /// Type of the elemenst in - /// Source sequence - /// Maximum size of each batch - /// Sequence of batches, whose size will always we , except for the last batch that will only hold the remaning items. If the source is empty, an empty sequence is returned. - [NotNull, LinqTunnel] - public static IEnumerable> Buffered([NotNull] this IEnumerable source, int batchSize) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (batchSize <= 0) throw new ArgumentException("Batch size must be greater than zero.", nameof(batchSize)); - - var list = new List(batchSize); - foreach (var item in source) - { - list.Add(item); - if (list.Count >= batchSize) - { - yield return list; - list.Clear(); - } - } - } - - #endregion - - #region Staying in the Monad... - - #region SelectMany... - - /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> selector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (selector == null) throw new ArgumentNullException(nameof(selector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.SelectMany(selector); - } - - return Flatten(source, new AsyncTransformExpression>(selector)); - } - - /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); - - return SelectMany(source, TaskHelpers.WithCancellation(asyncSelector)); - } - - /// Projects each element of an async sequence to an and flattens the resulting sequences into one async sequence. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.SelectMany(asyncSelector); - } - - return Flatten(source, new AsyncTransformExpression>(asyncSelector)); - } - - /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> collectionSelector, [NotNull] Func resultSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (collectionSelector == null) throw new ArgumentNullException(nameof(collectionSelector)); - if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.SelectMany(collectionSelector, resultSelector); - } - - return Flatten(source, new AsyncTransformExpression>(collectionSelector), resultSelector); - } - - /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncCollectionSelector == null) throw new ArgumentNullException(nameof(asyncCollectionSelector)); - if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); - - return SelectMany(source, TaskHelpers.WithCancellation(asyncCollectionSelector), resultSelector); - } - - /// Projects each element of an async sequence to an flattens the resulting sequences into one async sequence, and invokes a result selector function on each element therein. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectMany([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func>> asyncCollectionSelector, [NotNull] Func resultSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncCollectionSelector == null) throw new ArgumentNullException(nameof(asyncCollectionSelector)); - if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.SelectMany(asyncCollectionSelector, resultSelector); - } - - return Flatten(source, new AsyncTransformExpression>(asyncCollectionSelector), resultSelector); - } - - #endregion - - #region Select... - - /// Projects each element of an async sequence into a new form. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func selector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (selector == null) throw new ArgumentNullException(nameof(selector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Select(selector); - } - - return Map(source, new AsyncTransformExpression(selector)); - } - - /// Projects each element of an async sequence into a new form. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); - - return Select(source, TaskHelpers.WithCancellation(asyncSelector)); - } - - /// Projects each element of an async sequence into a new form. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Select([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Select(asyncSelector); - } - - return Map(source, new AsyncTransformExpression(asyncSelector)); - } - - #endregion - - #region Where... - - /// Filters an async sequence of values based on a predicate. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func predicate) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Where(predicate); - } - - return Filter(source, new AsyncFilterExpression(predicate)); - } - - /// Filters an async sequence of values based on a predicate. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncPredicate) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncPredicate == null) throw new ArgumentNullException(nameof(asyncPredicate)); - - return Where(source, TaskHelpers.WithCancellation(asyncPredicate)); - } - - /// Filters an async sequence of values based on a predicate. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Where([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncPredicate) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncPredicate == null) throw new ArgumentNullException(nameof(asyncPredicate)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Where(asyncPredicate); - } - - return Filter(source, new AsyncFilterExpression(asyncPredicate)); - } - - #endregion - - #region Take... - - /// Returns a specified number of contiguous elements from the start of an async sequence. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Take([NotNull] this IFdbAsyncEnumerable source, int count) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Take(count); - } - - return FdbAsyncEnumerable.Limit(source, count); - } - - #endregion - - #region TakeWhile... - - /// Returns elements from an async sequence as long as a specified condition is true, and then skips the remaining elements. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable TakeWhile([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func condition) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (condition == null) throw new ArgumentNullException(nameof(condition)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.TakeWhile(condition); - } - - return FdbAsyncEnumerable.Limit(source, condition); - } - - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable TakeWhile(this IFdbAsyncEnumerable source, [NotNull] Func condition, out QueryStatistics stopped) - { - var signal = new QueryStatistics(false); - stopped = signal; - - // to trigger the signal, we just intercept the condition returning false (which only happen once!) - Func wrapped = (x) => - { - if (condition(x)) return true; - signal.Update(true); - return false; - }; - - return TakeWhile(source, wrapped); - } - - #endregion - - #region Skip... - - /// Skips the first elements of an async sequence. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Skip([NotNull] this IFdbAsyncEnumerable source, int count) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be less than zero"); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.Skip(count); - } - - return FdbAsyncEnumerable.Offset(source, count); - } - - #endregion - - #region SelectAsync - - /// Projects each element of an async sequence into a new form. - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable SelectAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func> asyncSelector, FdbParallelQueryOptions options = null) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncSelector == null) throw new ArgumentNullException(nameof(asyncSelector)); - - return new FdbParallelSelectAsyncIterator(source, asyncSelector, options ?? new FdbParallelQueryOptions()); - } - - /// Always prefetch the next item from the inner sequence. - /// Type of the items in the source sequence - /// Source sequence that has a high latency, and from which we want to prefetch a set number of items. - /// Sequence that prefetch the next item, when outputing the current item. - /// - /// This iterator can help smooth out the query pipeline when every call to the inner sequence has a somewhat high latency (ex: reading the next page of results from the database). - /// Avoid prefetching from a source that is already reading from a buffer of results. - /// - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdbAsyncEnumerable source) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - return new FdbPrefetchingAsyncIterator(source, 1); - } - - /// Prefetch a certain number of items from the inner sequence, before outputing the results one by one. - /// Type of the items in the source sequence - /// Source sequence that has a high latency, and from which we want to prefetch a set number of items. - /// Maximum number of items to buffer from the source before they are consumed by the rest of the query. - /// Sequence that returns items from a buffer of prefetched list. - /// - /// This iterator can help smooth out the query pipeline when every call to the inner sequence has a somewhat high latency (ex: reading the next page of results from the database). - /// Avoid prefetching from a source that is already reading from a buffer of results. - /// - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Prefetch([NotNull] this IFdbAsyncEnumerable source, int prefetchCount) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (prefetchCount <= 0) throw new ArgumentOutOfRangeException(nameof(prefetchCount), prefetchCount, "Prefetch count must be at least one."); - - return new FdbPrefetchingAsyncIterator(source, prefetchCount); - } - - /// Buffers the items of a bursty sequence, into a sequence of variable-sized arrays made up of items that where produced in a very short timespan. - /// Type of the items in the source sequence - /// Source sequence, that produces bursts of items, produced from the same page of results, before reading the next page. - /// Maximum number of items to return in a single window. If more items arrive at the same time, a new window will be opened with the rest of the items. - /// Sequence of batches, where all the items of a single batch arrived at the same time. A batch is closed once the next call to MoveNext() on the inner sequence does not complete immediately. Batches can be smaller than . - /// - /// This should only be called on bursty asynchronous sequences, and when you want to process items in batches, without incurring the cost of latency between two pages of results. - /// You should avoid using this operator on sequences where each call to MoveNext() is asynchronous, since it would only produce batchs with only a single item. - /// - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Window([NotNull] this IFdbAsyncEnumerable source, int maxWindowSize) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (maxWindowSize <= 0) throw new ArgumentOutOfRangeException(nameof(maxWindowSize), maxWindowSize, "Window size must be at least one."); - - return new FdbWindowingAsyncIterator(source, maxWindowSize); - } - - /// Buffers the items of a source sequence, and outputs a sequence of fixed-sized arrays. - /// Type of the items in the source sequence - /// Source sequence that will be cut into chunks containing at most items. - /// Number of items per batch. The last batch may contain less items, but should never be empty. - /// Sequence of arrays of size , except the last batch which can have less items. - /// - /// This operator does not care about the latency of each item, and will always try to fill each batch completely, before outputing a result. - /// If you are working on an inner sequence that is bursty in nature, where items arrives in waves, you should use which attempts to minimize the latency by outputing incomplete batches if needed. - /// - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Batch([NotNull] this IFdbAsyncEnumerable source, int batchSize) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (batchSize <= 0) throw new ArgumentOutOfRangeException(nameof(batchSize), batchSize, "Batch size must be at least one."); - - return new FdbBatchingAsyncIterator(source, batchSize); - } - - #endregion - - #region Distinct... - - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Distinct([NotNull] this IFdbAsyncEnumerable source, IEqualityComparer comparer = null) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - comparer = comparer ?? EqualityComparer.Default; - - return new FdbDistinctAsyncIterator(source, comparer); - } - - #endregion - - #region OrderBy... - - [NotNull] - public static IFdbAsyncOrderedEnumerable OrderBy([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); - comparer = comparer ?? Comparer.Default; - - return new OrderedSequence(source, keySelector, comparer, descending: false, parent: null); - } - - [NotNull] - public static IFdbAsyncOrderedEnumerable OrderByDescending([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); - comparer = comparer ?? Comparer.Default; - - return new OrderedSequence(source, keySelector, comparer, descending: true, parent: null); - } - - [NotNull] - public static IFdbAsyncOrderedEnumerable ThenBy([NotNull] this IFdbAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) - { - if (source == null) throw new ArgumentNullException(nameof(keySelector)); - return source.CreateOrderedEnumerable(keySelector, comparer, descending: false); - } - - [NotNull] - public static IFdbAsyncOrderedEnumerable ThenByDescending([NotNull] this IFdbAsyncOrderedEnumerable source, [NotNull] Func keySelector, IComparer comparer = null) - { - if (source == null) throw new ArgumentNullException(nameof(keySelector)); - return source.CreateOrderedEnumerable(keySelector, comparer, descending: true); - } - - #endregion - - // If you are bored, maybe consider adding: - // - DefaultIfEmpty - // - Zip - // - OrderBy and OrderBy - // - GroupBy - - #endregion - - #region Leaving the Monad... - - /// Execute an action for each element of an async sequence - public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Action action, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (action == null) throw new ArgumentNullException(nameof(action)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.ExecuteAsync(action, ct); - } - else - { - return Run(source, FdbAsyncMode.All, action, ct); - } - } - - /// Execute an async action for each element of an async sequence - public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) - { - if (asyncAction == null) throw new ArgumentNullException(nameof(asyncAction)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.ExecuteAsync(TaskHelpers.WithCancellation(asyncAction), ct); - } - else - { - return ForEachAsync(source, TaskHelpers.WithCancellation(asyncAction), ct); - } - } - - /// Execute an async action for each element of an async sequence - public static Task ForEachAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func asyncAction, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncAction == null) throw new ArgumentNullException(nameof(asyncAction)); - - var iterator = source as FdbAsyncIterator; - if (iterator != null) - { - return iterator.ExecuteAsync(asyncAction, ct); - } - else - { - return Run(source, FdbAsyncMode.All, asyncAction, ct); - } - } - - /// Create an Hashset from an async sequence. - [ItemNotNull] - public static Task> ToHashsetAsync([NotNull] this IFdbAsyncEnumerable source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - return AggregateAsync( - source, - new Buffer(), - (buffer, x) => buffer.Add(x), - (buffer) => buffer.ToHashSet(comparer), - ct - ); - } - - /// Create a list from an async sequence. - [ItemNotNull] - public static Task> ToListAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - return AggregateAsync( - source, - new Buffer(), - (buffer, x) => buffer.Add(x), - (buffer) => buffer.ToList(), - ct - ); - } - - /// Create an array from an async sequence. - [ItemNotNull] - public static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - return AggregateAsync( - source, - new Buffer(), - (buffer, x) => buffer.Add(x), - (buffer) => buffer.ToArray(), - ct - ); - } - - /// Create an array from an async sequence, knowing a rough estimation of the number of elements. - [ItemNotNull] - internal static Task ToArrayAsync([NotNull] this IFdbAsyncEnumerable source, int estimatedSize, CancellationToken ct = default(CancellationToken)) - { - Contract.Requires(source != null && estimatedSize >= 0); - - return AggregateAsync( - source, - new List(estimatedSize), - (buffer, x) => buffer.Add(x), - (buffer) => buffer.ToArray(), - ct - ); - } - - /// Creates a Dictionary from an async sequence according to a specified key selector function and key comparer. - [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); - - return AggregateAsync( - source, - new Dictionary(comparer ?? EqualityComparer.Default), - (results, x) => { results[keySelector(x)] = x; }, - ct - ); - } - - /// Creates a Dictionary from an async sequence according to a specified key selector function, a comparer, and an element selector function. - [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func keySelector, [NotNull, InstantHandle] Func elementSelector, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (keySelector == null) throw new ArgumentNullException(nameof(keySelector)); - if (elementSelector == null) throw new ArgumentNullException(nameof(elementSelector)); - - return AggregateAsync( - source, - new Dictionary(comparer ?? EqualityComparer.Default), - (results, x) => { results[keySelector(x)] = elementSelector(x); }, - ct - ); - } - - /// Creates a Dictionary from an async sequence of pairs of keys and values. - [ItemNotNull] - public static Task> ToDictionaryAsync([NotNull] this IFdbAsyncEnumerable> source, IEqualityComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - return AggregateAsync( - source, - new Dictionary(comparer ?? EqualityComparer.Default), - (results, x) => { results[x.Key] = x.Value; }, - ct - ); - } - - /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - - ct.ThrowIfCancellationRequested(); - using (var iterator = source.GetEnumerator(FdbAsyncMode.All)) - { - Contract.Assert(iterator != null, "The sequence returned a null async iterator"); - - if (!(await iterator.MoveNextAsync(ct).ConfigureAwait(false))) - { - throw new InvalidOperationException("The sequence was empty"); - } - - var item = iterator.Current; - while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) - { - item = aggregator(item, iterator.Current); - } - - return item; - } - } - - /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - - var accumulate = seed; - await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); - return accumulate; - } - - /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - - var accumulate = seed; - await ForEachAsync(source, (x) => { aggregator(accumulate, x); }, ct).ConfigureAwait(false); - return accumulate; - } - - /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Func aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); - - var accumulate = seed; - await ForEachAsync(source, (x) => { accumulate = aggregator(accumulate, x); }, ct).ConfigureAwait(false); - return resultSelector(accumulate); - } - - /// Applies an accumulator function over an async sequence. - public static async Task AggregateAsync([NotNull] this IFdbAsyncEnumerable source, TAccumulate seed, [NotNull, InstantHandle] Action aggregator, [NotNull, InstantHandle] Func resultSelector, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (aggregator == null) throw new ArgumentNullException(nameof(aggregator)); - if (resultSelector == null) throw new ArgumentNullException(nameof(resultSelector)); - - var accumulate = seed; - await ForEachAsync(source, (x) => aggregator(accumulate, x), ct); - return resultSelector(accumulate); - } - - /// Returns the first element of an async sequence, or an exception if it is empty - public static Task FirstAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return rq.FirstAsync(); - - return Head(source, single: false, orDefault: false, ct: ct); - } - - /// Returns the first element of an async sequence, or the default value for the type if it is empty - public static Task FirstOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return rq.FirstOrDefaultAsync(); - - return Head(source, single: false, orDefault: true, ct: ct); - } - - /// Returns the first and only element of an async sequence, or an exception if it is empty or have two or more elements - /// Will need to call MoveNext at least twice to ensure that there is no second element. - public static Task SingleAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return rq.SingleAsync(); - - return Head(source, single: true, orDefault: false, ct: ct); - } - - /// Returns the first and only element of an async sequence, the default value for the type if it is empty, or an exception if it has two or more elements - /// Will need to call MoveNext at least twice to ensure that there is no second element. - public static Task SingleOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return rq.SingleOrDefaultAsync(); - - return Head(source, single: true, orDefault: true, ct: ct); - } - - /// Returns the last element of an async sequence, or an exception if it is empty - public static async Task LastAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return await rq.LastAsync(); - - bool found = false; - T last = default(T); - - await ForEachAsync(source, (x) => { found = true; last = x; }, ct).ConfigureAwait(false); - - if (!found) throw new InvalidOperationException("The sequence was empty"); - return last; - } - - /// Returns the last element of an async sequence, or the default value for the type if it is empty - public static async Task LastOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return await rq.LastOrDefaultAsync(); - - bool found = false; - T last = default(T); - - await ForEachAsync(source, (x) => { found = true; last = x; }, ct).ConfigureAwait(false); - - return found ? last : default(T); - } - - /// Returns the element at a specific location of an async sequence, or an exception if there are not enough elements - public static async Task ElementAtAsync([NotNull] this IFdbAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return await rq.Skip(index).SingleAsync(); - - int counter = index; - T item = default(T); - await Run( - source, - FdbAsyncMode.All, - (x) => - { - if (counter-- == 0) { item = x; return false; } - return true; - }, - ct - ).ConfigureAwait(false); - - if (counter >= 0) throw new InvalidOperationException("The sequence was too small"); - return item; - } - - /// Returns the element at a specific location of an async sequence, or the default value for the type if it there are not enough elements - public static async Task ElementAtOrDefaultAsync([NotNull] this IFdbAsyncEnumerable source, int index, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (index < 0) throw new ArgumentOutOfRangeException(nameof(index)); - ct.ThrowIfCancellationRequested(); - - var rq = source as FdbRangeQuery; - if (rq != null) return await rq.Skip(index).SingleAsync(); - - int counter = index; - T item = default(T); - - //TODO: use ExecuteAsync() if the source is an Iterator! - await Run( - source, - FdbAsyncMode.All, - (x) => - { - if (counter-- == 0) { item = x; return false; } - return true; - }, - ct - ).ConfigureAwait(false); - - if (counter >= 0) return default(T); - return item; - } - - /// Returns the number of elements in an async sequence. - public static async Task CountAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - int count = 0; - - await ForEachAsync(source, (_) => { ++count; }, ct).ConfigureAwait(false); - - return count; - } - - /// Returns a number that represents how many elements in the specified async sequence satisfy a condition. - public static async Task CountAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - - int count = 0; - - await ForEachAsync(source, (x) => { if (predicate(x)) ++count; }, ct).ConfigureAwait(false); - - return count; - } - - /// Returns the sum of all elements in the specified async sequence. - public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - ulong sum = 0; - - await ForEachAsync(source, (x) => { sum += x; }, ct).ConfigureAwait(false); - - return sum; - } - - /// Returns the sum of all elements in the specified async sequence that satisfy a condition. - public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - - ulong sum = 0; - - await ForEachAsync(source, (x) => { if (predicate(x)) sum += x; }, ct).ConfigureAwait(false); - - return sum; - } - - /// Returns the sum of all elements in the specified async sequence. - public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - long sum = 0; - - await ForEachAsync(source, (x) => { sum += x; }, ct).ConfigureAwait(false); - - return sum; - } - - /// Returns the sum of all elements in the specified async sequence that satisfy a condition. - public static async Task SumAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - - long sum = 0; - - await ForEachAsync(source, (x) => { if (predicate(x)) sum += x; }, ct).ConfigureAwait(false); - - return sum; - } - - /// Returns the smallest value in the specified async sequence - public static async Task MinAsync([NotNull] this IFdbAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - comparer = comparer ?? Comparer.Default; - - bool found = false; - T min = default(T); - - await ForEachAsync( - source, - (x) => - { - if (!found || comparer.Compare(x, min) < 0) - { - min = x; - found = true; - } - }, - ct - ).ConfigureAwait(false); - - if (!found) throw new InvalidOperationException("The sequence was empty"); - return min; - } - - /// Returns the largest value in the specified async sequence - public static async Task MaxAsync([NotNull] this IFdbAsyncEnumerable source, IComparer comparer = null, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - comparer = comparer ?? Comparer.Default; - - bool found = false; - T max = default(T); - - await ForEachAsync( - source, - (x) => - { - if (!found || comparer.Compare(x, max) > 0) - { - max = x; - found = true; - } - }, - ct - ).ConfigureAwait(false); - - if (!found) throw new InvalidOperationException("The sequence was empty"); - return max; - } - - /// Determines whether an async sequence contains any elements. - /// This is the logical equivalent to "source.Count() > 0" but can be better optimized by some providers - public static async Task AnyAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) - { - return await iterator.MoveNextAsync(ct).ConfigureAwait(false); - } - } - - /// Determines whether any element of an async sequence satisfies a condition. - public static async Task AnyAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - ct.ThrowIfCancellationRequested(); - - using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) - { - while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) - { - if (predicate(iterator.Current)) return true; - } - } - return false; - } - - /// Determines wether an async sequence contains no elements at all. - /// This is the logical equivalent to "source.Count() == 0" or "!source.Any()" but can be better optimized by some providers - public static async Task NoneAsync([NotNull] this IFdbAsyncEnumerable source, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - ct.ThrowIfCancellationRequested(); - - using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) - { - return !(await iterator.MoveNextAsync(ct).ConfigureAwait(false)); - } - } - - /// Determines whether none of the elements of an async sequence satisfies a condition. - public static async Task NoneAsync([NotNull] this IFdbAsyncEnumerable source, [NotNull, InstantHandle] Func predicate, CancellationToken ct = default(CancellationToken)) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (predicate == null) throw new ArgumentNullException(nameof(predicate)); - ct.ThrowIfCancellationRequested(); - - using (var iterator = source.GetEnumerator(FdbAsyncMode.Head)) - { - while (await iterator.MoveNextAsync(ct).ConfigureAwait(false)) - { - if (predicate(iterator.Current)) return false; - } - } - return true; - } - - #endregion - - #region Query Statistics... - - //TODO: move this somewhere else? - - public class QueryStatistics - { - public QueryStatistics() - { } - - public QueryStatistics(TData value) - { - this.Value = value; - } - - public TData Value { get; protected set; } - - public void Update(TData newValue) - { - this.Value = newValue; - } - } - - public class KeyValueSize - { - /// Total number of pairs of keys and values that have flowed through this point - public long Count { get; private set; } - - /// Total size of all keys and values combined - public long Size { get { return checked(this.KeySize + this.ValueSize); } } - - /// Total size of all keys combined - public long KeySize { get; private set; } - - /// Total size of all values combined - public long ValueSize { get; private set; } - - public void Add(int keySize, int valueSize) - { - this.Count++; - this.KeySize = checked(keySize + this.KeySize); - this.ValueSize = checked(valueSize + this.ValueSize); - } - } - - public class DataSize - { - /// Total number of items that have flowed through this point - public long Count { get; private set; } - - /// Total size of all items that have flowed through this point - public long Size { get; private set; } - - public void Add(int size) - { - this.Count++; - this.Size = checked(size + this.Size); - } - } - - /// Measure the number of items that pass through this point of the query - /// The values returned in are only safe to read once the query has ended - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable WithCountStatistics([NotNull] this IFdbAsyncEnumerable source, out QueryStatistics counter) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - var signal = new QueryStatistics(0); - counter = signal; - - // to count, we just increment the signal each type a value flows through here - Func wrapped = (x) => - { - signal.Update(checked(signal.Value + 1)); - return x; - }; - - return Select(source, wrapped); - } - - /// Measure the number and size of slices that pass through this point of the query - /// The values returned in are only safe to read once the query has ended - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable> WithSizeStatistics([NotNull] this IFdbAsyncEnumerable> source, out QueryStatistics statistics) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - var data = new KeyValueSize(); - statistics = new QueryStatistics(data); - - // to count, we just increment the signal each type a value flows through here - Func, KeyValuePair> wrapped = (kvp) => - { - data.Add(kvp.Key.Count, kvp.Value.Count); - return kvp; - }; - - return Select(source, wrapped); - } - - /// Measure the number and sizes of the keys and values that pass through this point of the query - /// The values returned in are only safe to read once the query has ended - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable WithSizeStatistics([NotNull] this IFdbAsyncEnumerable source, out QueryStatistics statistics) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - - var data = new DataSize(); - statistics = new QueryStatistics(data); - - // to count, we just increment the signal each type a value flows through here - Func wrapped = (x) => - { - data.Add(x.Count); - return x; - }; - - return Select(source, wrapped); - } - - /// Execute an action on each item passing through the sequence, without modifying the original sequence - /// The is execute inline before passing the item down the line, and should not block - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Observe([NotNull] this IFdbAsyncEnumerable source, [NotNull] Action handler) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (handler == null) throw new ArgumentNullException(nameof(handler)); - - return new FdbObserverIterator(source, new AsyncObserverExpression(handler)); - } - - /// Execute an action on each item passing through the sequence, without modifying the original sequence - /// The is execute inline before passing the item down the line, and should not block - [NotNull, LinqTunnel] - public static IFdbAsyncEnumerable Observe([NotNull] this IFdbAsyncEnumerable source, [NotNull] Func asyncHandler) - { - if (source == null) throw new ArgumentNullException(nameof(source)); - if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); - - return new FdbObserverIterator(source, new AsyncObserverExpression(asyncHandler)); - } - - #endregion - - } - -} diff --git a/FoundationDB.Client/Async/IAsyncEnumerable.cs b/FoundationDB.Client/Linq/IAsyncEnumerable.cs similarity index 85% rename from FoundationDB.Client/Async/IAsyncEnumerable.cs rename to FoundationDB.Client/Linq/IAsyncEnumerable.cs index e4488b6c1..069cb1a86 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/IAsyncEnumerable.cs @@ -26,8 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Linq { + using System.Threading; using JetBrains.Annotations; // note: these interfaces are modeled after the IAsyncEnumerable and IAsyncEnumerator found in Rx @@ -38,9 +39,11 @@ namespace FoundationDB.Async public interface IAsyncEnumerable { /// Gets an asynchronous enumerator over the sequence. + /// Token used to cancel the iterator from the outside + /// Defines how the enumerator will be used by the caller. The source provider can use the mode to optimize how the results are produced. /// Enumerator for asynchronous enumeration over the sequence. [NotNull] - IAsyncEnumerator GetEnumerator(); + IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint hint); } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Async/IAsyncEnumerator.cs b/FoundationDB.Client/Linq/IAsyncEnumerator.cs similarity index 91% rename from FoundationDB.Client/Async/IAsyncEnumerator.cs rename to FoundationDB.Client/Linq/IAsyncEnumerator.cs index 580201bb0..095b60640 100644 --- a/FoundationDB.Client/Async/IAsyncEnumerator.cs +++ b/FoundationDB.Client/Linq/IAsyncEnumerator.cs @@ -26,10 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Async +namespace Doxense.Linq { using System; - using System.Threading; using System.Threading.Tasks; // note: these interfaces are modeled after the IAsyncEnumerable and IAsyncEnumerator found in Rx @@ -40,12 +39,11 @@ namespace FoundationDB.Async public interface IAsyncEnumerator : IDisposable { /// Advances the enumerator to the next element in the sequence, returning the result asynchronously. - /// Cancellation token that can be used to cancel the operation. /// - /// Task containing the result of the operation: true if the enumerator was successfully advanced + /// Task containing the result of the operation: true if the enumerator was successfully advanced /// to the next element; false if the enumerator has passed the end of the sequence. /// - Task MoveNextAsync(CancellationToken ct); + Task MoveNextAsync(); /// Gets the current element in the iteration. T Current { get; } diff --git a/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs b/FoundationDB.Client/Linq/IAsyncOrderedEnumerable.cs similarity index 68% rename from FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs rename to FoundationDB.Client/Linq/IAsyncOrderedEnumerable.cs index 369e5b129..029583a2c 100644 --- a/FoundationDB.Client/Linq/IFdbAsyncOrderedEnumerable.cs +++ b/FoundationDB.Client/Linq/IAsyncOrderedEnumerable.cs @@ -26,20 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { - using FoundationDB.Async; - using FoundationDB.Client.Utils; using JetBrains.Annotations; using System; using System.Collections.Generic; - using System.Threading; - using System.Threading.Tasks; - public interface IFdbAsyncOrderedEnumerable : IFdbAsyncEnumerable + // note: these interfaces are modeled after the IAsyncEnumerable and IAsyncEnumerator found in Rx + //TODO: if/when async enumerables are avail in C#, we would just need to either remove these interfaces, or make them implement the real stuff + + /// Asynchronous version of the interface, allowing elements of the enumerable sequence to be retrieved asynchronously. + /// + public interface IAsyncOrderedEnumerable : IAsyncEnumerable { [NotNull, LinqTunnel] - IFdbAsyncOrderedEnumerable CreateOrderedEnumerable([NotNull] Func keySelector, IComparer comparer, bool descending); + IAsyncOrderedEnumerable CreateOrderedEnumerable([NotNull] Func keySelector, IComparer comparer, bool descending); } } diff --git a/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs b/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs deleted file mode 100644 index fb258399d..000000000 --- a/FoundationDB.Client/Linq/IFdbAsyncEnumerable.cs +++ /dev/null @@ -1,48 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Linq -{ - using FoundationDB.Async; - using JetBrains.Annotations; - using System; - - public interface IFdbAsyncEnumerable : IAsyncEnumerable - { - - /// - /// Gets an asynchronous enumerator over the sequence. - /// - /// Defines how the enumerator will be used by the caller. The source provider can use the mode to optimize how the results are produced. - /// Enumerator for asynchronous enumeration over the sequence. - [NotNull] - IFdbAsyncEnumerator GetEnumerator(FdbAsyncMode mode = FdbAsyncMode.Default); - - } - -} \ No newline at end of file diff --git a/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs b/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs deleted file mode 100644 index 1fe5794ee..000000000 --- a/FoundationDB.Client/Linq/IFdbAsyncEnumerator.cs +++ /dev/null @@ -1,38 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Linq -{ - using FoundationDB.Async; - using System; - public interface IFdbAsyncEnumerator : IAsyncEnumerator - { - //TODO: add method specific to FDB here - } - -} diff --git a/FoundationDB.Client/Linq/FdbParallelQueryOptions.cs b/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs similarity index 96% rename from FoundationDB.Client/Linq/FdbParallelQueryOptions.cs rename to FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs index fea5f3126..cd1e10b41 100644 --- a/FoundationDB.Client/Linq/FdbParallelQueryOptions.cs +++ b/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs @@ -26,13 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Linq +namespace Doxense.Linq { using System; using System.Threading.Tasks; /// Container class for all settings relative to parallel operations - public sealed class FdbParallelQueryOptions + public sealed class ParallelAsyncQueryOptions { /// Maximum number of concurrent async tasks that can run in parallel diff --git a/FoundationDB.Client/Linq/QueryStatistics.cs b/FoundationDB.Client/Linq/QueryStatistics.cs new file mode 100644 index 000000000..1eceba08e --- /dev/null +++ b/FoundationDB.Client/Linq/QueryStatistics.cs @@ -0,0 +1,89 @@ +#region BSD Licence +/* Copyright (c) 2013, Doxense SARL +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 + +namespace Doxense.Linq +{ + using System; + + public class QueryStatistics + { + public QueryStatistics() + { } + + public QueryStatistics(TData value) + { + this.Value = value; + } + + public TData Value { get; protected set; } + + public void Update(TData newValue) + { + this.Value = newValue; + } + } + + + public sealed class KeyValueSizeStatistics + { + /// Total number of pairs of keys and values that have flowed through this point + public long Count { get; private set; } + + /// Total size of all keys and values combined + public long Size => checked(this.KeySize + this.ValueSize); + + /// Total size of all keys combined + public long KeySize { get; private set; } + + /// Total size of all values combined + public long ValueSize { get; private set; } + + public void Add(int keySize, int valueSize) + { + this.Count++; + this.KeySize = checked(keySize + this.KeySize); + this.ValueSize = checked(valueSize + this.ValueSize); + } + } + + public sealed class DataSizeStatistics + { + /// Total number of items that have flowed through this point + public long Count { get; private set; } + + /// Total size of all items that have flowed through this point + public long Size { get; private set; } + + public void Add(int size) + { + this.Count++; + this.Size = checked(size + this.Size); + } + } + +} diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index 7163f009c..c83167039 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; @@ -34,10 +35,8 @@ namespace FoundationDB.Client using System.Linq; using System.Threading; using System.Threading.Tasks; - using SystemIO = System.IO; - using FoundationDB.Client.Utils; + using Doxense.Linq; using FoundationDB.Layers.Directories; - using FoundationDB.Linq; using JetBrains.Annotations; public static partial class Fdb diff --git a/FoundationDB.Client/Utils/ExceptionExtensions.cs b/FoundationDB.Client/Utils/ExceptionExtensions.cs index 76902e7da..b84bdf08b 100644 --- a/FoundationDB.Client/Utils/ExceptionExtensions.cs +++ b/FoundationDB.Client/Utils/ExceptionExtensions.cs @@ -6,7 +6,7 @@ // #endregion -namespace FoundationDB +namespace Doxense { using Doxense.Diagnostics.Contracts; using System; diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs index a8bec27f9..0fadf8990 100644 --- a/FoundationDB.Client/Utils/Slice.cs +++ b/FoundationDB.Client/Utils/Slice.cs @@ -40,8 +40,6 @@ namespace FoundationDB using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; - using FoundationDB.Client.Utils; /// Delimits a section of a byte array [ImmutableObject(true), PublicAPI, DebuggerDisplay("Count={Count}, Offset={Offset}"), DebuggerTypeProxy(typeof(Slice.DebugView))] @@ -2287,7 +2285,7 @@ public static Task FromStreamAsync([NotNull] Stream data, CancellationTok if (data.Length > int.MaxValue) throw new InvalidOperationException("Streams of more than 2GB are not supported"); //TODO: other checks? - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); int length; checked { length = (int)data.Length; } diff --git a/FoundationDB.Client/Utils/SliceListStream.cs b/FoundationDB.Client/Utils/SliceListStream.cs index 1503ab61e..b01a10ca6 100644 --- a/FoundationDB.Client/Utils/SliceListStream.cs +++ b/FoundationDB.Client/Utils/SliceListStream.cs @@ -33,8 +33,8 @@ namespace FoundationDB using System.IO; using System.Linq; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; using JetBrains.Annotations; /// Merge multiple slices into a single stream @@ -253,7 +253,7 @@ public override Task ReadAsync(byte[] buffer, int offset, int count, System if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(ct); + return Task.FromCanceled(ct); } try @@ -303,7 +303,7 @@ public override void Flush() public override Task FlushAsync(System.Threading.CancellationToken ct) { // Not supported, but don't throw here - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } #endregion diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/SliceStream.cs index 7b20c10cc..aa66e72b7 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/SliceStream.cs @@ -32,7 +32,6 @@ namespace FoundationDB using System.IO; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; /// Stream that wraps a Slice for reading /// This stream is optimized for blocking and async reads @@ -186,7 +185,7 @@ public override Task ReadAsync(byte[] buffer, int offset, int count, System if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation(ct); + return Task.FromCanceled(ct); } try @@ -199,7 +198,7 @@ public override Task ReadAsync(byte[] buffer, int offset, int count, System } catch (Exception e) { - return TaskHelpers.FromException(e); + return Task.FromException(e); } } @@ -212,7 +211,7 @@ public override Task CopyToAsync(Stream destination, int bufferSize, System.Thre if (!destination.CanWrite) throw new ArgumentException("The destination stream cannot be written to", nameof(destination)); int remaining = m_slice.Count - m_position; - if (remaining <= 0) return TaskHelpers.CompletedTask; + if (remaining <= 0) return Task.CompletedTask; // simulate the read m_position += remaining; @@ -244,7 +243,7 @@ public override void Write(byte[] buffer, int offset, int count) /// This methods is not supported public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { - return TaskHelpers.FromException(new NotSupportedException()); + return Task.FromException(new NotSupportedException()); } #endif @@ -259,7 +258,7 @@ public override void Flush() public override Task FlushAsync(System.Threading.CancellationToken ct) { // Not supported, but don't throw here - return TaskHelpers.CompletedTask; + return Task.CompletedTask; } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index fee380684..eacd476b2 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Layers.Collections { using System; @@ -33,10 +34,10 @@ namespace FoundationDB.Layers.Collections using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; - using FoundationDB.Async; + using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Linq; using JetBrains.Annotations; [DebuggerDisplay("Name={Name}, Subspace={Subspace}")] @@ -141,7 +142,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey id) /// Async sequence of pairs of keys and values, ordered by keys ascending. /// CAUTION: This can be dangerous if the map contains a lot of entries! You should always use .Take() to limit the number of results returned. [NotNull] - public IFdbAsyncEnumerable> All([NotNull] IFdbReadOnlyTransaction trans, FdbRangeOptions options = null) + public IAsyncEnumerable> All([NotNull] IFdbReadOnlyTransaction trans, FdbRangeOptions options = null) { if (trans == null) throw new ArgumentNullException(nameof(trans)); @@ -227,7 +228,7 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action { state = handler(state, DecodeItems(batch)); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; }, ct ); @@ -360,7 +361,7 @@ await Fdb.Bulk.ExportAsync( (batch, _, __) => { state = handler(state, DecodeItems(batch)); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; }, ct ); diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index a54e6a2b7..5c34910fc 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -26,20 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Layers.Collections { - using FoundationDB.Client; - using FoundationDB.Linq; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; using System.Threading.Tasks; + using Doxense.Linq; + using FoundationDB.Client; + using JetBrains.Annotations; /// Multimap that tracks the number of times a specific key/value pair has been inserted or removed. /// Type of the keys of the map /// Type of the values of the map - [DebuggerDisplay("Subspace={Subspace}")] + [DebuggerDisplay("Subspace={" + nameof(FdbMultiMap.Subspace) + "}")] public class FdbMultiMap { // Inspired by https://foundationdb.com/recipes/developer/multimaps @@ -64,8 +65,8 @@ public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues) /// Encoder for the key/value pairs public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues, ICompositeKeyEncoder encoder) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); this.Subspace = subspace; this.AllowNegativeValues = allowNegativeValues; @@ -96,10 +97,10 @@ public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues, ICompositeKe public Task AddAsync([NotNull] IFdbTransaction trans, TKey key, TValue value) { //note: this method does not need to be async, but subtract is, so it's better if both methods have the same shape. - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.AtomicAdd(this.Location.Keys.Encode(key, value), PlusOne); - return FoundationDB.Async.TaskHelpers.CompletedTask; + return Task.CompletedTask; } /// Decrements the count of an (index, value) pair in the multimap, and optionally removes it if the count reaches zero. @@ -109,7 +110,7 @@ public Task AddAsync([NotNull] IFdbTransaction trans, TKey key, TValue value) /// If the updated count reaches zero or less, and AllowNegativeValues is not set, the key will be cleared from the map. public async Task SubtractAsync([NotNull] IFdbTransaction trans, TKey key, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); Slice k = this.Location.Keys.Encode(key, value); if (this.AllowNegativeValues) @@ -137,7 +138,7 @@ public async Task SubtractAsync([NotNull] IFdbTransaction trans, TKey key, TValu /// Checks if a (key, value) pair exists public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, TKey key, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); var v = await trans.GetAsync(this.Location.Keys.Encode(key, value)).ConfigureAwait(false); return this.AllowNegativeValues ? v.IsPresent : v.ToInt64() > 0; @@ -151,7 +152,7 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T /// The count can be zero or negative if AllowNegativeValues is enable. public async Task GetCountAsync([NotNull] IFdbReadOnlyTransaction trans, TKey key, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); Slice v = await trans.GetAsync(this.Location.Keys.Encode(key, value)).ConfigureAwait(false); if (v.IsNullOrEmpty) return null; @@ -164,9 +165,9 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T /// /// [NotNull] - public IFdbAsyncEnumerable Get([NotNull] IFdbReadOnlyTransaction trans, TKey key) + public IAsyncEnumerable Get([NotNull] IFdbReadOnlyTransaction trans, TKey key) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); if (this.AllowNegativeValues) @@ -198,7 +199,7 @@ public Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey /// /// [NotNull] - public IFdbAsyncEnumerable> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) + public IAsyncEnumerable> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) { var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); @@ -232,7 +233,7 @@ public Task> GetCountsAsync([NotNull] IFdbReadOnlyTrans /// public void Remove([NotNull] IFdbTransaction trans, TKey key) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.ClearRange(KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key))); } @@ -244,7 +245,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key) /// public void Remove([NotNull] IFdbTransaction trans, TKey key, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.Clear(this.Location.Keys.Encode(key, value)); } diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 8d2acf328..740012f7a 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -28,16 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections { + using System; + using System.Collections.Generic; + using System.Threading; + using System.Threading.Tasks; using FoundationDB.Client; #if DEBUG using FoundationDB.Filters.Logging; #endif using JetBrains.Annotations; - using System; - using System.Collections.Generic; - using System.Threading; - using System.Threading.Tasks; - using FoundationDB.Async; /// /// Provides a high-contention Queue class @@ -130,7 +129,7 @@ public Task> PopAsync([NotNull] IFdbDatabase db, CancellationToken c if (ct.IsCancellationRequested) { - return TaskHelpers.FromCancellation>(ct); + return Task.FromCanceled>(ct); } if (this.HighContention) @@ -184,7 +183,7 @@ public Task ExportAsync(IFdbDatabase db, Action handler, CancellationTo handler(this.Encoder.DecodeValue(kv.Value), offset); ++offset; } - return TaskHelpers.CompletedTask; + return Task.CompletedTask; }, ct ); @@ -227,7 +226,7 @@ public Task ExportAsync(IFdbDatabase db, Action handler, Cancellation (kvs, offset, _) => { handler(this.Encoder.DecodeValues(kvs), offset); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; }, ct ); diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index 1c104865b..27ac9f9e3 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -32,8 +32,8 @@ namespace FoundationDB.Layers.Collections using System.Linq; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Linq; using JetBrains.Annotations; /// @@ -53,14 +53,14 @@ public class FdbRankedSet /// Subspace where the set will be stored public FdbRankedSet([NotNull] IKeySubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace.Using(TypeSystem.Tuples); } public Task OpenAsync([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return SetupLevelsAsync(trans); } @@ -72,7 +72,7 @@ public Task OpenAsync([NotNull] IFdbTransaction trans) /// public Task SizeAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return trans .GetRange(this.Subspace.Partition.ByKey(MAX_LEVELS - 1).Keys.ToRange()) @@ -82,7 +82,7 @@ public Task SizeAsync([NotNull] IFdbReadOnlyTransaction trans) public async Task InsertAsync([NotNull] IFdbTransaction trans, Slice key) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (await ContainsAsync(trans, key).ConfigureAwait(false)) { @@ -120,15 +120,15 @@ public async Task InsertAsync([NotNull] IFdbTransaction trans, Slice key) public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, Slice key) { - if (trans == null) throw new ArgumentNullException("trans"); - if (key.IsNull) throw new ArgumentException("Empty key not allowed in set", "key"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (key.IsNull) throw new ArgumentException("Empty key not allowed in set", nameof(key)); return (await trans.GetAsync(this.Subspace.Keys.Encode(0, key)).ConfigureAwait(false)).HasValue; } public async Task EraseAsync([NotNull] IFdbTransaction trans, Slice key) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (!(await ContainsAsync(trans, key).ConfigureAwait(false))) { @@ -154,8 +154,8 @@ public async Task EraseAsync([NotNull] IFdbTransaction trans, Slice key) public async Task Rank([NotNull] IFdbReadOnlyTransaction trans, Slice key) { - if (trans == null) throw new ArgumentNullException("trans"); - if (key.IsNull) throw new ArgumentException("Empty key not allowed in set", "key"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (key.IsNull) throw new ArgumentException("Empty key not allowed in set", nameof(key)); if (!(await ContainsAsync(trans, key).ConfigureAwait(false))) { diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index 3f8c60fb7..37ceb453c 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -32,8 +32,8 @@ namespace FoundationDB.Layers.Collections using System.Linq; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Linq; using JetBrains.Annotations; /// Represents a potentially sparse array in FoundationDB. @@ -225,7 +225,7 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction tr, long index) } /// [NOT YET IMPLEMENTED] Get a range of items in the Vector, returned as an async sequence. - public IFdbAsyncEnumerable GetRangeAsync([NotNull] IFdbReadOnlyTransaction tr, long startIndex, long endIndex, long step) + public IAsyncEnumerable GetRangeAsync([NotNull] IFdbReadOnlyTransaction tr, long startIndex, long endIndex, long step) { if (tr == null) throw new ArgumentNullException("tr"); diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index b185acac1..200ee0b41 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -30,12 +30,10 @@ namespace FoundationDB.Layers.Blobs { using System; using System.Collections.Generic; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using FoundationDB.Client; using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using JetBrains.Annotations; // THIS IS NOT AN OFFICIAL LAYER, JUST A PROTOTYPE TO TEST A FEW THINGS ! @@ -220,9 +218,8 @@ public void Delete(IFdbTransaction trans, ITuple id, params string[] fields) /// Return the list the names of all fields of an hashset /// Transaction that will be used for this request /// Unique identifier of the hashset - /// /// List of all fields. If the list is empty, the hashset does not exist - public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id, CancellationToken ct = default(CancellationToken)) + public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id) { //note: As of Beta2, FDB does not have a fdb_get_range that only return the keys. That means that we will have to also read the values from the db, in order to just get the names of the fields :( //TODO: find a way to optimize this ? @@ -236,7 +233,7 @@ public void Delete(IFdbTransaction trans, ITuple id, params string[] fields) return trans .GetRange(KeyRange.StartsWith(prefix)) .Select((kvp) => ParseFieldKey(STuple.Unpack(kvp.Key))) - .ToListAsync(ct); + .ToListAsync(); } #endregion diff --git a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs index b72761018..9297761e7 100644 --- a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs +++ b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs @@ -1,16 +1,15 @@ -using FoundationDB.Async; -using FoundationDB.Client; -using FoundationDB.Filters.Logging; -using System; -using System.Diagnostics; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - + namespace FoundationDB.Layers.Messaging { - + using System; + using System.Diagnostics; + using System.Linq; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using Doxense.Async; + using FoundationDB.Client; + using FoundationDB.Filters.Logging; public class WorkerPoolTest { diff --git a/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs b/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs index 4a06b6cc2..65a412cb5 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbExpressionHelpers.cs @@ -26,19 +26,19 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense.Async; +using Doxense.Linq; + namespace FoundationDB.Linq.Expressions { - using FoundationDB.Async; - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics.Contracts; - using System.Linq; using System.Linq.Expressions; - using System.Reflection; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using JetBrains.Annotations; /// Helper class for working with extension expressions public static class FdbExpressionHelpers @@ -97,12 +97,12 @@ private static Task Inline([NotNull] Func func { try { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); return Task.FromResult(func(trans)); } catch(Exception e) { - return TaskHelpers.FromException(e); + return Task.FromException(e); } } @@ -129,19 +129,19 @@ public static Expression ExecuteEnumerable([NotNull] Func> generator, [NotNull] Func, CancellationToken, Task> lambda, [NotNull] IFdbReadOnlyTransaction trans, CancellationToken ct) + internal static Task ExecuteEnumerable([NotNull] Func> generator, [NotNull] Func, CancellationToken, Task> lambda, [NotNull] IFdbReadOnlyTransaction trans, CancellationToken ct) { Contract.Requires(generator != null && lambda != null && trans != null); try { - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var enumerable = generator(trans); - if (enumerable == null) return TaskHelpers.FromException(new InvalidOperationException("Source query returned null")); + if (enumerable == null) return Task.FromException(new InvalidOperationException("Source query returned null")); return lambda(enumerable, ct); } catch (Exception e) { - return TaskHelpers.FromException(e); + return Task.FromException(e); } } diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs index 0d5b40f7b..39840089c 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryAsyncEnumerableExpression.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Linq.Expressions using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; using JetBrains.Annotations; @@ -40,7 +41,7 @@ namespace FoundationDB.Linq.Expressions public sealed class FdbQueryAsyncEnumerableExpression : FdbQuerySequenceExpression { - internal FdbQueryAsyncEnumerableExpression(IFdbAsyncEnumerable source) + internal FdbQueryAsyncEnumerableExpression(IAsyncEnumerable source) { Contract.Requires(source != null); this.Source = source; @@ -53,7 +54,7 @@ public override FdbQueryShape Shape } /// Source sequence of this expression - public IFdbAsyncEnumerable Source + public IAsyncEnumerable Source { [NotNull] get; private set; @@ -72,18 +73,18 @@ public override void WriteTo(FdbQueryExpressionStringBuilder builder) } /// Returns a new expression that will execute this query on a transaction and return a single result - public override Expression>>> CompileSingle() + public override Expression>>> CompileSingle() { return FdbExpressionHelpers.ToTask(CompileSequence()); } /// Returns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>> CompileSequence() + public override Expression>> CompileSequence() { var prmTrans = Expression.Parameter(typeof(IFdbReadOnlyTransaction), "trans"); - return Expression.Lambda>>( + return Expression.Lambda>>( Expression.Constant(this.Source), prmTrans ); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs index 4b4744892..d9519f26c 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpression.cs @@ -50,17 +50,10 @@ protected FdbQueryExpression(Type type) } /// Type of the results of the query - public override Type Type - { - [NotNull] - get { return m_type; } - } + public override Type Type => m_type; /// Always return - public override ExpressionType NodeType - { - get { return ExpressionType.Extension; } - } + public override ExpressionType NodeType => ExpressionType.Extension; /// Shape of the query public abstract FdbQueryShape Shape { get; } diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index 116d83b52..bdae87da6 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -26,17 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense.Linq; + namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using FoundationDB.Layers.Indexing; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Linq.Expressions; - using System.Reflection; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using JetBrains.Annotations; /// Helper class to construct Query Expressions public static class FdbQueryExpressions @@ -44,10 +44,10 @@ public static class FdbQueryExpressions /// Return a single result from the query [NotNull] - public static FdbQuerySingleExpression Single([NotNull] FdbQuerySequenceExpression source, string name, [NotNull] Expression, CancellationToken, Task>> lambda) + public static FdbQuerySingleExpression Single([NotNull] FdbQuerySequenceExpression source, string name, [NotNull] Expression, CancellationToken, Task>> lambda) { - if (source == null) throw new ArgumentNullException("source"); - if (lambda == null) throw new ArgumentNullException("lambda"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (lambda == null) throw new ArgumentNullException(nameof(lambda)); if (name == null) name = lambda.Name ?? "Lambda"; @@ -56,9 +56,9 @@ public static FdbQuerySingleExpression Single([NotNull] FdbQuerySequ /// Return a sequence of results from the query [NotNull] - public static FdbQueryAsyncEnumerableExpression Sequence([NotNull] IFdbAsyncEnumerable source) + public static FdbQueryAsyncEnumerableExpression Sequence([NotNull] IAsyncEnumerable source) { - if (source == null) throw new ArgumentNullException("source"); + if (source == null) throw new ArgumentNullException(nameof(source)); return new FdbQueryAsyncEnumerableExpression(source); } @@ -96,7 +96,7 @@ public static FdbQueryRangeExpression RangeStartsWith(ITuple tuple, FdbRangeOpti [NotNull] public static FdbQueryIntersectExpression Intersect(params FdbQuerySequenceExpression[] expressions) { - if (expressions == null) throw new ArgumentNullException("expressions"); + if (expressions == null) throw new ArgumentNullException(nameof(expressions)); if (expressions.Length <= 1) throw new ArgumentException("There must be at least two sequences to perform an intersection"); var type = expressions[0].Type; @@ -109,7 +109,7 @@ public static FdbQueryIntersectExpression Intersect(params FdbQuerySequenc [NotNull] public static FdbQueryUnionExpression Union(params FdbQuerySequenceExpression[] expressions) { - if (expressions == null) throw new ArgumentNullException("expressions"); + if (expressions == null) throw new ArgumentNullException(nameof(expressions)); if (expressions.Length <= 1) throw new ArgumentException("There must be at least two sequences to perform an intersection"); var type = expressions[0].Type; @@ -122,10 +122,10 @@ public static FdbQueryUnionExpression Union(params FdbQuerySequenceExpress [NotNull] public static FdbQueryTransformExpression Transform([NotNull] FdbQuerySequenceExpression source, [NotNull] Expression> transform) { - if (source == null) throw new ArgumentNullException("source"); - if (transform == null) throw new ArgumentNullException("transform"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (transform == null) throw new ArgumentNullException(nameof(transform)); - if (source.ElementType != typeof(T)) throw new ArgumentException(String.Format("Source sequence has type {0} that is not compatible with transform input type {1}", source.ElementType.Name, typeof(T).Name), "source"); + if (source.ElementType != typeof(T)) throw new ArgumentException(String.Format("Source sequence has type {0} that is not compatible with transform input type {1}", source.ElementType.Name, typeof(T).Name), nameof(source)); return new FdbQueryTransformExpression(source, transform); } @@ -134,10 +134,10 @@ public static FdbQueryTransformExpression Transform([NotNull] FdbQue [NotNull] public static FdbQueryFilterExpression Filter([NotNull] FdbQuerySequenceExpression source, [NotNull] Expression> filter) { - if (source == null) throw new ArgumentNullException("source"); - if (filter == null) throw new ArgumentNullException("filter"); + if (source == null) throw new ArgumentNullException(nameof(source)); + if (filter == null) throw new ArgumentNullException(nameof(filter)); - if (source.ElementType != typeof(T)) throw new ArgumentException(String.Format("Source sequence has type {0} that is not compatible with filter input type {1}", source.ElementType.Name, typeof(T).Name), "source"); + if (source.ElementType != typeof(T)) throw new ArgumentException(String.Format("Source sequence has type {0} that is not compatible with filter input type {1}", source.ElementType.Name, typeof(T).Name), nameof(source)); return new FdbQueryFilterExpression(source, filter); } @@ -146,7 +146,7 @@ public static FdbQueryFilterExpression Filter([NotNull] FdbQuerySequenceEx [NotNull] public static string ExplainSingle([NotNull] FdbQueryExpression expression, CancellationToken ct) { - if (expression == null) throw new ArgumentNullException("expression"); + if (expression == null) throw new ArgumentNullException(nameof(expression)); if (expression.Shape != FdbQueryShape.Single) throw new InvalidOperationException("Invalid shape (single expected)"); var expr = expression.CompileSingle(); @@ -158,7 +158,7 @@ public static string ExplainSingle([NotNull] FdbQueryExpression expression [NotNull] public static string ExplainSequence([NotNull] FdbQuerySequenceExpression expression) { - if (expression == null) throw new ArgumentNullException("expression"); + if (expression == null) throw new ArgumentNullException(nameof(expression)); if (expression.Shape != FdbQueryShape.Sequence) throw new InvalidOperationException("Invalid shape (sequence expected)"); var expr = expression.CompileSequence(); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs index 49c131e9b..c2b7c55ad 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryFilterExpression.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Linq.Expressions using System; using System.Linq.Expressions; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; using JetBrains.Annotations; @@ -78,7 +79,7 @@ public override void WriteTo(FdbQueryExpressionStringBuilder builder) /// Returns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>> CompileSequence() + public override Expression>> CompileSequence() { var lambda = this.Filter.Compile(); @@ -88,13 +89,13 @@ public override Expression> // (tr) => sourceEnumerable(tr).Where(lambda); - var body = FdbExpressionHelpers.RewriteCall, Func, IFdbAsyncEnumerable>>( + var body = FdbExpressionHelpers.RewriteCall, Func, IAsyncEnumerable>>( (sequence, predicate) => sequence.Where(predicate), FdbExpressionHelpers.RewriteCall(enumerable, prmTrans), Expression.Constant(lambda) ); - return Expression.Lambda>>(body, prmTrans); + return Expression.Lambda>>(body, prmTrans); } } diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs index adf6cc479..db179f004 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIndexLookupExpression.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Linq.Expressions using System.Globalization; using System.Linq.Expressions; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; using FoundationDB.Layers.Indexing; using JetBrains.Annotations; @@ -118,7 +119,7 @@ public FdbIndex Index /// Returns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>> CompileSequence() + public override Expression>> CompileSequence() { var prmTrans = Expression.Parameter(typeof(IFdbReadOnlyTransaction), "trans"); Expression body; @@ -127,7 +128,7 @@ public override Expression> { case ExpressionType.Equal: { - body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IFdbAsyncEnumerable>>( + body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IAsyncEnumerable>>( (index, trans, value, reverse) => index.Lookup(trans, value, reverse), Expression.Constant(this.Index, typeof(FdbIndex)), prmTrans, @@ -140,7 +141,7 @@ public override Expression> case ExpressionType.GreaterThan: case ExpressionType.GreaterThanOrEqual: { - body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IFdbAsyncEnumerable>>( + body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IAsyncEnumerable>>( (index, trans, value, reverse) => index.LookupGreaterThan(trans, value, this.Operator == ExpressionType.GreaterThanOrEqual, reverse), Expression.Constant(this.Index, typeof(FdbIndex)), prmTrans, @@ -153,7 +154,7 @@ public override Expression> case ExpressionType.LessThan: case ExpressionType.LessThanOrEqual: { - body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IFdbAsyncEnumerable>>( + body = FdbExpressionHelpers.RewriteCall, IFdbReadOnlyTransaction, V, bool, IAsyncEnumerable>>( (index, trans, value, reverse) => index.LookupLessThan(trans, value, this.Operator == ExpressionType.LessThanOrEqual, reverse), Expression.Constant(this.Index, typeof(FdbIndex)), prmTrans, @@ -169,7 +170,7 @@ public override Expression> } } - return Expression.Lambda>>(body, prmTrans); + return Expression.Lambda>>(body, prmTrans); } /// Returns a textual representation of expression diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs index d07480e30..1b6e9819c 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryIntersectExpression.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Linq.Expressions using System.Collections.Generic; using System.Linq.Expressions; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using FoundationDB.Client; using JetBrains.Annotations; @@ -105,7 +106,7 @@ public override void WriteTo(FdbQueryExpressionStringBuilder builder) /// Returns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>> CompileSequence() + public override Expression>> CompileSequence() { // compile the key selector var prmTrans = Expression.Parameter(typeof(IFdbReadOnlyTransaction), "trans"); @@ -117,13 +118,13 @@ public override Expression> enumerables[i] = FdbExpressionHelpers.RewriteCall(this.Expressions[i].CompileSequence(), prmTrans); } - var array = Expression.NewArrayInit(typeof(IFdbAsyncEnumerable), enumerables); + var array = Expression.NewArrayInit(typeof(IAsyncEnumerable), enumerables); Expression body; switch (this.MergeType) { case FdbQueryMergeType.Intersect: { - body = FdbExpressionHelpers.RewriteCall[], IComparer, IFdbAsyncEnumerable>>( + body = FdbExpressionHelpers.RewriteCall[], IComparer, IAsyncEnumerable>>( (sources, comparer) => FdbMergeQueryExtensions.Intersect(sources, comparer), array, Expression.Constant(this.KeyComparer, typeof(IComparer)) @@ -132,7 +133,7 @@ public override Expression> } case FdbQueryMergeType.Union: { - body = FdbExpressionHelpers.RewriteCall[], IComparer, IFdbAsyncEnumerable>>( + body = FdbExpressionHelpers.RewriteCall[], IComparer, IAsyncEnumerable>>( (sources, comparer) => FdbMergeQueryExtensions.Union(sources, comparer), array, Expression.Constant(this.KeyComparer, typeof(IComparer)) @@ -145,7 +146,7 @@ public override Expression> } } - return Expression.Lambda>>( + return Expression.Lambda>>( body, prmTrans ); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs index 5d177db9b..c8d393d94 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryRangeExpression.cs @@ -26,15 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Globalization; using System.Linq.Expressions; - using System.Threading; + using Doxense.Linq; + using FoundationDB.Client; + using JetBrains.Annotations; /// Expression that represents a GetRange query using a pair of key selectors public class FdbQueryRangeExpression : FdbQuerySequenceExpression> @@ -69,7 +70,7 @@ public override void WriteTo(FdbQueryExpressionStringBuilder builder) /// Returns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>>> CompileSequence() + public override Expression>>> CompileSequence() { var prmTrans = Expression.Parameter(typeof(IFdbReadOnlyTransaction), "trans"); @@ -80,7 +81,7 @@ public override Expression>>>( + return Expression.Lambda>>>( body, prmTrans ); @@ -95,4 +96,4 @@ public override string ToString() } -} \ No newline at end of file +} diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs index 92733490a..ddbf13a96 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQuerySequenceExpression.cs @@ -25,19 +25,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. */ #endregion - + namespace FoundationDB.Linq.Expressions { - using FoundationDB.Client; using JetBrains.Annotations; using System; using System.Linq.Expressions; using System.Threading; using System.Threading.Tasks; + using Doxense.Linq; + using FoundationDB.Client; /// Base class of all queries that return a sequence of elements (Ranges, Index lookups, ...) /// Type of items returned - public abstract class FdbQuerySequenceExpression : FdbQueryExpression> + public abstract class FdbQuerySequenceExpression : FdbQueryExpression> { /// Type of elements returned by the sequence public Type ElementType @@ -53,10 +54,10 @@ public override FdbQueryShape Shape } /// Returns a new expression that creates an async sequence that will execute this query on a transaction - public abstract Expression>> CompileSequence(); + public abstract Expression>> CompileSequence(); /// Returns a new expression that creates an async sequence that will execute this query on a transaction - public override Expression>>> CompileSingle() + public override Expression>>> CompileSingle() { //REVIEW: why is it called CompileSingle ?? return FdbExpressionHelpers.ToTask(CompileSequence()); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs b/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs index 975a8e489..a9102abcf 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQuerySingleExpression.cs @@ -28,9 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Expressions { - using FoundationDB.Async; - using FoundationDB.Client; - using JetBrains.Annotations; using System; using System.Diagnostics.Contracts; using System.Globalization; @@ -38,6 +35,9 @@ namespace FoundationDB.Linq.Expressions using System.Reflection; using System.Threading; using System.Threading.Tasks; + using Doxense.Linq; + using FoundationDB.Client; + using JetBrains.Annotations; /// Base class of all queries that return a single element /// Type of the elements of the source sequence @@ -45,40 +45,26 @@ namespace FoundationDB.Linq.Expressions public class FdbQuerySingleExpression : FdbQueryExpression { /// Create a new expression that returns a single result from a source sequence - public FdbQuerySingleExpression(FdbQuerySequenceExpression sequence, string name, Expression, CancellationToken, Task>> lambda) + public FdbQuerySingleExpression(FdbQuerySequenceExpression sequence, string name, Expression, CancellationToken, Task>> lambda) { Contract.Requires(sequence != null && lambda != null); this.Sequence = sequence; this.Name = name; - this.Lambda = lambda; + this.Handler = lambda; } /// Always returns - public override FdbQueryShape Shape - { - get { return FdbQueryShape.Single; } - } + public override FdbQueryShape Shape => FdbQueryShape.Single; /// Source sequence - public FdbQuerySequenceExpression Sequence - { - [NotNull] get; - private set; - } + [NotNull] + public FdbQuerySequenceExpression Sequence { get; } /// Name of this query - public string Name - { - get; - private set; - } + public string Name { get; } - /// Opeartion that is applied to and that returns a single result - public new Expression, CancellationToken, Task>> Lambda - { - [NotNull] get; - private set; - } + [NotNull] + public Expression, CancellationToken, Task>> Handler { get; } /// Apply a custom visitor to this expression public override Expression Accept(FdbQueryExpressionVisitor visitor) @@ -110,7 +96,7 @@ public override ExpressionReturns a new expression that creates an async sequence that will execute this query on a transaction [NotNull] - public override Expression>> CompileSequence() + public override Expression>> CompileSequence() { var lambda = this.Transform.Compile(); @@ -89,13 +90,13 @@ public override Expression> // (tr) => sourceEnumerable(tr).Select(lambda); - var body = FdbExpressionHelpers.RewriteCall, Func, IFdbAsyncEnumerable>>( + var body = FdbExpressionHelpers.RewriteCall, Func, IAsyncEnumerable>>( (sequence, selector) => sequence.Select(selector), FdbExpressionHelpers.RewriteCall(enumerable, prmTrans), Expression.Constant(lambda) ); - return Expression.Lambda>>(body, prmTrans); + return Expression.Lambda>>(body, prmTrans); } } diff --git a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs index 9545dea31..97716da8e 100644 --- a/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs +++ b/FoundationDB.Linq.Providers/FdbAsyncQueryable.cs @@ -28,16 +28,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq { - using FoundationDB.Async; - using FoundationDB.Client; - using FoundationDB.Layers.Indexing; - using FoundationDB.Linq.Expressions; - using FoundationDB.Linq.Providers; using System; using System.Collections.Generic; using System.Linq.Expressions; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; + using Doxense.Linq; + using FoundationDB.Client; + using FoundationDB.Layers.Indexing; + using FoundationDB.Linq.Expressions; + using FoundationDB.Linq.Providers; /// Extensions methods that help create a query expression tree public static class FdbAsyncQueryable @@ -60,7 +61,7 @@ public static class FdbAsyncQueryable /// Query that will use this database as a source public static IFdbDatabaseQueryable Query(this IFdbDatabase db) { - if (db == null) throw new ArgumentNullException("db"); + if (db == null) throw new ArgumentNullException(nameof(db)); return new FdbDatabaseQuery(db); } @@ -71,7 +72,7 @@ public static IFdbDatabaseQueryable Query(this IFdbDatabase db) /// Query that will return the keys from the specified public static IFdbAsyncSequenceQueryable> Range(this IFdbDatabaseQueryable query, KeySelectorPair range) { - if (query == null) throw new ArgumentNullException("query"); + if (query == null) throw new ArgumentNullException(nameof(query)); var expr = FdbQueryExpressions.Range(range); @@ -84,7 +85,7 @@ public static IFdbAsyncSequenceQueryable> Range(this /// Query that will return the keys that share the specified public static IFdbAsyncSequenceQueryable> RangeStartsWith(this IFdbDatabaseQueryable query, Slice prefix) { - if (query == null) throw new ArgumentNullException("query"); + if (query == null) throw new ArgumentNullException(nameof(query)); var expr = FdbQueryExpressions.RangeStartsWith(prefix); @@ -98,8 +99,8 @@ public static IFdbAsyncSequenceQueryable> RangeStarts /// Creates a new query on this index public static IFdbIndexQueryable Query(this FdbIndex index, IFdbDatabase db) { - if (index == null) throw new ArgumentNullException("index"); - if (db == null) throw new ArgumentNullException("db"); + if (index == null) throw new ArgumentNullException(nameof(index)); + if (db == null) throw new ArgumentNullException(nameof(db)); return new FdbIndexQuery(db, index); } @@ -117,8 +118,8 @@ public static IFdbAsyncSequenceQueryable Lookup(this IFdbIndex /// Projects each element of a sequence query into a new form. public static IFdbAsyncSequenceQueryable Select(this IFdbAsyncSequenceQueryable query, Expression> selector) { - if (query == null) throw new ArgumentNullException("query"); - if (selector == null) throw new ArgumentNullException("selector"); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); var sourceExpr = query.Expression as FdbQuerySequenceExpression; if (sourceExpr == null) throw new ArgumentException("query"); @@ -131,8 +132,8 @@ public static IFdbAsyncSequenceQueryable Select(this IFdbAsyncSequenceQ /// Filters a sequence query of values based on a predicate. public static IFdbAsyncSequenceQueryable Where(this IFdbAsyncSequenceQueryable query, Expression> predicate) { - if (query == null) throw new ArgumentNullException("query"); - if (predicate == null) throw new ArgumentNullException("predicate"); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (predicate == null) throw new ArgumentNullException(nameof(predicate)); var sourceExpr = query.Expression as FdbQuerySequenceExpression; if (sourceExpr == null) throw new ArgumentException("query"); @@ -143,12 +144,12 @@ public static IFdbAsyncSequenceQueryable Where(this IFdbAsyncSequenceQuery } /// Returns an async sequence that would return the results of this query as they arrive. - public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequenceQueryable query) + public static IAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequenceQueryable query) { - if (query == null) throw new ArgumentNullException("query"); + if (query == null) throw new ArgumentNullException(nameof(query)); var sequenceQuery = query as FdbAsyncSequenceQuery; - if (sequenceQuery == null) throw new ArgumentException("Source query type not supported", "query"); + if (sequenceQuery == null) throw new ArgumentException("Source query type not supported", nameof(query)); return sequenceQuery.ToEnumerable(); } @@ -156,7 +157,7 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequence /// Returns the first element of a sequence query public static Task CountAsync(this IFdbAsyncSequenceQueryable query, CancellationToken ct = default(CancellationToken)) { - if (query == null) throw new ArgumentNullException("query"); + if (query == null) throw new ArgumentNullException(nameof(query)); var expr = FdbQueryExpressions.Single( (FdbQuerySequenceExpression)query.Expression, @@ -170,8 +171,8 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequence /// Returns the first element of a sequence query public static Task FirstAsync(this IFdbAsyncSequenceQueryable query, CancellationToken ct = default(CancellationToken)) { - if (query == null) throw new ArgumentNullException("query"); - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var expr = FdbQueryExpressions.Single( (FdbQuerySequenceExpression)query.Expression, @@ -185,8 +186,8 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequence /// Returns the first element of a sequence query public static Task FirstOrDefaultAsync(this IFdbAsyncSequenceQueryable query, CancellationToken ct = default(CancellationToken)) { - if (query == null) throw new ArgumentNullException("query"); - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); var expr = FdbQueryExpressions.Single( (FdbQuerySequenceExpression)query.Expression, @@ -200,8 +201,8 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequence /// Immediately executes a sequence query and return a list of all the results once it has completed. public static Task> ToListAsync(this IFdbAsyncSequenceQueryable query, CancellationToken ct = default(CancellationToken)) { - if (query == null) throw new ArgumentNullException("query"); - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation>(ct); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (ct.IsCancellationRequested) return Task.FromCanceled>(ct); return query.Provider.ExecuteAsync>(query.Expression, ct); @@ -210,8 +211,8 @@ public static IFdbAsyncEnumerable ToAsyncEnumerable(this IFdbAsyncSequence /// Immediately executes a sequence query and return an array of all the results once it has completed. public static Task ToArrayAsync(this IFdbAsyncSequenceQueryable query, CancellationToken ct = default(CancellationToken)) { - if (query == null) throw new ArgumentNullException("query"); - if (ct.IsCancellationRequested) return TaskHelpers.FromCancellation(ct); + if (query == null) throw new ArgumentNullException(nameof(query)); + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); return query.Provider.ExecuteAsync(query.Expression, ct); } diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs index e59007947..63929e05d 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncQuery.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +using Doxense.Linq; + namespace FoundationDB.Linq.Providers { using FoundationDB.Client; @@ -57,21 +59,18 @@ protected FdbAsyncQuery([NotNull] IFdbReadOnlyTransaction trans, FdbQueryExpress } /// Query expression - public FdbQueryExpression Expression { get; private set; } + public FdbQueryExpression Expression { get; } /// Database used by the query (or null) - public IFdbDatabase Database { [CanBeNull] get; private set; } + public IFdbDatabase Database { [CanBeNull] get; } /// Transaction used by the query (or null) - public IFdbReadOnlyTransaction Transaction { [CanBeNull] get; private set; } + public IFdbReadOnlyTransaction Transaction { [CanBeNull] get; } /// Type of the elements returned by the query - public virtual Type Type { get { return this.Expression.Type; } } + public virtual Type Type => this.Expression.Type; - IFdbAsyncQueryProvider IFdbAsyncQueryable.Provider - { - get { return this; } - } + IFdbAsyncQueryProvider IFdbAsyncQueryable.Provider => this; /// Create a new query from a query expression public virtual IFdbAsyncQueryable CreateQuery(FdbQueryExpression expression) @@ -83,7 +82,7 @@ public virtual IFdbAsyncQueryable CreateQuery(FdbQueryExpression expression) /// Create a new typed query from a query expression public virtual IFdbAsyncQueryable CreateQuery([NotNull] FdbQueryExpression expression) { - if (expression == null) throw new ArgumentNullException("expression"); + if (expression == null) throw new ArgumentNullException(nameof(expression)); if (this.Transaction != null) return new FdbAsyncSingleQuery(this.Transaction, expression); @@ -94,7 +93,7 @@ public virtual IFdbAsyncQueryable CreateQuery([NotNull] FdbQueryExpression /// Create a new sequence query from a sequence expression public virtual IFdbAsyncSequenceQueryable CreateSequenceQuery([NotNull] FdbQuerySequenceExpression expression) { - if (expression == null) throw new ArgumentNullException("expression"); + if (expression == null) throw new ArgumentNullException(nameof(expression)); if (this.Transaction != null) return new FdbAsyncSequenceQuery(this.Transaction, expression); @@ -106,7 +105,7 @@ public virtual IFdbAsyncSequenceQueryable CreateSequenceQuery([NotNull] Fd /// Type of the expected result. Can be a for singleton queries or a for sequence queries public async Task ExecuteAsync([NotNull] FdbQueryExpression expression, CancellationToken ct) { - if (expression == null) throw new ArgumentNullException("ct"); + if (expression == null) throw new ArgumentNullException(nameof(ct)); ct.ThrowIfCancellationRequested(); var result = await ExecuteInternal(expression, typeof(R), ct).ConfigureAwait(false); @@ -180,7 +179,7 @@ protected virtual async Task ExecuteSingleInternal([NotNull] FdbQueryExp #region Sequence... [NotNull] - private Func> CompileSequence([NotNull] FdbQueryExpression expression) + private Func> CompileSequence([NotNull] FdbQueryExpression expression) { #if false //TODO: caching ! @@ -197,25 +196,25 @@ private Func> CompileSequence([N } [NotNull] - internal static IFdbAsyncEnumerator GetEnumerator([NotNull] FdbAsyncSequenceQuery sequence, FdbAsyncMode mode) + internal static IAsyncEnumerator GetEnumerator([NotNull] FdbAsyncSequenceQuery sequence, AsyncIterationHint mode) { var generator = sequence.CompileSequence(sequence.Expression); if (sequence.Transaction != null) { - return generator(sequence.Transaction).GetEnumerator(mode); + return generator(sequence.Transaction).GetEnumerator(sequence.Transaction.Cancellation, mode); } //BUGBUG: how do we get a CancellationToken without a transaction? var ct = CancellationToken.None; IFdbTransaction trans = null; - IFdbAsyncEnumerator iterator = null; + IAsyncEnumerator iterator = null; bool success = true; try { trans = sequence.Database.BeginTransaction(ct); - iterator = generator(trans).GetEnumerator(); + iterator = generator(trans).GetEnumerator(ct, mode); return new TransactionIterator(trans, iterator); } @@ -228,32 +227,29 @@ internal static IFdbAsyncEnumerator GetEnumerator([NotNull] FdbAsyncSequenceQ { if (!success) { - if (iterator != null) iterator.Dispose(); - if (trans != null) trans.Dispose(); + iterator?.Dispose(); + trans?.Dispose(); } } } - private sealed class TransactionIterator : IFdbAsyncEnumerator + private sealed class TransactionIterator : IAsyncEnumerator { - private readonly IFdbAsyncEnumerator m_iterator; + private readonly IAsyncEnumerator m_iterator; private readonly IFdbTransaction m_transaction; - public TransactionIterator(IFdbTransaction transaction, IFdbAsyncEnumerator iterator) + public TransactionIterator(IFdbTransaction transaction, IAsyncEnumerator iterator) { m_transaction = transaction; m_iterator = iterator; } - public Task MoveNextAsync(CancellationToken ct) + public Task MoveNextAsync() { - return m_iterator.MoveNextAsync(ct); + return m_iterator.MoveNextAsync(); } - public T Current - { - get { return m_iterator.Current; } - } + public T Current => m_iterator.Current; public void Dispose() { @@ -297,14 +293,14 @@ protected virtual async Task ExecuteSequenceInternal(FdbQueryExpression } else { - throw new InvalidOperationException(String.Format("Sequence result type {0} is not supported", resultType.Name)); + throw new InvalidOperationException($"Sequence result type {resultType.Name} is not supported"); } return result; } finally { - if (owned && trans != null) trans.Dispose(); + if (owned) trans?.Dispose(); } } diff --git a/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs b/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs index 017b5772a..7cdb45bc5 100644 --- a/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs +++ b/FoundationDB.Linq.Providers/Providers/FdbAsyncSequenceQuery.cs @@ -26,11 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Linq.Providers { using FoundationDB.Client; using FoundationDB.Linq.Expressions; using System; + using Doxense.Linq; /// Async LINQ query that returns an async sequence of items /// Type of the items in the sequence @@ -51,9 +53,9 @@ public FdbAsyncSequenceQuery(IFdbReadOnlyTransaction trans, FdbQueryExpression e public Type ElementType { get { return typeof(T); } } /// Return an async sequence that will return the results of this query - public IFdbAsyncEnumerable ToEnumerable(FdbAsyncMode mode = FdbAsyncMode.Default) + public IAsyncEnumerable ToEnumerable(AsyncIterationHint mode = AsyncIterationHint.Default) { - return FdbAsyncEnumerable.Create((_) => GetEnumerator(this, mode)); + return AsyncEnumerable.Create((_, __) => GetEnumerator(this, mode)); } } diff --git a/FoundationDB.Samples/Program.cs b/FoundationDB.Samples/Program.cs index 076eadd2f..cc1454548 100644 --- a/FoundationDB.Samples/Program.cs +++ b/FoundationDB.Samples/Program.cs @@ -8,7 +8,7 @@ namespace FoundationDB.Samples using System.Linq; using System.Threading; using System.Threading.Tasks; - using FoundationDB.Async; + using Doxense.Async; using FoundationDB.Client; using FoundationDB.Filters.Logging; using FoundationDB.Samples.Benchmarks; diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index cc14886c9..fe7cfe862 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -8,9 +8,9 @@ namespace FoundationDB.Samples.Tutorials using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Linq; using FoundationDB.Client; using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; public class ClassScheduling : IAsyncTest { diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index d1b4229b5..2af7b729c 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -26,20 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Tests.Sandbox { - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using FoundationDB.Linq; using System; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.Linq; - using System.Net; using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Linq; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; class Program { diff --git a/FoundationDB.Tests/Async/AsyncBufferFacts.cs b/FoundationDB.Tests/Async/AsyncBufferFacts.cs index 0d312b906..d69f114f8 100644 --- a/FoundationDB.Tests/Async/AsyncBufferFacts.cs +++ b/FoundationDB.Tests/Async/AsyncBufferFacts.cs @@ -26,19 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System.Diagnostics; - namespace FoundationDB.Async.Tests { - using FoundationDB.Async; - using FoundationDB.Client.Tests; - using NUnit.Framework; using System; using System.Collections.Generic; + using System.Diagnostics; using System.Linq; using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] public class AsyncBufferFacts : FdbTest diff --git a/FoundationDB.Tests/FdbTest.cs b/FoundationDB.Tests/FdbTest.cs index 1de8262b3..5323dae3e 100644 --- a/FoundationDB.Tests/FdbTest.cs +++ b/FoundationDB.Tests/FdbTest.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Layers.Directories; - using NUnit.Framework; using System; using System.Diagnostics; using System.Globalization; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Layers.Directories; + using NUnit.Framework; /// Base class for all FoundationDB tests public abstract class FdbTest diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index e526456cd..1099f30a6 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -28,15 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Linq; - using System.Text; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Layers.Tuples; + using NUnit.Framework; [TestFixture] public class KeyFacts diff --git a/FoundationDB.Tests/Layers/IndexingFacts.cs b/FoundationDB.Tests/Layers/IndexingFacts.cs index 9522cecf7..75abe0d58 100644 --- a/FoundationDB.Tests/Layers/IndexingFacts.cs +++ b/FoundationDB.Tests/Layers/IndexingFacts.cs @@ -28,16 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tables.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Layers.Indexing; - using FoundationDB.Layers.Directories; - using FoundationDB.Linq; - using NUnit.Framework; using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; using System; + using Doxense.Linq; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using FoundationDB.Layers.Indexing; + using NUnit.Framework; [TestFixture] public class IndexingFacts : FdbTest diff --git a/FoundationDB.Tests/Layers/QueuesFacts.cs b/FoundationDB.Tests/Layers/QueuesFacts.cs index a620fba64..b85708990 100644 --- a/FoundationDB.Tests/Layers/QueuesFacts.cs +++ b/FoundationDB.Tests/Layers/QueuesFacts.cs @@ -30,17 +30,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections.Tests { - using FoundationDB.Async; - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Filters.Logging; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Diagnostics; using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] public class QueuesFacts : FdbTest diff --git a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs index 185650f49..6c7585046 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs @@ -26,12 +26,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Linq.Tests { - using FoundationDB.Async; - using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Diagnostics; @@ -39,6 +36,13 @@ namespace FoundationDB.Linq.Tests using System.Runtime.ExceptionServices; using System.Threading; using System.Threading.Tasks; + using Doxense; + using Doxense.Async; + using Doxense.Linq; + using Doxense.Linq.Async.Iterators; + using FoundationDB.Client.Tests; + using FoundationDB.Layers.Tuples; + using NUnit.Framework; [TestFixture] public class FdbAsyncEnumerableFacts : FdbTest @@ -53,9 +57,9 @@ public async Task Test_Can_Convert_Enumerable_To_AsyncEnumerable() Assert.That(source, Is.Not.Null); var results = new List(); - using (var iterator = source.GetEnumerator()) + using (var iterator = source.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - while (await iterator.MoveNextAsync(CancellationToken.None)) + while (await iterator.MoveNextAsync()) { Assert.That(results.Count, Is.LessThan(10)); results.Add(iterator.Current); @@ -78,9 +82,9 @@ public async Task Test_Can_Convert_Enumerable_To_AsyncEnumerable_With_Async_Tran Assert.That(source, Is.Not.Null); var results = new List(); - using (var iterator = source.GetEnumerator()) + using (var iterator = source.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - while (await iterator.MoveNextAsync(CancellationToken.None)) + while (await iterator.MoveNextAsync()) { Assert.That(results.Count, Is.LessThan(10)); results.Add(iterator.Current); @@ -130,7 +134,7 @@ public async Task Test_Can_ToArrayAsync_Big() [Test] public async Task Test_Empty() { - var empty = FdbAsyncEnumerable.Empty(); + var empty = AsyncEnumerable.Empty(); Assert.That(empty, Is.Not.Null); var results = await empty.ToListAsync(); @@ -149,7 +153,7 @@ public async Task Test_Empty() [Test] public async Task Test_Singleton() { - var singleton = FdbAsyncEnumerable.Singleton(42); + var singleton = AsyncEnumerable.Singleton(42); Assert.That(singleton, Is.Not.Null); var results = await singleton.ToListAsync(); @@ -170,15 +174,15 @@ public async Task Test_Producer_Single() { // Func - var singleton = FdbAsyncEnumerable.Single(() => 42); + var singleton = AsyncEnumerable.Single(() => 42); Assert.That(singleton, Is.Not.Null); - using(var iterator = singleton.GetEnumerator()) + using(var iterator = singleton.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - var res = await iterator.MoveNextAsync(this.Cancellation); + var res = await iterator.MoveNextAsync(); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNextAsync(this.Cancellation); + res = await iterator.MoveNextAsync(); Assert.That(res, Is.False); } @@ -196,15 +200,15 @@ public async Task Test_Producer_Single() // Func> - singleton = FdbAsyncEnumerable.Single(() => Task.Delay(50).ContinueWith(_ => 42)); + singleton = AsyncEnumerable.Single(() => Task.Delay(50).ContinueWith(_ => 42)); Assert.That(singleton, Is.Not.Null); - using (var iterator = singleton.GetEnumerator()) + using (var iterator = singleton.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - var res = await iterator.MoveNextAsync(this.Cancellation); + var res = await iterator.MoveNextAsync(); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNextAsync(this.Cancellation); + res = await iterator.MoveNextAsync(); Assert.That(res, Is.False); } @@ -222,15 +226,15 @@ public async Task Test_Producer_Single() // Func> - singleton = FdbAsyncEnumerable.Single((ct) => Task.Delay(50, ct).ContinueWith(_ => 42)); + singleton = AsyncEnumerable.Single((ct) => Task.Delay(50, ct).ContinueWith(_ => 42)); Assert.That(singleton, Is.Not.Null); - using (var iterator = singleton.GetEnumerator()) + using (var iterator = singleton.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - var res = await iterator.MoveNextAsync(this.Cancellation); + var res = await iterator.MoveNextAsync(); Assert.That(res, Is.True); Assert.That(iterator.Current, Is.EqualTo(42)); - res = await iterator.MoveNextAsync(this.Cancellation); + res = await iterator.MoveNextAsync(); Assert.That(res, Is.False); } @@ -254,7 +258,7 @@ public async Task Test_Can_Select_Sync() var selected = source.Select(x => x + 1); Assert.That(selected, Is.Not.Null); - Assert.That(selected, Is.InstanceOf>()); + Assert.That(selected, Is.InstanceOf>()); var results = await selected.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 })); @@ -271,7 +275,7 @@ public async Task Test_Can_Select_Async() return x + 1; }); Assert.That(selected, Is.Not.Null); - Assert.That(selected, Is.InstanceOf>()); + Assert.That(selected, Is.InstanceOf>()); var results = await selected.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 })); @@ -284,11 +288,11 @@ public async Task Test_Can_Select_Multiple_Times() var squares = source.Select(x => (long)x * x); Assert.That(squares, Is.Not.Null); - Assert.That(squares, Is.InstanceOf>()); + Assert.That(squares, Is.InstanceOf>()); var roots = squares.Select(x => Math.Sqrt(x)); Assert.That(roots, Is.Not.Null); - Assert.That(roots, Is.InstanceOf>()); + Assert.That(roots, Is.InstanceOf>()); var results = await roots.ToListAsync(); Assert.That(results, Is.EqualTo(new double[] { 0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0 })); @@ -301,11 +305,11 @@ public async Task Test_Can_Select_Async_Multiple_Times() var squares = source.Select(x => Task.FromResult((long)x * x)); Assert.That(squares, Is.Not.Null); - Assert.That(squares, Is.InstanceOf>()); + Assert.That(squares, Is.InstanceOf>()); var roots = squares.Select(x => Task.FromResult(Math.Sqrt(x))); Assert.That(roots, Is.Not.Null); - Assert.That(roots, Is.InstanceOf>()); + Assert.That(roots, Is.InstanceOf>()); var results = await roots.ToListAsync(); Assert.That(results, Is.EqualTo(new double[] { 0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0 })); @@ -330,7 +334,7 @@ public async Task Test_Can_Take() var query = source.Take(10); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); + Assert.That(query, Is.InstanceOf>()); var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 })); @@ -346,7 +350,7 @@ public async Task Test_Can_Where_And_Take() .Where(x => x % 2 == 1) .Take(10); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); + Assert.That(query, Is.InstanceOf>()); var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 1, 3, 5, 7, 9, 11, 13, 15, 17, 19 })); @@ -361,7 +365,7 @@ public async Task Test_Can_Take_And_Where() .Take(10) .Where(x => x % 2 == 1); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); + Assert.That(query, Is.InstanceOf>()); var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 1, 3, 5, 7, 9 })); @@ -376,7 +380,7 @@ public async Task Test_Can_Combine_Where_Clauses() .Where(x => x % 2 == 1) .Where(x => x % 3 == 0); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); // should have been optimized + Assert.That(query, Is.InstanceOf>()); // should have been optimized var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 3, 9, 15, 21, 27, 33, 39 })); @@ -391,7 +395,7 @@ public async Task Test_Can_Skip_And_Where() .Skip(21) .Where(x => x % 2 == 1); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); + Assert.That(query, Is.InstanceOf>()); var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 21, 23, 25, 27, 29, 31, 33, 35, 37, 39, 41 })); @@ -406,7 +410,7 @@ public async Task Test_Can_Where_And_Skip() .Where(x => x % 2 == 1) .Skip(15); Assert.That(query, Is.Not.Null); - Assert.That(query, Is.InstanceOf>()); // should be optimized + Assert.That(query, Is.InstanceOf>()); // should be optimized var results = await query.ToListAsync(); Assert.That(results, Is.EqualTo(new int[] { 31, 33, 35, 37, 39, 41 })); @@ -431,7 +435,7 @@ public async Task Test_Can_Get_First() int first = await source.FirstAsync(); Assert.That(first, Is.EqualTo(42)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.FirstAsync().GetAwaiter().GetResult(), Throws.InstanceOf()); } @@ -443,7 +447,7 @@ public async Task Test_Can_Get_FirstOrDefault() int first = await source.FirstOrDefaultAsync(); Assert.That(first, Is.EqualTo(42)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); first = await source.FirstOrDefaultAsync(); Assert.That(first, Is.EqualTo(0)); @@ -456,7 +460,7 @@ public async Task Test_Can_Get_Single() int first = await source.SingleAsync(); Assert.That(first, Is.EqualTo(42)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.SingleAsync().GetAwaiter().GetResult(), Throws.InstanceOf()); source = Enumerable.Range(42, 3).ToAsyncEnumerable(); @@ -471,7 +475,7 @@ public async Task Test_Can_Get_SingleOrDefault() int first = await source.SingleOrDefaultAsync(); Assert.That(first, Is.EqualTo(42)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); first = await source.SingleOrDefaultAsync(); Assert.That(first, Is.EqualTo(0)); @@ -486,7 +490,7 @@ public async Task Test_Can_Get_Last() int first = await source.LastAsync(); Assert.That(first, Is.EqualTo(44)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.LastAsync().GetAwaiter().GetResult(), Throws.InstanceOf()); } @@ -498,7 +502,7 @@ public async Task Test_Can_Get_LastOrDefault() int first = await source.LastOrDefaultAsync(); Assert.That(first, Is.EqualTo(44)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); first = await source.LastOrDefaultAsync(); Assert.That(first, Is.EqualTo(0)); @@ -522,7 +526,7 @@ public async Task Test_Can_Get_ElementAt() Assert.That(() => source.ElementAtAsync(10).GetAwaiter().GetResult(), Throws.InstanceOf()); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.ElementAtAsync(0).GetAwaiter().GetResult(), Throws.InstanceOf()); } @@ -545,7 +549,7 @@ public async Task Test_Can_Get_ElementAtOrDefault() item = await source.ElementAtOrDefaultAsync(10); Assert.That(item, Is.EqualTo(0)); - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); item = await source.ElementAtOrDefaultAsync(0); Assert.That(item, Is.EqualTo(0)); item = await source.ElementAtOrDefaultAsync(42); @@ -627,7 +631,7 @@ public async Task Test_Can_Any() any = await source.AnyAsync(); Assert.That(any, Is.True); - any = await FdbAsyncEnumerable.Empty().AnyAsync(); + any = await AsyncEnumerable.Empty().AnyAsync(); Assert.That(any, Is.False); } @@ -642,7 +646,7 @@ public async Task Test_Can_Any_With_Predicate() any = await source.AnyAsync(x => x < 0); Assert.That(any, Is.False); - any = await FdbAsyncEnumerable.Empty().AnyAsync(x => x == 42); + any = await AsyncEnumerable.Empty().AnyAsync(x => x == 42); Assert.That(any, Is.False); } @@ -657,7 +661,7 @@ public async Task Test_Can_None() none = await source.NoneAsync(); Assert.That(none, Is.False); - none = await FdbAsyncEnumerable.Empty().NoneAsync(); + none = await AsyncEnumerable.Empty().NoneAsync(); Assert.That(none, Is.True); } @@ -672,7 +676,7 @@ public async Task Test_Can_None_With_Predicate() any = await source.NoneAsync(x => x < 0); Assert.That(any, Is.True); - any = await FdbAsyncEnumerable.Empty().NoneAsync(x => x == 42); + any = await AsyncEnumerable.Empty().NoneAsync(x => x == 42); Assert.That(any, Is.True); } @@ -719,7 +723,7 @@ public async Task Test_Can_Min() Assert.That(min, Is.EqualTo(items.Min())); // empty should fail - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.MinAsync().GetAwaiter().GetResult(), Throws.InstanceOf()); } @@ -746,7 +750,7 @@ public async Task Test_Can_Max() Assert.That(max, Is.EqualTo(items.Max())); // empty should fail - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); Assert.That(() => source.MaxAsync().GetAwaiter().GetResult(), Throws.InstanceOf()); } @@ -763,7 +767,7 @@ public async Task Test_Can_Sum_Signed() Assert.That(sum, Is.EqualTo(expected)); // empty should return 0 - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); sum = await source.SumAsync(); Assert.That(sum, Is.EqualTo(0)); } @@ -781,7 +785,7 @@ public async Task Test_Can_Sum_Unsigned() Assert.That(sum, Is.EqualTo(expected)); // empty should return 0 - source = FdbAsyncEnumerable.Empty(); + source = AsyncEnumerable.Empty(); sum = await source.SumAsync(); Assert.That(sum, Is.EqualTo(0)); } @@ -894,7 +898,7 @@ public async Task Test_Can_Batch() // empty - query = FdbAsyncEnumerable.Empty().Batch(20); + query = AsyncEnumerable.Empty().Batch(20); Assert.That(query, Is.Not.Null); results = await query.ToListAsync(); @@ -907,7 +911,7 @@ public async Task Test_Can_Window() // generate a source that stalls every 13 items, from 0 to 49 - var source = new FdbAnonymousAsyncGenerator((index, ct) => + var source = new AnonymousAsyncGenerator((index, ct) => { if (index >= 50) return Task.FromResult(Maybe.Nothing()); if (index % 13 == 0) return Task.Delay(100).ContinueWith((_) => Maybe.Return((int)index)); @@ -954,7 +958,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() Console.WriteLine("CONSTANT LATENCY GENERATOR:"); // this iterator waits on each item produced - var source = new FdbAnonymousAsyncGenerator((index, ct) => + var source = new AnonymousAsyncGenerator((index, ct) => { Interlocked.Increment(ref called); if (index >= 10) return Task.FromResult(Maybe.Nothing()); @@ -1004,7 +1008,7 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() Console.WriteLine("BURSTY GENERATOR:"); // this iterator produce burst of items - var source = new FdbAnonymousAsyncGenerator((index, ct) => + var source = new AnonymousAsyncGenerator((index, ct) => { Interlocked.Increment(ref called); if (index >= 10) return Task.FromResult(Maybe.Nothing()); @@ -1116,21 +1120,21 @@ public async Task Test_Exceptions_Are_Propagated_To_Caller() return x; }); - using (var iterator = query.GetEnumerator()) + using (var iterator = query.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { // first move next should succeed - bool res = await iterator.MoveNextAsync(CancellationToken.None); + bool res = await iterator.MoveNextAsync(); Assert.That(res, Is.True); // second move next should fail - var x = Assert.Throws(async () => await iterator.MoveNextAsync(CancellationToken.None), "Should have failed"); + var x = Assert.Throws(async () => await iterator.MoveNextAsync(), "Should have failed"); Assert.That(x.Message, Is.EqualTo("KABOOM")); // accessing current should rethrow the exception Assert.That(() => iterator.Current, Throws.InstanceOf()); // another attempt at MoveNext should fail immediately but with a different error - Assert.Throws(async () => await iterator.MoveNextAsync(CancellationToken.None)); + Assert.Throws(async () => await iterator.MoveNextAsync()); } } @@ -1190,7 +1194,7 @@ public async Task Test_Parallel_Select_Async() throw; } }, - new FdbParallelQueryOptions { MaxConcurrency = MAX_CONCURRENCY } + new ParallelAsyncQueryOptions { MaxConcurrency = MAX_CONCURRENCY } ); var results = await query.ToListAsync(token); @@ -1317,7 +1321,7 @@ public async Task Test_FdbASyncIteratorPump() #endif ct.ThrowIfCancellationRequested(); items.Add(x); - return TaskHelpers.CompletedTask; + return Task.CompletedTask; }, onCompleted: () => { @@ -1336,9 +1340,9 @@ public async Task Test_FdbASyncIteratorPump() } ); - using(var inner = source.GetEnumerator()) + using(var inner = source.GetEnumerator(this.Cancellation, AsyncIterationHint.Default)) { - var pump = new FdbAsyncIteratorPump(inner, queue); + var pump = new AsyncIteratorPump(inner, queue); Console.WriteLine("[PUMP] Start pumping on #" + Thread.CurrentThread.ManagedThreadId); sw.Start(); @@ -1456,7 +1460,7 @@ public async Task Test_AsyncLinq_vs_LinqToObject() for(int i=0;i query = SourceOfInts.ToAsyncEnumerable(); + IAsyncEnumerable query = SourceOfInts.ToAsyncEnumerable(); IEnumerable reference = SourceOfInts; IQueryable witness = Queryable.AsQueryable(SourceOfInts); diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index f52bdfbd4..c8f07de2f 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -26,15 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Linq.Expressions.Tests { + using System; + using System.Collections.Generic; + using System.Linq.Expressions; + using Doxense.Linq; using FoundationDB.Client; using FoundationDB.Layers.Indexing; using FoundationDB.Layers.Tuples; using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Linq.Expressions; [TestFixture] public class FdbQueryExpressionFacts @@ -59,7 +61,7 @@ public void Test_FdbQueryIndexLookupExpression() Assert.That(expr.Value, Is.Not.Null); Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -81,7 +83,7 @@ public void Test_FdbQueryIndexLookupExpression_From_Lambda() Assert.That(expr.Value, Is.Not.Null); Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -100,7 +102,7 @@ public void Test_FdbQueryRangeExpression() Assert.That(expr.Range.Begin.Key.ToString(), Is.EqualTo("<02>Foo<00>")); Assert.That(expr.Range.End.Key.ToString(), Is.EqualTo("<02>Foo<01>")); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable>))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -130,7 +132,7 @@ public void Test_FdbQueryIntersectExpression() Assert.That(expr.Terms[0], Is.SameAs(expr1)); Assert.That(expr.Terms[1], Is.SameAs(expr2)); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -160,7 +162,7 @@ public void Test_FdbQueryUnionExpression() Assert.That(expr.Terms[0], Is.SameAs(expr1)); Assert.That(expr.Terms[1], Is.SameAs(expr2)); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -179,7 +181,7 @@ public void Test_FdbQueryTransformExpression() Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); Assert.That(expr.Transform, Is.Not.Null); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); Assert.That(expr.ElementType, Is.EqualTo(typeof(string))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); @@ -198,7 +200,7 @@ public void Test_FdbQueryFilterExpression() Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); Assert.That(expr.Filter, Is.Not.Null); - Assert.That(expr.Type, Is.EqualTo(typeof(IFdbAsyncEnumerable>))); + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 8bf627551..930a7aea7 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -28,17 +28,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Filters.Logging; - using FoundationDB.Layers.Tuples; - using FoundationDB.Layers.Directories; - using FoundationDB.Linq; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Diagnostics; using System.Linq; - using System.Text; using System.Threading.Tasks; + using Doxense.Linq; + using Doxense.Linq.Async.Iterators; + using FoundationDB.Layers.Directories; + using FoundationDB.Layers.Tuples; + using NUnit.Framework; [TestFixture] public class RangeQueryFacts : FdbTest @@ -519,7 +518,7 @@ public async Task Test_Can_MergeSort() ); Assert.That(merge, Is.Not.Null); - Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); + Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); var results = await merge.ToListAsync(); Assert.That(results, Is.Not.Null); @@ -589,7 +588,7 @@ public async Task Test_Range_Intersect() ); Assert.That(merge, Is.Not.Null); - Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); + Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); var results = await merge.ToListAsync(); Assert.That(results, Is.Not.Null); @@ -661,7 +660,7 @@ public async Task Test_Range_Except() ); Assert.That(merge, Is.Not.Null); - Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); + Assert.That(merge, Is.InstanceOf, int, KeyValuePair>>()); var results = await merge.ToListAsync(); Assert.That(results, Is.Not.Null); From c557c827a54dc3118c1516dca0ff555d360d09a6 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 15:59:16 +0200 Subject: [PATCH 068/153] Fixed indentation --- README.md | 97 +++++++++++++++++++++++++++---------------------------- 1 file changed, 48 insertions(+), 49 deletions(-) diff --git a/README.md b/README.md index 616f0e679..64cc74660 100644 --- a/README.md +++ b/README.md @@ -10,52 +10,52 @@ How to use ```CSharp -// note: most operations require a valid CancellationToken, which you need to provide -CancellationToken token = ....; // host-provided cancellation token +// note: most operations require a valid CancellationToken, which you need to obtain from the context (HTTP request, component lifetime, timeout, ...) +CancellationToken cancel = ....; // Connect to the db "DB" using the default cluster file using (var db = await Fdb.OpenAsync()) { // we will use a "Test" directory to isolate our test data - var location = await db.Directory.CreateOrOpenAsync("Test", token); - // this location will remember the allocated prefix, and - // automatically add it as a prefix to all our keys + var location = await db.Directory.CreateOrOpenAsync("Test", cancel); + // this location will remember the allocated prefix, and + // automatically add it as a prefix to all our keys // we need a transaction to be able to make changes to the db - // note: production code should use "db.WriteAsync(..., token)" instead - using (var trans = db.BeginTransaction(token)) + // note: production code should use "db.WriteAsync(..., cancel)" instead + using (var trans = db.BeginTransaction(cancel)) { - // For our convenience, we will use the Tuple Encoding format for our keys, - // which is accessible via the "location.Keys" helper. We could have used - // any other encoding for the keys. Tuples are simple to use and have some - // intereseting ordering properties that make it easy to work with. - // => All our keys will be encoded as the packed tuple ({Test}, "foo"), - // making them very nice and compact. We could also use integers or GUIDs - // for the keys themselves. - + // For our convenience, we will use the Tuple Encoding format for our keys, + // which is accessible via the "location.Keys" helper. We could have used + // any other encoding for the keys. Tuples are simple to use and have some + // intereseting ordering properties that make it easy to work with. + // => All our keys will be encoded as the packed tuple ({Test}, "foo"), + // making them very nice and compact. We could also use integers or GUIDs + // for the keys themselves. + // Set "Hello" key to "World" - trans.Set( - location.Keys.Encode("Hello"), - Slice.FromString("World") // UTF-8 encoded string - ); + trans.Set( + location.Keys.Encode("Hello"), + Slice.FromString("World") // UTF-8 encoded string + ); // Set "Count" key to 42 trans.Set( - location.Keys.Encode("Count"), - Slice.FromInt32(42) // 1 byte - ); + location.Keys.Encode("Count"), + Slice.FromInt32(42) // 1 byte + ); // Atomically add 123 to "Total" trans.AtomicAdd( - location.Keys.Encode("Total"), - Slice.FromFixed32(123) // 4 bytes, Little Endian - ); + location.Keys.Encode("Total"), + Slice.FromFixed32(123) // 4 bytes, Little Endian + ); // Set bits 3, 9 and 30 in the bit map stored in the key "Bitmap" trans.AtomicOr( - location.Keys.Encode("Bitmap"), - Slice.FromFixed32((1 << 3) | (1 << 9) | (1 << 30)) // 4 bytes, Little Endian - ); + location.Keys.Encode("Bitmap"), + Slice.FromFixed32((1 << 3) | (1 << 9) | (1 << 30)) // 4 bytes, Little Endian + ); // commit the changes to the db await trans.CommitAsync(); @@ -64,9 +64,9 @@ using (var db = await Fdb.OpenAsync()) } // we also need a transaction to read from the db - // note: production code should use "db.ReadAsync(..., token)" instead. - using (var trans = db.BeginReadOnlyTransaction(token)) - { + // note: production code should use "db.ReadAsync(..., cancel)" instead. + using (var trans = db.BeginReadOnlyTransaction(cancel)) + { // Read ("Test", "Hello", ) as a string Slice value = await trans.GetAsync(location.Keys.Encode("Hello")); Console.WriteLine(value.ToUnicode()); // -> World @@ -74,7 +74,7 @@ using (var db = await Fdb.OpenAsync()) // Read ("Test", "Count", ) as an int value = await trans.GetAsync(location.Keys.Encode("Count")); Console.WriteLine(value.ToInt32()); // -> 42 - + // missing keys give a result of Slice.Nil, which is the equivalent // of "key not found". value = await trans.GetAsync(location.Keys.Encode("NotFound")); @@ -92,13 +92,13 @@ using (var db = await Fdb.OpenAsync()) // First we will create a subdirectory for our little array, // just so that is does not interfere with other things in the cluster. - var list = await location.CreateOrOpenAsync(db, "List", token); + var list = await location.CreateOrOpenAsync(db, "List", cancel); - // here we will use db.WriteAsync(...) that implements a retry loop. - // this helps protect you against intermitent failures by automatically - // retrying the lambda method you provided. - await db.WriteAsync((trans) => - { + // here we will use db.WriteAsync(...) that implements a retry loop. + // this helps protect you against intermitent failures by automatically + // retrying the lambda method you provided. + await db.WriteAsync((trans) => + { // add some data to the list with the format: (..., index) = value trans.Set(list.Keys.Encode(0), Slice.FromString("AAA")); trans.Set(list.Keys.Encode(1), Slice.FromString("BBB")); @@ -129,8 +129,8 @@ using (var db = await Fdb.OpenAsync()) // If something goes wrong with the database, this lambda will be called again, // until the problems goes away, or the retry loop decides that there is no point // in retrying anymore, and the exception will be re-thrown. - - }, token); // don't forget the cancellation token, which can stop the retry loop ! + + }, cancel); // don't forget the CancellationToken, which can stop the retry loop ! // We can read everything back in one shot, using an async "LINQ" query. var results = await db.QueryAsync((trans) => @@ -138,7 +138,7 @@ using (var db = await Fdb.OpenAsync()) // do a range query on the list subspace, which should return all the pairs // in the subspace, one for each entry in the array. // We exploit the fact that subspace.Tuples.ToRange() usually does not include - // the subspace prefix itself, because we don't want our counter to be returned + // the subspace prefix itself, because we don't want our counter to be returned // with the query itself. return trans // ask for all keys that are _inside_ our subspace @@ -157,11 +157,11 @@ using (var db = await Fdb.OpenAsync()) // fetch ALL the values from the db! .Where((kvp) => kvp.Key % 2 == 0); - // note that QueryAsync() is a shortcut for calling ReadAsync(...) and then - // calling ToListAsync() on the async LINQ Query. If you want to call a - // different operator than ToListAsync(), just use ReadAsync() + // note that QueryAsync() is a shortcut for calling ReadAsync(...) and then + // calling ToListAsync() on the async LINQ Query. If you want to call a + // different operator than ToListAsync(), just use ReadAsync() - }, token); + }, cancel); // results.Count -> 2 // results[0] -> KeyValuePair(0, "AAA") @@ -188,14 +188,14 @@ Please note that the above sample is ok for a simple HelloWorld.exe app, but for - You should NEVER block on Tasks by using .Wait() from non-async code. This will either dead-lock your application, or greatly degrade the performances. If you cannot do otherwise (ex: top-level call in a `void Main()` then at least wrap your code inside a `static async Task MainAsync(string[] args)` method, and do a `MainAsync(args).GetAwaiter().GetResult()`. -- Don't give in, and resist the tentation of passing `CancellationToken.None` everywhere! Try to obtain a valid `CancellationToken` from your execution context (HTTP host, Task Worker environment, ...). This will allow the environment to safely shutdown and abort all pending transactions, without any risks of data corruption. If you don't have any easy source (like in a unit test framework), then at list provide you own using a global `CancellationTokenSource` that you can `Cancel()` in your shutdown code path. From inside your transactional code, you can get back the token anytime via the `tr.Cancellation` property which will trigger if the transaction completes or is aborted. +- Don't give in, and resist the tenmptation of passing `CancellationToken.None` everywhere! Try to obtain a valid `CancellationToken` from your execution context (HTTP host, Task Worker environment, ...). This will allow the environment to safely shutdown and abort all pending transactions, without any risks of data corruption. If you don't have any easy source (like in a unit test framework), then at list provide you own using a global `CancellationTokenSource` that you can `Cancel()` in your shutdown code path. From inside your transactional code, you can get back the token anytime via the `tr.Cancellation` property which will trigger if the transaction completes or is aborted. How to build ------------ ### Visual Studio Solution -You will need Visual Studio .NET 2012 or 2013 and .NET 4.6.1 minimum to compile the solution. +You will need Visual Studio .NET 2017 and .NET 4.6.1 minimum to compile the solution. You will also need to obtain the 'fdb_c.dll' C API binding from the foundationdb.com wesite, by installing the client SDK: @@ -254,7 +254,7 @@ Hosting on IIS * The underlying client library will not run on a 32-bit Application Pool. You will need to move your web application to a 64-bit Application Pool. * If you are using IIS Express with an ASP.NET or ASP.NET MVC application from Visual Studio, you need to configure your IIS Express instance to run in 64-bit. With Visual Studio 2013, this can be done by checking Tools | Options | Projects and Solutions | Web Projects | Use the 64 bit version of IIS Express for web sites and projects * The fdb_c.dll library can only be started once per process. This makes impractical to run an web application running inside a dedicated Application Domain alongside other application, on a shared host process. See http://community.foundationdb.com/questions/1146/using-foundationdb-in-a-webapi-2-project for more details. The only current workaround is to have a dedicated host process for this application, by making it run inside its own Application Pool. -* If you don't use the host's cancellation token for transactions and retry loops, deadlock can occur if the FoundationDB cluster is unavailable or under very heavy load. Please consider also using safe values for the DefaultTimeout and DefaultRetryLimit settings. +* If you don't use the host's CancellationToken for transactions and retry loops, deadlock can occur if the FoundationDB cluster is unavailable or under very heavy load. Please consider also using safe values for the DefaultTimeout and DefaultRetryLimit settings. Hosting on OWIN --------------- @@ -298,4 +298,3 @@ Contributing ------------ * It is important to point out that this solution uses tabs instead of spaces for various reasons. In order to ease the transition for people who want to start contributing and avoid having to switch their Visual Studio configuration manually an .editorconfig file has been added to the root folder of the solution. The easiest way to use this is to install the [Extension for Visual Studio](http://visualstudiogallery.msdn.microsoft.com/c8bccfe2-650c-4b42-bc5c-845e21f96328). This will switch visual studio's settings for white space in csharp files to use tabs. - From 742a0903a190c101a5e7b85a550ffb383d05b0c0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 21 Apr 2018 16:06:40 +0200 Subject: [PATCH 069/153] Updated all links to the new web site --- README.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index 64cc74660..493423dcd 100644 --- a/README.md +++ b/README.md @@ -180,7 +180,7 @@ Please note that the above sample is ok for a simple HelloWorld.exe app, but for - You should NOT open a new connection (`Fdb.OpenAsync()`) everytime you need to read or write something. You should open a single database instance somewhere in your startup code, and use that instance everywhere. If you are using a Repository pattern, you can store the IFdbDatabase instance there. Another option is to use a Dependency Injection framework -- You should probably not create and transactions yourself (`db.CreateTransaction()`), and instead prefer using the standard retry loops implemented by `db.ReadAsync(...)`, `db.WriteAsync(...)` and `db.ReadWriteAsync(...)` which will handle all the gory details for you. They will ensure that your transactions are retried in case of conflicts or transient errors. See https://foundationdb.com/key-value-store/documentation/developer-guide.html#conflict-ranges +- You should probably not create and transactions yourself (`db.CreateTransaction()`), and instead prefer using the standard retry loops implemented by `db.ReadAsync(...)`, `db.WriteAsync(...)` and `db.ReadWriteAsync(...)` which will handle all the gory details for you. They will ensure that your transactions are retried in case of conflicts or transient errors. See https://apple.github.io/foundationdb/developer-guide.html#conflict-ranges - Use the `Tuple Layer` to encode and decode your keys, if possible. This will give you a better experience overall, since all the logging filters and key formatters will try to decode tuples by default, and display `(42, "hello", true)` instead of the cryptic `<15>*<02>hello<00><15><01>`. For simple values like strings (ex: JSON text) or 32-bit/64-bit numbers, you can also use `Slice.FromString(...)`, or `Slice.FromInt32(...)`. For composite values, you can also use the Tuple encoding, if the elements types are simple (string, numbers, dates, ...). You can also use custom encoders via the `IKeyEncoder` and `IValueEncoder`, which you can get from the helper class `KeyValueEncoders`, or roll your own by implementing these interfaces. @@ -197,9 +197,9 @@ How to build You will need Visual Studio .NET 2017 and .NET 4.6.1 minimum to compile the solution. -You will also need to obtain the 'fdb_c.dll' C API binding from the foundationdb.com wesite, by installing the client SDK: +You will also need to obtain the 'fdb_c.dll' C API binding from the foundationdb.org wesite, by installing the client SDK: -* Go to http://foundationdb.com/get/ and download the Windows x64 MSI. You can use the free Community edition that gives you unlimited server processes for development and testing. +* Go to https://www.foundationdb.org/download/ and download the Windows x64 MSI. You can use the free Community edition that gives you unlimited server processes for development and testing. * Install the MSI, selecting the default options. * Go to `C:\Program Files\foundationdb\bin\` and make sure that `fdb_c.dll` is there. * Open the FoundationDb.Client.sln file in Visual Studio 2012. @@ -228,7 +228,7 @@ If you get `System.UnauthorizedAccessException: Access to the path './build/outp When building for Mono/Linux this version will look for `libfdb_c.so` instead of `fdb_c.dll`. -More details on running FoundationDB on Linux can be found here: https://foundationdb.com/key-value-store/documentation/getting-started-linux.html +More details on running FoundationDB on Linux can be found here: https://apple.github.io/foundationdb/getting-started-linux.html How to build the NuGet packages ------------------------------- @@ -253,7 +253,7 @@ Hosting on IIS * The .NET API is async-only, and should only be called inside async methods. You should NEVER write something like `tr.GetAsync(...).Wait()` or 'tr.GetAsync(...).Result' because it will GREATLY degrade performances and prevent you from scaling up past a few concurrent requests. * The underlying client library will not run on a 32-bit Application Pool. You will need to move your web application to a 64-bit Application Pool. * If you are using IIS Express with an ASP.NET or ASP.NET MVC application from Visual Studio, you need to configure your IIS Express instance to run in 64-bit. With Visual Studio 2013, this can be done by checking Tools | Options | Projects and Solutions | Web Projects | Use the 64 bit version of IIS Express for web sites and projects -* The fdb_c.dll library can only be started once per process. This makes impractical to run an web application running inside a dedicated Application Domain alongside other application, on a shared host process. See http://community.foundationdb.com/questions/1146/using-foundationdb-in-a-webapi-2-project for more details. The only current workaround is to have a dedicated host process for this application, by making it run inside its own Application Pool. +* The fdb_c.dll library can only be started once per process. This makes impractical to run an web application running inside a dedicated Application Domain alongside other application, on a shared host process. The only current workaround is to have a dedicated host process for this application, by making it run inside its own Application Pool. * If you don't use the host's CancellationToken for transactions and retry loops, deadlock can occur if the FoundationDB cluster is unavailable or under very heavy load. Please consider also using safe values for the DefaultTimeout and DefaultRetryLimit settings. Hosting on OWIN @@ -265,7 +265,7 @@ Hosting on OWIN Implementation Notes -------------------- -Please refer to http://foundationdb.com/documentation/ to get an overview on the FoundationDB API, if you haven't already. +Please refer to https://apple.github.io/foundationdb/ to get an overview on the FoundationDB API, if you haven't already. This .NET binding has been modeled to be as close as possible to the other bindings (Python especially), while still having a '.NET' style API. @@ -292,7 +292,7 @@ Known Limitations * The LINQ API is still a work in progress, and may change a lot. Simple LINQ queries, like Select() or Where() on the result of range queries (to convert Slice key/values into oter types) should work. * You cannot unload the fdb C native client from the process once the netork thread has started. You can stop the network thread once, but it does not support being restarted. This can cause problems when running under ASP.NET. * FoundationDB does not support long running batch or range queries if they take too much time. Such queries will fail with a 'past_version' error. -* See https://foundationdb.com/documentation/known-limitations.html for other known limitations of the FoundationDB database. +* See https://apple.github.io/foundationdb/known-limitations.html for other known limitations of the FoundationDB database. Contributing ------------ From 80f4312c5aaf928c89634cad3767d964f6a93a81 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 11:26:44 +0200 Subject: [PATCH 070/153] [BREAKING API CHANGE] Upgraded the low level framework (Slice, Tuples, ...) to the latest version, and changed a lot of stuff! - Slice has been upgraded and optimized over the years. It can support interop with ReadOnlySpan but is currently #if'd out because it would require a dependency to System.memory - Tuples have been refactored as well. The encoding feature has been extracted from the core STuple<..> structs (which are similiar to ValueTuple<...> but more vector-like) - TuPack is the core serializer for the FDB Tuple wire format. - Subspaces and Key Encoders have been refactored into IKeySubspace for the base "no encoding" subspace, and IDynamicKeySubspace vs ITypedKeySubspace<...> for subspaces paired with an encoding scheme. - The whole Async LINQ library has been updated (perf and bugfix). The IAsyncEnumerable interface has also been changed to match the current state of the art. - Maybe has also been updated --- FdbShell/Commands/BasicCommands.cs | 14 +- FdbShell/Commands/FdbCliCommands.cs | 20 +- FdbShell/Program.cs | 6 +- ...onversionHelper.cs => ComparisonHelper.cs} | 17 +- .../{IFdbConverter.cs => ITypeConverter.cs} | 14 +- .../Converters/SimilarValueComparer.cs | 2 +- .../{FdbConverters.cs => TypeConverters.cs} | 423 +- FoundationDB.Client/Fdb.System.cs | 39 +- FoundationDB.Client/Fdb.cs | 4 +- FoundationDB.Client/FdbDatabase.cs | 1 + FoundationDB.Client/FdbKey.cs | 13 +- .../Filters/FdbDatabaseFilter.cs | 2 +- .../Logging/FdbTransactionLog.Commands.cs | 16 +- .../FoundationDB.Client.csproj | 38 +- .../Layers/Directories/FdbDirectoryLayer.cs | 21 +- .../Directories/FdbDirectoryPartition.cs | 7 +- .../Directories/FdbDirectorySubspace.cs | 8 +- .../Tuples/Encoding/ITupleSerializable.cs | 41 + .../Layers/Tuples/Encoding/PrefixedTuple.cs | 68 +- .../Layers/Tuples/Encoding/SlicedTuple.cs | 59 +- .../Layers/Tuples/Encoding/TupleCodec`1.cs | 14 +- .../Layers/Tuples/Encoding/TupleEncoder.cs | 975 +++++ .../TupleKeyEncoder.cs | 57 +- .../TupleKeyEncoding.cs | 5 +- .../Layers/Tuples/Encoding/TuplePacker.cs | 44 +- .../Layers/Tuples/Encoding/TuplePackers.cs | 873 +++- .../Layers/Tuples/Encoding/TupleParser.cs | 252 +- .../Layers/Tuples/Encoding/TupleReader.cs | 22 +- .../Layers/Tuples/Encoding/TupleTypes.cs | 15 +- .../Layers/Tuples/Encoding/TupleWriter.cs | 15 +- .../Layers/Tuples/FdbTupleAlias.cs | 2 +- .../Formatters/AnonymousTupleFormatter.cs | 14 +- .../Formatters/FormattableTupleFormatter.cs | 7 +- .../Formatters/GenericTupleFormatter.cs | 4 +- .../Tuples/Formatters/ITupleFormattable.cs | 7 +- .../Tuples/Formatters/ITupleFormatter.cs | 6 +- .../Tuples/Formatters/TupleFormatter.cs | 18 +- FoundationDB.Client/Layers/Tuples/ITuple.cs | 61 +- .../Layers/Tuples/JoinedTuple.cs | 77 +- .../Layers/Tuples/LinkedTuple.cs | 78 +- .../Layers/Tuples/ListTuple.cs | 100 +- .../Layers/Tuples/MemoizedTuple.cs | 22 +- FoundationDB.Client/Layers/Tuples/STuple.cs | 1786 ++++---- FoundationDB.Client/Layers/Tuples/STuple`1.cs | 230 +- FoundationDB.Client/Layers/Tuples/STuple`2.cs | 259 +- FoundationDB.Client/Layers/Tuples/STuple`3.cs | 285 +- FoundationDB.Client/Layers/Tuples/STuple`4.cs | 291 +- FoundationDB.Client/Layers/Tuples/STuple`5.cs | 338 +- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 484 +++ FoundationDB.Client/Layers/Tuples/TuPack.cs | 964 +++++ .../Layers/Tuples/TupleComparisons.cs | 37 +- .../Layers/Tuples/TupleExtensions.cs | 349 +- .../Layers/Tuples/TupleHelpers.cs | 235 ++ .../Layers/Tuples/TupleSerializer.cs | 138 + FoundationDB.Client/Native/FdbNative.cs | 2 +- .../Subspaces/DynamicKeySubspace.cs | 12 +- .../Subspaces/FdbDatabasePartition.cs | 2 +- FoundationDB.Client/Subspaces/IKeySubspace.cs | 2 + FoundationDB.Client/Subspaces/KeySubspace.cs | 6 +- .../Subspaces/TypedKeySubspace`1.cs | 7 +- .../Subspaces/TypedKeySubspace`2.cs | 7 +- .../Subspaces/TypedKeySubspace`3.cs | 7 +- .../Subspaces/TypedKeySubspace`4.cs | 7 +- .../Encoders/DynamicKeyEncoderBase.cs | 36 +- .../TypeSystem/Encoders/KeyValueEncoders.cs | 136 +- .../TypeSystem/ICompositeKeyEncoder.cs | 8 +- .../TypeSystem/IDynamicKeyEncoder.cs | 10 +- .../TypeSystem/IOrderedTypeCodec.cs | 2 + .../TypeSystem/IUnorderedTypeCodec.cs | 1 + FoundationDB.Client/TypeSystem/TypeCodec`1.cs | 5 +- FoundationDB.Client/TypeSystem/TypeSystem.cs | 5 +- FoundationDB.Client/Utils/Batched.cs | 1 + .../Utils/ExceptionExtensions.cs | 33 +- FoundationDB.Client/Utils/HashCodes.cs | 216 + .../Utils/Memory/BitHelpers.cs | 777 ++++ .../Utils/Memory/Slice.Comparer.cs | 89 + .../Utils/Memory/Slice.Encoding.cs | 2504 ++++++++++++ FoundationDB.Client/Utils/Memory/Slice.cs | 2519 ++++++++++++ .../Utils/{ => Memory}/SliceBuffer.cs | 132 +- .../Utils/{ => Memory}/SliceListStream.cs | 71 +- .../Utils/{ => Memory}/SlicePairComparer.cs | 5 +- .../Utils/Memory/SliceReader.cs | 423 ++ .../Utils/{ => Memory}/SliceStream.cs | 55 +- .../Utils/Memory/SliceWriter.cs | 2295 +++++++++++ .../Utils/Memory/UnsafeHelpers.cs | 3589 +++++++++++++++++ FoundationDB.Client/Utils/Slice.cs | 2508 ------------ FoundationDB.Client/Utils/SliceComparer.cs | 79 - FoundationDB.Client/Utils/SliceHelpers.cs | 734 ---- FoundationDB.Client/Utils/SliceReader.cs | 282 -- FoundationDB.Client/Utils/SliceWriter.cs | 879 ---- FoundationDB.Client/Utils/StringConverters.cs | 736 ++++ FoundationDB.Client/Utils/Uuid128.cs | 332 +- FoundationDB.Client/Utils/Uuid64.cs | 896 +++- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 2 +- .../Collections/FdbQueue`1.cs | 6 +- .../Indexes/FdbIndex`2.cs | 6 +- .../Interning/FdbStringIntern.cs | 3 +- .../Documents/FdbDocumentHandlers.cs | 7 +- .../Documents/FdbHashSetCollection.cs | 6 +- .../Bitmaps/CompressedBitmapBuilder.cs | 5 +- .../Bitmaps/CompressedBitmapWordIterator.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmapWriter.cs | 3 +- .../Indexes/Bitmaps/WordAlignHybridCoding.cs | 1 + .../Messaging/FdbWorkerPool.cs | 26 +- .../Messaging/WorkerPoolTest.cs | 6 +- .../Expressions/FdbQueryExpressions.cs | 7 +- .../Benchmarks/BenchRunner.cs | 4 +- .../MessageQueue/MessageQueueRunner.cs | 20 +- .../Tutorials/ClassScheduling.cs | 14 +- FoundationDB.Tests.Sandbox/Program.cs | 25 +- FoundationDB.Tests/Async/AsyncBufferFacts.cs | 296 +- FoundationDB.Tests/DatabaseBulkFacts.cs | 11 +- FoundationDB.Tests/DatabaseFacts.cs | 33 +- FoundationDB.Tests/Encoders/EncoderFacts.cs | 24 +- FoundationDB.Tests/Encoders/TypeCodecFacts.cs | 43 +- .../Indexing/CompressedBitmapsFacts.cs | 63 +- .../Indexing/SuperSlowUncompressedBitmap.cs | 2 +- .../Experimental/JsonNetCodec.cs | 7 +- .../Experimental/ProtobufCodec.cs | 9 +- .../Filters/LoggingFilterFacts.cs | 7 +- FoundationDB.Tests/FoundationDB.Tests.csproj | 13 +- FoundationDB.Tests/KeyFacts.cs | 152 +- FoundationDB.Tests/Layers/BlobFacts.cs | 7 +- FoundationDB.Tests/Layers/CounterFacts.cs | 4 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 28 +- .../Layers/DocumentCollectionFacts.cs | 4 +- FoundationDB.Tests/Layers/MapFacts.cs | 12 +- FoundationDB.Tests/Layers/MultiMapFacts.cs | 7 +- FoundationDB.Tests/Layers/RankedSetFacts.cs | 13 +- .../Layers/StringInternFacts.cs | 4 +- FoundationDB.Tests/Layers/TupleFacts.cs | 3041 -------------- FoundationDB.Tests/Layers/VectorFacts.cs | 29 +- ...erableFacts.cs => AsyncEnumerableFacts.cs} | 5 +- ...eryableFacts.cs => AsyncQueryableFacts.cs} | 11 +- .../Linq/FdbQueryExpressionFacts.cs | 11 +- FoundationDB.Tests/RangeQueryFacts.cs | 16 +- FoundationDB.Tests/SubspaceFacts.cs | 27 +- FoundationDB.Tests/TestHelpers.cs | 5 +- FoundationDB.Tests/TransactionFacts.cs | 103 +- FoundationDB.Tests/TransactionalFacts.cs | 1 + FoundationDB.Tests/Utils/ConversionFacts.cs | 12 +- .../Utils/FdbConvertersFacts.cs | 174 - .../Utils/SliceComparerFacts.cs | 29 +- FoundationDB.Tests/Utils/SliceFacts.cs | 2092 +++++++--- FoundationDB.Tests/Utils/SliceHelperFacts.cs | 112 - FoundationDB.Tests/Utils/SliceStreamFacts.cs | 13 +- FoundationDB.Tests/Utils/SliceWriterFacts.cs | 632 ++- FoundationDB.Tests/Utils/TupleFacts.cs | 2144 ++++++++++ .../Utils/TypeConvertersFacts.cs | 176 + .../{ => Utils}/Uuid128Facts.cs | 47 +- FoundationDB.Tests/Utils/Uuid64Facts.cs | 690 ++++ FoundationDB.Tests/Uuid64Facts.cs | 420 -- FoundationDb.Client.sln.DotSettings | 3 + README.md | 4 +- 154 files changed, 26679 insertions(+), 12287 deletions(-) rename FoundationDB.Client/Converters/{ConversionHelper.cs => ComparisonHelper.cs} (96%) rename FoundationDB.Client/Converters/{IFdbConverter.cs => ITypeConverter.cs} (86%) rename FoundationDB.Client/Converters/{FdbConverters.cs => TypeConverters.cs} (59%) create mode 100644 FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs create mode 100644 FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs rename FoundationDB.Client/Layers/Tuples/{TypeSystem => Encoding}/TupleKeyEncoder.cs (84%) rename FoundationDB.Client/Layers/Tuples/{TypeSystem => Encoding}/TupleKeyEncoding.cs (94%) create mode 100644 FoundationDB.Client/Layers/Tuples/STuple`6.cs create mode 100644 FoundationDB.Client/Layers/Tuples/TuPack.cs create mode 100644 FoundationDB.Client/Layers/Tuples/TupleHelpers.cs create mode 100644 FoundationDB.Client/Layers/Tuples/TupleSerializer.cs create mode 100644 FoundationDB.Client/Utils/HashCodes.cs create mode 100644 FoundationDB.Client/Utils/Memory/BitHelpers.cs create mode 100644 FoundationDB.Client/Utils/Memory/Slice.Comparer.cs create mode 100644 FoundationDB.Client/Utils/Memory/Slice.Encoding.cs create mode 100644 FoundationDB.Client/Utils/Memory/Slice.cs rename FoundationDB.Client/Utils/{ => Memory}/SliceBuffer.cs (66%) rename FoundationDB.Client/Utils/{ => Memory}/SliceListStream.cs (83%) rename FoundationDB.Client/Utils/{ => Memory}/SlicePairComparer.cs (96%) create mode 100644 FoundationDB.Client/Utils/Memory/SliceReader.cs rename FoundationDB.Client/Utils/{ => Memory}/SliceStream.cs (88%) create mode 100644 FoundationDB.Client/Utils/Memory/SliceWriter.cs create mode 100644 FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs delete mode 100644 FoundationDB.Client/Utils/Slice.cs delete mode 100644 FoundationDB.Client/Utils/SliceComparer.cs delete mode 100644 FoundationDB.Client/Utils/SliceHelpers.cs delete mode 100644 FoundationDB.Client/Utils/SliceReader.cs delete mode 100644 FoundationDB.Client/Utils/SliceWriter.cs create mode 100644 FoundationDB.Client/Utils/StringConverters.cs delete mode 100644 FoundationDB.Tests/Layers/TupleFacts.cs rename FoundationDB.Tests/Linq/{FdbAsyncEnumerableFacts.cs => AsyncEnumerableFacts.cs} (99%) rename FoundationDB.Tests/Linq/{FdbAsyncQueryableFacts.cs => AsyncQueryableFacts.cs} (97%) delete mode 100644 FoundationDB.Tests/Utils/FdbConvertersFacts.cs delete mode 100644 FoundationDB.Tests/Utils/SliceHelperFacts.cs create mode 100644 FoundationDB.Tests/Utils/TupleFacts.cs create mode 100644 FoundationDB.Tests/Utils/TypeConvertersFacts.cs rename FoundationDB.Tests/{ => Utils}/Uuid128Facts.cs (84%) create mode 100644 FoundationDB.Tests/Utils/Uuid64Facts.cs delete mode 100644 FoundationDB.Tests/Uuid64Facts.cs diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 9c97113a6..3c6d54ffa 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -11,10 +11,10 @@ namespace FdbShell using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using FoundationDB; using FoundationDB.Client; using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; public static class BasicCommands { @@ -43,7 +43,7 @@ public static async Task Dir(string[] path, ITuple extras, DirectoryBrowseOption { if (log == null) log = Console.Out; - log.WriteLine("# Listing {0}:", String.Join("/", path)); + log.WriteLine("# Listing {0}:", string.Join("/", path)); var parent = await TryOpenCurrentDirectoryAsync(path, db, ct); if (parent == null) @@ -54,7 +54,7 @@ public static async Task Dir(string[] path, ITuple extras, DirectoryBrowseOption if (parent.Layer.IsPresent) { - log.WriteLine("# Layer: {0}", parent.Layer.ToAsciiOrHexaString()); + log.WriteLine("# Layer: {0:P}", parent.Layer); } var folders = await Fdb.Directory.BrowseAsync(db, parent, ct); @@ -121,7 +121,7 @@ public static async Task CreateDirectory(string[] path, ITuple extras, IFdbDatab if (stuff.Key.IsPresent) { log.WriteLine("CAUTION: There is already some data under {0} !"); - log.WriteLine(" {0} = {1}", FdbKey.Dump(stuff.Key), stuff.Value.ToAsciiOrHexaString()); + log.WriteLine(" {0} = {1:V}", FdbKey.Dump(stuff.Key), stuff.Value); } } @@ -187,7 +187,7 @@ public static async Task ShowDirectoryLayer(string[] path, ITuple extras, IFdbDa if (dir.Layer == FdbDirectoryPartition.LayerId) log.WriteLine("# Directory {0} is a partition", String.Join("/", path)); else if (dir.Layer.IsPresent) - log.WriteLine("# Directory {0} has layer {1}", String.Join("/", path), dir.Layer.ToAsciiOrHexaString()); + log.WriteLine("# Directory {0} has layer {1:P}", String.Join("/", path), dir.Layer); else log.WriteLine("# Directory {0} does not have a layer defined", String.Join("/", path)); } @@ -203,7 +203,7 @@ public static async Task ChangeDirectoryLayer(string[] path, string layer, ITupl else { dir = await db.ReadWriteAsync((tr) => dir.ChangeLayerAsync(tr, Slice.FromString(layer)), ct); - log.WriteLine("# Directory {0} layer changed to {1}", String.Join("/", path), dir.Layer.ToAsciiOrHexaString()); + log.WriteLine("# Directory {0} layer changed to {1:P}", String.Join("/", path), dir.Layer); } } @@ -257,7 +257,7 @@ public static async Task Show(string[] path, ITuple extras, bool reverse, IFdbDa if (reverse) keys.Reverse(); foreach (var key in keys.Take(count)) { - log.WriteLine("...{0} = {1}", FdbKey.Dump(folder.ExtractKey(key.Key)), key.Value.ToAsciiOrHexaString()); + log.WriteLine("...{0} = {1:V}", FdbKey.Dump(folder.ExtractKey(key.Key)), key.Value); } if (!reverse && keys.Count == count + 1) { diff --git a/FdbShell/Commands/FdbCliCommands.cs b/FdbShell/Commands/FdbCliCommands.cs index eab5d23bb..a62fbda25 100644 --- a/FdbShell/Commands/FdbCliCommands.cs +++ b/FdbShell/Commands/FdbCliCommands.cs @@ -1,17 +1,13 @@ -using FoundationDB.Client; -using FoundationDB.Layers.Directories; -using FoundationDB.Layers.Tuples; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.IO; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - + namespace FdbShell { + using System; + using System.Diagnostics; + using System.IO; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + public static class FdbCliCommands { diff --git a/FdbShell/Program.cs b/FdbShell/Program.cs index d7719ec29..a93d4dffd 100644 --- a/FdbShell/Program.cs +++ b/FdbShell/Program.cs @@ -29,19 +29,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FdbShell { using System; - using System.Collections.Generic; - using System.Diagnostics; using System.IO; using System.Linq; using System.Net; - using System.Text; using System.Threading; using System.Threading.Tasks; using Doxense; + using Doxense.Collections.Tuples; using FoundationDB.Client; - using FoundationDB.Filters.Logging; using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; using Mono.Options; using Mono.Terminal; diff --git a/FoundationDB.Client/Converters/ConversionHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs similarity index 96% rename from FoundationDB.Client/Converters/ConversionHelper.cs rename to FoundationDB.Client/Converters/ComparisonHelper.cs index b632ef649..5547f07e6 100644 --- a/FoundationDB.Client/Converters/ConversionHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -26,13 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Converters + +namespace Doxense.Runtime.Converters { using System; using System.Collections.Concurrent; using System.Collections.Generic; using System.Globalization; using Doxense.Diagnostics.Contracts; + using FoundationDB; using JetBrains.Annotations; /// Helper classe used to compare object of "compatible" types @@ -103,18 +105,15 @@ internal static string TryAdaptToString(object value) { if (value == null) return null; - var s = value as string; - if (s != null) return s; + if (value is string s) return s; - if (value is char) return new string((char)value, 1); + if (value is char c) return new string(c, 1); - if (value is Slice) return ((Slice) value).ToAscii(); //REVIEW: or ToUnicode() ? + if (value is Slice slice) return slice.ToStringAscii(); //REVIEW: or ToUnicode() ? - var bstr = value as byte[]; - if (bstr != null) return Slice.Create(bstr).ToAscii(); //REVIEW: or ToUnicode() ? + if (value is byte[] bstr) return bstr.AsSlice().ToStringAscii(); //REVIEW: or ToUnicode() ? - var fmt = value as IFormattable; - if (fmt != null) return fmt.ToString(null, CultureInfo.InvariantCulture); + if (value is IFormattable fmt) return fmt.ToString(null, CultureInfo.InvariantCulture); return null; } diff --git a/FoundationDB.Client/Converters/IFdbConverter.cs b/FoundationDB.Client/Converters/ITypeConverter.cs similarity index 86% rename from FoundationDB.Client/Converters/IFdbConverter.cs rename to FoundationDB.Client/Converters/ITypeConverter.cs index 65a9fa941..c92550c30 100644 --- a/FoundationDB.Client/Converters/IFdbConverter.cs +++ b/FoundationDB.Client/Converters/ITypeConverter.cs @@ -26,27 +26,35 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Converters +namespace Doxense.Runtime.Converters { using System; + using JetBrains.Annotations; /// Base class of all value converters - public interface IFdbConverter + public interface ITypeConverter { + /// Type of the instance to be converted + [NotNull] Type Source { get; } + + /// Type of the result of the conversion + [NotNull] Type Destination { get; } + [Pure] object ConvertBoxed(object value); } /// Class that can convert values of type into values of type /// Source type /// Destination type - public interface IFdbConverter : IFdbConverter + public interface ITypeConverter : ITypeConverter { /// Converts a into a /// Value to convert /// Converted value + [Pure] TDestination Convert(TSource value); } diff --git a/FoundationDB.Client/Converters/SimilarValueComparer.cs b/FoundationDB.Client/Converters/SimilarValueComparer.cs index 8d52b855d..3657eb5d1 100644 --- a/FoundationDB.Client/Converters/SimilarValueComparer.cs +++ b/FoundationDB.Client/Converters/SimilarValueComparer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Converters +namespace Doxense.Runtime.Converters { using System; using System.Collections; diff --git a/FoundationDB.Client/Converters/FdbConverters.cs b/FoundationDB.Client/Converters/TypeConverters.cs similarity index 59% rename from FoundationDB.Client/Converters/FdbConverters.cs rename to FoundationDB.Client/Converters/TypeConverters.cs index 505d4c3d6..37b36a830 100644 --- a/FoundationDB.Client/Converters/FdbConverters.cs +++ b/FoundationDB.Client/Converters/TypeConverters.cs @@ -26,35 +26,44 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Converters +namespace Doxense.Runtime.Converters { using System; using System.Collections.Generic; using System.Globalization; using System.Linq.Expressions; + using System.Reflection; + using System.Runtime.CompilerServices; using System.Threading; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; /// Helper class to convert object from one type to another - public static class FdbConverters + public static class TypeConverters { + + /// Cache used to make the JIT inline all converters from ValueType to ValueType + private static class Cache + { + [NotNull] + public static readonly ITypeConverter Converter = GetConverter(); + } + #region Identity /// Simple converter where the source and destination types are the same /// Source and Destination type - private class Identity : IFdbConverter + private sealed class Identity : ITypeConverter { private static readonly bool IsReferenceType = typeof(T).IsClass; //TODO: nullables ? - public static readonly IFdbConverter Default = new Identity(); + public static readonly ITypeConverter Default = new Identity(); - public static readonly Func FromObject = (Func)FdbConverters.CreateCaster(typeof(T)); + public static readonly Func FromObject = (Func)TypeConverters.CreateCaster(typeof(T)); - public Type Source { get { return typeof(T); } } + public Type Source => typeof(T); - public Type Destination { get { return typeof(T); } } + public Type Destination => typeof(T); public T Convert(T value) { @@ -78,55 +87,56 @@ public static T Cast(object value) #region Anonymous /// Simple converter that wraps a lambda function - /// Source type - /// Destination type - private class Anonymous : IFdbConverter + /// Source type + /// Destination type + private sealed class Anonymous : ITypeConverter { - private Func Converter { get; set; } + [NotNull] + private Func Converter { get; } - public Anonymous([NotNull] Func converter) + public Anonymous([NotNull] Func converter) { - if (converter == null) throw new ArgumentNullException("converter"); + Contract.NotNull(converter, nameof(converter)); this.Converter = converter; } - public Type Source { get { return typeof(T); } } + public Type Source => typeof(TInput); - public Type Destination { get { return typeof(R); } } + public Type Destination => typeof(TOutput); - public R Convert(T value) + public TOutput Convert(TInput value) { return this.Converter(value); } public object ConvertBoxed(object value) { - return (object) this.Converter(Identity.FromObject(value)); + return this.Converter(Identity.FromObject(value)); } } - private class SubClass : IFdbConverter + private sealed class SubClass : ITypeConverter { - public static readonly IFdbConverter Default = new SubClass(); + public static readonly ITypeConverter Default = new SubClass(); private SubClass() { - if (!typeof(R).IsAssignableFrom(typeof(T))) throw new InvalidOperationException(String.Format("Type {0} is not a subclass of {1}", typeof(T).Name, typeof(R).Name)); + if (!typeof(TOutput).IsAssignableFrom(typeof(TInput))) throw new InvalidOperationException($"Type {typeof(TInput).Name} is not a subclass of {typeof(TOutput).Name}"); } - public R Convert(T value) + public TOutput Convert(TInput value) { - return (R)(object)value; + return (TOutput)(object)value; } public Type Source { - get { return typeof(T); } + get { return typeof(TInput); } } public Type Destination { - get { return typeof(R); } + get { return typeof(TOutput); } } public object ConvertBoxed(object value) @@ -138,22 +148,22 @@ public object ConvertBoxed(object value) #endregion /// Static ctor that initialize the default converters - static FdbConverters() + static TypeConverters() { RegisterDefaultConverters(); } /// Map of all known converters from T to R /// No locking required, because all changes will replace this instance with a new Dictionary - private static Dictionary Converters = new Dictionary(ComparisonHelper.TypePairComparer.Default); + private static Dictionary Converters = new Dictionary(ComparisonHelper.TypePairComparer.Default); /// Register all the default converters private static void RegisterDefaultConverters() { //TODO: there is too much generic type combinations! need to refactor this ... - RegisterUnsafe((value) => Slice.FromByte(value ? (byte)1 : default(byte))); - RegisterUnsafe((value) => Slice.FromByte(value ? (byte)1 : default(byte)).GetBytes()); + RegisterUnsafe((value) => Slice.FromByte(value ? (byte) 1 : default(byte))); + RegisterUnsafe((value) => Slice.FromByte(value ? (byte) 1 : default(byte)).GetBytes()); RegisterUnsafe((value) => value ? "true" : "false"); RegisterUnsafe((value) => value ? (sbyte)1 : default(sbyte)); RegisterUnsafe((value) => value ? (byte)1 : default(byte)); @@ -165,10 +175,11 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value ? 1UL : default(ulong)); RegisterUnsafe((value) => value ? 1.0d : default(double)); RegisterUnsafe((value) => value ? 1.0f : default(float)); + RegisterUnsafe((value) => value ? 1m : default(decimal)); - RegisterUnsafe((value) => Slice.FromInt32(value)); + RegisterUnsafe(Slice.FromInt32); RegisterUnsafe((value) => Slice.FromInt32(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(StringConverters.ToString); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => checked((byte)value)); @@ -178,12 +189,12 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value); RegisterUnsafe((value) => (ulong)value); RegisterUnsafe((value) => value); - RegisterUnsafe((value) => checked((float)value)); - RegisterUnsafe((value) => (FdbTupleAlias)value); + RegisterUnsafe((value) => value); // possible loss of precision + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromUInt64(value)); - RegisterUnsafe((value) => Slice.FromUInt64(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(Slice.FromUInt32); + RegisterUnsafe((value) => Slice.FromUInt32(value).GetBytes()); + RegisterUnsafe(StringConverters.ToString); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => checked((byte)value)); @@ -193,11 +204,12 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); - RegisterUnsafe((value) => checked((float)value)); + RegisterUnsafe((value) => value); // possible loss of precision + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromInt64(value)); + RegisterUnsafe(Slice.FromInt64); RegisterUnsafe((value) => Slice.FromInt64(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(StringConverters.ToString); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => checked((byte)value)); @@ -206,15 +218,16 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => checked((int)value)); RegisterUnsafe((value) => (uint)value); RegisterUnsafe((value) => (ulong)value); - RegisterUnsafe((value) => checked((double)value)); - RegisterUnsafe((value) => checked((float)value)); - RegisterUnsafe((value) => TimeSpan.FromTicks(value)); + RegisterUnsafe((value) => value); // possible loss of precision + RegisterUnsafe((value) => value); // possible loss of precision + RegisterUnsafe(TimeSpan.FromTicks); RegisterUnsafe((value) => new Uuid64(value)); RegisterUnsafe((value) => new System.Net.IPAddress(value)); + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromUInt64(value)); + RegisterUnsafe(Slice.FromUInt64); RegisterUnsafe((value) => Slice.FromUInt64(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(StringConverters.ToString); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => checked((byte)value)); @@ -223,14 +236,15 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => checked((int)value)); RegisterUnsafe((value) => checked((uint)value)); RegisterUnsafe((value) => checked((long)value)); - RegisterUnsafe((value) => checked((double)value)); - RegisterUnsafe((value) => checked((float)value)); + RegisterUnsafe((value) => value); // possible loss of precision + RegisterUnsafe((value) => value); // possible loss of precision RegisterUnsafe((value) => new Uuid64(value)); - RegisterUnsafe((value) => TimeSpan.FromTicks(checked((long)value))); + RegisterUnsafe((value) => TimeSpan.FromTicks(checked((long) value))); + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromInt32(value)); - RegisterUnsafe((value) => Slice.FromInt32(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(Slice.FromInt16); + RegisterUnsafe((value) => Slice.FromInt16(value).GetBytes()); + RegisterUnsafe((value) => StringConverters.ToString(value)); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => checked((byte)value)); @@ -241,11 +255,11 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => checked ((ulong)value)); RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); - RegisterUnsafe((value) => (FdbTupleAlias)value); + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromUInt64(value)); - RegisterUnsafe((value) => Slice.FromUInt64(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(Slice.FromUInt16); + RegisterUnsafe((value) => Slice.FromUInt16(value).GetBytes()); + RegisterUnsafe((value) => StringConverters.ToString(value)); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((byte)value)); RegisterUnsafe((value) => checked((sbyte)value)); @@ -256,10 +270,11 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromInt32(value)); - RegisterUnsafe((value) => Slice.FromInt32(value).GetBytes()); - RegisterUnsafe((value) => value.ToString(CultureInfo.InvariantCulture)); //TODO: string table! + RegisterUnsafe(Slice.FromByte); + RegisterUnsafe((value) => Slice.FromByte(value).GetBytes()); + RegisterUnsafe((value) => StringConverters.ToString(value)); RegisterUnsafe((value) => value != 0); RegisterUnsafe((value) => checked((sbyte)value)); RegisterUnsafe((value) => value); @@ -270,7 +285,7 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); - RegisterUnsafe((value) => (FdbTupleAlias)value); + RegisterUnsafe((value) => value); RegisterUnsafe((value) => Slice.FromInt64(value)); RegisterUnsafe((value) => Slice.FromInt64(value).GetBytes()); @@ -285,8 +300,9 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => checked((ulong)value)); RegisterUnsafe((value) => value); RegisterUnsafe((value) => value); + RegisterUnsafe((value) => value); - RegisterUnsafe((value) => Slice.FromSingle(value)); + RegisterUnsafe(Slice.FromSingle); RegisterUnsafe((value) => Slice.FromSingle(value).GetBytes()); RegisterUnsafe((value) => value.ToString("R", CultureInfo.InvariantCulture)); RegisterUnsafe((value) => !(value == 0f || float.IsNaN(value))); @@ -299,6 +315,7 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => checked((long)value)); RegisterUnsafe((value) => (ulong)value); RegisterUnsafe((value) => value); + RegisterUnsafe((value) => (decimal) value); // possible loss of precision RegisterUnsafe((value) => Slice.FromDouble(value)); RegisterUnsafe((value) => Slice.FromDouble(value).GetBytes()); @@ -312,41 +329,43 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => (uint)value); RegisterUnsafe((value) => checked((long)value)); RegisterUnsafe((value) => (ulong)value); - RegisterUnsafe((value) => checked((float)value)); + RegisterUnsafe((value) => (float)value); // possible loss of precision + RegisterUnsafe((value) => (decimal) value); // possible loss of precision RegisterUnsafe((value) => Slice.FromString(value)); - RegisterUnsafe((value) => Slice.FromString(value).GetBytes()); + RegisterUnsafe((value) => Slice.FromString(value).GetBytes()); //REVIEW: string=>byte[] use UTF-8, but byte[]=>string uses Base64 ? RegisterUnsafe((value) => !string.IsNullOrEmpty(value)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(sbyte) : SByte.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(byte) : Byte.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(short) : Int16.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(ushort) : UInt16.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(int) : Int32.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(uint) : UInt32.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(long) : Int64.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(ulong) : UInt64.Parse(value, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(float) : Single.Parse(value, NumberStyles.Float, CultureInfo.InvariantCulture)); - RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(double) : Double.Parse(value, NumberStyles.Float, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(sbyte) : sbyte.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(byte) : byte.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(short) : short.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(ushort) : ushort.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(int) : int.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(uint) : uint.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(long) : long.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(ulong) : ulong.Parse(value, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(float) : float.Parse(value, NumberStyles.Float, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(double) : double.Parse(value, NumberStyles.Float, CultureInfo.InvariantCulture)); + RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(decimal) : decimal.Parse(value, NumberStyles.Float, CultureInfo.InvariantCulture)); RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(Guid) : Guid.Parse(value)); RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(Uuid128) : Uuid128.Parse(value)); RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(Uuid64) : Uuid64.Parse(value)); RegisterUnsafe((value) => string.IsNullOrEmpty(value) ? default(System.Net.IPAddress) : System.Net.IPAddress.Parse(value)); - RegisterUnsafe((value) => Slice.Create(value)); - RegisterUnsafe((value) => value == null ? default(string) : value.Length == 0 ? String.Empty : System.Convert.ToBase64String(value)); + RegisterUnsafe((value) => value.AsSlice()); + RegisterUnsafe((value) => value == null ? default(string) : value.Length == 0 ? string.Empty : System.Convert.ToBase64String(value)); //REVIEW: string=>byte[] use UTF-8, but byte[]=>string uses Base64 ? RegisterUnsafe((value) => value != null && value.Length > 0); - RegisterUnsafe((value) => value == null ? default(sbyte) : Slice.Create(value).ToSByte()); - RegisterUnsafe((value) => value == null ? default(byte) : Slice.Create(value).ToByte()); - RegisterUnsafe((value) => value == null ? default(short) : Slice.Create(value).ToInt16()); - RegisterUnsafe((value) => value == null ? default(ushort) : Slice.Create(value).ToUInt16()); - RegisterUnsafe((value) => value == null ? 0 : Slice.Create(value).ToInt32()); - RegisterUnsafe((value) => value == null ? 0U : Slice.Create(value).ToUInt32()); - RegisterUnsafe((value) => value == null ? 0L : Slice.Create(value).ToInt64()); - RegisterUnsafe((value) => value == null ? 0UL : Slice.Create(value).ToUInt64()); + RegisterUnsafe((value) => value == null ? default(sbyte) : value.AsSlice().ToSByte()); + RegisterUnsafe((value) => value == null ? default(byte) : value.AsSlice().ToByte()); + RegisterUnsafe((value) => value == null ? default(short) : value.AsSlice().ToInt16()); + RegisterUnsafe((value) => value == null ? default(ushort) : value.AsSlice().ToUInt16()); + RegisterUnsafe((value) => value == null ? 0 : value.AsSlice().ToInt32()); + RegisterUnsafe((value) => value == null ? 0U : value.AsSlice().ToUInt32()); + RegisterUnsafe((value) => value == null ? 0L : value.AsSlice().ToInt64()); + RegisterUnsafe((value) => value == null ? 0UL : value.AsSlice().ToUInt64()); RegisterUnsafe((value) => value == null || value.Length == 0 ? default(Guid) : new Uuid128(value).ToGuid()); RegisterUnsafe((value) => value == null || value.Length == 0 ? default(Uuid128) : new Uuid128(value)); - RegisterUnsafe((value) => value == null || value.Length == 0 ? default(Uuid64) : new Uuid64(value)); - RegisterUnsafe((value) => value == null ? TimeSpan.Zero : TimeSpan.FromTicks(Slice.Create(value).ToInt64())); + RegisterUnsafe((value) => value != null ? Uuid64.Read(value) : default(Uuid64)); + RegisterUnsafe((value) => value == null ? TimeSpan.Zero : TimeSpan.FromTicks(value.AsSlice().ToInt64())); RegisterUnsafe((value) => value == null || value.Length == 0 ? default(System.Net.IPAddress) : new System.Net.IPAddress(value)); RegisterUnsafe((value) => Slice.FromGuid(value)); @@ -354,14 +373,14 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value.ToString("D", null)); RegisterUnsafe((value) => new Uuid128(value)); RegisterUnsafe((value) => value != Guid.Empty); - RegisterUnsafe((value) => new System.Net.IPAddress(new Uuid128(value).ToByteArray())); + RegisterUnsafe((value) => new System.Net.IPAddress(new Uuid128(value).ToByteArray())); //REVIEW: custom converter for Guid=>IPv6? RegisterUnsafe((value) => value.ToSlice()); RegisterUnsafe((value) => value.ToByteArray()); RegisterUnsafe((value) => value.ToString("D", null)); RegisterUnsafe((value) => value.ToGuid()); RegisterUnsafe((value) => value != Uuid128.Empty); - RegisterUnsafe((value) => new System.Net.IPAddress(value.ToByteArray())); + RegisterUnsafe((value) => new System.Net.IPAddress(value.ToByteArray())); //REVIEW: custom converter for Guid=>IPv6? RegisterUnsafe((value) => value.ToSlice()); RegisterUnsafe((value) => value.ToByteArray()); @@ -377,15 +396,13 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value.TotalSeconds); RegisterUnsafe((value) => value == TimeSpan.Zero); - RegisterUnsafe((value) => value != null ? Slice.Create(value.GetAddressBytes()) : Slice.Nil); - RegisterUnsafe((value) => value != null ? value.GetAddressBytes() : null); - RegisterUnsafe((value) => value != null ? value.ToString() : null); + RegisterUnsafe((value) => (value?.GetAddressBytes()).AsSlice()); + RegisterUnsafe((value) => value?.GetAddressBytes()); + RegisterUnsafe((value) => value?.ToString()); +#pragma warning disable 618 + RegisterUnsafe((value) => (int) (value?.Address ?? 0)); +#pragma warning restore 618 - RegisterUnsafe((value) => (byte)value); - RegisterUnsafe((value) => (int)value); - RegisterUnsafe((value) => Slice.FromByte((byte)value)); - - //REVIEW: this should go in the Tuples layer ! RegisterUnsafe((value) => value.GetBytes()); RegisterUnsafe((value) => value.ToUnicode()); RegisterUnsafe((value) => value.ToBool()); @@ -397,33 +414,36 @@ private static void RegisterDefaultConverters() RegisterUnsafe((value) => value.ToUInt32()); RegisterUnsafe((value) => value.ToInt64()); RegisterUnsafe((value) => value.ToUInt64()); + RegisterUnsafe((value) => value.ToSingle()); + RegisterUnsafe((value) => value.ToDouble()); + RegisterUnsafe((value) => value.ToDecimal()); RegisterUnsafe((value) => value.ToGuid()); RegisterUnsafe((value) => value.ToUuid128()); RegisterUnsafe((value) => value.ToUuid64()); RegisterUnsafe((value) => TimeSpan.FromTicks(value.ToInt64())); - RegisterUnsafe((value) => (FdbTupleAlias)value.ToByte()); - RegisterUnsafe((value) => !value.IsNullOrEmpty ? new System.Net.IPAddress(value.GetBytes()) : null); + RegisterUnsafe((value) => !value.IsNullOrEmpty ? new System.Net.IPAddress(value.GetBytesOrEmpty()) : null); } /// Helper method to throw an exception when we don't know how to convert from to /// Type of the source object /// Target type of the conversion - [ContractAnnotation("=> halt")] - private static void FailCannotConvert(Type source, Type destination) + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailCannotConvert(Type source, Type destination) { // prettyprint nullable type names to have something more usefull than "Nullable`1" //TODO: extend this to all generic types ? var nt = Nullable.GetUnderlyingType(source); - string sourceName = nt == null ? source.Name : String.Format("Nullable<{0}>", nt.Name); + string sourceName = nt == null ? source.Name : ("Nullable<" + nt.Name + ">"); nt = Nullable.GetUnderlyingType(destination); - string destinationName = nt == null ? destination.Name : String.Format("Nullable<{0}>", nt.Name); + string destinationName = nt == null ? destination.Name : ("Nullable<" + nt.Name + ">"); - throw new InvalidOperationException(String.Format("Cannot convert values of type {0} into {1}", sourceName, destinationName)); + return new InvalidOperationException($"Cannot convert values of type {sourceName} into {destinationName}"); } /// Create a new delegate that cast a boxed valued of type T (object) into a T /// Delegate that is of type Func<object, > - private static Delegate CreateCaster(Type type) + [NotNull] + private static Delegate CreateCaster([NotNull] Type type) { var prm = Expression.Parameter(typeof(object), "value"); //TODO: valuetype vs ref type ? @@ -433,48 +453,51 @@ private static Delegate CreateCaster(Type type) } /// Helper method that wraps a lambda function into a converter - /// Source type - /// Destination type - /// Lambda that converts a value of type into a value of type + /// Source type + /// Destination type + /// Lambda that converts a value of type into a value of type /// Converters that wraps the lambda - public static IFdbConverter Create([NotNull] Func converter) + [NotNull] + public static ITypeConverter Create([NotNull] Func converter) { - if (converter == null) throw new ArgumentNullException("converter"); - return new Anonymous(converter); + Contract.NotNull(converter, nameof(converter)); + return new Anonymous(converter); } /// Add a new known converter (without locking) - /// Source type - /// Destination type - /// Lambda that converts a value of type into a value of type - internal static void RegisterUnsafe([NotNull] Func converter) + /// Source type + /// Destination type + /// Lambda that converts a value of type into a value of type + internal static void RegisterUnsafe([NotNull] Func converter) { Contract.Requires(converter != null); - Converters[new ComparisonHelper.TypePair(typeof(T), typeof(R))] = new Anonymous(converter); + Converters[new ComparisonHelper.TypePair(typeof(TInput), typeof(TOutput))] = new Anonymous(converter); } /// Registers a new type converter - /// Source type - /// Destination type - /// Lambda that converts a value of type into a value of type - public static void Register([NotNull] Func converter) + /// Source type + /// Destination type + /// Lambda that converts a value of type into a value of type + public static void Register([NotNull] Func converter) { Contract.Requires(converter != null); - Register(new Anonymous(converter)); + Register(new Anonymous(converter)); } /// Registers a new type converter - /// Source type - /// Destination type - /// Instance that can convert values of type into a values of type - public static void Register([NotNull] IFdbConverter converter) + /// Source type + /// Destination type + /// Instance that can convert values of type into a values of type + public static void Register([NotNull] ITypeConverter converter) { - if (converter == null) throw new ArgumentNullException("converter"); + Contract.NotNull(converter, nameof(converter)); while (true) { var previous = Converters; - var dic = new Dictionary(previous, previous.Comparer); - dic[new ComparisonHelper.TypePair(typeof(T), typeof(R))] = converter; + var dic = new Dictionary(previous, previous.Comparer) + { + [new ComparisonHelper.TypePair(typeof(TInput), typeof(TOutput))] = converter + }; if (Interlocked.CompareExchange(ref Converters, dic, previous) == previous) { break; @@ -482,102 +505,130 @@ public static void Register([NotNull] IFdbConverter converter) } } - /// Returns a converter that converts s into s - /// Source type - /// Destination type + /// Returns a converter that converts s into s + /// Source type + /// Destination type /// Valid convertir for this types, or an exception if there are no known convertions /// No valid converter for these types was found [NotNull] - public static IFdbConverter GetConverter() + public static ITypeConverter GetConverter() { - - if (typeof(T) == typeof(R)) + if (typeof(TInput) == typeof(TOutput)) { // R == T : identity function - return (IFdbConverter)Identity.Default; + return (ITypeConverter) Identity.Default; } // Try to get from the known converters - IFdbConverter converter; - if (!Converters.TryGetValue(new ComparisonHelper.TypePair(typeof(T), typeof(R)), out converter)) + if (!Converters.TryGetValue(new ComparisonHelper.TypePair(typeof(TInput), typeof(TOutput)), out ITypeConverter converter)) { - if (typeof(R).IsAssignableFrom(typeof(T))) + if (typeof(TOutput).IsAssignableFrom(typeof(TInput))) { // T is a subclass of R, so it should work fine - return SubClass.Default; + return SubClass.Default; } //TODO: ..? - FailCannotConvert(typeof(T), typeof(R)); + throw FailCannotConvert(typeof(TInput), typeof(TOutput)); } - return (IFdbConverter)converter; + return (ITypeConverter) converter; + } + + /// Wrap a Tye Converter into a corresponding Func<....> + /// Source type + /// Destination type + /// Instance that can convert from to + /// Lambda function that, when called, invokes + [Pure, NotNull] + public static Func AsFunc([NotNull] this ITypeConverter converter) + { + return converter.Convert; } - /// Convert a value of type into type - /// Source type - /// Destination type + /// Convert a value of type into type + /// Source type + /// Destination type /// Value to convert /// Converted value - public static R Convert(T value) + [Pure, ContractAnnotation("null=>null")] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static TOutput Convert(TInput value) { //note: most of the types, T will be equal to R. We should get an optimized converter that will not box the values - return GetConverter().Convert(value); + return Cache.Converter.Convert(value); } /// Cast a boxed value (known to be of type ) into an unboxed value /// Runtime type of the value /// Value that is known to be of type , but is boxed into an object /// Original value casted into its runtime type + [Pure, ContractAnnotation("null=>null")] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static T Unbox(object value) { return Identity.FromObject(value); } - /// Convert a boxed value into type - /// Destination type + /// Convert a boxed value into type + /// Destination type /// Boxed value - /// Converted value, or an exception if there are no known convertions. The value null is converted into default() by convention + /// Converted value, or an exception if there are no known convertions. The value null is converted into default() by convention /// No valid converter for these types was found - [CanBeNull] - public static R ConvertBoxed(object value) + [Pure, CanBeNull] + public static T ConvertBoxed(object value) { - if (value == null) return default(R); + if (value == null) return default(T); var type = value.GetType(); - var targetType = typeof(R); + var targetType = typeof(T); // cast ! - if (targetType.IsAssignableFrom(type)) return (R)value; + if (targetType.IsAssignableFrom(type)) return (T) value; - IFdbConverter converter; - if (!Converters.TryGetValue(new ComparisonHelper.TypePair(type, targetType), out converter)) + if (!Converters.TryGetValue(new ComparisonHelper.TypePair(type, targetType), out ITypeConverter converter)) { // maybe it is a nullable type ? var nullableType = Nullable.GetUnderlyingType(targetType); - if (nullableType != null) - { // we already nullchecked value above, so we just have to convert it to the underlying type... + if (nullableType == null) throw FailCannotConvert(type, targetType); - // shortcut for converting a R into a Nullable ... - if (type == nullableType) return (R)value; + // we already nullchecked value above, so we just have to convert it to the underlying type... - // maybe we have a converter for the underlying type ? - if (Converters.TryGetValue(new ComparisonHelper.TypePair(type, nullableType), out converter)) - { - return (R)converter.ConvertBoxed(value); - } + // shortcut for converting a T into a Nullable ... + if (type == nullableType) return (T) value; + + // maybe we have a converter for the underlying type ? + if (Converters.TryGetValue(new ComparisonHelper.TypePair(type, nullableType), out converter)) + { + return (T) converter.ConvertBoxed(value); } - FailCannotConvert(type, targetType); } - return (R)converter.ConvertBoxed(value); + return (T) converter.ConvertBoxed(value); + } + + [NotNull] + private static MethodInfo GetConverterMethod(Type input, Type output) + { + var m = typeof(TypeConverters).GetMethod(nameof(GetConverter), BindingFlags.Static | BindingFlags.Public).MakeGenericMethod(input, output); + Contract.Assert(m != null); + return m; + } + + /// Create a boxed converter from to + [Pure, NotNull] + public static Func CreateBoxedConverter(Type outputType) + { + var converter = (ITypeConverter) GetConverterMethod(typeof(TInput), outputType).Invoke(null, Array.Empty()); + return (x) => converter.ConvertBoxed(x); } /// Converts all the elements of a sequence /// New sequence with all the converted elements - public static IEnumerable ConvertAll([NotNull] this IFdbConverter converter, [NotNull] IEnumerable items) + [Pure, NotNull] + public static IEnumerable ConvertAll([NotNull] this ITypeConverter converter, [NotNull] IEnumerable items) { - if (converter == null) throw new ArgumentNullException("converter"); - if (items == null) throw new ArgumentNullException("items"); + Contract.NotNull(converter, nameof(converter)); + Contract.NotNull(items, nameof(items)); foreach (var item in items) { @@ -588,32 +639,23 @@ public static IEnumerable ConvertAll([NotNull] this IFdbConverter /// Converts all the elements of a list /// New list with all the converted elements [NotNull] - public static List ConvertAll([NotNull] this IFdbConverter converter, [NotNull] List items) + public static List ConvertAll([NotNull] this ITypeConverter converter, [NotNull] List items) { - if (converter == null) throw new ArgumentNullException("converter"); - if (items == null) throw new ArgumentNullException("items"); + Contract.NotNull(converter, nameof(converter)); + Contract.NotNull(items, nameof(items)); -#if CORE_CLR - var list = new List(items.Count); - foreach (var item in items) - { - list.Add(converter.Convert(item)); - } - return list; -#else - return items.ConvertAll(converter.Convert); -#endif + return items.ConvertAll(converter.Convert); } /// Converts all the elements of an array /// New array with all the converted elements [NotNull] - public static R[] ConvertAll([NotNull] this IFdbConverter converter, [NotNull] T[] items) + public static TOutput[] ConvertAll([NotNull] this ITypeConverter converter, [NotNull] TInput[] items) { - if (converter == null) throw new ArgumentNullException("converter"); - if (items == null) throw new ArgumentNullException("items"); + Contract.NotNull(converter, nameof(converter)); + Contract.NotNull(items, nameof(items)); - var results = new R[items.Length]; + var results = new TOutput[items.Length]; for (int i = 0; i < items.Length; i++) { results[i] = converter.Convert(items[i]); @@ -621,6 +663,21 @@ public static R[] ConvertAll([NotNull] this IFdbConverter converter, return results; } + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string ToString(TInput value) + { + //note: raccourci pour Convert(..) dont le but est d'être inliné par le JIT en release + return Cache.Converter.Convert(value); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static TOutput FromString(string text) + { + //note: raccourci pour Convert(..) dont le but est d'être inliné par le JIT en release + return Cache.Converter.Convert(text); + } + } } diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 9a07152aa..00d0db065 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -28,16 +28,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define TRACE_COUNTING + namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using FoundationDB.Client.Status; using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; public static partial class Fdb @@ -51,41 +52,41 @@ public static class System // => eg. Fdb.System.MaxValue.Array[0] = 42; /// "\xFF\xFF" - public static readonly Slice MaxValue = Slice.FromAscii("\xFF\xFF"); + public static readonly Slice MaxValue = Slice.FromByteString("\xFF\xFF"); /// "\xFF\x00" - public static readonly Slice MinValue = Slice.FromAscii("\xFF\x00"); + public static readonly Slice MinValue = Slice.FromByteString("\xFF\x00"); /// "\xFF/backupDataFormat" - public static readonly Slice BackupDataFormat = Slice.FromAscii("\xFF/backupDataFormat"); + public static readonly Slice BackupDataFormat = Slice.FromByteString("\xFF/backupDataFormat"); /// "\xFF/conf/" - public static readonly Slice ConfigPrefix = Slice.FromAscii("\xFF/conf/"); + public static readonly Slice ConfigPrefix = Slice.FromByteString("\xFF/conf/"); /// "\xFF/coordinators" - public static readonly Slice Coordinators = Slice.FromAscii("\xFF/coordinators"); + public static readonly Slice Coordinators = Slice.FromByteString("\xFF/coordinators"); /// "\xFF/globals/" - public static readonly Slice GlobalsPrefix = Slice.FromAscii("\xFF/globals/"); + public static readonly Slice GlobalsPrefix = Slice.FromByteString("\xFF/globals/"); /// "\xFF/init_id" - public static readonly Slice InitId = Slice.FromAscii("\xFF/init_id"); + public static readonly Slice InitId = Slice.FromByteString("\xFF/init_id"); /// "\xFF/keyServer/(key_boundary)" => (..., node_id, ...) - public static readonly Slice KeyServers = Slice.FromAscii("\xFF/keyServers/"); + public static readonly Slice KeyServers = Slice.FromByteString("\xFF/keyServers/"); /// "\xFF/serverKeys/(node_id)/(key_boundary)" => ('' | '1') - public static readonly Slice ServerKeys = Slice.FromAscii("\xFF/serverKeys/"); + public static readonly Slice ServerKeys = Slice.FromByteString("\xFF/serverKeys/"); /// "\xFF/serverList/(node_id)" => (..., node_id, machine_id, datacenter_id, ...) - public static readonly Slice ServerList = Slice.FromAscii("\xFF/serverList/"); + public static readonly Slice ServerList = Slice.FromByteString("\xFF/serverList/"); /// "\xFF/workers/(ip:port)/..." => datacenter + machine + mclass - public static readonly Slice WorkersPrefix = Slice.FromAscii("\xFF/workers/"); + public static readonly Slice WorkersPrefix = Slice.FromByteString("\xFF/workers/"); #region JSON Status - private static readonly Slice StatusJsonKey = Slice.FromAscii("\xFF\xFF/status/json"); + private static readonly Slice StatusJsonKey = Slice.FromByteString("\xFF\xFF/status/json"); [ItemCanBeNull] public static async Task GetStatusAsync([NotNull] IFdbReadOnlyTransaction trans) @@ -148,7 +149,7 @@ public static async Task GetCoordinatorsAsync([NotNull] IFdbData if (coordinators.IsNull) throw new InvalidOperationException("Failed to read the list of coordinators from the cluster's system keyspace."); - return FdbClusterFile.Parse(coordinators.ToAscii()); + return FdbClusterFile.Parse(coordinators.ToStringAscii()); } /// Return the value of a configuration parameter (located under '\xFF/conf/') @@ -177,7 +178,7 @@ public static Task GetConfigParameterAsync([NotNull] IFdbDatabase db, [No public static Slice ConfigKey([NotNull] string name) { if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); - return ConfigPrefix + Slice.FromAscii(name); + return ConfigPrefix + Slice.FromByteString(name); } /// Return the corresponding key for a global attribute @@ -186,7 +187,7 @@ public static Slice ConfigKey([NotNull] string name) public static Slice GlobalsKey([NotNull] string name) { if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); - return GlobalsPrefix + Slice.FromAscii(name); + return GlobalsPrefix + Slice.FromByteString(name); } /// Return the corresponding key for a global attribute @@ -197,7 +198,7 @@ public static Slice WorkersKey([NotNull] string id, [NotNull] string name) { if (string.IsNullOrEmpty(id)) throw new ArgumentException("Id cannot be null or empty", nameof(id)); if (string.IsNullOrEmpty(name)) throw new ArgumentException("Attribute name cannot be null or empty", nameof(name)); - return WorkersPrefix + Slice.FromAscii(id) + Slice.FromChar('/') + Slice.FromAscii(name); + return WorkersPrefix + Slice.FromByteString(id) + Slice.FromChar('/') + Slice.FromByteString(name); } /// Returns the current storage engine mode of the cluster @@ -221,7 +222,7 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas default: { // welcome to the future! - return "unknown(" + value.ToAsciiOrHexaString() + ")"; + return "unknown(" + value.PrettyPrint() + ")"; } } } @@ -236,7 +237,7 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas public static async Task> GetBoundaryKeysAsync([NotNull] IFdbReadOnlyTransaction trans, Slice beginInclusive, Slice endExclusive) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - Contract.Assert(trans.Context != null && trans.Context.Database != null); + Contract.Assert(trans.Context?.Database != null); using (var shadow = trans.Context.Database.BeginReadOnlyTransaction(trans.Cancellation)) { diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 2fa06b8fe..8b82040f5 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -65,7 +65,7 @@ public static partial class Fdb internal const int MaxSafeApiVersion = FdbNative.FDB_API_MAX_VERSION; /// Default API version that will be selected, if the application does not specify otherwise. - internal const int DefaultApiVersion = 300; // v3.0.x + internal const int DefaultApiVersion = 510; // v5.1.x //INVARIANT: MinSafeApiVersion <= DefaultApiVersion <= MaxSafeApiVersion #endregion @@ -716,7 +716,7 @@ private static FdbError SetNetworkOption(FdbNetworkOption option, string value) /// Set the value of a network option on the database handler private static FdbError SetNetworkOption(FdbNetworkOption option, Slice value) { - SliceHelpers.EnsureSliceIsValid(ref value); + value.EnsureSliceIsValid(); unsafe { fixed (byte* ptr = value.Array) diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index c7f56bab1..1c91886b0 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -37,6 +37,7 @@ namespace FoundationDB.Client using System.Threading.Tasks; using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client.Core; using FoundationDB.Client.Native; using FoundationDB.Layers.Directories; diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 0de5c5298..e731cb764 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client; namespace FoundationDB { @@ -34,8 +33,10 @@ namespace FoundationDB using System.Collections.Generic; using System.Diagnostics; using System.Linq; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Memory; + using FoundationDB.Client; using JetBrains.Annotations; /// Factory class for keys @@ -304,7 +305,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) case 0xFF: { //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key[0, -1]); + tuple = TuPack.Unpack(key[0, -1]); suffix = "."; break; } @@ -312,7 +313,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) { var tmp = key[0, -1] + (byte)0; //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.STuple.Unpack(tmp); + tuple = TuPack.Unpack(tmp); suffix = " + 1"; break; } @@ -329,7 +330,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) if (key.Count > 2 && key[-1] == 0 && key[-2] != 0xFF) { //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key[0, -1]); + tuple = TuPack.Unpack(key[0, -1]); suffix = ".<00>"; } break; @@ -345,7 +346,7 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) if (tuple == null && !skip) { // attempt a regular decoding //***README*** if you break under here, see README in the last catch() block - tuple = FoundationDB.Layers.Tuples.STuple.Unpack(key); + tuple = TuPack.Unpack(key); } if (tuple != null) return tuple.ToString() + suffix; diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index b1935d73c..694086804 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Filters { using System; @@ -35,6 +34,7 @@ namespace FoundationDB.Filters using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; using JetBrains.Annotations; diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index 8b67d1fad..fc8e1df1b 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -293,7 +293,7 @@ public DirectoryKeyResolver(Dictionary knownSubspaces) p++; } - Array.Sort(prefixes, paths, SliceComparer.Default); + Array.Sort(prefixes, paths, Slice.Comparer.Default); this.Prefixes = prefixes; this.Paths = paths; } @@ -309,7 +309,7 @@ public static async Task BuildFromDirectoryLayer(IFdbReadO // Entries that correspond to subfolders have the form: NodeSubspace.Pack( (parent_prefix, 0, "child_name") ) = child_prefix var keys = await tr.GetRange(location.ToRange()).ToListAsync(); - var map = new Dictionary(SliceComparer.Default); + var map = new Dictionary(Slice.Comparer.Default); foreach (var entry in keys) { @@ -333,7 +333,7 @@ private bool TryLookup(Slice key, out Slice prefix, out string path) if (key.IsNullOrEmpty) return false; - int p = Array.BinarySearch(this.Prefixes, key, SliceComparer.Default); + int p = Array.BinarySearch(this.Prefixes, key, Slice.Comparer.Default); if (p >= 0) { // direct match! prefix = this.Prefixes[p]; @@ -465,7 +465,7 @@ public override int? ArgumentBytes public override string GetArguments(KeyResolver resolver) { - return String.Concat(resolver.Resolve(this.Key), " = ", this.Value.ToAsciiOrHexaString()); + return String.Concat(resolver.Resolve(this.Key), " = ", this.Value.PrettyPrint()); } } @@ -547,7 +547,7 @@ public override int? ArgumentBytes public override string GetArguments(KeyResolver resolver) { - return String.Concat(resolver.Resolve(this.Key), " ", this.Mutation.ToString(), " ", this.Param.ToAsciiOrHexaString()); + return String.Concat(resolver.Resolve(this.Key), " ", this.Mutation.ToString(), " ", this.Param.PrettyPrint()); } public override string ToString(KeyResolver resolver) @@ -630,7 +630,7 @@ public override string GetResult(KeyResolver resolver) protected override string Dump(Slice value) { - return value.ToAsciiOrHexaString(); + return value.PrettyPrint(); } } @@ -712,8 +712,8 @@ public override string GetResult(KeyResolver resolver) if (!this.Result.HasValue) return base.GetResult(resolver); var res = this.Result.Value; string s = String.Concat("[", res.Length.ToString(), "] {"); - if (res.Length > 0) s += res[0].ToAsciiOrHexaString(); - if (res.Length > 1) s += " ... " + res[res.Length - 1].ToAsciiOrHexaString(); + if (res.Length > 0) s += res[0].PrettyPrint(); + if (res.Length > 1) s += " ... " + res[res.Length - 1].PrettyPrint(); return s + " }"; } diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index be9de267d..2abec7a00 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -61,10 +61,16 @@ - + + + + + + + @@ -215,8 +221,8 @@ - - + + @@ -224,20 +230,24 @@ + + + + - - - - + + + + - - + + - - - + + + @@ -247,8 +257,8 @@ - - + + diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index c7040f497..f08d7b9c4 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -34,10 +34,11 @@ namespace FoundationDB.Layers.Directories using System.Linq; using System.Threading.Tasks; using JetBrains.Annotations; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Memory; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using FoundationDB.Filters.Logging; /// Provides a FdbDirectoryLayer class for managing directories in FoundationDB. @@ -51,9 +52,9 @@ public class FdbDirectoryLayer : IFdbDirectory { private const int SUBDIRS = 0; internal static readonly Version LayerVersion = new Version(1, 0, 0); - internal static readonly Slice LayerSuffix = Slice.FromAscii("layer"); - internal static readonly Slice HcaKey = Slice.FromAscii("hca"); - internal static readonly Slice VersionKey = Slice.FromAscii("version"); + internal static readonly Slice LayerSuffix = Slice.FromStringAscii("layer"); + internal static readonly Slice HcaKey = Slice.FromStringAscii("hca"); + internal static readonly Slice VersionKey = Slice.FromStringAscii("version"); /// Use this flag to make the Directory Layer start annotating the transactions with a descriptions of all operations. /// @@ -111,7 +112,7 @@ void IFdbDirectory.CheckLayer(Slice layer) { if (layer.IsPresent) { - throw new InvalidOperationException(String.Format("The directory layer {0} is not compatible with layer {1}.", this.FullName, layer.ToAsciiOrHexaString())); + throw new InvalidOperationException($"The directory layer {this.FullName} is not compatible with layer {layer:P}."); } } @@ -480,7 +481,7 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti public override string ToString() { - return String.Format("DirectoryLayer(path={0}, contents={1}, nodes={2})", this.FullName, this.ContentSubspace.GetPrefix().ToAsciiOrHexaString(), this.NodeSubspace.GetPrefix().ToAsciiOrHexaString()); + return String.Format("DirectoryLayer(path={0}, contents={1}, nodes={2})", this.FullName, this.ContentSubspace.GetPrefix().PrettyPrint(), this.NodeSubspace.GetPrefix().PrettyPrint()); } #endregion @@ -619,7 +620,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly if (layer.IsPresent && layer != existingNode.Layer) { - throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", path, layer.ToAsciiOrHexaString(), existingNode.Layer.ToAsciiOrHexaString())); + throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", path, layer.PrettyPrint(), existingNode.Layer.PrettyPrint())); } return ContentsOfNode(existingNode.Subspace, path, existingNode.Layer); } @@ -644,7 +645,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Ensure that there is no data already present under prefix {0}", prefix); if (await trans.GetRange(KeyRange.StartsWith(prefix)).AnyAsync().ConfigureAwait(false)) { - throw new InvalidOperationException(String.Format("The database has keys stored at the prefix chosen by the automatic prefix allocator: {0}", prefix.ToAsciiOrHexaString())); + throw new InvalidOperationException(String.Format("The database has keys stored at the prefix chosen by the automatic prefix allocator: {0}", prefix.PrettyPrint())); } // ensure that the prefix has not already been allocated @@ -876,8 +877,8 @@ private static void CheckVersion(Slice value, bool writeAccess) var minor = reader.ReadFixed32(); var upgrade = reader.ReadFixed32(); - if (major > LayerVersion.Major) throw new InvalidOperationException(String.Format("Cannot load directory with version {0}.{1}.{2} using directory layer {3}", major, minor, upgrade, LayerVersion)); - if (writeAccess && minor > LayerVersion.Minor) throw new InvalidOperationException(String.Format("Directory with version {0}.{1}.{2} is read-only when opened using directory layer {3}", major, minor, upgrade, LayerVersion)); + if (major > LayerVersion.Major) throw new InvalidOperationException($"Cannot load directory with version {major}.{minor}.{upgrade} using directory layer {FdbDirectoryLayer.LayerVersion}"); + if (writeAccess && minor > LayerVersion.Minor) throw new InvalidOperationException($"Directory with version {major}.{minor}.{upgrade} is read-only when opened using directory layer {FdbDirectoryLayer.LayerVersion}"); } private void InitializeDirectory([NotNull] IFdbTransaction trans) diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 0a60e3256..767d37270 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -28,9 +28,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Directories { - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using System; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + public class FdbDirectoryPartition : FdbDirectorySubspace { @@ -76,7 +77,7 @@ protected override FdbDirectoryLayer GetLayerForPath(ITuple relativeLocation) public override string ToString() { - return String.Format("DirectoryPartition(path={0}, prefix={1})", this.FullName, this.InternalKey.ToAsciiOrHexaString()); + return String.Format("DirectoryPartition(path={0}, prefix={1})", this.FullName, this.InternalKey.PrettyPrint()); } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index 8a1122f84..b67b194ca 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -32,9 +32,9 @@ namespace FoundationDB.Layers.Directories using System.Collections.Generic; using System.Diagnostics; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; /// A Directory Subspace represents the contents of a directory, but it also remembers the path with which it was opened and offers convenience methods to operate on the directory at that path. @@ -121,7 +121,7 @@ public void CheckLayer(Slice layer) { if (layer.IsPresent && layer != this.Layer) { - throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", this.FullName, this.Layer.ToAsciiOrHexaString(), layer.ToAsciiOrHexaString())); + throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", this.FullName, this.Layer.PrettyPrint(), layer.PrettyPrint())); } } @@ -435,11 +435,11 @@ public override string ToString() { if (this.Layer.IsNullOrEmpty) { - return String.Format("DirectorySubspace(path={0}, prefix={1})", this.FullName, this.InternalKey.ToAsciiOrHexaString()); + return String.Format("DirectorySubspace(path={0}, prefix={1})", this.FullName, this.InternalKey.PrettyPrint()); } else { - return String.Format("DirectorySubspace(path={0}, prefix={1}, layer={2})", this.FullName, this.InternalKey.ToAsciiOrHexaString(), this.Layer.ToAsciiOrHexaString()); + return String.Format("DirectorySubspace(path={0}, prefix={1}, layer={2})", this.FullName, this.InternalKey.PrettyPrint(), this.Layer.PrettyPrint()); } } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs new file mode 100644 index 000000000..c20072756 --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs @@ -0,0 +1,41 @@ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information +#endregion + +namespace Doxense.Collections.Tuples.Encoding +{ + using System; + using JetBrains.Annotations; + + /// Represents an object that can serialize itself using the Tuple Binary Encoding format + public interface ITupleSerializable //REVIEW: ITuplePackable? + { + /// Appends the packed bytes of this instance to the end of a buffer + /// Buffer that will received the packed bytes of this instance + void PackTo(ref TupleWriter writer); + + //note: there is not UnpackFrom, because it does not play way with constructors and readonly fields! + // => use ITupleSerializer for this! + } + + /// Represents an object that can serialize or deserialize tuples of type , using the Tuple Binary Encoding format + /// Type of tuples that can be processed by this instance + public interface ITupleSerializer //REVIEW: ITuplePacker ? + where TTuple : ITuple + { + /// Appends the packed bytes of an item to the end of a buffer + /// Buffer that will received the packed bytes of this instance + /// Tuple that will be packed + void PackTo(ref TupleWriter writer, ref TTuple tuple); + + /// Decode the packed bytes from a buffer, and return the corresponding item + /// Buffer that contains the bytes the decode + /// Receives the decoded tuple + /// + /// The value of will be updated to point to either the end of the buffer, or the next "element" if there are more bytes available. + /// + [Pure] + void UnpackFrom(ref TupleReader reader, out TTuple tuple); + + } +} diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs index b1e1ebb42..2dee08675 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs @@ -26,20 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples + +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Collections; using System.Collections.Generic; using System.Diagnostics; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Tuple that has a fixed abitrary binary prefix - [DebuggerDisplay("{ToString()}")] - public sealed class PrefixedTuple : ITuple + [DebuggerDisplay("{ToString(),nq}")] + public sealed class PrefixedTuple : ITuple, ITupleSerializable { // Used in scenario where we will append keys to a common base tuple // note: linked list are not very efficient, but we do not expect a very long chain, and the head will usually be a subspace or memoized tuple @@ -47,7 +47,7 @@ public sealed class PrefixedTuple : ITuple private Slice m_prefix; //PERF: readonly struct private readonly ITuple m_items; - internal PrefixedTuple(Slice prefix, ITuple items) + public PrefixedTuple(Slice prefix, ITuple items) { Contract.Requires(!prefix.IsNull && items != null); @@ -56,15 +56,16 @@ internal PrefixedTuple(Slice prefix, ITuple items) } /// Binary prefix to all the keys produced by this tuple - public Slice Prefix + public Slice Prefix => m_prefix; + + void ITupleSerializable.PackTo(ref TupleWriter writer) { - get { return m_prefix; } + PackTo(ref writer); } - - public void PackTo(ref TupleWriter writer) + internal void PackTo(ref TupleWriter writer) { writer.Output.WriteBytes(m_prefix); - m_items.PackTo(ref writer); + TupleEncoder.WriteTo(ref writer, m_items); } public Slice ToSlice() @@ -74,57 +75,48 @@ public Slice ToSlice() return writer.Output.ToSlice(); } - public int Count - { - get { return m_items.Count; } - } + public int Count => m_items.Count; - public object this[int index] - { - get { return m_items[index]; } - } + public object this[int index] => m_items[index]; - public ITuple this[int? fromIncluded, int? toExcluded] - { - get { return m_items[fromIncluded, toExcluded]; } - } + public ITuple this[int? fromIncluded, int? toExcluded] => m_items[fromIncluded, toExcluded]; - public R Get(int index) + public T Get(int index) { - return m_items.Get(index); + return m_items.Get(index); } - public R Last() + public T Last() { - return m_items.Last(); + return m_items.Last(); } - ITuple ITuple.Append(R value) + ITuple ITuple.Append(T value) { - return this.Append(value); + return Append(value); } ITuple ITuple.Concat(ITuple tuple) { - return this.Concat(tuple); + return Concat(tuple); } [NotNull] - public PrefixedTuple Append(R value) + public PrefixedTuple Append(T value) { - return new PrefixedTuple(m_prefix, m_items.Append(value)); + return new PrefixedTuple(m_prefix, m_items.Append(value)); } - [NotNull] + [Pure, NotNull] public PrefixedTuple Concat([NotNull] ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + Contract.NotNull(tuple, nameof(tuple)); if (tuple.Count == 0) return this; return new PrefixedTuple(m_prefix, m_items.Concat(tuple)); } - public void CopyTo([NotNull] object[] array, int offset) + public void CopyTo(object[] array, int offset) { m_items.CopyTo(array, offset); } @@ -143,7 +135,7 @@ public override string ToString() { //TODO: should we add the prefix to the string representation ? // => something like "(123, 'abc', true)" - return STuple.ToString(this); + return STuple.Formatter.ToString(this); } public override bool Equals(object obj) @@ -188,12 +180,12 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect return comparer.Equals(m_items, linked.m_items); } - return STuple.Equals(this, other, comparer); + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return STuple.CombineHashCodes( + return HashCodes.Combine( m_prefix.GetHashCode(), comparer.GetHashCode(m_items) ); diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs index 8439a1a3c..eaa48554b 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs @@ -26,17 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Collections; using System.Collections.Generic; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using Doxense.Runtime.Converters; /// Lazily-evaluated tuple that was unpacked from a key - internal sealed class SlicedTuple : ITuple + public sealed class SlicedTuple : ITuple, ITupleSerializable { // STuple.Unpack() splits a key into an array of slices (one for each item). We hold onto these slices, and only deserialize them if needed. // This is helpful because in most cases, the app code will only want to get the last few items (e.g: tuple[-1]) or skip the first few items (some subspace). @@ -61,40 +61,31 @@ public SlicedTuple(Slice[] slices, int offset, int count) m_count = count; } - public void PackTo(ref TupleWriter writer) + void ITupleSerializable.PackTo(ref TupleWriter writer) + { + PackTo(ref writer); + } + internal void PackTo(ref TupleWriter writer) { var slices = m_slices; - for (int n = m_count, p = m_offset; n > 0; n--) + int offset = m_offset; + int count = m_count; + for (int i = 0; i < count; i++) { - writer.Output.WriteBytes(slices[p++]); + writer.Output.WriteBytes(slices[i + offset]); } } - public Slice ToSlice() - { - // merge all the slices making up this segment - //TODO: should we get the sum of all slices to pre-allocated the buffer ? - var writer = new TupleWriter(); - PackTo(ref writer); - return writer.Output.ToSlice(); - } + public int Count => m_count; - public int Count - { - get { return m_count; } - } - - public object this[int index] - { - get { return TuplePackers.DeserializeBoxed(GetSlice(index)); } - } + public object this[int index] => TuplePackers.DeserializeBoxed(GetSlice(index)); public ITuple this[int? fromIncluded, int? toExcluded] { get { - int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? TupleHelpers.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? TupleHelpers.MapIndexBounded(toExcluded.Value, m_count) : m_count; int len = end - begin; if (len <= 0) return STuple.Empty; @@ -103,20 +94,20 @@ public object this[int index] } } - public R Get(int index) + public T Get(int index) { - return TuplePacker.Deserialize(GetSlice(index)); + return TuplePacker.Deserialize(GetSlice(index)); } - public R Last() + public T Last() { if (m_count == 0) throw new InvalidOperationException("Tuple is empty"); - return TuplePacker.Deserialize(m_slices[m_offset + m_count - 1]); + return TuplePacker.Deserialize(m_slices[m_offset + m_count - 1]); } public Slice GetSlice(int index) { - return m_slices[m_offset + STuple.MapIndex(index, m_count)]; + return m_slices[m_offset + TupleHelpers.MapIndex(index, m_count)]; } ITuple ITuple.Append(T value) @@ -154,7 +145,7 @@ public override string ToString() { //OPTIMIZE: this could be optimized, because it may be called a lot when logging is enabled on keys parsed from range reads // => each slice has a type prefix that could be used to format it to a StringBuilder faster, maybe? - return STuple.ToString(this); + return STuple.Formatter.ToString(this); } public override bool Equals(object obj) @@ -190,7 +181,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) return false; } - return STuple.Equals(this, other, comparer); + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) @@ -205,7 +196,7 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) int h = 0; for (int i = 0; i < m_count; i++) { - h = STuple.CombineHashCodes(h, comparer.GetHashCode(m_slices[i + m_offset])); + h = HashCodes.Combine(h, comparer.GetHashCode(m_slices[i + m_offset])); } if (canUseCache) m_hashCode = h; return h; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index 48123c07e..fd0aea49b 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -26,11 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples.Encoding { + using System; using FoundationDB.Client; using JetBrains.Annotations; - using System; + using Doxense.Memory; /// Type codec that uses the Tuple Encoding format /// Type of the values encoded by this codec @@ -54,28 +55,27 @@ public TupleCodec(T missingValue) public override Slice EncodeOrdered(T value) { - return STuple.EncodeKey(value); + return TupleEncoder.EncodeKey(value); } public override void EncodeOrderedSelfTerm(ref SliceWriter output, T value) { //HACKHACK: we lose the current depth! var writer = new TupleWriter(output); - TuplePacker.Encoder(ref writer, value); + TuplePackers.SerializeTo(ref writer, value); output = writer.Output; } public override T DecodeOrdered(Slice input) { - return STuple.DecodeKey(input); + return TuPack.DecodeKey(input); } public override T DecodeOrderedSelfTerm(ref SliceReader input) { //HACKHACK: we lose the current depth! var reader = new TupleReader(input); - T value; - bool res = STuple.DecodeNext(ref reader, out value); + bool res = TuPack.DecodeNext(ref reader, out T value); input = reader.Input; return res ? value : m_missingValue; } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs new file mode 100644 index 000000000..15fc5148f --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs @@ -0,0 +1,975 @@ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information +#endregion + +namespace Doxense.Collections.Tuples.Encoding +{ + using System; + using System.Collections.Generic; + using Doxense.Collections.Tuples; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; + + /// Helper class to encode and decode tuples to and from binary buffers + /// This class is intended for implementors of tuples, and should not be called directly by application code! + public static class TupleEncoder + { + + /// Internal helper that serializes the content of a Tuple into a TupleWriter, meant to be called by implementers of types. + /// Warning: This method will call into if inmplements + + internal static void WriteTo(ref TupleWriter writer, [NotNull] ITuple tuple) + { + Contract.Requires(tuple != null); + // ReSharper disable once SuspiciousTypeConversion.Global + if (tuple is ITupleSerializable ts) + { // optimized version + ts.PackTo(ref writer); + return; + } + + int n = tuple.Count; + // small tuples probably are faster with indexers + //REVIEW: when should we use indexers, and when should we use foreach? + if (n <= 4) + { + for (int i = 0; i < n; i++) + { + TuplePackers.SerializeObjectTo(ref writer, tuple[i]); + } + } + else + { + foreach (object item in tuple) + { + TuplePackers.SerializeObjectTo(ref writer, item); + } + } + } + + #region Packing... + + // Without prefix + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure] + public static Slice Pack([CanBeNull] TTuple tuple) + where TTuple : ITuple + { + if (tuple == null) return Slice.Nil; + //TODO: maybe optimize for Count==0 => Empty ? (calling .Count may not be fast for all tuples...) + var writer = new TupleWriter(); + WriteTo(ref writer, tuple); + return writer.ToSlice(); + } + + /// Pack an array of N-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [NotNull] + public static Slice[] Pack([NotNull] params TTuple[] tuples) //REVIEW: change name to PackRange or PackBatch? + where TTuple : ITuple + { + var empty = default(Slice); + return Pack(empty, tuples); + } + + public static void PackTo(ref SliceWriter writer, [CanBeNull] TTuple tuple) + where TTuple : ITuple + { + if (tuple != null) + { + var tw = new TupleWriter(writer); + WriteTo(ref tw, tuple); + writer = tw.Output; + } + } + + public static void Pack(ref TupleWriter writer, [CanBeNull] TTuple tuple) + where TTuple : ITuple + { + if (tuple != null) + { + WriteTo(ref writer, tuple); + } + } + + // With prefix + + /// Efficiently concatenate a prefix with the packed representation of a tuple + public static Slice Pack(Slice prefix, [CanBeNull] TTuple tuple) + where TTuple : ITuple + { + if (tuple == null || tuple.Count == 0) return prefix; + + var writer = new TupleWriter(32 + prefix.Count); + writer.Output.WriteBytes(prefix); + WriteTo(ref writer, tuple); + return writer.ToSlice(); + } + + /// Pack an array of N-tuples, all sharing the same buffer + /// Common prefix added to all the tuples + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] + [NotNull] + public static Slice[] Pack(Slice prefix, [NotNull] params TTuple[] tuples) + where TTuple : ITuple + { + Contract.NotNull(tuples, nameof(tuples)); + + // pre-allocate by supposing that each tuple will take at least 16 bytes + var writer = new TupleWriter(tuples.Length * (16 + prefix.Count)); + var next = new List(tuples.Length); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var tuple in tuples) + { + writer.Output.WriteBytes(prefix); + WriteTo(ref writer, tuple); + next.Add(writer.Output.Position); + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + /// Pack a sequence of N-tuples, all sharing the same buffer + /// Common prefix added to all the tuples + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] + [NotNull] + public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable tuples) + where TTuple : ITuple + { + Contract.NotNull(tuples, nameof(tuples)); + + // use optimized version for arrays + if (tuples is TTuple[] array) return Pack(prefix, array); + + var next = new List((tuples as ICollection)?.Count ?? 0); + var writer = new TupleWriter(next.Capacity * (16 + prefix.Count)); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var tuple in tuples) + { + writer.Output.WriteBytes(prefix); + WriteTo(ref writer, tuple); + next.Add(writer.Output.Position); + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + [NotNull] + public static Slice[] Pack(Slice prefix, [NotNull] TElement[] elements, Func transform) + where TTuple : ITuple + { + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(transform, nameof(transform)); + + var next = new List(elements.Length); + var writer = new TupleWriter(next.Capacity * (16 + prefix.Count)); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var element in elements) + { + var tuple = transform(element); + if (tuple == null) + { + next.Add(writer.Output.Position); + } + else + { + writer.Output.WriteBytes(prefix); + WriteTo(ref writer, tuple); + next.Add(writer.Output.Position); + } + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + [NotNull] + public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable elements, Func transform) + where TTuple : ITuple + { + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(transform, nameof(transform)); + + // use optimized version for arrays + if (elements is TElement[] array) return Pack(prefix, array, transform); + + var next = new List((elements as ICollection)?.Count ?? 0); + var writer = new TupleWriter(next.Capacity * (16 + prefix.Count)); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var element in elements) + { + var tuple = transform(element); + if (tuple == null) + { + next.Add(writer.Output.Position); + } + else + { + writer.Output.WriteBytes(prefix); + WriteTo(ref writer, tuple); + next.Add(writer.Output.Position); + } + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + // With prefix... + + /// Efficiently concatenate a prefix with the packed representation of a 1-tuple + [Pure] + public static Slice EncodePrefixedKey(Slice prefix, T value) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 2-tuple + [Pure] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 3-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 4-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + TuplePackers.SerializeTo(ref writer, value4); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 5-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + TuplePackers.SerializeTo(ref writer, value4); + TuplePackers.SerializeTo(ref writer, value5); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 6-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + TuplePackers.SerializeTo(ref writer, value4); + TuplePackers.SerializeTo(ref writer, value5); + TuplePackers.SerializeTo(ref writer, value6); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 7-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + TuplePackers.SerializeTo(ref writer, value4); + TuplePackers.SerializeTo(ref writer, value5); + TuplePackers.SerializeTo(ref writer, value6); + TuplePackers.SerializeTo(ref writer, value7); + return writer.ToSlice(); + } + + /// Efficiently concatenate a prefix with the packed representation of a 8-tuple + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, value1); + TuplePackers.SerializeTo(ref writer, value2); + TuplePackers.SerializeTo(ref writer, value3); + TuplePackers.SerializeTo(ref writer, value4); + TuplePackers.SerializeTo(ref writer, value5); + TuplePackers.SerializeTo(ref writer, value6); + TuplePackers.SerializeTo(ref writer, value7); + TuplePackers.SerializeTo(ref writer, value8); + return writer.ToSlice(); + } + + // EncodeKey... + + //REVIEW: do we really ned "Key" in the name? + // => we want to make it obvious that this is to pack ordered keys, but this could be used for anything else... + // => EncodeValues? (may be confused with unordered encoding) + // => EncodeItems? + // => Encode? + + /// Pack a 1-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.ToSlice(); + } + + /// Pack a 2-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.ToSlice(); + } + + /// Pack a 3-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.ToSlice(); + } + + /// Pack a 4-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.ToSlice(); + } + + /// Pack a 5-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.ToSlice(); + } + + /// Pack a 6-tuple directly into a slice + public static Slice Pack(Slice prefix, ref STuple tuple) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + tuple.PackTo(ref writer); + return writer.Output.ToSlice(); + } + + /// Pack a 1-tuple directly into a slice + public static Slice EncodeKey(T1 item1) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + return writer.ToSlice(); + } + + /// Pack a 2-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + return writer.ToSlice(); + } + + /// Pack a 3-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + return writer.ToSlice(); + } + + /// Pack a 4-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + TuplePackers.SerializeTo(ref writer, item4); + return writer.ToSlice(); + } + + /// Pack a 5-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + TuplePackers.SerializeTo(ref writer, item4); + TuplePackers.SerializeTo(ref writer, item5); + return writer.ToSlice(); + } + + /// Pack a 6-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + TuplePackers.SerializeTo(ref writer, item4); + TuplePackers.SerializeTo(ref writer, item5); + TuplePackers.SerializeTo(ref writer, item6); + return writer.Output.ToSlice(); + } + + /// Pack a 6-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + TuplePackers.SerializeTo(ref writer, item4); + TuplePackers.SerializeTo(ref writer, item5); + TuplePackers.SerializeTo(ref writer, item6); + TuplePackers.SerializeTo(ref writer, item7); + return writer.ToSlice(); + } + + /// Pack a 6-tuple directly into a slice + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + var writer = new TupleWriter(); + TuplePackers.SerializeTo(ref writer, item1); + TuplePackers.SerializeTo(ref writer, item2); + TuplePackers.SerializeTo(ref writer, item3); + TuplePackers.SerializeTo(ref writer, item4); + TuplePackers.SerializeTo(ref writer, item5); + TuplePackers.SerializeTo(ref writer, item6); + TuplePackers.SerializeTo(ref writer, item7); + TuplePackers.SerializeTo(ref writer, item8); + return writer.ToSlice(); + } + + [NotNull] + public static Slice[] EncodeKeys([NotNull] IEnumerable keys) + { + var empty = default(Slice); + return EncodePrefixedKeys(empty, keys); + } + + /// Pack a 1-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + writer = tw.Output; + } + + /// Pack a 2-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + writer = tw.Output; + } + + /// Pack a 3-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + writer = tw.Output; + } + + /// Pack a 4-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + TuplePackers.SerializeTo(ref tw, item4); + writer = tw.Output; + } + + /// Pack a 5-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + TuplePackers.SerializeTo(ref tw, item4); + TuplePackers.SerializeTo(ref tw, item5); + writer = tw.Output; + } + + /// Pack a 6-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + TuplePackers.SerializeTo(ref tw, item4); + TuplePackers.SerializeTo(ref tw, item5); + TuplePackers.SerializeTo(ref tw, item6); + writer = tw.Output; + } + + /// Pack a 6-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + TuplePackers.SerializeTo(ref tw, item4); + TuplePackers.SerializeTo(ref tw, item5); + TuplePackers.SerializeTo(ref tw, item6); + TuplePackers.SerializeTo(ref tw, item7); + writer = tw.Output; + } + + /// Pack a 6-tuple directly into a slice + public static void WriteKeysTo(ref SliceWriter writer, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + var tw = new TupleWriter(writer); + TuplePackers.SerializeTo(ref tw, item1); + TuplePackers.SerializeTo(ref tw, item2); + TuplePackers.SerializeTo(ref tw, item3); + TuplePackers.SerializeTo(ref tw, item4); + TuplePackers.SerializeTo(ref tw, item5); + TuplePackers.SerializeTo(ref tw, item6); + TuplePackers.SerializeTo(ref tw, item7); + TuplePackers.SerializeTo(ref tw, item8); + writer = tw.Output; + } + + /// Merge a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) + { + Contract.NotNull(keys, nameof(keys)); + + // use optimized version for arrays + if (keys is T[] array) return EncodePrefixedKeys(prefix, array); + + var next = new List((keys as ICollection)?.Count ?? 0); + var writer = new TupleWriter(); + var packer = TuplePacker.Encoder; + + //TODO: use multiple buffers if item count is huge ? + + bool hasPrefix = prefix.IsPresent; + + foreach (var key in keys) + { + if (hasPrefix) writer.Output.WriteBytes(prefix); + packer(ref writer, key); + next.Add(writer.Output.Position); + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + [NotNull] + public static Slice[] EncodeKeys([NotNull] params T[] keys) + { + var empty = default(Slice); + return EncodePrefixedKeys(empty, keys); + } + + /// Merge an array of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) + { + Contract.NotNull(keys, nameof(keys)); + + // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize + var writer = new TupleWriter(keys.Length * (prefix.Count + 8)); + var next = new List(keys.Length); + var packer = TuplePacker.Encoder; + + //TODO: use multiple buffers if item count is huge ? + + foreach (var key in keys) + { + if (prefix.Count > 0) writer.Output.WriteBytes(prefix); + packer(ref writer, key); + next.Add(writer.Output.Position); + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + /// Merge an array of elements, all sharing the same buffer + /// Type of the elements + /// Type of the keys extracted from the elements + /// Sequence of elements to pack + /// Lambda that extract the key from each element + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] Func selector) + { + var empty = default(Slice); + return EncodePrefixedKeys(empty, elements, selector); + } + + /// Merge an array of elements with a same prefix, all sharing the same buffer + /// Type of the elements + /// Type of the keys extracted from the elements + /// Prefix shared by all keys (can be empty) + /// Sequence of elements to pack + /// Lambda that extract the key from each element + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) + { + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(selector, nameof(selector)); + + // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize + var writer = new TupleWriter(elements.Length * (prefix.Count + 8)); + var next = new List(elements.Length); + var packer = TuplePacker.Encoder; + + //TODO: use multiple buffers if item count is huge ? + + foreach (var value in elements) + { + if (prefix.Count > 0) writer.Output.WriteBytes(prefix); + packer(ref writer, selector(value)); + next.Add(writer.Output.Position); + } + + return Slice.SplitIntoSegments(writer.Output.Buffer, 0, next); + } + + /// Pack a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the prefix tuple + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodePrefixedKeys([NotNull] TTuple prefix, [NotNull] IEnumerable keys) + where TTuple : ITuple + { + Contract.NotNullAllowStructs(prefix, nameof(prefix)); + var head = Pack(prefix); + return EncodePrefixedKeys(head, keys); + } + + /// Pack a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the prefix tuple + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] EncodePrefixedKeys([NotNull] TTuple prefix, [NotNull] params T1[] keys) + where TTuple : ITuple + { + Contract.NotNullAllowStructs(prefix, nameof(prefix)); + + var head = Pack(prefix); + return EncodePrefixedKeys(head, keys); + } + + #endregion + + #region Unpacking... + + /// Unpack a tuple from a serialied key blob + /// Binary key containing a previously packed tuple + /// Unpacked tuple, or the empty tuple if the key is + /// If is equal to + [NotNull] + public static ITuple Unpack(Slice packedKey) + { + if (packedKey.IsNull) throw new ArgumentNullException(nameof(packedKey)); + if (packedKey.Count == 0) return STuple.Empty; + + return TuplePackers.Unpack(packedKey, false); + } + + /// Unpack a tuple from a binary representation + /// Binary key containing a previously packed tuple, or Slice.Nil + /// Unpacked tuple, the empty tuple if is equal to , or null if the key is + [CanBeNull] + public static ITuple UnpackOrDefault(Slice packedKey) + { + if (packedKey.IsNull) return null; + if (packedKey.Count == 0) return STuple.Empty; + return TuplePackers.Unpack(packedKey, false); + } + + /// Unpack a tuple and only return its first element + /// Type of the first value in the decoded tuple + /// Slice that should be entirely parsable as a tuple + /// Decoded value of the first item in the tuple + public static T DecodeFirst(Slice packedKey) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the first element of an empty tuple"); + + var slice = TuplePackers.UnpackFirst(packedKey); + if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); + + return TuplePacker.Deserialize(slice); + } + + /// Unpack a tuple and only return its last element + /// Type of the last value in the decoded tuple + /// Slice that should be entirely parsable as a tuple + /// Decoded value of the last item in the tuple + public static T DecodeLast(Slice packedKey) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the last element of an empty tuple"); + + var slice = TuplePackers.UnpackLast(packedKey); + if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); + + return TuplePacker.Deserialize(slice); + } + + /// Unpack the value of a singleton tuple + /// Type of the single value in the decoded tuple + /// Slice that should contain the packed representation of a tuple with a single element + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than one element. + public static void DecodeKey(Slice packedKey, out STuple tuple) //REVIEW: or T1 instead of STuple ? + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack a single value out of an empty tuple"); + + var slice = TuplePackers.UnpackSingle(packedKey); + if (slice.IsNull) throw new InvalidOperationException("Failed to unpack singleton tuple"); + + tuple = new STuple(TuplePacker.Deserialize(slice)); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) //REVIEW: or T1 instead of STuple ? + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than two items"); + + tuple = new STuple(item1); + } + + /// Unpack a key containing two elements + /// Slice that should contain the packed representation of a tuple with two elements + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than two elements. + public static void DecodeKey(Slice packedKey, out STuple tuple) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); + + var reader = new TupleReader(packedKey); + DecodeKey(ref reader, out tuple); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out T2 item2)) throw new FormatException("Failed to decode second item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than two items"); + tuple = new STuple(item1, item2); + } + + + public static void DecodeKey(ref TupleReader reader, out T1 item1, out T2 item2) + { + if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than two items"); + } + + /// Unpack a key containing three elements + /// Slice that should contain the packed representation of a tuple with three elements + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than three elements. + public static void DecodeKey(Slice packedKey, out STuple tuple) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); + + var reader = new TupleReader(packedKey); + DecodeKey(ref reader, out tuple); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out T2 item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out T3 item3)) throw new FormatException("Failed to decode third item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than three items"); + tuple = new STuple(item1, item2, item3); + } + + public static void DecodeKey(ref TupleReader reader, out T1 item1, out T2 item2, out T3 item3) + { + if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than three items"); + } + + /// Unpack a key containing four elements + /// Slice that should contain the packed representation of a tuple with four elements + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than four elements. + public static void DecodeKey(Slice packedKey, out STuple tuple) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); + + var reader = new TupleReader(packedKey); + DecodeKey(ref reader, out tuple); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out T2 item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out T3 item3)) throw new FormatException("Failed to decode third item"); + if (!DecodeNext(ref reader, out T4 item4)) throw new FormatException("Failed to decode fourth item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); + tuple = new STuple(item1, item2, item3, item4); + } + + public static void DecodeKey(ref TupleReader reader, out T1 item1, out T2 item2, out T3 item3, out T4 item4) + { + if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); + if (!DecodeNext(ref reader, out item4)) throw new FormatException("Failed to decode fourth item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); + } + + /// Unpack a key containing five elements + /// Slice that should contain the packed representation of a tuple with five elements + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than five elements. + public static void DecodeKey(Slice packedKey, out STuple tuple) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); + + var reader = new TupleReader(packedKey); + DecodeKey(ref reader, out tuple); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out T2 item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out T3 item3)) throw new FormatException("Failed to decode third item"); + if (!DecodeNext(ref reader, out T4 item4)) throw new FormatException("Failed to decode fourth item"); + if (!DecodeNext(ref reader, out T5 item5)) throw new FormatException("Failed to decode fifth item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); + tuple = new STuple(item1, item2, item3, item4, item5); + } + + public static void DecodeKey(ref TupleReader reader, out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5) + { + if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); + if (!DecodeNext(ref reader, out item4)) throw new FormatException("Failed to decode fourth item"); + if (!DecodeNext(ref reader, out item5)) throw new FormatException("Failed to decode fifth item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); + } + + /// Unpack a key containing six elements + /// Slice that should contain the packed representation of a tuple with six elements + /// Receives the decoded tuple + /// Throws an exception if the tuple is empty of has more than six elements. + public static void DecodeKey(Slice packedKey, out STuple tuple) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); + + var reader = new TupleReader(packedKey); + DecodeKey(ref reader, out tuple); + } + + public static void DecodeKey(ref TupleReader reader, out STuple tuple) + { + if (!DecodeNext(ref reader, out T1 item1)) throw new FormatException("Failed to decode first item"); + if (!DecodeNext(ref reader, out T2 item2)) throw new FormatException("Failed to decode second item"); + if (!DecodeNext(ref reader, out T3 item3)) throw new FormatException("Failed to decode third item"); + if (!DecodeNext(ref reader, out T4 item4)) throw new FormatException("Failed to decode fourth item"); + if (!DecodeNext(ref reader, out T5 item5)) throw new FormatException("Failed to decode fifth item"); + if (!DecodeNext(ref reader, out T6 item6)) throw new FormatException("Failed to decode sixth item"); + if (reader.Input.HasMore) throw new FormatException("The key contains more than six items"); + tuple = new STuple(item1, item2, item3, item4, item5, item6); + } + + /// Unpack the next item in the tuple, and advance the cursor + /// Type of the next value in the tuple + /// Reader positionned at the start of the next item to read + /// If decoding succeedsd, receives the decoded value. + /// True if the decoded succeeded (and receives the decoded value). False if the tuple has reached the end. + public static bool DecodeNext(ref TupleReader input, out T value) + { + if (!input.Input.HasMore) + { + value = default(T); + return false; + } + + var slice = TupleParser.ParseNext(ref input); + value = TuplePacker.Deserialize(slice); + return true; + } + + #endregion + + } + +} diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs similarity index 84% rename from FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs index 277d88e11..0d4ec147b 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs @@ -26,11 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples + +namespace Doxense.Collections.Tuples.Encoding { using System; + using Doxense.Collections.Tuples; + using Doxense.Memory; + using FoundationDB; using FoundationDB.Client; + /// Encoder for variable-length elements, that uses the Tuple Binary Encoding format public sealed class TupleKeyEncoder : IDynamicKeyEncoder { @@ -44,15 +49,10 @@ public IKeyEncoding Encoding get { return TypeSystem.Tuples; } } - public KeyRange ToRange(Slice prefix) - { - return STuple.ToRange(prefix); - } - public void PackKey(ref SliceWriter writer, ITuple items) { var tw = new TupleWriter(writer); - STuple.Pack(ref tw, items); + TupleEncoder.WriteTo(ref tw, items); writer = tw.Output; } @@ -142,88 +142,95 @@ public void EncodeKey(ref SliceWriter writer, T1 public ITuple UnpackKey(Slice packed) { - return STuple.Unpack(packed); + return TuPack.Unpack(packed); } public T DecodeKey(Slice packed) { - return STuple.DecodeKey(packed); + return TuPack.DecodeKey(packed); } public T DecodeKeyFirst(Slice packed) { - return STuple.DecodeFirst(packed); + return TuPack.DecodeFirst(packed); } public T DecodeKeyLast(Slice packed) { - return STuple.DecodeLast(packed); + return TuPack.DecodeLast(packed); } public STuple DecodeKey(Slice packed) { - return STuple.DecodeKey(packed); + return TuPack.DecodeKey(packed); } public STuple DecodeKey(Slice packed) { - return STuple.DecodeKey(packed); + return TuPack.DecodeKey(packed); } public STuple DecodeKey(Slice packed) { - return STuple.DecodeKey(packed); + return TuPack.DecodeKey(packed); } public STuple DecodeKey(Slice packed) { - return STuple.DecodeKey(packed); + return TuPack.DecodeKey(packed); } + public KeyRange ToRange(Slice prefix) + { + return TuPack.ToRange(prefix); + } + + public KeyRange ToRange(Slice prefix, ITuple items) { - return STuple.ToRange(prefix, items); + return TuPack.ToRange(prefix, items); } public KeyRange ToKeyRange(Slice prefix, T1 item1) { - return STuple.ToRange(prefix, STuple.Create(item1)); + return TuPack.ToRange(prefix, STuple.Create(item1)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { - return STuple.ToRange(prefix, STuple.Create(item1, item2)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6, item7)); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - return STuple.ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); + return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6, item7, item8)); } + } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs similarity index 94% rename from FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs rename to FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs index 9a29e0fee..ff460024e 100644 --- a/FoundationDB.Client/Layers/Tuples/TypeSystem/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs @@ -27,11 +27,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples.Encoding { using System; using FoundationDB.Client; + /// Encoding that uses the Tuple Binary Encoding format public sealed class TupleKeyEncoding : IKeyEncoding { public IDynamicKeyEncoder GetDynamicEncoder() @@ -59,4 +60,4 @@ public ICompositeKeyEncoder GetEncoder() return KeyValueEncoders.Tuples.CompositeKey(); } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs index bc12202c6..1fc6a469d 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs @@ -26,33 +26,54 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client; -using System; - -namespace FoundationDB.Layers.Tuples + namespace Doxense.Collections.Tuples.Encoding { + using System; + using System.Runtime.CompilerServices; + using Doxense.Memory; + using FoundationDB; + using JetBrains.Annotations; /// Helper class that can serialize values of type to the tuple binary format /// Type of values to be serialized public static class TuplePacker { + [NotNull] internal static readonly TuplePackers.Encoder Encoder = TuplePackers.GetSerializer(required: true); + [NotNull] internal static readonly Func Decoder = TuplePackers.GetDeserializer(required: true); - /// Serialize a into a binary buffer + /// Serialize a using a Tuple Writer /// Target buffer /// Value that will be serialized - /// The buffer does not need to be preallocated. -#if !NET_4_0 - [System.Runtime.CompilerServices.MethodImpl(System.Runtime.CompilerServices.MethodImplOptions.AggressiveInlining)] -#endif + /// + /// The buffer does not need to be preallocated. + /// This method supports embedded tuples. + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, T value) { Encoder(ref writer, value); } + /// Serialize a into a binary buffer + /// Target buffer + /// Value that will be serialized + /// + /// The buffer does not need to be preallocated. + /// This method DOES NOT support embedded tupels, and assumes that we are serializing a top-level Tuple! + /// If you need support for embedded tuples, use instead! + /// + public static void SerializeTo(ref SliceWriter writer, T value) + { + var tw = new TupleWriter(writer); + Encoder(ref tw, value); + writer = tw.Output; + //REVIEW: we loose the depth information here! :( + } + /// Serialize a value of type into a tuple segment /// Value that will be serialized /// Slice that contains the binary representation of @@ -66,14 +87,11 @@ public static Slice Serialize(T value) /// Deserialize a tuple segment into a value of type /// Slice that contains the binary representation of a tuple item /// Decoded value, or an exception if the item type is not compatible -#if !NET_4_0 - [System.Runtime.CompilerServices.MethodImpl(System.Runtime.CompilerServices.MethodImplOptions.AggressiveInlining)] -#endif + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static T Deserialize(Slice slice) { return Decoder(slice); } } - } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index 67f8639cb..a8e8cc9b6 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -26,16 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Collections.Generic; using System.Globalization; + using System.Linq; using System.Linq.Expressions; using System.Reflection; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Helper methods used during serialization of values to the tuple binary format @@ -48,87 +52,171 @@ public static class TuplePackers /// Returns a lambda that will be able to serialize values of type /// Type of values to serialize - /// Reusable action that knows how to serialize values of type into binary buffers, or an exception if the type is not supported - [ContractAnnotation("true => notnull")] + /// Reusable action that knows how to serialize values of type into binary buffers, or that throws an exception if the type is not supported + [CanBeNull, ContractAnnotation("true => notnull")] internal static Encoder GetSerializer(bool required) { - var encoder = (Encoder)GetSerializerFor(typeof(T)); + //note: this method is only called once per initializing of TuplePackers to create the cached delegate. + + var encoder = (Encoder) GetSerializerFor(typeof(T)); if (encoder == null && required) { - encoder = delegate { throw new InvalidOperationException(String.Format("Does not know how to serialize values of type {0} into keys", typeof(T).Name)); }; + encoder = delegate { throw new InvalidOperationException($"Does not know how to serialize values of type '{typeof(T).Name}' into keys"); }; } return encoder; } + [CanBeNull] private static Delegate GetSerializerFor([NotNull] Type type) { - if (type == null) throw new ArgumentNullException("type"); + Contract.NotNull(type, nameof(type)); if (type == typeof(object)) { // return a generic serializer that will inspect the runtime type of the object - return new Encoder(TuplePackers.SerializeObjectTo); + return new Encoder(SerializeObjectTo); } + // look for well-known types that have their own (non-generic) TuplePackers.SerializeTo(...) method var typeArgs = new[] { typeof(TupleWriter).MakeByRefType(), type }; - var method = typeof(TuplePackers).GetMethod("SerializeTo", BindingFlags.Static | BindingFlags.Public, null, typeArgs, null); + var method = typeof(TuplePackers).GetMethod(nameof(SerializeTo), BindingFlags.Static | BindingFlags.Public, binder: null, types: typeArgs, modifiers: null); if (method != null) { // we have a direct serializer return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } - // maybe if it is a tuple ? + // maybe it is a nullable type ? + var nullableType = Nullable.GetUnderlyingType(type); + if (nullableType != null) + { // nullable types can reuse the underlying type serializer + method = typeof(TuplePackers).GetMethod(nameof(SerializeNullableTo), BindingFlags.Static | BindingFlags.Public); + if (method != null) + { + return method.MakeGenericMethod(nullableType).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); + } + } + + // maybe it is a tuple ? if (typeof(ITuple).IsAssignableFrom(type)) { - method = typeof(TuplePackers).GetMethod("SerializeTupleTo", BindingFlags.Static | BindingFlags.Public); + // If so, try to use the corresponding TuplePackers.SerializeTupleTo(...) method + method = typeof(TuplePackers).GetMethod(nameof(SerializeTupleTo), BindingFlags.Static | BindingFlags.Public); if (method != null) { return method.MakeGenericMethod(type).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } } + // Can it transform itself into a tuple? if (typeof(ITupleFormattable).IsAssignableFrom(type)) { - method = typeof(TuplePackers).GetMethod("SerializeFormattableTo", BindingFlags.Static | BindingFlags.Public); + // If so, try to use the corresponding TuplePackers.SerializeFormattableTo(...) method + method = typeof(TuplePackers).GetMethod(nameof(SerializeFormattableTo), BindingFlags.Static | BindingFlags.Public); if (method != null) { return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } } - var nullableType = Nullable.GetUnderlyingType(type); - if (nullableType != null) - { // nullable types can reuse the underlying type serializer - method = typeof(TuplePackers).GetMethod("SerializeNullableTo", BindingFlags.Static | BindingFlags.Public); +#if ENABLE_VALUETUPLES + if ((type.Name == nameof(System.ValueTuple) || type.Name.StartsWith(nameof(System.ValueTuple) + "`", StringComparison.Ordinal)) && type.Namespace == "System") + { + typeArgs = type.GetGenericArguments(); + method = FindValueTupleSerializerMethod(typeArgs); if (method != null) { - return method.MakeGenericMethod(nullableType).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); + return method.MakeGenericMethod(typeArgs).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } } +#endif - // TODO: look for a static SerializeTo(BWB, T) method on the type itself ? + // TODO: look for a static SerializeTo(ref TupleWriter, T) method on the type itself ? // no luck.. return null; } +#if ENABLE_VALUETUPLES + private static MethodInfo FindValueTupleSerializerMethod(Type[] args) + { + //note: we want to find the correct SerializeValueTuple<...>(ref TupleWriter, ValueTuple<...>), but this cannot be done with Type.GetMethod(...) directly + // => we have to scan for all methods with the correct name, and the same number of Type Arguments than the ValueTuple. + return typeof(TuplePackers) + .GetMethods(BindingFlags.Static | BindingFlags.Public) + .SingleOrDefault(m => m.Name == nameof(SerializeValueTupleTo) && m.GetGenericArguments().Length == args.Length); + } +#endif + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static void SerializeTo(ref TupleWriter writer, T value) + { + // + // - In Release builds, this will be cleaned up and inlined by the JIT as a direct invokatino of the correct WriteXYZ method + // - In Debug builds, we have to disabled this, because it would be too slow + //IMPORTANT: only ValueTypes and they must have a corresponding Write$TYPE$(ref TupleWriter, $TYPE) in TupleParser! +#if !DEBUG + if (typeof(T) == typeof(bool)) { TupleParser.WriteBool(ref writer, (bool) (object) value); return; } + if (typeof(T) == typeof(int)) { TupleParser.WriteInt32(ref writer, (int) (object) value); return; } + if (typeof(T) == typeof(long)) { TupleParser.WriteInt64(ref writer, (long) (object) value); return; } + if (typeof(T) == typeof(uint)) { TupleParser.WriteUInt32(ref writer, (uint) (object) value); return; } + if (typeof(T) == typeof(ulong)) { TupleParser.WriteUInt64(ref writer, (ulong) (object) value); return; } + if (typeof(T) == typeof(short)) { TupleParser.WriteInt32(ref writer, (short) (object) value); return; } + if (typeof(T) == typeof(ushort)) { TupleParser.WriteUInt32(ref writer, (ushort) (object) value); return; } + if (typeof(T) == typeof(sbyte)) { TupleParser.WriteInt32(ref writer, (sbyte) (object) value); return; } + if (typeof(T) == typeof(byte)) { TupleParser.WriteUInt32(ref writer, (byte) (object) value); return; } + if (typeof(T) == typeof(float)) { TupleParser.WriteSingle(ref writer, (float) (object) value); return; } + if (typeof(T) == typeof(double)) { TupleParser.WriteDouble(ref writer, (double) (object) value); return; } + if (typeof(T) == typeof(char)) { TupleParser.WriteChar(ref writer, (char) (object) value); return; } + if (typeof(T) == typeof(Guid)) { TupleParser.WriteGuid(ref writer, (Guid) (object) value); return; } + if (typeof(T) == typeof(Uuid128)) { TupleParser.WriteUuid128(ref writer, (Uuid128) (object) value); return; } + if (typeof(T) == typeof(Uuid64)) { TupleParser.WriteUuid64(ref writer, (Uuid64) (object) value); return; } + if (typeof(T) == typeof(decimal)) { TupleParser.WriteDecimal(ref writer, (decimal) (object) value); return; } + if (typeof(T) == typeof(Slice)) { TupleParser.WriteBytes(ref writer, (Slice) (object) value); return; } + + if (typeof(T) == typeof(bool?)) { TupleParser.WriteBool(ref writer, (bool?) (object) value); return; } + if (typeof(T) == typeof(int?)) { TupleParser.WriteInt32(ref writer, (int?) (object) value); return; } + if (typeof(T) == typeof(long?)) { TupleParser.WriteInt64(ref writer, (long?) (object) value); return; } + if (typeof(T) == typeof(uint?)) { TupleParser.WriteUInt32(ref writer, (uint?) (object) value); return; } + if (typeof(T) == typeof(ulong?)) { TupleParser.WriteUInt64(ref writer, (ulong?) (object) value); return; } + if (typeof(T) == typeof(short?)) { TupleParser.WriteInt32(ref writer, (short?) (object) value); return; } + if (typeof(T) == typeof(ushort?)) { TupleParser.WriteUInt32(ref writer, (ushort?) (object) value); return; } + if (typeof(T) == typeof(sbyte?)) { TupleParser.WriteInt32(ref writer, (sbyte?) (object) value); return; } + if (typeof(T) == typeof(byte?)) { TupleParser.WriteUInt32(ref writer, (byte?) (object) value); return; } + if (typeof(T) == typeof(float?)) { TupleParser.WriteSingle(ref writer, (float?) (object) value); return; } + if (typeof(T) == typeof(double?)) { TupleParser.WriteDouble(ref writer, (double?) (object) value); return; } + if (typeof(T) == typeof(char?)) { TupleParser.WriteChar(ref writer, (char?) (object) value); return; } + if (typeof(T) == typeof(Guid?)) { TupleParser.WriteGuid(ref writer, (Guid?) (object) value); return; } + if (typeof(T) == typeof(Uuid128?)) { TupleParser.WriteUuid128(ref writer, (Uuid128?) (object) value); return; } + if (typeof(T) == typeof(Uuid64?)) { TupleParser.WriteUuid64(ref writer, (Uuid64?) (object) value); return; } + if (typeof(T) == typeof(decimal?)) { TupleParser.WriteDecimal(ref writer, (decimal?) (object) value); return; } +#endif + // + + // invoke the encoder directly + TuplePacker.Encoder(ref writer, value); + } + /// Serialize a nullable value, by checking for null at runtime /// Underling type of the nullable type /// Target buffer /// Nullable value to serialize /// Uses the underlying type's serializer if the value is not null + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeNullableTo(ref TupleWriter writer, T? value) where T : struct { if (value == null) TupleParser.WriteNil(ref writer); else - TuplePacker.Encoder(ref writer, value.Value); + SerializeTo(ref writer, value.Value); } - /// Serialize an untyped object, by checking its type at runtime + /// Serialize an untyped object, by checking its type at runtime [VERY SLOW] /// Target buffer /// Untyped value whose type will be inspected at runtime - /// May throw at runtime if the type is not supported + /// + /// May throw at runtime if the type is not supported. + /// This method will be very slow! Please consider using typed tuples instead! + /// public static void SerializeObjectTo(ref TupleWriter writer, object value) { if (value == null) @@ -143,46 +231,39 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) case TypeCode.Empty: case TypeCode.Object: { - byte[] bytes = value as byte[]; - if (bytes != null) + if (value is byte[] bytes) { SerializeTo(ref writer, bytes); return; } - if (value is Slice) - { - SerializeTo(ref writer, (Slice)value); - return; - } - - if (value is Guid) + if (value is Slice slice) { - SerializeTo(ref writer, (Guid)value); + SerializeTo(ref writer, slice); return; } - if (value is Uuid128) + if (value is Guid g) { - SerializeTo(ref writer, (Uuid128)value); + SerializeTo(ref writer, g); return; } - if (value is Uuid64) + if (value is Uuid128 u128) { - SerializeTo(ref writer, (Uuid64)value); + SerializeTo(ref writer, u128); return; } - if (value is TimeSpan) + if (value is Uuid64 u64) { - SerializeTo(ref writer, (TimeSpan)value); + SerializeTo(ref writer, u64); return; } - if (value is FdbTupleAlias) + if (value is TimeSpan ts) { - SerializeTo(ref writer, (FdbTupleAlias)value); + SerializeTo(ref writer, ts); return; } @@ -264,58 +345,54 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) SerializeTo(ref writer, (float)value); return; } + case TypeCode.Decimal: + { + SerializeTo(ref writer, (decimal)value); + return; + } } - var tuple = value as ITuple; - if (tuple != null) + if (value is ITuple tuple) { SerializeTupleTo(ref writer, tuple); return; } - var fmt = value as ITupleFormattable; - if (fmt != null) + if (value is ITupleFormattable fmt) { tuple = fmt.ToTuple(); - if (tuple == null) throw new InvalidOperationException(String.Format("An instance of type {0} returned a null Tuple while serialiazing", value.GetType().Name)); + if (tuple == null) throw new InvalidOperationException($"An instance of type '{value.GetType().Name}' returned a null Tuple while serialiazing"); SerializeTupleTo(ref writer, tuple); return; } // Not Supported ? - throw new NotSupportedException(String.Format("Doesn't know how to serialize objects of type {0} into Tuple Encoding format", value.GetType().Name)); + throw new NotSupportedException($"Doesn't know how to serialize objects of type '{value.GetType().Name}' into Tuple Encoding format"); } /// Writes a slice as a byte[] array + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, Slice value) { - if (value.IsNull) - { - TupleParser.WriteNil(ref writer); - } - else if (value.Offset == 0 && value.Count == value.Array.Length) - { - TupleParser.WriteBytes(ref writer, value.Array); - } - else - { - TupleParser.WriteBytes(ref writer, value.Array, value.Offset, value.Count); - } + TupleParser.WriteBytes(ref writer, value); } /// Writes a byte[] array + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, byte[] value) { TupleParser.WriteBytes(ref writer, value); } /// Writes an array segment as a byte[] array + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, ArraySegment value) { - SerializeTo(ref writer, Slice.Create(value)); + TupleParser.WriteBytes(ref writer, value); } /// Writes a char as Unicode string + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, char value) { TupleParser.WriteChar(ref writer, value); @@ -323,49 +400,50 @@ public static void SerializeTo(ref TupleWriter writer, char value) /// Writes a boolean as an integer /// Uses 0 for false, and -1 for true + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, bool value) { TupleParser.WriteBool(ref writer, value); } /// Writes a boolean as an integer or null + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, bool? value) { - if (value == null) - { // null => 00 - TupleParser.WriteNil(ref writer); - } - else - { - TupleParser.WriteBool(ref writer, value.Value); - } + //REVIEW: only method for a nullable type? add others? or remove this one? + TupleParser.WriteBool(ref writer, value); } /// Writes a signed byte as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, sbyte value) { TupleParser.WriteInt32(ref writer, value); } /// Writes an unsigned byte as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, byte value) { TupleParser.WriteByte(ref writer, value); } /// Writes a signed word as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, short value) { TupleParser.WriteInt32(ref writer, value); } /// Writes an unsigned word as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, ushort value) { TupleParser.WriteUInt32(ref writer, value); } /// Writes a signed int as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, int value) { TupleParser.WriteInt32(ref writer, value); @@ -378,36 +456,48 @@ public static void SerializeTo(ref TupleWriter writer, uint value) } /// Writes a signed long as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, long value) { TupleParser.WriteInt64(ref writer, value); } /// Writes an unsigned long as an integer + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, ulong value) { TupleParser.WriteUInt64(ref writer, value); } /// Writes a 32-bit IEEE floating point number + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, float value) { TupleParser.WriteSingle(ref writer, value); } /// Writes a 64-bit IEEE floating point number + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, double value) { TupleParser.WriteDouble(ref writer, value); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void SerializeTo(ref TupleWriter writer, decimal value) + { + TupleParser.WriteDecimal(ref writer, value); + } + /// Writes a string as an Unicode string + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, string value) { TupleParser.WriteString(ref writer, value); } /// Writes a DateTime converted to the number of days since the Unix Epoch and stored as a 64-bit decimal + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, DateTime value) { // The problem of serializing DateTime: TimeZone? Precision? @@ -422,12 +512,11 @@ public static void SerializeTo(ref TupleWriter writer, DateTime value) // => JS binding MAY support decoding of 64-bit floats in the future, in which case the value would be preserved exactly. const long UNIX_EPOCH_EPOCH = 621355968000000000L; - double ms = (value.ToUniversalTime().Ticks - UNIX_EPOCH_EPOCH) / (double)TimeSpan.TicksPerDay; - - TupleParser.WriteDouble(ref writer, ms); + TupleParser.WriteDouble(ref writer, (value.ToUniversalTime().Ticks - UNIX_EPOCH_EPOCH) / (double)TimeSpan.TicksPerDay); } /// Writes a TimeSpan converted to to a number seconds encoded as a 64-bit decimal + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, TimeSpan value) { // We have the same precision problem with storing DateTimes: @@ -442,6 +531,7 @@ public static void SerializeTo(ref TupleWriter writer, TimeSpan value) } /// Writes a Guid as a 128-bit UUID + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, Guid value) { //REVIEW: should we consider serializing Guid.Empty as <14> (integer 0) ? or maybe <01><00> (empty bytestring) ? @@ -450,12 +540,14 @@ public static void SerializeTo(ref TupleWriter writer, Guid value) } /// Writes a Uuid as a 128-bit UUID + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, Uuid128 value) { TupleParser.WriteUuid128(ref writer, value); } /// Writes a Uuid as a 64-bit UUID + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void SerializeTo(ref TupleWriter writer, Uuid64 value) { TupleParser.WriteUuid64(ref writer, value); @@ -464,26 +556,21 @@ public static void SerializeTo(ref TupleWriter writer, Uuid64 value) /// Writes an IPaddress as a 32-bit (IPv4) or 128-bit (IPv6) byte array public static void SerializeTo(ref TupleWriter writer, System.Net.IPAddress value) { - TupleParser.WriteBytes(ref writer, value != null ? value.GetAddressBytes() : null); - } - - public static void SerializeTo(ref TupleWriter writer, FdbTupleAlias value) - { - Contract.Requires(Enum.IsDefined(typeof(FdbTupleAlias), value)); - - writer.Output.WriteByte((byte)value); + TupleParser.WriteBytes(ref writer, value?.GetAddressBytes()); } + /// Serialize an embedded tuples public static void SerializeTupleTo(ref TupleWriter writer, TTuple tuple) where TTuple : ITuple { Contract.Requires(tuple != null); TupleParser.BeginTuple(ref writer); - tuple.PackTo(ref writer); + TupleEncoder.WriteTo(ref writer, tuple); TupleParser.EndTuple(ref writer); } + /// Serialize an embedded tuple formattable public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormattable formattable) { if (formattable == null) @@ -493,44 +580,108 @@ public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormatta } var tuple = formattable.ToTuple(); - if (tuple == null) throw new InvalidOperationException(String.Format("Custom formatter {0}.ToTuple() cannot return null", formattable.GetType().Name)); + if (tuple == null) throw new InvalidOperationException($"Custom formatter {formattable.GetType().Name}.ToTuple() cannot return null"); TupleParser.BeginTuple(ref writer); - tuple.PackTo(ref writer); + TupleEncoder.WriteTo(ref writer, tuple); TupleParser.EndTuple(ref writer); } +#if ENABLE_VALUETUPLES + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + SerializeTo(ref writer, tuple.Item5); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + SerializeTo(ref writer, tuple.Item5); + SerializeTo(ref writer, tuple.Item6); + TupleParser.EndTuple(ref writer); + } + +#endif + #endregion #region Deserializers... - private static readonly Dictionary s_sliceUnpackers = InitializeDefaultUnpackers(); + private static readonly Dictionary WellKnownUnpackers = InitializeDefaultUnpackers(); [NotNull] private static Dictionary InitializeDefaultUnpackers() { - var map = new Dictionary(); - - map[typeof(Slice)] = new Func(TuplePackers.DeserializeSlice); - map[typeof(byte[])] = new Func(TuplePackers.DeserializeBytes); - map[typeof(bool)] = new Func(TuplePackers.DeserializeBoolean); - map[typeof(string)] = new Func(TuplePackers.DeserializeString); - map[typeof(sbyte)] = new Func(TuplePackers.DeserializeSByte); - map[typeof(short)] = new Func(TuplePackers.DeserializeInt16); - map[typeof(int)] = new Func(TuplePackers.DeserializeInt32); - map[typeof(long)] = new Func(TuplePackers.DeserializeInt64); - map[typeof(byte)] = new Func(TuplePackers.DeserializeByte); - map[typeof(ushort)] = new Func(TuplePackers.DeserializeUInt16); - map[typeof(uint)] = new Func(TuplePackers.DeserializeUInt32); - map[typeof(ulong)] = new Func(TuplePackers.DeserializeUInt64); - map[typeof(float)] = new Func(TuplePackers.DeserializeSingle); - map[typeof(double)] = new Func(TuplePackers.DeserializeDouble); - map[typeof(Guid)] = new Func(TuplePackers.DeserializeGuid); - map[typeof(Uuid128)] = new Func(TuplePackers.DeserializeUuid128); - map[typeof(Uuid64)] = new Func(TuplePackers.DeserializeUuid64); - map[typeof(TimeSpan)] = new Func(TuplePackers.DeserializeTimeSpan); - map[typeof(DateTime)] = new Func(TuplePackers.DeserializeDateTime); - map[typeof(System.Net.IPAddress)] = new Func(TuplePackers.DeserializeIPAddress); + var map = new Dictionary + { + [typeof(Slice)] = new Func(TuplePackers.DeserializeSlice), + [typeof(byte[])] = new Func(TuplePackers.DeserializeBytes), + [typeof(bool)] = new Func(TuplePackers.DeserializeBoolean), + [typeof(string)] = new Func(TuplePackers.DeserializeString), + [typeof(char)] = new Func(TuplePackers.DeserializeChar), + [typeof(sbyte)] = new Func(TuplePackers.DeserializeSByte), + [typeof(short)] = new Func(TuplePackers.DeserializeInt16), + [typeof(int)] = new Func(TuplePackers.DeserializeInt32), + [typeof(long)] = new Func(TuplePackers.DeserializeInt64), + [typeof(byte)] = new Func(TuplePackers.DeserializeByte), + [typeof(ushort)] = new Func(TuplePackers.DeserializeUInt16), + [typeof(uint)] = new Func(TuplePackers.DeserializeUInt32), + [typeof(ulong)] = new Func(TuplePackers.DeserializeUInt64), + [typeof(float)] = new Func(TuplePackers.DeserializeSingle), + [typeof(double)] = new Func(TuplePackers.DeserializeDouble), + [typeof(Guid)] = new Func(TuplePackers.DeserializeGuid), + [typeof(Uuid128)] = new Func(TuplePackers.DeserializeUuid128), + [typeof(Uuid64)] = new Func(TuplePackers.DeserializeUuid64), + [typeof(TimeSpan)] = new Func(TuplePackers.DeserializeTimeSpan), + [typeof(DateTime)] = new Func(TuplePackers.DeserializeDateTime), + [typeof(System.Net.IPAddress)] = new Func(TuplePackers.DeserializeIPAddress), + [typeof(ITuple)] = new Func(TuplePackers.DeserializeTuple), + }; // add Nullable versions for all these types return map; @@ -544,36 +695,60 @@ internal static Func GetDeserializer(bool required) { Type type = typeof(T); - Delegate decoder; - if (s_sliceUnpackers.TryGetValue(type, out decoder)) - { - return (Func)decoder; + if (WellKnownUnpackers.TryGetValue(type, out var decoder)) + { // We already know how to decode this type + return (Func) decoder; } - //TODO: handle nullable types? + // Nullable var underlyingType = Nullable.GetUnderlyingType(typeof(T)); - if (underlyingType != null && s_sliceUnpackers.TryGetValue(underlyingType, out decoder)) + if (underlyingType != null && WellKnownUnpackers.TryGetValue(underlyingType, out decoder)) + { + return (Func) MakeNullableDeserializer(type, underlyingType, decoder); + } + + // STuple<...> + if (typeof(ITuple).IsAssignableFrom(type)) { - decoder = MakeNullableDeserializer(type, underlyingType, decoder); - if (decoder != null) return (Func)decoder; + if (type.IsValueType && type.IsGenericType && type.Name.StartsWith(nameof(STuple) + "`", StringComparison.Ordinal)) + return (Func) MakeSTupleDeserializer(type); } - if (required) +#if ENABLE_VALUETUPLES + if ((type.Name == nameof(ValueTuple) || type.Name.StartsWith(nameof(ValueTuple) + "`", StringComparison.Ordinal)) && type.Namespace == "System") { - return (_) => { throw new InvalidOperationException(String.Format("Does not know how to deserialize keys into values of type {0}", typeof(T).Name)); }; + return (Func) MakeValueTupleDeserializer(type); } - else - { // when all else fails... - return (value) => FdbConverters.ConvertBoxed(DeserializeBoxed(value)); +#endif + + if (required) + { // will throw at runtime + return MakeNotSupportedDeserializer(); } + // when all else fails... + return MakeConvertBoxedDeserializer(); + } + + [Pure, NotNull] + private static Func MakeNotSupportedDeserializer() + { + return (_) => throw new InvalidOperationException($"Does not know how to deserialize keys into values of type {typeof(T).Name}"); + } + + [Pure, NotNull] + private static Func MakeConvertBoxedDeserializer() + { + return (value) => TypeConverters.ConvertBoxed(DeserializeBoxed(value)); } /// Check if a tuple segment is the equivalent of 'Nil' + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] internal static bool IsNilSegment(Slice slice) { return slice.IsNullOrEmpty || slice[0] == TupleTypes.Nil; } + [Pure, NotNull] private static Delegate MakeNullableDeserializer([NotNull] Type nullableType, [NotNull] Type type, [NotNull] Delegate decoder) { Contract.Requires(nullableType != null && type != null && decoder != null); @@ -582,7 +757,7 @@ private static Delegate MakeNullableDeserializer([NotNull] Type nullableType, [N var prmSlice = Expression.Parameter(typeof(Slice), "slice"); var body = Expression.Condition( // IsNilSegment(slice) ? - Expression.Call(typeof(TuplePackers).GetMethod("IsNilSegment", BindingFlags.Static | BindingFlags.NonPublic), prmSlice), + Expression.Call(typeof(TuplePackers).GetMethod(nameof(IsNilSegment), BindingFlags.Static | BindingFlags.NonPublic), prmSlice), // True => default(Nullable) Expression.Default(nullableType), // False => decoder(slice) @@ -592,6 +767,62 @@ private static Delegate MakeNullableDeserializer([NotNull] Type nullableType, [N return Expression.Lambda(body, prmSlice).Compile(); } + [Pure, NotNull] + private static Delegate MakeSTupleDeserializer(Type type) + { + Contract.Requires(type != null); + + // (slice) => TuPack.DeserializeTuple(slice) + + var targs = type.GetGenericArguments(); + var method = typeof(TuplePackers) + .GetMethods() + .Single(m => + { // find the matching "DeserializeTuple(Slice)" method that we want to call + if (m.Name != nameof(DeserializeTuple)) return false; + if (!m.IsGenericMethod || m.GetGenericArguments().Length != targs.Length) return false; + var args = m.GetParameters(); + if (args.Length != 1 && args[0].ParameterType != typeof(Slice)) return false; + return true; + }) + .MakeGenericMethod(targs); + + var prmSlice = Expression.Parameter(typeof(Slice), "slice"); + var body = Expression.Call(method, prmSlice); + + return Expression.Lambda(body, prmSlice).Compile(); + } + +#if ENABLE_VALUETUPLES + + [Pure, NotNull] + private static Delegate MakeValueTupleDeserializer(Type type) + { + Contract.Requires(type != null); + + // (slice) => TuPack.DeserializeValueTuple(slice) + + var targs = type.GetGenericArguments(); + var method = typeof(TuplePackers) + .GetMethods() + .Single(m => + { // find the matching "DeserializeValueTuple(Slice)" method that we want to call + if (m.Name != nameof(DeserializeValueTuple)) return false; + if (!m.IsGenericMethod || m.GetGenericArguments().Length != targs.Length) return false; + var args = m.GetParameters(); + if (args.Length != 1 && args[0].ParameterType != typeof(Slice)) return false; + return true; + }) + .MakeGenericMethod(targs); + + var prmSlice = Expression.Parameter(typeof(Slice), "slice"); + var body = Expression.Call(method, prmSlice); + + return Expression.Lambda(body, prmSlice).Compile(); + } + +#endif + /// Deserialize a packed element into an object by choosing the most appropriate type at runtime /// Slice that contains a single packed element /// Decoded element, in the type that is the best fit. @@ -620,14 +851,14 @@ public static object DeserializeBoxed(Slice slice) { case TupleTypes.Single: return TupleParser.ParseSingle(slice); case TupleTypes.Double: return TupleParser.ParseDouble(slice); + //TODO: Triple + case TupleTypes.Decimal: return TupleParser.ParseDecimal(slice); case TupleTypes.Uuid128: return TupleParser.ParseGuid(slice); case TupleTypes.Uuid64: return TupleParser.ParseUuid64(slice); - case TupleTypes.AliasDirectory: return FdbTupleAlias.Directory; - case TupleTypes.AliasSystem: return FdbTupleAlias.System; } } - throw new FormatException(String.Format("Cannot convert tuple segment with unknown type code {0}", type)); + throw new FormatException($"Cannot convert tuple segment with unknown type code 0x{type:X}"); } /// Deserialize a slice into a type that implements ITupleFormattable @@ -680,6 +911,8 @@ public static Slice DeserializeSlice(Slice slice) case TupleTypes.Single: return Slice.FromSingle(TupleParser.ParseSingle(slice)); case TupleTypes.Double: return Slice.FromDouble(TupleParser.ParseDouble(slice)); + //TODO: triple + case TupleTypes.Decimal: return Slice.FromDecimal(TupleParser.ParseDecimal(slice)); case TupleTypes.Uuid128: return Slice.FromGuid(TupleParser.ParseGuid(slice)); case TupleTypes.Uuid64: return Slice.FromUuid64(TupleParser.ParseUuid64(slice)); @@ -691,11 +924,11 @@ public static Slice DeserializeSlice(Slice slice) return Slice.FromUInt64(DeserializeUInt64(slice)); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a Slice", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a Slice"); } /// Deserialize a tuple segment into a byte array - [CanBeNull] //REVIEW: because of Slice.GetBytes() + [CanBeNull, MethodImpl(MethodImplOptions.AggressiveInlining)] //REVIEW: because of Slice.GetBytes() public static byte[] DeserializeBytes(Slice slice) { return DeserializeSlice(slice).GetBytes(); @@ -716,17 +949,259 @@ public static ITuple DeserializeTuple(Slice slice) } case TupleTypes.Bytes: { - return STuple.Unpack(TupleParser.ParseBytes(slice)); + return TupleEncoder.Unpack(TupleParser.ParseBytes(slice)); } case TupleTypes.TupleStart: { return TupleParser.ParseTuple(slice); } + default: + { + throw new FormatException("Cannot convert tuple segment into a Tuple"); + } + } + } + + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } + } + return res; + } + + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } + } + return res; + } + + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } + } + return res; + + } + + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } } + return res; - throw new FormatException("Cannot convert tuple segment into a Tuple"); } + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } + } + return res; + } + + [Pure] + public static STuple DeserializeTuple(Slice slice) + { + var res = default(STuple); + if (slice.IsPresent) + { + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + break; + } + case TupleTypes.Bytes: + { + TupleEncoder.DecodeKey(TupleParser.ParseBytes(slice), out res); + break; + } + case TupleTypes.TupleStart: + { + var reader = TupleReader.Embedded(slice); + TupleEncoder.DecodeKey(ref reader, out res); + break; + } + default: + { + throw new FormatException($"Cannot convert tuple segment into a {res.GetType().Name}"); + } + } + } + return res; + } + +#if ENABLE_VALUETUPLES + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ValueTuple DeserializeValueTuple(Slice slice) + { + return DeserializeTuple(slice).ToValueTuple(); + } + +#endif + /// Deserialize a tuple segment into a Boolean /// Slice that contains a single packed element public static bool DeserializeBoolean(Slice slice) @@ -755,18 +1230,27 @@ public static bool DeserializeBoolean(Slice slice) case TupleTypes.Single: { //TODO: should NaN considered to be false ? + //=> it is the "null" of the floats, so if we do, 'null' should also be considered false + // ReSharper disable once CompareOfFloatsByEqualityOperator return 0f != TupleParser.ParseSingle(slice); } case TupleTypes.Double: { //TODO: should NaN considered to be false ? - return 0f != TupleParser.ParseDouble(slice); + //=> it is the "null" of the floats, so if we do, 'null' should also be considered false + // ReSharper disable once CompareOfFloatsByEqualityOperator + return 0d != TupleParser.ParseDouble(slice); + } + //TODO: triple + case TupleTypes.Decimal: + { + return 0m != TupleParser.ParseDecimal(slice); } } //TODO: should we handle weird cases like strings "True" and "False"? - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a boolean", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a boolean"); } /// Deserialize a tuple segment into an Int16 @@ -809,7 +1293,7 @@ public static long DeserializeInt64(Slice slice) } } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a signed integer", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a signed integer"); } /// Deserialize a tuple segment into an UInt32 @@ -853,7 +1337,7 @@ public static ulong DeserializeUInt64(Slice slice) } } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into an unsigned integer", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into an unsigned integer"); } public static float DeserializeSingle(Slice slice) @@ -865,11 +1349,12 @@ public static float DeserializeSingle(Slice slice) { case TupleTypes.Nil: { + //REVIEW: or should we retourne NaN? return 0; } case TupleTypes.Utf8: { - return Single.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + return float.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } case TupleTypes.Single: { @@ -877,16 +1362,20 @@ public static float DeserializeSingle(Slice slice) } case TupleTypes.Double: { - return (float)TupleParser.ParseDouble(slice); + return (float) TupleParser.ParseDouble(slice); + } + case TupleTypes.Decimal: + { + return (float) TupleParser.ParseDecimal(slice); } } if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { - return checked((float)DeserializeInt64(slice)); + return DeserializeInt64(slice); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a Single", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a Single"); } public static double DeserializeDouble(Slice slice) @@ -898,28 +1387,38 @@ public static double DeserializeDouble(Slice slice) { case TupleTypes.Nil: { + //REVIEW: or should we retourne NaN? return 0; } case TupleTypes.Utf8: { - return Double.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); + return double.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } case TupleTypes.Single: { - return (double)TupleParser.ParseSingle(slice); + return TupleParser.ParseSingle(slice); } case TupleTypes.Double: { return TupleParser.ParseDouble(slice); } + case TupleTypes.Decimal: + { + return (double) TupleParser.ParseDecimal(slice); + } } if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) { - return checked((double)DeserializeInt64(slice)); + return DeserializeInt64(slice); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a Double", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a Double"); + } + + public static decimal DeserializeDecimal(Slice slice) + { + throw new NotImplementedException(); } /// Deserialize a tuple segment into a DateTime (UTC) @@ -952,6 +1451,14 @@ public static DateTime DeserializeDateTime(Slice slice) long ticks = UNIX_EPOCH_TICKS + (long)(TupleParser.ParseDouble(slice) * TimeSpan.TicksPerDay); return new DateTime(ticks, DateTimeKind.Utc); } + + case TupleTypes.Decimal: + { + const long UNIX_EPOCH_TICKS = 621355968000000000L; + //note: we can't user TimeSpan.FromDays(...) because it rounds to the nearest millisecond! + long ticks = UNIX_EPOCH_TICKS + (long)(TupleParser.ParseDecimal(slice) * TimeSpan.TicksPerDay); + return new DateTime(ticks, DateTimeKind.Utc); + } } // If we have an integer, we consider it to be a number of Ticks (Windows Only) @@ -960,7 +1467,7 @@ public static DateTime DeserializeDateTime(Slice slice) return new DateTime(DeserializeInt64(slice), DateTimeKind.Utc); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a DateTime", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a DateTime"); } /// Deserialize a tuple segment into a TimeSpan @@ -983,10 +1490,20 @@ public static TimeSpan DeserializeTimeSpan(Slice slice) { // "HH:MM:SS.fffff" return TimeSpan.Parse(TupleParser.ParseUnicode(slice), CultureInfo.InvariantCulture); } + case TupleTypes.Single: + { // Number of seconds + //note: We can't use TimeSpan.FromSeconds(...) because it rounds to the nearest millisecond! + return new TimeSpan((long) (TupleParser.ParseSingle(slice) * TimeSpan.TicksPerSecond)); + } case TupleTypes.Double: { // Number of seconds //note: We can't use TimeSpan.FromSeconds(...) because it rounds to the nearest millisecond! - return new TimeSpan((long)(TupleParser.ParseDouble(slice) * (double)TimeSpan.TicksPerSecond)); + return new TimeSpan((long) (TupleParser.ParseDouble(slice) * TimeSpan.TicksPerSecond)); + } + case TupleTypes.Decimal: + { // Number of seconds + //note: We can't use TimeSpan.FromSeconds(...) because it rounds to the nearest millisecond! + return new TimeSpan((long) (TupleParser.ParseDecimal(slice) * TimeSpan.TicksPerSecond)); } } @@ -996,7 +1513,44 @@ public static TimeSpan DeserializeTimeSpan(Slice slice) return new TimeSpan(DeserializeInt64(slice)); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a TimeSpan", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a TimeSpan"); + } + + /// Deserialize a tuple segment into a Unicode character + /// Slice that contains a single packed element + public static char DeserializeChar(Slice slice) + { + if (slice.IsNullOrEmpty) return '\0'; + + byte type = slice[0]; + switch (type) + { + case TupleTypes.Nil: + { + return '\0'; + } + case TupleTypes.Bytes: + { + var s = TupleParser.ParseBytes(slice); + if (s.Count == 0) return '\0'; + if (s.Count == 1) return (char) s[0]; + throw new FormatException($"Cannot convert buffer of size {s.Count} into a Char"); + } + case TupleTypes.Utf8: + { + var s = TupleParser.ParseUnicode(slice); + if (s.Length == 0) return '\0'; + if (s.Length == 1) return s[0]; + throw new FormatException($"Cannot convert string of size {s.Length} into a Char"); + } + } + + if (type <= TupleTypes.IntPos8 && type >= TupleTypes.IntNeg8) + { + return (char) TupleParser.ParseInt64(type, slice); + } + + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a Char"); } /// Deserialize a tuple segment into a Unicode string @@ -1029,6 +1583,10 @@ public static string DeserializeString(Slice slice) { return TupleParser.ParseDouble(slice).ToString(CultureInfo.InvariantCulture); } + case TupleTypes.Decimal: + { + return TupleParser.ParseDecimal(slice).ToString(CultureInfo.InvariantCulture); + } case TupleTypes.Uuid128: { return TupleParser.ParseGuid(slice).ToString(); @@ -1044,7 +1602,7 @@ public static string DeserializeString(Slice slice) return TupleParser.ParseInt64(type, slice).ToString(CultureInfo.InvariantCulture); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a String", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a String"); } /// Deserialize a tuple segment into Guid @@ -1072,7 +1630,7 @@ public static Guid DeserializeGuid(Slice slice) //REVIEW: should we allow converting a Uuid64 into a Guid? This looks more like a bug than an expected behavior... } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into a System.Guid", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a System.Guid"); } /// Deserialize a tuple segment into 128-bit UUID @@ -1100,7 +1658,7 @@ public static Uuid128 DeserializeUuid128(Slice slice) //REVIEW: should we allow converting a Uuid64 into a Uuid128? This looks more like a bug than an expected behavior... } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into an Uuid128", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into an Uuid128"); } /// Deserialize a tuple segment into 64-bit UUID @@ -1115,11 +1673,11 @@ public static Uuid64 DeserializeUuid64(Slice slice) { case TupleTypes.Bytes: { // expect binary representation as a 16-byte array - return new Uuid64(TupleParser.ParseBytes(slice)); + return Uuid64.Read(TupleParser.ParseBytes(slice)); } case TupleTypes.Utf8: { // expect text representation - return new Uuid64(TupleParser.ParseUnicode(slice)); + return Uuid64.Parse(TupleParser.ParseUnicode(slice)); } case TupleTypes.Uuid64: { @@ -1133,7 +1691,7 @@ public static Uuid64 DeserializeUuid64(Slice slice) } // we don't support negative numbers! - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into an Uuid64", type)); + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into an Uuid64"); } /// Deserialize a tuple segment into Guid @@ -1149,7 +1707,7 @@ public static System.Net.IPAddress DeserializeIPAddress(Slice slice) { case TupleTypes.Bytes: { - return new System.Net.IPAddress(TupleParser.ParseBytes(slice).GetBytes()); + return new System.Net.IPAddress(TupleParser.ParseBytes(slice).GetBytesOrEmpty()); } case TupleTypes.Utf8: { @@ -1157,7 +1715,7 @@ public static System.Net.IPAddress DeserializeIPAddress(Slice slice) } case TupleTypes.Uuid128: { // could be an IPv6 encoded as a 128-bits UUID - return new System.Net.IPAddress(slice.GetBytes()); + return new System.Net.IPAddress(slice.GetBytesOrEmpty()); } } @@ -1168,24 +1726,26 @@ public static System.Net.IPAddress DeserializeIPAddress(Slice slice) return new System.Net.IPAddress(value); } - throw new FormatException(String.Format("Cannot convert tuple segment of type 0x{0:X} into System.Net.IPAddress", type)); - } - - public static FdbTupleAlias DeserializeAlias(Slice slice) - { - if (slice.Count != 1) throw new FormatException("Cannot convert tuple segment into this type"); - return (FdbTupleAlias)slice[0]; + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into System.Net.IPAddress"); } /// Unpack a tuple from a buffer /// Slice that contains the packed representation of a tuple with zero or more elements + /// /// Decoded tuple [NotNull] internal static SlicedTuple Unpack(Slice buffer, bool embedded) { var reader = new TupleReader(buffer); if (embedded) reader.Depth = 1; + return Unpack(ref reader); + } + /// Unpack a tuple from a buffer + /// Reader positionned on the start of the packed representation of a tuple with zero or more elements + /// Decoded tuple + internal static SlicedTuple Unpack(ref TupleReader reader) + { // most tuples will probably fit within (prefix, sub-prefix, id, key) so pre-allocating with 4 should be ok... var items = new Slice[4]; @@ -1202,9 +1762,10 @@ internal static SlicedTuple Unpack(Slice buffer, bool embedded) } if (reader.Input.HasMore) throw new FormatException("Parsing of tuple failed failed before reaching the end of the key"); - return new SlicedTuple(p == 0 ? Slice.EmptySliceArray : items, 0, p); + return new SlicedTuple(p == 0 ? Array.Empty() : items, 0, p); } + /// Ensure that a slice is a packed tuple that contains a single and valid element /// Slice that should contain the packed representation of a singleton tuple /// Decoded slice of the single element in the singleton tuple diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs index 576400166..ca7a07fdf 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs @@ -26,12 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples + namespace Doxense.Collections.Tuples.Encoding { using System; + using System.Runtime.CompilerServices; using System.Text; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; + using Doxense.Memory; using JetBrains.Annotations; /// Helper class that contains low-level encoders for the tuple binary format @@ -48,7 +50,7 @@ public static void WriteNil(ref TupleWriter writer) } else { // inside a tuple, NILs are escaped as <00> - writer.Output.WriteByte2(TupleTypes.Nil, 0xFF); + writer.Output.WriteBytes(TupleTypes.Nil, 0xFF); } } @@ -57,7 +59,24 @@ public static void WriteBool(ref TupleWriter writer, bool value) // To be compatible with other bindings, we will encode False as the number 0, and True as the number 1 if (value) { // true => 15 01 - writer.Output.WriteByte2(TupleTypes.IntPos1, 1); + writer.Output.WriteBytes(TupleTypes.IntPos1, 1); + } + else + { // false => 14 + writer.Output.WriteByte(TupleTypes.IntZero); + } + } + + public static void WriteBool(ref TupleWriter writer, bool? value) + { + // To be compatible with other bindings, we will encode False as the number 0, and True as the number 1 + if (value == null) + { // null => 00 + writer.Output.WriteByte(TupleTypes.Nil); + } + else if (value.Value) + { // true => 15 01 + writer.Output.WriteBytes(TupleTypes.IntPos1, 1); } else { // false => 14 @@ -76,7 +95,7 @@ public static void WriteByte(ref TupleWriter writer, byte value) } else { // 1..255: frequent for array index - writer.Output.WriteByte2(TupleTypes.IntPos1, value); + writer.Output.WriteBytes(TupleTypes.IntPos1, value); } } @@ -95,13 +114,13 @@ public static void WriteInt32(ref TupleWriter writer, int value) if (value > 0) { // 1..255: frequent for array index - writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); + writer.Output.WriteBytes(TupleTypes.IntPos1, (byte)value); return; } if (value > -256) { // -255..-1 - writer.Output.WriteByte2(TupleTypes.IntNeg1, (byte)(255 + value)); + writer.Output.WriteBytes(TupleTypes.IntNeg1, (byte)(255 + value)); return; } } @@ -109,6 +128,12 @@ public static void WriteInt32(ref TupleWriter writer, int value) WriteInt64Slow(ref writer, value); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteInt32(ref TupleWriter writer, int? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteInt32(ref writer, value.Value); + } + /// Writes an Int64 at the end, and advance the cursor /// Target buffer /// Signed QWORD, 64 bits, High Endian @@ -124,13 +149,13 @@ public static void WriteInt64(ref TupleWriter writer, long value) if (value > 0) { // 1..255: frequent for array index - writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); + writer.Output.WriteBytes(TupleTypes.IntPos1, (byte)value); return; } if (value > -256) { // -255..-1 - writer.Output.WriteByte2(TupleTypes.IntNeg1, (byte)(255 + value)); + writer.Output.WriteBytes(TupleTypes.IntNeg1, (byte)(255 + value)); return; } } @@ -138,6 +163,13 @@ public static void WriteInt64(ref TupleWriter writer, long value) WriteInt64Slow(ref writer, value); } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteInt64(ref TupleWriter writer, long? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteInt64(ref writer, value.Value); + } + private static void WriteInt64Slow(ref TupleWriter writer, long value) { // we are only called for values <= -256 or >= 256 @@ -195,7 +227,7 @@ public static void WriteUInt32(ref TupleWriter writer, uint value) } else { // 1..255 - writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); + writer.Output.WriteBytes(TupleTypes.IntPos1, (byte)value); } } else @@ -204,6 +236,12 @@ public static void WriteUInt32(ref TupleWriter writer, uint value) } } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteUInt32(ref TupleWriter writer, uint? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteUInt32(ref writer, value.Value); + } + /// Writes an UInt64 at the end, and advance the cursor /// Target buffer /// Signed QWORD, 64 bits, High Endian @@ -217,7 +255,7 @@ public static void WriteUInt64(ref TupleWriter writer, ulong value) } else { // 1..255 - writer.Output.WriteByte2(TupleTypes.IntPos1, (byte)value); + writer.Output.WriteBytes(TupleTypes.IntPos1, (byte)value); } } else @@ -226,6 +264,12 @@ public static void WriteUInt64(ref TupleWriter writer, ulong value) } } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteUInt64(ref TupleWriter writer, ulong? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteUInt64(ref writer, value.Value); + } + private static void WriteUInt64Slow(ref TupleWriter writer, ulong value) { // We are only called for values >= 256 @@ -295,6 +339,13 @@ public static void WriteSingle(ref TupleWriter writer, float value) writer.Output.Position = p + 5; } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteSingle(ref TupleWriter writer, float? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteSingle(ref writer, value.Value); + } + /// Writes an Double at the end, and advance the cursor /// Target buffer /// IEEE Floating point, 64 bits, High Endian @@ -335,17 +386,21 @@ public static void WriteDouble(ref TupleWriter writer, double value) writer.Output.Position = p + 9; } - /// Writes a binary string - public static void WriteBytes(ref TupleWriter writer, byte[] value) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteDouble(ref TupleWriter writer, double? value) { - if (value == null) - { - WriteNil(ref writer); - } - else - { - WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value); - } + if (!value.HasValue) WriteNil(ref writer); else WriteDouble(ref writer, value.Value); + } + + public static void WriteDecimal(ref TupleWriter writer, decimal value) + { + throw new NotImplementedException(); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteDecimal(ref TupleWriter writer, decimal? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteDecimal(ref writer, value.Value); } /// Writes a string encoded in UTF-8 @@ -357,7 +412,7 @@ public static unsafe void WriteString(ref TupleWriter writer, string value) } else if (value.Length == 0) { // "02 00" - writer.Output.WriteByte2(TupleTypes.Utf8, 0x00); + writer.Output.WriteBytes(TupleTypes.Utf8, 0x00); } else { @@ -384,7 +439,7 @@ internal static unsafe void WriteChars(ref TupleWriter writer, char[] value, int } else { // "02 00" - writer.Output.WriteByte2(TupleTypes.Utf8, 0x00); + writer.Output.WriteBytes(TupleTypes.Utf8, 0x00); } } else @@ -484,11 +539,10 @@ private static unsafe bool TryWriteUnescapedUtf8String(ref TupleWriter writer, c // 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 { - int charsUsed, bytesUsed; - encoder.Convert(ptr, remaining, buf, bufLen, true, out charsUsed, out bytesUsed, out done); + encoder.Convert(ptr, remaining, buf, bufLen, true, out int charsUsed, out int bytesUsed, out done); if (bytesUsed > 0) { - writer.Output.WriteBytes(buf, bytesUsed); + writer.Output.WriteBytes(buf, (uint) bytesUsed); } remaining -= charsUsed; ptr += charsUsed; @@ -510,15 +564,15 @@ public static void WriteChar(ref TupleWriter writer, char value) if (value == 0) { // NUL => "00 0F" // note: \0 is the only unicode character that will produce a zero byte when converted in UTF-8 - writer.Output.WriteByte4(TupleTypes.Utf8, 0x00, 0xFF, 0x00); + writer.Output.WriteBytes(TupleTypes.Utf8, 0x00, 0xFF, 0x00); } else if (value < 0x80) { // 0x00..0x7F => 0xxxxxxx - writer.Output.WriteByte3(TupleTypes.Utf8, (byte)value, 0x00); + writer.Output.WriteBytes(TupleTypes.Utf8, (byte)value, 0x00); } else if (value < 0x800) { // 0x80..0x7FF => 110xxxxx 10xxxxxx => two bytes - writer.Output.WriteByte4(TupleTypes.Utf8, (byte)(0xC0 | (value >> 6)), (byte)(0x80 | (value & 0x3F)), 0x00); + writer.Output.WriteBytes(TupleTypes.Utf8, (byte)(0xC0 | (value >> 6)), (byte)(0x80 | (value & 0x3F)), 0x00); } else { // 0x800..0xFFFF => 11110xxx 10xxxxxx 10xxxxxx 10xxxxxx @@ -532,6 +586,25 @@ public static void WriteChar(ref TupleWriter writer, char value) } } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteChar(ref TupleWriter writer, char? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteChar(ref writer, value.Value); + } + + /// Writes a binary string + public static void WriteBytes(ref TupleWriter writer, byte[] value) + { + if (value == null) + { + WriteNil(ref writer); + } + else + { + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value); + } + } + /// Writes a binary string public static void WriteBytes(ref TupleWriter writer, [NotNull] byte[] value, int offset, int count) { @@ -541,7 +614,31 @@ public static void WriteBytes(ref TupleWriter writer, [NotNull] byte[] value, in /// Writes a binary string public static void WriteBytes(ref TupleWriter writer, ArraySegment value) { - WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value.Array, value.Offset, value.Count); + if (value.Count == 0 && value.Array == null) + { // default(ArraySegment) ~= null + WriteNil(ref writer); + } + else + { + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value.Array, value.Offset, value.Count); + } + } + + /// Writes a binary string + public static void WriteBytes(ref TupleWriter writer, Slice value) + { + if (value.IsNull) + { + WriteNil(ref writer); + } + else if (value.Offset == 0 && value.Count == value.Array.Length) + { + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value.Array); + } + else + { + WriteNulEscapedBytes(ref writer, TupleTypes.Bytes, value.Array, value.Offset, value.Count); + } } /// Writes a buffer with all instances of 0 escaped as '00 FF' @@ -554,6 +651,7 @@ internal static void WriteNulEscapedBytes(ref TupleWriter writer, byte type, [No for (int i = offset, end = offset + count; i < end; ++i) { if (value[i] == 0) ++n; + //TODO: optimize this! } writer.Output.EnsureBytes(n + 2); @@ -564,13 +662,14 @@ internal static void WriteNulEscapedBytes(ref TupleWriter writer, byte type, [No { if (n == count) { // no NULs in the string, can copy all at once - SliceHelpers.CopyBytesUnsafe(buffer, p, value, offset, n); + UnsafeHelpers.CopyUnsafe(buffer, p, value, offset, n); p += n; } else { // we need to escape all NULs for(int i = offset, end = offset + count; i < end; ++i) { + //TODO: optimize this! byte b = value[i]; buffer[p++] = b; if (b == 0) buffer[p++] = 0xFF; @@ -600,7 +699,7 @@ private static void WriteNulEscapedBytes(ref TupleWriter writer, byte type, [Not { if (n == value.Length) { // no NULs in the string, can copy all at once - SliceHelpers.CopyBytesUnsafe(buffer, p, value, 0, n); + UnsafeHelpers.CopyUnsafe(buffer, p, value, 0, n); p += n; } else @@ -621,14 +720,14 @@ public static void WriteGuid(ref TupleWriter writer, Guid value) { writer.Output.EnsureBytes(17); writer.Output.UnsafeWriteByte(TupleTypes.Uuid128); - unsafe - { - // UUIDs are stored using the RFC 4122 standard, so we need to swap some parts of the System.Guid + // Guids should be stored using the RFC 4122 standard, so we need to swap some parts of the System.Guid (handled by Uuid128) + writer.Output.UnsafeWriteUuid128(new Uuid128(value)); + } - byte* ptr = stackalloc byte[16]; - Uuid128.Write(value, ptr); - writer.Output.UnsafeWriteBytes(ptr, 16); - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteGuid(ref TupleWriter writer, Guid? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteGuid(ref writer, value.Value); } /// Writes a RFC 4122 encoded 128-bit UUID @@ -636,12 +735,13 @@ public static void WriteUuid128(ref TupleWriter writer, Uuid128 value) { writer.Output.EnsureBytes(17); writer.Output.UnsafeWriteByte(TupleTypes.Uuid128); - unsafe - { - byte* ptr = stackalloc byte[16]; - value.WriteTo(ptr); - writer.Output.UnsafeWriteBytes(ptr, 16); - } + writer.Output.UnsafeWriteUuid128(value); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteUuid128(ref TupleWriter writer, Uuid128? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteUuid128(ref writer, value.Value); } /// Writes a 64-bit UUID @@ -649,12 +749,13 @@ public static void WriteUuid64(ref TupleWriter writer, Uuid64 value) { writer.Output.EnsureBytes(9); writer.Output.UnsafeWriteByte(TupleTypes.Uuid64); - unsafe - { - byte* ptr = stackalloc byte[8]; - value.WriteTo(ptr); - writer.Output.UnsafeWriteBytes(ptr, 8); - } + writer.Output.UnsafeWriteUuid64(value); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteUuid64(ref TupleWriter writer, Uuid64? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteUuid64(ref writer, value.Value); } /// Mark the start of a new embedded tuple @@ -702,7 +803,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 +820,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 buffer.AsSlice(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); @@ -733,7 +834,7 @@ internal static ArraySegment UnescapeByteStringSlow([NotNull] byte[] buffe int i = 0; if (offsetOfFirstZero > 0) { - SliceHelpers.CopyBytesUnsafe(tmp, 0, buffer, offset, offsetOfFirstZero); + UnsafeHelpers.CopyUnsafe(tmp, 0, buffer, offset, offsetOfFirstZero); p += offsetOfFirstZero; i = offsetOfFirstZero; } @@ -749,7 +850,7 @@ internal static ArraySegment UnescapeByteStringSlow([NotNull] byte[] buffe tmp[i++] = b; } - return new ArraySegment(tmp, 0, i); + return tmp.AsSlice(0, i); } /// Parse a tuple segment containing a byte array @@ -772,7 +873,7 @@ public static string ParseAscii(Slice slice) var decoded = UnescapeByteString(slice.Array, slice.Offset + 1, slice.Count - 2); - return Slice.DefaultEncoding.GetString(decoded.Array, decoded.Offset, decoded.Count); + return Encoding.Default.GetString(decoded.Array, decoded.Offset, decoded.Count); } /// Parse a tuple segment containing a unicode string @@ -856,6 +957,19 @@ public static double ParseDouble(Slice slice) return value; } + /// Parse a tuple segment containing a quadruple precision number (float128) + public static decimal ParseDecimal(Slice slice) + { + Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Decimal); + + if (slice.Count != 17) + { + throw new FormatException("Slice has invalid size for a Decimal"); + } + + throw new NotImplementedException(); + } + /// Parse a tuple segment containing a 128-bit GUID public static Guid ParseGuid(Slice slice) { @@ -867,7 +981,7 @@ public static Guid ParseGuid(Slice slice) } // We store them in RFC 4122 under the hood, so we need to reverse them to the MS format - return Uuid128.Convert(new Slice(slice.Array, slice.Offset + 1, 16)); + return Uuid128.Convert(slice.Substring(1, 16)); } /// Parse a tuple segment containing a 128-bit UUID @@ -880,7 +994,7 @@ public static Uuid128 ParseUuid128(Slice slice) throw new FormatException("Slice has invalid size for a 128-bit UUID"); } - return new Uuid128(new Slice(slice.Array, slice.Offset + 1, 16)); + return new Uuid128(slice.Substring(1, 16)); } /// Parse a tuple segment containing a 64-bit UUID @@ -893,7 +1007,7 @@ public static Uuid64 ParseUuid64(Slice slice) throw new FormatException("Slice has invalid size for a 64-bit UUID"); } - return new Uuid64(new Slice(slice.Array, slice.Offset + 1, 8)); + return Uuid64.Read(slice.Substring(1, 8)); } #endregion @@ -962,6 +1076,16 @@ public static Slice ParseNext(ref TupleReader reader) return reader.Input.ReadBytes(9); } + case TupleTypes.Triple: + { // <22>(10 bytes) + return reader.Input.ReadBytes(11); + } + + case TupleTypes.Decimal: + { // <23>(16 bytes) + return reader.Input.ReadBytes(17); + } + case TupleTypes.Uuid128: { // <30>(16 bytes) return reader.Input.ReadBytes(17); @@ -987,7 +1111,7 @@ public static Slice ParseNext(ref TupleReader reader) return reader.Input.ReadBytes(1 + bytes); } - throw new FormatException(String.Format("Invalid tuple type byte {0} at index {1}/{2}", type, reader.Input.Position, reader.Input.Buffer.Count)); + throw new FormatException($"Invalid tuple type byte {type} at index {reader.Input.Position}/{reader.Input.Buffer.Count}"); } /// Read an embedded tuple, without parsing it @@ -1015,7 +1139,7 @@ internal static Slice ReadEmbeddedTupleBytes(ref TupleReader reader) // else: ignore this token, it will be processed later if the tuple is unpacked and accessed } - throw new FormatException(String.Format("Truncated embedded tuple started at index {0}/{1}", start, reader.Input.Buffer.Count)); + throw new FormatException($"Truncated embedded tuple started at index {start}/{reader.Input.Buffer.Count}"); } /// Skip a number of tokens @@ -1042,7 +1166,7 @@ public static T VisitNext(ref TupleReader reader, Func> 1; // first round down to one less than a power of 2 + v |= v >> 1; // first round down to one less than a power of 2 v |= v >> 2; v |= v >> 4; v |= v >> 8; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs index 0a2e83f78..d53c3f667 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs @@ -26,11 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples + namespace Doxense.Collections.Tuples.Encoding { - using FoundationDB.Client; using System; using System.Diagnostics; + using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; [DebuggerDisplay("{Input.Position}/{Input.Buffer.Count} @ {Depth}")] public struct TupleReader @@ -38,17 +41,32 @@ public struct TupleReader public SliceReader Input; public int Depth; + [ MethodImpl(MethodImplOptions.AggressiveInlining)] public TupleReader(Slice buffer) { this.Input = new SliceReader(buffer); this.Depth = 0; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TupleReader(Slice buffer, int depth) + { + this.Input = new SliceReader(buffer); + this.Depth = depth; + } + public TupleReader(SliceReader input) { this.Input = input; this.Depth = 0; } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static TupleReader Embedded(Slice packed) + { + Contract.Requires(packed.Count >= 2 && packed[0] == TupleTypes.TupleStart && packed[-1] == 0); + return new TupleReader(packed.Substring(1, packed.Count - 2), 1); + } } } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs index 71e133b9a..85cf0664f 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs @@ -26,9 +26,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +using FoundationDB; + +namespace Doxense.Collections.Tuples.Encoding { - using FoundationDB.Client; using System; /// @@ -73,6 +74,10 @@ internal static class TupleTypes internal const byte Single = 32; /// Double precision decimals (64-bit, Big-Endian) [DRAFT] internal const byte Double = 33; + /// Triple precision decimals (80-bit, Big-Endian) [DRAFT] + internal const byte Triple = 34; //note: javascript numbers + /// Quadruple precision decimals (128-bit, Big-Endian) [DRAFT] + internal const byte Decimal = 35; /// RFC4122 UUID (128 bits) [DRAFT] internal const byte Uuid128 = 48; @@ -88,7 +93,7 @@ internal static class TupleTypes internal const byte AliasSystem = 255; /// Return the type of a tuple segment, from its header - public static TupleSegmentType DecodeSegmentType(ref Slice segment) + public static TupleSegmentType DecodeSegmentType(Slice segment) { if (segment.Count == 0) return TupleSegmentType.Nil; @@ -101,6 +106,8 @@ public static TupleSegmentType DecodeSegmentType(ref Slice segment) case TupleStart: return TupleSegmentType.Tuple; case Single: return TupleSegmentType.Single; case Double: return TupleSegmentType.Double; + case Triple: return TupleSegmentType.Triple; + case Decimal: return TupleSegmentType.Decimal; case Uuid128: return TupleSegmentType.Uuid128; case Uuid64: return TupleSegmentType.Uuid64; } @@ -125,6 +132,8 @@ public enum TupleSegmentType Integer = 20, Single = 32, Double = 33, + Triple = 34, + Decimal = 35, Uuid128 = 48, Uuid64 = 49, } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs index 093176d42..d43716688 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs @@ -26,15 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples.Encoding { - using FoundationDB.Client; using System; using System.Diagnostics; + using System.Runtime.CompilerServices; + using Doxense.Memory; [DebuggerDisplay("{Output.Position}/{Output.Buffer.Length} @ {Depth}")] public struct TupleWriter { + //TODO: Could we use "ref SliceWriter" here with C# 7? + public SliceWriter Output; public int Depth; @@ -49,7 +52,13 @@ public TupleWriter(int capacity) this.Output = new SliceWriter(capacity); this.Depth = 0; } - + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice ToSlice() + { + return this.Output.ToSlice(); + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs b/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs index bfa27aae1..cadcc0d53 100644 --- a/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs +++ b/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs index 45cf18398..618358502 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs @@ -26,20 +26,22 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; + using JetBrains.Annotations; + using Doxense.Diagnostics.Contracts; /// Customer formatter that will called the provided lambda functions to convert to and from a tuple - internal sealed class AnonymousTupleFormatter : ITupleFormatter + public sealed class AnonymousTupleFormatter : ITupleFormatter { private readonly Func m_to; private readonly Func m_from; - public AnonymousTupleFormatter(Func to, Func from) + public AnonymousTupleFormatter([NotNull] Func to, [NotNull] Func from) { - if (to == null) throw new ArgumentNullException(nameof(to)); - if (from == null) throw new ArgumentNullException(nameof(@from)); + Contract.NotNull(to, nameof(to)); + Contract.NotNull(from, nameof(from)); m_to = to; m_from = from; @@ -52,7 +54,7 @@ public ITuple ToTuple(T key) public T FromTuple(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); return m_from(tuple); } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs index 19a18bc39..eeaab9b63 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs @@ -26,12 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; + using Doxense.Diagnostics.Contracts; /// Specialized formatter for types that implement ITupleFormattable - internal sealed class FormattableTupleFormatter : ITupleFormatter + public sealed class FormattableTupleFormatter : ITupleFormatter where T : ITupleFormattable, new() { public ITuple ToTuple(T key) @@ -42,7 +43,7 @@ public ITuple ToTuple(T key) public T FromTuple(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); var key = new T(); key.FromTuple(tuple); return key; diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs index 7b432f075..5ee1c9396 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs @@ -26,12 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; /// Simple key formatter that maps a value into a singleton tuple, and back - internal sealed class GenericTupleFormatter : ITupleFormatter + public sealed class GenericTupleFormatter : ITupleFormatter { public ITuple ToTuple(T key) diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs index cb2645056..05eceaaf4 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; @@ -36,11 +36,14 @@ namespace FoundationDB.Layers.Tuples public interface ITupleFormattable { /// Return the tuple representation of this instance - ITuple ToTuple(); + ITuple ToTuple(); //REVIEW: [NotNull] ? /// Load a tuple representation into a newly created instance /// + [Obsolete("Use an ITupleFormater instead!")] void FromTuple(ITuple tuple); + //REVIEW: REMOVE THIS! This does not work well with private ctors and readonly + // => use ITupleFormatter to serialize/deseralize stuff } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs index 9e11904ab..d02de5f75 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; @@ -39,12 +39,12 @@ public interface ITupleFormatter /// Convert a key into a tuple sequence /// Key to convert to a tuple /// Tuple that represent the key (can contain a single item for primitive keys, or several items for composite keys) - ITuple ToTuple(TKey key); + ITuple ToTuple(TKey key); //REVIEW: [NotNull] result? /// Convert a tuple sequence into a key /// Tuple to convert back into a key /// Key that corresponds to the tuple - TKey FromTuple(ITuple tuple); + TKey FromTuple(ITuple tuple); //REVIEW: [NotNull] tuple? } } diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs index 49d3ab565..15a9094e9 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs @@ -26,9 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; + using JetBrains.Annotations; + using Doxense.Diagnostics.Contracts; /// Helper class to get or create tuple formatters public static class TupleFormatter @@ -36,6 +38,7 @@ public static class TupleFormatter private static ITupleFormatter s_default; /// Return the default tuple formatter for this type + [NotNull] public static ITupleFormatter Default { get @@ -54,22 +57,24 @@ public static ITupleFormatter Default /// Lambda that is called to convert a value into a tuple. It SHOULD NOT return null. /// Lambda that is called to convert a tuple back into a value. It CAN return null. /// Custom formatter - public static ITupleFormatter Create(Func from, Func to) + [NotNull] + public static ITupleFormatter Create([NotNull] Func from, [NotNull] Func to) { return new AnonymousTupleFormatter(from, to); } /// Create a formatter that just add or remove a prefix to values - public static ITupleFormatter CreateAppender(ITuple prefix) + [NotNull] + public static ITupleFormatter CreateAppender([NotNull] ITuple prefix) { - if (prefix == null) throw new ArgumentNullException(nameof(prefix)); + Contract.NotNull(prefix, nameof(prefix)); return new AnonymousTupleFormatter( (value) => prefix.Append(value), (tuple) => { - if (tuple.Count != prefix.Count + 1) throw new ArgumentException("Tuple size is invalid", nameof(tuple)); - if (!STuple.StartsWith(tuple, prefix)) throw new ArgumentException("Tuple does not start with the expected prefix", nameof(tuple)); + if (tuple.Count != prefix.Count + 1) throw new ArgumentException("Tuple size is invalid", "tuple"); + if (!TupleHelpers.StartsWith(tuple, prefix)) throw new ArgumentException("Tuple does not start with the expected prefix", "tuple"); return tuple.Last(); } ); @@ -77,6 +82,7 @@ public static ITupleFormatter CreateAppender(ITuple prefix) /// Creates and instance of a tuple formatter that is best suited for this type + [NotNull] private static ITupleFormatter CreateDefaultFormatter() { var type = typeof(T); diff --git a/FoundationDB.Client/Layers/Tuples/ITuple.cs b/FoundationDB.Client/Layers/Tuples/ITuple.cs index cfe12bf17..d36baffbd 100644 --- a/FoundationDB.Client/Layers/Tuples/ITuple.cs +++ b/FoundationDB.Client/Layers/Tuples/ITuple.cs @@ -26,22 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { - using FoundationDB.Client; using JetBrains.Annotations; using System; using System.Collections.Generic; using System.ComponentModel; - /// Represents a Tuple of N elements + /// Represents a Tuple of N elements of any type [ImmutableObject(true)] [CannotApplyEqualityOperator] - public interface ITuple : IEnumerable, IEquatable, IReadOnlyCollection -#if !NET_4_0 - , IReadOnlyList - , System.Collections.IStructuralEquatable -#endif + public interface ITuple : IEquatable, IReadOnlyList, System.Collections.IStructuralEquatable { // Tuples should, by default, behave as closely to Python's tuples as possible. See http://docs.python.org/2/tutorial/datastructures.html#tuples-and-sequences @@ -58,58 +53,40 @@ public interface ITuple : IEnumerable, IEquatable, IReadOnlyColl // - Getting the substring of a tuple should as fast as possible, if possible O(1). For list-based tuples, it should return a view of the list (offset/count) and avoid copying the list // - If an operation returns an empty tuple, then it should return the STuple.Empty singleton instance // - If an operation does not change the tuple (like Append(STuple.Empty), or tuple.Substring(0)), then the tuple should return itself - // - If the same tuple will be packed frequently, it should be memoized (converted into a FdbMemoizedTuple) - -#if NET_4_0 - /// [DANGEROUS] Return an item of the tuple, given its position - /// Position of the item (if negative, means relative from the end) - /// Value of the item - /// The type of the returned value will be either null, string, byte[], Guid, long or ulong. You should use tuple.Get<T>(...) instead if you are working with non standard values! - /// If is outside the bounds of the tuple - /// - /// ("Hello", "World", 123,)[0] => "Hello" - /// ("Hello", "World", 123,)[-1] => 123L - /// - object this[int index] { get; } -#endif + // - If the same tuple will be packed frequently, it should be memoized (converted into a MemoizedTuple) /// Return a section of the tuple /// Starting offset of the sub-tuple to return, or null to select from the start. Negative values means from the end /// Ending offset (excluded) of the sub-tuple to return or null to select until the end. Negative values means from the end. /// Tuple that include all items in the current tuple whose offset are greather than or equal to and strictly less than . The tuple may be smaller than expected if the range is larger than the parent tuple. If the range does not intersect with the tuple, the Empty tuple will be returned. - ITuple this[int? fromIncluded, int? toExcluded] { [NotNull] get; } + ITuple this[int? fromIncluded, int? toExcluded] { [NotNull, Pure] get; } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . + /// Value of the item at position , adapted into type . /// If is outside the bounds of the tuple /// /// ("Hello", "World", 123,).Get<string>(0) => "Hello" /// ("Hello", "World", 123,).Get<int>(-1) => 123 /// ("Hello", "World", 123,).Get<string>(-1) => "123" /// - T Get(int index); - - /// Return the typed value of the last item in the tuple - /// Expected type of the item - /// Value of the last item of this tuple, adapted into type - /// Equivalent of tuple.Get<T>(-1) - T Last(); + [Pure] + TItem Get(int index); /// Create a new Tuple by appending a single new value at the end of this tuple - /// Type of the new value + /// Type of the new value /// Value that will be appended at the end /// New tuple with the new value /// ("Hello,").Append("World") => ("Hello", "World",) - /// If is an , then it will be appended as a single element. If you need to append the *items* of a tuple, you must call - [NotNull] - ITuple Append(T value); + /// If is an , then it will be appended as a single element. If you need to append the *items* of a tuple, you must call + [Pure, NotNull] + ITuple Append(TItem value); /// Create a new Tuple by appending the items of another tuple at the end of this tuple /// Tuple whose items must be appended at the end of the current tuple /// New tuple with the new values, or the same instance if is empty. - [NotNull] + [Pure, NotNull] ITuple Concat([NotNull] ITuple tuple); /// Copy all items of the tuple into an array at a specific location @@ -121,16 +98,6 @@ public interface ITuple : IEnumerable, IEquatable, IReadOnlyColl /// void CopyTo([NotNull] object[] array, int offset); - /// Appends the packed bytes of this instance to the end of a buffer - /// Buffer that will received the packed bytes of this instance - void PackTo(ref TupleWriter writer); - - /// Pack this instance into a Slice - /// - /// ("Hello", "World", 123).ToSlice() => '\x02Hello\x00\x02World\x00\x15\x7B' - /// - Slice ToSlice(); - } } diff --git a/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs b/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs index daed93900..27f64f4b0 100644 --- a/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs @@ -26,19 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { - using FoundationDB.Client; - using FoundationDB.Client.Converters; using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Runtime.Converters; + using Doxense.Diagnostics.Contracts; /// Tuple that represents the concatenation of two tuples - [DebuggerDisplay("{ToString()}")] - public sealed class JoinedTuple : ITuple + [DebuggerDisplay("{ToString(),nq}")] + public sealed class JoinedTuple : ITuple, ITupleSerializable { // Uses cases: joining a 'subspace' tuple (customerId, 'Users', ) with a 'key' tuple (userId, 'Contacts', 123, ) @@ -56,8 +57,8 @@ public sealed class JoinedTuple : ITuple public JoinedTuple(ITuple head, ITuple tail) { - if (head == null) throw new ArgumentNullException("head"); - if (tail == null) throw new ArgumentNullException("tail"); + Contract.NotNull(head, nameof(head)); + Contract.NotNull(tail, nameof(tail)); this.Head = head; this.Tail = tail; @@ -65,34 +66,30 @@ public JoinedTuple(ITuple head, ITuple tail) m_count = m_split + tail.Count; } - public void PackTo(ref TupleWriter writer) - { - this.Head.PackTo(ref writer); - this.Tail.PackTo(ref writer); - } - public Slice ToSlice() + void ITupleSerializable.PackTo(ref TupleWriter writer) { - var writer = new TupleWriter(); PackTo(ref writer); - return writer.Output.ToSlice(); } - public override string ToString() + internal void PackTo(ref TupleWriter writer) { - return STuple.ToString(this); + TupleEncoder.WriteTo(ref writer, this.Head); + TupleEncoder.WriteTo(ref writer, this.Tail); } - public int Count + public override string ToString() { - get { return m_count; } + return STuple.Formatter.ToString(this); } + public int Count => m_count; + public object this[int index] { get { - index = STuple.MapIndex(index, m_count); + index = TupleHelpers.MapIndex(index, m_count); return index < m_split ? this.Head[index] : this.Tail[index - m_split]; } } @@ -101,8 +98,8 @@ public object this[int index] { get { - int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? TupleHelpers.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? TupleHelpers.MapIndexBounded(toExcluded.Value, m_count) : m_count; if (end <= begin) return STuple.Empty; @@ -111,31 +108,21 @@ public object this[int index] { // all selected items are in the tail return this.Tail[begin - p, end - p]; } - else if (end <= p) + if (end <= p) { // all selected items are in the head return this.Head[begin, end]; } - else - { // selected items are both in head and tail - return new JoinedTuple(this.Head[begin, null], this.Tail[null, end - p]); - } + // selected items are both in head and tail + return new JoinedTuple(this.Head[begin, null], this.Tail[null, end - p]); } } public T Get(int index) { - index = STuple.MapIndex(index, m_count); + index = TupleHelpers.MapIndex(index, m_count); return index < m_split ? this.Head.Get(index) : this.Tail.Get(index - m_split); } - public T Last() - { - if (this.Tail.Count > 0) - return this.Tail.Last(); - else - return this.Head.Last(); - } - ITuple ITuple.Append(T value) { return new LinkedTuple(this, value); @@ -147,10 +134,9 @@ public LinkedTuple Append(T value) return new LinkedTuple(this, value); } - [NotNull] - public ITuple Concat([NotNull] ITuple tuple) + public ITuple Concat(ITuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + Contract.NotNull(tuple, nameof(tuple)); int n1 = tuple.Count; if (n1 == 0) return this; @@ -161,11 +147,8 @@ public ITuple Concat([NotNull] ITuple tuple) { // it's getting bug, merge to a new List tuple return new ListTuple(this.Head, this.Tail, tuple); } - else - { - // REVIEW: should we always concat with the tail? - return new JoinedTuple(this.Head, this.Tail.Concat(tuple)); - } + // REVIEW: should we always concat with the tail? + return new JoinedTuple(this.Head, this.Tail.Concat(tuple)); } public void CopyTo(object[] array, int offset) @@ -235,9 +218,9 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return STuple.CombineHashCodes( - this.Head != null ? this.Head.GetHashCode(comparer) : 0, - this.Tail != null ? this.Tail.GetHashCode(comparer) : 0 + return HashCodes.Combine( + HashCodes.Compute(this.Head, comparer), + HashCodes.Compute(this.Tail, comparer) ); } } diff --git a/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs b/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs index a52bfd45d..b8aca3537 100644 --- a/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs @@ -26,20 +26,19 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { - using System.Collections; + using JetBrains.Annotations; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; - using JetBrains.Annotations; + using Doxense.Runtime.Converters; /// Tuple that adds a value at the end of an already existing tuple /// Type of the last value of the tuple - [DebuggerDisplay("{ToString()}")] - public sealed class LinkedTuple : ITuple + [DebuggerDisplay("{ToString(),nq}")] + public sealed class LinkedTuple : ITuple, ITupleSerializable { //TODO: consider changing this to a struct ? @@ -56,9 +55,9 @@ public sealed class LinkedTuple : ITuple public readonly int Depth; /// Append a new value at the end of an existing tuple - internal LinkedTuple(ITuple head, T tail) + public LinkedTuple([NotNull] ITuple head, T tail) { - Contract.Requires(head != null); + Contract.NotNull(head, nameof(head)); this.Head = head; this.Tail = tail; @@ -66,25 +65,20 @@ internal LinkedTuple(ITuple head, T tail) } /// Pack this tuple into a buffer - public void PackTo(ref TupleWriter writer) + void ITupleSerializable.PackTo(ref TupleWriter writer) { - this.Head.PackTo(ref writer); - TuplePacker.SerializeTo(ref writer, this.Tail); + PackTo(ref writer); } - /// Pack this tuple into a slice - public Slice ToSlice() + /// Pack this tuple into a buffer + internal void PackTo(ref TupleWriter writer) { - var writer = new TupleWriter(); - PackTo(ref writer); - return writer.Output.ToSlice(); + TupleEncoder.WriteTo(ref writer, this.Head); + TuplePacker.SerializeTo(ref writer, this.Tail); } /// Returns the number of elements in this tuple - public int Count - { - get { return this.Depth + 1; } - } + public int Count => this.Depth + 1; public object this[int index] { @@ -98,39 +92,39 @@ public object this[int index] public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } - public R Get(int index) + public TItem Get(int index) { - if (index == this.Depth || index == -1) return FdbConverters.Convert(this.Tail); + if (index == this.Depth || index == -1) return TypeConverters.Convert(this.Tail); if (index < -1) index++; - return this.Head.Get(index); + return this.Head.Get(index); } - public R Last() + public T Last { - return FdbConverters.Convert(this.Tail); + [Pure] + get { return this.Tail; } } - ITuple ITuple.Append(R value) + ITuple ITuple.Append(TItem value) { - return this.Append(value); + return this.Append(value); } [NotNull] - public LinkedTuple Append(R value) + public LinkedTuple Append(TItem value) { - return new LinkedTuple(this, value); + return new LinkedTuple(this, value); } - [NotNull] - public ITuple Concat([NotNull] ITuple tuple) + public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); } - public void CopyTo([NotNull] object[] array, int offset) + public void CopyTo(object[] array, int offset) { this.Head.CopyTo(array, offset); array[offset + this.Depth] = this.Tail; @@ -152,22 +146,22 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public override string ToString() { - return STuple.ToString(this); + return STuple.Formatter.ToString(this); } public override bool Equals(object obj) { - return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); + return obj != null && ((System.Collections.IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); } public bool Equals(ITuple other) { - return !object.ReferenceEquals(other, null) && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); + return !object.ReferenceEquals(other, null) && ((System.Collections.IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } public override int GetHashCode() { - return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); + return ((System.Collections.IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); } bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) @@ -186,13 +180,13 @@ bool System.Collections.IStructuralEquatable.Equals(object other, System.Collect return this.Head.Equals(linked.Tail, comparer); } - return STuple.Equals(this, other, comparer); + return TupleHelpers.Equals(this, other, comparer); } - int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) + int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return STuple.CombineHashCodes( - this.Head != null ? this.Head.GetHashCode(comparer) : 0, + return HashCodes.Combine( + HashCodes.Compute(this.Head, comparer), comparer.GetHashCode(this.Tail) ); } diff --git a/FoundationDB.Client/Layers/Tuples/ListTuple.cs b/FoundationDB.Client/Layers/Tuples/ListTuple.cs index 4fa95de86..781b6ff3f 100644 --- a/FoundationDB.Client/Layers/Tuples/ListTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/ListTuple.cs @@ -26,20 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.Linq; + using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using Doxense.Runtime.Converters; + using JetBrains.Annotations; /// Tuple that can hold any number of untyped items - public sealed class ListTuple : ITuple + public sealed class ListTuple : ITuple, ITupleSerializable { - // We could use a FdbListTuple for tuples where all items are of type T, and FdbListTuple could derive from FdbListTuple. + // We could use a ListTuple for tuples where all items are of type T, and ListTuple could derive from ListTuple. // => this could speed up a bit the use case of STuple.FromArray or STuple.FromSequence /// List of the items in the tuple. @@ -53,7 +54,7 @@ public sealed class ListTuple : ITuple private int? m_hashCode; /// Create a new tuple from a sequence of items (copied) - internal ListTuple(IEnumerable items) + public ListTuple([NotNull, InstantHandle] IEnumerable items) { m_items = items.ToArray(); m_count = m_items.Length; @@ -61,7 +62,7 @@ internal ListTuple(IEnumerable items) /// Wrap a List of items /// The list should not mutate and should not be exposed to anyone else! - internal ListTuple(object[] items, int offset, int count) + public ListTuple(object[] items, int offset, int count) { Contract.Requires(items != null && offset >= 0 && count >= 0); Contract.Requires(offset + count <= items.Length, "inner item array is too small"); @@ -72,12 +73,10 @@ internal ListTuple(object[] items, int offset, int count) } /// Create a new list tuple by merging the items of two tuples together - /// - /// - internal ListTuple(ITuple a, ITuple b) + public ListTuple(ITuple a, ITuple b) { - if (a == null) throw new ArgumentNullException("a"); - if (b == null) throw new ArgumentNullException("b"); + Contract.NotNull(a, nameof(a)); + Contract.NotNull(b, nameof(b)); int nA = a.Count; int nB = b.Count; @@ -91,11 +90,11 @@ internal ListTuple(ITuple a, ITuple b) } /// Create a new list tuple by merging the items of three tuples together - internal ListTuple(ITuple a, ITuple b, ITuple c) + public ListTuple(ITuple a, ITuple b, ITuple c) { - if (a == null) throw new ArgumentNullException("a"); - if (b == null) throw new ArgumentNullException("b"); - if (c == null) throw new ArgumentNullException("c"); + Contract.NotNull(a, nameof(a)); + Contract.NotNull(b, nameof(b)); + Contract.NotNull(c, nameof(c)); int nA = a.Count; int nB = b.Count; @@ -110,25 +109,16 @@ internal ListTuple(ITuple a, ITuple b, ITuple c) if (nC > 0) c.CopyTo(m_items, nA + nB); } - public int Count - { - get { return m_count; } - } + public int Count => m_count; - public object this[int index] - { - get - { - return m_items[m_offset + STuple.MapIndex(index, m_count)]; - } - } + public object this[int index] => m_items[m_offset + TupleHelpers.MapIndex(index, m_count)]; public ITuple this[int? fromIncluded, int? toExcluded] { get { - int begin = fromIncluded.HasValue ? STuple.MapIndexBounded(fromIncluded.Value, m_count) : 0; - int end = toExcluded.HasValue ? STuple.MapIndexBounded(toExcluded.Value, m_count) : m_count; + int begin = fromIncluded.HasValue ? TupleHelpers.MapIndexBounded(fromIncluded.Value, m_count) : 0; + int end = toExcluded.HasValue ? TupleHelpers.MapIndexBounded(toExcluded.Value, m_count) : m_count; int len = end - begin; if (len <= 0) return STuple.Empty; @@ -141,23 +131,23 @@ public object this[int index] } } - public R Get(int index) + public TItem Get(int index) { - return FdbConverters.ConvertBoxed(this[index]); + return TypeConverters.ConvertBoxed(this[index]); } - public R Last() + public TItem Last() { if (m_count == 0) throw new InvalidOperationException("Tuple is empty"); - return FdbConverters.ConvertBoxed(m_items[m_offset + m_count - 1]); + return TypeConverters.ConvertBoxed(m_items[m_offset + m_count - 1]); } - ITuple ITuple.Append(T value) + ITuple ITuple.Append(TItem value) { - return this.Append(value); + return Append(value); } - public ListTuple Append(T value) + public ListTuple Append(TItem value) { var list = new object[m_count + 1]; Array.Copy(m_items, m_offset, list, 0, m_count); @@ -167,7 +157,7 @@ public ListTuple Append(T value) public ListTuple AppendRange(object[] items) { - if (items == null) throw new ArgumentNullException("items"); + Contract.NotNull(items, nameof(items)); if (items.Length == 0) return this; @@ -179,7 +169,7 @@ public ListTuple AppendRange(object[] items) public ListTuple Concat(ListTuple tuple) { - if (tuple == null) throw new ArgumentNullException("tuple"); + Contract.NotNull(tuple, nameof(tuple)); if (tuple.m_count == 0) return this; if (m_count == 0) return tuple; @@ -192,8 +182,7 @@ public ListTuple Concat(ListTuple tuple) public ListTuple Concat(ITuple tuple) { - var _ = tuple as ListTuple; - if (_ != null) return Concat(_); + if (tuple is ListTuple lt) return Concat(lt); int count = tuple.Count; if (count == 0) return this; @@ -206,7 +195,7 @@ public ListTuple Concat(ITuple tuple) ITuple ITuple.Concat(ITuple tuple) { - return this.Concat(tuple); + return Concat(tuple); } public void CopyTo(object[] array, int offset) @@ -236,24 +225,27 @@ private static IEnumerator Enumerate(object[] items, int offset, int cou } } - public void PackTo(ref TupleWriter writer) + + void ITupleSerializable.PackTo(ref TupleWriter writer) { - for (int i = 0; i < m_count; i++) - { - TuplePackers.SerializeObjectTo(ref writer, m_items[i + m_offset]); - } + PackTo(ref writer); } - public Slice ToSlice() + internal void PackTo(ref TupleWriter writer) { - var writer = new TupleWriter(); - PackTo(ref writer); - return writer.Output.ToSlice(); + //REVIEW: this is VERY slow! + int count = m_count; + var items = m_items; + int offset = m_offset; + for (int i = 0; i < count; i++) + { + TuplePackers.SerializeObjectTo(ref writer, items[i + offset]); + } } public override string ToString() { - return STuple.ToString(m_items, m_offset, m_count); + return STuple.Formatter.ToString(m_items, m_offset, m_count); } private bool CompareItems(IEnumerable theirs, IEqualityComparer comparer) @@ -311,7 +303,7 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) } } - return STuple.Equals(this, other, comparer); + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) @@ -327,8 +319,8 @@ int IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer compar for (int i = 0; i < m_count; i++) { var item = m_items[i + m_offset]; - - h = STuple.CombineHashCodes(h, comparer.GetHashCode(item)); + + h = HashCodes.Combine(h, comparer.GetHashCode(item)); } if (canUseCache) m_hashCode = h; return h; diff --git a/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs index 32740bfa8..9840289a4 100644 --- a/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs @@ -26,15 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using Doxense.Runtime.Converters; /// Represents an immutable tuple where the packed bytes are cached [DebuggerDisplay("{ToString()}")] @@ -67,12 +67,12 @@ public int Count public object this[int index] { - get { return m_items[STuple.MapIndex(index, m_items.Length)]; } + get { return m_items[TupleHelpers.MapIndex(index, m_items.Length)]; } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } public void PackTo(ref TupleWriter writer) @@ -105,14 +105,14 @@ public object[] ToArray() public R Get(int index) { - return FdbConverters.ConvertBoxed(this[index]); + return TypeConverters.ConvertBoxed(this[index]); } public R Last() { int n = m_items.Length; if (n == 0) throw new InvalidOperationException("Tuple is emtpy"); - return FdbConverters.ConvertBoxed(m_items[n - 1]); + return TypeConverters.ConvertBoxed(m_items[n - 1]); } ITuple ITuple.Append(T value) @@ -147,7 +147,7 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() public override string ToString() { - return STuple.ToString(m_items, 0, m_items.Length); + return STuple.Formatter.ToString(m_items, 0, m_items.Length); } public override bool Equals(object obj) @@ -165,7 +165,7 @@ public bool Equals(ITuple other) return m_packed.Equals(memoized.m_packed); } - return STuple.Equals(this, other, SimilarValueComparer.Default); + return TupleHelpers.Equals(this, other, SimilarValueComparer.Default); } public override int GetHashCode() @@ -175,12 +175,12 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) { - return STuple.Equals(this, other, comparer); + return TupleHelpers.Equals(this, other, comparer); } int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) { - return STuple.StructuralGetHashCode(this, comparer); + return TupleHelpers.StructuralGetHashCode(this, comparer); } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index a57d499a6..47aae9396 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -26,185 +26,196 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; - using System.Collections; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; using System.Linq; + using System.Runtime.CompilerServices; using System.Text; + using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; + using FoundationDB; using JetBrains.Annotations; /// Factory class for Tuples [PublicAPI] - public static class STuple + public struct STuple : ITuple, ITupleSerializable { + //note: We cannot use 'Tuple' because it's already used by the BCL in the System namespace, and we cannot use 'Tuples' either because it is part of the namespace... + /// Empty tuple /// Not to be mistaken with a 1-tuple containing 'null' ! - public static readonly ITuple Empty = new EmptyTuple(); - - /// Empty tuple (singleton that is used as a base for other tuples) - private sealed class EmptyTuple : ITuple - { + [NotNull] + public static ITuple Empty => new STuple(); - public int Count => 0; + #region Empty Tuple - object IReadOnlyList.this[int index] => throw new InvalidOperationException("Tuple is empty"); + public int Count => 0; - public ITuple this[int? from, int? to] => this; - //REVIEW: should we throw if from/to are not null, 0 or -1 ? - public R Get(int index) - { - throw new InvalidOperationException("Tuple is empty"); - } + object IReadOnlyList.this[int index] => throw new InvalidOperationException("Tuple is empty"); - R ITuple.Last() - { - throw new InvalidOperationException("Tuple is empty"); - } + //REVIEW: should we throw if from/to are not null, 0 or -1 ? + public ITuple this[int? from, int? to] => this; - public ITuple Append(T1 value) - { - return new STuple(value); - } + public TItem Get(int index) + { + throw new InvalidOperationException("Tuple is empty"); + } - public ITuple Concat(ITuple tuple) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - if (tuple is EmptyTuple || tuple.Count == 0) return this; - return tuple; - } + public ITuple Append(T1 value) => new STuple(value); - public void PackTo(ref TupleWriter writer) - { - //NO-OP - } + public ITuple Concat(ITuple tuple) + { + Contract.NotNull(tuple, nameof(tuple)); + if (tuple.Count == 0) return this; + return tuple; + } - public Slice ToSlice() - { - return Slice.Empty; - } + void ITupleSerializable.PackTo(ref TupleWriter writer) + { + PackTo(ref writer); + } - public void CopyTo(object[] array, int offset) - { - //NO-OP - } + internal void PackTo(ref TupleWriter writer) + { + //NO-OP + } - public IEnumerator GetEnumerator() - { - yield break; - } + public void CopyTo(object[] array, int offset) + { + //NO-OP + } - System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() - { - return this.GetEnumerator(); - } + public IEnumerator GetEnumerator() + { + yield break; + } - public override string ToString() - { - return "()"; - } + System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() + { + return this.GetEnumerator(); + } - public override int GetHashCode() - { - return 0; - } + public override string ToString() + { + return "()"; + } - public bool Equals(ITuple value) - { - return value != null && value.Count == 0; - } + public override int GetHashCode() + { + return 0; + } - public override bool Equals(object obj) - { - return Equals(obj as ITuple); - } + public bool Equals(ITuple value) + { + return value != null && value.Count == 0; + } - bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) - { - var tuple = other as ITuple; - return tuple != null && tuple.Count == 0; - } + public override bool Equals(object obj) + { + return Equals(obj as ITuple); + } - int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) - { - return 0; - } + bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) + { + return other is ITuple tuple && tuple.Count == 0; + } + int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) + { + return 0; } + #endregion + #region Creation - /// Create a new 1-tuple, holding only one item - /// This is the non-generic equivalent of STuple.Create<object>() - [NotNull] - public static ITuple CreateBoxed(object item) + /// Create a new empty tuple with 0 elements + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] + public static STuple Create() { - return new STuple(item); + //note: redundant with STuple.Empty, but is here to fit nicely with the other Create overloads + return new STuple(); } /// Create a new 1-tuple, holding only one item - [DebuggerStepThrough] + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] public static STuple Create(T1 item1) { return new STuple(item1); } /// Create a new 2-tuple, holding two items - [DebuggerStepThrough] + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] public static STuple Create(T1 item1, T2 item2) { return new STuple(item1, item2); } /// Create a new 3-tuple, holding three items - [DebuggerStepThrough] + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] public static STuple Create(T1 item1, T2 item2, T3 item3) { return new STuple(item1, item2, item3); } /// Create a new 4-tuple, holding four items - [DebuggerStepThrough] + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] public static STuple Create(T1 item1, T2 item2, T3 item3, T4 item4) { return new STuple(item1, item2, item3, item4); } /// Create a new 5-tuple, holding five items - [DebuggerStepThrough] + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] public static STuple Create(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { return new STuple(item1, item2, item3, item4, item5); } + /// Create a new 6-tuple, holding six items + [MethodImpl(MethodImplOptions.AggressiveInlining), DebuggerStepThrough] + public static STuple Create(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + return new STuple(item1, item2, item3, item4, item5, item6); + } + /// Create a new N-tuple, from N items /// Items to wrap in a tuple /// If you already have an array of items, you should call instead. Mutating the array, would also mutate the tuple! [NotNull] public static ITuple Create([NotNull] params object[] items) { - if (items == null) throw new ArgumentNullException(nameof(items)); + Contract.NotNull(items, nameof(items)); //note: this is a convenience method for people that wants to pass more than 3 args arguments, and not have to call CreateRange(object[]) method - if (items.Length == 0) return STuple.Empty; + if (items.Length == 0) return new STuple(); // We don't copy the array, and rely on the fact that the array was created by the compiler and that nobody will get a reference on it. return new ListTuple(items, 0, items.Length); } + /// Create a new 1-tuple, holding only one item + /// This is the non-generic equivalent of STuple.Create<object>() + [NotNull] + public static ITuple CreateBoxed(object item) + { + return new STuple(item); + } + /// Create a new N-tuple that wraps an array of untyped items /// If the original array is mutated, the tuple will reflect the changes! [NotNull] public static ITuple Wrap([NotNull] object[] items) { //note: this method only exists to differentiate between Create(object[]) and Create() - if (items == null) throw new ArgumentException("items"); + Contract.NotNull(items, nameof(items)); return FromObjects(items, 0, items.Length, copy: false); } @@ -221,7 +232,7 @@ public static ITuple Wrap([NotNull] object[] items, int offset, int count) public static ITuple FromObjects([NotNull] object[] items) { //note: this method only exists to differentiate between Create(object[]) and Create() - if (items == null) throw new ArgumentException("items"); + Contract.NotNull(items, nameof(items)); return FromObjects(items, 0, items.Length, copy: true); } @@ -237,10 +248,10 @@ public static ITuple FromObjects([NotNull] object[] items, int offset, int count [NotNull] public static ITuple FromObjects([NotNull] object[] items, int offset, int count, bool copy) { - if (items == null) throw new ArgumentNullException(nameof(items)); - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); - if (offset + count > items.Length) throw new ArgumentOutOfRangeException(nameof(count), "Source array is too small"); + Contract.NotNull(items, nameof(items)); + Contract.Positive(offset, nameof(offset)); + Contract.Positive(count, nameof(count)); + Contract.LessOrEqual(offset + count, items.Length, nameof(count), "Source array is too small"); if (count == 0) return STuple.Empty; @@ -263,7 +274,7 @@ public static ITuple FromObjects([NotNull] object[] items, int offset, int count [NotNull] public static ITuple FromArray([NotNull] T[] items) { - if (items == null) throw new ArgumentNullException(nameof(items)); + Contract.NotNull(items, nameof(items)); return FromArray(items, 0, items.Length); } @@ -272,21 +283,23 @@ public static ITuple FromArray([NotNull] T[] items) [NotNull] public static ITuple FromArray([NotNull] T[] items, int offset, int count) { - if (items == null) throw new ArgumentNullException(nameof(items)); - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); - if (offset + count > items.Length) throw new ArgumentOutOfRangeException(nameof(count), "Source array is too small"); + Contract.NotNull(items, nameof(items)); + Contract.Positive(offset, nameof(offset)); + Contract.Positive(count, nameof(count)); + Contract.LessOrEqual(offset + count, items.Length, nameof(count), "Source array is too small"); - switch(count) + switch (count) { - case 0: return STuple.Empty; - case 1: return STuple.Create(items[offset]); - case 2: return STuple.Create(items[offset], items[offset + 1]); - case 3: return STuple.Create(items[offset], items[offset + 1], items[offset + 2]); - case 4: return STuple.Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3]); + case 0: return Create(); + case 1: return Create(items[offset]); + case 2: return Create(items[offset], items[offset + 1]); + case 3: return Create(items[offset], items[offset + 1], items[offset + 2]); + case 4: return Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3]); + case 5: return Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3], items[offset + 4]); + case 6: return Create(items[offset], items[offset + 1], items[offset + 2], items[offset + 3], items[offset + 4], items[offset + 5]); default: { // copy the items in a temp array - //TODO: we would probably benefit from having an FdbListTuple here! + //TODO: we would probably benefit from having an ListTuple here! var tmp = new object[count]; Array.Copy(items, offset, tmp, 0, count); return new ListTuple(tmp, 0, count); @@ -298,23 +311,21 @@ public static ITuple FromArray([NotNull] T[] items, int offset, int count) [NotNull] public static ITuple FromEnumerable([NotNull] IEnumerable items) { - if (items == null) throw new ArgumentNullException(nameof(items)); + Contract.NotNull(items, nameof(items)); - var arr = items as T[]; - if (arr != null) + if (items is T[] arr) { return FromArray(arr, 0, arr.Length); } // may already be a tuple (because it implements IE) - var tuple = items as ITuple; - if (tuple != null) + if (items is ITuple tuple) { return tuple; } object[] tmp = items.Cast().ToArray(); - //TODO: we would probably benefit from having an FdbListTuple here! + //TODO: we would probably benefit from having an ListTuple here! return new ListTuple(tmp, 0, tmp.Length); } @@ -322,718 +333,83 @@ public static ITuple FromEnumerable([NotNull] IEnumerable items) [NotNull] public static ITuple Concat([NotNull] ITuple head, [NotNull] ITuple tail) { - if (head == null) throw new ArgumentNullException(nameof(head)); - if (tail == null) throw new ArgumentNullException(nameof(tail)); - - int n1 = head.Count; - if (n1 == 0) return tail; - - int n2 = tail.Count; - if (n2 == 0) return head; - - return new JoinedTuple(head, tail); - } - - #endregion - - #region Packing... - - // Without prefix - - /// Pack a tuple into a slice - /// Tuple that must be serialized into a binary slice - public static Slice Pack([NotNull] ITuple tuple) - { - //note: this is redundant with tuple.ToSlice() - // => maybe we should remove this method? - - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - return tuple.ToSlice(); - } - - /// Pack an array of N-tuples, all sharing the same buffer - /// Sequence of N-tuples to pack - /// Array containing the buffer segment of each packed tuple - /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] - [NotNull] - public static Slice[] Pack([NotNull] params ITuple[] tuples) - { - return Pack(Slice.Nil, tuples); - } - - /// Pack a sequence of N-tuples, all sharing the same buffer - /// Sequence of N-tuples to pack - /// Array containing the buffer segment of each packed tuple - /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] - [NotNull] - public static Slice[] Pack([NotNull] IEnumerable tuples) - { - return Pack(Slice.Nil, tuples); - } - - // With prefix - - public static void Pack(ref TupleWriter writer, [CanBeNull] ITuple tuple) - { - if (tuple == null || tuple.Count == 0) return; - tuple.PackTo(ref writer); - } - - /// Efficiently concatenate a prefix with the packed representation of a tuple - public static Slice Pack(Slice prefix, [CanBeNull] ITuple tuple) - { - if (tuple == null || tuple.Count == 0) return prefix; - - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); - return writer.Output.ToSlice(); - } - - /// Pack an array of N-tuples, all sharing the same buffer - /// Commong prefix added to all the tuples - /// Sequence of N-tuples to pack - /// Array containing the buffer segment of each packed tuple - /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] - [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] params ITuple[] tuples) - { - if (tuples == null) throw new ArgumentNullException(nameof(tuples)); - - // pre-allocate by supposing that each tuple will take at least 16 bytes - var writer = new TupleWriter(tuples.Length * (16 + prefix.Count)); - var next = new List(tuples.Length); - - //TODO: use multiple buffers if item count is huge ? - - foreach (var tuple in tuples) - { - writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); - next.Add(writer.Output.Position); - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - /// Pack a sequence of N-tuples, all sharing the same buffer - /// Commong prefix added to all the tuples - /// Sequence of N-tuples to pack - /// Array containing the buffer segment of each packed tuple - /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] - [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable tuples) - { - if (tuples == null) throw new ArgumentNullException(nameof(tuples)); - - // use optimized version for arrays - var array = tuples as ITuple[]; - if (array != null) return Pack(prefix, array); - - var next = new List(); - var writer = new TupleWriter(); - - //TODO: use multiple buffers if item count is huge ? - - foreach (var tuple in tuples) - { - writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); - next.Add(writer.Output.Position); - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] TElement[] elements, Func transform) - { - if (elements == null) throw new ArgumentNullException(nameof(elements)); - if (transform == null) throw new ArgumentNullException(nameof(transform)); - - var next = new List(elements.Length); - var writer = new TupleWriter(); - - //TODO: use multiple buffers if item count is huge ? - - foreach (var element in elements) - { - var tuple = transform(element); - if (tuple == null) - { - next.Add(writer.Output.Position); - } - else - { - writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); - next.Add(writer.Output.Position); - } - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - [NotNull] - public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable elements, Func transform) - { - if (elements == null) throw new ArgumentNullException(nameof(elements)); - if (transform == null) throw new ArgumentNullException(nameof(transform)); - - // use optimized version for arrays - var array = elements as TElement[]; - if (array != null) return Pack(prefix, array, transform); - - var next = new List(); - var writer = new TupleWriter(); - - //TODO: use multiple buffers if item count is huge ? - - foreach (var element in elements) - { - var tuple = transform(element); - if (tuple == null) - { - next.Add(writer.Output.Position); - } - else - { - writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); - next.Add(writer.Output.Position); - } - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - #endregion - - #region Encode - - //REVIEW: EncodeKey/EncodeKeys? Encode/EncodeRange? EncodeValues? EncodeItems? - - /// Pack a 1-tuple directly into a slice - public static Slice EncodeKey(T1 item1) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - return writer.Output.ToSlice(); - } - - /// Pack a 2-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - return writer.Output.ToSlice(); - } - - /// Pack a 3-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - return writer.Output.ToSlice(); - } - - /// Pack a 4-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - TuplePacker.SerializeTo(ref writer, item4); - return writer.Output.ToSlice(); - } - - /// Pack a 5-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - TuplePacker.SerializeTo(ref writer, item4); - TuplePacker.SerializeTo(ref writer, item5); - return writer.Output.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - TuplePacker.SerializeTo(ref writer, item4); - TuplePacker.SerializeTo(ref writer, item5); - TuplePacker.SerializeTo(ref writer, item6); - return writer.Output.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - TuplePacker.SerializeTo(ref writer, item4); - TuplePacker.SerializeTo(ref writer, item5); - TuplePacker.SerializeTo(ref writer, item6); - TuplePacker.SerializeTo(ref writer, item7); - return writer.Output.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) - { - var writer = new TupleWriter(); - TuplePacker.SerializeTo(ref writer, item1); - TuplePacker.SerializeTo(ref writer, item2); - TuplePacker.SerializeTo(ref writer, item3); - TuplePacker.SerializeTo(ref writer, item4); - TuplePacker.SerializeTo(ref writer, item5); - TuplePacker.SerializeTo(ref writer, item6); - TuplePacker.SerializeTo(ref writer, item7); - TuplePacker.SerializeTo(ref writer, item8); - return writer.Output.ToSlice(); - } - - [NotNull] - public static Slice[] EncodeKeys([NotNull] IEnumerable keys) - { - return EncodePrefixedKeys(Slice.Nil, keys); - } - - /// Merge a sequence of keys with a same prefix, all sharing the same buffer - /// Type of the keys - /// Prefix shared by all keys - /// Sequence of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) - { - if (prefix == null) throw new ArgumentNullException(nameof(prefix)); - if (keys == null) throw new ArgumentNullException(nameof(keys)); - - // use optimized version for arrays - var array = keys as T[]; - if (array != null) return EncodePrefixedKeys(prefix, array); - - var next = new List(); - var writer = new TupleWriter(); - var packer = TuplePacker.Encoder; - - //TODO: use multiple buffers if item count is huge ? - - foreach (var key in keys) - { - if (prefix.IsPresent) writer.Output.WriteBytes(prefix); - packer(ref writer, key); - next.Add(writer.Output.Position); - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - [NotNull] - public static Slice[] EncodeKeys([NotNull] params T[] keys) - { - return EncodePrefixedKeys(Slice.Nil, keys); - } - - /// Merge an array of keys with a same prefix, all sharing the same buffer - /// Type of the keys - /// Prefix shared by all keys - /// Sequence of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) - { - if (keys == null) throw new ArgumentNullException(nameof(keys)); - - // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize - var writer = new TupleWriter(keys.Length * (prefix.Count + 8)); - var next = new List(keys.Length); - var packer = TuplePacker.Encoder; - - //TODO: use multiple buffers if item count is huge ? - - foreach (var key in keys) - { - if (prefix.Count > 0) writer.Output.WriteBytes(prefix); - packer(ref writer, key); - next.Add(writer.Output.Position); - } - - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); - } - - /// Merge an array of elements, all sharing the same buffer - /// Type of the elements - /// Type of the keys extracted from the elements - /// Sequence of elements to pack - /// Lambda that extract the key from each element - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] Func selector) - { - return EncodePrefixedKeys(Slice.Empty, elements, selector); - } - - /// Merge an array of elements with a same prefix, all sharing the same buffer - /// Type of the elements - /// Type of the keys extracted from the elements - /// Prefix shared by all keys (can be empty) - /// Sequence of elements to pack - /// Lambda that extract the key from each element - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) - { - if (elements == null) throw new ArgumentNullException(nameof(elements)); - if (selector == null) throw new ArgumentNullException(nameof(selector)); - - // pre-allocate by guessing that each key will take at least 8 bytes. Even if 8 is too small, we should have at most one or two buffer resize - var writer = new TupleWriter(elements.Length * (prefix.Count + 8)); - var next = new List(elements.Length); - var packer = TuplePacker.Encoder; - - //TODO: use multiple buffers if item count is huge ? - - foreach (var value in elements) - { - if (prefix.Count > 0) writer.Output.WriteBytes(prefix); - packer(ref writer, selector(value)); - next.Add(writer.Output.Position); - } + Contract.NotNull(head, nameof(head)); + Contract.NotNull(tail, nameof(tail)); - return FdbKey.SplitIntoSegments(writer.Output.Buffer, 0, next); + return head.Count == 0 ? tail + : tail.Count == 0 ? head + : new JoinedTuple(head, tail); } - /// Pack a sequence of keys with a same prefix, all sharing the same buffer - /// Type of the keys - /// Prefix shared by all keys - /// Sequence of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] IEnumerable keys) - { - if (prefix == null) throw new ArgumentNullException(nameof(prefix)); +#if ENABLE_VALUETUPLES - return EncodePrefixedKeys(prefix.ToSlice(), keys); - } - - /// Pack a sequence of keys with a same prefix, all sharing the same buffer - /// Type of the keys - /// Prefix shared by all keys - /// Sequence of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] params T[] keys) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (prefix == null) throw new ArgumentNullException(nameof(prefix)); - - return EncodePrefixedKeys(prefix.ToSlice(), keys); + return new STuple(tuple.Item1); } - #endregion - - #region Unpacking... - - /// Unpack a tuple from a serialied key blob - /// Binary key containing a previously packed tuple - /// Unpacked tuple, or the empty tuple if the key is - /// If is equal to - [NotNull] - public static ITuple Unpack(Slice packedKey) + [Pure] + public static STuple Create(ref ValueTuple tuple) { - if (packedKey.IsNull) throw new ArgumentNullException(nameof(packedKey)); - if (packedKey.Count == 0) return STuple.Empty; - - return TuplePackers.Unpack(packedKey, false); + return new STuple(tuple.Item1); } - /// Unpack a tuple from a binary representation - /// Binary key containing a previously packed tuple, or Slice.Nil - /// Unpacked tuple, the empty tuple if is equal to , or null if the key is - [CanBeNull] - public static ITuple UnpackOrDefault(Slice packedKey) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (packedKey.IsNull) return null; - if (packedKey.Count == 0) return STuple.Empty; - return TuplePackers.Unpack(packedKey, false); + return new STuple(tuple.Item1, tuple.Item2); } - /// Unpack a tuple and only return its first element - /// Type of the first value in the decoded tuple - /// Slice that should be entirely parsable as a tuple - /// Decoded value of the first item in the tuple - public static T DecodeFirst(Slice packedKey) + [Pure] + public static STuple Create(ref ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the first element of an empty tuple"); - - var slice = TuplePackers.UnpackFirst(packedKey); - if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); - - return TuplePacker.Deserialize(slice); + return new STuple(tuple.Item1, tuple.Item2); } - /// Unpack a tuple and only return its last element - /// Type of the last value in the decoded tuple - /// Slice that should be entirely parsable as a tuple - /// Decoded value of the last item in the tuple - public static T DecodeLast(Slice packedKey) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the last element of an empty tuple"); - - var slice = TuplePackers.UnpackLast(packedKey); - if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); - - return TuplePacker.Deserialize(slice); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); } - /// Unpack the value of a singleton tuple - /// Type of the single value in the decoded tuple - /// Slice that should contain the packed representation of a tuple with a single element - /// Decoded value of the only item in the tuple. Throws an exception if the tuple is empty of has more than one element. - public static T DecodeKey(Slice packedKey) + [Pure] + public static STuple Create(ref ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack a single value out of an empty tuple"); - - var slice = TuplePackers.UnpackSingle(packedKey); - if (slice.IsNull) throw new InvalidOperationException("Failed to unpack singleton tuple"); - - return TuplePacker.Deserialize(slice); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); } - /// Unpack a key containing two elements - /// Slice that should contain the packed representation of a tuple with two elements - /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static STuple DecodeKey(Slice packedKey) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); - - var reader = new TupleReader(packedKey); - - T1 item1; - if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); - - T2 item2; - if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); - - if (reader.Input.HasMore) throw new FormatException("The key contains more than two items"); - - return Create(item1, item2); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } - /// Unpack a key containing three elements - /// Slice that should contain the packed representation of a tuple with three elements - /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static STuple DecodeKey(Slice packedKey) + [Pure] + public static STuple Create(ref ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); - - var reader = new TupleReader(packedKey); - - T1 item1; - if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); - - T2 item2; - if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); - - T3 item3; - if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); - - if (reader.Input.HasMore) throw new FormatException("The key contains more than three items"); - - return Create(item1, item2, item3); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } - /// Unpack a key containing four elements - /// Slice that should contain the packed representation of a tuple with four elements - /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static STuple DecodeKey(Slice packedKey) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); - - var reader = new TupleReader(packedKey); - - T1 item1; - if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); - - T2 item2; - if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); - - T3 item3; - if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); - - T4 item4; - if (!DecodeNext(ref reader, out item4)) throw new FormatException("Failed to decode fourth item"); - - if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); - - return Create(item1, item2, item3, item4); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } - /// Unpack a key containing five elements - /// Slice that should contain the packed representation of a tuple with five elements - /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. - public static STuple DecodeKey(Slice packedKey) + [Pure] + public static STuple Create(ValueTuple tuple) { - if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack an empty tuple"); - - var reader = new TupleReader(packedKey); - - T1 item1; - if (!DecodeNext(ref reader, out item1)) throw new FormatException("Failed to decode first item"); - - T2 item2; - if (!DecodeNext(ref reader, out item2)) throw new FormatException("Failed to decode second item"); - - T3 item3; - if (!DecodeNext(ref reader, out item3)) throw new FormatException("Failed to decode third item"); - - T4 item4; - if (!DecodeNext(ref reader, out item4)) throw new FormatException("Failed to decode fourth item"); - - T5 item5; - if (!DecodeNext(ref reader, out item5)) throw new FormatException("Failed to decode fiftyh item"); - - if (reader.Input.HasMore) throw new FormatException("The key contains more than four items"); - - return Create(item1, item2, item3, item4, item5); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } - /// Unpack the next item in the tuple, and advance the cursor - /// Type of the next value in the tuple - /// Reader positionned at the start of the next item to read - /// If decoding succeedsd, receives the decoded value. - /// True if the decoded succeeded (and receives the decoded value). False if the tuple has reached the end. - public static bool DecodeNext(ref TupleReader input, out T value) + [Pure] + public static STuple Create(ref ValueTuple tuple) { - if (!input.Input.HasMore) - { - value = default(T); - return false; - } - - var slice = TupleParser.ParseNext(ref input); - value = TuplePacker.Deserialize(slice); - return true; + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } - #endregion - - #region PackWithPrefix... - - //note: they are equivalent to the Pack<...>() methods, they only take a binary prefix - - /// Efficiently concatenate a prefix with the packed representation of a 1-tuple - public static Slice EncodePrefixedKey(Slice prefix, T value) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 2-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 3-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 4-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - TuplePacker.Encoder(ref writer, value4); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 5-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - TuplePacker.Encoder(ref writer, value4); - TuplePacker.Encoder(ref writer, value5); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 6-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - TuplePacker.Encoder(ref writer, value4); - TuplePacker.Encoder(ref writer, value5); - TuplePacker.Encoder(ref writer, value6); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 7-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - TuplePacker.Encoder(ref writer, value4); - TuplePacker.Encoder(ref writer, value5); - TuplePacker.Encoder(ref writer, value6); - TuplePacker.Encoder(ref writer, value7); - return writer.Output.ToSlice(); - } - - /// Efficiently concatenate a prefix with the packed representation of a 8-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) - { - var writer = new TupleWriter(); - writer.Output.WriteBytes(prefix); - TuplePacker.Encoder(ref writer, value1); - TuplePacker.Encoder(ref writer, value2); - TuplePacker.Encoder(ref writer, value3); - TuplePacker.Encoder(ref writer, value4); - TuplePacker.Encoder(ref writer, value5); - TuplePacker.Encoder(ref writer, value6); - TuplePacker.Encoder(ref writer, value7); - TuplePacker.Encoder(ref writer, value8); - return writer.Output.ToSlice(); - } +#endif #endregion @@ -1057,388 +433,612 @@ public static bool Equals(ITuple left, ITuple right) public static bool Equivalent(ITuple left, ITuple right) { if (object.ReferenceEquals(left, null)) return object.ReferenceEquals(right, null); - return !object.ReferenceEquals(right, null) && Equals(left, right, TupleComparisons.Default); - } - - /// Create a range that selects all tuples that are stored under the specified subspace: 'prefix\x00' <= k < 'prefix\xFF' - /// Subspace binary prefix (that will be excluded from the range) - /// Range including all possible tuples starting with the specified prefix. - /// STuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) - public static KeyRange ToRange(Slice prefix) - { - if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); - - //note: there is no guarantee that prefix is a valid packed tuple (could be any exotic binary prefix) + return !object.ReferenceEquals(right, null) && TupleHelpers.Equals(left, right, TupleComparisons.Default); + } + + public static class Formatter + { + + private const string TokenNull = "null"; + private const string TokenFalse = "false"; + private const string TokenTrue = "true"; + private const string TokenDoubleQuote = "\""; + private const string TokenSingleQuote = "'"; + private const string TokenTupleEmpty = "()"; + private const string TokenTupleSep = ", "; + private const string TokenTupleClose = ")"; + private const string TokenTupleSingleClose = ",)"; + + /// Converts any object into a displayable string, for logging/debugging purpose + /// Object to stringify + /// String representation of the object + /// + /// Stringify<{REF_TYPE}>(null) => "nil" + /// Stringify<string>{string}("hello") => "\"hello\"" + /// Stringify<int>(123) => "123" + /// Stringify<double>(123.4d) => "123.4" + /// Stringify<bool>(true) => "true" + /// Stringify<char>('Z') => "'Z'" + /// Stringify<Slice>((...) => hexa decimal string ("01 23 45 67 89 AB CD EF") + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(T item) + { + if (default(T) == null) + { + if (item == null) return TokenNull; + } + // ! + if (typeof(T) == typeof(int)) return Stringify((int) (object) item); + if (typeof(T) == typeof(uint)) return Stringify((uint) (object) item); + if (typeof(T) == typeof(long)) return Stringify((long) (object) item); + if (typeof(T) == typeof(ulong)) return Stringify((ulong) (object) item); + if (typeof(T) == typeof(bool)) return Stringify((bool) (object) item); + if (typeof(T) == typeof(char)) return Stringify((char) (object) item); + if (typeof(T) == typeof(Slice)) return Stringify((Slice)(object)item); + if (typeof(T) == typeof(double)) return Stringify((double) (object) item); + if (typeof(T) == typeof(float)) return Stringify((float) (object) item); + if (typeof(T) == typeof(Guid)) return Stringify((Guid) (object) item); + if (typeof(T) == typeof(Uuid128)) return Stringify((Uuid128) (object) item); + if (typeof(T) == typeof(Uuid64)) return Stringify((Uuid64) (object) item); + // + if (typeof(T) == typeof(string)) return Stringify((string) (object) item); + + // some other type + return StringifyInternal(item); + } - // prefix => [ prefix."\0", prefix."\xFF" ) - return new KeyRange( - prefix + FdbKey.MinValue, - prefix + FdbKey.MaxValue - ); - } + /// Converts any object into a displayable string, for logging/debugging purpose + /// Object to stringify + /// String representation of the object + /// + /// Stringify(null) => "nil" + /// Stringify("hello") => "\"hello\"" + /// Stringify(123) => "123" + /// Stringify(123.4d) => "123.4" + /// Stringify(true) => "true" + /// Stringify('Z') => "'Z'" + /// Stringify((Slice)...) => hexa decimal string ("01 23 45 67 89 AB CD EF") + /// + [NotNull] + internal static string StringifyBoxed(object item) + { + switch (item) + { + case null: return TokenNull; + case string s: return Stringify(s); + case int i: return Stringify(i); + case long l: return Stringify(l); + case uint u: return Stringify(u); + case ulong ul: return Stringify(ul); + case bool b: return Stringify(b); + case char c: return Stringify(c); + case Slice sl: return Stringify(sl); + case double d: return Stringify(d); + case float f: return Stringify(f); + case Guid guid: return Stringify(guid); + case Uuid128 u128: return Stringify(u128); + case Uuid64 u64: return Stringify(u64); + } - /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// STuple.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. - public static KeyRange ToRange([NotNull] ITuple tuple) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + // some other type + return StringifyInternal(item); + } - // tuple => [ packed."\0", packed."\xFF" ) - var packed = tuple.ToSlice(); + private static string StringifyInternal(object item) + { + if (item is byte[] bytes) return Stringify(bytes.AsSlice()); + if (item is Slice slice) return Stringify(slice); + if (item is ArraySegment buffer) return Stringify(buffer.AsSlice()); + //TODO: Span, ReadOnlySpan, Memory, ReadOnlyMemory, ... + if (item is IFormattable f) return f.ToString(null, CultureInfo.InvariantCulture); + + // This will probably not give a meaningful result ... :( + return item.ToString(); + } - return new KeyRange( - packed + FdbKey.MinValue, - packed + FdbKey.MaxValue - ); - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + //TODO: escape the string? If it contains \0 or control chars, it can cause problems in the console or debugger output + public static string Stringify(string item) => TokenDoubleQuote + item + TokenDoubleQuote; /* "hello" */ - /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// STuple.ToRange(Slice.FromInt32(42), STuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. - /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. - public static KeyRange ToRange(Slice prefix, [NotNull] ITuple tuple) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(bool item) => item ? TokenTrue : TokenFalse; - // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) - var packed = prefix + tuple.ToSlice(); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(int item) => StringConverters.ToString(item); - return new KeyRange( - packed + FdbKey.MinValue, - packed + FdbKey.MaxValue - ); - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(uint item) => StringConverters.ToString(item); - private const string TokenNull = "null"; - private const string TokenDoubleQuote = "\""; - private const string TokenSingleQuote = "'"; - private const string TokenOpenBracket = "{"; - private const string TokenCloseBracket = "}"; - private const string TokenTupleEmpty = "()"; - private const string TokenTupleSep = ", "; - private const string TokenTupleClose = ")"; - private const string TokenTupleSingleClose = ",)"; - - /// Converts any object into a displayble string, for logging/debugging purpose - /// Object to stringify - /// String representation of the object - /// - /// Stringify(null) => "nil" - /// Stringify("hello") => "\"hello\"" - /// Stringify(123) => "123" - /// Stringify(123.4) => "123.4" - /// Stringify(true) => "true" - /// Stringify(Slice) => hexa decimal string ("01 23 45 67 89 AB CD EF") - /// - [NotNull] - internal static string Stringify(object item) - { - if (item == null) return TokenNull; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(long item) => StringConverters.ToString(item); - var s = item as string; - //TODO: escape the string? If it contains \0 or control chars, it can cause problems in the console or debugger output - if (s != null) return TokenDoubleQuote + s + TokenDoubleQuote; /* "hello" */ + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(ulong item) => StringConverters.ToString(item); - if (item is int) return ((int)item).ToString(null, CultureInfo.InvariantCulture); - if (item is long) return ((long)item).ToString(null, CultureInfo.InvariantCulture); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(double item) => item.ToString("R", CultureInfo.InvariantCulture); - if (item is char) return TokenSingleQuote + new string((char)item, 1) + TokenSingleQuote; /* 'X' */ + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(float item) => item.ToString("R", CultureInfo.InvariantCulture); - if (item is Slice) return ((Slice)item).ToAsciiOrHexaString(); - if (item is byte[]) return Slice.Create((byte[]) item).ToAsciiOrHexaString(); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(char item) => TokenSingleQuote + new string(item, 1) + TokenSingleQuote; /* 'X' */ - if (item is FdbTupleAlias) return TokenOpenBracket + ((FdbTupleAlias)item).ToString() + TokenCloseBracket; /* {X} */ + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(Slice item) => item.IsNull ? "null" : '`' + Slice.Dump(item, item.Count) + '`'; - // decimals need the "R" representation to have all the digits - if (item is double) return ((double)item).ToString("R", CultureInfo.InvariantCulture); - if (item is float) return ((float)item).ToString("R", CultureInfo.InvariantCulture); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(byte[] item) => Stringify(item.AsSlice()); - if (item is Guid) return ((Guid)item).ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx} */ - if (item is Uuid128) return ((Uuid128)item).ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx} */ - if (item is Uuid64) return ((Uuid64)item).ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxxxxxx} */ + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(ArraySegment item) => Stringify(item.AsSlice()); - var f = item as IFormattable; - if (f != null) return f.ToString(null, CultureInfo.InvariantCulture); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(Guid item) => item.ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx} */ - // This will probably not give a meaningful result ... :( - return item.ToString(); - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(Uuid128 item) => item.ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx} */ - /// Converts a list of object into a displaying string, for loggin/debugging purpose - /// Array containing items to stringfy - /// Start offset of the items to convert - /// Number of items to convert - /// String representation of the tuple in the form "(item1, item2, ... itemN,)" - /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\",) - [NotNull] - internal static string ToString(object[] items, int offset, int count) - { - if (items == null) return String.Empty; - Contract.Requires(offset >= 0 && count >= 0); + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string Stringify(Uuid64 item) => item.ToString("B", CultureInfo.InstalledUICulture); /* {xxxxxxxx-xxxxxxxx} */ - if (count <= 0) - { // empty tuple: "()" - return TokenTupleEmpty; - } - - var sb = new StringBuilder(); - sb.Append('(').Append(Stringify(items[offset++])); - - if (count == 1) - { // singleton tuple : "(X,)" - return sb.Append(TokenTupleSingleClose).ToString(); - } - - while (--count > 0) + /// Converts a list of object into a displaying string, for loggin/debugging purpose + /// Array containing items to stringfy + /// Start offset of the items to convert + /// Number of items to convert + /// String representation of the tuple in the form "(item1, item2, ... itemN,)" + /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\",) + [NotNull] + public static string ToString(object[] items, int offset, int count) { - sb.Append(TokenTupleSep /* ", " */).Append(Stringify(items[offset++])); - } - return sb.Append(TokenTupleClose /* ",)" */).ToString(); - } + if (items == null) return String.Empty; + Contract.Requires(offset >= 0 && count >= 0); - /// Converts a sequence of object into a displaying string, for loggin/debugging purpose - /// Sequence of items to stringfy - /// String representation of the tuple in the form "(item1, item2, ... itemN,)" - /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\") - [NotNull] - internal static string ToString(IEnumerable items) - { - if (items == null) return String.Empty; - using (var enumerator = items.GetEnumerator()) - { - if (!enumerator.MoveNext()) - { // empty tuple : "()" + if (count <= 0) + { // empty tuple: "()" return TokenTupleEmpty; } var sb = new StringBuilder(); - sb.Append('(').Append(Stringify(enumerator.Current)); - bool singleton = true; - while (enumerator.MoveNext()) + sb.Append('('); + sb.Append(StringifyBoxed(items[offset++])); + + if (count == 1) + { // singleton tuple : "(X,)" + return sb.Append(TokenTupleSingleClose).ToString(); + } + + while (--count > 0) { - singleton = false; - sb.Append(TokenTupleSep).Append(Stringify(enumerator.Current)); + sb.Append(TokenTupleSep /* ", " */).Append(StringifyBoxed(items[offset++])); } - // add a trailing ',' for singletons - return sb.Append(singleton ? TokenTupleSingleClose : TokenTupleClose).ToString(); + return sb.Append(TokenTupleClose /* ",)" */).ToString(); } - } - /// Default (non-optimized) implementation of ITuple.this[long?, long?] - /// Tuple to slice - /// Start offset of the section (included) - /// End offset of the section (included) - /// New tuple only containing items inside this section - [NotNull] - internal static ITuple Splice([NotNull] ITuple tuple, int? fromIncluded, int? toExcluded) - { - Contract.Requires(tuple != null); - int count = tuple.Count; - if (count == 0) return STuple.Empty; - - int start = fromIncluded.HasValue ? MapIndexBounded(fromIncluded.Value, count) : 0; - int end = toExcluded.HasValue ? MapIndexBounded(toExcluded.Value, count) : count; + /// Converts a sequence of object into a displaying string, for loggin/debugging purpose + /// Sequence of items to stringfy + /// String representation of the tuple in the form "(item1, item2, ... itemN,)" + /// ToString(STuple.Create("hello", 123, true, "world")) => "(\"hello\", 123, true, \"world\") + [NotNull] + public static string ToString(IEnumerable items) + { + if (items == null) return string.Empty; - int len = end - start; + if (items is object[] arr) return ToString(arr, 0, arr.Length); - if (len <= 0) return STuple.Empty; - if (start == 0 && len == count) return tuple; - switch(len) - { - case 1: return new ListTuple(new object[] { tuple[start] }, 0, 1); - case 2: return new ListTuple(new object[] { tuple[start], tuple[start + 1] }, 0, 2); - default: + using (var enumerator = items.GetEnumerator()) { - var items = new object[len]; - //note: can be slow for tuples using linked-lists, but hopefully they will have their own Slice implementation... - int q = start; - for (int p = 0; p < items.Length; p++) + if (!enumerator.MoveNext()) + { // empty tuple : "()" + return TokenTupleEmpty; + } + + var sb = new StringBuilder(); + sb.Append('(').Append(StringifyBoxed(enumerator.Current)); + bool singleton = true; + while (enumerator.MoveNext()) { - items[p] = tuple[q++]; + singleton = false; + sb.Append(TokenTupleSep).Append(StringifyBoxed(enumerator.Current)); } - return new ListTuple(items, 0, len); + // add a trailing ',' for singletons + return sb.Append(singleton ? TokenTupleSingleClose : TokenTupleClose).ToString(); } } + } - /// Default (non-optimized) implementation for ITuple.StartsWith() - /// Larger tuple - /// Smaller tuple - /// True if starts with (or is equal to) - internal static bool StartsWith([NotNull] ITuple a, [NotNull] ITuple b) + /// Hleper to parse strings back into tuples + public static class Deformatter { - Contract.Requires(a != null && b != null); - if (object.ReferenceEquals(a, b)) return true; - int an = a.Count; - int bn = b.Count; - if (bn > an) return false; - if (bn == 0) return true; // note: 'an' can only be 0 because of previous test - for (int i = 0; i < bn; i++) + [Pure, NotNull] + public static ITuple Parse([NotNull] string expression) { - if (!object.Equals(a[i], b[i])) return false; + Contract.NotNullOrWhiteSpace(expression, nameof(expression)); + var parser = new Parser(expression.Trim()); + var tuple = parser.ParseExpression(); + if (parser.HasMore) throw new FormatException("Unexpected token after final ')' in Tuple expression."); + return tuple; } - return true; - } - /// Default (non-optimized) implementation for ITuple.EndsWith() - /// Larger tuple - /// Smaller tuple - /// True if starts with (or is equal to) - internal static bool EndsWith([NotNull] ITuple a, [NotNull] ITuple b) - { - Contract.Requires(a != null && b != null); - if (object.ReferenceEquals(a, b)) return true; - int an = a.Count; - int bn = b.Count; + /// Parse a tuple expression at the start of a string + /// String who starts with a valid Tuple expression, with optional extra characters + /// First item is the parsed tuple, and the second item is the rest of the string (or null if we consumed the whole expression) + public static void ParseNext(string expression, out ITuple tuple, out string tail) + { + Contract.NotNullOrWhiteSpace(expression, nameof(expression)); + if (string.IsNullOrWhiteSpace(expression)) + { + tuple = null; + tail = null; + return; + } - if (bn > an) return false; - if (bn == 0) return true; // note: 'an' can only be 0 because of previous test + var parser = new Parser(expression.Trim()); + tuple = parser.ParseExpression(); + string s = parser.GetTail(); + tail = string.IsNullOrWhiteSpace(s) ? null : s.Trim(); + } - int offset = an - bn; - for (int i = 0; i < bn; i++) + private struct Parser { - if (!object.Equals(a[offset + i], b[i])) return false; - } - return true; - } - /// Helper to copy the content of a tuple at a specific position in an array - /// Updated offset just after the last element of the copied tuple - internal static int CopyTo([NotNull] ITuple tuple, [NotNull] object[] array, int offset) - { - Contract.Requires(tuple != null && array != null && offset >= 0); + private const char EOF = '\xFFFF'; - foreach (var item in tuple) - { - array[offset++] = item; - } - return offset; - } + public Parser(string expression) + { + this.Expression = expression; + this.Cursor = 0; + } - /// Maps a relative index into an absolute index - /// Relative index in the tuple (from the end if negative) - /// Size of the tuple - /// Absolute index from the start of the tuple, or exception if outside of the tuple - /// If the absolute index is outside of the tuple (<0 or >=) - internal static int MapIndex(int index, int count) - { - int offset = index; - if (offset < 0) offset += count; - if (offset < 0 || offset >= count) FailIndexOutOfRange(index, count); - return offset; - } + public readonly string Expression; + private int Cursor; - /// Maps a relative index into an absolute index - /// Relative index in the tuple (from the end if negative) - /// Size of the tuple - /// Absolute index from the start of the tuple. Truncated to 0 if index is before the start of the tuple, or to if the index is after the end of the tuple - internal static int MapIndexBounded(int index, int count) - { - if (index < 0) index += count; - return Math.Max(Math.Min(index, count), 0); - } + public bool HasMore => this.Cursor < this.Expression.Length; - [ContractAnnotation("=> halt")] - internal static void FailIndexOutOfRange(int index, int count) - { - throw new IndexOutOfRangeException(String.Format("Index {0} is outside of the tuple's range (0..{1})", index, count - 1)); - } + [CanBeNull] + public string GetTail() => this.Cursor < this.Expression.Length ? this.Expression.Substring(this.Cursor) : null; - internal static int CombineHashCodes(int h1, int h2) - { - return ((h1 << 5) + h1) ^ h2; - } + private char ReadNext() + { + int p = this.Cursor; + string s = this.Expression; + if ((uint) p >= (uint) s.Length) return EOF; + char c = s[p]; + this.Cursor = p + 1; + return c; + } - internal static int CombineHashCodes(int h1, int h2, int h3) - { - int h = ((h1 << 5) + h1) ^ h2; - return ((h << 5) + h) ^ h3; - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private char PeekNext() + { + int p = this.Cursor; + string s = this.Expression; + return (uint) p < (uint) s.Length ? s[p] : EOF; + } - internal static int CombineHashCodes(int h1, int h2, int h3, int h4) - { - return CombineHashCodes(CombineHashCodes(h1, h2), CombineHashCodes(h3, h4)); - } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private void Advance() + { + ++this.Cursor; + } - internal static int CombineHashCodes(int h1, int h2, int h3, int h4, int h5) - { - return CombineHashCodes(CombineHashCodes(h1, h2, h3), CombineHashCodes(h4, h5)); - } + private bool TryReadKeyword(string keyword) + { + //IMPORTANT: 'keyword' doit être en lowercase! + int p = this.Cursor; + string s = this.Expression; + int r = keyword.Length; + if ((uint) (p + r) > (uint) s.Length) return false; // not enough + for (int i = 0; i < r; i++) + { + if (char.ToLowerInvariant(s[p + i]) != keyword[i]) return false; + } + this.Cursor = p + r; + return true; + } - internal static bool Equals(ITuple left, object other, [NotNull] IEqualityComparer comparer) - { - return object.ReferenceEquals(left, null) ? other == null : STuple.Equals(left, other as ITuple, comparer); - } + /// Parse a tuple + [Pure, NotNull] + public ITuple ParseExpression() + { - internal static bool Equals(ITuple x, ITuple y, [NotNull] IEqualityComparer comparer) - { - if (object.ReferenceEquals(x, y)) return true; - if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; + char c = ReadNext(); + if (c != '(') + { + throw new FormatException("Invalid tuple expression. Valid tuple must start with '(' and end with ')'."); + } - return x.Count == y.Count && DeepEquals(x, y, comparer); - } + bool expectItem = true; - internal static bool DeepEquals([NotNull] ITuple x, [NotNull] ITuple y, [NotNull] IEqualityComparer comparer) - { - Contract.Requires(x != null && y != null && comparer != null); + var items = new List(); + while (true) + { + c = PeekNext(); + switch (c) + { + case ')': + { + //note: we accept a terminal ',' without the last item, to allow "(123,)" as a valid tuple. + if (expectItem && items.Count > 1) throw new FormatException("Missing item before last ',' in Tuple expression"); + Advance(); + return items.Count == 0 ? STuple.Empty : new ListTuple(items); + } + case EOF: + { + throw new FormatException("Missing ')' at the end of tuple expression."); + } + + case ',': + { + if (expectItem) throw new FormatException("Missing ',' before next item in Tuple expression."); + Advance(); + expectItem = true; + break; + } + + case '"': + { // string literal + string s = ReadStringLiteral(); + items.Add(s); + expectItem = false; + break; + } + case '\'': + { // single char literal + Advance(); + char x = ReadNext(); + c = ReadNext(); + if (c != '\'') throw new FormatException("Missing quote after character. Single quotes are for single characters. For strings, use double quotes!"); + items.Add(x); + expectItem = false; + break; + } + case '{': + { // Guid + Guid g = ReadGuidLiteral(); + items.Add(g); + expectItem = false; + break; + } + case '(': + { // embedded tuple! + var sub = ParseExpression(); + items.Add(sub); + expectItem = false; + break; + } + + default: + { + if (char.IsWhiteSpace(c)) + { // ignore whitespaces + Advance(); + break; + } + + if (char.IsDigit(c) || c == '-') + { // number! + items.Add(ReadNumberLiteral()); + expectItem = false; + break; + } + + if (c == 't' || c == 'T') + { // true? + if (!TryReadKeyword("true")) throw new FormatException("Unrecognized keyword in Tuple expression. Did you meant to write 'true' instead?"); + items.Add(true); + expectItem = false; + break; + } + + if (c == 'f' || c == 'F') + { // false? + if (!TryReadKeyword("false")) throw new FormatException("Unrecognized keyword in Tuple expression. Did you meant to write 'false' instead?"); + items.Add(false); + expectItem = false; + break; + } + + throw new FormatException($"Invalid token '{c}' in Tuple expression."); + } + } + } + } - using (var xs = x.GetEnumerator()) - using (var ys = y.GetEnumerator()) - { - while (xs.MoveNext()) + private object ReadNumberLiteral() { - if (!ys.MoveNext()) return false; + bool dec = false; + bool neg = false; + bool exp = false; + + string s = this.Expression; + int start = this.Cursor; + int end = s.Length; + int p = start; + ulong x = 0; + + char c = s[p]; + if (c == '-') + { + neg = true; + } + else if (c != '+') + { + x = (ulong) (c - '0'); + } + ++p; - if (!comparer.Equals(xs.Current, ys.Current)) return false; - } + while (p < end) + { + c = s[p]; + if (char.IsDigit(c)) + { + x = checked(x * 10 + (ulong) (c - '0')); + ++p; + continue; + } + + if (c == '.') + { + if (dec) throw new FormatException("Redundant '.' in number that already has a decimal point."); + if (exp) throw new FormatException("Unexpected '.' in exponent part of number."); + dec = true; + ++p; + continue; + } + + if (c == ',' || c == ')' || char.IsWhiteSpace(c)) + { + break; + } + + if (c == 'E') + { + if (dec) throw new FormatException("Redundant 'E' in number that already has an exponent."); + exp = true; + ++p; + continue; + } + + if (c == '-' || c == '+') + { + if (!exp) throw new FormatException("Unexpected sign in number."); + ++p; + continue; + } + + throw new FormatException($"Unexpected token '{c}' while parsing number in Tuple expression."); + } - return !ys.MoveNext(); - } - } + this.Cursor = p; - internal static int StructuralGetHashCode(ITuple tuple, [NotNull] IEqualityComparer comparer) - { - Contract.Requires(comparer != null); + if (!dec && !exp) + { + if (neg) + { + if (x < int.MaxValue) return -((int) x); + if (x < long.MaxValue) return -((long) x); + if (x == 1UL + long.MaxValue) return long.MinValue; + throw new OverflowException("Parsed number is too large"); + } + + if (x <= int.MaxValue) return (int) x; + if (x <= long.MaxValue) return (long) x; + return x; + } - if (object.ReferenceEquals(tuple, null)) - { - return comparer.GetHashCode(null); - } + return double.Parse(s.Substring(start, p - start), CultureInfo.InvariantCulture); + } - int h = 0; - foreach(var item in tuple) - { - h = CombineHashCodes(h, comparer.GetHashCode(item)); - } - return h; - } + private string ReadStringLiteral() + { + string s = this.Expression; + int p = this.Cursor; + int end = p + s.Length; - internal static int StructuralCompare(ITuple x, ITuple y, [NotNull] IComparer comparer) - { - Contract.Requires(comparer != null); + // main loop is optimistic and assumes that the string will not be escaped. + // If we find the first instance of '\', then we switch to a secondary loop that uses a StringBuilder to decode each character - if (object.ReferenceEquals(x, y)) return 0; - if (object.ReferenceEquals(x, null)) return -1; - if (object.ReferenceEquals(y, null)) return 1; + char c = s[p++]; + if (c != '"') throw new FormatException("Expected '\"' token is missing in Tuple expression"); + int start = p; - using (var xs = x.GetEnumerator()) - using (var ys = y.GetEnumerator()) - { - while (xs.MoveNext()) - { - if (!ys.MoveNext()) return 1; + while (p < end) + { + c = s[p]; + if (c == '"') + { + this.Cursor = p + 1; + return s.Substring(start, p - start); + } + + if (c == '\\') + { // string is escaped, will need to decode the content + ++p; + goto parse_escaped_string; + } + ++p; + } + goto truncated_string; - int cmp = comparer.Compare(xs.Current, ys.Current); - if (cmp != 0) return cmp; + parse_escaped_string: + bool escape = true; + var sb = new StringBuilder(); + if (p > start + 1) sb.Append(s.Substring(start, p - start - 1)); // copy what we have parsed so far + while (p < end) + { + c = s[p]; + if (c == '"') + { + if (escape) + { + escape = false; + } + else + { + this.Cursor = p + 1; + return sb.ToString(); + } + } + else if (c == '\\') + { + if (!escape) + { // start of escape sequence + escape = true; + ++p; + continue; + } + escape = false; + } + else if (escape) + { + if (c == 't') c = '\t'; + else if (c == 'r') c = '\r'; + else if (c == 'n') c = '\n'; + //TODO: \x## and \u#### syntax! + else throw new FormatException($"Unrecognized '\\{c}' token while parsing string in Tuple expression"); + escape = false; + } + ++p; + sb.Append(c); + } + truncated_string: + throw new FormatException("Missing double quote at end of string in Tuple expression"); + } + private Guid ReadGuidLiteral() + { + var s = this.Expression; + int p = this.Cursor; + int end = s.Length; + char c = s[p]; + if (s[p] != '{') throw new FormatException($"Unexpected token '{c}' at start of GUID in Tuple expression"); + ++p; + int start = p; + while (p < end) + { + c = s[p]; + if (c == '}') + { + string lit = s.Substring(start, p - start); + // Shortcut: "{} or {0} means "00000000-0000-0000-0000-000000000000" + Guid g = lit == "" || lit == "0" ? Guid.Empty : Guid.Parse(lit); + this.Cursor = p + 1; + return g; + } + ++p; + } + + throw new FormatException("Invalid GUID in Tuple expression."); } - return ys.MoveNext() ? -1 : 0; + } } #endregion - } - } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs index 6b1e57759..302f4b230 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -26,21 +26,28 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Tuple that holds only one item /// Type of the item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif { // This is mostly used by code that create a lot of temporary singleton, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -54,41 +61,41 @@ public STuple(T1 item1) this.Item1 = item1; } - public int Count { get { return 1; } } + public int Count => 1; public object this[int index] { get { - if (index > 0 || index < -1) STuple.FailIndexOutOfRange(index, 1); + if (index > 0 || index < -1) return TupleHelpers.FailIndexOutOfRange(index, 1); return this.Item1; } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . - public R Get(int index) + /// Value of the item at position , adapted into type . + public TItem Get(int index) { - if (index > 0 || index < -1) STuple.FailIndexOutOfRange(index, 1); - return FdbConverters.Convert(this.Item1); + if (index > 0 || index < -1) return TupleHelpers.FailIndexOutOfRange(index, 1); + return TypeConverters.Convert(this.Item1); } - /// Return the typed value of the last item in the tuple - R ITuple.Last() + void ITupleSerializable.PackTo(ref TupleWriter writer) { - return FdbConverters.Convert(this.Item1); + PackTo(ref writer); } - public void PackTo(ref TupleWriter writer) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) { - TuplePacker.Encoder(ref writer, this.Item1); + TupleSerializer.Default.PackTo(ref writer, ref this); } ITuple ITuple.Append(T2 value) @@ -100,7 +107,8 @@ ITuple ITuple.Append(T2 value) /// Tuple that will be added as an embedded item /// New tuple with one extra item /// If you want to append the *items* of , and not the tuple itself, please call ! - [NotNull] + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public STuple Append(T2 value) { return new STuple(this.Item1, value); @@ -109,20 +117,29 @@ public STuple Append(T2 value) /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - [NotNull] - public ITuple Concat([NotNull] ITuple tuple) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); } /// Copy the item of this singleton into an array at the specified offset - public void CopyTo([NotNull] object[] array, int offset) + public void CopyTo(object[] array, int offset) { array[offset] = this.Item1; } + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1) + { + item1 = this.Item1; + } + + /// Execute a lambda Action with the content of this tuple /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void With([NotNull] Action lambda) { lambda(this.Item1); @@ -131,7 +148,8 @@ public void With([NotNull] Action lambda) /// Execute a lambda Function with the content of this tuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple - public R With([NotNull] Func lambda) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) { return lambda(this.Item1); } @@ -146,15 +164,10 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() return this.GetEnumerator(); } - public Slice ToSlice() - { - return STuple.EncodeKey(this.Item1); - } - public override string ToString() { // singleton tuples end with a trailing ',' - return "(" + STuple.Stringify(this.Item1) + ",)"; + return "(" + STuple.Formatter.Stringify(this.Item1) + ",)"; } public override bool Equals(object obj) @@ -167,6 +180,12 @@ public bool Equals(ITuple other) return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1); + } + public override int GetHashCode() { return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); @@ -185,11 +204,17 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is STuple) + if (other is STuple stuple) + { + return comparer.Equals(this.Item1, stuple.Item1); + } +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) { - return comparer.Equals(this.Item1, ((STuple)other).Item1); + return comparer.Equals(this.Item1, vtuple.Item1); } - return STuple.Equals(this, other, comparer); +#endif + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) @@ -197,17 +222,156 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) return comparer.GetHashCode(this.Item1); } - public static implicit operator STuple(Tuple t) + [Pure] + public static implicit operator STuple([NotNull] Tuple t) { - if (t == null) throw new ArgumentNullException("t"); + Contract.NotNull(t, nameof(t)); return new STuple(t.Item1); } + [Pure, NotNull] public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1); } +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, tuple.Item1); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, tuple.Item1, tuple.Item2); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return new ValueTuple(this.Item1); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple(STuple t) + { + return new ValueTuple(t.Item1); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1); + } + +#endif + + public sealed class Comparer : IComparer> + { + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + return Comparer1.Compare(x.Item1, y.Item1); + } + } + + public sealed class EqualityComparer : IEqualityComparer> + { + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1); + } + + public int GetHashCode(STuple obj) + { + return Comparer1.GetHashCode(obj.Item1); + } + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs index 3cf96db23..08b7461d4 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -26,25 +26,32 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Tuple that holds a pair of items /// Type of the first item /// Type of the second item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif { // This is mostly used by code that create a lot of temporary pair, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the pair public readonly T1 Item1; @@ -58,7 +65,7 @@ public STuple(T1 item1, T2 item2) this.Item2 = item2; } - public int Count { get { return 2; } } + public int Count => 2; public object this[int index] { @@ -68,46 +75,56 @@ public object this[int index] { case 0: case -2: return this.Item1; case 1: case -1: return this.Item2; - default: STuple.FailIndexOutOfRange(index, 2); return null; + default: return TupleHelpers.FailIndexOutOfRange(index, 2); } } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . - public R Get(int index) + /// Value of the item at position , adapted into type . + public TItem Get(int index) { switch(index) { - case 0: case -2: return FdbConverters.Convert(this.Item1); - case 1: case -1: return FdbConverters.Convert(this.Item2); - default: STuple.FailIndexOutOfRange(index, 2); return default(R); + case 0: case -2: return TypeConverters.Convert(this.Item1); + case 1: case -1: return TypeConverters.Convert(this.Item2); + default: return TupleHelpers.FailIndexOutOfRange(index, 2); } } /// Return the value of the last item in the tuple public T2 Last { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] get { return this.Item2; } } - /// Return the typed value of the last item in the tuple - R ITuple.Last() + /// Return a tuple without the first item + public STuple Tail + { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new STuple(this.Item2); } + } + + void ITupleSerializable.PackTo(ref TupleWriter writer) { - return FdbConverters.Convert(this.Item2); + PackTo(ref writer); } - public void PackTo(ref TupleWriter writer) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) { - TuplePacker.Encoder(ref writer, this.Item1); - TuplePacker.Encoder(ref writer, this.Item2); + TupleSerializer.Default.PackTo(ref writer, ref this); } ITuple ITuple.Append(T3 value) @@ -119,10 +136,11 @@ ITuple ITuple.Append(T3 value) /// Value that will be added as an embedded item /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! + [MethodImpl(MethodImplOptions.AggressiveInlining)] public STuple Append(T3 value) { return new STuple(this.Item1, this.Item2, value); - // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). + // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). // ex: if we have N possible types, then there could be N^3 possible variants of STuple that the JIT has to deal with. // => if this starts becoming a problem, then we should return a list tuple ! } @@ -130,7 +148,8 @@ public STuple Append(T3 value) /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - public ITuple Concat([NotNull] ITuple tuple) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); } @@ -142,8 +161,17 @@ public void CopyTo(object[] array, int offset) array[offset + 1] = this.Item2; } + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1, out T2 item2) + { + item1 = this.Item1; + item2 = this.Item2; + } + /// Execute a lambda Action with the content of this tuple /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void With([NotNull] Action lambda) { lambda(this.Item1, this.Item2); @@ -152,7 +180,8 @@ public void With([NotNull] Action lambda) /// Execute a lambda Function with the content of this tuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple - public R With([NotNull] Func lambda) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) { return lambda(this.Item1, this.Item2); } @@ -168,14 +197,14 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() return this.GetEnumerator(); } - public Slice ToSlice() - { - return STuple.EncodeKey(this.Item1, this.Item2); - } - public override string ToString() { - return "(" + STuple.Stringify(this.Item1) + ", " + STuple.Stringify(this.Item2) + ")"; + return string.Concat( + "(", + STuple.Formatter.Stringify(this.Item1), ", ", + STuple.Formatter.Stringify(this.Item2), + ")" + ); } public override bool Equals(object obj) @@ -188,6 +217,13 @@ public bool Equals(ITuple other) return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2); + } + public override int GetHashCode() { return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); @@ -205,37 +241,182 @@ public override int GetHashCode() || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } - bool System.Collections.IStructuralEquatable.Equals(object other, System.Collections.IEqualityComparer comparer) + bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is STuple) + if (other is STuple stuple) + { + return comparer.Equals(this.Item1, stuple.Item1) + && comparer.Equals(this.Item2, stuple.Item2); + } +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) { - var tuple = (STuple)other; - return comparer.Equals(this.Item1, tuple.Item1) - && comparer.Equals(this.Item2, tuple.Item2); + return comparer.Equals(this.Item1, vtuple.Item1) + && comparer.Equals(this.Item2, vtuple.Item2); } - return STuple.Equals(this, other, comparer); +#endif + return TupleHelpers.Equals(this, other, comparer); } - int System.Collections.IStructuralEquatable.GetHashCode(System.Collections.IEqualityComparer comparer) + int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return STuple.CombineHashCodes( + return HashCodes.Combine( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2) ); } - public static implicit operator STuple(Tuple t) + [Pure] + public static implicit operator STuple([NotNull] Tuple t) { - if (t == null) throw new ArgumentNullException("t"); + Contract.NotNull(t, nameof(t)); return new STuple(t.Item1, t.Item2); } + [Pure, NotNull] public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2); } +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + t.Item2 = this.Item2; + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, tuple.Item1); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, tuple.Item1, tuple.Item2); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, tuple.Item1, tuple.Item2, tuple.Item3); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return ValueTuple.Create(this.Item1, this.Item2); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1, t.Item2); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple(STuple t) + { + return ValueTuple.Create(t.Item1, t.Item2); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); + } + +#endif + + public sealed class Comparer : IComparer> + { + + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + private static readonly Comparer Comparer2 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + int cmp = Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = Comparer2.Compare(x.Item2, y.Item2); + return cmp; + } + + } + + public sealed class EqualityComparer : IEqualityComparer> + { + + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer2 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1) + && Comparer2.Equals(x.Item2, y.Item2); + } + + public int GetHashCode(STuple obj) + { + return HashCodes.Combine( + Comparer1.GetHashCode(obj.Item1), + Comparer2.GetHashCode(obj.Item2) + ); + } + + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs index 8966d7fd8..968c7d6a4 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -26,16 +26,19 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; - using System.Text; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Tuple that can hold three items @@ -43,10 +46,13 @@ namespace FoundationDB.Layers.Tuples /// Type of the second item /// Type of the third item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif { // This is mostly used by code that create a lot of temporary triplet, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the triplet public readonly T1 Item1; @@ -63,7 +69,7 @@ public STuple(T1 item1, T2 item2, T3 item3) this.Item3 = item3; } - public int Count { get { return 3; } } + public int Count => 3; public object this[int index] { @@ -74,48 +80,57 @@ public object this[int index] case 0: case -3: return this.Item1; case 1: case -2: return this.Item2; case 2: case -1: return this.Item3; - default: STuple.FailIndexOutOfRange(index, 3); return null; + default: return TupleHelpers.FailIndexOutOfRange(index, 3); } } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . - public R Get(int index) + /// Value of the item at position , adapted into type . + public TItem Get(int index) { switch(index) { - case 0: case -3: return FdbConverters.Convert(this.Item1); - case 1: case -2: return FdbConverters.Convert(this.Item2); - case 2: case -1: return FdbConverters.Convert(this.Item3); - default: STuple.FailIndexOutOfRange(index, 3); return default(R); + case 0: case -3: return TypeConverters.Convert(this.Item1); + case 1: case -2: return TypeConverters.Convert(this.Item2); + case 2: case -1: return TypeConverters.Convert(this.Item3); + default: return TupleHelpers.FailIndexOutOfRange(index, 3); } } /// Return the value of the last item in the tuple public T3 Last { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] get { return this.Item3; } } - /// Return the typed value of the last item in the tuple - R ITuple.Last() + /// Return a tuple without the first item + public STuple Tail + { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new STuple(this.Item2, this.Item3); } + } + + void ITupleSerializable.PackTo(ref TupleWriter writer) { - return FdbConverters.Convert(this.Item3); + PackTo(ref writer); } - public void PackTo(ref TupleWriter writer) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) { - TuplePacker.Encoder(ref writer, this.Item1); - TuplePacker.Encoder(ref writer, this.Item2); - TuplePacker.Encoder(ref writer, this.Item3); + TupleSerializer.Default.PackTo(ref writer, ref this); } ITuple ITuple.Append(T4 value) @@ -128,19 +143,21 @@ ITuple ITuple.Append(T4 value) /// Value that will be added as an embedded item /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! - [NotNull] + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public STuple Append(T4 value) { // Here, the caller was explicitly using the STuple struct so probably care about memory footprint, so we keep returning a struct return new STuple(this.Item1, this.Item2, this.Item3, value); - // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). + // Note: By create a STuple we risk an explosion of the number of combinations of Ts which could potentially cause problems at runtime (too many variants of the same generic types). // ex: if we have N possible types, then there could be N^4 possible variants of STuple that the JIT has to deal with. // => if this starts becoming a problem, then we should return a list tuple ! } /// Copy all the items of this tuple into an array at the specified offset - [NotNull] + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public STuple Append(ITuple value) { //note: this override exists to prevent the explosion of tuple types such as STuple, STuple, STuple> ! @@ -150,8 +167,8 @@ public STuple Append(ITuple value) /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items - [NotNull] - public ITuple Concat([NotNull] ITuple tuple) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); } @@ -163,8 +180,18 @@ public void CopyTo(object[] array, int offset) array[offset + 2] = this.Item3; } + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1, out T2 item2, out T3 item3) + { + item1 = this.Item1; + item2 = this.Item2; + item3 = this.Item3; + } + /// Execute a lambda Action with the content of this tuple /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void With([NotNull] Action lambda) { lambda(this.Item1, this.Item2, this.Item3); @@ -173,7 +200,8 @@ public void With([NotNull] Action lambda) /// Execute a lambda Function with the content of this tuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple - public R With([NotNull] Func lambda) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) { return lambda(this.Item1, this.Item2, this.Item3); } @@ -190,18 +218,15 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() return this.GetEnumerator(); } - public Slice ToSlice() - { - return STuple.EncodeKey(this.Item1, this.Item2, this.Item3); - } - public override string ToString() { - return new StringBuilder(32).Append('(') - .Append(STuple.Stringify(this.Item1)).Append(", ") - .Append(STuple.Stringify(this.Item2)).Append(", ") - .Append(STuple.Stringify(this.Item3)).Append(')') - .ToString(); + return string.Concat( + "(", + STuple.Formatter.Stringify(this.Item1), ", ", + STuple.Formatter.Stringify(this.Item2), ", ", + STuple.Formatter.Stringify(this.Item3), + ")" + ); } public override bool Equals(object obj) @@ -214,6 +239,14 @@ public bool Equals(ITuple other) return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, other.Item3); + } + public override int GetHashCode() { return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); @@ -238,36 +271,192 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is STuple) + if (other is STuple stuple) { - var tuple = (STuple)other; - return comparer.Equals(this.Item1, tuple.Item1) - && comparer.Equals(this.Item2, tuple.Item2) - && comparer.Equals(this.Item3, tuple.Item3); + return comparer.Equals(this.Item1, stuple.Item1) + && comparer.Equals(this.Item2, stuple.Item2) + && comparer.Equals(this.Item3, stuple.Item3); } - return STuple.Equals(this, other, comparer); +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) + { + return comparer.Equals(this.Item1, vtuple.Item1) + && comparer.Equals(this.Item2, vtuple.Item2) + && comparer.Equals(this.Item3, vtuple.Item3); + } +#endif + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return STuple.CombineHashCodes( + return HashCodes.Combine( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3) ); } - public static implicit operator STuple(Tuple t) + [Pure] + public static implicit operator STuple([NotNull] Tuple t) { - if (t == null) throw new ArgumentNullException("t"); + Contract.NotNull(t, nameof(t)); return new STuple(t.Item1, t.Item2, t.Item3); } + [Pure, NotNull] public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3); } +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + t.Item2 = this.Item2; + t.Item3 = this.Item3; + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, tuple.Item1); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, tuple.Item1, tuple.Item2); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, tuple.Item1, tuple.Item2, tuple.Item3); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return ValueTuple.Create(this.Item1, this.Item2, this.Item3); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1, t.Item2, t.Item3); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple (STuple t) + { + return ValueTuple.Create(t.Item1, t.Item2, t.Item3); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, this.Item3); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3); + } + +#endif + + public sealed class Comparer : IComparer> + { + + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + private static readonly Comparer Comparer2 = Comparer.Default; + private static readonly Comparer Comparer3 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + int cmp = Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = Comparer2.Compare(x.Item2, y.Item2); + if (cmp == 0) cmp = Comparer3.Compare(x.Item3, y.Item3); + return cmp; + } + + } + + public sealed class EqualityComparer : IEqualityComparer> + { + + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer2 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer3 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1) + && Comparer2.Equals(x.Item2, y.Item2) + && Comparer3.Equals(x.Item3, y.Item3); + } + + public int GetHashCode(STuple obj) + { + return HashCodes.Combine( + Comparer1.GetHashCode(obj.Item1), + Comparer2.GetHashCode(obj.Item2), + Comparer3.GetHashCode(obj.Item3) + ); + } + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs index a6bc908a3..f406534e8 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -26,16 +26,19 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; - using System.Text; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; using JetBrains.Annotations; /// Tuple that can hold four items @@ -44,10 +47,13 @@ namespace FoundationDB.Layers.Tuples /// Type of the third item /// Type of the fourth item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the quartet public readonly T1 Item1; @@ -69,7 +75,7 @@ public STuple(T1 item1, T2 item2, T3 item3, T4 item4) } /// Number of items in this tuple - public int Count { get { return 4; } } + public int Count => 4; /// Return the Nth item in this tuple public object this[int index] @@ -82,50 +88,58 @@ public object this[int index] case 1: case -3: return this.Item2; case 2: case -2: return this.Item3; case 3: case -1: return this.Item4; - default: STuple.FailIndexOutOfRange(index, 4); return null; + default: return TupleHelpers.FailIndexOutOfRange(index, 4); } } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . - public R Get(int index) + /// Value of the item at position , adapted into type . + public TItem Get(int index) { switch(index) { - case 0: case -4: return FdbConverters.Convert(this.Item1); - case 1: case -3: return FdbConverters.Convert(this.Item2); - case 2: case -2: return FdbConverters.Convert(this.Item3); - case 3: case -1: return FdbConverters.Convert(this.Item4); - default: STuple.FailIndexOutOfRange(index, 4); return default(R); + case 0: case -4: return TypeConverters.Convert(this.Item1); + case 1: case -3: return TypeConverters.Convert(this.Item2); + case 2: case -2: return TypeConverters.Convert(this.Item3); + case 3: case -1: return TypeConverters.Convert(this.Item4); + default: return TupleHelpers.FailIndexOutOfRange(index, 4); } } /// Return the value of the last item in the tuple public T4 Last { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] get { return this.Item4; } } - /// Return the typed value of the last item in the tuple - R ITuple.Last() + /// Return a tuple without the first item + public STuple Tail + { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new STuple(this.Item2, this.Item3, this.Item4); } + } + + void ITupleSerializable.PackTo(ref TupleWriter writer) { - return FdbConverters.Convert(this.Item4); + PackTo(ref writer); } - public void PackTo(ref TupleWriter writer) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) { - TuplePacker.Encoder(ref writer, this.Item1); - TuplePacker.Encoder(ref writer, this.Item2); - TuplePacker.Encoder(ref writer, this.Item3); - TuplePacker.Encoder(ref writer, this.Item4); + TupleSerializer.Default.PackTo(ref writer, ref this); } ITuple ITuple.Append(T5 value) @@ -138,17 +152,19 @@ ITuple ITuple.Append(T5 value) /// Value that will be added as an embedded item /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! - [NotNull] + [NotNull, Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public LinkedTuple Append(T5 value) { // the caller probably cares about the return type, since it is using a struct, but whatever tuple type we use will end up boxing this tuple on the heap, and we will loose type information. - // but, by returning a FdbLinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) + // but, by returning a LinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) return new LinkedTuple(this, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items + [MethodImpl(MethodImplOptions.AggressiveInlining)] public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); @@ -163,8 +179,19 @@ public void CopyTo(object[] array, int offset) array[offset + 3] = this.Item4; } + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1, out T2 item2, out T3 item3, out T4 item4) + { + item1 = this.Item1; + item2 = this.Item2; + item3 = this.Item3; + item4 = this.Item4; + } + /// Execute a lambda Action with the content of this tuple /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void With([NotNull] Action lambda) { lambda(this.Item1, this.Item2, this.Item3, this.Item4); @@ -173,7 +200,8 @@ public void With([NotNull] Action lambda) /// Execute a lambda Function with the content of this tuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple - public R With([NotNull] Func lambda) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) { return lambda(this.Item1, this.Item2, this.Item3, this.Item4); } @@ -191,19 +219,16 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() return this.GetEnumerator(); } - public Slice ToSlice() - { - return STuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4); - } - public override string ToString() { - return new StringBuilder(48).Append('(') - .Append(STuple.Stringify(this.Item1)).Append(", ") - .Append(STuple.Stringify(this.Item2)).Append(", ") - .Append(STuple.Stringify(this.Item3)).Append(", ") - .Append(STuple.Stringify(this.Item4)).Append(')') - .ToString(); + return string.Concat( + "(", + STuple.Formatter.Stringify(this.Item1), ", ", + STuple.Formatter.Stringify(this.Item2), ", ", + STuple.Formatter.Stringify(this.Item3), ", ", + STuple.Formatter.Stringify(this.Item4), + ")" + ); } public override bool Equals(object obj) @@ -216,6 +241,15 @@ public bool Equals(ITuple other) return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, other.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, other.Item4); + } + public override int GetHashCode() { return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); @@ -242,20 +276,28 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is STuple) + if (other is STuple stuple) + { + return comparer.Equals(this.Item1, stuple.Item1) + && comparer.Equals(this.Item2, stuple.Item2) + && comparer.Equals(this.Item3, stuple.Item3) + && comparer.Equals(this.Item4, stuple.Item4); + } +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) { - var tuple = (STuple)other; - return comparer.Equals(this.Item1, tuple.Item1) - && comparer.Equals(this.Item2, tuple.Item2) - && comparer.Equals(this.Item3, tuple.Item3) - && comparer.Equals(this.Item4, tuple.Item4); + return comparer.Equals(this.Item1, vtuple.Item1) + && comparer.Equals(this.Item2, vtuple.Item2) + && comparer.Equals(this.Item3, vtuple.Item3) + && comparer.Equals(this.Item4, vtuple.Item4); } - return STuple.Equals(this, other, comparer); +#endif + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return STuple.CombineHashCodes( + return HashCodes.Combine( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3), @@ -263,17 +305,168 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) ); } - public static implicit operator STuple(Tuple t) + [Pure] + public static implicit operator STuple([NotNull] Tuple t) { - if (t == null) throw new ArgumentNullException("t"); + Contract.NotNull(t, nameof(t)); return new STuple(t.Item1, t.Item2, t.Item3, t.Item4); } + [Pure, NotNull] public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4); } +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + t.Item2 = this.Item2; + t.Item3 = this.Item3; + t.Item4 = this.Item4; + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, this.Item4, tuple.Item1); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, this.Item4, tuple.Item1, tuple.Item2); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1, t.Item2, t.Item3, t.Item4); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple(STuple t) + { + return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, this.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, this.Item4); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4); + } + +#endif + + public sealed class Comparer : IComparer> + { + + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + private static readonly Comparer Comparer2 = Comparer.Default; + private static readonly Comparer Comparer3 = Comparer.Default; + private static readonly Comparer Comparer4 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + int cmp = Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = Comparer2.Compare(x.Item2, y.Item2); + if (cmp == 0) cmp = Comparer3.Compare(x.Item3, y.Item3); + if (cmp == 0) cmp = Comparer4.Compare(x.Item4, y.Item4); + return cmp; + } + + } + + public sealed class EqualityComparer : IEqualityComparer> + { + + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer2 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer3 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer4 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1) + && Comparer2.Equals(x.Item2, y.Item2) + && Comparer3.Equals(x.Item3, y.Item3) + && Comparer4.Equals(x.Item4, y.Item4); + } + + public int GetHashCode(STuple obj) + { + return HashCodes.Combine( + Comparer1.GetHashCode(obj.Item1), + Comparer2.GetHashCode(obj.Item2), + Comparer3.GetHashCode(obj.Item3), + Comparer4.GetHashCode(obj.Item4) + ); + } + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index 0625f103b..1d310ade8 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -1,54 +1,36 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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. - */ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections; using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; - using System.Text; - using FoundationDB.Client; - using FoundationDB.Client.Converters; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; using JetBrains.Annotations; - /// Tuple that can hold four items + /// Tuple that can hold five items /// Type of the 1st item /// Type of the 2nd item /// Type of the 3rd item /// Type of the 4th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. - // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost /// First element of the tuple public readonly T1 Item1; @@ -73,7 +55,7 @@ public STuple(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) } /// Number of items in this tuple - public int Count { get { return 5; } } + public int Count => 5; /// Return the Nth item in this tuple public object this[int index] @@ -87,52 +69,59 @@ public object this[int index] case 2: case -3: return this.Item3; case 3: case -2: return this.Item4; case 4: case -1: return this.Item5; - default: STuple.FailIndexOutOfRange(index, 5); return null; + default: return TupleHelpers.FailIndexOutOfRange(index, 5); } } } public ITuple this[int? fromIncluded, int? toExcluded] { - get { return STuple.Splice(this, fromIncluded, toExcluded); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } } /// Return the typed value of an item of the tuple, given its position - /// Expected type of the item + /// Expected type of the item /// Position of the item (if negative, means relative from the end) - /// Value of the item at position , adapted into type . - public R Get(int index) + /// Value of the item at position , adapted into type . + public TItem Get(int index) { switch(index) { - case 0: case -5: return FdbConverters.Convert(this.Item1); - case 1: case -4: return FdbConverters.Convert(this.Item2); - case 2: case -3: return FdbConverters.Convert(this.Item3); - case 3: case -2: return FdbConverters.Convert(this.Item4); - case 4: case -1: return FdbConverters.Convert(this.Item5); - default: STuple.FailIndexOutOfRange(index, 5); return default(R); + case 0: case -5: return TypeConverters.Convert(this.Item1); + case 1: case -4: return TypeConverters.Convert(this.Item2); + case 2: case -3: return TypeConverters.Convert(this.Item3); + case 3: case -2: return TypeConverters.Convert(this.Item4); + case 4: case -1: return TypeConverters.Convert(this.Item5); + default: return TupleHelpers.FailIndexOutOfRange(index, 5); } } /// Return the value of the last item in the tuple public T5 Last { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] get { return this.Item5; } } - /// Return the typed value of the last item in the tuple - R ITuple.Last() + /// Return a tuple without the first item + public STuple Tail { - return FdbConverters.Convert(this.Item5); + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new STuple(this.Item2, this.Item3, this.Item4, this.Item5); } } - public void PackTo(ref TupleWriter writer) + void ITupleSerializable.PackTo(ref TupleWriter writer) { - TuplePacker.Encoder(ref writer, this.Item1); - TuplePacker.Encoder(ref writer, this.Item2); - TuplePacker.Encoder(ref writer, this.Item3); - TuplePacker.Encoder(ref writer, this.Item4); - TuplePacker.Encoder(ref writer, this.Item5); + PackTo(ref writer); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) + { + TupleSerializer.Default.PackTo(ref writer, ref this); } ITuple ITuple.Append(T6 value) @@ -145,17 +134,17 @@ ITuple ITuple.Append(T6 value) /// Value that will be added as an embedded item /// New tuple with one extra item /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! - [NotNull] - public LinkedTuple Append(T6 value) + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public STuple Append(T6 value) { - // the caller probably cares about the return type, since it is using a struct, but whatever tuple type we use will end up boxing this tuple on the heap, and we will loose type information. - // but, by returning a FdbLinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) - return new LinkedTuple(this, value); + return new STuple(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, value); } /// Appends the items of a tuple at the end of the current tuple. /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items + [MethodImpl(MethodImplOptions.AggressiveInlining)] public ITuple Concat(ITuple tuple) { return STuple.Concat(this, tuple); @@ -171,8 +160,20 @@ public void CopyTo(object[] array, int offset) array[offset + 4] = this.Item5; } + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5) + { + item1 = this.Item1; + item2 = this.Item2; + item3 = this.Item3; + item4 = this.Item4; + item5 = this.Item5; + } + /// Execute a lambda Action with the content of this tuple /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] public void With([NotNull] Action lambda) { lambda(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); @@ -181,7 +182,8 @@ public void With([NotNull] Action lambda) /// Execute a lambda Function with the content of this tuple /// Action that will be passed the content of this tuple as parameters /// Result of calling with the items of this tuple - public R With([NotNull] Func lambda) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) { return lambda(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); } @@ -200,20 +202,18 @@ System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() return this.GetEnumerator(); } - public Slice ToSlice() - { - return STuple.EncodeKey(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); - } - public override string ToString() { - return new StringBuilder(48).Append('(') - .Append(STuple.Stringify(this.Item1)).Append(", ") - .Append(STuple.Stringify(this.Item2)).Append(", ") - .Append(STuple.Stringify(this.Item3)).Append(", ") - .Append(STuple.Stringify(this.Item4)).Append(", ") - .Append(STuple.Stringify(this.Item5)).Append(')') - .ToString(); + return string.Join("", new[] + { + "(", + STuple.Formatter.Stringify(this.Item1), ", ", + STuple.Formatter.Stringify(this.Item2), ", ", + STuple.Formatter.Stringify(this.Item3), ", ", + STuple.Formatter.Stringify(this.Item4), ", ", + STuple.Formatter.Stringify(this.Item5), + ")" + }); } public override bool Equals(object obj) @@ -226,6 +226,16 @@ public bool Equals(ITuple other) return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, other.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, other.Item4) + && SimilarValueComparer.Default.Equals(this.Item5, other.Item5); + } + public override int GetHashCode() { return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); @@ -254,21 +264,30 @@ public override int GetHashCode() bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { if (other == null) return false; - if (other is STuple) + if (other is STuple stuple) { - var tuple = (STuple)other; - return comparer.Equals(this.Item1, tuple.Item1) - && comparer.Equals(this.Item2, tuple.Item2) - && comparer.Equals(this.Item3, tuple.Item3) - && comparer.Equals(this.Item4, tuple.Item4) - && comparer.Equals(this.Item5, tuple.Item5); + return comparer.Equals(this.Item1, stuple.Item1) + && comparer.Equals(this.Item2, stuple.Item2) + && comparer.Equals(this.Item3, stuple.Item3) + && comparer.Equals(this.Item4, stuple.Item4) + && comparer.Equals(this.Item5, stuple.Item5); } - return STuple.Equals(this, other, comparer); +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) + { + return comparer.Equals(this.Item1, vtuple.Item1) + && comparer.Equals(this.Item2, vtuple.Item2) + && comparer.Equals(this.Item3, vtuple.Item3) + && comparer.Equals(this.Item4, vtuple.Item4) + && comparer.Equals(this.Item5, vtuple.Item5); + } +#endif + return TupleHelpers.Equals(this, other, comparer); } int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) { - return STuple.CombineHashCodes( + return HashCodes.Combine( comparer.GetHashCode(this.Item1), comparer.GetHashCode(this.Item2), comparer.GetHashCode(this.Item3), @@ -277,17 +296,170 @@ int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) ); } - public static implicit operator STuple(Tuple t) + [Pure] + public static implicit operator STuple([NotNull] Tuple t) { - if (t == null) throw new ArgumentNullException("t"); + Contract.NotNull(t, nameof(t)); return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } + [Pure, NotNull] public static explicit operator Tuple(STuple t) { return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + t.Item2 = this.Item2; + t.Item3 = this.Item3; + t.Item4 = this.Item4; + t.Item5 = this.Item5; + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [Pure] + public STuple Concat(ValueTuple tuple) + { + return new STuple(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, tuple.Item1); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple(STuple t) + { + return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, this.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, this.Item4) + && SimilarValueComparer.Default.Equals(this.Item5, this.Item5); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + && SimilarValueComparer.Default.Equals(left.Item5, right.Item5); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + && SimilarValueComparer.Default.Equals(left.Item5, right.Item5); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + || !SimilarValueComparer.Default.Equals(left.Item5, right.Item5); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + || !SimilarValueComparer.Default.Equals(left.Item5, right.Item5); + } + +#endif + + public sealed class Comparer : IComparer> + { + + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + private static readonly Comparer Comparer2 = Comparer.Default; + private static readonly Comparer Comparer3 = Comparer.Default; + private static readonly Comparer Comparer4 = Comparer.Default; + private static readonly Comparer Comparer5 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + int cmp = Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = Comparer2.Compare(x.Item2, y.Item2); + if (cmp == 0) cmp = Comparer3.Compare(x.Item3, y.Item3); + if (cmp == 0) cmp = Comparer4.Compare(x.Item4, y.Item4); + if (cmp == 0) cmp = Comparer5.Compare(x.Item5, y.Item5); + return cmp; + } + + } + + public sealed class EqualityComparer : IEqualityComparer> + { + + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer2 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer3 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer4 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer5 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1) + && Comparer2.Equals(x.Item2, y.Item2) + && Comparer3.Equals(x.Item3, y.Item3) + && Comparer4.Equals(x.Item4, y.Item4) + && Comparer5.Equals(x.Item5, y.Item5); + } + + public int GetHashCode(STuple obj) + { + return HashCodes.Combine( + Comparer1.GetHashCode(obj.Item1), + Comparer2.GetHashCode(obj.Item2), + Comparer3.GetHashCode(obj.Item3), + Comparer4.GetHashCode(obj.Item4), + Comparer5.GetHashCode(obj.Item5) + ); + } + } + } } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs new file mode 100644 index 000000000..751723789 --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -0,0 +1,484 @@ +#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. +// See License.MD for license information +#endregion + +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples +{ + using System; + using System.Collections; + using System.Collections.Generic; + using System.ComponentModel; + using System.Diagnostics; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Runtime.Converters; + using JetBrains.Annotations; + + /// Tuple that can hold five items + /// Type of the 1st item + /// Type of the 2nd item + /// Type of the 3rd item + /// Type of the 4th item + /// Type of the 5th item + /// Type of the 5th item + [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] + public struct STuple : ITuple, ITupleSerializable, IEquatable> +#if ENABLE_VALUETUPLES + , IEquatable> +#endif + { + // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. + // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost + + /// First element of the tuple + public readonly T1 Item1; + /// Second element of the tuple + public readonly T2 Item2; + /// Third element of the tuple + public readonly T3 Item3; + /// Fourth element of the tuple + public readonly T4 Item4; + /// Fifth of the tuple + public readonly T5 Item5; + /// Sixth and last element of the tuple + public readonly T6 Item6; + + /// Create a tuple containing for items + [DebuggerStepThrough] + public STuple(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + this.Item1 = item1; + this.Item2 = item2; + this.Item3 = item3; + this.Item4 = item4; + this.Item5 = item5; + this.Item6 = item6; + } + + /// Number of items in this tuple + public int Count => 6; + + /// Return the Nth item in this tuple + public object this[int index] + { + get + { + switch (index) + { + case 0: case -6: return this.Item1; + case 1: case -5: return this.Item2; + case 2: case -4: return this.Item3; + case 3: case -3: return this.Item4; + case 4: case -2: return this.Item5; + case 5: case -1: return this.Item6; + default: return TupleHelpers.FailIndexOutOfRange(index, 6); + } + } + } + + public ITuple this[int? fromIncluded, int? toExcluded] + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return TupleHelpers.Splice(this, fromIncluded, toExcluded); } + } + + /// Return the typed value of an item of the tuple, given its position + /// Expected type of the item + /// Position of the item (if negative, means relative from the end) + /// Value of the item at position , adapted into type . + public TItem Get(int index) + { + switch(index) + { + case 0: case -6: return TypeConverters.Convert(this.Item1); + case 1: case -5: return TypeConverters.Convert(this.Item2); + case 2: case -4: return TypeConverters.Convert(this.Item3); + case 3: case -3: return TypeConverters.Convert(this.Item4); + case 4: case -2: return TypeConverters.Convert(this.Item5); + case 5: case -1: return TypeConverters.Convert(this.Item6); + default: return TupleHelpers.FailIndexOutOfRange(index, 6); + } + } + + /// Return the value of the last item in the tuple + public T6 Last + { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Item6; } + } + + /// Return a tuple without the first item + public STuple Tail + { + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return new STuple(this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); } + } + + void ITupleSerializable.PackTo(ref TupleWriter writer) + { + PackTo(ref writer); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal void PackTo(ref TupleWriter writer) + { + TupleSerializer.Default.PackTo(ref writer, ref this); + } + + ITuple ITuple.Append(T7 value) + { + // the caller doesn't care about the return type, so just box everything into a list tuple + return new ListTuple(new object[7] { this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6, value }, 0, 7); + } + + /// Appends a single new item at the end of the current tuple. + /// Value that will be added as an embedded item + /// New tuple with one extra item + /// If is a tuple, and you want to append the *items* of this tuple, and not the tuple itself, please call ! + [NotNull, Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public LinkedTuple Append(T7 value) + { + // the caller probably cares about the return type, since it is using a struct, but whatever tuple type we use will end up boxing this tuple on the heap, and we will loose type information. + // but, by returning a LinkedTuple, the tuple will still remember the exact type, and efficiently serializer/convert the values (without having to guess the type) + return new LinkedTuple(this, value); + } + + /// Appends the items of a tuple at the end of the current tuple. + /// Tuple whose items are to be appended at the end + /// New tuple composed of the current tuple's items, followed by 's items + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ITuple Concat(ITuple tuple) + { + return STuple.Concat(this, tuple); + } + + /// Copy all the items of this tuple into an array at the specified offset + public void CopyTo(object[] array, int offset) + { + array[offset] = this.Item1; + array[offset + 1] = this.Item2; + array[offset + 2] = this.Item3; + array[offset + 3] = this.Item4; + array[offset + 4] = this.Item5; + array[offset + 5] = this.Item6; + } + + [EditorBrowsable(EditorBrowsableState.Never)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Deconstruct(out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5, out T6 item6) + { + item1 = this.Item1; + item2 = this.Item2; + item3 = this.Item3; + item4 = this.Item4; + item5 = this.Item5; + item6 = this.Item6; + } + + /// Execute a lambda Action with the content of this tuple + /// Action that will be passed the content of this tuple as parameters + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void With([NotNull] Action lambda) + { + lambda(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); + } + + /// Execute a lambda Function with the content of this tuple + /// Action that will be passed the content of this tuple as parameters + /// Result of calling with the items of this tuple + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public TItem With([NotNull] Func lambda) + { + return lambda(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); + } + + public IEnumerator GetEnumerator() + { + yield return this.Item1; + yield return this.Item2; + yield return this.Item3; + yield return this.Item4; + yield return this.Item5; + yield return this.Item6; + } + + System.Collections.IEnumerator System.Collections.IEnumerable.GetEnumerator() + { + return this.GetEnumerator(); + } + + public override string ToString() + { + return string.Join("", new[] + { + "(", + STuple.Formatter.Stringify(this.Item1), ", ", + STuple.Formatter.Stringify(this.Item2), ", ", + STuple.Formatter.Stringify(this.Item3), ", ", + STuple.Formatter.Stringify(this.Item4), ", ", + STuple.Formatter.Stringify(this.Item5), ", ", + STuple.Formatter.Stringify(this.Item6), + ")" + }); + } + + public override bool Equals(object obj) + { + return obj != null && ((IStructuralEquatable)this).Equals(obj, SimilarValueComparer.Default); + } + + public bool Equals(ITuple other) + { + return other != null && ((IStructuralEquatable)this).Equals(other, SimilarValueComparer.Default); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(STuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, other.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, other.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, other.Item4) + && SimilarValueComparer.Default.Equals(this.Item5, other.Item5) + && SimilarValueComparer.Default.Equals(this.Item6, other.Item6); + } + + public override int GetHashCode() + { + return ((IStructuralEquatable)this).GetHashCode(SimilarValueComparer.Default); + } + + public static bool operator ==(STuple left, STuple right) + { + var comparer = SimilarValueComparer.Default; + return comparer.Equals(left.Item1, right.Item1) + && comparer.Equals(left.Item2, right.Item2) + && comparer.Equals(left.Item3, right.Item3) + && comparer.Equals(left.Item4, right.Item4) + && comparer.Equals(left.Item5, right.Item5) + && comparer.Equals(left.Item6, right.Item6); + } + + public static bool operator !=(STuple left, STuple right) + { + var comparer = SimilarValueComparer.Default; + return !comparer.Equals(left.Item1, right.Item1) + || !comparer.Equals(left.Item2, right.Item2) + || !comparer.Equals(left.Item3, right.Item3) + || !comparer.Equals(left.Item4, right.Item4) + || !comparer.Equals(left.Item5, right.Item5) + || !comparer.Equals(left.Item6, right.Item6); + } + + bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) + { + if (other == null) return false; + if (other is STuple stuple) + { + return comparer.Equals(this.Item1, stuple.Item1) + && comparer.Equals(this.Item2, stuple.Item2) + && comparer.Equals(this.Item3, stuple.Item3) + && comparer.Equals(this.Item4, stuple.Item4) + && comparer.Equals(this.Item5, stuple.Item5) + && comparer.Equals(this.Item6, stuple.Item6); + } +#if ENABLE_VALUETUPLES + if (other is ValueTuple vtuple) + { + return comparer.Equals(this.Item1, vtuple.Item1) + && comparer.Equals(this.Item2, vtuple.Item2) + && comparer.Equals(this.Item3, vtuple.Item3) + && comparer.Equals(this.Item4, vtuple.Item4) + && comparer.Equals(this.Item5, vtuple.Item5) + && comparer.Equals(this.Item6, vtuple.Item6); + } +#endif + return TupleHelpers.Equals(this, other, comparer); + } + + int IStructuralEquatable.GetHashCode(IEqualityComparer comparer) + { + return HashCodes.Combine( + comparer.GetHashCode(this.Item1), + comparer.GetHashCode(this.Item2), + comparer.GetHashCode(this.Item3), + comparer.GetHashCode(this.Item4), + comparer.GetHashCode(this.Item5), + comparer.GetHashCode(this.Item6) + ); + } + + [Pure] + public static implicit operator STuple([NotNull] Tuple tuple) + { + Contract.NotNull(tuple, nameof(tuple)); + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); + } + + [Pure, NotNull] + public static explicit operator Tuple(STuple tuple) + { + return new Tuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); + } + +#if ENABLE_VALUETUPLES + + // interop with System.ValueTuple + + public void Fill(ref ValueTuple t) + { + t.Item1 = this.Item1; + t.Item2 = this.Item2; + t.Item3 = this.Item3; + t.Item4 = this.Item4; + t.Item5 = this.Item5; + t.Item6 = this.Item6; + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public ValueTuple ToValueTuple() + { + return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator STuple(ValueTuple t) + { + return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5, t.Item6); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ValueTuple(STuple t) + { + return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5, t.Item6); + } + + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + bool IEquatable>.Equals(ValueTuple other) + { + return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) + && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) + && SimilarValueComparer.Default.Equals(this.Item3, this.Item3) + && SimilarValueComparer.Default.Equals(this.Item4, this.Item4) + && SimilarValueComparer.Default.Equals(this.Item5, this.Item5) + && SimilarValueComparer.Default.Equals(this.Item6, this.Item6); + } + + public static bool operator ==(STuple left, ValueTuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + && SimilarValueComparer.Default.Equals(left.Item5, right.Item5) + && SimilarValueComparer.Default.Equals(left.Item6, right.Item6); + } + + public static bool operator ==(ValueTuple left, STuple right) + { + return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + && SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + && SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + && SimilarValueComparer.Default.Equals(left.Item5, right.Item5) + && SimilarValueComparer.Default.Equals(left.Item6, right.Item6); + } + + public static bool operator !=(STuple left, ValueTuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + || !SimilarValueComparer.Default.Equals(left.Item5, right.Item5) + || !SimilarValueComparer.Default.Equals(left.Item6, right.Item6); + } + + public static bool operator !=(ValueTuple left, STuple right) + { + return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) + || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) + || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3) + || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4) + || !SimilarValueComparer.Default.Equals(left.Item5, right.Item5) + || !SimilarValueComparer.Default.Equals(left.Item6, right.Item6); + } + +#endif + + public sealed class Comparer : IComparer> + { + + public static Comparer Default { [NotNull] get; } = new Comparer(); + + private static readonly Comparer Comparer1 = Comparer.Default; + private static readonly Comparer Comparer2 = Comparer.Default; + private static readonly Comparer Comparer3 = Comparer.Default; + private static readonly Comparer Comparer4 = Comparer.Default; + private static readonly Comparer Comparer5 = Comparer.Default; + private static readonly Comparer Comparer6 = Comparer.Default; + + private Comparer() { } + + public int Compare(STuple x, STuple y) + { + int cmp = Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = Comparer2.Compare(x.Item2, y.Item2); + if (cmp == 0) cmp = Comparer3.Compare(x.Item3, y.Item3); + if (cmp == 0) cmp = Comparer4.Compare(x.Item4, y.Item4); + if (cmp == 0) cmp = Comparer5.Compare(x.Item5, y.Item5); + if (cmp == 0) cmp = Comparer6.Compare(x.Item6, y.Item6); + return cmp; + } + + } + + public sealed class EqualityComparer : IEqualityComparer> + { + + public static EqualityComparer Default { [NotNull] get; } = new EqualityComparer(); + + private static readonly EqualityComparer Comparer1 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer2 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer3 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer4 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer5 = EqualityComparer.Default; + private static readonly EqualityComparer Comparer6 = EqualityComparer.Default; + + private EqualityComparer() { } + + public bool Equals(STuple x, STuple y) + { + return Comparer1.Equals(x.Item1, y.Item1) + && Comparer2.Equals(x.Item2, y.Item2) + && Comparer3.Equals(x.Item3, y.Item3) + && Comparer4.Equals(x.Item4, y.Item4) + && Comparer5.Equals(x.Item5, y.Item5) + && Comparer6.Equals(x.Item6, y.Item6); + } + + public int GetHashCode(STuple obj) + { + return HashCodes.Combine( + Comparer1.GetHashCode(obj.Item1), + Comparer2.GetHashCode(obj.Item2), + Comparer3.GetHashCode(obj.Item3), + Comparer4.GetHashCode(obj.Item4), + Comparer5.GetHashCode(obj.Item5), + Comparer6.GetHashCode(obj.Item6) + ); + } + } + + } +} diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs new file mode 100644 index 000000000..26fe00069 --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -0,0 +1,964 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples +{ + using System; + using System.Collections.Generic; + using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Memory; + using FoundationDB; + using JetBrains.Annotations; + + /// Tuple Binary Encoding + public static class TuPack + { + + #region Packing... + + // Without prefix + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack([CanBeNull] TTuple tuple) + where TTuple : ITuple + { + return TupleEncoder.Pack(tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(STuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, ref tuple); + } + +#if ENABLE_VALUETUPLES + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(ValueTuple tuple) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuple.ToSTuple()); + } + +#endif + + /// Pack an array of N-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params ITuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 1-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 2-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 3-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 4-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 5-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack an array of 6-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull] params STuple[] tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Pack a sequence of N-tuples, all sharing the same buffer + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack([ ("Foo", 1), ("Foo", 2) ]) => [ "\x02Foo\x00\x15\x01", "\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples([NotNull, InstantHandle] this IEnumerable tuples) + { + var empty = default(Slice); + return TupleEncoder.Pack(empty, tuples); + } + + /// Efficiently write the packed representation of a tuple + /// Output buffer + /// Tuple that must be serialized into a binary slice + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void PackTo(ref SliceWriter writer, [CanBeNull] TTuple tuple) + where TTuple : ITuple + { + TupleEncoder.PackTo(ref writer, tuple); + } + + // With prefix + + /// Efficiently concatenate a prefix with the packed representation of a tuple + /// Prefix added to the start of the packed slice + /// Pack a tuple into a slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, [CanBeNull] TTuple tuple) + where TTuple : ITuple + { + return TupleEncoder.Pack(prefix, tuple); + } + + /// Pack a tuple into a slice + /// Prefix added to the start of the packed slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack a tuple into a slice + /// Prefix added to the start of the packed slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack a tuple into a slice + /// Prefix added to the start of the packed slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack a tuple into a slice + /// Prefix added to the start of the packed slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack a tuple into a slice + /// Prefix added to the start of the packed slice + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack a tuple into a slice + /// Common prefix added to all the tuples + /// Tuple that must be serialized into a binary slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice Pack(Slice prefix, STuple tuple) + { + return TupleEncoder.Pack(prefix, ref tuple); + } + + /// Pack an array of N-tuples, all sharing the same buffer + /// Common prefix added to all the tuples + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples(Slice prefix, [NotNull] params ITuple[] tuples) + { + return TupleEncoder.Pack(prefix, tuples); + } + + /// Pack a sequence of N-tuples, all sharing the same buffer + /// Common prefix added to all the tuples + /// Sequence of N-tuples to pack + /// Array containing the buffer segment of each packed tuple + /// BatchPack("abc", [ ("Foo", 1), ("Foo", 2) ]) => [ "abc\x02Foo\x00\x15\x01", "abc\x02Foo\x00\x15\x02" ] + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples(Slice prefix, [NotNull] IEnumerable tuples) + { + return TupleEncoder.Pack(prefix, tuples); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples(Slice prefix, [NotNull] TElement[] elements, Func transform) + where TTuple : ITuple + { + return TupleEncoder.Pack(prefix, elements, transform); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] PackTuples(Slice prefix, [NotNull] IEnumerable elements, Func transform) + where TTuple : ITuple + { + return TupleEncoder.Pack(prefix, elements, transform); + } + + #endregion + + #region Encode + + //REVIEW: EncodeKey/EncodeKeys? Encode/EncodeRange? EncodeValues? EncodeItems? + + /// Pack a 1-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1) + { + return TupleEncoder.EncodeKey(item1); + } + + /// Pack a 2-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2) + { + return TupleEncoder.EncodeKey(item1, item2); + } + + /// Pack a 3-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) + { + return TupleEncoder.EncodeKey(item1, item2, item3); + } + + /// Pack a 4-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) + { + return TupleEncoder.EncodeKey(item1, item2, item3, item4); + } + + /// Pack a 5-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5); + } + + /// Pack a 6-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6); + } + + /// Pack a 6-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6, item7); + } + + /// Pack a 6-tuple directly into a slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6, item7, item8); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodeKeys([NotNull] IEnumerable keys) + { + var empty = default(Slice); + return TupleEncoder.EncodePrefixedKeys(empty, keys); + } + + /// Merge a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) + { + return TupleEncoder.EncodePrefixedKeys(prefix, keys); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodeKeys([NotNull] params T[] keys) + { + var empty = default(Slice); + return TupleEncoder.EncodePrefixedKeys(empty, keys); + } + + /// Merge an array of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) + { + return TupleEncoder.EncodePrefixedKeys(prefix, keys); + } + + /// Merge an array of elements, all sharing the same buffer + /// Type of the elements + /// Type of the keys extracted from the elements + /// Sequence of elements to pack + /// Lambda that extract the key from each element + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] Func selector) + { + var empty = default(Slice); + return TupleEncoder.EncodePrefixedKeys(empty, elements, selector); + } + + /// Merge an array of elements with a same prefix, all sharing the same buffer + /// Type of the elements + /// Type of the keys extracted from the elements + /// Prefix shared by all keys (can be empty) + /// Sequence of elements to pack + /// Lambda that extract the key from each element + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) + { + return TupleEncoder.EncodePrefixedKeys(prefix, elements, selector); + } + + /// Pack a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] IEnumerable keys) + { + Contract.NotNull(prefix, nameof(prefix)); + + return EncodePrefixedKeys(Pack(prefix), keys); + } + + /// Pack a sequence of keys with a same prefix, all sharing the same buffer + /// Type of the keys + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] params T[] keys) + { + Contract.NotNull(prefix, nameof(prefix)); + + return EncodePrefixedKeys(Pack(prefix), keys); + } + + #endregion + + #region Ranges... + + /// Create a range that selects all tuples that are stored under the specified subspace: 'prefix\x00' <= k < 'prefix\xFF' + /// Subspace binary prefix (that will be excluded from the range) + /// Range including all possible tuples starting with the specified prefix. + /// FdbTuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) + [Pure] + public static KeyRange ToRange(Slice prefix) + { + if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); + //note: there is no guarantee that prefix is a valid packed tuple (could be any exotic binary prefix) + + // prefix => [ prefix."\0", prefix."\xFF" ) + return new KeyRange( + prefix + 0x00, + prefix + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange([NotNull] TTuple tuple) + where TTuple : ITuple + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(Slice.FromInt32(42), FdbTuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. + /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. + [Pure] + public static KeyRange ToRange(Slice prefix, [NotNull] TTuple tuple) + where TTuple : ITuple + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(Slice prefix, STuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.Pack(prefix, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + #endregion + + #region Unpacking... + + /// Unpack a tuple from a serialied key blob + /// Binary key containing a previously packed tuple + /// Unpacked tuple, or the empty tuple if the key is + /// If is equal to + [Pure, NotNull] + public static ITuple Unpack(Slice packedKey) + { + if (packedKey.IsNull) throw new ArgumentNullException(nameof(packedKey), "Cannot unpack tuple from Nil"); + if (packedKey.Count == 0) return STuple.Empty; + + return TuplePackers.Unpack(packedKey, embedded: false); + } + + /// Unpack a tuple from a binary representation + /// Binary key containing a previously packed tuple, or Slice.Nil + /// Unpacked tuple, the empty tuple if is equal to , or null if the key is + [Pure, CanBeNull] + public static ITuple UnpackOrDefault(Slice packedKey) + { + if (packedKey.IsNull) return null; + if (packedKey.Count == 0) return STuple.Empty; + return TuplePackers.Unpack(packedKey, embedded: false); + } + + /// Unpack a tuple and only return its first element + /// Type of the first value in the decoded tuple + /// Slice that should be entirely parsable as a tuple + /// Decoded value of the first item in the tuple + [Pure] + public static T DecodeFirst(Slice packedKey) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the first element of an empty tuple"); + + var slice = TuplePackers.UnpackFirst(packedKey); + if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); + + return TuplePacker.Deserialize(slice); + } + + /// Unpack a tuple and only return its last element + /// Type of the last value in the decoded tuple + /// Slice that should be entirely parsable as a tuple + /// Decoded value of the last item in the tuple + [Pure] + public static T DecodeLast(Slice packedKey) + { + if (packedKey.IsNullOrEmpty) throw new InvalidOperationException("Cannot unpack the last element of an empty tuple"); + + var slice = TuplePackers.UnpackLast(packedKey); + if (slice.IsNull) throw new InvalidOperationException("Failed to unpack tuple"); + + return TuplePacker.Deserialize(slice); + } + + /// Unpack the value of a singleton tuple + /// Type of the single value in the decoded tuple + /// Slice that should contain the packed representation of a tuple with a single element + /// Decoded value of the only item in the tuple. Throws an exception if the tuple is empty of has more than one element. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static T1 DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple.Item1; + } + + /// Unpack a key containing two elements + /// Slice that should contain the packed representation of a tuple with two elements + /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static STuple DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple; + } + + /// Unpack a key containing three elements + /// Slice that should contain the packed representation of a tuple with three elements + /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static STuple DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple; + } + + /// Unpack a key containing four elements + /// Slice that should contain the packed representation of a tuple with four elements + /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static STuple DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple; + } + + /// Unpack a key containing five elements + /// Slice that should contain the packed representation of a tuple with five elements + /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static STuple DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple; + } + + /// Unpack a key containing six elements + /// Slice that should contain the packed representation of a tuple with six elements + /// Decoded value of the elements int the tuple. Throws an exception if the tuple is empty of has more than elements. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static STuple DecodeKey(Slice packedKey) + { + TupleEncoder.DecodeKey(packedKey, out STuple tuple); + return tuple; + } + + /// Unpack the next item in the tuple, and advance the cursor + /// Type of the next value in the tuple + /// Reader positionned at the start of the next item to read + /// If decoding succeedsd, receives the decoded value. + /// True if the decoded succeeded (and receives the decoded value). False if the tuple has reached the end. + public static bool DecodeNext(ref TupleReader input, out T value) + { + if (!input.Input.HasMore) + { + value = default(T); + return false; + } + + var slice = TupleParser.ParseNext(ref input); + value = TuplePacker.Deserialize(slice); + return true; + } + + #endregion + + #region EncodePrefixedKey... + + //note: they are equivalent to the Pack<...>() methods, they only take a binary prefix + + /// Efficiently concatenate a prefix with the packed representation of a 1-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value) + { + return TupleEncoder.EncodePrefixedKey(prefix, value); + } + + /// Efficiently concatenate a prefix with the packed representation of a 2-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2); + } + + /// Efficiently concatenate a prefix with the packed representation of a 3-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3); + } + + /// Efficiently concatenate a prefix with the packed representation of a 4-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4); + } + + /// Efficiently concatenate a prefix with the packed representation of a 5-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5); + } + + /// Efficiently concatenate a prefix with the packed representation of a 6-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6); + } + + /// Efficiently concatenate a prefix with the packed representation of a 7-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6, value7); + } + + /// Efficiently concatenate a prefix with the packed representation of a 8-tuple + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) + { + return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6, value7, value8); + } + + #endregion + + } + +} diff --git a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs index cb3559d1d..535b390de 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs @@ -26,14 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +namespace Doxense.Collections.Tuples { - using FoundationDB.Client.Converters; - using JetBrains.Annotations; using System; using System.Collections; using System.Collections.Generic; using System.Runtime.CompilerServices; + using Doxense.Runtime.Converters; + using JetBrains.Annotations; /// Helper class for tuple comparisons public static class TupleComparisons @@ -44,8 +44,10 @@ public static class TupleComparisons /// Tuple comparer that uses the default BCL object comparison ("123" != 123 != 123L != 123.0d) public static readonly EqualityComparer Bcl = new EqualityComparer(EqualityComparer.Default); +#if false /// Tuple comparer that compared the packed bytes (slow!) public static readonly BinaryComparer Binary = new BinaryComparer(); +#endif public sealed class EqualityComparer : IEqualityComparer, IEqualityComparer { @@ -66,7 +68,7 @@ public bool Equals(ITuple x, ITuple y) public int GetHashCode(ITuple obj) { - return obj != null ? obj.GetHashCode(m_comparer) : 0; + return HashCodes.Compute(obj, m_comparer); } public new bool Equals(object x, object y) @@ -74,8 +76,7 @@ public int GetHashCode(ITuple obj) if (object.ReferenceEquals(x, y)) return true; if (x == null || y == null) return false; - var t = x as ITuple; - if (t != null) return t.Equals(y, m_comparer); + if (x is ITuple t) return t.Equals(y, m_comparer); t = y as ITuple; if (t != null) return t.Equals(x, m_comparer); @@ -94,7 +95,9 @@ public int GetHashCode(object obj) return RuntimeHelpers.GetHashCode(obj); } } - + +#if false + public sealed class BinaryComparer : IEqualityComparer, IEqualityComparer { internal BinaryComparer() @@ -136,6 +139,8 @@ public int GetHashCode(object obj) } } +#endif + /// Create a new instance that compares a single item position in two tuples /// Type of the item to compare /// Offset of the item to compare (can be negative) @@ -203,10 +208,10 @@ public CompositeComparer(int offset, IComparer comparer) /// Offset in the tuples where the comparison starts /// If negative, comparison starts from the end. - public int Offset { get; private set; } + public int Offset { get; } /// Comparer for the first element (at possition ) - public IComparer Comparer { get; private set; } + public IComparer Comparer { get; } /// Compare a single item in both tuples /// First tuple @@ -258,13 +263,13 @@ public CompositeComparer(int offset, IComparer comparer1, IComparer comp /// Offset in the tuples where the comparison starts /// If negative, comparison starts from the end. - public int Offset { get; private set; } + public int Offset { get; } /// Comparer for the first element (at possition ) - public IComparer Comparer1 { get; private set; } + public IComparer Comparer1 { get; } /// Comparer for the second element (at possition + 1) - public IComparer Comparer2 { get; private set; } + public IComparer Comparer2 { get; } /// Compare up to two items in both tuples /// First tuple @@ -326,16 +331,16 @@ public CompositeComparer(int offset, IComparer comparer1, IComparer comp /// Offset in the tuples where the comparison starts /// If negative, comparison starts from the end. - public int Offset { get; private set; } + public int Offset { get; } /// Comparer for the first element (at possition ) - public IComparer Comparer1 { get; private set; } + public IComparer Comparer1 { get; } /// Comparer for the second element (at possition + 1) - public IComparer Comparer2 { get; private set; } + public IComparer Comparer2 { get; } /// Comparer for the third element (at possition + 2) - public IComparer Comparer3 { get; private set; } + public IComparer Comparer3 { get; } /// Compare up to three items in both tuples /// First tuple diff --git a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs index 8a3e8797c..2e19f7fcf 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs @@ -26,11 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Layers.Tuples +//#define ENABLE_VALUETUPLES + +namespace Doxense.Collections.Tuples { using System; using System.Collections.Generic; - using FoundationDB.Client; + using System.ComponentModel; + using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; /// Add extensions methods that deal with tuples on various types @@ -41,6 +45,7 @@ public static class TupleExtensions /// Returns true if the tuple is either null or empty [ContractAnnotation("null => true")] + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool IsNullOrEmpty(this ITuple tuple) { return tuple == null || tuple.Count == 0; @@ -57,7 +62,7 @@ public static bool IsSingleton(this ITuple tuple) [NotNull, ItemCanBeNull] public static object[] ToArray([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); var items = new object[tuple.Count]; if (items.Length > 0) @@ -71,7 +76,7 @@ public static object[] ToArray([NotNull] this ITuple tuple) [NotNull] public static T[] ToArray([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); var items = new T[tuple.Count]; if (items.Length > 0) @@ -84,35 +89,40 @@ public static T[] ToArray([NotNull] this ITuple tuple) return items; } - /// Returns a byte array containing the packed version of a tuple - [CanBeNull] - public static byte[] GetBytes([NotNull] this ITuple tuple) - { - return tuple.ToSlice().GetBytes(); - } - /// Returns the typed value of the first item in this tuple /// Expected type of the first item /// Value of the first item, adapted into type . + [Pure] + [ContractAnnotation("null => true")] public static T First([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); return tuple.Get(0); } + /// Return the typed value of the last item in the tuple + /// Expected type of the item + /// Value of the last item of this tuple, adapted into type + /// Equivalent of tuple.Get<T>(-1) + [Pure] + [ContractAnnotation("null => true")] + public static T Last([NotNull] this ITuple tuple) + { + return tuple.Get(-1); + } + /// Appends two values at the end of a tuple [NotNull] public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - return new JoinedTuple(tuple, STuple.Create(value1, value2)); + Contract.NotNull(tuple, nameof(tuple)); + return new JoinedTuple(tuple, STuple.Create(value1, value2)); } /// Appends three values at the end of a tuple [NotNull] public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2, T3 value3) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); return new JoinedTuple(tuple, STuple.Create(value1, value2, value3)); } @@ -120,86 +130,10 @@ public static ITuple Append([NotNull] this ITuple tuple, T1 value1, [NotNull] public static ITuple Append([NotNull] this ITuple tuple, T1 value1, T2 value2, T3 value3, T4 value4) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); return new JoinedTuple(tuple, STuple.Create(value1, value2, value3, value4)); } - /// Creates a key range containing all children of this tuple, from tuple.pack()+'\0' to tuple.pack()+'\xFF' - /// Tuple that is the suffix of all keys - /// Range of all keys suffixed by the tuple. The tuple itself will not be included - public static KeyRange ToRange([NotNull] this ITuple tuple) - { - return ToRange(tuple, false); - } - - /// Creates a key range containing all children of tuple, optionally including the tuple itself. - /// Tuple that is the prefix of all keys - /// If true, the tuple key itself is included, if false only the children keys are included - /// Range of all keys suffixed by the tuple. The tuple itself will be included if is true - public static KeyRange ToRange([NotNull] this ITuple tuple, bool includePrefix) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - - // We want to allocate only one byte[] to store both keys, and map both Slice to each chunk - // So we will serialize the tuple two times in the same writer - - var writer = new TupleWriter(); - - tuple.PackTo(ref writer); - writer.Output.EnsureBytes(writer.Output.Position + 2); - if (!includePrefix) writer.Output.WriteByte(0); - int p0 = writer.Output.Position; - - tuple.PackTo(ref writer); - writer.Output.WriteByte(0xFF); - int p1 = writer.Output.Position; - - return new KeyRange( - new Slice(writer.Output.Buffer, 0, p0), - new Slice(writer.Output.Buffer, p0, p1 - p0) - ); - } - - /// Creates pre-packed and isolated copy of this tuple - /// - /// Create a copy of the tuple that can be reused frequently to pack values - /// If the tuple is already memoized, the current instance will be returned - [CanBeNull, ContractAnnotation("null => null")] - public static MemoizedTuple Memoize(this ITuple tuple) - { - if (tuple == null) return null; - - var memoized = tuple as MemoizedTuple ?? new MemoizedTuple(tuple.ToArray(), tuple.ToSlice()); - - return memoized; - } - - /// Unpack a tuple from this slice - /// - /// Unpacked tuple if the slice contains data, STuple.Empty if the slice is empty, or null if the slice is Slice.Nil - [NotNull] - public static ITuple ToTuple(this Slice slice) - { - //note: this method is here to allow a fluent API with method chaining, like "something.ToFoundationDbKey().ToTuple().With((int x, int y) => .....)" - return STuple.Unpack(slice); - } - - /// Unpack a tuple from this slice - /// - /// Unpacked tuple if the slice contains data, STuple.Empty if the slice is empty, or null if the slice is Slice.Nil - [CanBeNull] - public static ITuple ToTupleOrDefault(this Slice slice) - { - //note: this method is here to allow a fluent API with method chaining, like "something.ToFoundationDbKey().ToTuple().With((int x, int y) => .....)" - - if (slice.IsNullOrEmpty) - { - return slice.HasValue ? STuple.Empty : null; - } - - return STuple.Unpack(slice); - } - /// Returns a substring of the current tuple /// Current tuple /// Offset from the start of the current tuple (negative value means from the end) @@ -207,7 +141,7 @@ public static ITuple ToTupleOrDefault(this Slice slice) [NotNull] public static ITuple Substring([NotNull] this ITuple tuple, int offset) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); return tuple[offset, null]; } @@ -220,8 +154,8 @@ public static ITuple Substring([NotNull] this ITuple tuple, int offset) [NotNull] public static ITuple Substring([NotNull] this ITuple tuple, int offset, int count) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative."); + Contract.NotNull(tuple, nameof(tuple)); + Contract.Positive(count, nameof(count)); if (count == 0) return STuple.Empty; @@ -257,11 +191,11 @@ public static ITuple Truncate([NotNull] this ITuple tuple, int count) /// True if the beginning of is equal to or if both tuples are identical public static bool StartsWith([NotNull] this ITuple left, [NotNull] ITuple right) { - if (left == null) throw new ArgumentNullException(nameof(left)); - if (right == null) throw new ArgumentNullException(nameof(right)); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); //REVIEW: move this on ITuple interface ? - return STuple.StartsWith(left, right); + return TupleHelpers.StartsWith(left, right); } /// Test if the end of current tuple is equal to another tuple @@ -270,11 +204,11 @@ public static bool StartsWith([NotNull] this ITuple left, [NotNull] ITuple right /// True if the end of is equal to or if both tuples are identical public static bool EndsWith([NotNull] this ITuple left, [NotNull] ITuple right) { - if (left == null) throw new ArgumentNullException(nameof(left)); - if (right == null) throw new ArgumentNullException(nameof(right)); + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); //REVIEW: move this on ITuple interface ? - return STuple.EndsWith(left, right); + return TupleHelpers.EndsWith(left, right); } /// Transform a tuple of N elements into a list of N singletons @@ -283,7 +217,7 @@ public static bool EndsWith([NotNull] this ITuple left, [NotNull] ITuple right) /// (123, ABC, false,).Explode() => [ (123,), (ABC,), (false,) ] public static IEnumerable Explode([NotNull] this ITuple tuple) { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); + Contract.NotNull(tuple, nameof(tuple)); int p = 0; int n = tuple.Count; @@ -294,22 +228,6 @@ public static IEnumerable Explode([NotNull] this ITuple tuple) } } - /// Returns a key that is immediately after the packed representation of this tuple - /// This is the equivalent of manually packing the tuple and incrementing the resulting slice - public static Slice Increment([NotNull] this ITuple tuple) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - return FdbKey.Increment(tuple.ToSlice()); - } - - /// Returns a Key Selector pair that defines the range of all items contained under this tuple - public static KeySelectorPair ToSelectorPair([NotNull] this ITuple tuple) - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - - return KeySelectorPair.StartsWith(tuple.ToSlice()); - } - /// Verify that this tuple has the expected size /// Tuple which must be of a specific size /// Expected number of items in this tuple @@ -355,18 +273,29 @@ public static ITuple OfSizeAtMost(this ITuple tuple, int size) [ContractAnnotation("=> halt")] internal static void ThrowInvalidTupleSize(ITuple tuple, int expected, int test) { - if (tuple == null) - { - throw new ArgumentNullException(nameof(tuple)); - } + Contract.NotNull(tuple, nameof(tuple)); switch(test) { - case 1: throw new InvalidOperationException(String.Format("This operation requires a tuple of size {0} or less, but this tuple has {1} elements", expected, tuple.Count)); - case -1: throw new InvalidOperationException(String.Format("This operation requires a tuple of size {0} or more, but this tuple has {1} elements", expected, tuple.Count)); - default: throw new InvalidOperationException(String.Format("This operation requires a tuple of size {0}, but this tuple has {1} elements", expected, tuple.Count)); + case 1: throw new InvalidOperationException($"This operation requires a tuple of size {expected} or less, but this tuple has {tuple.Count} elements"); + case -1: throw new InvalidOperationException($"This operation requires a tuple of size {expected} or more, but this tuple has {tuple.Count} elements"); + default: throw new InvalidOperationException($"This operation requires a tuple of size {expected}, but this tuple has {tuple.Count} elements"); } } + /// Creates pre-packed and isolated copy of this tuple + /// + /// Create a copy of the tuple that can be reused frequently to pack values + /// If the tuple is already memoized, the current instance will be returned + [CanBeNull, ContractAnnotation("null => null")] + public static MemoizedTuple Memoize(this ITuple tuple) + { + if (tuple == null) return null; + + var memoized = tuple as MemoizedTuple ?? new MemoizedTuple(tuple.ToArray(), TuPack.Pack(tuple)); + + return memoized; + } + /// Returns a typed version of a tuple of size 1 /// Expected type of the single element /// Tuple that must be of size 1 @@ -387,7 +316,7 @@ public static STuple As([NotNull] this ITuple tuple) tuple.OfSize(2); return new STuple( tuple.Get(0), - tuple.Get(1) + tuple.Get(1) ); } @@ -403,7 +332,7 @@ public static STuple As([NotNull] this ITuple tuple) return new STuple( tuple.Get(0), tuple.Get(1), - tuple.Get(2) + tuple.Get(2) ); } @@ -421,7 +350,7 @@ public static STuple As([NotNull] this ITuple tu tuple.Get(0), tuple.Get(1), tuple.Get(2), - tuple.Get(3) + tuple.Get(3) ); } @@ -441,7 +370,29 @@ public static STuple As([NotNull] this I tuple.Get(1), tuple.Get(2), tuple.Get(3), - tuple.Get(4) + tuple.Get(4) + ); + } + + /// Returns a typed version of a tuple of size 5 + /// Expected type of the first element + /// Expected type of the second element + /// Expected type of the third element + /// Expected type of the fourth element + /// Expected type of the fifth element + /// Expected type of the sixth element + /// Tuple that must be of size 5 + /// Equivalent tuple, with its elements converted to the specified types + public static STuple As([NotNull] this ITuple tuple) + { + tuple.OfSize(6); + return new STuple( + tuple.Get(0), + tuple.Get(1), + tuple.Get(2), + tuple.Get(3), + tuple.Get(4), + tuple.Get(5) ); } @@ -532,8 +483,7 @@ public static void With([NotNull] this ITuple tu /// If has not the expected size public static TResult With([NotNull] this ITuple tuple, [NotNull] Func lambda) { - OfSize(tuple, 1); - return lambda(tuple.Get(0)); + return lambda(tuple.OfSize(1).Get(0)); } /// Execute a lambda Function with the content of this tuple @@ -615,6 +565,143 @@ public static TResult With([NotNull] th #endregion + #region Deconstruction (C#7) + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1) + { + item1 = value.OfSize(1).Get(0); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2) + { + value.OfSize(2); + item1 = value.Get(0); + item2 = value.Get(1); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3) + { + value.OfSize(3); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3, out T4 item4) + { + value.OfSize(4); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + item4 = value.Get(3); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5) + { + value.OfSize(5); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + item4 = value.Get(3); + item5 = value.Get(4); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5, out T6 item6) + { + value.OfSize(6); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + item4 = value.Get(3); + item5 = value.Get(4); + item6 = value.Get(5); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5, out T6 item6, out T7 item7) + { + value.OfSize(7); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + item4 = value.Get(3); + item5 = value.Get(4); + item6 = value.Get(5); + item7 = value.Get(6); + } + + [EditorBrowsable(EditorBrowsableState.Never)] + public static void Deconstruct(this ITuple value, out T1 item1, out T2 item2, out T3 item3, out T4 item4, out T5 item5, out T6 item6, out T7 item7, out T8 item8) + { + value.OfSize(8); + item1 = value.Get(0); + item2 = value.Get(1); + item3 = value.Get(2); + item4 = value.Get(3); + item5 = value.Get(4); + item6 = value.Get(5); + item7 = value.Get(6); + item8 = value.Get(7); + } + + #endregion + + #region ValueTuple (C#7) + +#if ENABLE_VALUETUPLES + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return default(STuple); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1, tuple.Item2); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); + } + + [Pure] + public static STuple ToSTuple(this ValueTuple tuple) + { + return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); + } + +#endif + + #endregion + } } diff --git a/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs b/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs new file mode 100644 index 000000000..118e1d61f --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs @@ -0,0 +1,235 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Collections.Tuples +{ + using System; + using System.Collections; + using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + + public static class TupleHelpers + { + + /// Default (non-optimized) implementation of ITuple.this[long?, long?] + /// Tuple to slice + /// Start offset of the section (included) + /// End offset of the section (included) + /// New tuple only containing items inside this section + [NotNull] + public static ITuple Splice([NotNull] ITuple tuple, int? fromIncluded, int? toExcluded) + { + Contract.Requires(tuple != null); + int count = tuple.Count; + if (count == 0) return STuple.Empty; + + int start = fromIncluded.HasValue ? MapIndexBounded(fromIncluded.Value, count) : 0; + int end = toExcluded.HasValue ? MapIndexBounded(toExcluded.Value, count) : count; + + int len = end - start; + + if (len <= 0) return STuple.Empty; + if (start == 0 && len == count) return tuple; + switch (len) + { + case 1: + return new ListTuple(new[] { tuple[start] }, 0, 1); + case 2: + return new ListTuple(new[] { tuple[start], tuple[start + 1] }, 0, 2); + default: + { + var items = new object[len]; + //note: can be slow for tuples using linked-lists, but hopefully they will have their own Slice implementation... + int q = start; + for (int p = 0; p < items.Length; p++) + { + items[p] = tuple[q++]; + } + return new ListTuple(items, 0, len); + } + } + } + + /// Default (non-optimized) implementation for ITuple.StartsWith() + /// Larger tuple + /// Smaller tuple + /// True if starts with (or is equal to) + public static bool StartsWith([NotNull] ITuple a, [NotNull] ITuple b) + { + Contract.Requires(a != null && b != null); + if (object.ReferenceEquals(a, b)) return true; + int an = a.Count; + int bn = b.Count; + + if (bn > an) return false; + if (bn == 0) return true; // note: 'an' can only be 0 because of previous test + + for (int i = 0; i < bn; i++) + { + if (!object.Equals(a[i], b[i])) return false; + } + return true; + } + + /// Default (non-optimized) implementation for ITuple.EndsWith() + /// Larger tuple + /// Smaller tuple + /// True if starts with (or is equal to) + public static bool EndsWith([NotNull] ITuple a, [NotNull] ITuple b) + { + Contract.Requires(a != null && b != null); + if (object.ReferenceEquals(a, b)) return true; + int an = a.Count; + int bn = b.Count; + + if (bn > an) return false; + if (bn == 0) return true; // note: 'an' can only be 0 because of previous test + + int offset = an - bn; + for (int i = 0; i < bn; i++) + { + if (!object.Equals(a[offset + i], b[i])) return false; + } + return true; + } + + /// Helper to copy the content of a tuple at a specific position in an array + /// Updated offset just after the last element of the copied tuple + public static int CopyTo([NotNull] ITuple tuple, [NotNull] object[] array, int offset) + { + Contract.Requires(tuple != null && array != null && offset >= 0); + + foreach (var item in tuple) + { + array[offset++] = item; + } + return offset; + } + + /// Maps a relative index into an absolute index + /// Relative index in the tuple (from the end if negative) + /// Size of the tuple + /// Absolute index from the start of the tuple, or exception if outside of the tuple + /// If the absolute index is outside of the tuple (<0 or >=) + public static int MapIndex(int index, int count) + { + int offset = index; + if (offset < 0) offset += count; + if (offset < 0 || offset >= count) return FailIndexOutOfRange(index, count); + return offset; + } + + /// Maps a relative index into an absolute index + /// Relative index in the tuple (from the end if negative) + /// Size of the tuple + /// Absolute index from the start of the tuple. Truncated to 0 if index is before the start of the tuple, or to if the index is after the end of the tuple + public static int MapIndexBounded(int index, int count) + { + if (index < 0) index += count; + return Math.Max(Math.Min(index, count), 0); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T FailIndexOutOfRange(int index, int count) + { + throw new IndexOutOfRangeException($"Index {index} is outside of the tuple's range (0..{count - 1})"); + } + + public static bool Equals(ITuple left, object other, [NotNull] IEqualityComparer comparer) + { + return object.ReferenceEquals(left, null) ? other == null : Equals(left, other as ITuple, comparer); + } + + public static bool Equals(ITuple x, ITuple y, [NotNull] IEqualityComparer comparer) + { + if (object.ReferenceEquals(x, y)) return true; + if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; + + return x.Count == y.Count && DeepEquals(x, y, comparer); + } + + public static bool DeepEquals([NotNull] ITuple x, [NotNull] ITuple y, [NotNull] IEqualityComparer comparer) + { + Contract.Requires(x != null && y != null && comparer != null); + + using (var xs = x.GetEnumerator()) + using (var ys = y.GetEnumerator()) + { + while (xs.MoveNext()) + { + if (!ys.MoveNext()) return false; + + if (!comparer.Equals(xs.Current, ys.Current)) return false; + } + + return !ys.MoveNext(); + } + } + + public static int StructuralGetHashCode(ITuple tuple, [NotNull] IEqualityComparer comparer) + { + Contract.Requires(comparer != null); + + if (object.ReferenceEquals(tuple, null)) + { + return comparer.GetHashCode(null); + } + + int h = 0; + foreach (var item in tuple) + { + h = HashCodes.Combine(h, comparer.GetHashCode(item)); + } + return h; + } + + public static int StructuralCompare(ITuple x, ITuple y, [NotNull] IComparer comparer) + { + Contract.Requires(comparer != null); + + if (object.ReferenceEquals(x, y)) return 0; + if (object.ReferenceEquals(x, null)) return -1; + if (object.ReferenceEquals(y, null)) return 1; + + using (var xs = x.GetEnumerator()) + using (var ys = y.GetEnumerator()) + { + while (xs.MoveNext()) + { + if (!ys.MoveNext()) return 1; + + int cmp = comparer.Compare(xs.Current, ys.Current); + if (cmp != 0) return cmp; + + } + return ys.MoveNext() ? -1 : 0; + } + } + } +} diff --git a/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs b/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs new file mode 100644 index 000000000..038486b45 --- /dev/null +++ b/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs @@ -0,0 +1,138 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Collections.Tuples.Encoding +{ + using System; + using JetBrains.Annotations; + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + TuplePackers.SerializeTo(ref writer, tuple.Item2); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + TuplePackers.SerializeTo(ref writer, tuple.Item2); + TuplePackers.SerializeTo(ref writer, tuple.Item3); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + TuplePackers.SerializeTo(ref writer, tuple.Item2); + TuplePackers.SerializeTo(ref writer, tuple.Item3); + TuplePackers.SerializeTo(ref writer, tuple.Item4); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + TuplePackers.SerializeTo(ref writer, tuple.Item2); + TuplePackers.SerializeTo(ref writer, tuple.Item3); + TuplePackers.SerializeTo(ref writer, tuple.Item4); + TuplePackers.SerializeTo(ref writer, tuple.Item5); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } + + public sealed class TupleSerializer : ITupleSerializer> + { + public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); + + public void PackTo(ref TupleWriter writer, ref STuple tuple) + { + TuplePackers.SerializeTo(ref writer, tuple.Item1); + TuplePackers.SerializeTo(ref writer, tuple.Item2); + TuplePackers.SerializeTo(ref writer, tuple.Item3); + TuplePackers.SerializeTo(ref writer, tuple.Item4); + TuplePackers.SerializeTo(ref writer, tuple.Item5); + TuplePackers.SerializeTo(ref writer, tuple.Item6); + } + + public void UnpackFrom(ref TupleReader reader, out STuple tuple) + { + TupleEncoder.DecodeKey(ref reader, out tuple); + } + } +} diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 41f341305..c8d087f36 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -711,7 +711,7 @@ public static FdbError FutureGetKey(FutureHandle future, out Slice key) } else { - key = Slice.Create(ptr, keyLength); + key = Slice.Copy(ptr, keyLength); } return err; } diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 750e14874..7aa8a103d 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -26,15 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Linq; + using Doxense.Collections.Tuples; + using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Memory; using JetBrains.Annotations; public class DynamicKeySubspace : KeySubspace, IDynamicKeySubspace @@ -113,7 +113,7 @@ public KeyRange ToRange([NotNull] ITupleFormattable item) /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix - /// This is a shortcut for + /// This is a shortcut for public Slice this[[NotNull] ITuple tuple] => Pack(tuple); /// Convert an item into a key of this subspace @@ -333,7 +333,7 @@ public Slice[] EncodeMany(IEnumerableUnpack a key of this subspace, back into a tuple - /// Key that was produced by a previous call to + /// Key that was produced by a previous call to /// Original tuple public ITuple Unpack(Slice packed) { @@ -365,7 +365,7 @@ private static T[] BatchDecode(IEnumerable packed, IKeySubspace subspa } /// Unpack a batch of keys of this subspace, back into an array of tuples - /// Sequence of keys that were produced by a previous call to or + /// Sequence of keys that were produced by a previous call to or /// Array containing the original tuples public ITuple[] UnpackMany(IEnumerable packed) { diff --git a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs index e7d863ee8..5bf238fd0 100644 --- a/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs +++ b/FoundationDB.Client/Subspaces/FdbDatabasePartition.cs @@ -84,7 +84,7 @@ void IFdbDirectory.CheckLayer(Slice layer) { if (layer.IsPresent && layer != this.Layer) { - throw new InvalidOperationException($"The directory {this.FullName} is a partition which is not compatible with layer {layer.ToAsciiOrHexaString()}."); + throw new InvalidOperationException($"The directory {this.FullName} is a partition which is not compatible with layer {layer:P}."); } } diff --git a/FoundationDB.Client/Subspaces/IKeySubspace.cs b/FoundationDB.Client/Subspaces/IKeySubspace.cs index c436796d7..086a01068 100644 --- a/FoundationDB.Client/Subspaces/IKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/IKeySubspace.cs @@ -26,10 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; using System.Collections.Generic; + using Doxense.Memory; using JetBrains.Annotations; /// Represents a sub-partition of the global key space. diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 6ae622ebd..185950c99 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -26,13 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Memory; using JetBrains.Annotations; /// Adds a prefix on every keys, to group them inside a common subspace @@ -120,7 +122,7 @@ public static IDynamicKeySubspace CreateDynamic([NotNull] ITuple tuple, IKeyEnco { Contract.NotNull(tuple, nameof(tuple)); var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new DynamicKeySubspace(tuple.ToSlice(), true, encoder); + return new DynamicKeySubspace(TuPack.Pack(tuple), copy: true, encoder: encoder); } [Pure, NotNull] diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs index 010265f25..fe6f003a5 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs @@ -26,14 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Collections.Tuples; + using Doxense.Memory; using JetBrains.Annotations; /// Subspace that knows how to encode and decode its key @@ -103,7 +102,7 @@ public T Decode(Slice packed) public KeyRange ToRange(T value) { //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value)); + return TuPack.ToRange(Encode(value)); } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs index 6a7f0a472..00e4c0e5f 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -26,15 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; - using FoundationDB.Layers.Tuples; + using Doxense.Collections.Tuples; + using Doxense.Memory; /// Subspace that knows how to encode and decode its key /// Type of the first item of the keys handled by this subspace @@ -108,7 +107,7 @@ public STuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2) { //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2)); + return TuPack.ToRange(Encode(value1, value2)); } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index 1b658f547..c3c3a9f57 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -26,15 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { using System; using System.Collections.Generic; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using JetBrains.Annotations; - using FoundationDB.Layers.Tuples; /// Subspace that knows how to encode and decode its key /// Type of the first item of the keys handled by this subspace @@ -111,7 +110,7 @@ public STuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2, T3 value3) { //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2, value3)); + return TuPack.ToRange(Encode(value1, value2, value3)); } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index 5a13c2ec2..394ad0857 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -26,14 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB.Client.Utils; - namespace FoundationDB.Client { using System; using System.Collections.Generic; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Memory; using JetBrains.Annotations; /// Subspace that knows how to encode and decode its key @@ -120,7 +119,7 @@ public STuple Decode(Slice packed) public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) { //REVIEW: which semantic for ToRange() should we use? - return STuple.ToRange(Encode(value1, value2, value3, value4)); + return TuPack.ToRange(Encode(value1, value2, value3, value4)); } } diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index 96ba147cb..861c972be 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -1,8 +1,38 @@ -using System; -using FoundationDB.Layers.Tuples; +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + namespace FoundationDB.Client { + using System; + using Doxense.Collections.Tuples; + using Doxense.Memory; + public abstract class DynamicKeyEncoderBase : IDynamicKeyEncoder { @@ -139,4 +169,4 @@ public virtual KeyRange ToKeyRange(Slice prefix, return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index e04ac2c3b..0c32aaea9 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -26,13 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Linq; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using FoundationDB.Layers.Tuples; + using Doxense.Memory; using JetBrains.Annotations; /// Helper class for all key/value encoders @@ -444,7 +446,7 @@ public override Slice EncodeComposite(STuple items, int count) { Contract.Requires(count > 0); - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); return writer.ToSlice(); @@ -482,7 +484,7 @@ public override Slice EncodeComposite(STuple items, int count) { Contract.Requires(count > 0 && count <= 3); - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); if (count >= 3) m_codec3.EncodeOrderedSelfTerm(ref writer, items.Item3); @@ -500,7 +502,7 @@ public override STuple DecodeComposite(Slice encoded, int count) if (count >= 1) key1 = m_codec1.DecodeOrderedSelfTerm(ref reader); if (count >= 2) key2 = m_codec2.DecodeOrderedSelfTerm(ref reader); if (count >= 3) key3 = m_codec3.DecodeOrderedSelfTerm(ref reader); - if (reader.HasMore) throw new InvalidOperationException(String.Format("Unexpected data at the end of composite key after {0} items", count)); + if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); return STuple.Create(key1, key2, key3); } @@ -510,7 +512,7 @@ public override STuple DecodeComposite(Slice encoded, int count) [NotNull] public static IKeyEncoder Bind([NotNull] IOrderedTypeCodec codec) { - if (codec == null) throw new ArgumentNullException("codec"); + if (codec == null) throw new ArgumentNullException(nameof(codec)); return new OrderedKeyEncoder(codec); } @@ -519,8 +521,8 @@ public static IKeyEncoder Bind([NotNull] IOrderedTypeCodec codec) [NotNull] public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2) { - if (codec1 == null) throw new ArgumentNullException("codec1"); - if (codec2 == null) throw new ArgumentNullException("codec2"); + if (codec1 == null) throw new ArgumentNullException(nameof(codec1)); + if (codec2 == null) throw new ArgumentNullException(nameof(codec2)); return new CodecCompositeKeyEncoder(codec1, codec2); } @@ -529,9 +531,9 @@ public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCo [NotNull] public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2, [NotNull] IOrderedTypeCodec codec3) { - if (codec1 == null) throw new ArgumentNullException("codec1"); - if (codec2 == null) throw new ArgumentNullException("codec2"); - if (codec3 == null) throw new ArgumentNullException("codec2"); + if (codec1 == null) throw new ArgumentNullException(nameof(codec1)); + if (codec2 == null) throw new ArgumentNullException(nameof(codec2)); + if (codec3 == null) throw new ArgumentNullException(nameof(codec2)); return new CodecCompositeKeyEncoder(codec1, codec2, codec3); } @@ -567,7 +569,7 @@ public static class Unordered [NotNull] public static IKeyEncoder Bind([NotNull] IUnorderedTypeCodec codec) { - if (codec == null) throw new ArgumentNullException("codec"); + if (codec == null) throw new ArgumentNullException(nameof(codec)); var encoder = codec as IKeyEncoder; if (encoder != null) return encoder; @@ -619,7 +621,7 @@ public static IValueEncoder GuidEncoder [NotNull] public static IValueEncoder Bind([NotNull] IUnorderedTypeCodec codec) { - if (codec == null) throw new ArgumentNullException("codec"); + if (codec == null) throw new ArgumentNullException(nameof(codec)); var encoder = codec as IValueEncoder; if (encoder != null) return encoder; @@ -700,24 +702,24 @@ private TupleKeyEncoder() { } public Slice EncodeKey(T key) { - return STuple.EncodeKey(key); + return TuPack.EncodeKey(key); } public T DecodeKey(Slice encoded) { if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return STuple.DecodeKey(encoded); + return TuPack.DecodeKey(encoded); } public Slice EncodeValue(T key) { - return STuple.EncodeKey(key); + return TuPack.EncodeKey(key); } public T DecodeValue(Slice encoded) { if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return STuple.DecodeKey(encoded); + return TuPack.DecodeKey(encoded); } } @@ -733,17 +735,17 @@ public override Slice EncodeComposite(STuple key, int items) { switch (items) { - case 2: return key.ToSlice(); - case 1: return STuple.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException("items", items, "Item count must be either 1 or 2"); + case 2: return TuPack.EncodeKey(key.Item1, key.Item2); + case 1: return TuPack.EncodeKey(key.Item1); + default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be either 1 or 2"); } } public override STuple DecodeComposite(Slice encoded, int items) { - if (items < 1 || items > 2) throw new ArgumentOutOfRangeException("items", items, "Item count must be either 1 or 2"); + if (items < 1 || items > 2) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be either 1 or 2"); - var t = STuple.Unpack(encoded).OfSize(items); + var t = TuPack.Unpack(encoded).OfSize(items); Contract.Assert(t != null); return STuple.Create( @@ -764,18 +766,18 @@ public override Slice EncodeComposite(STuple key, int items) { switch (items) { - case 3: return key.ToSlice(); - case 2: return STuple.EncodeKey(key.Item1, key.Item2); - case 1: return STuple.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 3"); + case 3: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3); + case 2: return TuPack.EncodeKey(key.Item1, key.Item2); + case 1: return TuPack.EncodeKey(key.Item1); + default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 3"); } } public override STuple DecodeComposite(Slice encoded, int items) { - if (items < 1 || items > 3) throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 3"); + if (items < 1 || items > 3) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 3"); - var t = STuple.Unpack(encoded).OfSize(items); + var t = TuPack.Unpack(encoded).OfSize(items); Contract.Assert(t != null); return STuple.Create( @@ -797,19 +799,19 @@ public override Slice EncodeComposite(STuple key, int items) { switch (items) { - case 4: return key.ToSlice(); - case 3: return STuple.EncodeKey(key.Item1, key.Item2, key.Item3); - case 2: return STuple.EncodeKey(key.Item1, key.Item2); - case 1: return STuple.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 4"); + case 4: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3, key.Item4); + case 3: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3); + case 2: return TuPack.EncodeKey(key.Item1, key.Item2); + case 1: return TuPack.EncodeKey(key.Item1); + default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 4"); } } public override STuple DecodeComposite(Slice encoded, int items) { - if (items < 1 || items > 4) throw new ArgumentOutOfRangeException("items", items, "Item count must be between 1 and 4"); + if (items < 1 || items > 4) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 4"); - var t = STuple.Unpack(encoded).OfSize(items); + var t = TuPack.Unpack(encoded).OfSize(items); return STuple.Create( t.Get(0), @@ -870,8 +872,8 @@ public static IValueEncoder Value() [NotNull] public static IKeyEncoder Bind([NotNull] Func encoder, [NotNull] Func decoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (decoder == null) throw new ArgumentNullException("decoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (decoder == null) throw new ArgumentNullException(nameof(decoder)); return new Singleton(encoder, decoder); } @@ -879,8 +881,8 @@ public static IKeyEncoder Bind([NotNull] Func encoder, [NotNull] [NotNull] public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params T[] values) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (values == null) throw new ArgumentNullException("values"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (values == null) throw new ArgumentNullException(nameof(values)); var slices = new Slice[values.Length]; for (int i = 0; i < values.Length; i++) @@ -894,9 +896,9 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotN [NotNull] public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable elements, Func selector) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (elements == null) throw new ArgumentNullException("elements"); - if (selector == null) throw new ArgumentNullException("selector"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (elements == null) throw new ArgumentNullException(nameof(elements)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); TElement[] arr; ICollection coll; @@ -928,9 +930,9 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder([NotNull] this IKeyEncoder encoder, [NotNull] TElement[] elements, Func selector) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (elements == null) throw new ArgumentNullException("elements"); - if (selector == null) throw new ArgumentNullException("selector"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (elements == null) throw new ArgumentNullException(nameof(elements)); + if (selector == null) throw new ArgumentNullException(nameof(selector)); var slices = new Slice[elements.Length]; for (int i = 0; i < elements.Length; i++) @@ -944,8 +946,8 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable values) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (values == null) throw new ArgumentNullException("values"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (values == null) throw new ArgumentNullException(nameof(values)); // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed @@ -974,8 +976,8 @@ public static IEnumerable EncodeKeys([NotNull] this IKeyEncoder enc [NotNull] public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params Slice[] slices) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (slices == null) throw new ArgumentNullException("slices"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (slices == null) throw new ArgumentNullException(nameof(slices)); var values = new T[slices.Length]; for (int i = 0; i < slices.Length; i++) @@ -989,8 +991,8 @@ public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] [NotNull] public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] KeyValuePair[] items) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (items == null) throw new ArgumentNullException("items"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (items == null) throw new ArgumentNullException(nameof(items)); var values = new T[items.Length]; for (int i = 0; i < items.Length; i++) @@ -1004,8 +1006,8 @@ public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] [NotNull] public static IEnumerable DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable slices) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (slices == null) throw new ArgumentNullException("slices"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (slices == null) throw new ArgumentNullException(nameof(slices)); // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) @@ -1015,14 +1017,14 @@ public static IEnumerable DecodeKeys([NotNull] this IKeyEncoder encoder /// Returns a partial encoder that will only encode the first element public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); return new HeadEncoder(encoder); } /// Returns a partial encoder that will only encode the first element public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); return new HeadEncoder(encoder); } @@ -1030,7 +1032,7 @@ public static HeadEncoder Head([NotNull] this IComposite /// Returns a partial encoder that will only encode the first element public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); return new HeadEncoder(encoder); } @@ -1038,7 +1040,7 @@ public static HeadEncoder Head([NotNull] this IC /// Returns a partial encoder that will only encode the first and second elements public static PairEncoder Pair([NotNull] this ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); return new PairEncoder(encoder); } @@ -1046,7 +1048,7 @@ public static PairEncoder Pair([NotNull] this IComposite /// Returns a partial encoder that will only encode the first and second elements public static PairEncoder Pair([NotNull] this ICompositeKeyEncoder encoder) { - if (encoder == null) throw new ArgumentNullException("encoder"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); return new PairEncoder(encoder); } @@ -1059,8 +1061,8 @@ public static PairEncoder Pair([NotNull] this IC [NotNull] public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] params T[] values) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (values == null) throw new ArgumentNullException("values"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (values == null) throw new ArgumentNullException(nameof(values)); var slices = new Slice[values.Length]; for (int i = 0; i < values.Length; i++) @@ -1075,8 +1077,8 @@ public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [ [NotNull] public static IEnumerable EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable values) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (values == null) throw new ArgumentNullException("values"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (values == null) throw new ArgumentNullException(nameof(values)); // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed @@ -1104,8 +1106,8 @@ public static IEnumerable EncodeValues([NotNull] this IValueEncoder [NotNull] public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] params Slice[] slices) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (slices == null) throw new ArgumentNullException("slices"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (slices == null) throw new ArgumentNullException(nameof(slices)); var values = new T[slices.Length]; for (int i = 0; i < slices.Length; i++) @@ -1120,8 +1122,8 @@ public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotN [NotNull] public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] KeyValuePair[] items) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (items == null) throw new ArgumentNullException("items"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (items == null) throw new ArgumentNullException(nameof(items)); var values = new T[items.Length]; for (int i = 0; i < items.Length; i++) @@ -1136,8 +1138,8 @@ public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotN [NotNull] public static IEnumerable DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable slices) { - if (encoder == null) throw new ArgumentNullException("encoder"); - if (slices == null) throw new ArgumentNullException("slices"); + if (encoder == null) throw new ArgumentNullException(nameof(encoder)); + if (slices == null) throw new ArgumentNullException(nameof(slices)); // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) diff --git a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs index 3d3f4c6d0..c1f64ca2d 100644 --- a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs @@ -26,11 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using FoundationDB.Layers.Tuples; - namespace FoundationDB.Client { + using System; + using Doxense.Collections.Tuples; + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { Slice EncodeKey(T1 value1, T2 value2, T3 value3, T4 value4); @@ -53,4 +53,4 @@ public interface ICompositeKeyEncoder : IKeyEncoder TTuple DecodeComposite(Slice encoded, int items); } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs index 8312636bd..5cb9b5736 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs @@ -26,12 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using FoundationDB.Layers.Tuples; -using JetBrains.Annotations; - namespace FoundationDB.Client { + using System; + using Doxense.Collections.Tuples; + using Doxense.Memory; + using JetBrains.Annotations; /// Encoder that can process keys of variable size and types public interface IDynamicKeyEncoder @@ -330,4 +330,4 @@ public interface IDynamicKeyEncoder } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs index bc3326eea..c99b461f7 100644 --- a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs @@ -26,9 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client { using System; + using Doxense.Memory; public interface IOrderedTypeCodec { diff --git a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs index fb7bdd866..60cd9a5f0 100644 --- a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using Doxense.Memory; public interface IUnorderedTypeCodec { diff --git a/FoundationDB.Client/TypeSystem/TypeCodec`1.cs b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs index 5424672fa..06300d7cc 100644 --- a/FoundationDB.Client/TypeSystem/TypeCodec`1.cs +++ b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using Doxense.Memory; public abstract class TypeCodec : IOrderedTypeCodec, IUnorderedTypeCodec { @@ -39,7 +40,7 @@ public abstract class TypeCodec : IOrderedTypeCodec, IUnorderedTypeCodec BE LIABLE FOR ANY */ #endregion + namespace FoundationDB.Client //REVIEW: what namespace? { using System; using JetBrains.Annotations; - using FoundationDB.Layers.Tuples; + using Doxense.Collections.Tuples.Encoding; public static class TypeSystem { @@ -51,4 +52,4 @@ static TypeSystem() } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/Utils/Batched.cs b/FoundationDB.Client/Utils/Batched.cs index fa6f329b1..e01b4a300 100644 --- a/FoundationDB.Client/Utils/Batched.cs +++ b/FoundationDB.Client/Utils/Batched.cs @@ -31,6 +31,7 @@ namespace FoundationDB using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using JetBrains.Annotations; internal static class Batched diff --git a/FoundationDB.Client/Utils/ExceptionExtensions.cs b/FoundationDB.Client/Utils/ExceptionExtensions.cs index b84bdf08b..a77734c54 100644 --- a/FoundationDB.Client/Utils/ExceptionExtensions.cs +++ b/FoundationDB.Client/Utils/ExceptionExtensions.cs @@ -1,9 +1,29 @@ -#region Copyright Doxense 2005-2015 -// -// All rights are reserved. Reproduction or transmission in whole or in part, in -// any form or by any means, electronic, mechanical or otherwise, is prohibited -// without the prior written consent of the copyright owner. -// +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace Doxense @@ -118,5 +138,4 @@ public static Exception Unwrap([NotNull] this Exception self) return self; } } - } diff --git a/FoundationDB.Client/Utils/HashCodes.cs b/FoundationDB.Client/Utils/HashCodes.cs new file mode 100644 index 000000000..11609e6b6 --- /dev/null +++ b/FoundationDB.Client/Utils/HashCodes.cs @@ -0,0 +1,216 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense +{ + using System; + using System.Collections; + using System.Runtime.CompilerServices; + using JetBrains.Annotations; + + /// Helper methods to work with hashcodes + internal static class HashCodes + { + //REVIEW: dplacer dans le namespace "Doxense" tout court? => c'est utilis dans des tonnes de classes Model POCO + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(long value) + { + return unchecked((int) value) ^ (int) (value >> 32); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(ulong value) + { + return unchecked((int)value) ^ (int)(value >> 32); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(bool value) + { + return value ? 1 : 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(Guid value) + { + return value.GetHashCode(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(int? value) + { + return value ?? -1; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(long? value) + { + return value.HasValue ? Compute(value.Value) : -1; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(ulong? value) + { + return value.HasValue ? Compute(value.Value) : -1; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(bool? value) + { + return value.HasValue ? Compute(value.Value) : -1; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(Guid? value) + { + return value?.GetHashCode() ?? -1; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(string value) + { + return value?.GetHashCode() ?? 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(T value) + where T : class + { + return value?.GetHashCode() ?? 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(T? value) + where T : struct + { + return value.GetValueOrDefault().GetHashCode(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Compute(T value, System.Collections.IEqualityComparer comparer) + where T : IStructuralEquatable + { + return value?.GetHashCode(comparer) ?? 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Combine(int h1, int h2) + { + return ((h1 << 5) + h1) ^ h2; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Combine(int h1, int h2, int h3) + { + int h = ((h1 << 5) + h1) ^ h2; + return ((h << 5) + h) ^ h3; + } + + [Pure] + public static int Combine(int h1, int h2, int h3, int h4) + { + return Combine(Combine(h1, h2), Combine(h3, h4)); + } + + [Pure] + public static int Combine(int h1, int h2, int h3, int h4, int h5) + { + return Combine(Combine(h1, h2, h3), Combine(h4, h5)); + } + + [Pure] + public static int Combine(int h1, int h2, int h3, int h4, int h5, int h6) + { + return Combine(Combine(h1, h2, h3), Combine(h4, h5, h6)); + } + + /// Test that both hash codes, if present, have the same value + /// False IIF h1 != nul && h2 != null && h1 != h2; otherisse, True + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool SameOrMissing(int? h1, int? h2) + { + return !h1.HasValue || !h2.HasValue || h1.Value == h2.Value; + } + + #region Flags... + + // Combines one or more booleans into a single value (one bit per boolean) + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a) + { + return (a ? 1 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b) + { + return (a ? 1 : 0) | (b ? 2 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c, bool d) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0) | (d ? 8 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c, bool d, bool e) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0) | (d ? 8 : 0) | (e ? 16 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c, bool d, bool e, bool f) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0) | (d ? 8 : 0) | (e ? 16 : 0) | (f ? 32 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c, bool d, bool e, bool f, bool g) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0) | (d ? 8 : 0) | (e ? 16 : 0) | (f ? 32 : 0) | (g ? 64 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Flags(bool a, bool b, bool c, bool d, bool e, bool f, bool g, bool h) + { + return (a ? 1 : 0) | (b ? 2 : 0) | (c ? 4 : 0) | (d ? 8 : 0) | (e ? 16 : 0) | (f ? 32 : 0) | (g ? 64 : 0) | (h ? 128 : 0); + } + + #endregion + + } +} diff --git a/FoundationDB.Client/Utils/Memory/BitHelpers.cs b/FoundationDB.Client/Utils/Memory/BitHelpers.cs new file mode 100644 index 000000000..1f8505a98 --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/BitHelpers.cs @@ -0,0 +1,777 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Memory +{ + using System; + using System.Diagnostics; + using System.Runtime.CompilerServices; + using JetBrains.Annotations; + + /// Helper methods to work with bits + [PublicAPI] + [DebuggerNonUserCode] + public static class BitHelpers + { + + #region Power of Twos + + /// Round a number to the next power of 2 + /// Positive integer that will be rounded up (if not already a power of 2) + /// Smallest power of 2 that is greater than or equal to + /// Will return 1 for = 0 (because 0 is not a power of 2 !), and will throw for < 0 + /// If is greater than 2^31 and would overflow + [Pure] + public static uint NextPowerOfTwo(uint x) + { + // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two + + // special cases + if (x == 0) return 1; + if (x > (1U << 31)) throw UnsafeHelpers.Errors.PowerOfTwoOverflow(); + + --x; + x |= (x >> 1); + x |= (x >> 2); + x |= (x >> 4); + x |= (x >> 8); + x |= (x >> 16); + return x + 1; + } + + /// Round a number to the next power of 2 + /// Positive integer that will be rounded up (if not already a power of 2) + /// Smallest power of 2 that is greater then or equal to + /// Will return 1 for = 0 (because 0 is not a power 2 !), and will throws for < 0 + /// If is negative, or it is greater than 2^30 and would overflow. + [Pure] + public static int NextPowerOfTwo(int x) + { + // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two + + // special cases + if (x == 0) return 1; + if ((uint)x > (1U << 30)) throw UnsafeHelpers.Errors.PowerOfTwoNegative(); + + --x; + x |= (x >> 1); + x |= (x >> 2); + x |= (x >> 4); + x |= (x >> 8); + x |= (x >> 16); + return x + 1; + } + + /// Round a number to the next power of 2 + /// Positive integer that will be rounded up (if not already a power of 2) + /// Smallest power of 2 that is greater than or equal to + /// Will return 1 for = 0 (because 0 is not a power of 2 !), and will throw for < 0 + /// If is greater than 2^63 and would overflow + [Pure] + public static ulong NextPowerOfTwo(ulong x) + { + // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two + + // special cases + if (x == 0) return 1; + if (x > (1UL << 63)) throw UnsafeHelpers.Errors.PowerOfTwoOverflow(); + + --x; + x |= (x >> 1); + x |= (x >> 2); + x |= (x >> 4); + x |= (x >> 8); + x |= (x >> 16); + x |= (x >> 32); + return x + 1; + } + + /// Round a number to the next power of 2 + /// Positive integer that will be rounded up (if not already a power of 2) + /// Smallest power of 2 that is greater then or equal to + /// Will return 1 for = 0 (because 0 is not a power 2 !), and will throws for < 0 + /// If is negative, or it is greater than 2^62 and would overflow. + [Pure] + public static long NextPowerOfTwo(long x) + { + // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two + + // special cases + if (x == 0) return 1; + if ((ulong) x > (1UL << 62)) throw UnsafeHelpers.Errors.PowerOfTwoNegative(); + + --x; + x |= (x >> 1); + x |= (x >> 2); + x |= (x >> 4); + x |= (x >> 8); + x |= (x >> 16); + x |= (x >> 32); + return x + 1; + } + + /// Test if a number is a power of 2 + /// True if is expressible as 2^i (i>=0) + /// 0 is NOT considered to be a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool IsPowerOfTwo(int x) + { + return x > 0 & unchecked((x & (x - 1)) == 0); + } + + /// Test if a number is a power of 2 + /// True if is expressible as 2^i (i>=0) + /// 0 is NOT considered to be a power of 2 + /// This methods guarantees that IsPowerOfTwo(x) == (NextPowerOfTwo(x) == x) + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool IsPowerOfTwo(uint x) + { + return x != 0 & unchecked((x & (x - 1)) == 0); + } + + /// Test if a number is a power of 2 + /// True if is expressible as 2^i (i>=0) + /// 0 is NOT considered to be a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool IsPowerOfTwo(long x) + { + return x > 0 & unchecked((x & (x - 1)) == 0); + } + + /// Test if a number is a power of 2 + /// True if is expressible as 2^i (i>=0) + /// 0 is NOT considered to be a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool IsPowerOfTwo(ulong x) + { + return x != 0 & unchecked((x & (x - 1)) == 0); + } + + #endregion + + #region Alignment / Padding... + + //REVIEW: align/padding should probably be moved somewhere else because it does not really have anything to do bith bit twiddling... + + /// Round a size to a multiple of a specific value + /// Minimum size required + /// Final size must be a multiple of this number + /// Result cannot be less than this value + /// Size rounded up to the next multiple of , or 0 if is negative + /// For aligments that are powers of two, will be faster + /// If the rounded size overflows over 2 GB + [Pure] + public static int Align(int size, [Positive] int alignment, int minimum = 0) + { + //Contract.Requires(alignment > 0); + long x = Math.Max(size, minimum); + x += alignment - 1; + x /= alignment; + x *= alignment; + return checked((int) x); + } + + /// Round a size to a multiple of power of two + /// Minimum size required + /// Must be a power two + /// Size rounded up to the next multiple of + /// If the rounded size overflows over 2 GB + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int AlignPowerOfTwo(int size, [PowerOfTwo] int powerOfTwo = 16) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + if (size <= 0) + { + return size < 0 ? 0 : powerOfTwo; + } + int mask = powerOfTwo - 1; + // force an exception if we overflow above 2GB + return checked(size + mask) & ~mask; + } + + /// Round a size to a multiple of a specific value + /// Minimum size required + /// Final size must be a multiple of this number + /// Result cannot be less than this value + /// Size rounded up to the next multiple of . + /// + /// For aligments that are powers of two, will be faster. + /// + /// If the rounded size overflows over 2 GB + [Pure] + public static uint Align(uint size, uint alignment, uint minimum = 0) + { + //Contract.Requires(alignment > 0); + ulong x = Math.Max(size, minimum); + x += alignment - 1; + x /= alignment; + x *= alignment; + return checked((uint) x); + } + + /// Round a size to a multiple of power of two + /// Minimum size required + /// Must be a power two + /// Size rounded up to the next multiple of + /// If the rounded size overflows over 4 GB + [Pure] + public static uint AlignPowerOfTwo(uint size, [PowerOfTwo] uint powerOfTwo = 16U) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + if (size == 0) return powerOfTwo; + uint mask = powerOfTwo - 1; + // force an exception if we overflow above 4GB + return checked(size + mask) & ~mask; + } + + /// Round a size to a multiple of a specific value + /// Minimum size required + /// Final size must be a multiple of this number + /// Result cannot be less than this value + /// Size rounded up to the next multiple of , or 0 if is negative + /// For aligments that are powers of two, will be faster + /// If the rounded size overflows over 2^63 + [Pure] + public static long Align(long size, [Positive] long alignment, long minimum = 0) + { + //Contract.Requires(alignment > 0); + long x = Math.Max(size, minimum); + // we have to divide first and check the modulo, because adding (aligment+1) before could overflow at the wrong time + long y = x /alignment; + if (x % alignment != 0) ++y; + return checked(y * alignment); + } + + /// Round a size to a multiple of power of two + /// Minimum size required + /// Must be a power two + /// Size rounded up to the next multiple of + /// If the rounded size overflows over long.MaxValue + [Pure] + public static long AlignPowerOfTwo(long size, [PowerOfTwo] long powerOfTwo = 16L) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + if (size <= 0) + { + return size < 0 ? 0 : powerOfTwo; + } + // force an exception if we overflow above ulong.MaxValue + long mask = powerOfTwo - 1; + return checked(size + mask) & ~mask; + } + + /// Round a size to a multiple of a specific value + /// Minimum size required + /// Final size must be a multiple of this number + /// Result cannot be less than this value + /// Size rounded up to the next multiple of . + /// + /// For aligments that are powers of two, will be faster. + /// + /// If the rounded size overflows over 2^63 + [Pure] + public static ulong Align(ulong size, ulong alignment, ulong minimum = 0) + { + //Contract.Requires(alignment > 0); + ulong x = Math.Max(size, minimum); + // we have to divide first and check the modulo, because adding (aligment+1) before could overflow at the wrong time + ulong y = x / alignment; + if (x % alignment != 0) ++y; + return checked(y * alignment); + } + + /// Round a size to a multiple of power of two + /// Minimum size required + /// Must be a power two + /// Size rounded up to the next multiple of + /// If the rounded size overflows over ulong.MaxValue + [Pure] + public static ulong AlignPowerOfTwo(ulong size, [PowerOfTwo] ulong powerOfTwo = 16UL) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + + if (size == 0) + { + return powerOfTwo; + } + // force an exception if we overflow above ulong.MaxValue + ulong mask = powerOfTwo - 1; + return checked(size + mask) & ~mask; + } + + /// Computes the number of padding bytes needed to align a buffer to a specific alignment + /// Size of the buffer + /// Alignement required (must be a power of two) + /// Number of padding bytes required to end up with a buffer size multiple of . Returns 0 if the buffer is already aligned + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int PaddingPowerOfTwo(int size, [PowerOfTwo] int powerOfTwo = 16) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + return (~size + 1) & (powerOfTwo - 1); + + } + + /// Computes the number of padding bytes needed to align a buffer to a specific alignment + /// Size of the buffer + /// Alignement required (must be a power of two) + /// Number of padding bytes required to end up with a buffer size multiple of . Returns 0 if the buffer is already aligned + /// Result is unspecified if is 0 or not a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint PaddingPowerOfTwo(uint size, [PowerOfTwo] uint powerOfTwo = 16) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + return (~size + 1) & (powerOfTwo - 1); + } + + /// Computes the number of padding bytes needed to align a buffer to a specific alignment + /// Size of the buffer + /// Alignement required (must be a power of two) + /// Number of padding bytes required to end up with a buffer size multiple of . Returns 0 if the buffer is already aligned + /// Result is unspecified if is 0 or not a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static long PaddingPowerOfTwo(long size, [PowerOfTwo] long powerOfTwo = 16) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + return (~size + 1) & (powerOfTwo - 1); + + } + + /// Computes the number of padding bytes needed to align a buffer to a specific alignment + /// Size of the buffer + /// Alignement required (must be a power of two) + /// Number of padding bytes required to end up with a buffer size multiple of . Returns 0 if the buffer is already aligned + /// Result is unspecified if is 0 or not a power of 2 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong PaddingPowerOfTwo(ulong size, [PowerOfTwo] ulong powerOfTwo = 16) + { + //Contract.Requires(BitHelpers.IsPowerOfTwo(powerOfTwo)); + return (~size + 1) & (powerOfTwo - 1); + } + + #endregion + + #region CountBits... + + // CountBits(x) == POPCNT == number of bits that are set to 1 in a word + // - CountBits(0) == 0 + // - CountBits(8) == 1 + // - CountBits(42) == 3 + // - CountBits(uint.MaxValue) == 32 + + /// Count the number of bits set to 1 in a 32-bit signed integer + /// Value between 0 and 32 + [Pure] //REVIEW: force inline or not? + public static int CountBits(int value) + { + // cf https://graphics.stanford.edu/~seander/bithacks.html#CountBitsSet64 + // PERF: this averages ~2ns/op OnMyMachine(tm) + value = value - ((value >> 1) & 0x55555555); + value = (value & 0x33333333) + ((value >> 2) & 0x33333333); + value = ((value + (value >> 4) & 0xF0F0F0F) * 0x1010101) >> (32 - 8); + return value; + } + + /// Count the number of bits set to 1 in a 32-bit unsigned integer + /// Value between 0 and 32 + [Pure] //REVIEW: force inline or not? + public static int CountBits(uint value) + { + // cf https://graphics.stanford.edu/~seander/bithacks.html#CountBitsSet64 + // PERF: this averages ~2ns/op OnMyMachine(tm) + value = value - ((value >> 1) & 0x55555555); + value = (value & 0x33333333) + ((value >> 2) & 0x33333333); + value = ((value + (value >> 4) & 0xF0F0F0F) * 0x1010101) >> (32 - 8); + return (int) value; + } + + /// Count the number of bits set to 1 in a 64-bit signed integer + /// Value between 0 and 64 + [Pure] //REVIEW: force inline or not? + public static int CountBits(long value) + { + // cf https://graphics.stanford.edu/~seander/bithacks.html#CountBitsSet64 + // PERF: this averages ~2.5ns/op OnMyMachine(tm) + value = value - ((value >> 1) & 0x5555555555555555); + value = (value & 0x3333333333333333) + ((value >> 2) & 0x3333333333333333); + value = ((value + (value >> 4) & 0x0F0F0F0F0F0F0F0F) * 0x0101010101010101) >> (64 - 8); + return (int) value; + } + + /// Count the number of bits set to 1 in a 32-bit unsigned integer + /// Value between 0 and 64 + [Pure] //REVIEW: force inline or not? + public static int CountBits(ulong value) + { + // cf https://graphics.stanford.edu/~seander/bithacks.html#CountBitsSet64 + // PERF: this averages ~2.5ns/op OnMyMachine(tm) + value = value - ((value >> 1) & 0x5555555555555555); + value = (value & 0x3333333333333333) + ((value >> 2) & 0x3333333333333333); + value = ((value + (value >> 4) & 0x0F0F0F0F0F0F0F0F) * 0x0101010101010101) >> (64 - 8); + return (int) value; + } + + #endregion + + #region MostSignificantBit... + + // MostSignificantBit(x) == Highest bit index (0..63) of the first bit set to 1 + // - MostSignificantBit(1) == 0 + // - MostSignificantBit(8) == 3 + // - MostSignificantBit(42) == 5 + // - MostSignificantBit(uint.MaxValue) == 31 + // Remark: if the value can be 0, the convention is to return to the word size (32 or 64) + // - MostSignificantBit(default(uint)) == 32 + // - MostSignificantBit(default(ulong)) == 64 + // MostSignificantBitNonZeroXX(x) is a no-branch variant which is undefined for x == 0 + + private static readonly int[] MultiplyDeBruijnBitPosition32 = new int[32] + { + 0, 9, 1, 10, 13, 21, 2, 29, 11, 14, 16, 18, 22, 25, 3, 30, + 8, 12, 20, 28, 15, 17, 24, 7, 19, 27, 23, 6, 26, 5, 4, 31 + }; + + private static readonly int[] MultiplyDeBruijnBitPosition64 = new int[64] + { + 63, 0, 58, 1, 59, 47, 53, 2, + 60, 39, 48, 27, 54, 33, 42, 3, + 61, 51, 37, 40, 49, 18, 28, 20, + 55, 30, 34, 11, 43, 14, 22, 4, + 62, 57, 46, 52, 38, 26, 32, 41, + 50, 36, 17, 19, 29, 10, 13, 21, + 56, 45, 25, 31, 35, 16, 9, 12, + 44, 24, 15, 8, 23, 7, 6, 5 + }; + + /// Return the position of the highest bit that is set + /// Value between 0 and 32 + /// + /// Result is 32 if is 0. + /// If the value of is known to be non-zero, then you can call directly. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int MostSignificantBit(int v) + { + return v == 0 ? 32 : MostSignificantBitNonZero32((uint) v); + } + + /// Return the position of the highest bit that is set + /// Value between 0 and 32 + /// + /// Result is 32 if is 0. + /// If the value of is known to be non-zero, then you can call directly. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int MostSignificantBit(uint v) + { + return v == 0 ? 32 : MostSignificantBitNonZero32(v); + } + + /// Return the position of the highest bit that is set + /// Result is unspecified if is 0. + [Pure] //REVIEW: force inline or not? + public static int MostSignificantBitNonZero32(uint v) + { + // from: http://graphics.stanford.edu/~seander/bithacks.html#IntegerLogDeBruijn + v |= v >> 1; // first round down to one less than a power of 2 + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + + var r = (v * 0x07C4ACDDU) >> 27; + return MultiplyDeBruijnBitPosition32[r & 31]; + } + + /// Return the position of the highest bit that is set + /// Value between 0 and 64 + /// + /// Result is 64 if is 0. + /// If the value of is known to be non-zero, then you can call directly. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int MostSignificantBit(long v) + { + return v == 0 ? 64 : MostSignificantBitNonZero64((ulong) v); + } + + /// Return the position of the highest bit that is set + /// Value between 0 and 64 + /// + /// Result is 64 if is zero. + /// If the value of is known to be non-zero, then you can call directly. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int MostSignificantBit(ulong v) + { + return v == 0 ? 64 : MostSignificantBitNonZero64(v); + } + + /// Return the position of the highest bit that is set + /// Result is unspecified if is 0. + [Pure] //REVIEW: force inline or not? + public static int MostSignificantBitNonZero64(ulong nonZero) + { + ulong v = nonZero; + v |= v >> 1; + v |= v >> 2; + v |= v >> 4; + v |= v >> 8; + v |= v >> 16; + v |= v >> 32; + + var r = ((v - (v >> 1)) * 0x07EDD5E59A4E28C2UL) >> 58; + return MultiplyDeBruijnBitPosition64[r & 63]; + } + + #endregion + + #region LeastSignificantBit... + + // LeastSignificantBit(x) == Smallest bit index (0..63) of the first bit set to 1 + // - LeastSignificantBit(1) == 0 + // - LeastSignificantBit(8) == 3 + // - LeastSignificantBit(42) == 2 + // - LeastSignificantBit(uint.MaxValue) = 0 + // Remark: if the value is 0, the convention is to return to the word size (32 or 64) + // - LeastSignificantBit(default(uint)) == 32 + // - LeastSignificantBit(default(ulong)) == 64 + // LeastSignificantBitNonZeroXX(x) is a no-branch variant which is undefined for x == 0 + + /// Return the position of the lowest bit that is set + /// Value between 0 and 32 + /// Result is 32 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBit(int v) + { + return v == 0 ? 32 : LeastSignificantBitNonZero32(v); + } + + /// Return the position of the lowest bit that is set + /// Value between 0 and 32 + /// Result is 32 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBit(uint v) + { + return v == 0 ? 32 : LeastSignificantBitNonZero32(v); + } + + /// Return the position of the lowest bit that is set + /// Result is unspecified if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBitNonZero32(long nonZero) + { + // This solution does not have any branch, but conversion to float may not be fast enough on some architecture... + //PERF: this averages 2.5ns/op OnMyMachine() + unsafe + { + //note: nonZero must be a long, because -int.MaxValue would overflow on 32-bit + var d = (float) (nonZero & -nonZero); + return (int) (((*(uint*) &d) >> 23) - 0x7f); + //note: this returns -127 if w == 0, which is "negative" + } + } + + /// Return the position of the lowest bit that is set + /// Value between 0 and 64 + /// Result is 64 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBit(ulong v) + { + return v == 0 ? 64 : LeastSignificantBitNonZero64((long) v); + } + + /// Return the position of the lowest bit that is set + /// Value between 0 and 64 + /// Result is 64 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBit(long v) + { + return v == 0 ? 64 : LeastSignificantBitNonZero64(v); + } + + /// Return the position of the lowest bit that is set + /// Result is unspecified if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LeastSignificantBitNonZero64(long nonZero) + { + // This solution does not have any branch, but conversion to double may not be fast enough on some architecture... + //PERF: this averages 2.5ns/op OnMyMachine() + unsafe + { + // isolated LS1B to double + var d = (double)(nonZero & -nonZero); + // exponent is in bits 52 to 62 (11 bits) + ulong l = *((ulong*)&d); + ulong exp = (l >> 52) & ((1 << 11) - 1); + return (int)(exp - 1023); + //note: this returns -1023 if w == 0, which is "negative" + } + } + + #endregion + + #region FirstNonZeroByte... + + // FirstNonZeroByte(x) == offset of the first byte in a multi-byte word, that has at least one bit set to 1 + // - FirstNonZeroByte(0x000042) == 0 + // - FirstNonZeroByte(0x004200) == 1 + // - FirstNonZeroByte(0x004201) == 0 + // - FirstNonZeroByte(0x420000) == 2 + // - FirstNonZeroByte(0x420001) == 0 + // Remark: if the value is 0, the convention is to return to the word size in bytes (4 or 8) + // - FirstNonZeroByte(default(uint)) == 4 + // - FirstNonZeroByte(default(ulong)) == 8 + + /// Return the offset of the first non-zero byte + /// Value between 0 and 4 + /// Returns 4 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int FirstNonZeroByte(int v) + { + return v == 0 ? 4 : (LeastSignificantBitNonZero32(v) >> 3); + } + + /// Return the offset of the first non-zero byte + /// Value between 0 and 4 + /// Returns 4 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int FirstNonZeroByte(uint v) + { + return v == 0 ? 4 : (LeastSignificantBitNonZero32((int) v) >> 3); + } + + /// Return the offset of the first non-zero byte + /// Value between 0 and 8 + /// Returns 8 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int FirstNonZeroByte(long v) + { + return v == 0 ? 8 : (LeastSignificantBitNonZero64(v) >> 3); + } + + /// Return the offset of the first non-zero byte + /// Value between 0 and 8 + /// Returns 8 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int FirstNonZeroByte(ulong v) + { + return v == 0 ? 8 : (LeastSignificantBitNonZero64((long) v) >> 3); + } + + #endregion + + #region LastNonZeroByte... + + // LastNonZeroByte(x) == offset of the first byte in a multi-byte word, that has at least one bit set to 1 + // - LastNonZeroByte(0x000042) == 0 + // - LastNonZeroByte(0x004200) == 1 + // - LastNonZeroByte(0x004201) == 1 + // - LastNonZeroByte(0x420000) == 2 + // - LastNonZeroByte(0x420001) == 2 + // Remark: if the value is 0, the convention is to return to the word size in bytes (4 or 8) + // - LastNonZeroByte(default(uint)) == 4 + // - LastNonZeroByte(default(ulong)) == 8 + + /// Return the offset of the last non-zero byte + /// Returns 4 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LastNonZeroByte(int v) + { + return v == 0 ? 4 : (MostSignificantBitNonZero32((uint) v) >> 3); + } + + /// Return the offset of the last non-zero byte + /// Returns 4 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LastNonZeroByte(uint v) + { + return v == 0 ? 4 : (MostSignificantBitNonZero32(v) >> 3); + } + + /// Return the offset of the last non-zero byte + /// Returns 8 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LastNonZeroByte(long v) + { + return v == 0 ? 8 : (MostSignificantBitNonZero64((ulong) v) >> 3); + } + + /// Return the offset of the last non-zero byte + /// Returns 8 if is 0 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LastNonZeroByte(ulong v) + { + return v == 0 ? 8 : (MostSignificantBitNonZero64(v) >> 3); + } + + #endregion + + #region RotL/RotR... + + /// Rotate bits to the left (ROTL) + /// RotL32(0x12345678, 4) = 0x23456781 + /// Equivalent of the 'rotl' CRT function, or the 'ROL' x86 instruction + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint RotL32(uint x, int n) + { + return (x << n) | (x >> (32 - n)); + } + + /// Rotate bits to the right (ROTR) + /// RotR32(0x12345678, 4) = 0x81234567 + /// Equivalent of the 'rotr' CRT function, or the 'ROR' x86 instruction + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint RotR32(uint x, int n) + { + return (x >> n) | (x << (32 - n)); + } + + /// Rotate bits to the left (ROTL64) + /// RotL64(0x0123456789ABCDEF, 4) = 0x123456789ABCDEF0 + /// Equivalent of the '_rotl64' CRT function, or the 'ROL' x64 instruction + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong RotL64(ulong x, int n) + { + return (x << n) | (x >> (64 - n)); + } + + /// Rotate bits to the right (ROTR64) + /// RotR64(0x0123456789ABCDEF, 4) = 0xF0123456789ABCDE + /// Equivalent of the '_rotr64' CRT function, or the 'ROR' x64 instruction + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong RotR64(ulong x, int n) + { + return (x >> n) | (x << (64 - n)); + } + + #endregion + + } + +} diff --git a/FoundationDB.Client/Utils/Memory/Slice.Comparer.cs b/FoundationDB.Client/Utils/Memory/Slice.Comparer.cs new file mode 100644 index 000000000..f5f367792 --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/Slice.Comparer.cs @@ -0,0 +1,89 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace System +{ + using System; + using System.Collections.Generic; + using Doxense.Memory; + + public partial struct Slice + { + + /// Performs optimized equality and comparison checks on Slices + public sealed class Comparer : IComparer, IEqualityComparer, IComparer> + { + /// Default instance of the slice comparator + public static readonly Comparer Default = new Comparer(); + + private Comparer() + { } + + /// Lexicographically compare two slices and returns an indication of their relative sort order. + /// Slice compared with + /// Slice compared with + /// Returns a NEGATIVE value if is LESS THAN , ZERO if is EQUAL TO , and a POSITIVE value if is GREATER THAN . + /// + /// If both and are nil or empty, the comparison will return ZERO. If only is nil or empty, it will return a NEGATIVE value. If only is nil or empty, it will return a POSITIVE value. + /// There are no guarantees that non-zero results will be exactly -1 or +1. You should always use comparison operators or the sign of the returned value, instead of testing for equality with -1 or +1. + /// + public int Compare(Slice x, Slice 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 UnsafeHelpers.Compare(x.Array, x.Offset, x.Count, y.Array, y.Offset, y.Count); + } + + /// Checks if two slices are equal. + /// Slice compared with + /// Slice compared with + /// true if and have the same size and contain the same sequence of bytes; otherwise, false. + public bool Equals(Slice x, Slice y) + { + return x.Count == y.Count && UnsafeHelpers.SameBytes(x.Array, x.Offset, y.Array, y.Offset, y.Count); + } + + /// Computes the hash code of a slice + /// A slice + /// A 32-bit signed hash coded calculated from all the bytes in the slice + public int GetHashCode(Slice obj) + { + return obj.Array == null ? 0 : UnsafeHelpers.ComputeHashCode(obj.Array, obj.Offset, obj.Count); + } + + int IComparer>.Compare(KeyValuePair x, KeyValuePair y) + { + // only compare the keys + return Compare(x.Key, y.Key); + } + + } + } + +} diff --git a/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs b/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs new file mode 100644 index 000000000..14ba1ee15 --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs @@ -0,0 +1,2504 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +//#define ENABLE_SPAN + +namespace System +{ + using System; + using System.Globalization; + using System.Runtime.CompilerServices; + using System.Text; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; + + public partial struct Slice + { + + #region FromXXX... + + /// Decode a Base64 encoded string into a slice + [Pure] + public static Slice FromBase64(string base64String) + { + return base64String == null ? Slice.Nil : base64String.Length == 0 ? Slice.Empty : Convert.FromBase64String(base64String).AsSlice(); + } + + #region 8-bit integers... + + /// Encode an unsigned 8-bit integer into a slice + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] //used as a shortcut by a lot of other methods + public static Slice FromByte(byte value) + { + return new Slice(ByteSprite, value, 1); + } + + /// Encode an unsigned 8-bit integer into a slice + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] //used as a shortcut by a lot of other methods + public static Slice FromByte(int value) + { + if ((uint) value > 255) ThrowHelper.ThrowArgumentOutOfRangeException(nameof(value)); + return new Slice(ByteSprite, value, 1); + } + + #endregion + + #region 16-bit integers + + /// Encode a signed 16-bit integer into a variable size slice (1 or 2 bytes) in little-endian + [Pure] + public static Slice FromInt16(short value) + { + if (value >= 0) + { + if (value <= 255) + { + return Slice.FromByte((byte)value); + } + return new Slice(new byte[] { (byte)value, (byte)(value >> 8) }, 0, 2); + } + + return FromFixed16(value); + } + + /// Encode a signed 16-bit integer into a 2-byte slice in little-endian + [Pure] + public static Slice FromFixed16(short value) + { + return new Slice(new byte[2] { (byte) value, (byte) (value >> 8) }, 0, 2); + } + + /// Encode an unsigned 16-bit integer into a variable size slice (1 or 2 bytes) in little-endian + [Pure] + public static Slice FromUInt16(ushort value) + { + if (value <= 255) + { + return Slice.FromByte((byte)value); + } + else + { + return FromFixedU16(value); + } + } + + /// Encode an unsigned 16-bit integer into a 2-byte slice in little-endian + /// 0x1122 => 11 22 + [Pure] + public static Slice FromFixedU16(ushort value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[2] { (byte) value, (byte) (value >> 8) }, 0, 2); + } + + /// Encode an unsigned 16-bit integer into a 2-byte slice in big-endian + /// 0x1122 => 22 11 + [Pure] + public static Slice FromFixedU16BE(ushort value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[2] { (byte) (value >> 8), (byte) value }, 0, 4); + } + + /// Encode an unsigned 16-bit integer into 7-bit encoded unsigned int (aka 'Varint16') + [Pure] + public static Slice FromVarint16(ushort value) + { + if (value < 128) + { + return FromByte((byte)value); + } + else + { + var writer = new SliceWriter(3); + writer.WriteVarInt16(value); + return writer.ToSlice(); + } + } + + #endregion + + #region 32-bit integers + + /// Encode a signed 32-bit integer into a variable size slice (1 to 4 bytes) in little-endian + [Pure] + public static Slice FromInt32(int value) + { + if (value >= 0) + { + if (value <= (1 << 8) - 1) + { + return Slice.FromByte((byte)value); + } + if (value <= (1 << 16) - 1) + { + //TODO: possible micro optimization is for values like 0x100, 0x201, 0x1413 or 0x4342, where we could use 2 consecutive bytes in the ByteSprite, + return new Slice(new byte[2] { (byte)value, (byte)(value >> 8) }, 0, 2); + } + if (value <= (1 << 24) - 1) + { + return new Slice(new byte[3] { (byte)value, (byte)(value >> 8), (byte)(value >> 16) }, 0, 3); + } + } + + return FromFixed32(value); + } + + /// Encode a signed 32-bit integer into a 4-byte slice in little-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed32(int value) + { + return new Slice(new byte[4] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24) }, 0, 4); + } + + /// Encode a signed 32-bit integer into a variable size slice (1 to 4 bytes) in big-endian + [Pure] + public static Slice FromInt32BE(int value) + { + if (value >= 0) + { + if (value <= (1 << 8) - 1) + { + return Slice.FromByte((byte)value); + } + if (value <= (1 << 16) - 1) + { + //TODO: possible micro optimization is for values like 0x100, 0x201, 0x1413 or 0x4342, where we could use 2 consecutive bytes in the ByteSprite, + return new Slice(new byte[2] { (byte) (value >> 8), (byte) value }, 0, 2); + } + if (value <= (1 << 24) - 1) + { + return new Slice(new byte[3] { (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 3); + } + } + return FromFixed32BE(value); + } + + /// Encode a signed 32-bit integer into a 4-byte slice in big-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed32BE(int value) + { + return new Slice(new byte[4] { (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value, }, 0, 4); + } + + /// Encode an unsigned 32-bit integer into a variable size slice (1 to 4 bytes) in little-endian + [Pure] + public static Slice FromUInt32(uint value) + { + if (value <= (1 << 8) - 1) + { + return FromByte((byte) value); + } + if (value <= (1 << 16) - 1) + { + return new Slice(new byte[2] { (byte) value, (byte) (value >> 8) }, 0, 2); + } + if (value <= (1 << 24) - 1) + { + return new Slice(new byte[3] { (byte) value, (byte) (value >> 8), (byte) (value >> 16) }, 0, 3); + } + return FromFixedU32(value); + } + + /// Encode an unsigned 32-bit integer into a 4-byte slice in little-endian + /// 0x11223344 => 11 22 33 44 + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixedU32(uint value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[4] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24) }, 0, 4); + } + + /// Encode an unsigned 32-bit integer into a variable size slice (1 to 4 bytes) in big-endian + [Pure] + public static Slice FromUInt32BE(uint value) + { + if (value <= (1 << 8) - 1) + { + return FromByte((byte)value); + } + if (value <= (1 << 16) - 1) + { + return new Slice(new byte[2] { (byte) (value >> 8), (byte) value }, 0, 2); + } + if (value <= (1 << 24) - 1) + { + return new Slice(new byte[3] { (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 3); + } + return FromFixedU32BE(value); + } + + /// Encode an unsigned 32-bit integer into a 4-byte slice in big-endian + /// 0x11223344 => 44 33 22 11 + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixedU32BE(uint value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[4] { (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 4); + } + + /// Encode an unsigned 32-bit integer into 7-bit encoded unsigned int (aka 'Varint32') + [Pure] + public static Slice FromVarint32(uint value) + { + if (value <= 127) + { // single byte slices are cached + return FromByte((byte)value); + } + + var writer = new SliceWriter(value <= (1 << 14) - 1 ? 2 : 5); + writer.WriteVarInt32(value); + return writer.ToSlice(); + } + + #endregion + + #region 64-bit integers + + /// Encode a signed 64-bit integer into a variable size slice (1 to 8 bytes) in little-endian + [Pure] + public static Slice FromInt64(long value) + { + if (value >= 0) + { + if (value <= (1L << 32) - 1) + { + return FromInt32((int) value); + } + if (value <= (1L << 40) - 1) + { + return new Slice(new byte[5] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32) }, 0, 5); + } + if (value <= (1L << 48) - 1) + { + return new Slice(new byte[6] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40) }, 0, 6); + } + if (value <= (1L << 56) - 1) + { + return new Slice(new byte[7] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40), (byte) (value >> 48) }, 0, 7); + } + } + + return FromFixed64(value); + } + + /// Encode a signed 64-bit integer into a 8-byte slice in little-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed64(long value) + { + return new Slice(new byte[8] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40), (byte) (value >> 48), (byte) (value >> 56) }, 0, 8); + } + + /// Encode a signed 64-bit integer into a variable size slice (1 to 8 bytes) in big-endian + [Pure] + public static Slice FromInt64BE(long value) + { + if (value >= 0) + { + if (value <= (1L << 32) - 1) + { + return FromInt32BE((int) value); + } + if (value <= (1L << 40) - 1) + { + return new Slice(new byte[5] { (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 5); + } + if (value <= (1L << 48) - 1) + { + return new Slice(new byte[6] { (byte) (value >> 40), (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 6); + } + if (value <= (1L << 56) - 1) + { + return new Slice(new byte[7] { (byte) (value >> 48), (byte) (value >> 40), (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 7); + } + } + + return FromFixed64BE(value); + } + + /// Encode a signed 64-bit integer into a 8-byte slice in big-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed64BE(long value) + { + return new Slice(new byte[8] { (byte) (value >> 56), (byte) (value >> 48), (byte) (value >> 40), (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 8); + } + + /// Encode an unsigned 64-bit integer into a variable size slice (1 to 8 bytes) in little-endian + [Pure] + public static Slice FromUInt64(ulong value) + { + if (value <= (1UL << 32) - 1) + { + return FromUInt32((uint) value); + } + if (value <= (1UL << 40) - 1) + { + return new Slice(new byte[5] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32) }, 0, 5); + } + if (value <= (1UL << 48) - 1) + { + return new Slice(new byte[6] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40) }, 0, 6); + } + if (value <= (1UL << 56) - 1) + { + return new Slice(new byte[7] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40), (byte) (value >> 48) }, 0, 7); + } + return FromFixedU64(value); + } + + /// Encode an unsigned 64-bit integer into a 8-byte slice in little-endian + /// 0x1122334455667788 => 11 22 33 44 55 66 77 88 + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixedU64(ulong value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[8] { (byte) value, (byte) (value >> 8), (byte) (value >> 16), (byte) (value >> 24), (byte) (value >> 32), (byte) (value >> 40), (byte) (value >> 48), (byte) (value >> 56) }, 0, 8); + } + + /// Encode an unsigned 64-bit integer into a variable size slice (1 to 8 bytes) in big-endian + [Pure] + public static Slice FromUInt64BE(ulong value) + { + if (value <= (1UL << 32) - 1) + { + return FromInt32BE((int) value); + } + if (value <= (1UL << 40) - 1) + { + return new Slice(new byte[5] { (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 5); + } + if (value <= (1UL << 48) - 1) + { + return new Slice(new byte[6] { (byte)(value >> 40), (byte)(value >> 32), (byte)(value >> 24), (byte)(value >> 16), (byte)(value >> 8), (byte)value }, 0, 6); + } + if (value <= (1UL << 56) - 1) + { + return new Slice(new byte[7] { (byte) (value >> 48), (byte) (value >> 40), (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 7); + } + return FromFixedU64BE(value); + } + + /// Encode an unsigned 64-bit integer into a 8-byte slice in big-endian + /// 0x1122334455667788 => 88 77 66 55 44 33 22 11 + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixedU64BE(ulong value) //REVIEW: we could drop the 'U' here + { + return new Slice(new byte[8] { (byte) (value >> 56), (byte) (value >> 48), (byte) (value >> 40), (byte) (value >> 32), (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }, 0, 8); + } + + /// Encode an unsigned 64-bit integer into 7-bit encoded unsigned int (aka 'Varint64') + [Pure] + public static Slice FromVarint64(ulong value) + { + if (value <= 127) + { // single byte slices are cached + return FromByte((byte)value); + } + + SliceWriter writer; + if (value <= uint.MaxValue) + { + writer = new SliceWriter(value <= (1 << 14) - 1 ? 2 : 5); + writer.WriteVarInt32((uint) value); + } + else + { + writer = new SliceWriter(10); + writer.WriteVarInt64(value); + } + return writer.ToSlice(); + } + + #endregion + + #region 128-bit integers + + // we model 128-bit integers as two 64-bit integers (low and high) + + /// Encode a signed 128-bit integer into a 16-byte slice in little-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed128(long lo, long hi) + { + return new Slice( + new byte[16] + { + (byte) (lo), (byte) (lo >> 8), (byte) (lo >> 16), (byte) (lo >> 24), (byte) (lo >> 32), (byte) (lo >> 40), (byte) (lo >> 48), (byte) (lo >> 56), + (byte) (hi), (byte) (hi >> 8), (byte) (hi >> 16), (byte) (hi >> 24), (byte) (hi >> 32), (byte) (hi >> 40), (byte) (hi >> 48), (byte) (hi >> 56), + }, + 0, + 16 + ); + } + + /// Encode a signed 128-bit integer into a 16-byte slice in big-endian + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromFixed128BE(long lo, long hi) + { + return new Slice( + new byte[16] + { + (byte) (hi >> 56), (byte) (hi >> 48), (byte) (hi >> 40), (byte) (hi >> 32), (byte) (hi >> 24), (byte) (hi >> 16), (byte) (hi >> 8), (byte) (hi), + (byte) (lo >> 56), (byte) (lo >> 48), (byte) (lo >> 40), (byte) (lo >> 32), (byte) (lo >> 24), (byte) (lo >> 16), (byte) (lo >> 8), (byte) (lo), + }, + 0, + 16 + ); + } + + #endregion + + #region decimals + + /// Encode a 32-bit decimal into an 4-byte slice + [Pure] + public static Slice FromSingle(float value) + { + //TODO: may not work on BE platforms? + byte[] tmp = new byte[4]; + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + *((float*)ptr) = value; + } + } + return new Slice(tmp, 0, 4); + } + + /// Encode a 32-bit decimal into an 4-byte slice (in network order) + [Pure] + public static Slice FromSingleBE(float value) + { + //TODO: may not work on BE platforms? + byte[] tmp = new byte[4]; + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + *((uint*)ptr) = UnsafeHelpers.ByteSwap32(*(uint*) &value); + } + } + return new Slice(tmp, 0, 4); + } + + /// Encode a 64-bit decimal into an 8-byte slice + [Pure] + public static Slice FromDouble(double value) + { + //TODO: may not work on BE platforms? + byte[] tmp = new byte[8]; + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + *((double*) ptr) = value; + } + } + return new Slice(tmp, 0, 8); + } + + /// Encode a 64-bit decimal into an 8-byte slice (in network order) + [Pure] + public static Slice FromDoubleBE(double value) + { + //TODO: may not work on BE platforms? + byte[] tmp = new byte[8]; + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + *((ulong*)ptr) = UnsafeHelpers.ByteSwap64(*(ulong*) &value); + } + } + return new Slice(tmp, 0, 8); + } + + /// Encode a 128-bit decimal into an 16-byte slice + public static Slice FromDecimal(decimal value) + { + //TODO: may not work on BE platforms? + byte[] tmp = new byte[16]; + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + *((decimal*) ptr) = value; + } + } + return new Slice(tmp, 0, 16); + } + + #endregion + + /// Create a 16-byte slice containing a System.Guid encoding according to RFC 4122 (Big Endian) + /// WARNING: Slice.FromGuid(guid).GetBytes() will not produce the same result as guid.ToByteArray() ! + /// If you need to produce Microsoft compatible byte arrays, use Slice.Create(guid.ToByteArray()) but then you shoud NEVER use Slice.ToGuid() to decode such a value ! + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromGuid(Guid value) + { + // UUID are stored using the RFC4122 format (Big Endian), while .NET's System.GUID use Little Endian + // => we will convert the GUID into a UUID under the hood, and hope that it gets converted back when read from the db + + return new Uuid128(value).ToSlice(); + } + + /// Create a 16-byte slice containing an RFC 4122 compliant 128-bit UUID + /// You should never call this method on a slice created from the result of calling System.Guid.ToByteArray() ! + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromUuid128(Uuid128 value) + { + // UUID should already be in the RFC 4122 ordering + return value.ToSlice(); + } + + /// Create an 8-byte slice containing an 64-bit UUID + [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromUuid64(Uuid64 value) + { + return value.ToSlice(); + } + + /// Encoding used to produce ASCII slices + [NotNull] + internal static readonly ASCIIEncoding AsciiEncoding = new ASCIIEncoding(); + + /// Encoding used to produce ANSI slices + [NotNull] + internal static readonly Encoding DefaultEncoding = Encoding.Default; + + /// Encoding used to produce UTF-8 slices + [NotNull] + internal static readonly UTF8Encoding Utf8NoBomEncoding = new UTF8Encoding(encoderShouldEmitUTF8Identifier: false, throwOnInvalidBytes: true); + + /// Dangerously create a slice containing string converted to the local ANSI code page. All non-ANSI characters may be corrupted or converted to '?', and this slice may not decode properly on a different system. + /// + /// WARNING: if you put a string that contains non-ANSI chars, it will be silently corrupted! This should only be used to store keywords or 'safe' strings, and when the decoding will only happen on the same system, or systems using the same codepage. + /// Slices encoded by this method are not guaranteed to be decoded without loss. YOU'VE BEEN WARNED! + /// + [Pure] + public static Slice FromStringAnsi([CanBeNull] string text) + { + return text == null ? Slice.Nil + : text.Length == 0 ? Slice.Empty + : new Slice(DefaultEncoding.GetBytes(text)); + } + + /// Create a slice from an ASCII string, where all the characters map directory into bytes (0..255). The string will be checked before being encoded. + /// + /// This method will check each character and fail if at least one is greater than 255. + /// Slices encoded by this method are only guaranteed to roundtrip if decoded with . If the original string only contained ASCII characters (0..127) then it can also be decoded by . + /// The only difference between this method and is that the later will truncate non-ASCII characters to their lowest 8 bits, while the former will throw an exception. + /// + /// If at least one character is greater than 255. + [Pure] + public static Slice FromStringAscii([CanBeNull] string value) + { + if (value == null) return Slice.Nil; + if (value.Length == 0) return Slice.Empty; + byte[] _ = null; + unsafe + { + fixed(char* chars = value) + { + return ConvertByteStringChecked(chars, value.Length, ref _); + } + } + } + +#if ENABLE_SPAN + /// Create a slice from an ASCII string, where all the characters map directory into bytes (0..255). The string will be checked before being encoded. + /// + /// This method will check each character and fail if at least one is greater than 255. + /// Slices encoded by this method are only guaranteed to roundtrip if decoded with . If the original string only contained ASCII characters (0..127) then it can also be decoded by . + /// The only difference between this method and is that the later will truncate non-ASCII characters to their lowest 8 bits, while the former will throw an exception. + /// + /// If at least one character is greater than 255. + [Pure] + public static Slice FromStringAscii(ReadOnlySpan value) + { + if (value.Length == 0) return Slice.Empty; + byte[] _ = null; + return ConvertByteStringChecked(value, ref _); + } + + /// Create a slice from an ASCII string, where all the characters map directory into bytes (0..255). The string will be checked before being encoded. + /// + /// This method will check each character and fail if at least one is greater than 255. + /// Slices encoded by this method are only guaranteed to roundtrip if decoded with . If the original string only contained ASCII characters (0..127) then it can also be decoded by . + /// The only difference between this method and is that the later will truncate non-ASCII characters to their lowest 8 bits, while the former will throw an exception. + /// + /// If at least one character is greater than 255. + [Pure] + public static Slice FromStringAscii(ReadOnlySpan value, ref byte[] buffer) + { + if (value.Length == 0) return Slice.Empty; + return ConvertByteStringChecked(value, ref buffer); + } + + /// Create a slice from an byte string, where all the characters map directly into bytes (0..255), without performing any validation + /// + /// This method does not make any effort to detect characters above 255, which will be truncated to their lower 8 bits, introducing corruption when the string will be decoded. Please MAKE SURE to not call this with untrusted data. + /// Slices encoded by this method are ONLY compatible with UTF-8 encoding if all characters are between 0 and 127. If this is not the case, then decoding it as an UTF-8 sequence may introduce corruption. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromByteString([CanBeNull] string value) + { + if (value == null) return Slice.Nil; + byte[] _ = null; + return FromByteString(value.AsSpan(), ref _); + } + + /// Create a slice from an byte string, where all the characters map directly into bytes (0..255), without performing any validation + /// + /// This method does not make any effort to detect characters above 255, which will be truncated to their lower 8 bits, introducing corruption when the string will be decoded. Please MAKE SURE to not call this with untrusted data. + /// Slices encoded by this method are ONLY compatible with UTF-8 encoding if all characters are between 0 and 127. If this is not the case, then decoding it as an UTF-8 sequence may introduce corruption. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromByteString(ReadOnlySpan value) + { + byte[] _ = default; + return FromByteString(value, ref _); + } + + /// Create a slice from an byte string, where all the characters map directly into bytes (0..255), without performing any validation + /// + /// This method does not make any effort to detect characters above 255, which will be truncated to their lower 8 bits, introducing corruption when the string will be decoded. Please MAKE SURE to not call this with untrusted data. + /// Slices encoded by this method are ONLY compatible with UTF-8 encoding if all characters are between 0 and 127. If this is not the case, then decoding it as an UTF-8 sequence may introduce corruption. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromByteString(ReadOnlySpan value, ref byte[] buffer) + { + return value.Length != 0 ? ConvertByteStringNoCheck(value, ref buffer) : Slice.Empty; + } + + [Pure] + internal static Slice ConvertByteStringChecked(ReadOnlySpan value, ref byte[] buffer) + { + int n = value.Length; + if (n == 1) + { + char c = value[0]; + if (c > 0xFF) goto InvalidChar; + if (buffer?.Length > 0) + { + buffer[0] = (byte) c; + return new Slice(buffer, 0, 1); + } + return FromByte((byte) c); + } + + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, n); + if (!TryConvertBytesStringChecked(new Span(tmp, 0, n), value)) goto InvalidChar; + return new Slice(tmp, 0, n); + InvalidChar: + throw ThrowHelper.FormatException("The specified string contains characters that cannot be safely truncated to 8 bits. If you are encoding natural text, you should use UTF-8 encoding."); + } + + [Pure] + private static bool TryConvertBytesStringChecked(Span buffer, ReadOnlySpan value) + { + int n = value.Length; + if ((uint) buffer.Length < (uint) n) return false; + unsafe + { + fixed (byte* pBytes = &MemoryMarshal.GetReference(buffer)) + fixed (char* pChars = &MemoryMarshal.GetReference(value)) + { + char* inp = pChars; + byte* outp = pBytes; + + while (n > 0) + { + char c = *inp; + if (c > 0xFF) return false; + *outp++ = (byte)(*inp++); + --n; + } + } + } + return true; + } +#else + + /// Create a slice from an byte string, where all the characters map directly into bytes (0..255), without performing any validation + /// + /// This method does not make any effort to detect characters above 255, which will be truncated to their lower 8 bits, introducing corruption when the string will be decoded. Please MAKE SURE to not call this with untrusted data. + /// Slices encoded by this method are ONLY compatible with UTF-8 encoding if all characters are between 0 and 127. If this is not the case, then decoding it as an UTF-8 sequence may introduce corruption. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromByteString([CanBeNull] string value) + { + if (value == null) return Slice.Nil; + byte[] _ = null; + unsafe + { + fixed(char* chars = value) + { + return FromByteString(chars, value.Length, ref _); + } + } + } + + /// Create a slice from an byte string, where all the characters map directly into bytes (0..255), without performing any validation + /// + /// This method does not make any effort to detect characters above 255, which will be truncated to their lower 8 bits, introducing corruption when the string will be decoded. Please MAKE SURE to not call this with untrusted data. + /// Slices encoded by this method are ONLY compatible with UTF-8 encoding if all characters are between 0 and 127. If this is not the case, then decoding it as an UTF-8 sequence may introduce corruption. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static unsafe Slice FromByteString(char* chars, int numChars, ref byte[] buffer) + { + return numChars != 0 ? ConvertByteStringNoCheck(chars, numChars, ref buffer) : Slice.Empty; + } + + [Pure] + internal static unsafe Slice ConvertByteStringChecked(char* value, int n, ref byte[] buffer) + { + if (n == 1) + { + char c = value[0]; + if (c > 0xFF) goto InvalidChar; + if (buffer?.Length > 0) + { + buffer[0] = (byte) c; + return new Slice(buffer, 0, 1); + } + return FromByte((byte) c); + } + + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, n); + if (!TryConvertBytesStringChecked(new Slice(tmp, 0, n), value, n)) goto InvalidChar; + return new Slice(tmp, 0, n); + InvalidChar: + throw ThrowHelper.FormatException("The specified string contains characters that cannot be safely truncated to 8 bits. If you are encoding natural text, you should use UTF-8 encoding."); + } + + [Pure] + private static unsafe bool TryConvertBytesStringChecked(Slice buffer, char* value, int n) + { + if ((uint) buffer.Count < (uint) n) return false; + unsafe + { + fixed (byte* pBytes = &buffer.DangerousGetPinnableReference()) + { + char* inp = value; + byte* outp = pBytes; + + while (n > 0) + { + char c = *inp; + if (c > 0xFF) return false; + *outp++ = (byte)(*inp++); + --n; + } + } + } + return true; + } +#endif + +#if ENABLE_SPAN + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// This method is optimized for strings that usually contain only ASCII characters. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromString([CanBeNull] string value) + { + //REVIEW: what if people call FromString"\xFF/some/system/path") by mistake? + // Should be special case when the string starts with \xFF (or \xFF\xFF)? What about \xFE ? + if (value == null) return default(Slice); + byte[] _ = null; + return FromString(value.AsSpan(), ref _); + } + + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// This method is optimized for strings that usually contain only ASCII characters. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromString(ReadOnlySpan value) + { + byte[] _ = null; + return FromString(value, ref _); + } + + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// This method is optimized for strings that usually contain only ASCII characters. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + public static Slice FromString(ReadOnlySpan value, ref byte[] buffer) + { + if (value.Length == 0) return Empty; + if (UnsafeHelpers.IsAsciiString(value)) + { + return ConvertByteStringNoCheck(value, ref buffer); + } + + unsafe + { + fixed (char* chars = &MemoryMarshal.GetReference(value)) + { + int capa = Utf8Encoder.GetByteCount(chars, value.Length); + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, capa); + fixed (byte* ptr = &tmp[0]) + { + if (Utf8NoBomEncoding.GetBytes(chars, value.Length, ptr, capa) != capa) + { +#if DEBUG + // uhoh, on a une désynchro entre GetByteCount() et ce que l'encoding a réellement généré?? + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + throw new InvalidOperationException("UTF-8 byte capacity estimation failed."); + } + return new Slice(tmp, 0, capa); + } + } + } + } + +#else + + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// This method is optimized for strings that usually contain only ASCII characters. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromString([CanBeNull] string value) + { + //REVIEW: what if people call FromString"\xFF/some/system/path") by mistake? + // Should be special case when the string starts with \xFF (or \xFF\xFF)? What about \xFE ? + if (value == null) return default(Slice); + byte[] _ = null; + unsafe + { + fixed(char* chars = value) + { + return FromString(chars, value.Length, ref _); + } + } + } + + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// This method is optimized for strings that usually contain only ASCII characters. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + public static unsafe Slice FromString(char* chars, int numChars, ref byte[] buffer) + { + if (numChars == 0) return Empty; + if (UnsafeHelpers.IsAsciiString(chars, numChars)) + { + return ConvertByteStringNoCheck(chars, numChars, ref buffer); + } + + int capa = Encoding.UTF8.GetByteCount(chars, numChars); + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, capa); + fixed (byte* ptr = &tmp[0]) + { + if (Utf8NoBomEncoding.GetBytes(chars, numChars, ptr, capa) != capa) + { +#if DEBUG + // uhoh, on a une désynchro entre GetByteCount() et ce que l'encoding a réellement généré?? + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + throw new InvalidOperationException("UTF-8 byte capacity estimation failed."); + } + return new Slice(tmp, 0, capa); + } + } + +#endif + + /// Create a slice containing the UTF-8 bytes of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + public static Slice FromStringUtf8([CanBeNull] string value) + { + //REVIEW: what if people call FromString"\xFF/some/system/path") by mistake? + // Should be special case when the string starts with \xFF (or \xFF\xFF)? What about \xFE ? + return value == null ? Slice.Nil + : value.Length == 0 ? Slice.Empty + : new Slice(Utf8NoBomEncoding.GetBytes(value)); + } + +#if ENABLE_SPAN + /// Create a slice containing the UTF-8 bytes of subsection of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, ContractAnnotation("=> buffer:notnull")] + [Obsolete("Use FromStringUtf8(ReadOnlySpan, ...) instead")] + public static Slice FromStringUtf8([NotNull] string value, [Positive] int offset, [Positive] int count, ref byte[] buffer, out bool asciiOnly) + { + if (count == 0) + { + asciiOnly = true; + return Empty; + } + return FromStringUtf8(value.AsSpan(offset, count), ref buffer, out asciiOnly); + } + + /// Create a slice containing the UTF-8 bytes of subsection of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + public static Slice FromStringUtf8(ReadOnlySpan value) + { + if (value.Length == 0) return Empty; + byte[] __ = null; + return FromStringUtf8(value, ref __, out _); + } + + /// Create a slice containing the UTF-8 bytes of subsection of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + public static Slice FromStringUtf8(ReadOnlySpan value, ref byte[] buffer, out bool asciiOnly) + { + if (value.Length == 0) + { + asciiOnly = true; + return Empty; + } + + unsafe + { + //note: there is no direct way to GetBytes(..) from a segment of a string, without going to char pointers :( + fixed (char* inp = &MemoryMarshal.GetReference(value)) + { + int len = Utf8NoBomEncoding.GetByteCount(inp, value.Length); + Contract.Assert(len > 0); + + //TODO: we could optimize conversion if we know it is only ascii! + asciiOnly = len == value.Length; + + // write UTF-8 bytes to buffer + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, len); + fixed (byte* outp = &tmp[0]) + { + //TODO: PERF: if len == count, we know it is ASCII only and could optimize for that case? + if (len != Utf8NoBomEncoding.GetBytes(inp, value.Length, outp, len)) + { +#if DEBUG + // uhoh, y a mismatch entre GetByteCount() et l'encoding UTF-8! + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + throw new InvalidOperationException("UTF-8 string size estimation failed."); + } + return new Slice(tmp, 0, len); + } + } + } + } +#else + /// Create a slice containing the UTF-8 bytes of subsection of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, ContractAnnotation("=> buffer:notnull")] + [Obsolete("Use FromStringUtf8(ReadOnlySpan, ...) instead")] + public static Slice FromStringUtf8([NotNull] string value, [Positive] int offset, [Positive] int count, ref byte[] buffer, out bool asciiOnly) + { + Contract.DoesNotOverflow(value, offset, count); + if (count == 0) + { + asciiOnly = true; + return Empty; + } + + unsafe + { + fixed(char* chars = value) + { + return FromStringUtf8(chars + offset, count, ref buffer, out asciiOnly); + } + } + } + + /// Create a slice containing the UTF-8 bytes of subsection of the string . + /// + /// The slice will NOT include the UTF-8 BOM. + /// This method will not try to identify ASCII-only strings: + /// - If the string provided can ONLY contain ASCII, you should use . + /// - If it is more frequent for the string to be ASCII-only than having UNICODE characters, consider using . + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + public static unsafe Slice FromStringUtf8(char* chars, int numChars, ref byte[] buffer, out bool asciiOnly) + { + if (numChars == 0) + { + asciiOnly = true; + return Empty; + } + + //note: there is no direct way to GetBytes(..) from a segment of a string, without going to char pointers :( + int len = Utf8NoBomEncoding.GetByteCount(chars, numChars); + Contract.Assert(len > 0); + + //TODO: we could optimize conversion if we know it is only ascii! + asciiOnly = len == numChars; + + // write UTF-8 bytes to buffer + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, len); + fixed (byte* outp = &tmp[0]) + { + //TODO: PERF: if len == count, we know it is ASCII only and could optimize for that case? + if (len != Utf8NoBomEncoding.GetBytes(chars, numChars, outp, len)) + { +#if DEBUG + // uhoh, y a mismatch entre GetByteCount() et l'encoding UTF-8! + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + throw new InvalidOperationException("UTF-8 string size estimation failed."); + } + return new Slice(tmp, 0, len); + } + } +#endif + +#if ENABLE_SPAN + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromStringUtf8WithBom([CanBeNull] string value) + { + //REVIEW: what if people call FromString"\xFF/some/system/path") by mistake? + // Should be special case when the string starts with \xFF (or \xFF\xFF)? What about \xFE ? + if (value == null) return default; + byte[] _ = null; + return FromStringUtf8WithBom(value.AsSpan(), ref _); + } + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromStringUtf8WithBom(ReadOnlySpan value) + { + byte[] _ = null; + return FromStringUtf8WithBom(value, ref _); + } + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + public static Slice FromStringUtf8WithBom(ReadOnlySpan value, ref byte[] buffer) + { + if (value.Length == 0) + { + //note: cannot use a singleton buffer because it could be mutated by the caller! + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, 8); + tmp[0] = 0xEF; + tmp[1] = 0xBB; + tmp[2] = 0xBF; + return new Slice(tmp, 0, 3); + } + unsafe + { + fixed (char* pchars = &MemoryMarshal.GetReference(value)) + { + int capa = checked(3 + Utf8NoBomEncoding.GetByteCount(pchars, value.Length)); + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, capa); + fixed (byte* outp = &tmp[0]) + { + outp[0] = 0xEF; + outp[1] = 0xBB; + outp[2] = 0xBF; + Utf8NoBomEncoding.GetBytes(pchars, value.Length, outp + 3, tmp.Length - 3); + } + return new Slice(tmp, 0, capa); + } + } + } + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + private static Slice ConvertByteStringNoCheck(ReadOnlySpan value, ref byte[] buffer) + { + int len = value.Length; + if (len == 0) return Empty; + if (len == 1) return FromByte((byte) value[0]); + + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, len); + unsafe + { + fixed (byte* pBytes = &tmp[0]) + fixed (char* pChars = &MemoryMarshal.GetReference(value)) + { + byte* outp = pBytes; + byte* stop = pBytes + len; + char* inp = pChars; + while (outp < stop) + { + *outp++ = (byte) *inp++; + } + } + } + return new Slice(tmp, 0, len); + } +#else + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromStringUtf8WithBom([CanBeNull] string value) + { + //REVIEW: what if people call FromString"\xFF/some/system/path") by mistake? + // Should be special case when the string starts with \xFF (or \xFF\xFF)? What about \xFE ? + if (value == null) return default(Slice); + byte[] _ = null; + unsafe + { + fixed(char* chars = value) + { + return FromStringUtf8WithBom(chars, value.Length, ref _); + } + } + } + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + public static unsafe Slice FromStringUtf8WithBom(char* chars, int numChars, ref byte[] buffer) + { + if (numChars == 0) + { + //note: cannot use a singleton buffer because it could be mutated by the caller! + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, 8); + tmp[0] = 0xEF; + tmp[1] = 0xBB; + tmp[2] = 0xBF; + return new Slice(tmp, 0, 3); + } + else + { + int capa = checked(3 + Utf8NoBomEncoding.GetByteCount(chars, numChars)); + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, capa); + fixed (byte* outp = &tmp[0]) + { + outp[0] = 0xEF; + outp[1] = 0xBB; + outp[2] = 0xBF; + Utf8NoBomEncoding.GetBytes(chars, numChars, outp + 3, tmp.Length - 3); + } + return new Slice(tmp, 0, capa); + } + } + + /// Create a slice containing the UTF-8 bytes of the string , prefixed by the UTF-8 BOM. + /// + /// If the string is null, an empty slice is returned. + /// If the string is empty, the UTF-8 BOM is returned. + /// DO NOT call this method to encode special strings that contain binary prefixes, like "\xFF/some/system/path" or "\xFE\x01\x02\x03", because they do not map to UTF-8 directly. + /// For these case, or when you known that the string only contains ASCII only (with 100% certainty), you should use . + /// + [Pure] + private static unsafe Slice ConvertByteStringNoCheck(char* chars, int numChars, ref byte[] buffer) + { + if (numChars == 0) return Empty; + if (numChars == 1) return FromByte((byte) chars[0]); + + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, numChars); + unsafe + { + fixed (byte* pBytes = &tmp[0]) + { + byte* outp = pBytes; + byte* stop = pBytes + numChars; + char* inp = chars; + while (outp < stop) + { + *outp++ = (byte) *inp++; + } + } + } + return new Slice(tmp, 0, numChars); + } +#endif + + /// Create a slice that holds the UTF-8 encoded representation of + /// + /// 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 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice FromChar(char value) + { + if (value < 128) + { // ASCII + return FromByte((byte)value); + } + + byte[] _ = null; + return FromChar(value, ref _); + } + + /// Create a slice that holds the UTF-8 encoded representation of + /// 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 + [Pure] + public static Slice FromChar(char value, ref byte[] buffer) + { + if (value < 128) + { // ASCII + return Slice.FromByte((byte)value); + } + + // note: Encoding.UTF8.GetMaxByteCount(1) returns 6, but allocate 8 to stay aligned + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, 8); + unsafe + { + fixed (byte* ptr = &tmp[0]) + { + int n = Utf8NoBomEncoding.GetBytes(&value, 1, ptr, tmp.Length); + return n == 1 ? FromByte(tmp[0]) : new Slice(tmp, 0, n); + } + } + } + + /// Convert an hexadecimal digit (0-9A-Fa-f) into the corresponding decimal value + /// Hexadecimal digit (case insensitive) + /// Decimal value between 0 and 15, or an exception + [Pure] + private static int NibbleToDecimal(char c) + { + int x = c - 48; + if (x < 10) return x; + if (x >= 17 && x <= 42) return x - 7; + if (x >= 49 && x <= 74) return x - 39; + return ThrowInputNotValidHexadecimalDigit(); + } + + private static int ThrowInputNotValidHexadecimalDigit() + { + throw FailInputNotValidHexadecimalDigit(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static FormatException FailInputNotValidHexadecimalDigit() + { + return ThrowHelper.FormatException("Input is not a valid hexadecimal digit"); + } + + /// Convert an hexadecimal encoded string ("1234AA7F") into a slice + /// String contains a sequence of pairs of hexadecimal digits with no separating spaces. + /// Slice containing the decoded byte array, or an exeception if the string is empty or has an odd length + [Pure] + public static Slice FromHexa([CanBeNull] string hexaString) + { + if (string.IsNullOrEmpty(hexaString)) return hexaString == null ? Slice.Nil : Slice.Empty; + + if (hexaString.IndexOf(' ') > 0) + { // remove spaces + hexaString = hexaString.Replace(" ", ""); + } + + if ((hexaString.Length & 1) != 0) throw new ArgumentException("Hexadecimal string must be of even length", nameof(hexaString)); + + var buffer = new byte[hexaString.Length >> 1]; + for (int i = 0; i < hexaString.Length; i += 2) + { + buffer[i >> 1] = (byte)((NibbleToDecimal(hexaString[i]) << 4) | NibbleToDecimal(hexaString[i + 1])); + } + return new Slice(buffer); + } + + #endregion + + #region ToXXX + + /// Stringify a slice containing characters in the operating system's current ANSI codepage + /// Decoded string, or null if the slice is + /// + /// Calling this method on a slice that is not ANSI, or was generated with different codepage than the current process, will return a corrupted string! + /// This method should ONLY be used to interop with the Win32 API or unamanged libraries that require the ANSI codepage! + /// You SHOULD *NOT* use this to expose data to other systems or locale (via sockets, files, ...) + /// If you are decoding natural text, you should probably change the encoding at the source to be UTF-8! + /// If you are decoding identifiers or keywords that are known to be ASCII only, you should use instead (safe). + /// If these identifiers can contain 'special' bytes (like \xFF or \xFE), you should use instead (unsafe). + /// + [Pure, CanBeNull] + public string ToStringAnsi() + { + if (this.Count == 0) return this.Array != null ? String.Empty : default(string); + //note: Encoding.GetString() will do the bound checking for us + return Slice.DefaultEncoding.GetString(this.Array, this.Offset, this.Count); + } + + /// Stringify a slice containing 7-bit ASCII characters only + /// Decoded string, or null if the slice is null + /// + /// This method should ONLY be used to decoded data that is GUARANTEED to be in the range 0..127. + /// This method will THROW if any byte in the slice has bit 7 set to 1 (ie: >= 0x80) + /// If you are decoding identifiers or keywords with 'special' bytes (like \xFF or \xFE), you should use instead. + /// If you are decoding natural text, or text from unknown origin, you should use or instead. + /// If you are attempting to decode a string obtain from a Win32 or unamanged library call, you should use instead. + /// + [Pure, CanBeNull] + public string ToStringAscii() + { + if (this.Count == 0) + { + return this.Array != null ? String.Empty : default(string); + } + if (UnsafeHelpers.IsAsciiBytes(this.Array, this.Offset, this.Count)) + { + return UnsafeHelpers.ConvertToByteString(this.Array, this.Offset, this.Count); + } + throw new DecoderFallbackException("The slice contains at least one non-ASCII character"); + } + + /// Stringify a slice containing only ASCII chars + /// ASCII string, or null if the slice is null + [Pure, CanBeNull] + public string ToByteString() //REVIEW: rename to ToStringSOMETHING(): ToStringByte()? ToStringRaw()? + { + return this.Count == 0 + ? (this.Array != null ? String.Empty : default(string)) + : UnsafeHelpers.ConvertToByteString(this.Array, this.Offset, this.Count); + } + + + + /// Stringify a slice containing either 7-bit ASCII, or UTF-8 characters + /// Decoded string, or null if the slice is null. The encoding will be automatically detected + /// + /// This should only be used for slices produced by any of the , , , or methods. + /// This is NOT compatible with slices produced by or encoded with any specific encoding or code page. + /// This method will NOT automatically remove the UTF-8 BOM if present (use if you need this) + /// + [Pure, CanBeNull] + public string ToUnicode() //REVIEW: rename this to ToStringUnicode() ? + { + var array = this.Array; + int count = this.Count; + int offset = this.Offset; + return count == 0 ? (array != null ? String.Empty : default(string)) + : UnsafeHelpers.IsAsciiBytes(array, offset, count) ? UnsafeHelpers.ConvertToByteString(array, offset, count) + : Utf8NoBomEncoding.GetString(array, offset, count); + } + + [Pure] + private static bool HasUtf8Bom([NotNull] byte[] array, int offset, int count) + { + return count >= 3 + && (uint) (offset + count) <= (uint) array.Length + && array[offset + 0] == 0xEF + && array[offset + 1] == 0xBB + && array[offset + 2] == 0xBF; + } + + /// Decode a slice that is known to contain an UTF-8 encoded string with an optional UTF-8 BOM + /// Decoded string, or null if the slice is null + /// If the slice contains one or more invalid UTF-8 sequences + /// + /// This method will THROW if the slice does not contain valid UTF-8 sequences. + /// This method will remove any UTF-8 BOM if present. If you need to keep the BOM as the first character of the string, use + /// + [Pure, CanBeNull] + public string ToStringUtf8() + { + int count = this.Count; + var array = this.Array; + if (count == 0) return array != null ? String.Empty : default(string); + + // detect BOM + int offset = this.Offset; + if (HasUtf8Bom(array, offset, count)) + { // skip it! + offset += 3; + count -= 3; + if (count == 0) return String.Empty; + } + return Slice.Utf8NoBomEncoding.GetString(array, offset, count); + } + + /// Converts a slice using Base64 encoding + [Pure, CanBeNull] + public string ToBase64() + { + if (this.Count == 0) return this.Array != null ? String.Empty : default(string); + //note: Convert.ToBase64String() will do the bound checking for us + return Convert.ToBase64String(this.Array, this.Offset, this.Count); + } + + /// Converts a slice into a string with each byte encoded into hexadecimal (lowercase) + /// If true, produces lowercase hexadecimal (a-f); otherwise, produces uppercase hexadecimal (A-F) + /// "0123456789abcdef" + [Pure, NotNull] + public string ToHexaString(bool lower = false) + { + return FormatHexaString(this.Array, this.Offset, this.Count, '\0', lower); + } + + /// Converts a slice into a string with each byte encoded into hexadecimal (uppercase) separated by a char + /// Character used to separate the hexadecimal pairs (ex: ' ') + /// If true, produces lowercase hexadecimal (a-f); otherwise, produces uppercase hexadecimal (A-F) + /// "01 23 45 67 89 ab cd ef" + [Pure, NotNull] + public string ToHexaString(char sep, bool lower = false) + { + return FormatHexaString(this.Array, this.Offset, this.Count, sep, lower); + } + + [Pure, NotNull] + internal static string FormatHexaString(byte[] buffer, int offset, int count, char sep, bool lower) + { + if (count == 0) return String.Empty; + UnsafeHelpers.EnsureBufferIsValidNotNull(buffer, offset, count); + + var sb = new StringBuilder(count * (sep == '\0' ? 2 : 3)); + int letters = lower ? 87 : 55; + while (count-- > 0) + { + if (sep != '\0' && sb.Length > 0) sb.Append(sep); + byte b = buffer[offset++]; + int x = b >> 4; + sb.Append((char)(x + (x < 10 ? 48 : letters))); + x = b & 0xF; + sb.Append((char)(x + (x < 10 ? 48 : letters))); + } + return sb.ToString(); + } + + [NotNull] + private static StringBuilder EscapeString(StringBuilder sb, [NotNull] byte[] buffer, int offset, int count, [NotNull] Encoding encoding) + { + if (sb == null) sb = new StringBuilder(count + 16); + foreach (var c in encoding.GetChars(buffer, offset, count)) + { + if ((c >= ' ' && c <= '~') || (c >= 880 && c <= 2047) || (c >= 12352 && c <= 12591)) + sb.Append(c); + else if (c == 0) + sb.Append(@"\0"); + else if (c == '\n') + sb.Append(@"\n"); + else if (c == '\r') + sb.Append(@"\r"); + else if (c == '\t') + sb.Append(@"\t"); + else if (c > 127) + sb.Append(@"\u").Append(((int)c).ToString("x4", CultureInfo.InvariantCulture)); + else // pas clean! + sb.Append(@"\x").Append(((int)c).ToString("x2", CultureInfo.InvariantCulture)); + } + return sb; + } + + /// Helper method that dumps the slice as a string (if it contains only printable ascii chars) or an hex array if it contains non printable chars. It should only be used for logging and troubleshooting ! + /// Returns either "'abc'", "<00 42 7F>", or "{ ...JSON... }". Returns "''" for Slice.Empty, and "" for + [Pure, NotNull] + public string PrettyPrint() + { + if (this.Count == 0) return this.Array != null ? "''" : String.Empty; + return PrettyPrint(this.Array, this.Offset, this.Count, 1024); //REVIEW: constant for max size! + } + + /// Helper method that dumps the slice as a string (if it contains only printable ascii chars) or an hex array if it contains non printable chars. It should only be used for logging and troubleshooting ! + /// Truncate the slice if it exceeds this size + /// Returns either "'abc'", "<00 42 7F>", or "{ ...JSON... }". Returns "''" for Slice.Empty, and "" for + [Pure, NotNull] + public string PrettyPrint(int maxLen) + { + if (this.Count == 0) return this.Array != null ? "''" : String.Empty; + return PrettyPrint(this.Array, this.Offset, this.Count, maxLen); + } + + [Pure, NotNull] + internal static string PrettyPrint([NotNull] byte[] buffer, int offset, int count, int maxLen) + { + if (count == 0) return "''"; + + // look for UTF-8 BOM + if (count >= 3 && buffer[offset] == 0xEF && buffer[offset + 1] == 0xBB && buffer[offset + 2] == 0xBF) + { // this is supposed to be an UTF-8 string + return EscapeString(new StringBuilder(count).Append('\''), buffer, offset + 3, Math.Min(count - 3, maxLen), Slice.Utf8NoBomEncoding).Append('\'').ToString(); + } + + if (count >= 2) + { + // look for JSON objets or arrays + if ((buffer[offset] == '{' && buffer[offset + count - 1] == '}') || (buffer[offset] == '[' && buffer[offset + count - 1] == ']')) + { + try + { + if (count <= maxLen) + { + return EscapeString(new StringBuilder(count + 16), buffer, offset, count, Slice.Utf8NoBomEncoding).ToString(); + } + else + { + return + EscapeString(new StringBuilder(count + 16), buffer, offset, maxLen, Slice.Utf8NoBomEncoding) + .Append("[\u2026]") + .Append(buffer[offset + count - 1]) + .ToString(); + } + } + catch (System.Text.DecoderFallbackException) + { + // sometimes, binary data "looks" like valid JSON but is not, so we just ignore it (even if we may have done a bunch of work for nothing) + } + } + } + + // do a first path on the slice to look for binary of possible text + bool mustEscape = false; + int n = count; + int p = offset; + while (n-- > 0) + { + byte b = buffer[p++]; + if (b >= 32 && b < 127) continue; + + // we accept via escaping the following special chars: CR, LF, TAB + if (b == 0 || b == 10 || b == 13 || b == 9) + { + mustEscape = true; + continue; + } + + //TODO: are there any chars above 128 that could be accepted ? + + // this looks like binary + //return "<" + FormatHexaString(buffer, offset, count, ' ', false) + ">"; + return Slice.Dump(new Slice(buffer, offset, count), maxLen); + } + + if (!mustEscape) + { // only printable chars found + if (count <= maxLen) + { + return "'" + Encoding.ASCII.GetString(buffer, offset, count) + "'"; + } + else + { + return "'" + Encoding.ASCII.GetString(buffer, offset, maxLen) + "[\u2026]'"; // Unicode for '...' + } + } + // some escaping required + if (count <= maxLen) + { + return EscapeString(new StringBuilder(count + 2).Append('\''), buffer, offset, count, Slice.Utf8NoBomEncoding).Append('\'').ToString(); + } + else + { + return EscapeString(new StringBuilder(count + 2).Append('\''), buffer, offset, maxLen, Slice.Utf8NoBomEncoding).Append("[\u2026]'").ToString(); + } + } + + /// Converts a slice into a byte + /// Value of the first and only byte of the slice, or 0 if the slice is null or empty. + /// If the slice has more than one byte + [Pure] + public byte ToByte() + { + switch (this.Count) + { + case 0: return 0; + case 1: return this.Array[this.Offset]; + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(1); + } + } + + /// Converts a slice into a signed byte (-128..+127) + /// Value of the first and only byte of the slice, or 0 if the slice is null or empty. + /// If the slice has more than one byte + [Pure] + public sbyte ToSByte() + { + switch (this.Count) + { + case 0: return 0; + case 1: return (sbyte)this.Array[this.Offset]; + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(1); + } + } + + /// Converts a slice into a boolean. + /// False if the slice is empty, or is equal to the byte 0; otherwise, true. + [Pure] + public bool ToBool() + { + EnsureSliceIsValid(); + // Anything appart from nil/empty, or the byte 0 itself is considered truthy. + return this.Count > 1 || (this.Count == 1 && this.Array[this.Offset] != 0); + //TODO: consider checking if the slice consist of only zeroes ? (ex: Slice.FromFixed32(0) could be considered falsy ...) + } + + #region 16 bits... + + /// Converts a slice into a little-endian encoded, signed 16-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 2 bytes + /// If there are more than 2 bytes in the slice + [Pure] + public short ToInt16() + { + switch (this.Count) + { + case 0: return 0; + case 1: return this.Array[this.Offset]; + case 2: return (short)(this.Array[this.Offset] | (this.Array[this.Offset + 1] << 8)); + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(2); + } + } + + /// Converts a slice into a big-endian encoded, signed 16-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 2 bytes + /// If there are more than 2 bytes in the slice + [Pure] + public short ToInt16BE() + { + EnsureSliceIsValid(); + switch (this.Count) + { + case 0: return 0; + case 1: return this.Array[this.Offset]; + case 2: return (short)(this.Array[this.Offset + 1] | (this.Array[this.Offset] << 8)); + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(2); + + } + } + + /// Converts a slice into a little-endian encoded, unsigned 16-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 2 bytes + /// If there are more than 2 bytes in the slice + [Pure] + public ushort ToUInt16() + { + EnsureSliceIsValid(); + switch (this.Count) + { + case 0: return 0; + case 1: return this.Array[this.Offset]; + case 2: return (ushort)(this.Array[this.Offset] | (this.Array[this.Offset + 1] << 8)); + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(2); + } + } + + /// Converts a slice into a little-endian encoded, unsigned 16-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 2 bytes + /// If there are more than 2 bytes in the slice + [Pure] + public ushort ToUInt16BE() + { + EnsureSliceIsValid(); + switch (this.Count) + { + case 0: return 0; + case 1: return this.Array[this.Offset]; + case 2: return (ushort)(this.Array[this.Offset + 1] | (this.Array[this.Offset] << 8)); + default: + if (this.Count < 0) throw UnsafeHelpers.Errors.SliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(2); + + } + } + + /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 2) + /// Decoded unsigned short. + /// If is less than zero, or more than 2. + [Pure] + public ushort ReadUInt16(int offset, int bytes) + { + if ((uint) bytes > 2) goto fail; + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + switch (bytes) + { + case 0: return 0; + case 1: return buffer[p]; + default: return (ushort)(buffer[p] | (buffer[p + 1] << 8)); + } + fail: + throw new ArgumentOutOfRangeException(nameof(bytes)); + } + + /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 2) + /// Decoded unsigned short. + /// If is less than zero, or more than 2. + [Pure] + public ushort ReadUInt16BE(int offset, int bytes) + { + if ((uint) bytes > 2) goto fail; + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + switch (bytes) + { + case 0: return 0; + case 1: return buffer[p]; + default: return (ushort)(buffer[p + 1] | (buffer[p] << 8)); + } + fail: + throw new ArgumentOutOfRangeException(nameof(bytes)); + } + + #endregion + + #region 24 bits... + + //note: all 'Int24' and 'UInt24' are represented in memory as Int32/UInt32 using only the lowest 24 bits (upper 8 bits will be IGNORED) + //note: 'FF FF' is equivalent to '00 FF FF', so is considered to be positive (= 65535) + + /// Converts a slice into a little-endian encoded, signed 24-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 3 bytes + /// If there are more than 3 bytes in the slice + [Pure] + public int ToInt24() + { + EnsureSliceIsValid(); + int count = this.Count; + if (count == 0) return 0; + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + switch (count) + { + case 1: return *ptr; + case 2: return UnsafeHelpers.LoadUInt16LE(ptr); // cannot be negative + case 3: return UnsafeHelpers.LoadInt24LE(ptr); + } + } + } + if (count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(3); + } + + /// Converts a slice into a big-endian encoded, signed 24-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 3 bytes + /// If there are more than 3 bytes in the slice + [Pure] + public int ToInt24BE() + { + EnsureSliceIsValid(); + int count = this.Count; + if (count == 0) return 0; + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + switch (count) + { + case 1: return *ptr; + case 2: return UnsafeHelpers.LoadUInt16BE(ptr); + case 3: return UnsafeHelpers.LoadInt24BE(ptr); + } + } + } + if (count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(3); + } + + /// Converts a slice into a little-endian encoded, unsigned 24-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 3 bytes + /// If there are more than 3 bytes in the slice + [Pure] + public uint ToUInt24() + { + EnsureSliceIsValid(); + int count = this.Count; + if (count == 0) return 0; + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + switch (count) + { + case 1: return *ptr; + case 2: return UnsafeHelpers.LoadUInt16LE(ptr); + case 3: return UnsafeHelpers.LoadUInt24LE(ptr); + } + } + } + if (count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(3); + } + + /// Converts a slice into a little-endian encoded, unsigned 24-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 3 bytes + /// If there are more than 3 bytes in the slice + [Pure] + public uint ToUInt24BE() + { + EnsureSliceIsValid(); + int count = this.Count; + if (count == 0) return 0; + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + switch (count) + { + case 1: return *ptr; + case 2: return UnsafeHelpers.LoadUInt16BE(ptr); + case 3: return UnsafeHelpers.LoadUInt24BE(ptr); + } + } + } + if (count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(3); + } + + /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 2) + /// Decoded unsigned short. + /// If is less than zero, or more than 3. + [Pure] + public uint ReadUInt24(int offset, int bytes) + { + if ((uint) bytes > 3) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + switch (bytes) + { + case 0: return 0; + case 1: return buffer[p]; + case 2: return (uint)(buffer[p] | (buffer[p + 1] << 8)); + default: return (uint)(buffer[p] | (buffer[p + 1] << 8) | (buffer[p + 2] << 16)); + } + } + + /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 2) + /// Decoded unsigned short. + /// If is less than zero, or more than 3. + [Pure] + public ushort ReadUInt24BE(int offset, int bytes) + { + if ((uint) bytes > 3) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + switch (bytes) + { + case 0: return 0; + case 1: return buffer[p]; + case 2: return (ushort)(buffer[p + 1] | (buffer[p] << 8)); + default: return (ushort)(buffer[p + 2] | (buffer[p + 1] << 8) | (buffer[p] << 16)); + } + } + + #endregion + + #region 32 bits... + + /// Converts a slice into a little-endian encoded, signed 32-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 4 bytes + /// If there are more than 4 bytes in the slice + [Pure] + public int ToInt32() + { + // note: we ensure that offset is not negative by doing a cast to uint + uint off = checked((uint)this.Offset); + var arr = this.Array; // if null, whill throw later with a nullref + switch (this.Count) // if negative, will throw in the default case below + { + case 0: return 0; + case 1: return arr[off]; + case 2: return arr[off] | (arr[off + 1] << 8); + case 3: return arr[off] | (arr[off + 1] << 8) | (arr[off + 2] << 16); + case 4: return arr[off] | (arr[off + 1] << 8) | (arr[off + 2] << 16) | (arr[off + 3] << 24); + default: + { + if (this.Count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(4); + } + } + } + + /// Converts a slice into a big-endian encoded, signed 32-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 4 bytes + /// If there are more than 4 bytes in the slice + [Pure] + public int ToInt32BE() + { + // note: we ensure that offset is not negative by doing a cast to uint + uint off = checked((uint)this.Offset); + var arr = this.Array; // if null, whill throw later with a nullref + switch (this.Count) // if negative, will throw in the default case below + { + case 0: return 0; + case 1: return arr[off]; + case 2: return (arr[off] << 8) | arr[off + 1]; + case 3: return (arr[off] << 16) | (arr[off + 1] << 8) | arr[off + 2]; + case 4: return (arr[off] << 24) | (arr[off + 1] << 16) | (arr[off + 2] << 8) | arr[off + 3]; + default: + { + if (this.Count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(4); + } + } + } + + /// Converts a slice into a little-endian encoded, unsigned 32-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes + /// If there are more than 4 bytes in the slice + [Pure] + public uint ToUInt32() + { + // note: we ensure that offset is not negative by doing a cast to uint + uint off = checked((uint)this.Offset); + var arr = this.Array; // if null, whill throw later with a nullref + switch (this.Count) // if negative, will throw in the default case below + { + case 0: return 0; + case 1: return arr[off]; + case 2: return (uint)(arr[off] | (arr[off + 1] << 8)); + case 3: return (uint)(arr[off] | (arr[off + 1] << 8) | (arr[off + 2] << 16)); + case 4: return (uint)(arr[off] | (arr[off + 1] << 8) | (arr[off + 2] << 16) | (arr[off + 3] << 24)); + default: + { + if (this.Count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(4); + } + } + } + + /// Converts a slice into a big-endian encoded, unsigned 32-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes + /// If there are more than 4 bytes in the slice + [Pure] + public uint ToUInt32BE() + { + // note: we ensure that offset is not negative by doing a cast to uint + uint off = checked((uint)this.Offset); + var arr = this.Array; // if null, whill throw later with a nullref + switch (this.Count) // if negative, will throw in the default case below + { + case 0: return 0; + case 1: return arr[off]; + case 2: return (uint)((arr[off] << 8) | arr[off + 1]); + case 3: return (uint)((arr[off] << 16) | (arr[off + 1] << 8) | arr[off + 2]); + case 4: return (uint)((arr[off] << 24) | (arr[off + 1] << 16) | (arr[off + 2] << 8) | arr[off + 3]); + default: + { + if (this.Count < 0) UnsafeHelpers.Errors.ThrowSliceCountNotNeg(); + return UnsafeHelpers.Errors.ThrowSliceTooLargeForConversion(4); + } + } + } + + /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 4) + /// Decoded unsigned integer. + /// If is less than zero, or more than 4. + [Pure] + public uint ReadUInt32(int offset, int bytes) + { + if (bytes == 0) return 0; + if ((uint) bytes > 4) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset) + bytes - 1; + + uint value = buffer[p--]; + while (--bytes > 0) + { + value = (value << 8) | buffer[p--]; + } + return value; + } + + /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 4) + /// Decoded unsigned integer. + /// If is less than zero, or more than 4. + [Pure] + public uint ReadUInt32BE(int offset, int bytes) + { + if (bytes == 0) return 0; + if ((uint) bytes > 4) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + + uint value = buffer[p++]; + while (--bytes > 0) + { + value = (value << 8) | buffer[p++]; + } + return value; + } + + #endregion + + #region 64 bits... + + /// Converts a slice into a little-endian encoded, signed 64-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 8 bytes + /// If there are more than 8 bytes in the slice + [Pure] + public long ToInt64() + { + return this.Count <= 4 ? ToUInt32() : ToInt64Slow(); + } + + [Pure] + private long ToInt64Slow() + { + int n = this.Count; + if ((uint) n > 8) goto fail; + EnsureSliceIsValid(); + + var buffer = this.Array; + int p = this.Offset + n - 1; + + long value = buffer[p--]; + while (--n > 0) + { + value = (value << 8) | buffer[p--]; + } + + return value; + fail: + throw new FormatException("Cannot convert slice into an Int64 because it is larger than 8 bytes"); + } + + /// Converts a slice into a big-endian encoded, signed 64-bit integer. + /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 8 bytes + /// If there are more than 8 bytes in the slice + [Pure] + public long ToInt64BE() + { + return this.Count <= 4 ? ToInt32BE() : ToInt64BESlow(); + } + + [Pure] + private long ToInt64BESlow() + { + int n = this.Count; + if (n == 0) return 0L; + if ((uint) n > 8) goto fail; + EnsureSliceIsValid(); + + var buffer = this.Array; + int p = this.Offset; + + long value = buffer[p++]; + while (--n > 0) + { + value = (value << 8) | buffer[p++]; + } + return value; + fail: + throw new FormatException("Cannot convert slice into an Int64 because it is larger than 8 bytes"); + } + + /// Converts a slice into a little-endian encoded, unsigned 64-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes + /// If there are more than 8 bytes in the slice + [Pure] + public ulong ToUInt64() + { + int n = this.Count; + if (n == 0) return 0L; + if ((uint) n > 8) goto fail; + EnsureSliceIsValid(); + + var buffer = this.Array; + int p = this.Offset + n - 1; + + ulong value = buffer[p--]; + while (--n > 0) + { + value = (value << 8) | buffer[p--]; + } + return value; + fail: + throw new FormatException("Cannot convert slice into an UInt64 because it is larger than 8 bytes"); + } + + /// Converts a slice into a little-endian encoded, unsigned 64-bit integer. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes + /// If there are more than 8 bytes in the slice + [Pure] + public ulong ToUInt64BE() + { + int n = this.Count; + if (n == 0) return 0L; + if ((uint) n > 8) goto fail; + EnsureSliceIsValid(); + + var buffer = this.Array; + int p = this.Offset; + + ulong value = buffer[p++]; + while (--n > 0) + { + value = (value << 8) | buffer[p++]; + } + return value; + fail: + throw new FormatException("Cannot convert slice into an UInt64 because it is larger than 8 bytes"); + } + + /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 8) + /// Decoded unsigned integer. + /// If is less than zero, or more than 8. + [Pure] + public ulong ReadUInt64(int offset, int bytes) + { + if (bytes == 0) return 0UL; + if ((uint) bytes > 8) goto fail; + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset) + bytes - 1; + + ulong value = buffer[p--]; + while (--bytes > 0) + { + value = (value << 8) | buffer[p--]; + } + return value; + fail: + throw new ArgumentOutOfRangeException(nameof(bytes)); + } + + /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice + /// Relative offset of the first byte + /// Number of bytes to read (up to 8) + /// Decoded unsigned integer. + /// If is less than zero, or more than 8. + [Pure] + public ulong ReadUInt64BE(int offset, int bytes) + { + if (bytes == 0) return 0UL; + if ((uint) bytes > 8) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + var buffer = this.Array; + int p = UnsafeMapToOffset(offset); + + ulong value = buffer[p++]; + while (--bytes > 0) + { + value = (value << 8) | buffer[p++]; + } + return value; + } + + /// Converts a slice into a 64-bit UUID. + /// Uuid decoded from the Slice. + /// The slice can either be an 8-byte array, or an ASCII string of 16, 17 or 19 chars + [Pure] + public Uuid64 ToUuid64() + { + if (this.Count == 0) return default(Uuid64); + EnsureSliceIsValid(); + + switch (this.Count) + { + case 8: + { // binary (8 bytes) + return Uuid64.Read(this); + } + + case 16: // hex16 + case 17: // hex8-hex8 + case 19: // {hex8-hex8} + { + // ReSharper disable once AssignNullToNotNullAttribute + return Uuid64.Parse(this.ToByteString()); + } + } + + throw new FormatException("Cannot convert slice into an Uuid64 because it has an incorrect size"); + } + + #endregion + + #region Floating Point... + + /// Converts a slice into a 32-bit IEEE floating point. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes + /// If there are less or more than 4 bytes in the slice + [Pure] + public float ToSingle() + { + if (this.Count == 0) return 0f; + if (this.Count != 4) goto fail; + EnsureSliceIsValid(); + + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + return *((float*)ptr); + } + } + fail: + throw new FormatException("Cannot convert slice into a Single because it is not exactly 4 bytes long."); + } + + /// Converts a slice into a 32-bit IEEE floating point (in network order). + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes + /// If there are less or more than 4 bytes in the slice + [Pure] + public float ToSingleBE() + { + if (this.Count == 0) return 0f; + if (this.Count != 4) goto fail; + EnsureSliceIsValid(); + + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + uint tmp = UnsafeHelpers.ByteSwap32(*(uint*)ptr); + return *((float*) &tmp); + } + } + fail: + throw new FormatException("Cannot convert slice into a Single because it is not exactly 4 bytes long."); + } + + /// Converts a slice into a 64-bit IEEE floating point. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes + /// If there are less or more than 8 bytes in the slice + [Pure] + public double ToDouble() + { + if (this.Count == 0) return 0d; + if (this.Count != 8) goto fail; + EnsureSliceIsValid(); + + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + return *((double*) ptr); + } + } + fail: + throw new FormatException("Cannot convert slice into a Double because it is not exactly 8 bytes long."); + } + + /// Converts a slice into a 64-bit IEEE floating point (in network order). + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes + /// If there are less or more than 8 bytes in the slice + [Pure] + public double ToDoubleBE() + { + if (this.Count == 0) return 0d; + if (this.Count != 8) goto fail; + EnsureSliceIsValid(); + + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + ulong tmp = UnsafeHelpers.ByteSwap64(*(ulong*)ptr); + return *((double*) &tmp); + } + } + fail: + throw new FormatException("Cannot convert slice into a Double because it is not exactly 8 bytes long."); + } + + /// Converts a slice into a 128-bit IEEE floating point. + /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes + /// If there are less or more than 8 bytes in the slice + [Pure] + public decimal ToDecimal() + { + if (this.Count == 0) return 0m; + if (this.Count != 16) goto fail; + EnsureSliceIsValid(); + + unsafe + { + fixed (byte* ptr = &DangerousGetPinnableReference()) + { + return *((decimal*)ptr); + } + } + fail: + throw new FormatException("Cannot convert slice into a Decimal because it is not exactly 16 bytes long."); + } + + #endregion + + #region 128 bits... + + /// Converts a slice into a Guid. + /// Native Guid decoded from the Slice. + /// The slice can either be a 16-byte RFC4122 GUID, or an ASCII string of 36 chars + [Pure] + public Guid ToGuid() + { + if (this.Count == 0) return default(Guid); + EnsureSliceIsValid(); + + if (this.Count == 16) + { // direct byte array + + // UUID are stored using the RFC4122 format (Big Endian), while .NET's System.GUID use Little Endian + // we need to swap the byte order of the Data1, Data2 and Data3 chunks, to ensure that Guid.ToString() will return the proper value. + + return new Uuid128(this).ToGuid(); + } + + if (this.Count == 36) + { // string representation (ex: "da846709-616d-4e82-bf55-d1d3e9cde9b1") + // ReSharper disable once AssignNullToNotNullAttribute + return Guid.Parse(this.ToByteString()); + } + + throw new FormatException("Cannot convert slice into a Guid because it has an incorrect size"); + } + + /// Converts a slice into a 128-bit UUID. + /// Uuid decoded from the Slice. + /// The slice can either be a 16-byte RFC4122 GUID, or an ASCII string of 36 chars + [Pure] + public Uuid128 ToUuid128() + { + if (this.Count == 0) return default(Uuid128); + EnsureSliceIsValid(); + + if (this.Count == 16) + { + return new Uuid128(this); + } + + if (this.Count == 36) + { + // ReSharper disable once AssignNullToNotNullAttribute + return Uuid128.Parse(ToByteString()); + } + + throw new FormatException("Cannot convert slice into an Uuid128 because it has an incorrect size"); + } + + #endregion + + #endregion + } +} diff --git a/FoundationDB.Client/Utils/Memory/Slice.cs b/FoundationDB.Client/Utils/Memory/Slice.cs new file mode 100644 index 000000000..c46e5f395 --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/Slice.cs @@ -0,0 +1,2519 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +//#define ENABLE_SPAN + +namespace System +{ + using System; + using System.Collections.Generic; + using System.ComponentModel; + using System.Diagnostics; + using System.IO; + using System.Linq; + using System.Runtime.CompilerServices; + using System.Runtime.InteropServices; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; + + /// Delimits a section of a byte array + /// A Slice if the logical equivalent to a ReadOnlySpan<byte> + [PublicAPI, ImmutableObject(true), DebuggerDisplay("Count={Count}, Offset={Offset}"), DebuggerTypeProxy(typeof(Slice.DebugView))] + [DebuggerNonUserCode] //remove this when you need to troubleshoot this class! + public /*readonly*/ partial struct Slice : IEquatable, IEquatable>, IEquatable, IComparable, IFormattable + { + #region Static Members... + + /// Null slice ("no segment") + public static readonly Slice Nil = default(Slice); + + /// Empty slice ("segment of 0 bytes") + //note: we allocate a 1-byte array so that we can get a pointer to &slice.Array[slice.Offset] even for the empty slice + public static readonly Slice Empty = new Slice(new byte[1], 0, 0); + + /// Cached array of bytes from 0 to 255 + [NotNull] + internal static readonly byte[] ByteSprite = CreateByteSprite(); + + private static byte[] CreateByteSprite() + { + var tmp = new byte[256]; + for (int i = 0; i < tmp.Length; i++) tmp[i] = (byte) i; + return tmp; + } + + #endregion + + //REVIEW: Layout: should we maybe swap things around? .Count seems to be the most often touched field before the rest + // => Should it be Array/Offset/Count (current), or Count/Offset/Array ? + + /// Pointer to the buffer (or null for ) + public readonly byte[] Array; + + /// Offset of the first byte of the slice in the parent buffer + public readonly int Offset; + + /// Number of bytes in the slice + public readonly int Count; + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal Slice([NotNull] byte[] array, int offset, int count) + { + //Paranoid.Requires(array != null && offset >= 0 && offset <= array.Length && count >= 0 && offset + count <= array.Length); + this.Array = array; + this.Offset = offset; + this.Count = count; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal Slice([NotNull] byte[] array) + { + //Paranoid.Requires(array != null); + this.Array = array; + this.Offset = 0; + this.Count = array.Length; + } + + /// Creates a slice mapping a section of a buffer, without any sanity checks or buffer optimization + /// Original buffer + /// Offset into buffer + /// Number of bytes + /// Slice that maps this segment of buffer. + /// + /// Slice.CreateUnsafe(buffer, 1, 5) => Slice { Array = buffer, Offset = 1, Count = 5 } + /// + /// + /// Use this method ONLY if you are 100% sure that the slice will be valid. Failure to do so may introduce memory corruption! + /// Also, please note that this method will NOT optimize the case where count == 0, and will keep a reference to the original buffer! + /// The caller is responsible for handle that scenario if it is important! + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice CreateUnsafe([NotNull] byte[] buffer, [Positive] int offset, [Positive] int count) + { + Contract.Requires(buffer != null && (uint) offset <= (uint) buffer.Length && (uint) count <= (uint) (buffer.Length - offset)); + return new Slice(buffer, offset, count); + } + + /// Creates a slice mapping a section of a buffer, without any sanity checks or buffer optimization + /// Original buffer + /// Offset into buffer + /// Number of bytes + /// Slice that maps this segment of buffer. + /// + /// Slice.CreateUnsafe(buffer, 1, 5) => Slice { Array = buffer, Offset = 1, Count = 5 } + /// + /// + /// Use this method ONLY if you are 100% sure that the slice will be valid. Failure to do so may introduce memory corruption! + /// Also, please note that this method will NOT optimize the case where count == 0, and will keep a reference to the original buffer! + /// The caller is responsible for handle that scenario if it is important! + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice CreateUnsafe([NotNull] byte[] buffer, uint offset, uint count) + { + Contract.Requires(buffer != null && offset <= (uint) buffer.Length && count <= ((uint) buffer.Length - offset)); + return new Slice(buffer, (int) offset, (int) count); + } + + /// Creates a new empty slice of a specified size containing all zeroes + public static Slice Create(int size) + { + Contract.Positive(size, nameof(size)); + return size != 0 ? new Slice(new byte[size]) : Slice.Empty; + } + + /// Creates a new empty slice of a specified size containing all zeroes + [Pure] + public static Slice Create(uint size) + { + Contract.LessOrEqual(size, int.MaxValue, nameof(size)); + return size != 0 ? new Slice(new byte[size]) : Slice.Empty; + } + + /// Creates a new slice with a copy of the array + [Pure] + public static Slice Copy(byte[] source) + { + Contract.NotNull(source, nameof(source)); + if (source.Length == 0) return Empty; + return Copy(source, 0, source.Length); + } + +#if ENABLE_SPAN + + /// Creates a new slice with a copy of the array segment + [Pure] + public static Slice Copy(byte[] source, int offset, int count) + { + return Copy(new ReadOnlySpan(source, offset, count)); + } + + /// Creates a new slice with a copy of the span + [Pure] + public static Slice Copy(ReadOnlySpan source) + { + if (source.Length == 0) return Empty; + var tmp = source.ToArray(); + return new Slice(tmp, 0, source.Length); + } + + /// Creates a new slice with a copy of the span, using a scratch buffer + [Pure] + public static Slice Copy(ReadOnlySpan source, [CanBeNull] ref byte[] buffer) + { + if (source.Length == 0) return Empty; + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, BitHelpers.NextPowerOfTwo(source.Length)); + UnsafeHelpers.Copy(tmp, 0, source); + return new Slice(tmp, 0, source.Length); + } + +#else + + /// Creates a new slice with a copy of the array segment + [Pure] + public static Slice Copy(byte[] source, int offset, int count) + { + if (count == 0) return source == null ? Nil : Empty; + var tmp = new byte[count]; + UnsafeHelpers.Copy(tmp, 0, source, offset, count); + return new Slice(tmp, 0, count); + } + + /// Creates a new slice with a copy of the span, using a scratch buffer + [Pure] + public static Slice Copy(Slice source, [CanBeNull] ref byte[] buffer) + { + if (source.Count == 0) return source.Array == null ? default(Slice) : Empty; + var tmp = UnsafeHelpers.EnsureCapacity(ref buffer, BitHelpers.NextPowerOfTwo(source.Count)); + UnsafeHelpers.Copy(tmp, 0, source.Array, source.Offset, source.Count); + return new Slice(tmp, 0, source.Count); + } + +#endif + + /// Creates a new slice with a copy of an unmanaged memory buffer + /// Pointer to unmanaged buffer + /// Number of bytes in the buffer + /// Slice with a managed copy of the data + [Pure] + public static Slice Copy(IntPtr source, int count) + { + unsafe + { + return Copy((byte*) source.ToPointer(), count); + } + } + + /// Creates a new slice with a copy of an unmanaged memory buffer + /// Pointer to unmanaged buffer + /// Number of bytes in the buffer + /// Slice with a managed copy of the data + [Pure] + public static unsafe Slice Copy(void * source, int count) + { + return Copy((byte*) source, count); + } + + + /// Creates a new slice with a copy of an unmanaged memory buffer + /// Pointer to unmanaged buffer + /// Number of bytes in the buffer + /// Slice with a managed copy of the data + [Pure] + public static unsafe Slice Copy(byte* source, int count) + { + if (count == 0) + { + return source == null ? default(Slice) : Empty; + } + Contract.PointerNotNull(source, nameof(source)); + Contract.Positive(count, nameof(count)); + + if (count == 1) + { // Use the sprite cache + return Slice.FromByte(*source); + } + + var bytes = new byte[count]; + UnsafeHelpers.CopyUnsafe(bytes, 0, source, (uint) count); + return new Slice(bytes, 0, count); + } + +#if ENABLE_SPAN + /// Return a copy of the memory content of an array of item + public static Slice CopyMemory(ReadOnlySpan items) + where T : struct + { + return Copy(MemoryMarshal.AsBytes(items)); + } + + /// Return a copy of the memory content of an array of item + public static Slice CopyMemory(ReadOnlySpan items, [CanBeNull] ref byte[] buffer) + where T : struct + { + return Copy(MemoryMarshal.AsBytes(items), ref buffer); + } +#endif + + /// Implicitly converts a Slice into an ArraySegment<byte> + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ArraySegment(Slice value) + { + return value.HasValue ? new ArraySegment(value.Array, value.Offset, value.Count) : default(ArraySegment); + } + + /// Implicitly converts an ArraySegment<byte> into a Slice + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator Slice(ArraySegment value) + { + if (value.Count == 0) return value.Array == null ? default(Slice) : Slice.Empty; + return new Slice(value.Array, value.Offset, value.Count); + } + +#if ENABLE_SPAN + /// Converts a Slice into an Span<byte> + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static explicit operator Span(Slice value) + { + //note: explicit because casting to writable Span MAY be dangerous, and we need opt-in from the caller! + return new Span(value.Array, value.Offset, value.Count); + } + + /// Implicitly converts a Slice into an ReadOnlySpan<byte> + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator ReadOnlySpan(Slice value) + { + //note: implicit because casting to non-writable ReadOnlySpan is safe + return new ReadOnlySpan(value.Array, value.Offset, value.Count); + } +#endif + + /// Returns true is the slice is not null + /// An empty slice is NOT considered null + public bool HasValue + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Array != null; } + } + + /// Returns true if the slice is null + /// An empty slice is NOT considered null + public bool IsNull + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Array == null; } + } + + /// Return true if the slice is not null but contains 0 bytes + /// A null slice is NOT empty + public bool IsEmpty + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Count == 0 && this.Array != null; } + } + + /// Returns true if the slice is null or empty, or false if it contains at least one byte + public bool IsNullOrEmpty + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Count == 0; } + } + + /// Returns true if the slice contains at least one byte, or false if it is null or empty + public bool IsPresent + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Count > 0; } + } + + /// Replace with + /// The same slice if it is not ; otherwise, + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice OrEmpty() + { + return this.Count > 0? this : Empty; + } + + /// Return a byte array containing all the bytes of the slice, or null if the slice is null + /// Byte array with a copy of the slice, or null + [Pure, CanBeNull] + public byte[] GetBytes() + { + int len = this.Count; + if (len == 0) return this.Array == null ? null : System.Array.Empty(); + EnsureSliceIsValid(); + + var tmp = new byte[len]; + UnsafeHelpers.CopyUnsafe(tmp, 0, this.Array, this.Offset, len); + return tmp; + } + + /// Return a byte array containing all the bytes of the slice, or and empty array if the slice is null or empty + /// Byte array with a copy of the slice + [Pure, NotNull] + public byte[] GetBytesOrEmpty() + { + //note: this is a convenience method for code where dealing with null is a pain, or where it has already checked IsNull + int len = this.Count; + if (len == 0) return System.Array.Empty(); + EnsureSliceIsValid(); + + var tmp = new byte[len]; + UnsafeHelpers.CopyUnsafe(tmp, 0, this.Array, this.Offset, len); + return tmp; + } + + /// Return a byte array containing a subset of the bytes of the slice, or null if the slice is null + /// Byte array with a copy of a subset of the slice, or null + [Pure, NotNull] + public byte[] GetBytes(int offset, int count) + { + //TODO: throw if this.Array == null ? (what does "Slice.Nil.GetBytes(..., 0)" mean ?) + + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset)); + + int len = this.Count; + if ((uint) count > (uint) len || (uint) count > (uint) (len - offset)) throw new ArgumentOutOfRangeException(nameof(count)); + + if (count == 0) return System.Array.Empty(); + EnsureSliceIsValid(); + + var tmp = new byte[count]; + UnsafeHelpers.CopyUnsafe(tmp, 0, this.Array, this.Offset + offset, count); + return tmp; + } + + /// Return a SliceReader that can decode this slice into smaller fields + [Obsolete("Use ToSliceReader() instead")] + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public SliceReader GetReader() + { + return new SliceReader(this); + } + + /// Return a SliceReader that can decode this slice into smaller fields + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public SliceReader ToSliceReader() + { + return new SliceReader(this); + } + + /// Return a stream that wraps this slice + /// Stream that will read the slice from the start. + /// + /// You can use this method to convert text into specific encodings, load bitmaps (JPEG, PNG, ...), or any serialization format that requires a Stream or TextReader instance. + /// Disposing this stream will have no effect on the slice. + /// + [Pure, NotNull] + public SliceStream ToSliceStream() + { + EnsureSliceIsValid(); + return new SliceStream(this); + } + + /// Returns a new slice that contains an isolated copy of the buffer + /// Slice that is equivalent, but is isolated from any changes to the buffer + [Pure] + public Slice Memoize() + { + if (this.Count == 0) return this.Array == null ? Slice.Nil : Slice.Empty; + // ReSharper disable once AssignNullToNotNullAttribute + return new Slice(GetBytes()); + } + + /// Map an offset in the slice into the absolute offset in the buffer, without any bound checking + /// Relative offset (negative values mean from the end) + /// Absolute offset in the buffer + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private int UnsafeMapToOffset(int index) + { + return this.Offset + NormalizeIndex(index); + } + + /// Map an offset in the slice into the absolute offset in the buffer + /// Relative offset (negative values mean from the end) + /// Absolute offset in the buffer + /// If the index is outside the slice + [MethodImpl(MethodImplOptions.AggressiveInlining)] + private int MapToOffset(int index) + { + int p = NormalizeIndex(index); + if ((uint) p >= (uint) this.Count) UnsafeHelpers.Errors.ThrowIndexOutOfBound(index); + return checked(this.Offset + p); + } + + /// Normalize negative index values into offset from the start + /// Relative offset (negative values mean from the end) + /// Relative offset from the start of the slice + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + private int NormalizeIndex(int index) + { + return index < 0 ? checked(index + this.Count) : index; + } + + /// Returns the value of one byte in the slice + /// Offset of the byte (negative values means start from the end) + public byte this[int index] + { + [MethodImpl(MethodImplOptions.AggressiveInlining)] + get { return this.Array[MapToOffset(index)]; } + } + +#if ENABLE_SPAN + /// Returns a reference to a specific position in the slice + [MethodImpl(MethodImplOptions.AggressiveInlining)] + [EditorBrowsable(EditorBrowsableState.Never)] + public ref readonly byte ItemRef(int index) + { + return ref this.Array[MapToOffset(index)]; + } +#endif + + /// Returns a substring of the current slice that fits withing the specified index range + /// The starting position of the substring. Positive values means from the start, negative values means from the end + /// The end position (excluded) of the substring. Positive values means from the start, negative values means from the end + /// Subslice + public Slice this[int start, int end] + { + get + { + start = NormalizeIndex(start); + end = NormalizeIndex(end); + + // bound check + if (start < 0) start = 0; + if (end > this.Count) end = this.Count; + + if (start >= end) return Slice.Empty; + if (start == 0 && end == this.Count) return this; + + checked { return new Slice(this.Array, this.Offset + start, end - start); } + } + } + + /// + /// Returns a reference to the first byte in the slice. + /// If the slice is empty, returns a reference to the location where the first character would have been stored. + /// Such a reference can be used for pinning but must never be dereferenced. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + [EditorBrowsable(EditorBrowsableState.Never)] + public ref byte DangerousGetPinnableReference() + { + //note: this is the equivalent of MemoryMarshal.GetReference(..) and does not check for the 0-length case! + return ref this.Array[this.Offset]; + } + +#if ENABLE_SPAN + /// + /// Returns a reference to the 0th element of the Span. If the Span is empty, returns null reference. + /// It can be used for pinning and is required to support the use of span within a fixed statement. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + [EditorBrowsable(EditorBrowsableState.Never)] + public ref byte GetPinnableReference() + { + unsafe + { + return ref (this.Count != 0) ? ref this.Array[this.Offset] : ref Unsafe.AsRef(null); + } + } +#endif + + /// Copy this slice into another buffer, and move the cursor + /// Buffer where to copy this slice + /// Offset into the destination buffer + public void WriteTo([NotNull] byte[] buffer, ref int cursor) + { + //note: CopyBytes will validate all the parameters + int count = this.Count; + UnsafeHelpers.Copy(buffer, cursor, this.Array, this.Offset, count); + cursor += count; + } + + public void CopyTo(Slice destination) + { + if (destination.Count < this.Count) throw UnsafeHelpers.Errors.SliceBufferTooSmall(); + UnsafeHelpers.Copy(destination.Array, destination.Offset, this.Array, this.Offset, this.Count); + } + +#if ENABLE_SPAN + public void CopyTo(Span destination) + { + if (destination.Length < this.Count) throw UnsafeHelpers.Errors.SliceBufferTooSmall(); + UnsafeHelpers.Copy(destination, this.Array, this.Offset, this.Count); + } +#endif + + /// Copy this slice into another buffer + /// Buffer where to copy this slice + /// Offset into the destination buffer + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void CopyTo([NotNull] byte[] buffer, int offset) + { + UnsafeHelpers.Copy(buffer, offset, this.Array, this.Offset, this.Count); + } + + /// Copy this slice into memory and return the advanced cursor + /// Pointer where to copy this slice + /// Pointer to the next byte after the last availble position in the output buffer + /// Copy will fail if there is not enough space in the output buffer (ie: if it would writer at or after ) + [NotNull] + public unsafe byte* CopyToUnsafe([NotNull] byte* ptr, [NotNull] byte* end) + { + if (ptr == null | end == null) throw new ArgumentNullException(ptr == null ? nameof(ptr) : nameof(end)); + long count = this.Count; + byte* next = ptr + count; + if (next > end) throw new ArgumentException("Slice is too large to fit in the specified output buffer"); + if (count > 0) + { + fixed (byte* bytes = &DangerousGetPinnableReference()) + { + Buffer.MemoryCopy(bytes, ptr, count, count); + } + } + return next; + } + + /// Try to copy this slice into memory and return the advanced cursor, if the destination is large enough + /// Pointer where to copy this slice + /// Pointer to the next byte after the last availble position in the output buffer + /// Point to the advanced memory position, or null if the destination buffer was too small + [CanBeNull] + public unsafe byte* TryCopyToUnsafe([NotNull] byte* ptr, [NotNull] byte* end) + { + if (ptr == null | end == null) throw new ArgumentNullException(ptr == null ? nameof(ptr) : nameof(end)); + long count = this.Count; + byte* next = ptr + count; + if (next > end) return null; + if (count > 0) + { + fixed (byte* bytes = &DangerousGetPinnableReference()) + { + Buffer.MemoryCopy(bytes, ptr, count, count); + } + } + return next; + } + + /// Copy this slice into memory and return the advanced cursor + /// Pointer where to copy this slice + /// Capacity of the output buffer + /// Copy will fail if there is not enough space in the output buffer (ie: if it would writer at or after ) + public IntPtr CopyTo(IntPtr ptr, long count) + { + unsafe + { + byte* p = (byte*) ptr.ToPointer(); + return (IntPtr) CopyToUnsafe(p, p + count); + } + } + + /// Copy this slice into memory and return the advanced cursor + /// Pointer where to copy this slice + /// Capacity of the output buffer + /// Updated pointer after the copy, of if the destination buffer was too small + public bool TryCopyTo(IntPtr ptr, long count) + { + unsafe + { + byte* p = (byte*) ptr.ToPointer(); + return null != TryCopyToUnsafe(p, p + count); + } + } + + /// Retrieves a substring from this instance. The substring starts at a specified character position. + /// The starting position of the substring. Positive values mmeans from the start, negative values means from the end + /// A slice that is equivalent to the substring that begins at (from the start or the end depending on the sign) in this instance, or Slice.Empty if is equal to the length of the slice. + /// The substring does not copy the original data, and refers to the same buffer as the original slice. Any change to the parent slice's buffer will be seen by the substring. You must call Memoize() on the resulting substring if you want a copy + /// {"ABCDE"}.Substring(0) => {"ABC"} + /// {"ABCDE"}.Substring(1} => {"BCDE"} + /// {"ABCDE"}.Substring(-2} => {"DE"} + /// {"ABCDE"}.Substring(5} => Slice.Empty + /// Slice.Empty.Substring(0) => Slice.Empty + /// Slice.Nil.Substring(0) => Slice.Emtpy + /// + /// indicates a position not within this instance, or is less than zero + [Pure] + public Slice Substring(int offset) + { + int len = this.Count; + + // negative values mean from the end + if (offset < 0) offset += this.Count; + //REVIEW: TODO: get rid of negative indexing, and create a different "substring from the end" method? + + // bound check + if ((uint) offset > (uint) len) UnsafeHelpers.Errors.ThrowOffsetOutsideSlice(); + + int r = len - offset; + return r != 0 ? new Slice(this.Array, this.Offset + offset, r) : Slice.Empty; + } + + /// Retrieves a substring from this instance. The substring starts at a specified character position and has a specified length. + /// The starting position of the substring. Positive values means from the start, negative values means from the end + /// Number of bytes in the substring + /// A slice that is equivalent to the substring of length that begins at (from the start or the end depending on the sign) in this instance, or Slice.Empty if count is zero. + /// The substring does not copy the original data, and refers to the same buffer as the original slice. Any change to the parent slice's buffer will be seen by the substring. You must call Memoize() on the resulting substring if you want a copy + /// {"ABCDE"}.Substring(0, 3) => {"ABC"} + /// {"ABCDE"}.Substring(1, 3} => {"BCD"} + /// {"ABCDE"}.Substring(-2, 2} => {"DE"} + /// Slice.Empty.Substring(0, 0) => Slice.Empty + /// Slice.Nil.Substring(0, 0) => Slice.Emtpy + /// + /// plus indicates a position not within this instance, or or is less than zero + [Pure] + public Slice Substring(int offset, int count) + { + if (count == 0) return Slice.Empty; + int len = this.Count; + + // bound check + if ((uint) offset >= (uint) len || (uint) count > (uint)(len - offset)) UnsafeHelpers.Errors.ThrowOffsetOutsideSlice(); + + return new Slice(this.Array, this.Offset + offset, count); + } + + /// Truncate the slice if its size exceeds the specified length. + /// Maximum size. + /// Slice of at most the specified size, or smaller if the original slice does not exceed the size. + /// + /// Smaller than maxSize is unmodified{"Hello, World!"}.Truncate(20) => {"Hello, World!"} + /// Larger than maxSize is truncated{"Hello, World!"}.Truncate(5) => {"Hello"} + /// Truncating to 0 returns Empty (or Nil){"Hello, World!"}.Truncate(0) == Slice.Empty + /// + [Pure] + public Slice Truncate([Positive] int maxSize) + { + //note: the only difference with Substring(0, maxSize) is that we don't throw if the slice is smaller than ! + Contract.Positive(maxSize, nameof(maxSize)); + + if (maxSize == 0) return this.Array == null ? Nil : Empty; + return this.Count <= maxSize ? this : new Slice(this.Array, this.Offset, maxSize); + } + + /// Returns a slice array that contains the sub-slices in this instance that are delimited by the specified separator + /// The slice that delimits the sub-slices in this instance. + /// to omit empty array elements from the array returned; or to include empty array elements in the array returned. + /// An array whose elements contains the sub-slices in this instance that are delimited by the value of . + [Pure] + public Slice[] Split(Slice separator, StringSplitOptions options = StringSplitOptions.None) + { + return Split(this, separator, options); + } + + [Pure] + public Slice[] Split(int stride) + { + return Split(this, stride); + } + + /// Reports the zero-based index of the first occurence of the specified slice in this instance. + /// The slice to seek + /// The zero-based index of if that slice is found, or -1 if it is not. If is , then the return value is -1. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public int IndexOf(Slice value) + { + return Find(this, value); + } + + /// Reports the zero-based index of the first occurence of the specified slice in this instance. The search starts at a specified position. + /// The slice to seek + /// The search starting position + /// The zero-based index of if that slice is found, or -1 if it is not. If is , then the return value is startIndex + [Pure] + public int IndexOf(Slice value, int startIndex) + { + return Substring(startIndex).IndexOf(value); + } + + /// Reports the zero-based index of the first occurence of the specified byte in this instance. + /// The byte to seek + /// The zero-based index of if that slice is found, or -1 if it is not. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public int IndexOf(byte value) + { + return Find(this, value); + } + + /// Reports the zero-based index of the first occurence of the specified byte in this instance. The search starts at a specified position. + /// The byte to seek + /// The search starting position + /// The zero-based index of if that byte is found, or -1 if it is not. + [Pure] + public int IndexOf(byte value, int startIndex) + { + int len = this.Count; + if ((uint) startIndex >= (uint) len) UnsafeHelpers.Errors.ThrowOffsetOutsideSlice(); + + var tmp = new Slice(this.Array, this.Offset + startIndex, len - startIndex); + int idx = Find(tmp, value); + return idx >= 0 ? checked(startIndex + idx) : -1; + } + + /// Determines whether the beginning of this slice instance matches a specified slice. + /// The slice to compare + /// true if matches the beginning of this slice; otherwise, false + [Pure] + public bool StartsWith(Slice value) + { + if (!value.HasValue) throw ThrowHelper.ArgumentNullException(nameof(value)); + + int count = value.Count; + + // any strings starts with the empty string + if (count == 0) return true; + + // prefix cannot be bigger + if ((uint) count > (uint) this.Count) return false; + + return UnsafeHelpers.SameBytes(this.Array, this.Offset, value.Array, value.Offset, count); + } + + /// Determines whether the end of this slice instance matches a specified slice. + /// The slice to compare to the substring at the end of this instance. + /// true if matches the end of this slice; otherwise, false + [Pure] + public bool EndsWith(Slice value) + { + if (!value.HasValue) throw ThrowHelper.ArgumentNullException(nameof(value)); + + // any strings ends with the empty string + int count = value.Count; + if (count == 0) return true; + + // suffix cannot be bigger + int len = this.Count; + if ((uint) count > (uint) len) return false; + + return UnsafeHelpers.SameBytes(this.Array, this.Offset + (len - count), value.Array, value.Offset, count); + } + + /// Equivalent of StartsWith, but the returns false if both slices are identical + [Pure] + public bool PrefixedBy(Slice parent) + { + int count = parent.Count; + + // empty is a parent of everyone + if (count == 0) return true; + + // we must have at least one more byte then the parent + if (this.Count <= count) return false; + + // must start with the same bytes + return UnsafeHelpers.SameBytes(parent.Array, parent.Offset, this.Array, this.Offset, count); + } + + /// Equivalent of EndsWith, but the returns false if both slices are identical + [Pure] + public bool SuffixedBy(Slice parent) + { + // empty is a parent of everyone + int count = parent.Count; + if (count == 0) return true; + + // empty is not a child of anything + int len = this.Count; + if (len == 0) return false; + + // we must have at least one more byte then the parent + if (len <= count) return false; + + // must start with the same bytes + return UnsafeHelpers.SameBytes(parent.Array, parent.Offset + (len - count), this.Array, this.Offset, count); + } + + /// Append/Merge a slice at the end of the current slice + /// Slice that must be appended + /// Merged slice if both slices are contigous, or a new slice containg the content of the current slice, followed by the tail slice. Or Slice.Empty if both parts are nil or empty + [Pure] + public Slice Concat(Slice tail) + { + if (tail.Count == 0) return this.Count > 0 ? this: Slice.Empty; + if (this.Count == 0) return tail; + + tail.EnsureSliceIsValid(); + this.EnsureSliceIsValid(); + + // special case: adjacent segments ? + if (object.ReferenceEquals(this.Array, tail.Array) && this.Offset + this.Count == tail.Offset) + { + return new Slice(this.Array, this.Offset, this.Count + tail.Count); + } + + byte[] tmp = new byte[this.Count + tail.Count]; + UnsafeHelpers.CopyUnsafe(tmp, 0, this.Array, this.Offset, this.Count); + UnsafeHelpers.CopyUnsafe(tmp, this.Count, tail.Array, tail.Offset, tail.Count); + return new Slice(tmp); + } + + /// Append an array of slice at the end of the current slice, all sharing the same buffer + /// Slices that must be appended + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull] + public Slice[] ConcatRange([NotNull] Slice[] slices) + { + Contract.NotNull(slices, nameof(slices)); + EnsureSliceIsValid(); + + // pre-allocate by computing final buffer capacity + var prefixSize = this.Count; + var capacity = slices.Sum((slice) => prefixSize + slice.Count); + var writer = new SliceWriter(capacity); + var next = new List(slices.Length); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var slice in slices) + { + writer.WriteBytes(this); + writer.WriteBytes(slice); + next.Add(writer.Position); + } + + return SplitIntoSegments(writer.Buffer, 0, next); + } + + /// Append a sequence of slice at the end of the current slice, all sharing the same buffer + /// Slices that must be appended + /// Array of slices (for all keys) that share the same underlying buffer + [Pure, NotNull] + public Slice[] ConcatRange([NotNull] IEnumerable slices) + { + Contract.NotNull(slices, nameof(slices)); + + // use optimized version for arrays + if (slices is Slice[] array) return ConcatRange(array); + + var next = new List(); + var writer = default(SliceWriter); + + //TODO: use multiple buffers if item count is huge ? + + foreach (var slice in slices) + { + writer.WriteBytes(this); + writer.WriteBytes(slice); + next.Add(writer.Position); + } + + return SplitIntoSegments(writer.Buffer, 0, next); + + } + + /// Split a buffer containing multiple contiguous segments into an array of segments + /// Buffer containing all the segments + /// Offset of the start of the first segment + /// Array containing, for each segment, the offset of the following segment + /// Array of segments + /// SplitIntoSegments("HelloWorld", 0, [5, 10]) => [{"Hello"}, {"World"}] + [NotNull] + public static Slice[] SplitIntoSegments([NotNull] byte[] buffer, int start, [NotNull] List endOffsets) + { + Contract.Requires(buffer != null && endOffsets != null); + var result = new Slice[endOffsets.Count]; + int i = 0; + int p = start; + foreach (var end in endOffsets) + { + result[i++] = new Slice(buffer, p, end - p); + p = end; + } + + return result; + } + + /// Concatenate two slices together + public static Slice Concat(Slice a, Slice b) + { + return a.Concat(b); + } + + /// Concatenate three slices together + public static Slice Concat(Slice a, Slice b, Slice c) + { + int count = a.Count + b.Count + c.Count; + if (count == 0) return Slice.Empty; + var writer = new SliceWriter(count); + writer.WriteBytes(a); + writer.WriteBytes(b); + writer.WriteBytes(c); + return writer.ToSlice(); + } + + /// Concatenate an array of slices into a single slice + public static Slice Concat(params Slice[] args) + { + int count = 0; + for (int i = 0; i < args.Length; i++) count += args[i].Count; + if (count == 0) return Slice.Empty; + var writer = new SliceWriter(count); + for (int i = 0; i < args.Length; i++) writer.WriteBytes(args[i]); + return writer.ToSlice(); + } + + /// Adds a prefix to a list of slices + /// Prefix to add to all the slices + /// List of slices to process + /// Array of slice that all start with and followed by the corresponding entry in + /// This method is optmized to reduce the amount of memory allocated + [Pure, NotNull] + public static Slice[] ConcatRange(Slice prefix, IEnumerable slices) + { + Contract.NotNull(slices, nameof(slices)); + + if (prefix.IsNullOrEmpty) + { // nothing to do, but we still need to copy the array + return slices.ToArray(); + } + + Slice[] res; + Slice[] arr; + ICollection coll; + + if ((arr = slices as Slice[]) != null) + { // fast-path for arrays (most frequent with range reads) + + // we wil use a SliceBuffer to store all the keys produced in as few byte[] arrays as needed + + // precompute the exact size needed + int totalSize = prefix.Count * arr.Length; + for (int i = 0; i < arr.Length; i++) totalSize += arr[i].Count; + var buf = new SliceBuffer(Math.Min(totalSize, 64 * 1024)); + + res = new Slice[arr.Length]; + for (int i = 0; i < arr.Length; i++) + { + res[i] = buf.Intern(prefix, arr[i], aligned: false); + } + } + else if ((coll = slices as ICollection) != null) + { // collection (size known) + + //TODO: also use a SliceBuffer since we could precompute the total size... + + res = new Slice[coll.Count]; + int p = 0; + foreach (var suffix in coll) + { + res[p++] = prefix.Concat(suffix); + } + } + else + { // streaming sequence (size unknown) + + //note: we can only scan the list once, so would be no way to get a sensible value for the buffer's page size + var list = new List(); + foreach (var suffix in slices) + { + list.Add(prefix.Concat(suffix)); + } + res = list.ToArray(); + } + + return res; + } + + /// Reports the zero-based index of the first occurrence of the specified slice in this source. + /// The slice Input slice + /// The slice to seek + /// Offset of the match if positive, or no occurence was found if negative + [Pure] + public static int Find(Slice source, Slice value) + { + const int NOT_FOUND = -1; + + source.EnsureSliceIsValid(); + source.EnsureSliceIsValid(); + + int m = value.Count; + if (m == 0) return 0; + + int n = source.Count; + if (n == 0) return NOT_FOUND; + + if (m == n) return source.Equals(value) ? 0 : NOT_FOUND; + if (m <= n) + { + byte[] src = source.Array; + int p = source.Offset; + byte firstByte = value[0]; + + // note: this is a very simplistic way to find a value, and is optimized for the case where the separator is only one byte (most common) + while (n-- > 0) + { + if (src[p++] == firstByte) + { // possible match ? + if (m == 1 || UnsafeHelpers.SameBytesUnsafe(src, p, value.Array, value.Offset + 1, m - 1)) + { + return p - source.Offset - 1; + } + } + } + } + + return NOT_FOUND; + } + + /// Reports the zero-based index of the first occurrence of the specified byte in this source. + /// The slice Input slice + /// The byte to find + /// Offset of the match if positive, or the byte was not found if negative + [Pure] + public static int Find(Slice source, byte value) + { + source.EnsureSliceIsValid(); + + const int NOT_FOUND = -1; + int n = source.Count; + if (n == 0) return NOT_FOUND; + unsafe + { + //TODO: Optimize this! + fixed (byte* ptr = &source.DangerousGetPinnableReference()) + { + byte* inp = ptr; + while (n-- > 0) + { + if (*inp == value) + { // match + return checked((int)(inp - ptr)); + } + ++inp; + } + } + } + return NOT_FOUND; + } + + /// Concatenates all the elements of a slice array, using the specified separator between each element. + /// The slice to use as a separator. Can be empty. + /// An array that contains the elements to concatenate. + /// A slice that consists of the elements in a value delimited by the slice. If is an empty array, the method returns . + /// If is null. + public static Slice Join(Slice separator, [NotNull] Slice[] values) + { + Contract.NotNull(values, nameof(values)); + + int count = values.Length; + if (count == 0) return Slice.Empty; + if (count == 1) return values[0]; + return Join(separator, values, 0, count); + } + + /// Concatenates the specified elements of a slice array, using the specified separator between each element. + /// The slice to use as a separator. Can be empty. + /// An array that contains the elements to concatenate. + /// The first element in to use. + /// The number of elements of to use. + /// A slice that consists of the slices in delimited by the slice. -or- if is zero, has no elements, or and all the elements of are . + /// If is null. + /// If or is less than zero. -or- plus is greater than the number of elements in . + public static Slice Join(Slice separator, [NotNull] Slice[] values, int startIndex, int count) + { + // Note: this method is modeled after String.Join() and should behave the same + // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) + + Contract.NotNull(values, nameof(values)); + + if (startIndex < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); + if (startIndex > values.Length - count) throw ThrowHelper.ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); + + if (count == 0) return Slice.Empty; + if (count == 1) return values[startIndex]; + + int size = 0; + for (int i = 0; i < values.Length; i++) size += values[i].Count; + size += (values.Length - 1) * separator.Count; + + // if the size overflows, that means that the resulting buffer would need to be >= 2 GB, which is not possible! + if (size < 0) throw new OutOfMemoryException(); + + //note: we want to make sure the buffer of the writer will be the exact size (so that we can use the result as a byte[] without copying again) + var tmp = new byte[size]; + var writer = new SliceWriter(tmp); + for (int i = 0; i < values.Length; i++) + { + if (i > 0) writer.WriteBytes(separator); + writer.WriteBytes(values[i]); + } + Contract.Assert(writer.Buffer.Length == size); + return writer.ToSlice(); + } + + /// Concatenates the specified elements of a slice sequence, using the specified separator between each element. + /// The slice to use as a separator. Can be empty. + /// A sequence will return the elements to concatenate. + /// A slice that consists of the slices in delimited by the slice. -or- if has no elements, or and all the elements of are . + /// If is null. + public static Slice Join(Slice separator, [NotNull] IEnumerable values) + { + Contract.NotNull(values, nameof(values)); + var array = (values as Slice[]) ?? values.ToArray(); + return Join(separator, array, 0, array.Length); + } + + /// Concatenates the specified elements of a slice array, using the specified separator between each element. + /// The slice to use as a separator. Can be empty. + /// An array that contains the elements to concatenate. + /// The first element in to use. + /// The number of elements of to use. + /// A byte array that consists of the slices in delimited by the slice. -or- an emtpy array if is zero, has no elements, or and all the elements of are . + /// If is null. + /// If or is less than zero. -or- plus is greater than the number of elements in . + [NotNull] + public static byte[] JoinBytes(Slice separator, [NotNull] Slice[] values, int startIndex, int count) + { + // Note: this method is modeled after String.Join() and should behave the same + // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) + + Contract.NotNull(values, nameof(values)); + //REVIEW: support negative indexing ? + if (startIndex < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); + if (startIndex > values.Length - count) throw ThrowHelper.ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); + + if (count == 0) return System.Array.Empty(); + if (count == 1) return values[startIndex].GetBytes() ?? System.Array.Empty(); + + int size = 0; + for (int i = 0; i < count; i++) size = checked(size + values[startIndex + i].Count); + size = checked(size + (count - 1) * separator.Count); + + // if the size overflows, that means that the resulting buffer would need to be >= 2 GB, which is not possible! + if (size < 0) throw new OutOfMemoryException(); + + //note: we want to make sure the buffer of the writer will be the exact size (so that we can use the result as a byte[] without copying again) + var tmp = new byte[size]; + int p = 0; + for (int i = 0; i < count; i++) + { + if (i > 0) separator.WriteTo(tmp, ref p); + values[startIndex + i].WriteTo(tmp, ref p); + } + Contract.Assert(p == tmp.Length); + return tmp; + } + + /// Concatenates the specified elements of a slice sequence, using the specified separator between each element. + /// The slice to use as a separator. Can be empty. + /// A sequence will return the elements to concatenate. + /// A byte array that consists of the slices in delimited by the slice. -or- an empty array if has no elements, or and all the elements of are . + /// If is null. + [NotNull] + public static byte[] JoinBytes(Slice separator, [NotNull] IEnumerable values) + { + Contract.NotNull(values, nameof(values)); + var array = (values as Slice[]) ?? values.ToArray(); + return JoinBytes(separator, array, 0, array.Length); + } + + /// Returns a slice array that contains the sub-slices in that are delimited by . A parameter specifies whether to return empty array elements. + /// Input slice that must be split into sub-slices + /// Separator that delimits the sub-slices in . Cannot be empty or nil + /// to omit empty array alements from the array returned; or to include empty array elements in the array returned. + /// An array whose elements contain the sub-slices that are delimited by . + /// If is empty, or if is not one of the values. + /// If does not contain the delimiter, the returned array consists of a single element that repeats the input, or an empty array if input is itself empty. + /// To reduce memory usage, the sub-slices returned in the array will all share the same underlying buffer of the input slice. + [NotNull] + public static Slice[] Split(Slice input, Slice separator, StringSplitOptions options = StringSplitOptions.None) + { + // this method is made to behave the same way as String.Split(), especially the following edge cases + // - Empty.Split(..., StringSplitOptions.None) => { Empty } + // - Empty.Split(..., StringSplitOptions.RemoveEmptyEntries) => { } + // differences: + // - If input is Nil, it is considered equivalent to Empty + // - If separator is Nil or Empty, the method throws + + var list = new List(); + + if (separator.Count <= 0) throw ThrowHelper.ArgumentException(nameof(separator), "Separator must have at least one byte"); + if (options < StringSplitOptions.None || options > StringSplitOptions.RemoveEmptyEntries) throw ThrowHelper.ArgumentException(nameof(options)); + + bool skipEmpty = options.HasFlag(StringSplitOptions.RemoveEmptyEntries); + if (input.Count == 0) + { + return skipEmpty ? System.Array.Empty() : new[] { Slice.Empty }; + } + + while (input.Count > 0) + { + int p = Find(input, separator); + if (p < 0) + { // last chunk + break; + } + if (p == 0) + { // empty chunk + if (!skipEmpty) list.Add(Slice.Empty); + } + else + { + list.Add(input.Substring(0, p)); + } + // note: we checked earlier that separator.Count > 0, so we are guaranteed to advance the cursor + input = input.Substring(p + separator.Count); + } + + if (input.Count > 0 || !skipEmpty) + { + list.Add(input); + } + + return list.ToArray(); + } + + /// Returns a slice array that contains the sub-slices in by cutting fixed-length chunks or size . + /// Input slice that must be split into sub-slices + /// Size of each chunk that will be cut from . Must be greater or equal to 1. + /// + /// An array whose elements contain the sub-slices, each of size , except the last slice that may be smaller if the length of is not a multiple of . + /// If is then the array will be empty. + /// If it is then the array will we of length 1 and contain the empty slice. + /// + /// To reduce memory usage, the sub-slices returned in the array will all share the same underlying buffer of the input slice. + [NotNull] + public static Slice[] Split(Slice input, int stride) + { + Contract.GreaterOrEqual(stride, 1, nameof (stride)); + + if (input.IsNull) return System.Array.Empty(); + + if (input.Count <= stride) + { // single element + return new [] { input }; + } + + // how many slices? (last one may be incomplete) + int count = (input.Count + (stride - 1)) / stride; + var result = new Slice[count]; + + int p = 0; + int r = input.Count; + for(int i = 0; i < result.Length; i++) + { + Contract.Assert(r >= 0); + result[i] = new Slice(input.Array, input.Offset + p, Math.Min(r, stride)); + p += stride; + r -= stride; + } + + return result; + } + + /// Returns the first key lexicographically that does not have the passed in as a prefix + /// Slice to increment + /// New slice that is guaranteed to be the first key lexicographically higher than which does not have as a prefix + /// If the last byte is already equal to 0xFF, it will rollover to 0x00 and the next byte will be incremented. + /// If the Slice is equal to Slice.Nil + /// If the Slice is the empty string or consists only of 0xFF bytes + /// + /// Slice.Increment(Slice.FromString("ABC")) => "ABD" + /// Slice.Increment(Slice.FromHexa("01 FF")) => { 02 } + /// + public static Slice Increment(Slice slice) + { + if (slice.IsNull) throw ThrowHelper.ArgumentException(nameof(slice), "Cannot increment null buffer"); + + int lastNonFfByte; + var tmp = slice.GetBytesOrEmpty(); + for (lastNonFfByte = tmp.Length - 1; lastNonFfByte >= 0; --lastNonFfByte) + { + if (tmp[lastNonFfByte] != 0xFF) + { + ++tmp[lastNonFfByte]; + break; + } + } + + if (lastNonFfByte < 0) + { + throw ThrowHelper.ArgumentException(nameof(slice), "Cannot increment key"); //TODO: PoneyDB.Errors.CannotIncrementKey(); + } + + return new Slice(tmp, 0, lastNonFfByte + 1); + } + + /// Merge an array of keys with a same prefix, all sharing the same buffer + /// Prefix shared by all keys + /// Array of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) + { + Contract.NotNull(keys, nameof(keys)); + + //REVIEW: merge this code with Slice.ConcatRange! + + if (keys.Length == 0) return System.Array.Empty(); + + // we can pre-allocate exactly the buffer by computing the total size of all keys + int size = keys.Length * prefix.Count; + for (int i = 0; i < keys.Length; i++) size += keys[i].Count; + + var writer = new SliceWriter(size); + var next = new List(keys.Length); + + //TODO: use multiple buffers if item count is huge ? + bool hasPrefix = prefix.IsPresent; + foreach (var key in keys) + { + if (hasPrefix) writer.WriteBytes(prefix); + writer.WriteBytes(key); + next.Add(writer.Position); + } + + return SplitIntoSegments(writer.Buffer, 0, next); + } + + /// Merge a sequence of keys with a same prefix, all sharing the same buffer + /// Prefix shared by all keys + /// Sequence of keys to pack + /// Array of slices (for all keys) that share the same underlying buffer + [NotNull] + public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) + { + Contract.NotNull(keys, nameof(keys)); + + //REVIEW: merge this code with Slice.ConcatRange! + + // use optimized version for arrays + if (keys is Slice[] array) return Merge(prefix, array); + + // pre-allocate with a count if we can get one... + var next = keys is ICollection coll ? new List(coll.Count) : new List(); + var writer = default(SliceWriter); + + //TODO: use multiple buffers if item count is huge ? + + bool hasPrefix = prefix.IsPresent; + foreach (var key in keys) + { + if (hasPrefix) writer.WriteBytes(prefix); + writer.WriteBytes(key); + next.Add(writer.Position); + } + + return SplitIntoSegments(writer.Buffer, 0, next); + } + + /// Creates a new slice that contains the same byte repeated + /// Byte that will fill the slice + /// Number of bytes + /// New slice that contains times the byte . + public static Slice Repeat(byte value, int count) + { + Contract.Positive(count, nameof(count), "count"); + if (count == 0) return Slice.Empty; + + var res = new byte[count]; + UnsafeHelpers.Fill(res, 0, count, value); + return new Slice(res); + } + + /// Creates a new slice that contains the same byte repeated + /// ASCII character (between 0 and 255) that will fill the slice. If is greater than 0xFF, only the 8 lowest bits will be used + /// Number of bytes + /// New slice that contains times the byte . + public static Slice Repeat(char value, int count) + { + Contract.Positive(count, nameof(count), "count"); + if (count == 0) return Slice.Empty; + + var res = new byte[count]; + UnsafeHelpers.Fill(res, 0, count, (byte) value); + return new Slice(res); + } + + /// Create a new slice filled with random bytes taken from a random number generator + /// Pseudo random generator to use (needs locking if instance is shared) + /// Number of random bytes to generate + /// Slice of bytes taken from + /// Warning: is not thread-safe ! If the instance is shared between threads, then it needs to be locked before calling this method. + public static Slice Random([NotNull] Random prng, int count) + { + Contract.NotNull(prng, nameof(prng)); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); + if (count == 0) return Slice.Empty; + + var bytes = new byte[count]; + prng.NextBytes(bytes); + return new Slice(bytes, 0, count); + } + + /// Create a new slice filled with random bytes taken from a cryptographic random number generator + /// Random generator to use (needs locking if instance is shared) + /// Number of random bytes to generate + /// If true, produce a sequence of non-zero bytes. + /// Slice of bytes taken from + /// Warning: All RNG implementations may not be thread-safe ! If the instance is shared between threads, then it may need to be locked before calling this method. + public static Slice Random([NotNull] System.Security.Cryptography.RandomNumberGenerator rng, int count, bool nonZeroBytes = false) + { + Contract.NotNull(rng, nameof(rng)); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); + if (count == 0) return Slice.Empty; + + var bytes = new byte[count]; + + if (nonZeroBytes) + rng.GetNonZeroBytes(bytes); + else + rng.GetBytes(bytes); + + return new Slice(bytes, 0, count); + } + + /// Returns the lowest of two keys + /// First key + /// Second key + /// The key that is BEFORE the other, using lexicographical order + /// If both keys are equal, then is returned + public static Slice Min(Slice a, Slice b) + { + return a.CompareTo(b) <= 0 ? a : b; + } + + /// Returns the lowest of three keys + /// First key + /// Second key + /// Second key + /// The key that is BEFORE the other two, using lexicographical order + public static Slice Min(Slice a, Slice b, Slice c) + { + return a.CompareTo(b) <= 0 + ? (a.CompareTo(c) <= 0 ? a : c) + : (b.CompareTo(c) <= 0 ? b : c); + } + + public static Slice Min(params Slice[] values) + { + switch (values.Length) + { + case 0: return Slice.Nil; + case 1: return values[0]; + case 2: return Min(values[0], values[1]); + case 3: return Min(values[0], values[1], values[3]); + default: + { + Slice min = values[0]; + for (int i = 1; i < values.Length; i++) + { + if (values[i].CompareTo(min) < 0) min = values[i]; + } + return min; + } + } + } + + /// Returns the highest of two keys + /// First key + /// Second key + /// The key that is AFTER the other, using lexicographical order + /// If both keys are equal, then is returned + public static Slice Max(Slice a, Slice b) + { + return a.CompareTo(b) >= 0 ? a : b; + } + + /// Returns the highest of three keys + /// First key + /// Second key + /// Second key + /// The key that is AFTER the other two, using lexicographical order + public static Slice Max(Slice a, Slice b, Slice c) + { + return a.CompareTo(b) >= 0 + ? (a.CompareTo(c) >= 0 ? a : c) + : (b.CompareTo(c) >= 0 ? b : c); + } + + public static Slice Max(params Slice[] values) + { + switch (values.Length) + { + case 0: return Slice.Nil; + case 1: return values[0]; + case 2: return Max(values[0], values[1]); + case 3: return Max(values[0], values[1], values[3]); + default: + { + Slice max = values[0]; + for (int i = 1; i < values.Length; i++) + { + if (values[i].CompareTo(max) > 0) max = values[i]; + } + return max; + } + } + } + + #region Slice arithmetics... + + /// Compare two slices for equality + /// True if the slices contains the same bytes + public static bool operator ==(Slice a, Slice b) + { + return a.Equals(b); + } + + /// Compare two slices for inequality + /// True if the slices do not contain the same bytes + public static bool operator !=(Slice a, Slice b) + { + return !a.Equals(b); + } + + /// Compare two slices + /// True if is lexicographically less than ; otherwise, false. + public static bool operator <(Slice a, Slice b) + { + return a.CompareTo(b) < 0; + } + + /// Compare two slices + /// True if is lexicographically less than or equal to ; otherwise, false. + public static bool operator <=(Slice a, Slice b) + { + return a.CompareTo(b) <= 0; + } + + /// Compare two slices + /// True if is lexicographically greater than ; otherwise, false. + public static bool operator >(Slice a, Slice b) + { + return a.CompareTo(b) > 0; + } + + /// Compare two slices + /// True if is lexicographically greater than or equal to ; otherwise, false. + public static bool operator >=(Slice a, Slice b) + { + return a.CompareTo(b) >= 0; + } + + /// Append/Merge two slices together + /// First slice + /// Second slice + /// Merged slices if both slices are contigous, or a new slice containg the content of the first slice, followed by the second + public static Slice operator +(Slice a, Slice b) + { + return a.Concat(b); + } + + /// Appends a byte at the end of the slice + /// First slice + /// Byte to append at the end + /// New slice with the byte appended + public static Slice operator +(Slice a, byte b) + { + if (a.Count == 0) return Slice.FromByte(b); + var tmp = new byte[a.Count + 1]; + UnsafeHelpers.CopyUnsafe(tmp, 0, a.Array, a.Offset, a.Count); + tmp[a.Count] = b; + return new Slice(tmp); + } + + /// Remove bytes at the end of slice + /// Smaller slice + public static Slice operator -(Slice s, int n) + { + if (n < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(n), "Cannot subtract a negative number from a slice"); + if (n > s.Count) throw ThrowHelper.ArgumentOutOfRangeException(nameof(n), "Cannout substract more bytes than the slice contains"); + + if (n == 0) return s; + if (n == s.Count) return Slice.Empty; + + return new Slice(s.Array, s.Offset, s.Count - n); + } + + // note: We also need overloads with Nullable's to be able to do things like "if (slice == null)", "if (slice != null)" or "if (null != slice)". + // For structs that have "==" / "!=" operators, the compiler will think that when you write "slice == null", you really mean "(Slice?)slice == default(Slice?)", and that would ALWAYS false if you don't have specialized overloads to intercept. + + /// Determines whether two specified instances of are equal + public static bool operator ==(Slice? a, Slice? b) + { + return a.GetValueOrDefault().Equals(b.GetValueOrDefault()); + } + + /// Determines whether two specified instances of are not equal + public static bool operator !=(Slice? a, Slice? b) + { + return !a.GetValueOrDefault().Equals(b.GetValueOrDefault()); + } + + /// Determines whether one specified is less than another specified . + public static bool operator <(Slice? a, Slice? b) + { + return a.GetValueOrDefault() < b.GetValueOrDefault(); + } + + /// Determines whether one specified is less than or equal to another specified . + public static bool operator <=(Slice? a, Slice? b) + { + return a.GetValueOrDefault() <= b.GetValueOrDefault(); + } + + /// Determines whether one specified is greater than another specified . + public static bool operator >(Slice? a, Slice? b) + { + return a.GetValueOrDefault() > b.GetValueOrDefault(); + } + + /// Determines whether one specified is greater than or equal to another specified . + public static bool operator >=(Slice? a, Slice? b) + { + return a.GetValueOrDefault() >= b.GetValueOrDefault(); + } + + /// Concatenates two together. + public static Slice operator +(Slice? a, Slice? b) + { + // note: makes "slice + null" work! + return a.GetValueOrDefault().Concat(b.GetValueOrDefault()); + } + + #endregion + + /// Returns a printable representation of the key + /// You can roundtrip the result of calling slice.ToString() by passing it to (string) and get back the original slice. + public override string ToString() + { + return Dump(this); + } + + public string ToString(string format) + { + return ToString(format, null); + } + + /// Formats the slice using the specified encoding + /// A single format specifier that indicates how to format the value of this Slice. The parameter can be "N", "D", "X", or "P". If format is null or an empty string (""), "D" is used. A lower case character will usually produce lowercased hexadecimal letters. + /// This paramater is not used + /// + /// + /// The format D is the default, and produce a round-trippable version of the slice, using <XX> tokens for non-printables bytes. + /// The format N (or n) produces a compact hexadecimal string (without separators). + /// The format X (or x) produces an hexadecimal string with spaces between each bytes. + /// The format P is the equivalent of calling . + /// + public string ToString(string format, IFormatProvider provider) + { + switch (format ?? "D") + { + case "D": + case "d": + return Dump(this); + + case "N": + return ToHexaString(lower: false); + case "n": + return ToHexaString(lower: true); + + case "X": + return ToHexaString(' ', lower: false); + case "x": + return ToHexaString(' ', lower: true); + + case "P": + case "p": + return PrettyPrint(); + + case "K": + case "k": + return PrettyPrint(); //TODO: Key ! (cf USlice) + + case "V": + case "v": + return PrettyPrint(); //TODO: Value ! (cf USlice) + + default: + throw new FormatException("Format is invalid or not supported"); + } + } + + /// Returns a printable representation of a key + /// This may not be efficient, so it should only be use for testing/logging/troubleshooting + [NotNull] + public static string Dump(Slice value, int maxSize = 1024) //REVIEW: rename this to Encode(..) or Escape(..) + { + if (value.Count == 0) return value.HasValue ? "" : ""; + + value.EnsureSliceIsValid(); + + var buffer = value.Array; + int count = Math.Min(value.Count, maxSize); + int pos = value.Offset; + + var sb = new StringBuilder(count + 16); + while (count-- > 0) + { + int c = buffer[pos++]; + if (c < 32 || c >= 127 || c == 60) + { + sb.Append('<'); + int x = c >> 4; + sb.Append((char)(x + (x < 10 ? 48 : 55))); + x = c & 0xF; + sb.Append((char)(x + (x < 10 ? 48 : 55))); + sb.Append('>'); + } + else + { + sb.Append((char)c); + } + } + if (value.Count > maxSize) sb.Append("[\u2026]"); // Unicode for '...' + return sb.ToString(); + } + + /// Decode the string that was generated by slice.ToString() or Slice.Dump(), back into the original slice + /// This may not be efficient, so it should only be use for testing/logging/troubleshooting + public static Slice Unescape(string value) //REVIEW: rename this to Decode() if we changed Dump() to Encode() + { + var writer = default(SliceWriter); + for (int i = 0; i < value.Length; i++) + { + char c = value[i]; + if (c == '<') + { + if (value[i + 3] != '>') throw new FormatException($"Invalid escape character at offset {i}"); + c = (char)(NibbleToDecimal(value[i + 1]) << 4 | NibbleToDecimal(value[i + 2])); + i += 3; + } + writer.WriteByte((byte)c); + } + return writer.ToSlice(); + } + + #region Streams... + + /// Read the content of a stream into a slice + /// Source stream, that must be in a readable state + /// Slice containing the stream content (or if the stream is ) + /// If is null. + /// If the size of the stream exceeds or if it does not support reading. + public static Slice FromStream([NotNull] Stream data) + { + Contract.NotNull(data, nameof(data)); + + // special case for empty values + if (data == Stream.Null) return Slice.Nil; + if (!data.CanRead) throw ThrowHelper.InvalidOperationException("Cannot read from provided stream"); + + if (data.Length == 0) return Slice.Empty; + if (data.Length > int.MaxValue) throw ThrowHelper.InvalidOperationException("Streams of more than 2GB are not supported"); + //TODO: other checks? + + int length; + checked { length = (int)data.Length; } + + if (data is MemoryStream || data is UnmanagedMemoryStream) // other types of already completed streams ? + { // read synchronously + return LoadFromNonBlockingStream(data, length); + } + + // read asynchronoulsy + return LoadFromBlockingStream(data, length); + } + + /// Asynchronously read the content of a stream into a slice + /// Source stream, that must be in a readable state + /// Optional cancellation token for this operation + /// Slice containing the stream content (or if the stream is ) + /// If is null. + /// If the size of the stream exceeds or if it does not support reading. + public static Task FromStreamAsync([NotNull] Stream data, CancellationToken ct) + { + Contract.NotNull(data, nameof(data)); + + // special case for empty values + if (data == Stream.Null) return Task.FromResult(Slice.Nil); + if (!data.CanRead) throw ThrowHelper.InvalidOperationException("Cannot read from provided stream"); + + if (data.Length == 0) return Task.FromResult(Slice.Empty); + if (data.Length > int.MaxValue) throw ThrowHelper.InvalidOperationException("Streams of more than 2GB are not supported"); + //TODO: other checks? + + if (ct.IsCancellationRequested) return Task.FromCanceled(ct); + + int length; + checked { length = (int)data.Length; } + + if (data is MemoryStream || data is UnmanagedMemoryStream) // other types of already completed streams ? + { // read synchronously + return Task.FromResult(LoadFromNonBlockingStream(data, length)); + } + + // read asynchronoulsy + return LoadFromBlockingStreamAsync(data, length, 0, ct); + } + + /// Read from a non-blocking stream that already contains all the data in memory (MemoryStream, UnmanagedStream, ...) + /// Source stream + /// Number of bytes to read from the stream + /// Slice containing the loaded data + private static Slice LoadFromNonBlockingStream([NotNull] Stream source, int length) + { + Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue); + + if (source is MemoryStream ms) + { // Already holds onto a byte[] + + //note: should be use GetBuffer() ? It can throws and is dangerous (could mutate) + return ms.ToArray().AsSlice(); + } + + // read it in bulk, without buffering + + var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? + + // note: reading should usually complete with only one big read, but loop until completed, just to be sure + int p = 0; + int r = length; + while (r > 0) + { + int n = source.Read(buffer, p, r); + if (n <= 0) throw ThrowHelper.InvalidOperationException($"Unexpected end of stream at {p:N0} / {length:N0} bytes"); + p += n; + r -= n; + } + Contract.Assert(r == 0 && p == length); + + return buffer.AsSlice(); + } + + /// Synchronously read from a blocking stream (FileStream, NetworkStream, ...) + /// Source stream + /// Number of bytes to read from the stream + /// If non zero, max amount of bytes to read in one chunk. If zero, tries to read everything at once + /// Slice containing the loaded data + private static Slice LoadFromBlockingStream([NotNull] Stream source, int length, int chunkSize = 0) + { + Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue && chunkSize >= 0); + + if (chunkSize == 0) chunkSize = int.MaxValue; + + var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? + + // note: reading should usually complete with only one big read, but loop until completed, just to be sure + int p = 0; + int r = length; + while (r > 0) + { + int c = Math.Max(r, chunkSize); + int n = source.Read(buffer, p, c); + if (n <= 0) throw ThrowHelper.InvalidOperationException($"Unexpected end of stream at {p:N0} / {length:N0} bytes"); + p += n; + r -= n; + } + Contract.Assert(r == 0 && p == length); + + return buffer.AsSlice(); + } + + /// Asynchronously read from a blocking stream (FileStream, NetworkStream, ...) + /// Source stream + /// Number of bytes to read from the stream + /// If non zero, max amount of bytes to read in one chunk. If zero, tries to read everything at once + /// Optional cancellation token for this operation + /// Slice containing the loaded data + private static async Task LoadFromBlockingStreamAsync([NotNull] Stream source, int length, int chunkSize, CancellationToken ct) + { + Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue && chunkSize >= 0); + + if (chunkSize == 0) chunkSize = int.MaxValue; + + var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? + + // note: reading should usually complete with only one big read, but loop until completed, just to be sure + int p = 0; + int r = length; + while (r > 0) + { + int c = Math.Min(r, chunkSize); + int n = await source.ReadAsync(buffer, p, c, ct); + if (n <= 0) throw ThrowHelper.InvalidOperationException($"Unexpected end of stream at {p:N0} / {length:N0} bytes"); + p += n; + r -= n; + } + Contract.Assert(r == 0 && p == length); + + return buffer.AsSlice(); + } + + #endregion + + #region Equality, Comparison... + + /// Checks if an object is equal to the current slice + /// Object that can be either another slice, a byte array, or a byte array segment. + /// true if the object represents a sequence of bytes that has the same size and same content as the current slice. + public override bool Equals(object obj) + { + switch (obj) + { + case null: return this.Array == null; + case Slice slice: return Equals(slice); + case ArraySegment segment: return Equals(segment); + case byte[] bytes: return Equals(bytes); + } + return false; + } + + /// Gets the hash code for this slice + /// A 32-bit signed hash code calculated from all the bytes in the slice. + public override int GetHashCode() + { + EnsureSliceIsValid(); + return this.Array == null ? 0 : UnsafeHelpers.ComputeHashCodeUnsafe(this.Array, this.Offset, this.Count); + } + + /// Checks if another slice is equal to the current slice. + /// Slice compared with the current instance + /// true if both slices have the same size and contain the same sequence of bytes; otherwise, false. + public bool Equals(Slice other) + { + other.EnsureSliceIsValid(); + this.EnsureSliceIsValid(); + + // note: Slice.Nil != Slice.Empty + if (this.Array == null) return other.Array == null; + if (other.Array == null) return false; + + return this.Count == other.Count && UnsafeHelpers.SameBytesUnsafe(this.Array, this.Offset, other.Array, other.Offset, this.Count); + } + + /// Lexicographically compare this slice with another one, and return an indication of their relative sort order + /// Slice to compare with this instance + /// Returns a NEGATIVE value if the current slice is LESS THAN , ZERO if it is EQUAL TO , and a POSITIVE value if it is GREATER THAN . + /// If both this instance and are Nil or Empty, the comparison will return ZERO. If only is Nil or Empty, it will return a NEGATIVE value. If only this instance is Nil or Empty, it will return a POSITIVE value. + public int CompareTo(Slice other) + { + if (this.Count == 0) return other.Count == 0 ? 0 : -1; + if (other.Count == 0) return +1; + other.EnsureSliceIsValid(); + this.EnsureSliceIsValid(); + return UnsafeHelpers.CompareUnsafe(this.Array, this.Offset, this.Count, other.Array, other.Offset, other.Count); + } + + /// Checks if the content of a byte array segment matches the current slice. + /// Byte array segment compared with the current instance + /// true if both segment and slice have the same size and contain the same sequence of bytes; otherwise, false. + public bool Equals(ArraySegment other) + { + return this.Count == other.Count && UnsafeHelpers.SameBytes(this.Array, this.Offset, other.Array, other.Offset, this.Count); + } + + /// Checks if the content of a byte array matches the current slice. + /// Byte array compared with the current instance + /// true if the both array and slice have the same size and contain the same sequence of bytes; otherwise, false. + public bool Equals(byte[] other) + { + if (other == null) return this.Array == null; + return this.Count == other.Length && UnsafeHelpers.SameBytes(this.Array, this.Offset, other, 0, this.Count); + } + + #endregion + + #region Sanity Checking... + + /// Verifies that the and fields represent a valid location in + /// This method is inlined for best performance + /// If the slice is not a valid section of a buffer + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void EnsureSliceIsValid() + { + // Conditions for a slice to be valid: + // - Count equal to 0 (other fields are ignored) + // - Count greather than 0 and Array not null and all the bytes of the slice are contained in the underlying buffer + + int count = this.Count; + if (count != 0) + { + var array = this.Array; + if (array == null || (uint) count > (long) array.Length - (uint) this.Offset) + { + throw MalformedSlice(this); + } + } + } + + /// Reject an invalid slice by throw an error with the appropriate diagnostic message. + /// Slice that is being naugthy + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception MalformedSlice(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! + // You should walk up the stack to see what is going on ! + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + + if (slice.Offset < 0) return UnsafeHelpers.Errors.SliceOffsetNotNeg(); + if (slice.Count < 0) return UnsafeHelpers.Errors.SliceCountNotNeg(); + if (slice.Count > 0) + { + if (slice.Array == null) return UnsafeHelpers.Errors.SliceBufferNotNull(); + if (slice.Offset + slice.Count > slice.Array.Length) return UnsafeHelpers.Errors.SliceBufferTooSmall(); + } + // maybe it's Lupus ? + return UnsafeHelpers.Errors.SliceInvalid(); + } + + #endregion + + /// Return the sum of the size of all the slices with an additionnal prefix + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Combined total size of all the slices and the prefixes + public static int GetTotalSize(int prefix, [NotNull] Slice[] slices) + { + long size = prefix * slices.Length; + for (int i = 0; i < slices.Length; i++) + { + size += slices[i].Count; + } + return checked((int)size); + } + + /// Return the sum of the size of all the slices with an additionnal prefix + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Combined total size of all the slices and the prefixes + public static int GetTotalSize(int prefix, [NotNull] Slice?[] slices) + { + long size = prefix * slices.Length; + for (int i = 0; i < slices.Length; i++) + { + size += slices[i].GetValueOrDefault().Count; + } + return checked((int)size); + } + + /// Return the sum of the size of all the slices with an additionnal prefix + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Combined total size of all the slices and the prefixes + public static int GetTotalSize(int prefix, [NotNull] List slices) + { + long size = prefix * slices.Count; + foreach (var val in slices) + { + size += val.Count; + } + return checked((int)size); + } + + /// Return the sum of the size of all the slices with an additionnal prefix + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Combined total size of all the slices and the prefixes + public static int GetTotalSize(int prefix, [NotNull] List slices) + { + long size = prefix * slices.Count; + foreach (var val in slices) + { + size += val.GetValueOrDefault().Count; + } + return checked((int)size); + } + + /// Return the sum of the size of all the slices with an additionnal prefix, and test if they all share the same buffer + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Receives null if at least two slices are stored in a different buffer. If not null, return the common buffer for all the keys + /// Combined total size of all the slices and the prefixes + public static int GetTotalSizeAndCommonStore(int prefix, [NotNull] Slice[] slices, out byte[] commonStore) + { + if (slices.Length == 0) + { + commonStore = null; + return 0; + } + byte[] store = slices[0].Array; + if (slices.Length == 1) + { + commonStore = store; + return prefix + slices[0].Count; + } + + bool sameStore = true; + long size = slices[0].Count + slices.Length * prefix; + for (int i = 1; i < slices.Length; i++) + { + size += slices[i].Count; + sameStore &= (slices[i].Array == store); + } + commonStore = sameStore ? store : null; + return checked((int)size); + } + + /// Return the sum of the size of all the slices with an additionnal prefix, and test if they all share the same buffer + /// Size of a prefix that would be added before each slice + /// Array of slices + /// Receives null if at least two slices are stored in a different buffer. If not null, return the common buffer for all the keys + /// Combined total size of all the slices and the prefixes + public static int GetTotalSizeAndCommonStore(int prefix, [NotNull] List slices, out byte[] commonStore) + { + Contract.Requires(slices != null); + if (slices.Count == 0) + { + commonStore = null; + return 0; + } + byte[] store = slices[0].Array; + if (slices.Count == 1) + { + commonStore = store; + return prefix + slices[0].Count; + } + + bool sameStore = true; + long size = slices[0].Count + slices.Count * prefix; + foreach (var val in slices) + { + size += val.Count; + sameStore &= (val.Array == store); + } + commonStore = sameStore ? store : null; + return checked((int)size); + } + + /// Structure that keeps buffers from moving in memory during GC collections + /// + /// Caller must ensure that this structure is properly Disposed in all executions paths once the buffers are not needed anymore! + /// It is safe to call Dispose() multiple times (though the buffers will be unpinned on the first call) + /// + public struct Pinned : IDisposable + { + + /// GC Handle on the main buffer + internal GCHandle Handle; + + /// Additionnal GC Handles (optionnal) + internal readonly GCHandle[] Handles; + + internal object Owner; + + internal Pinned([NotNull] object owner, [NotNull] byte[] buffer, [CanBeNull] List extra) + { + Contract.Requires(owner != null && buffer != null); + + this.Owner = buffer; + this.Handle = GCHandle.Alloc(buffer, GCHandleType.Pinned); + if (extra == null || extra.Count == 0) + { + this.Handles = null; + } + else + { + var handles = new GCHandle[extra.Count]; + this.Handles = handles; + int p = 0; + foreach (var chunk in extra) + { + handles[p++] = GCHandle.Alloc(chunk.Array, GCHandleType.Pinned); + } + handles[p] = GCHandle.Alloc(buffer); + } + } + + public bool IsAllocated => this.Handle.IsAllocated; + + public void Dispose() + { + if (this.Owner != null) + { + if (this.Handle.IsAllocated) this.Handle.Free(); + var handles = this.Handles; + if (handles != null) + { + for (int i = 0; i < handles.Length; i++) + { + if (handles[i].IsAllocated) handles[i].Free(); + } + } + this.Owner = null; + } + } + } + + [UsedImplicitly(ImplicitUseTargetFlags.WithMembers)] + private sealed class DebugView + { + private readonly Slice m_slice; + + public DebugView(Slice slice) + { + m_slice = slice; + } + + public int Count => m_slice.Count; + + public byte[] Data + { + get + { + if (m_slice.Count == 0) return m_slice.Array == null ? null : System.Array.Empty(); + if (m_slice.Offset == 0 && m_slice.Count == m_slice.Array.Length) return m_slice.Array; + var tmp = new byte[m_slice.Count]; + System.Array.Copy(m_slice.Array, m_slice.Offset, tmp, 0, m_slice.Count); + return tmp; + } + } + + public string Content => Slice.Dump(m_slice, maxSize: 1024); + + /// Encoding using only for display purpose: we don't want to throw in the 'Text' property if the input is not text! + [NotNull] + private static readonly UTF8Encoding Utf8NoBomEncodingNoThrow = new UTF8Encoding(encoderShouldEmitUTF8Identifier: false, throwOnInvalidBytes: false); + + public string Text + { + get + { + if (m_slice.Count == 0) return m_slice.Array == null ? null : String.Empty; + return EscapeString(new StringBuilder(m_slice.Count + 16), m_slice.Array, m_slice.Offset, m_slice.Count, Utf8NoBomEncodingNoThrow).ToString(); + } + } + + public string Hexa + { + get + { + if (m_slice.Count == 0) return m_slice.Array == null ? null : String.Empty; + return m_slice.Count <= 1024 + ? m_slice.ToHexaString(' ') + : m_slice.Substring(0, 1024).ToHexaString(' ') + "[\u2026]"; + } + } + + } + + } + + /// Helper methods for Slice + public static class SliceExtensions + { + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.NoInlining)] + private static Slice EmptyOrNil(byte[] array) + { + //note: we consider the "empty" or "nil" case less frequent, so we handle it in a non-inlined method + return array == null ? default(Slice) : Slice.Empty; + } + + /// Handle the Nil/Empty memoization + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.NoInlining)] + private static Slice EmptyOrNil([CanBeNull] byte[] array, int count) + { + //note: we consider the "empty" or "nil" case less frequent, so we handle it in a non-inlined method + if (array == null) return count == 0 ? default(Slice) : throw UnsafeHelpers.Errors.BufferArrayNotNull(); + return Slice.Empty; + } + + /// Return a slice that wraps the whole array + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice([CanBeNull] this byte[] bytes) + { + return bytes != null && bytes.Length > 0 ? new Slice(bytes, 0, bytes.Length) : EmptyOrNil(bytes); + } + + /// Return the tail of the array, starting from the specified offset + /// Underlying buffer to slice + /// Offset to the first byte of the slice + /// + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice([NotNull] this byte[] bytes, [Positive] int offset) + { + //note: this method is DANGEROUS! Caller may thing that it is passing a count instead of an offset. + Contract.NotNull(bytes, nameof(bytes)); + if ((uint) offset > (uint) bytes.Length) UnsafeHelpers.Errors.ThrowBufferArrayToSmall(); + return bytes.Length != 0 ? new Slice(bytes, offset, bytes.Length - offset) : Slice.Empty; + } + + /// Return a slice from the sub-section of the byte array + /// Underlying buffer to slice + /// Offset to the first element of the slice (if not empty) + /// Number of bytes to take + /// + /// Slice that maps the corresponding sub-section of the array. + /// If then either Slice.Empty or Slice.Nil will be returned, in order to not keep a reference to the whole buffer. + /// + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice([CanBeNull] this byte[] bytes, [Positive] int offset, [Positive] int count) + { + //note: this method will frequently be called with offset==0, so we should optimize for this case! + if (bytes == null | count == 0) return EmptyOrNil(bytes, count); + + // bound check + // ReSharper disable once PossibleNullReferenceException + if ((uint) offset >= (uint) bytes.Length || (uint) count > (uint) (bytes.Length - offset)) UnsafeHelpers.Errors.ThrowOffsetOutsideSlice(); + + return new Slice(bytes, offset, count); + } + + /// Return a slice from the sub-section of the byte array + /// Underlying buffer to slice + /// Offset to the first element of the slice (if not empty) + /// Number of bytes to take + /// + /// Slice that maps the corresponding sub-section of the array. + /// If then either Slice.Empty or Slice.Nil will be returned, in order to not keep a reference to the whole buffer. + /// + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice([CanBeNull] this byte[] bytes, uint offset, uint count) + { + //note: this method will frequently be called with offset==0, so we should optimize for this case! + if (bytes == null | count == 0) return EmptyOrNil(bytes, (int) count); + + // bound check + if (offset >= (uint) bytes.Length || count > ((uint) bytes.Length - offset)) UnsafeHelpers.Errors.ThrowOffsetOutsideSlice(); + + return new Slice(bytes, (int) offset, (int) count); + } + + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice(this ArraySegment self) + { + // We trust the ArraySegment ctor to valide the arguments before hand. + // If somehow the arguments were corrupted (intentionally or not), then the same problem could have happened with the slice anyway! + + // ReSharper disable once AssignNullToNotNullAttribute + return self.Count != 0 ? new Slice(self.Array, self.Offset, self.Count) : EmptyOrNil(self.Array, self.Count); + } + + /// Return a slice from the sub-section of an array segment + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Slice AsSlice(this ArraySegment self, int offset, int count) + { + return AsSlice(self).Substring(offset, count); + } + +#if ENABLE_SPAN + /// Convert this into the equivalent ReadOnlySpan<byte>. + /// Both and will be converted into an empty span + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ReadOnlySpan AsReadOnlySpan(this Slice self) + { + return new ReadOnlySpan(self.Array, self.Offset, self.Count); + } + + /// Convert this into the equivalent ReadOnlySpan<byte>. + /// If is + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ReadOnlySpan AsReadOnlySpan(this Slice self, int start) + { + var x = self.Substring(start); + return new ReadOnlySpan(x.Array, x.Offset, x.Count); + } + + /// Convert this into the equivalent ReadOnlySpan<byte>. + /// If is + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ReadOnlySpan AsReadOnlySpan(this Slice self, int start, int length) + { + var x = self.Substring(start, length); + return new ReadOnlySpan(x.Array, x.Offset, x.Count); + } + + /// Convert this into the equivalent Span<byte>. + /// Both and will be converted into an empty span + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Span AsSpan(this Slice self) + { + return new Span(self.Array, self.Offset, self.Count); + } + + /// Convert this into the equivalent Span<byte>. + /// If is + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Span AsSpan(this Slice self, int start) + { + var x = self.Substring(start); + return new Span(x.Array, x.Offset, x.Count); + } + + /// Convert this into the equivalent Span<byte>. + /// If is + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Span AsSpan(this Slice self, int start, int length) + { + var x = self.Substring(start, length); + return new Span(x.Array, x.Offset, x.Count); + } +#endif + + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static SliceReader ToSliceReader(this byte[] self) + { + return new SliceReader(self); + } + + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static SliceReader ToSliceReader(this byte[] self, int count) + { + return new SliceReader(self, 0, count); + } + + [Pure, DebuggerNonUserCode, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static SliceReader ToSliceReader(this byte[] self, int offset, int count) + { + return new SliceReader(self, offset, count); + } + + [Pure, NotNull, DebuggerNonUserCode] + public static SliceStream AsStream(this Slice slice) //REVIEW: => ToStream() ? + { + if (slice.IsNull) throw ThrowHelper.InvalidOperationException("Slice cannot be null"); + //TODO: have a singleton for the emtpy slice ? + return new SliceStream(slice); + } + +#if ENABLE_SPAN + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyTo(this ReadOnlySpan source, Slice destination) + { + source.CopyTo(new Span(destination.Array, destination.Offset, destination.Count)); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyTo(this Span source, Slice destination) + { + if (source.Length > 0) source.CopyTo(new Span(destination.Array, destination.Offset, destination.Count)); + } +#endif + + } + +} diff --git a/FoundationDB.Client/Utils/SliceBuffer.cs b/FoundationDB.Client/Utils/Memory/SliceBuffer.cs similarity index 66% rename from FoundationDB.Client/Utils/SliceBuffer.cs rename to FoundationDB.Client/Utils/Memory/SliceBuffer.cs index 35c1a8d6a..1ff5961f0 100644 --- a/FoundationDB.Client/Utils/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/Memory/SliceBuffer.cs @@ -26,20 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace Doxense.Memory { using System; using System.Collections.Generic; using System.Diagnostics; using Doxense.Diagnostics.Contracts; + using FoundationDB; using JetBrains.Annotations; /// Buffer that can be used to efficiently store multiple slices into as few chunks as possible /// - /// This class is usefull to centralize a lot of temporary slices whose lifetime is linked to a specific operation. Dropping the referce to the buffer will automatically reclaim all the slices that were stored with it. + /// This class is usefull to centralize a lot of temporary slices whose lifetime is linked to a specific operation. Dropping the reference to the buffer will automatically reclaim all the slices that were stored with it. /// This class is not thread safe. /// - [DebuggerDisplay("Pos={m_pos}, Remaining={m_remaining}, PageSize={m_pageSize}, Used={m_used+m_pos}, Allocated={m_allocated+m_pos+m_remaining}")] + [DebuggerDisplay("Pos={m_pos}, Remaining={m_remaining}, PageSize={m_pageSize}, Size={Size}, Allocated={Allocated}")] public sealed class SliceBuffer { private const int DefaultPageSize = 256; @@ -69,27 +70,18 @@ public SliceBuffer() /// Initial page size public SliceBuffer(int pageSize) { - if (pageSize < 0) throw new ArgumentOutOfRangeException("pageSize", "Page size cannt be less than zero"); - m_pageSize = pageSize == 0 ? DefaultPageSize : SliceHelpers.Align(pageSize); + if (pageSize < 0) throw new ArgumentOutOfRangeException(nameof(pageSize), "Page size cannt be less than zero"); + m_pageSize = pageSize == 0 ? DefaultPageSize : BitHelpers.AlignPowerOfTwo(pageSize, 16); } /// Gets the number of bytes used by all the slice allocated in this buffer - public int Size - { - get { return m_used + m_pos; } - } + public int Size => m_used + m_pos; /// Gets the total memory size allocated to store all the slices in this buffer - public int Allocated - { - get { return m_allocated + m_pos + m_remaining; } - } + public int Allocated => m_allocated + m_pos + m_remaining; /// Number of memory pages used by this buffer - public int PageCount - { - get { return m_chunks == null ? 1 : (m_chunks.Count + 1); } - } + public int PageCount => m_chunks?.Count + 1 ?? 1; /// Return the list of all the pages used by this buffer /// Array of pages used by the buffer @@ -97,76 +89,11 @@ public int PageCount public Slice[] GetPages() { var pages = new Slice[this.PageCount]; - if (m_chunks != null) m_chunks.CopyTo(pages); + m_chunks?.CopyTo(pages); pages[pages.Length - 1] = new Slice(m_current, 0, m_pos); return pages; } - /// Copy a pair of keys into the buffer, and return a new identical pair - /// Key range - /// Equivalent pair of keys, that are backed by the buffer. - public KeyRange InternRange(KeyRange range) - { - //TODO: if end is prefixed by begin, we could merge both keys (frequent when dealing with ranges on tuples that add \xFF - return new KeyRange( - Intern(range.Begin, aligned: true), - Intern(range.End, aligned: true) - ); - } - - /// Copy a pair of keys into the buffer, and return a new identical pair - /// Begin key of the range - /// End key of the range - /// Equivalent pair of keys, that are backed by the buffer. - public KeyRange InternRange(Slice begin, Slice end) - { - //TODO: if end is prefixed by begin, we could merge both keys (frequent when dealing with ranges on tuples that add \xFF - return new KeyRange( - Intern(begin, aligned: true), - Intern(end, aligned: true) - ); - } - - /// Copy a key into the buffer, and return a new range containing only that key - /// Key to copy to the buffer - /// Range equivalent to [key, key + '\0') that is backed by the buffer. - public KeyRange InternRangeFromKey(Slice key) - { - // Since the end key only adds \0 to the begin key, we can reuse the same bytes by making both overlap - var tmp = Intern(key, Slice.FromByte(0), aligned: true); - - return new KeyRange( - tmp.Substring(0, key.Count), - tmp - ); - } - - /// Copy a key selector into the buffer, and return a new identical selector - /// Key selector to copy to the buffer - /// Equivalent key selector that is backed by the buffer. - public KeySelector InternSelector(KeySelector selector) - { - return new KeySelector( - Intern(selector.Key, aligned: true), - selector.OrEqual, - selector.Offset - ); - } - - /// Copy a pair of key selectors into the buffer, and return a new identical pair - /// Pair of key selectors to copy to the buffer - /// Equivalent pair of key selectors that is backed by the buffer. - public KeySelectorPair InternSelectorPair(KeySelectorPair pair) - { - var begin = Intern(pair.Begin.Key, default(Slice), aligned: true); - var end = Intern(pair.End.Key, default(Slice), aligned: true); - - return new KeySelectorPair( - new KeySelector(begin, pair.Begin.OrEqual, pair.Begin.Offset), - new KeySelector(end, pair.End.OrEqual, pair.End.Offset) - ); - } - /// Allocate an empty space in the buffer /// Number of bytes to allocate /// If true, align the start of the slice with the default padding size. @@ -174,7 +101,7 @@ public KeySelectorPair InternSelectorPair(KeySelectorPair pair) /// There is NO garantees that the allocated slice will be pre-filled with zeroes. public Slice Allocate(int count, bool aligned = false) { - if (count < 0) throw new ArgumentException("Cannot allocate less than zero bytes.", "count"); + if (count < 0) throw new ArgumentException("Cannot allocate less than zero bytes.", nameof(count)); const int ALIGNMENT = 4; @@ -183,19 +110,20 @@ public Slice Allocate(int count, bool aligned = false) return Slice.Empty; } - int start = m_pos; - int extra = aligned ? (ALIGNMENT - (start & (ALIGNMENT - 1))) : 0; - if (count + extra > m_remaining) + int p = m_pos; + int r = m_remaining; + int extra = aligned ? (ALIGNMENT - (p & (ALIGNMENT - 1))) : 0; + if (count + extra > r) { // does not fit return AllocateFallback(count); } Contract.Assert(m_current != null && m_pos >= 0); - m_pos += count + extra; - m_remaining -= count + extra; + m_pos = p + (count + extra); + m_remaining = r - (count + extra); Contract.Ensures(m_remaining >= 0); //note: we rely on the fact that the buffer was pre-filled with zeroes - return Slice.Create(m_current, start + extra, count); + return new Slice(m_current, p + extra, count); } private Slice AllocateFallback(int count) @@ -224,7 +152,7 @@ private Slice AllocateFallback(int count) m_pos = count; m_remaining = pageSize - count; - return Slice.Create(buffer, 0, count); + return new Slice(buffer, 0, count); } /// Copy a slice into the buffer, with optional alignement, and return a new identical slice. @@ -239,11 +167,11 @@ public Slice Intern(Slice data, bool aligned = false) return data.Memoize(); } - SliceHelpers.EnsureSliceIsValid(ref data); + data.EnsureSliceIsValid(); // allocate the slice var slice = Allocate(data.Count, aligned); - SliceHelpers.CopyBytesUnsafe(slice.Array, slice.Offset, data.Array, data.Offset, data.Count); + UnsafeHelpers.CopyUnsafe(slice.Array, slice.Offset, data.Array, data.Offset, data.Count); return slice; } @@ -261,12 +189,12 @@ internal Slice Intern(Slice data, Slice suffix, bool aligned = false) return suffix.Count > 0 ? suffix : data.Array == null ? Slice.Nil : Slice.Empty; } - SliceHelpers.EnsureSliceIsValid(ref data); - SliceHelpers.EnsureSliceIsValid(ref suffix); + data.EnsureSliceIsValid(); + suffix.EnsureSliceIsValid(); var slice = Allocate(data.Count + suffix.Count, aligned); - SliceHelpers.CopyBytesUnsafe(slice.Array, slice.Offset, data.Array, data.Offset, data.Count); - SliceHelpers.CopyBytesUnsafe(slice.Array, slice.Offset + data.Count, suffix.Array, suffix.Offset, suffix.Count); + UnsafeHelpers.CopyUnsafe(slice.Array, slice.Offset, data.Array, data.Offset, data.Count); + UnsafeHelpers.CopyUnsafe(slice.Array, slice.Offset + data.Count, suffix.Array, suffix.Offset, suffix.Count); return slice; } @@ -302,7 +230,15 @@ private void Reset(bool keep) m_used = 0; if (!keep) m_current = null; } - } + /// Return a lock that will prevent the underlying byte arrays used by this buffer from moving around in memory during the next GC. + /// Lock instance that MUST be disposed to release the GC lock. + /// Any data added to the buffer WHILE the buffer is pinned MAY NOT be pinned itself! For safety, caller should make sure to write everything to the buffer before pinning it + public Slice.Pinned Pin() + { + return new Slice.Pinned(this, m_current, m_chunks); + } + + } } diff --git a/FoundationDB.Client/Utils/SliceListStream.cs b/FoundationDB.Client/Utils/Memory/SliceListStream.cs similarity index 83% rename from FoundationDB.Client/Utils/SliceListStream.cs rename to FoundationDB.Client/Utils/Memory/SliceListStream.cs index b01a10ca6..d422ba0d3 100644 --- a/FoundationDB.Client/Utils/SliceListStream.cs +++ b/FoundationDB.Client/Utils/Memory/SliceListStream.cs @@ -26,17 +26,20 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace Doxense.Memory { using System; using System.Collections.Generic; using System.IO; using System.Linq; + using System.Runtime.CompilerServices; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using FoundationDB; using JetBrains.Annotations; + //REVIEW: this is somewhat similar to the proposed new IBufferList (from System.IO.Pipeline) ? + /// Merge multiple slices into a single stream public sealed class SliceListStream : Stream { @@ -49,13 +52,13 @@ public sealed class SliceListStream : Stream internal SliceListStream([NotNull] Slice[] slices) { - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(slices, nameof(slices)); Init(slices); } public SliceListStream([NotNull] IEnumerable slices) { - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(slices, nameof(slices)); Init(slices.ToArray()); } @@ -72,31 +75,19 @@ private void Init([NotNull] Slice[] slices) #region Seeking... - public override bool CanSeek - { - get { return m_slices != null; } - } + public override bool CanSeek => m_slices != null; public override long Position { - get - { - return m_position; - } - set - { - Seek(value, SeekOrigin.Begin); - } + get => m_position; + set => Seek(value, SeekOrigin.Begin); } - public override long Length - { - get { return m_length; } - } + public override long Length => m_length; public override long Seek(long offset, SeekOrigin origin) { - if (m_slices == null) StreamIsClosed(); + if (m_slices == null) throw StreamIsClosed(); if (offset > int.MaxValue) throw new ArgumentOutOfRangeException(nameof(offset)); switch (origin) @@ -157,10 +148,7 @@ public override void SetLength(long value) #region Reading... - public override bool CanRead - { - get { return m_position < m_length; } - } + public override bool CanRead => m_position < m_length; private bool AdvanceToNextSlice() { @@ -198,7 +186,7 @@ public override int Read(byte[] buffer, int offset, int count) { ValidateBuffer(buffer, offset, count); - if (m_slices == null) StreamIsClosed(); + if (m_slices == null) throw StreamIsClosed(); Contract.Ensures(m_position >= 0 && m_position <= m_length); @@ -245,8 +233,6 @@ public override int Read(byte[] buffer, int offset, int count) return read; } -#if !NET_4_0 - public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { ValidateBuffer(buffer, offset, count); @@ -266,35 +252,26 @@ public override Task ReadAsync(byte[] buffer, int offset, int count, System } catch (Exception e) { - return TaskHelpers.FromException(e); + return Task.FromException(e); } } -#endif - #endregion #region Writing... - public override bool CanWrite - { - get { return false; } - } + public override bool CanWrite => false; public override void Write(byte[] buffer, int offset, int count) { throw new NotSupportedException(); } -#if !NET_4_0 - public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { - return TaskHelpers.FromException(new NotSupportedException()); + return Task.FromException(new NotSupportedException()); } -#endif - public override void Flush() { // Not supported, but don't throw here @@ -310,16 +287,16 @@ public override Task FlushAsync(System.Threading.CancellationToken ct) private static void ValidateBuffer(byte[] buffer, int offset, int count) { - if (buffer == null) throw new ArgumentNullException(nameof(buffer)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); - if (offset > buffer.Length - count) throw new ArgumentException("Offset and count must fit inside the buffer"); + Contract.NotNull(buffer, nameof(buffer)); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); + if (offset < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); + if (offset > buffer.Length - count) throw ThrowHelper.ArgumentException(nameof(offset), "Offset and count must fit inside the buffer"); } - [ContractAnnotation("=> halt")] - private static void StreamIsClosed() + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception StreamIsClosed() { - throw new ObjectDisposedException(null, "The stream was already closed"); + return ThrowHelper.ObjectDisposedException("The stream was already closed"); } protected override void Dispose(bool disposing) diff --git a/FoundationDB.Client/Utils/SlicePairComparer.cs b/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs similarity index 96% rename from FoundationDB.Client/Utils/SlicePairComparer.cs rename to FoundationDB.Client/Utils/Memory/SlicePairComparer.cs index 209f52fad..65c3d9f08 100644 --- a/FoundationDB.Client/Utils/SlicePairComparer.cs +++ b/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs @@ -26,15 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace Doxense.Memory { using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; + using FoundationDB; /// Performs optimized equality and comparison checks on key/value pairs of public sealed class SlicePairComparer : IComparer>, IEqualityComparer> { + //TODO: move this inside Slmice? (Slice.PairComparer.Default ...) + private const int BOTH = 0; private const int KEY_ONLY = 1; private const int VALUE_ONLY = 2; diff --git a/FoundationDB.Client/Utils/Memory/SliceReader.cs b/FoundationDB.Client/Utils/Memory/SliceReader.cs new file mode 100644 index 000000000..c54c16fae --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/SliceReader.cs @@ -0,0 +1,423 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Memory +{ + using System; + using System.Diagnostics; + using System.Runtime.CompilerServices; + using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB; + using JetBrains.Annotations; + + /// Helper class that holds the internal state used to parse tuples from slices + /// This struct MUST be passed by reference! + [PublicAPI, DebuggerDisplay("{Position}/{Buffer.Count}, NextByte={PeekByte()}")] + [DebuggerNonUserCode] //remove this when you need to troubleshoot this class! + public struct SliceReader + { + + /// Buffer containing the tuple being parsed + public readonly Slice Buffer; + + /// Current position inside the buffer + public int Position; + + /// Creates a new reader over a slice + /// Slice that will be used as the underlying buffer + public SliceReader(Slice buffer) + { + buffer.EnsureSliceIsValid(); + this.Buffer = buffer; + this.Position = 0; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public SliceReader(Slice buffer, int offset) + { + buffer.EnsureSliceIsValid(); + this.Buffer = buffer.Substring(offset); + this.Position = 0; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public SliceReader([NotNull] byte[] buffer) + { + this.Buffer = new Slice(buffer, 0, buffer.Length); + this.Position = 0; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public SliceReader([NotNull] byte[] buffer, int offset, int count) + { + this.Buffer = new Slice(buffer, offset, count); + this.Position = 0; + } + + /// Returns true if there are more bytes to parse + public bool HasMore => this.Position < this.Buffer.Count; + + /// Returns the number of bytes remaining + public int Remaining => Math.Max(0, this.Buffer.Count - this.Position); + + /// Returns a slice with all the bytes read so far in the buffer + public Slice Head => this.Buffer.Substring(0, this.Position); + + /// Returns a slice with all the remaining bytes in the buffer + public Slice Tail => this.Buffer.Substring(this.Position); + + /// Ensure that there are at least bytes remaining in the buffer + [MethodImpl(MethodImplOptions.AggressiveInlining)] + [DebuggerNonUserCode] + public void EnsureBytes(int count) + { + if (count < 0 || checked(this.Position + count) > this.Buffer.Count) throw ThrowNotEnoughBytes(count); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + [DebuggerNonUserCode] + private static Exception ThrowNotEnoughBytes(int count) + { + return ThrowHelper.FormatException($"The buffer does not have enough data to satisfy a read of {count} byte(s)"); + } + + /// Return the value of the next byte in the buffer, or -1 if we reached the end + [Pure] + public int PeekByte() + { + int p = this.Position; + return p < this.Buffer.Count ? this.Buffer[p] : -1; + } + + /// Return the value of the byte at a specified offset from the current position, or -1 if this is after the end, or before the start + [Pure] + public int PeekByteAt(int offset) + { + int p = this.Position + offset; + return p < this.Buffer.Count && p >= 0 ? this.Buffer[p] : -1; + } + + public Slice PeekBytes(int count) + { + return this.Buffer.Substring(this.Position, count); + } + + /// Attempt to peek at the next bytes from the reader, without advancing the pointer + /// Number of bytes to peek + /// Receives the corresponding slice if there are enough bytes remaining. + /// If true, the next are available in . If false, there are not enough bytes remaining in the buffer. + public bool TryPeekBytes(int count, out Slice bytes) + { + if (this.Remaining < count) + { + bytes = default(Slice); + return false; + } + bytes = this.Buffer.Substring(this.Position, count); + return true; + } + + /// Skip the next bytes of the buffer + public void Skip(int count) + { + EnsureBytes(count); + + this.Position += count; + } + + /// Read the next byte from the buffer + public byte ReadByte() + { + EnsureBytes(1); + + int p = this.Position; + byte b = this.Buffer[p]; + this.Position = p + 1; + return b; + } + + /// Read the next bytes from the buffer + public Slice ReadBytes(int count) + { + if (count == 0) return Slice.Empty; + + EnsureBytes(count); + int p = this.Position; + this.Position = p + count; + return this.Buffer.Substring(p, count); + } + + /// Read the next bytes from the buffer + public Slice ReadBytes(uint count) + { + int n = checked((int) count); + EnsureBytes(n); + + int p = this.Position; + this.Position = p + n; + return this.Buffer.Substring(p, n); + } + + /// Read until returns true, or we reach the end of the buffer + [Pure] + public Slice ReadWhile([NotNull] Func handler) + { + unsafe + { + int start = this.Position; + int count = 0; + fixed (byte* bytes = &this.Buffer.DangerousGetPinnableReference()) + { + byte* ptr = bytes; + byte* end = bytes + this.Remaining; + while (ptr < end) + { + if (!handler(*ptr, count)) + { + break; + } + ++ptr; + ++count; + } + this.Position = start + count; + return this.Buffer.Substring(start, count); + } + } + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice ReadToEnd() + { + return ReadBytes(this.Remaining); + } + + /// Read the next 2 bytes as an unsigned 16-bit integer, encoded in little-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ushort ReadFixed16() + { + return ReadBytes(2).ToUInt16(); + } + + /// Read the next 3 bytes as an unsigned 24-bit integer, encoded in little-endian + /// Bits 24 to 31 will always be zero + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public uint ReadFixed24() + { + return ReadBytes(3).ToUInt24(); + } + + + /// Read the next 4 bytes as an unsigned 32-bit integer, encoded in little-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public uint ReadFixed32() + { + return ReadBytes(4).ToUInt32(); + } + + /// Read the next 8 bytes as an unsigned 64-bit integer, encoded in little-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ulong ReadFixed64() + { + return ReadBytes(8).ToUInt64(); + } + + /// Read the next 2 bytes as an unsigned 16-bit integer, encoded in big-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ushort ReadFixed16BE() + { + return ReadBytes(2).ToUInt16BE(); + } + + /// Read the next 3 bytes as an unsigned 24-bit integer, encoded in big-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public uint ReadFixed24BE() + { + return ReadBytes(3).ToUInt24BE(); + } + + /// Read the next 4 bytes as an unsigned 32-bit integer, encoded in big-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public uint ReadFixed32BE() + { + return ReadBytes(4).ToUInt32BE(); + } + + /// Read the next 8 bytes as an unsigned 64-bit integer, encoded in big-endian + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ulong ReadFixed64BE() + { + return ReadBytes(8).ToUInt64BE(); + } + + /// Read the next 4 bytes as an IEEE 32-bit floating point number + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public float ReadSingle() + { + return ReadBytes(4).ToSingle(); + } + + /// Read the next 8 bytes as an IEEE 64-bit floating point number + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public double ReadDouble() + { + return ReadBytes(8).ToDouble(); + } + + /// Read an encoded nul-terminated byte array from the buffer + [Pure] + public Slice ReadByteString() + { + var buffer = this.Buffer.Array; + int start = this.Buffer.Offset + this.Position; + int p = start; + int end = this.Buffer.Offset + this.Buffer.Count; + + while (p < end) + { + byte b = buffer[p++]; + if (b == 0) + { + //TODO: decode \0\xFF ? + if (p < end && buffer[p] == 0xFF) + { + // skip the next byte and continue + p++; + continue; + } + + this.Position = p - this.Buffer.Offset; + return new Slice(buffer, start, p - start); + } + } + + throw ThrowHelper.FormatException("Truncated byte string (expected terminal NUL not found)"); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint16') from the buffer, and advances the cursor + /// Can Read up to 3 bytes from the input + [Pure] + public ushort ReadVarInt16() + { + //note: this could read up to 21 bits of data, so we check for overflow + return checked((ushort)ReadVarInt(3)); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor + /// Can Read up to 5 bytes from the input + [Pure] + public uint ReadVarInt32() + { + //note: this could read up to 35 bits of data, so we check for overflow + return checked((uint)ReadVarInt(5)); + } + + /// Reads a 7-bit encoded unsigned long (aka 'Varint32') from the buffer, and advances the cursor + /// Can Read up to 10 bytes from the input + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ulong ReadVarInt64() + { + return ReadVarInt(10); + } + + /// Reads a Base 128 Varint from the input + /// Maximum number of bytes allowed (5 for 32 bits, 10 for 64 bits) + private ulong ReadVarInt(int count) + { + var buffer = this.Buffer.Array; + int p = this.Buffer.Offset + this.Position; + int end = this.Buffer.Offset + this.Buffer.Count; + + ulong x = 0; + int s = 0; + + // read bytes until the MSB is unset + while (count-- > 0) + { + if (p > end) throw ThrowHelper.FormatException("Truncated Varint"); + byte b = buffer[p++]; + + x |= (b & 0x7FUL) << s; + if (b < 0x80) + { + this.Position = p - this.Buffer.Offset; + return x; + } + s += 7; + } + throw ThrowHelper.FormatException("Malformed Varint"); + } + + /// Reads a variable sized slice, by first reading its size (stored as a Varint32) and then the data + [Pure] + public Slice ReadVarBytes() + { + uint size = ReadVarInt32(); + if (size > int.MaxValue) throw ThrowHelper.FormatException("Malformed variable-sized array"); + if (size == 0) return Slice.Empty; + return ReadBytes((int)size); + } + + /// Reads an utf-8 encoded string prefixed by a variable-sized length + [Pure, NotNull] + public string ReadVarString() + { + var str = ReadVarBytes(); + return str.ToStringUtf8(); + } + + /// Reads a string prefixed by a variable-sized length, using the specified encoding + /// Encoding used for this string (or UTF-8 if null) + [Pure, NotNull] + public string ReadVarString([CanBeNull] Encoding encoding) + { + if (encoding == null || encoding.Equals(Encoding.UTF8)) + { // optimized path for utf-8 + return ReadVarString(); + } + // generic decoding + var bytes = ReadVarBytes(); + return bytes.Count > 0 ? encoding.GetString(bytes.Array, bytes.Offset, bytes.Count) : string.Empty; + } + + /// Reads a 128-bit UUID + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid128 ReadUuid128() + { + return ReadBytes(16).ToUuid128(); + } + + /// Reads a 64-bit UUID + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid64 ReadUuid64() + { + return ReadBytes(8).ToUuid64(); + } + } + +} diff --git a/FoundationDB.Client/Utils/SliceStream.cs b/FoundationDB.Client/Utils/Memory/SliceStream.cs similarity index 88% rename from FoundationDB.Client/Utils/SliceStream.cs rename to FoundationDB.Client/Utils/Memory/SliceStream.cs index aa66e72b7..7fb2bfd4e 100644 --- a/FoundationDB.Client/Utils/SliceStream.cs +++ b/FoundationDB.Client/Utils/Memory/SliceStream.cs @@ -26,12 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace Doxense.Memory { using System; using System.IO; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using FoundationDB; + using JetBrains.Annotations; /// Stream that wraps a Slice for reading /// This stream is optimized for blocking and async reads @@ -50,29 +52,17 @@ public SliceStream(Slice slice) #region Seeking... /// Returns true if the underlying slice is not null - public override bool CanSeek - { - get { return m_slice.HasValue; } - } + public override bool CanSeek => m_slice.HasValue; /// Gets or sets the current position in the underlying slice public override long Position { - get - { - return m_position; - } - set - { - Seek(value, SeekOrigin.Begin); - } + get => m_position; + set => Seek(value, SeekOrigin.Begin); } /// Getes the length of the underlying slice - public override long Length - { - get { return m_slice.Count; } - } + public override long Length => m_slice.Count; /// Seeks to a specific location in the underlying slice public override long Seek(long offset, SeekOrigin origin) @@ -126,10 +116,7 @@ public override void SetLength(long value) #region Reading... /// Returns true unless the current position is after the end of the underlying slice - public override bool CanRead - { - get { return m_position < m_slice.Count; } - } + public override bool CanRead => m_position < m_slice.Count; /// Reads from byte from the underyling slice and advances the position within the slice by one byte, or returns -1 if the end of the slice has been reached. public override int ReadByte() @@ -176,8 +163,6 @@ public override int Read(byte[] buffer, int offset, int count) return remaining; } -#if !NET_4_0 - /// Asynchronously reads a sequence of bytes from the underlying slice and advances the position within the slice by the number of bytes read. public override Task ReadAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { @@ -207,7 +192,7 @@ public override Task CopyToAsync(Stream destination, int bufferSize, System.Thre { Contract.Ensures(m_position >= 0 && m_position <= m_slice.Count); - if (destination == null) throw new ArgumentNullException(nameof(destination)); + Contract.NotNull(destination, nameof(destination)); if (!destination.CanWrite) throw new ArgumentException("The destination stream cannot be written to", nameof(destination)); int remaining = m_slice.Count - m_position; @@ -220,16 +205,16 @@ public override Task CopyToAsync(Stream destination, int bufferSize, System.Thre return destination.WriteAsync(m_slice.Array, m_slice.Offset, remaining, ct); } -#endif - #endregion #region Writing... /// Always return false - public override bool CanWrite + public override bool CanWrite => false; + + public override void WriteByte(byte value) { - get { return false; } + throw new NotSupportedException(); } /// This methods is not supported @@ -238,16 +223,12 @@ public override void Write(byte[] buffer, int offset, int count) throw new NotSupportedException(); } -#if !NET_4_0 - /// This methods is not supported public override Task WriteAsync(byte[] buffer, int offset, int count, System.Threading.CancellationToken ct) { return Task.FromException(new NotSupportedException()); } -#endif - /// This methods does nothing. public override void Flush() { @@ -265,15 +246,15 @@ public override Task FlushAsync(System.Threading.CancellationToken ct) private static void ValidateBuffer(byte[] buffer, int offset, int count) { - if (buffer == null) throw new ArgumentNullException(nameof(buffer)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero"); - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); - if (offset > buffer.Length - count) throw new ArgumentException("Offset and count must fit inside the buffer"); + Contract.NotNull(buffer, nameof(buffer)); + if (count < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Count cannot be less than zero."); + if ((uint) offset > buffer.Length - count) throw ThrowHelper.ArgumentException(nameof(offset), "Buffer is too small."); } + [ContractAnnotation("=> halt")] private static void StreamIsClosed() { - throw new ObjectDisposedException(null, "The stream was already closed"); + throw ThrowHelper.ObjectDisposedException("The stream was already closed"); } /// Closes the stream diff --git a/FoundationDB.Client/Utils/Memory/SliceWriter.cs b/FoundationDB.Client/Utils/Memory/SliceWriter.cs new file mode 100644 index 000000000..0deaad781 --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/SliceWriter.cs @@ -0,0 +1,2295 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +//#define ENABLE_ARRAY_POOL +//#define ENABLE_SPAN + +namespace Doxense.Memory +{ + using System; + using System.Diagnostics; + using System.Globalization; + using System.Runtime.CompilerServices; + using System.Text; + using Doxense.Diagnostics.Contracts; + using FoundationDB; + using JetBrains.Annotations; +#if ENABLE_SPAN + using System.Runtime.InteropServices; +#endif + + /// Slice buffer that emulates a pseudo-stream using a byte array that will automatically grow in size, if necessary + /// This struct MUST be passed by reference! + [PublicAPI, DebuggerDisplay("Position={Position}, Capacity={Capacity}"), DebuggerTypeProxy(typeof(SliceWriter.DebugView))] + [DebuggerNonUserCode] //remove this when you need to troubleshoot this class! + public struct SliceWriter + { + // Invariant + // * Valid data always start at offset 0 + // * 'this.Position' is equal to the current size as well as the offset of the next available free spot + // * 'this.Buffer' is either null (meaning newly created stream), or is at least as big as this.Position + + #region Private Members... + + /// Buffer holding the data + public byte[] Buffer; + + /// Position in the buffer ( == number of already written bytes) + public int Position; + + #endregion + + #region Constructors... + + /// Create a new empty binary buffer with an initial allocated size + /// Initial capacity of the buffer + public SliceWriter(int capacity) + { + Contract.Positive(capacity, nameof(capacity)); + +#if ENABLE_ARRAY_POOL + this.Buffer = capacity == 0 ? Array.Empty() : ArrayPool.Shared.Rent(capacity); +#else + this.Buffer = capacity == 0 ? Array.Empty() : new byte[capacity]; +#endif + this.Position = 0; + } + + /// Create a new binary writer using an existing buffer + /// Initial buffer + /// Since the content of the will be modified, only a temporary or scratch buffer should be used. If the writer needs to grow, a new buffer will be allocated. + public SliceWriter([NotNull] byte[] buffer) + : this(buffer, 0) + { } + + /// Create a new binary buffer using an existing buffer and with the cursor to a specific location + /// Since the content of the will be modified, only a temporary or scratch buffer should be used. If the writer needs to grow, a new buffer will be allocated. + public SliceWriter([NotNull] byte[] buffer, int index) + { + Contract.NotNull(buffer, nameof(buffer)); + Contract.Between(index, 0, buffer.Length, nameof(index)); + + this.Buffer = buffer; + this.Position = index; + } + + /// Creates a new binary buffer, initialized by copying pre-existing data + /// Data that will be copied at the start of the buffer + /// Optional initial capacity of the buffer + /// The cursor will already be placed at the end of the prefix + public SliceWriter(Slice prefix, int capacity = 0) + { + prefix.EnsureSliceIsValid(); + Contract.Positive(capacity, nameof(capacity)); + + int n = prefix.Count; + Contract.Assert(n >= 0); + + if (capacity == 0) + { // most frequent usage is to add a packed integer at the end of a prefix + capacity = BitHelpers.AlignPowerOfTwo(n + 8, 16); + } + else + { + capacity = BitHelpers.AlignPowerOfTwo(Math.Max(capacity, n), 16); + } + +#if ENABLE_ARRAY_POOL + var buffer = ArrayPool.Shared.Rent(capacity); +#else + var buffer = new byte[capacity]; +#endif + if (n > 0) prefix.CopyTo(buffer, 0); + + this.Buffer = buffer; + this.Position = n; + } + + #endregion + + #region Public Properties... + + /// Returns true if the buffer contains at least some data + public bool HasData => this.Position > 0; + + /// Capacity of the internal buffer + public int Capacity => this.Buffer?.Length ?? 0; + + /// Return the byte at the specified index + /// Index in the buffer (0-based if positive, from the end if negative) + public byte this[int index] + { + [Pure] + get + { + int pos = this.Position; + Contract.Assert(this.Buffer != null && pos >= 0); + //note: we will get bound checking for free in release builds + if (index < 0) index += pos; + if ((uint) index >= pos) throw ThrowHelper.IndexOutOfRangeException(); + return this.Buffer[index]; + } + } + + /// Returns a slice pointing to a segment inside the buffer + /// The starting position of the substring. Positive values means from the start, negative values means from the end + /// The end position (excluded) of the substring. Positive values means from the start, negative values means from the end + /// Slice that corresponds to the section selected. If the if equal to or greater than then an empty Slice is returned + /// If either or is outside of the currently allocated buffer. + public Slice this[int? beginInclusive, int? endExclusive] + { + [Pure] + get + { + int from = beginInclusive ?? 0; + int pos = this.Position; + int until = endExclusive ?? pos; + + // remap negative indexes + if (from < 0) from += pos; + if (until < 0) until += pos; + + // bound check + if ((uint) from >= pos) throw ThrowHelper.ArgumentOutOfRangeException(nameof(beginInclusive), beginInclusive, "The start index must be inside the bounds of the buffer."); + if ((uint) until > pos) throw ThrowHelper.ArgumentOutOfRangeException(nameof(endExclusive), endExclusive, "The end index must be inside the bounds of the buffer."); + + // chop chop + int count = until - from; + return count > 0 ? new Slice(this.Buffer, from, count) : Slice.Empty; + } + } + + #endregion + + /// Returns a byte array filled with the contents of the buffer + /// The buffer is copied in the byte array. And change to one will not impact the other + [Pure, NotNull] + public byte[] GetBytes() + { + int p = this.Position; + if (p == 0) return Array.Empty(); + + var bytes = new byte[p]; + if (p > 0) + { + Contract.Assert(this.Buffer != null && this.Buffer.Length >= this.Position); + UnsafeHelpers.CopyUnsafe(bytes, 0, this.Buffer, 0, bytes.Length); + } + return bytes; + } + + /// Returns a buffer segment pointing to the content of the buffer + /// Any change to the segment will change the buffer ! + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public ArraySegment ToArraySegment() + { + return ToSlice(); + } + + /// Returns a slice pointing to the content of the buffer + /// Any change to the slice will change the buffer ! + [Pure] + public Slice ToSlice() + { + var buffer = this.Buffer; + var p = this.Position; + if (buffer == null | p == 0) + { // empty buffer + return Slice.Empty; + } + Contract.Assert(buffer.Length >= p, "Current position is outside of the buffer"); + return new Slice(buffer, 0, p); + } + + /// Returns a slice pointing to the first bytes of the buffer + /// Size of the segment to return. + /// Slice that contains the first bytes written to this buffer + /// Any change to the slice will change the buffer ! + /// + /// ({HELLO WORLD}).Head(5) => {HELLO} + /// ({HELLO WORLD}).Head(1) => {H} + /// {{HELLO WORLD}).Head(0) => {} + /// + /// If is less than zero, or larger than the current buffer size + [Pure] + public Slice Head(int count) + { + if (count == 0) return Slice.Empty; + if ((uint) count > this.Position) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Buffer is too small"); + return new Slice(this.Buffer, 0, count); + } + + /// Returns a slice pointing to the first bytes of the buffer + /// Size of the segment to return. + /// Slice that contains the first bytes written to this buffer + /// Any change to the slice will change the buffer ! + /// + /// ({HELLO WORLD}).Head(5) => {HELLO} + /// ({HELLO WORLD}).Head(1) => {H} + /// {{HELLO WORLD}).Head(0) => {} + /// + /// If is less than zero, or larger than the current buffer size + [Pure] + public Slice Head(uint count) + { + if (count == 0) return Slice.Empty; + if (count > this.Position) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Buffer is too small"); + return new Slice(this.Buffer, 0, (int) count); + } + + /// Returns a slice pointer to the last bytes of the buffer + /// Size of the segment to return. + /// Slice that contains the last bytes written to this buffer + /// Any change to the slice will change the buffer ! + /// + /// ({HELLO WORLD}).Tail(5) => {WORLD} + /// ({HELLO WORLD}).Tail(1) => {D} + /// {{HELLO WORLD}).Tail(0) => {} + /// + /// If is less than zero, or larger than the current buffer size + public Slice Tail(int count) + { + if (count == 0) return Slice.Empty; + int p = this.Position; + if ((uint) count > p) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Buffer is too small"); + return new Slice(this.Buffer, p - count, count); + } + + /// Returns a slice pointer to the last bytes of the buffer + /// Size of the segment to return. + /// Slice that contains the last bytes written to this buffer + /// Any change to the slice will change the buffer ! + /// + /// ({HELLO WORLD}).Tail(5) => {WORLD} + /// ({HELLO WORLD}).Tail(1) => {D} + /// {{HELLO WORLD}).Tail(0) => {} + /// + /// If is less than zero, or larger than the current buffer size + public Slice Tail(uint count) + { + if (count == 0) return Slice.Empty; + int p = this.Position; + if (count > p) throw ThrowHelper.ArgumentOutOfRangeException(nameof(count), "Buffer is too small"); + return new Slice(this.Buffer, p - (int) count, (int) count); + } + + /// Returns a slice pointing to a segment inside the buffer + /// Offset of the segment from the start of the buffer + /// Any change to the slice will change the buffer ! + /// If is less then zero, or after the current position + [Pure] + public Slice Substring(int offset) + { + int p = this.Position; + if (offset < 0 || offset > p) throw ThrowHelper.ArgumentException(nameof(offset), "Offset must be inside the buffer"); + int count = p - offset; + return count > 0 ? new Slice(this.Buffer, offset, p - offset) : Slice.Empty; + } + + /// Returns a slice pointing to a segment inside the buffer + /// Offset of the segment from the start of the buffer + /// Size of the segment + /// Any change to the slice will change the buffer ! + /// If either or are less then zero, or do not fit inside the current buffer + [Pure] + public Slice Substring(int offset, int count) + { + int p = this.Position; + if ((uint) offset >= p) throw ThrowHelper.ArgumentException(nameof(offset), "Offset must be inside the buffer"); + if (count < 0 | offset + count > p) throw ThrowHelper.ArgumentException(nameof(count), "The buffer is too small"); + + return count > 0 ? new Slice(this.Buffer, offset, count) : Slice.Empty; + } + + /// Truncate the buffer by setting the cursor to the specified position. + /// New size of the buffer + /// If the buffer was smaller, it will be resized and filled with zeroes. If it was biffer, the cursor will be set to the specified position, but previous data will not be deleted. + public void SetLength(int position) + { + Contract.Requires(position >= 0); + + int p = this.Position; + if (p < position) + { + int missing = position - p; + var buffer = EnsureBytes(missing); + //TODO: native memset() ? + Array.Clear(buffer, p, missing); + } + this.Position = position; + } + + /// Delete the first N bytes of the buffer, and shift the remaining to the front + /// Number of bytes to remove at the head of the buffer + /// New size of the buffer (or 0 if it is empty) + /// This should be called after every successfull write to the underlying stream, to update the buffer. + public int Flush(int bytes) //REVIEW: plutot renommer en "RemoveHead"? ou faire un vrai "RemoveAt(offset, count)" ? + { + if (bytes == 0) return this.Position; + if (bytes < 0) throw ThrowHelper.ArgumentOutOfRangeException(nameof(bytes)); + + if (bytes < this.Position) + { // copy the left over data to the start of the buffer + int remaining = this.Position - bytes; + UnsafeHelpers.CopyUnsafe(this.Buffer, 0, this.Buffer, bytes, remaining); + this.Position = remaining; + return remaining; + } + else + { + //REVIEW: should we throw if there are less bytes in the buffer than we want to flush ? + this.Position = 0; + return 0; + } + } + + /// Empties the current buffer after a succesfull write + /// If true, fill the existing buffer with zeroes, if it is reused, to ensure that no previous data can leak. + /// If the current buffer is large enough, and less than 1/8th was used, then it will be discarded and a new smaller one will be allocated as needed + public void Reset(bool zeroes = false) + { + if (this.Position > 0) + { + Contract.Assert(this.Buffer != null && this.Buffer.Length >= this.Position); + // reduce size ? + // If the buffer exceeds 64K and we used less than 1/8 of it the last time, we will "shrink" the buffer + if (this.Buffer.Length > 65536 && this.Position <= (this.Buffer.Length >> 3)) + { // kill the buffer + this.Buffer = null; + //TODO: return to a central buffer pool? + } + else if (zeroes) + { // Clear it + unsafe + { + fixed (byte* ptr = this.Buffer) + { + UnsafeHelpers.ClearUnsafe(ptr, checked((uint)this.Buffer.Length)); + } + } + } + this.Position = 0; + } + } + + /// Advance the cursor of the buffer without writing anything, and return the previous position + /// Number of bytes to skip + /// Pad value (0xFF by default) + /// Position of the cursor BEFORE moving it. Can be used as a marker to go back later and fill some value + /// Will fill the skipped bytes with + public int Skip(int skip, byte pad = 0xFF) + { + Contract.Requires(skip >= 0); + + var buffer = EnsureBytes(skip); + int p = this.Position; + if (skip == 0) return p; + if (skip <= 8) + { + for (int i = 0; i < skip; i++) + { + buffer[p + i] = pad; + } + } + else + { + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + UnsafeHelpers.FillUnsafe(ptr, checked((uint) skip), pad); + } + } + } + this.Position = p + skip; + return p; + } + + /// Advance the cursor by the specified amount, and return the skipped over chunk (that can be filled later by the caller) + /// Number of bytes to allocate + /// Pad value (0xFF by default) + /// Slice that corresponds to the reserved segment in the buffer + /// Will fill the reserved segment with and the cursor will be positionned immediately after the segment. + public Slice Allocate(int count, byte pad = 0xFF) + { + Contract.Positive(count, nameof(count)); + if (count == 0) return Slice.Empty; + + int offset = Skip(count, pad); + return new Slice(this.Buffer, offset, count); + } + + /// Advance the cursor by the amount required end up on an aligned byte position + /// Number of bytes to align to + /// Pad value (0 by default) + public void Align(int aligment, byte pad = 0) + { + Contract.Requires(aligment > 0); + int r = this.Position % aligment; + if (r > 0) Skip(aligment - r, pad); + } + + /// Rewinds the cursor to a previous position in the buffer, while saving the current position + /// Will receive the current cursor position + /// Previous position in the buffer + public void Rewind(out int cursor, int position) + { + Contract.Requires(position >= 0 && position <= this.Position); + cursor = this.Position; + this.Position = position; + } + + #region Bytes... + + /// Add a byte to the end of the buffer, and advance the cursor + /// Byte, 8 bits + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteByte(byte value) + { + var buffer = EnsureBytes(1); + int p = this.Position; + buffer[p] = value; + this.Position = p + 1; + } + + /// Add a byte to the end of the buffer, and advance the cursor + /// Byte, 8 bits + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteByte(int value) + { + var buffer = EnsureBytes(1); + int p = this.Position; + buffer[p] = (byte) value; + this.Position = p + 1; + } + + /// Add a byte to the end of the buffer, and advance the cursor + /// Byte, 8 bits + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteByte(sbyte value) + { + var buffer = EnsureBytes(1); + int p = this.Position; + buffer[p] = (byte) value; + this.Position = p + 1; + } + + /// Add a 1-byte boolean to the end of the buffer, and advance the cursor + /// Boolean, encoded as either 0 or 1. + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteByte(bool value) + { + var buffer = EnsureBytes(1); + int p = this.Position; + buffer[p] = value ? (byte) 1 : (byte) 0; + this.Position = p + 1; + } + + /// Dangerously write a sigle byte at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void UnsafeWriteByte(byte value) + { + Contract.Requires(this.Buffer != null && this.Position < this.Buffer.Length); + this.Buffer[this.Position++] = value; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBytes(byte value1, byte value2) + { + var buffer = EnsureBytes(2); + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + this.Position = p + 2; + } + + /// Dangerously write two bytes at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void UnsafeWriteBytes(byte value1, byte value2) + { + Contract.Requires(this.Buffer != null && this.Position + 1 < this.Buffer.Length); + int p = this.Position; + this.Buffer[p] = value1; + this.Buffer[p + 1] = value2; + this.Position = p + 2; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBytes(byte value1, byte value2, byte value3) + { + var buffer = EnsureBytes(3); + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + this.Position = p + 3; + } + + /// Dangerously write three bytes at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void UnsafeWriteBytes(byte value1, byte value2, byte value3) + { + Contract.Requires(this.Buffer != null && this.Position + 2 < this.Buffer.Length); + var buffer = this.Buffer; + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + this.Position = p + 3; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBytes(byte value1, byte value2, byte value3, byte value4) + { + var buffer = EnsureBytes(4); + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + buffer[p + 3] = value4; + this.Position = p + 4; + } + + /// Dangerously write four bytes at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void UnsafeWriteBytes(byte value1, byte value2, byte value3, byte value4) + { + Contract.Requires(this.Buffer != null && this.Position + 3 < this.Buffer.Length); + var buffer = this.Buffer; + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + buffer[p + 3] = value4; + this.Position = p + 4; + } + + /// Dangerously write five bytes at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBytes(byte value1, byte value2, byte value3, byte value4, byte value5) + { + var buffer = EnsureBytes(5); + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + buffer[p + 3] = value4; + buffer[p + 4] = value5; + this.Position = p + 5; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void UnsafeWriteBytes(byte value1, byte value2, byte value3, byte value4, byte value5) + { + Contract.Requires(this.Buffer != null && this.Position + 4 < this.Buffer.Length); + var buffer = this.Buffer; + int p = this.Position; + buffer[p] = value1; + buffer[p + 1] = value2; + buffer[p + 2] = value3; + buffer[p + 3] = value4; + buffer[p + 4] = value5; + this.Position = p + 5; + } + + /// Write a byte array to the end of the buffer + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBytes([CanBeNull] byte[] data) + { + if (data != null) + { + WriteBytes(data, 0, data.Length); + } + } + + /// Write a chunk of a byte array to the end of the buffer + public void WriteBytes(byte[] data, int offset, int count) + { + if (count > 0) + { + UnsafeHelpers.EnsureBufferIsValidNotNull(data, offset, count); + int p = this.Position; + UnsafeHelpers.CopyUnsafe(EnsureBytes(count), p, data, offset, count); + this.Position = checked(p + count); + } + } + + /// Write a chunk of a byte array to the end of the buffer, with a prefix + public void WriteBytes(byte prefix, byte[] data, int offset, int count) + { + if (count >= 0) + { + if (count > 0) UnsafeHelpers.EnsureBufferIsValidNotNull(data, offset, count); + var buffer = EnsureBytes(count + 1); + int p = this.Position; + buffer[p] = prefix; + if (count > 0) UnsafeHelpers.CopyUnsafe(buffer, p + 1, data, offset, count); + this.Position = checked(p + 1 + count); + } + } + + /// Dangerously write a chunk of memory to the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + public void UnsafeWriteBytes(byte[] data, int offset, int count) + { + Contract.Requires(this.Buffer != null && this.Position >= 0 && data != null && count >= 0 && this.Position + count <= this.Buffer.Length && offset >= 0 && offset + count <= data.Length); + + if (count > 0) + { + int p = this.Position; + UnsafeHelpers.CopyUnsafe(this.Buffer, p, data, offset, count); + this.Position = checked(p + count); + } + } + + /// Write a segment of bytes to the end of the buffer + public void WriteBytes(Slice data) + { + data.EnsureSliceIsValid(); + + int count = data.Count; + if (count > 0) + { + int p = this.Position; + UnsafeHelpers.CopyUnsafe(EnsureBytes(count), p, data.Array, data.Offset, count); + this.Position = checked(p + count); + } + } + + /// Write a segment of bytes to the end of the buffer + public void WriteBytes(ref Slice data) + { + data.EnsureSliceIsValid(); + + int count = data.Count; + if (count > 0) + { + int p = this.Position; + UnsafeHelpers.CopyUnsafe(EnsureBytes(count), p, data.Array, data.Offset, count); + this.Position = checked(p + count); + } + } + +#if ENABLE_SPAN + /// Write a segment of bytes to the end of the buffer + public void WriteBytes(ReadOnlySpan data) + { + int count = data.Length; + if (count > 0) + { + int p = this.Position; + UnsafeHelpers.CopyUnsafe(EnsureBytes(count), p, data); + } + } +#endif + + /// Write a segment of bytes to the end of the buffer, with a prefix + public void WriteBytes(byte prefix, Slice data) + { + data.EnsureSliceIsValid(); + + int count = data.Count; + var buffer = EnsureBytes(count + 1); + int p = this.Position; + buffer[p] = prefix; + if (count > 0) UnsafeHelpers.CopyUnsafe(buffer, p + 1, data.Array, data.Offset, count); + this.Position = checked(p + count + 1); + } + +#if ENABLE_SPAN + /// Write a segment of bytes to the end of the buffer, with a prefix + public void WriteBytes(byte prefix, ReadOnlySpan data) + { + int count = data.Length; + var buffer = EnsureBytes(count + 1); + int p = this.Position; + buffer[p] = prefix; + if (count > 0) + { + UnsafeHelpers.CopyUnsafe(buffer, p + 1, data); + } + this.Position = checked(p + count + 1); + } +#endif + + /// Write a segment of bytes to the end of the buffer + public unsafe void WriteBytes(byte* data, uint count) + { + if (count == 0) return; + if (data == null) throw ThrowHelper.ArgumentNullException(nameof(data)); + + var buffer = EnsureBytes(count); + int p = this.Position; + Contract.Assert(buffer != null && p >= 0 && p + count <= buffer.Length); + + //note: we compute the end offset BEFORE, to protect against arithmetic overflow + int q = checked((int)(p + count)); + UnsafeHelpers.CopyUnsafe(buffer, p, data, count); + this.Position = q; + } + + /// Append a segment of bytes with a prefix to the end of the buffer + /// Byte added before the data + /// Pointer to the start of the data to append + /// Number of bytes to append (excluding the prefix) + public unsafe void WriteBytes(byte prefix, byte* data, uint count) + { + if (count != 0 && data == null) throw ThrowHelper.ArgumentNullException(nameof(data)); + + var buffer = EnsureBytes(count + 1); + int p = this.Position; + Contract.Assert(buffer != null && p >= 0 && p + 1 + count <= buffer.Length); + + //note: we compute the end offset BEFORE, to protect against arithmetic overflow + int q = checked((int)(p + 1 +count)); + buffer[p] = prefix; + UnsafeHelpers.CopyUnsafe(buffer, p + 1, data, count); + this.Position = q; + } + + /// Dangerously write a segment of bytes at the end of the buffer, without any capacity checks! + /// + /// This method DOES NOT check the buffer capacity before writing, and caller MUST have resized the buffer beforehand! + /// Failure to do so may introduce memory correction (buffer overflow!). + /// This should ONLY be used in performance-sensitive code paths that have been audited thoroughly! + /// + public unsafe void UnsafeWriteBytes(byte* data, uint count) + { + if (count != 0) + { + int p = this.Position; + Contract.Requires(this.Buffer != null && p >= 0 && data != null && p + count <= this.Buffer.Length); + + int q = checked((int)(p + count)); + UnsafeHelpers.CopyUnsafe(this.Buffer, p, data, count); + this.Position = q; + } + } + + // Appending is used when the caller want to get a Slice that points to the location where the bytes where written in the internal buffer + + /// Append a byte array to the end of the buffer + public Slice AppendBytes(byte[] data) + { + if (data == null) return Slice.Empty; + return AppendBytes(data, 0, data.Length); + } + + /// Append a chunk of a byte array to the end of the buffer + [Pure] + public Slice AppendBytes(byte[] data, int offset, int count) + { + if (count == 0) return Slice.Empty; + + UnsafeHelpers.EnsureBufferIsValidNotNull(data, offset, count); + int p = this.Position; + var buffer = EnsureBytes(count); + UnsafeHelpers.CopyUnsafe(buffer, p, data, offset, count); + this.Position = checked(p + count); + return new Slice(buffer, p, count); + } + + /// Append a segment of bytes to the end of the buffer + /// Buffer containing the data to append + /// Slice that maps the interned data using the writer's buffer. + /// If you do not need the resulting Slice, you should call instead! + [Pure] + public Slice AppendBytes(Slice data) + { + data.EnsureSliceIsValid(); + + int count = data.Count; + if (count == 0) return Slice.Empty; + + int p = this.Position; + var buffer = EnsureBytes(count); + UnsafeHelpers.CopyUnsafe(buffer, p, data.Array, data.Offset, count); + this.Position = checked(p + count); + return new Slice(buffer, p, count); + } + + /// Write a segment of bytes to the end of the buffer + /// Buffer containing the data to append + /// Slice that maps the interned data using the writer's buffer. + /// If you do not need the resulting Slice, you should call instead! + [Pure] + public Slice AppendBytes(ref Slice data) + { + data.EnsureSliceIsValid(); + + int count = data.Count; + if (count == 0) return Slice.Empty; + + int p = this.Position; + var buffer = EnsureBytes(count); + UnsafeHelpers.CopyUnsafe(buffer, p, data.Array, data.Offset, count); + this.Position = checked(p + count); + return new Slice(buffer, p, count); + } + + /// Append a segment of bytes to the end of the buffer + /// Pointer to the start of the data to append + /// Number of bytes to append + /// Slice that maps to the section of buffer that contains the appended data + /// If you do not need the resulting Slice, you should call instead! + [Pure] + public unsafe Slice AppendBytes(byte* data, uint count) + { + if (count == 0) return Slice.Empty; + if (data == null) throw ThrowHelper.ArgumentNullException(nameof(data)); + + var buffer = EnsureBytes(count); + int p = this.Position; + Contract.Assert(buffer != null && p >= 0 && p + count <= buffer.Length); + + int q = checked((int)(p + count)); + UnsafeHelpers.CopyUnsafe(buffer, p, data, count); + this.Position = q; + return new Slice(buffer, p, q - p); + } + + #endregion + + #region Fixed, Little-Endian + + /// Writes a 16-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed16(short value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(2)[p]) + { + UnsafeHelpers.StoreInt16LE(ptr, value); + } + } + this.Position = p + 2; + } + + /// Writes a 16-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed16(ushort value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(2)[p]) + { + UnsafeHelpers.StoreUInt16LE(ptr, value); + } + } + this.Position = p + 2; + } + + /// Writes a 16-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed24(int value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(3)[p]) + { + UnsafeHelpers.StoreUInt24LE(ptr, (uint) value); + } + } + this.Position = p + 3; + } + + /// Writes a 16-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed24(uint value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(3)[p]) + { + UnsafeHelpers.StoreUInt24LE(ptr, value); + } + } + this.Position = p + 3; + } + + /// Writes a 32-bit signed integer, using little-endian encoding + /// Advances the cursor by 4 bytes + public void WriteFixed32(int value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(4)[p]) + { + UnsafeHelpers.WriteFixed32Unsafe(ptr, (uint) value); + } + } + this.Position = p + 4; + } + + /// Writes a 32-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 4 bytes + public void WriteFixed32(uint value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(4)[p]) + { + UnsafeHelpers.WriteFixed32Unsafe(ptr, value); + } + } + this.Position = p + 4; + } + + /// Writes a 64-bit signed integer, using little-endian encoding + /// Advances the cursor by 8 bytes + public void WriteFixed64(long value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(8)[p]) + { + UnsafeHelpers.WriteFixed64Unsafe(ptr, (ulong) value); + } + } + this.Position = p + 8; + } + + /// Writes a 64-bit unsigned integer, using little-endian encoding + /// Advances the cursor by 8 bytes + public void WriteFixed64(ulong value) + { + int p = this.Position; + unsafe + { + fixed (byte* ptr = &EnsureBytes(8)[p]) + { + UnsafeHelpers.WriteFixed64Unsafe(ptr, value); + } + } + this.Position = p + 8; + } + + #endregion + + #region Fixed, Big-Endian + + /// Writes a 16-bit signed integer, using big-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed16BE(int value) + { + var buffer = EnsureBytes(2); + int p = this.Position; + buffer[p] = (byte)(value >> 8); + buffer[p + 1] = (byte)value; + this.Position = p + 2; + } + + /// Writes a 16-bit unsigned integer, using big-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed16BE(uint value) + { + var buffer = EnsureBytes(2); + int p = this.Position; + buffer[p] = (byte)(value >> 8); + buffer[p + 1] = (byte)value; + this.Position = p + 2; + } + + /// Writes a 24-bit signed integer, using big-endian encoding + /// Advances the cursor by 2 bytes + public void WriteFixed24BE(int value) + { + var buffer = EnsureBytes(3); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + UnsafeHelpers.StoreInt24BE(ptr, value); + } + } + this.Position = p + 3; + } + + /// Writes a 24-bit unsigned integer, using big-endian encoding + /// Advances the cursor by 3 bytes + public void WriteFixed24BE(uint value) + { + var buffer = EnsureBytes(3); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + UnsafeHelpers.StoreUInt24BE(ptr, value); + } + } + this.Position = p + 3; + } + + /// Writes a 32-bit signed integer, using big-endian encoding + /// Advances the cursor by 4 bytes + public void WriteFixed32BE(int value) + { + var buffer = EnsureBytes(4); + int p = this.Position; + buffer[p] = (byte)(value >> 24); + buffer[p + 1] = (byte)(value >> 16); + buffer[p + 2] = (byte)(value >> 8); + buffer[p + 3] = (byte)(value); + this.Position = p + 4; + } + + /// Writes a 32-bit unsigned integer, using big-endian encoding + /// Advances the cursor by 4 bytes + public void WriteFixed32BE(uint value) + { + var buffer = EnsureBytes(4); + int p = this.Position; + buffer[p] = (byte)(value >> 24); + buffer[p + 1] = (byte)(value >> 16); + buffer[p + 2] = (byte)(value >> 8); + buffer[p + 3] = (byte)(value); + this.Position = p + 4; + } + + /// Writes a 64-bit signed integer, using big-endian encoding + /// Advances the cursor by 8 bytes + public void WriteFixed64BE(long value) + { + var buffer = EnsureBytes(8); + int p = this.Position; + buffer[p] = (byte)(value >> 56); + buffer[p + 1] = (byte)(value >> 48); + buffer[p + 2] = (byte)(value >> 40); + buffer[p + 3] = (byte)(value >> 32); + buffer[p + 4] = (byte)(value >> 24); + buffer[p + 5] = (byte)(value >> 16); + buffer[p + 6] = (byte)(value >> 8); + buffer[p + 7] = (byte)(value); + this.Position = p + 8; + } + + /// Writes a 64-bit unsigned integer, using big-endian encoding + /// Advances the cursor by 8 bytes + public void WriteFixed64BE(ulong value) + { + var buffer = EnsureBytes(8); + int p = this.Position; + buffer[p] = (byte)(value >> 56); + buffer[p + 1] = (byte)(value >> 48); + buffer[p + 2] = (byte)(value >> 40); + buffer[p + 3] = (byte)(value >> 32); + buffer[p + 4] = (byte)(value >> 24); + buffer[p + 5] = (byte)(value >> 16); + buffer[p + 6] = (byte)(value >> 8); + buffer[p + 7] = (byte)(value); + this.Position = p + 8; + } + + #endregion + + #region Decimals... + + public void WriteSingle(float value) + { + var buffer = EnsureBytes(4); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + *((int*)ptr) = *(int*)(&value); + } + } + this.Position = p + 4; + } + + public void WriteSingle(byte prefix, float value) + { + var buffer = EnsureBytes(5); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + ptr[0] = prefix; + *((int*)(ptr + 1)) = *(int*)(&value); + } + } + this.Position = p + 5; + } + + public void WriteDouble(double value) + { + var buffer = EnsureBytes(8); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + *((long*)ptr) = *(long*)(&value); + } + } + this.Position = p + 8; + } + + public void WriteDouble(byte prefix, double value) + { + var buffer = EnsureBytes(9); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + ptr[0] = prefix; + *((long*)(ptr + 1)) = *(long*)(&value); + } + } + this.Position = p + 9; + } + + #endregion + + #region Variable size + + #region VarInts... + + /// Writes a 7-bit encoded unsigned int (aka 'Varint16') at the end, and advances the cursor + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteVarInt16(ushort value) + { + if (value < (1 << 7)) + { + WriteByte((byte)value); + } + else + { + WriteVarInt16Slow(value); + } + } + + private void WriteVarInt16Slow(ushort value) + { + const uint MASK = 128; + //note: value is known to be >= 128 + if (value < (1 << 14)) + { + WriteBytes( + (byte)(value | MASK), + (byte)(value >> 7) + ); + } + else + { + WriteBytes( + (byte)(value | MASK), + (byte)((value >> 7) | MASK), + (byte)(value >> 14) + ); + } + + } + + /// Writes a 7-bit encoded unsigned int (aka 'Varint32') at the end, and advances the cursor + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteVarInt32(uint value) + { + if (value < (1 << 7)) + { + WriteByte((byte) value); + } + else + { + WriteVarInt32Slow(value); + } + } + + private void WriteVarInt32Slow(uint value) + { + const uint MASK = 128; + //note: value is known to be >= 128 + if (value < (1 << 14)) + { + WriteBytes( + (byte)(value | MASK), + (byte)(value >> 7) + ); + } + else if (value < (1 << 21)) + { + WriteBytes( + (byte)(value | MASK), + (byte)((value >> 7) | MASK), + (byte)(value >> 14) + ); + } + else if (value < (1 << 28)) + { + WriteBytes( + (byte)(value | MASK), + (byte)((value >> 7) | MASK), + (byte)((value >> 14) | MASK), + (byte)(value >> 21) + ); + } + else + { + WriteBytes( + (byte)(value | MASK), + (byte)((value >> 7) | MASK), + (byte)((value >> 14) | MASK), + (byte)((value >> 21) | MASK), + (byte)(value >> 28) + ); + } + } + + /// Writes a 7-bit encoded unsigned long (aka 'Varint64') at the end, and advances the cursor + public void WriteVarInt64(ulong value) + { + //note: if the size if 64-bits, we probably expact values to always be way above 128 so no need to optimize for this case here + + const uint MASK = 128; + // max encoded size is 10 bytes + var buffer = EnsureBytes(UnsafeHelpers.SizeOfVarInt(value)); + int p = this.Position; + while (value >= MASK) + { + buffer[p++] = (byte) ((value & (MASK - 1)) | MASK); + value >>= 7; + } + buffer[p++] = (byte) value; + this.Position = p; + } + + #endregion + + #region VarBytes... + + /// Writes a length-prefixed byte array, and advances the cursor + public void WriteVarBytes(Slice value) + { + //REVIEW: what should we do for Slice.Nil ? + + value.EnsureSliceIsValid(); + int n = value.Count; + if (n >= 128) + { + WriteVarBytesSlow(value); + return; + } + + var buffer = EnsureBytes(n + 1); + int p = this.Position; + // write the count (single byte) + buffer[p] = (byte)n; + // write the bytes + if (n > 0) UnsafeHelpers.CopyUnsafe(buffer, p + 1, value.Array, value.Offset, n); + this.Position = checked(p + n + 1); + } + +#if ENABLE_SPAN + /// Writes a length-prefixed byte array, and advances the cursor + public void WriteVarBytes(ReadOnlySpan value) + { + int n = value.Length; + if (n >= 128) + { + WriteVarBytesSlow(value); + return; + } + + var buffer = EnsureBytes(n + 1); + int p = this.Position; + // write the count (single byte) + buffer[p] = (byte)n; + // write the bytes + if (n > 0) + { + UnsafeHelpers.CopyUnsafe(buffer, p + 1, value); + } + this.Position = checked(p + n + 1); + } +#endif + +#if ENABLE_SPAN + private void WriteVarBytesSlow(ReadOnlySpan value) + { + int n = value.Length; + EnsureBytes(checked(n + 5)); + // write the count + WriteVarInt32((uint) n); + // write the bytes + int p = this.Position; + UnsafeHelpers.CopyUnsafe(this.Buffer, p, value); + this.Position = checked(p + n); + } +#else + private void WriteVarBytesSlow(Slice value) + { + int n = value.Count; + EnsureBytes(checked(n + 5)); + // write the count + WriteVarInt32((uint) n); + // write the bytes + int p = this.Position; + UnsafeHelpers.CopyUnsafe(this.Buffer, p, value.Array, value.Offset, n); + this.Position = checked(p + n); + } +#endif + + /// Writes a length-prefixed byte array, and advances the cursor + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteVarBytes([NotNull] byte[] bytes) + { + Contract.Requires(bytes != null); + WriteVarBytes(bytes.AsSlice()); + } + + /// Writes a length-prefixed byte array, and advances the cursor + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteVarBytes([NotNull] byte[] bytes, int offset, int count) + { + Contract.Requires(count == 0 || bytes != null); + WriteVarBytes(bytes.AsSlice(offset, count)); + } + + public unsafe void WriteVarBytes(byte* data, uint count) + { + if (count >= 128) + { + WriteVarBytesSlow(data, count); + return; + } + + var buffer = EnsureBytes(count + 1); + int p = this.Position; + // write the count (single byte) + buffer[p] = (byte) count; + // write the bytes + if (count > 0) + { + Contract.Assert(data != null); + UnsafeHelpers.CopyUnsafe(buffer, p + 1, data, count); + } + this.Position = checked(p + (int) count + 1); + } + + private unsafe void WriteVarBytesSlow(byte* data, uint n) + { + Contract.Assert(data != null); + + // 32-bit varint may take up to 5 bytes + EnsureBytes(n + 5); + + // write the count + WriteVarInt32(n); + // write the bytes + int p = this.Position; + UnsafeHelpers.CopyUnsafe(this.Buffer, p, data, n); + this.Position = checked((int)(p + n)); + } + + #endregion + + #region VarString... + + // all VarStrings are encoded as a VarInt that contains the number of following encoded bytes + // => caller MUST KNOWN the encoding! (usually UTF-8) + // => the string's length is NOT stored! + + /// Write a variabe-sized string, using the specified encoding + /// + /// + public void WriteVarString(string value, Encoding encoding = null) + { + if (encoding == null) + { + WriteVarStringUtf8(value); + return; + } + int byteCount = encoding.GetByteCount(value); + if (byteCount == 0) + { + WriteByte(0); + return; + } + WriteVarInt32((uint) byteCount); + int p = this.Position; + int n = encoding.GetBytes(s: value, charIndex: 0, charCount: value.Length, bytes: this.Buffer, byteIndex: p); + this.Position = checked(p + n); + } + + /// Write a variable-sized string, encoded using UTF-8 + /// String to append + /// The null and empty string will be stored the same way. Caller must use a different technique if they must be stored differently. + public void WriteVarStringUtf8(string value) + { + // Format: + // - VarInt Number of following bytes + // - Byte[] UTF-8 encoded bytes + // Examples: + // - "" => { 0x00 } + // - "ABC" => { 0x03 'A' 'B' 'C' } + // - "Héllo" => { 0x06 'h' 0xC3 0xA9 'l' 'l' 'o' } + + // We need to know the encoded size beforehand, because we need to write the size first! + int byteCount = Encoding.UTF8.GetByteCount(value); + if (byteCount == 0) + { // nul or empty string + WriteByte(0); + } + else if (byteCount == value.Length) + { // ASCII! + WriteVarAsciiInternal(value); + } + else + { // contains non-ASCII characters, we will need to encode + WriteVarStringUtf8Internal(value, byteCount); + } + } + + private void WriteVarStringUtf8Internal(string value, int byteCount) + { + Contract.Assert(value != null && byteCount > 0 && byteCount >= value.Length); + EnsureBytes(byteCount + UnsafeHelpers.SizeOfVarBytes(byteCount)); + WriteVarInt32((uint)byteCount); + int p = this.Position; + int n = Encoding.UTF8.GetBytes(s: value, charIndex: 0, charCount: value.Length, bytes: this.Buffer, byteIndex: p); + this.Position = checked(p + n); + } + + /// Write a variable-sized string, which is known to only contain ASCII characters (0..127) + /// This is faster than when the caller KNOWS that the string is ASCII only. This should only be used with keywords and constants, NOT with user input! + /// If the string contains characters above 127 + public void WriteVarStringAscii(string value) + { + if (string.IsNullOrEmpty(value)) + { + WriteByte(0); + } + else + { + WriteVarAsciiInternal(value); + } + } + + /// Write a variable string that is known to only contain ASCII characters + private unsafe void WriteVarAsciiInternal(string value) + { + // Caller must ensure that string is ASCII only! (otherwise it will be corrupted) + Contract.Requires(!string.IsNullOrEmpty(value)); + + int len = value.Length; + var buffer = EnsureBytes(len + UnsafeHelpers.SizeOfVarBytes(len)); + int p = this.Position; + + fixed (byte* bytes = &buffer[p]) + fixed (char* chars = value) + { + var outp = UnsafeHelpers.WriteVarInt32Unsafe(bytes, (uint) value.Length); + p += (int) (outp - bytes); + int mask = 0; + for (int i = 0; i < len; i++) + { + var c = chars[i]; + mask |= c; + outp[i] = (byte)c; + } + if (mask >= 128) throw ThrowHelper.ArgumentException(nameof(value), "The specified string must only contain ASCII characters."); + } + this.Position = checked(p + value.Length); + } + + #endregion + + #endregion + + #region UUIDs... + + /// Write a 128-bit UUID, and advances the cursor + public void WriteUuid128(Uuid128 value) + { + var buffer = EnsureBytes(16); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + value.WriteToUnsafe(ptr); + } + } + this.Position = p + 16; + } + + /// Write a 128-bit UUID, and advances the cursor + public void UnsafeWriteUuid128(Uuid128 value) + { + Contract.Requires(this.Buffer != null && this.Position + 15 < this.Buffer.Length); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &this.Buffer[p]) + { + value.WriteToUnsafe(ptr); + } + } + this.Position = p + 16; + } + + /// Write a 128-bit UUID, and advances the cursor + public void WriteUuid64(Uuid64 value) + { + var buffer = EnsureBytes(8); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &buffer[p]) + { + value.WriteToUnsafe(ptr); + } + } + this.Position = p + 8; + } + + /// Write a 128-bit UUID, and advances the cursor + public void UnsafeWriteUuid64(Uuid64 value) + { + Contract.Requires(this.Buffer != null && this.Position + 7 < this.Buffer.Length); + int p = this.Position; + unsafe + { + fixed (byte* ptr = &this.Buffer[p]) + { + value.WriteToUnsafe(ptr); + } + } + this.Position = p + 8; + } + + #endregion + + #region Fixed-Size Text + + /// Write a string using UTF-8 + /// Text to write + /// Number of bytes written + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public int WriteString(string value) + { + return WriteStringUtf8(value); + } + +#if ENABLE_SPAN + /// Write a string using UTF-8 + /// Text to write + /// Number of bytes written + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public int WriteString(ReadOnlySpan value) + { + return WriteStringUtf8(value); + } +#endif + + /// Write a string using the specified encoding + /// Text to write + /// Encoding used to convert the text to bytes + /// Number of bytes written + public int WriteString(string value, Encoding encoding) + { + if (string.IsNullOrEmpty(value)) return 0; + + // In order to estimate the required capacity, we try to guess for very small strings, but compute the actual value for larger strings, + // so that we don't waste to much memory (up to 6x the string length in the worst case scenario) + var buffer = EnsureBytes(value.Length > 128 ? encoding.GetByteCount(value) : encoding.GetMaxByteCount(value.Length)); + + int p = this.Position; + int n = encoding.GetBytes(value, 0, value.Length, buffer, p); + this.Position = p + n; + return n; + } + + /// Write a string using UTF-8 + /// Text to write + /// Number of bytes written + public int WriteStringUtf8(string value) + { + if (string.IsNullOrEmpty(value)) return 0; + + // In order to estimate the required capacity, we try to guess for very small strings, but compute the actual value for larger strings, + // so that we don't waste to much memory (up to 6x the string length in the worst case scenario) + var buffer = EnsureBytes(value.Length > 128 + ? Encoding.UTF8.GetByteCount(value) + : Encoding.UTF8.GetMaxByteCount(value.Length)); + + int p = this.Position; + int n = Encoding.UTF8.GetBytes(s: value, charIndex: 0, charCount: value.Length, bytes: buffer, byteIndex: p); + this.Position = checked(p + n); + return n; + } + +#if ENABLE_SPAN + /// Write a string using UTF-8 + /// Number of bytes written + public int WriteStringUtf8(char[] chars, int offset, int count) + { + return WriteStringUtf8(new ReadOnlySpan(chars, offset, count)); + } + + /// Write a string using UTF-8 + /// Number of bytes written + public int WriteStringUtf8(ReadOnlySpan chars) + { + int count = chars.Length; + if (count == 0) return 0; + + unsafe + { + fixed (char* inp = &MemoryMarshal.GetReference(chars)) + { + // pour estimer la capacité, on fait une estimation a la louche pour des petites strings, mais on va calculer la bonne valeur pour des string plus grandes, + // afin d'éviter de gaspiller trop de mémoire (potentiellement jusqu'a 6 fois la taille) + var buffer = EnsureBytes(count > 128 + ? Encoding.UTF8.GetByteCount(inp, count) + : Encoding.UTF8.GetMaxByteCount(count)); + + int p = this.Position; + fixed (byte* outp = &buffer[p]) + { + int n = Encoding.UTF8.GetBytes(chars: inp, charCount: count, bytes: outp, byteCount: buffer.Length - p); + this.Position = checked(p + n); + return n; + } + } + } + } +#endif + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailInvalidUtf8CodePoint() + { + return new DecoderFallbackException("Failed to encode invalid Unicode CodePoint into UTF-8"); + } + + /// Write a string that only contains ASCII + /// String with characters only in the 0..127 range + /// Faster than when writing Magic Strings or ascii keywords + /// Number of bytes written + public int WriteStringAscii(string value) + { + Contract.Requires(value != null); + + var buffer = EnsureBytes(value.Length); + int p = this.Position; + foreach (var c in value) + { + buffer[p++] = (byte) c; + } + this.Position = p; + return value.Length; + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBase10(int value) + { + if ((uint) value <= 9) + { + WriteByte('0' + value); + } + else + { + WriteBase10Slow(value); + } + } + + public void WriteBase10(long value) + { + if ((ulong) value <= 9) + { + WriteByte('0' + (int) value); + } + else if (value <= int.MaxValue) + { + WriteBase10Slow((int) value); + } + else + { + WriteBase10Slower(value); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteBase10(uint value) + { + if (value <= 9) + { + WriteByte('0' + (int) value); + } + else + { + WriteBase10Slow(value); + } + } + + public void WriteBase10(ulong value) + { + if (value <= 9) + { + WriteByte('0' + (int) value); + } + else if (value <= uint.MaxValue) + { + WriteBase10Slow((uint) value); + } + else + { + WriteBase10Slower(value); + } + } + + private void WriteBase10Slow(int value) + { + if (value < 0) + { // negative numbers + if (value == int.MinValue) + { // cannot do Abs(MinValue), so special case for this one + WriteStringAscii("-2147483648"); + return; + } + WriteByte('-'); + value = -value; + } + + if (value < 10) + { + WriteByte((byte) ('0' + value)); + } + else if (value < 100) + { + WriteBytes( + (byte) ('0' + (value / 10)), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 1000) + { + WriteBytes( + (byte) ('0' + (value / 100)), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 10 * 1000) + { + WriteBytes( + (byte) ('0' + (value / 1000)), + (byte) ('0' + (value / 100) % 10), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 100 * 1000) + { + WriteBytes( + (byte) ('0' + (value / 10000)), + (byte) ('0' + (value / 1000) % 10), + (byte) ('0' + (value / 100) % 10), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else + { + WriteBase10Slower(value); + } + } + + private void WriteBase10Slower(long value) + { + //TODO: OPTIMIZE: sans allocations? + WriteStringAscii(value.ToString(CultureInfo.InvariantCulture)); + } + + private void WriteBase10Slow(uint value) + { + if (value < 10) + { + WriteByte((byte) ('0' + value)); + } + else if (value < 100) + { + WriteBytes( + (byte) ('0' + (value / 10)), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 1000) + { + WriteBytes( + (byte) ('0' + (value / 100)), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 10 * 1000) + { + WriteBytes( + (byte) ('0' + (value / 1000)), + (byte) ('0' + (value / 100) % 10), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else if (value < 100 * 1000) + { + WriteBytes( + (byte) ('0' + (value / 10000)), + (byte) ('0' + (value / 1000) % 10), + (byte) ('0' + (value / 100) % 10), + (byte) ('0' + (value / 10) % 10), + (byte) ('0' + (value % 10)) + ); + } + else + { + WriteBase10Slower(value); + } + } + + private void WriteBase10Slower(ulong value) + { + //TODO: OPTIMIZE: sans allocations? + WriteStringAscii(value.ToString(CultureInfo.InvariantCulture)); + } + + #endregion + + #region Patching + + #region 8-bits... + + /// Overwrite a section of the buffer that was already written, with the specified data + /// Offset from the start of the buffer where to start replacing + /// Data that will overwrite the buffer at the specified + /// You must ensure that replaced section does not overlap with the current position! + public void PatchBytes(int index, Slice data) + { + if (index + data.Count > this.Position) throw ThrowHelper.IndexOutOfRangeException(); + data.CopyTo(this.Buffer, index); + } + + /// Overwrite a section of the buffer that was already written, with the specified data + /// You must ensure that replaced section does not overlap with the current position! + public void PatchBytes(int index, byte[] buffer, int offset, int count) + { + if (index + count > this.Position) throw ThrowHelper.IndexOutOfRangeException(); + System.Buffer.BlockCopy(buffer, offset, this.Buffer, index, count); + } + + /// Overwrite a byte of the buffer that was already written + /// You must ensure that replaced byte is before the current position! + public void PatchByte(int index, byte value) + { + if ((uint) index >= this.Position) throw ThrowHelper.IndexOutOfRangeException(); + this.Buffer[index] = value; + } + + /// Overwrite a byte of the buffer that was already written + /// You must ensure that replaced byte is before the current position! + public void PatchByte(int index, int value) + { + //note: convenience method, because C# compiler likes to produce 'int' when combining bits together + if ((uint) index >= this.Position) throw ThrowHelper.IndexOutOfRangeException(); + this.Buffer[index] = (byte) value; + } + + #endregion + + #region 16-bits... + + /// Overwrite a word of the buffer that was already written + /// You must ensure that replaced word is before the current position! + public void PatchInt16(int index, short value) + { + if (index + 2 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed16Unsafe(ptr, (ushort) value); + } + } + } + + /// Overwrite a word of the buffer that was already written + /// You must ensure that replaced word is before the current position! + public void PatchUInt16(int index, ushort value) + { + if (index + 2 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed16Unsafe(ptr, value); + } + } + } + + /// Overwrite a word of the buffer that was already written + /// You must ensure that replaced word is before the current position! + public void PatchInt16BE(int index, short value) + { + if (index + 2 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed16BEUnsafe(ptr, (ushort) value); + } + } + } + + /// Overwrite a word of the buffer that was already written + /// You must ensure that replaced word is before the current position! + public void PatchUInt16BE(int index, ushort value) + { + if (index + 2 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed16BEUnsafe(ptr, value); + } + } + } + + #endregion + + #region 32-bits... + + /// Overwrite a dword of the buffer that was already written + /// You must ensure that replaced dword is before the current position! + public void PatchInt32(int index, int value) + { + if (index + 4 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed32Unsafe(ptr, (uint) value); + } + } + } + + /// Overwrite a dword of the buffer that was already written + /// You must ensure that replaced dword is before the current position! + public void PatchUInt32(int index, uint value) + { + if (index + 4 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed32Unsafe(ptr, value); + } + } + } + + /// Overwrite a dword of the buffer that was already written + /// You must ensure that replaced dword is before the current position! + public void PatchInt32BE(int index, int value) + { + if (index + 4 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed32BEUnsafe(ptr, (uint) value); + } + } + } + + /// Overwrite a dword of the buffer that was already written + /// You must ensure that replaced dword is before the current position! + public void PatchUInt32BE(int index, uint value) + { + if (index + 4 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed32BEUnsafe(ptr, value); + } + } + } + + #endregion + + #region 64-bits... + + /// Overwrite a qword of the buffer that was already written + /// You must ensure that replaced qword is before the current position! + public void PatchInt64(int index, long value) + { + if (index + 8 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed64Unsafe(ptr, (ulong) value); + } + } + } + + /// Overwrite a qword of the buffer that was already written + /// You must ensure that replaced qword is before the current position! + public void PatchUInt64(int index, ulong value) + { + if (index + 8 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed64Unsafe(ptr, value); + } + } + } + + /// Overwrite a qword of the buffer that was already written + /// You must ensure that replaced qword is before the current position! + public void PatchInt64BE(int index, long value) + { + if (index + 8 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed64BEUnsafe(ptr, (ulong) value); + } + } + } + + /// Overwrite a qword of the buffer that was already written + /// You must ensure that replaced qword is before the current position! + public void PatchUInt64BE(int index, ulong value) + { + if (index + 8 > this.Position) ThrowHelper.ThrowIndexOutOfRangeException(); + unsafe + { + fixed (byte* ptr = &this.Buffer[index]) + { + UnsafeHelpers.WriteFixed64BEUnsafe(ptr, value); + } + } + } + + #endregion + + #endregion + + /// Return the remaining capacity in the current underlying buffer + public int RemainingCapacity + { + get + { + var buffer = this.Buffer; + if (buffer == null || this.Position >= buffer.Length) return 0; + return buffer.Length - this.Position; + } + } + + /// Ensures that we can fit the specified amount of data at the end of the buffer + /// Number of bytes that will be written + /// If the buffer is too small, it will be resized, and all previously written data will be copied + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public byte[] EnsureBytes(int count) + { + //REVIEW: en C#7 on pourrait retourner le tuple (buffer, pos) ! + + Contract.Requires(count >= 0); + var buffer = this.Buffer; + if (buffer == null || this.Position + count > buffer.Length) + { + buffer = GrowBuffer(ref this.Buffer, this.Position + count); + Contract.Ensures(buffer != null && buffer.Length >= this.Position + count); + } + return buffer; + } + +#if ENABLE_ARRAY_POOL + + /// Ensures that we can fit the specified amount of data at the end of the buffer + /// Number of bytes that will be written + /// + /// If the buffer is too small, it will be resized, and all previously written data will be copied + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public byte[] EnsureBytes(int count, ArrayPool pool) + { + //REVIEW: en C#7 on pourrait retourner le tuple (buffer, pos) ! + + Contract.Requires(count >= 0); + var buffer = this.Buffer; + if (buffer == null || this.Position + count > buffer.Length) + { + buffer = GrowBuffer(ref this.Buffer, this.Position + count, pool); + Contract.Ensures(buffer != null && buffer.Length >= this.Position + count); + } + return buffer; + } + +#endif + + /// Ensures that we can fit the specified amount of data at the end of the buffer + /// Number of bytes that will be written + /// If the buffer is too small, it will be resized, and all previously written data will be copied + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public byte[] EnsureBytes(uint count) + { + return EnsureBytes(checked((int) count)); + } + + /// Ensures that we can fit data at a specifc offset in the buffer + /// Offset into the buffer (from the start) + /// Number of bytes that will be written at this offset + /// If the buffer is too small, it will be resized, and all previously written data will be copied + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void EnsureOffsetAndSize(int offset, int count) + { + Contract.Requires(offset >= 0 && count >= 0); + if (this.Buffer == null || offset + count > this.Buffer.Length) + { + GrowBuffer(ref this.Buffer, offset + count); + } + } + + /// Resize a buffer by doubling its capacity + /// Reference to the variable holding the buffer to create/resize. If null, a new buffer will be allocated. If not, the content of the buffer will be copied into the new buffer. + /// Mininum guaranteed buffer size after resizing. + /// The buffer will be resized to the maximum between the previous size multiplied by 2, and . The capacity will always be rounded to a multiple of 16 to reduce memory fragmentation + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static byte[] GrowBuffer( + ref byte[] buffer, + int minimumCapacity = 0 +#if ENABLE_ARRAY_POOL + , ArrayPool pool = null +#endif + ) + { + Contract.Requires(minimumCapacity >= 0); + + // double the size of the buffer, or use the minimum required + long newSize = Math.Max(buffer == null ? 0 : (((long) buffer.Length) << 1), minimumCapacity); + + // .NET (as of 4.5) cannot allocate an array with more than 2^31 - 1 items... + if (newSize > 0x7fffffffL) throw FailCannotGrowBuffer(); + + // round up to 16 bytes, to reduce fragmentation + int size = BitHelpers.AlignPowerOfTwo((int) newSize, 16); + +#if ENABLE_ARRAY_POOL + if (pool == null) + { + Array.Resize(ref buffer, size); + } + else + { // use the pool to resize the buffer + pool.Resize(ref buffer, size); + } +#else + Array.Resize(ref buffer, size); +#endif + return buffer; + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailCannotGrowBuffer() + { +#if DEBUG + // If you breakpoint here, that means that you probably have an uncheked maximum buffer size, or a runaway while(..) { append(..) } code in your layer code ! + // => you should ALWAYS ensure a reasonable maximum size of your allocations ! + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + // note: some methods in the BCL do throw an OutOfMemoryException when attempting to allocated more than 2^31 + return new OutOfMemoryException("Buffer cannot be resized, because it would exceed the maximum allowed size"); + } + + [UsedImplicitly(ImplicitUseTargetFlags.WithMembers)] + private sealed class DebugView + { + + public DebugView(SliceWriter writer) + { + this.Data = new Slice(writer.Buffer, 0, writer.Position); + this.Position = writer.Position; + this.Capacity = writer.Buffer.Length; + } + + public Slice Data { get; } + + public int Position { get; } + + public int Capacity { get; } + + } + + } + +} diff --git a/FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs b/FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs new file mode 100644 index 000000000..1ae85413a --- /dev/null +++ b/FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs @@ -0,0 +1,3589 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +// If defined, means that the host process will ALWAYS run in a Little Endian context, and we can use some optimizations to speed up encoding and decoding values to and from memory buffers. +// If undefined, then fallback to architecture-agnostic way of handling bit and little endian values +// note: when enabled, the code assumes that the CPU supports unaligned stores and loads +#define EXPECT_LITTLE_ENDIAN_HOST + +// Enable the use of Span and ReadOnlySpan +//#define ENABLE_SPAN + +//note: we would like to use Vector from System.Numerics.Vectors (which is converted to SIMD by the JIT), but this is not really practical just yet: +// - v4.0 of the assembly does NOT have Vector, which was removed between beta, and only came back in 4.1-beta +// - the ctor Vector(byte* ptr, int offset) is currently private, which means that we cannot use it with unsafe pointers yet +// - there does not seem to be any SIMD way to implement memcmp with the current Vector API, unless doing some trickery with substracting and looking for 0s + +namespace Doxense.Memory +{ + using System; + using System.Diagnostics; + using System.IO; + using System.Runtime.CompilerServices; + using System.Runtime.ConstrainedExecution; + using System.Runtime.InteropServices; + using System.Security; + using JetBrains.Annotations; + using Doxense.Diagnostics.Contracts; + + /// Helper methods for dealing with unmanaged memory. HANDLE WITH CARE! + /// Use of this class is unsafe. YOU HAVE BEEN WARNED! + [DebuggerNonUserCode] // <-- remove this when debugging the class itself! + public static unsafe class UnsafeHelpers + { + +#if EXPECT_LITTLE_ENDIAN_HOST + private const bool IsLittleEndian = true; +#else + //note: should be optimized as a const by the JIT! + private static readonly bool IsLittleEndian = BitConverter.IsLittleEndian; +#endif + + /// Validates that and represent a valid location in + /// If is 0, then is allowed to be null + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValid(byte[] array, int offset, int count) + { + // note: same test has for a Slice + if (count != 0 && (array == null || (uint) offset > (uint) array.Length || (uint) count > (uint) (array.Length - offset))) + { + throw Errors.MalformedBuffer(array, offset, count); + } + } + + /// Validates that and represent a valid location in + /// If is 0, then is allowed to be null + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValid(byte[] array, uint offset, uint count) + { + // note: same test has for a Slice + if (count != 0 && (array == null || (long) count > (long) array.Length - offset)) + { + throw Errors.MalformedBuffer(array, offset, count); + } + } + + /// Validates that and represent a valid location in + /// is not allowed to be null, even if is 0. + [ContractAnnotation("array:null => halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValidNotNull(byte[] array, int offset, int count) + { + // note: same test has for a Slice + if (array == null || (uint) offset > (uint) array.Length || (uint) count > (uint) (array.Length - offset)) + { + throw Errors.MalformedBuffer(array, offset, count); + } + } + + /// Validates that and represent a valid location in + /// is not allowed to be null, even if is 0. + [ContractAnnotation("array:null => halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValidNotNull(byte[] array, uint offset, uint count) + { + // note: same test has for a Slice + if (array == null || (long) count > (long) array.Length - offset) + { + throw Errors.MalformedBuffer(array, offset, count); + } + } + + /// Validates that an unmanged buffer represents a valid memory location + /// If is 0, then is allowed to be null + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValid(byte* bytes, long count) + { + if (count != 0 & (bytes == null || count < 0)) + { + throw Errors.MalformedBuffer(bytes, count); + } + } + + /// Validates that an unmanaged buffer represents a valid memory location + /// is not allowed to be null, even if is 0. + [ContractAnnotation("bytes:null => halt")] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void EnsureBufferIsValidNotNull(byte* bytes, long count) + { + if (bytes == null || count < 0) + { + throw Errors.MalformedBuffer(bytes, count); + } + } + + /// Compare two byte segments for equality + /// Left buffer + /// Start offset in left buffer + /// Right buffer + /// Start offset in right buffer + /// Number of bytes to compare + /// true if all bytes are the same in both segments + [Pure] + public static bool SameBytes(byte[] left, int leftOffset, byte[] right, int rightOffset, int count) + { + EnsureBufferIsValid(left, leftOffset, count); + EnsureBufferIsValid(right, rightOffset, count); + + if (left == null || right == null) return left == right; + return SameBytesUnsafe(left, leftOffset, right, rightOffset, count); + } + +#if ENABLE_SPAN + /// Compare two spans for equality + /// Left buffer + /// Right buffer + /// true if all bytes are the same in both segments + public static bool SameBytes(ReadOnlySpan left, ReadOnlySpan right) + { + if (left.Length != right.Length) return false; + //REVIEW: is there a more direct wait to compare two spans ?? (did not find anything in ReadOnlySpan, MemoryExtensions nor MemoryMarshal ... ?) + fixed (byte* pLeft = &MemoryMarshal.GetReference(left)) + fixed (byte* pRight = &MemoryMarshal.GetReference(right)) + { + //TODO: version of comapre that is optimized for equality checks! + return 0 == CompareUnsafe(pLeft, pRight, (uint) left.Length); + } + } +#endif + + /// Compare two byte segments for equality, without validating the arguments + /// Left buffer + /// Start offset in left buffer + /// Right buffer + /// Start offset in right buffer + /// Number of bytes to compare + /// true if all bytes are the same in both segments + [Pure] + 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 (count == 0 || (object.ReferenceEquals(left, right) && leftOffset == rightOffset)) + { // empty, or same segment of the same buffer + return true; + } + + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) + { + //TODO: version of comapre that is optimized for equality checks! + return 0 == CompareUnsafe(pLeft, pRight, checked((uint)count)); + } + } + + /// Compare two byte buffers lexicographically + /// Left buffer + /// Right buffer + /// Returns zero if both buffers 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" + /// + [Pure] + public static int Compare([NotNull] byte[] left, [NotNull] byte[] right) + { + Contract.NotNull(left, nameof(left)); + Contract.NotNull(right, nameof(right)); + return CompareUnsafe(left, 0, left.Length, right, 0, right.Length); + } + + /// 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" + /// + [Pure] + public static int Compare([NotNull] byte[] left, int leftOffset, int leftCount, [NotNull] byte[] right, int rightOffset, int rightCount) + { + EnsureBufferIsValidNotNull(left, leftOffset, leftCount); + EnsureBufferIsValidNotNull(right, rightOffset, rightCount); + + return CompareUnsafe(left, leftOffset, leftCount, right, rightOffset, rightCount); + } + + /// Compare two byte segments lexicographically, without validating the arguments + /// 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" + /// + [Pure] + public static int CompareUnsafe([NotNull] byte[] left, int leftOffset, int leftCount, [NotNull] byte[] right, int rightOffset, int rightCount) + { + Contract.Requires(left != null && right != null && leftOffset >= 0 && leftCount >= 0 && rightOffset >= 0 && rightCount >= 0); + + if (object.ReferenceEquals(left, right) && leftCount == rightCount && leftOffset == rightOffset) + { // same segment in the same buffer + return 0; + } + + fixed (byte* pLeft = &left[leftOffset]) + fixed (byte* pRight = &right[rightOffset]) + { + return CompareUnsafe(pLeft, (uint) leftCount, pRight, (uint) rightCount); + } + } + + /// Ensure that the specified temporary buffer is large enough + /// Pointer to a temporary scratch buffer (previous data will not be maintained) + /// Minimum expected capacity + /// Same buffer if it was large enough, or a new allocated buffer with length greater than or equal to + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte[] EnsureCapacity(ref byte[] buffer, int minCapacity) + { + if (buffer == null || buffer.Length < minCapacity) + { + buffer = AllocateAligned(minCapacity); + } + return buffer; + } + + /// Ensure that the specified temporary buffer is large enough + /// Pointer to a temporary scratch buffer (previous data will not be maintained) + /// Minimum expected capacity + /// Same buffer if it was large enough, or a new allocated buffer with length greater than or equal to + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte[] EnsureCapacity(ref byte[] buffer, uint minCapacity) + { + if (minCapacity > int.MaxValue) throw FailBufferTooLarge(minCapacity); + if (buffer == null || buffer.Length < (int) minCapacity) + { + buffer = AllocateAligned((int) minCapacity); + } + return buffer; + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte[] AllocateAligned(int minCapacity) + { + if (minCapacity < 0) throw FailBufferTooLarge(minCapacity); //note: probably an integer overlofw (unsigned -> signed) + return new byte[BitHelpers.AlignPowerOfTwo(minCapacity, 8)]; + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailBufferTooLarge(long minCapacity) + { + return new ArgumentOutOfRangeException(nameof(minCapacity), minCapacity, "Cannot allocate buffer larger than 2GB."); + } + + /// 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! + [DebuggerStepThrough] + public static void Copy(byte[] dst, int dstOffset, byte[] src, int srcOffset, int count) + { + if (count > 0) + { + EnsureBufferIsValidNotNull(dst, dstOffset, count); + EnsureBufferIsValidNotNull(src, srcOffset, count); + + fixed (byte* pDst = &dst[dstOffset]) // throw if dst == null or dstOffset outside of the array + fixed (byte* pSrc = &src[srcOffset]) // throw if src == null or srcOffset outside of the array + { + Buffer.MemoryCopy(pSrc, pDst, dst.Length - dstOffset, count); + } + } + } + + /// 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! + [DebuggerStepThrough] + public static void Copy(byte[] dst, uint dstOffset, byte[] src, uint srcOffset, uint count) + { + if (count > 0) + { + EnsureBufferIsValidNotNull(dst, dstOffset, count); + EnsureBufferIsValidNotNull(src, srcOffset, count); + + fixed (byte* pDst = &dst[dstOffset]) // throw if dst == null or dstOffset outside of the array + fixed (byte* pSrc = &src[srcOffset]) // throw if src == null or srcOffset outside of the array + { + Buffer.MemoryCopy(pSrc, pDst, dst.Length - dstOffset, count); + } + } + } + +#if ENABLE_SPAN + public static void Copy(Span destination, byte[] src, int srcOffset, int count) + { + if (count > 0) + { + new ReadOnlySpan(src, srcOffset, count).CopyTo(destination); + } + } + + public static void Copy(Span destination, Slice source) + { + if (source.Count > 0) + { + new ReadOnlySpan(source.Array, source.Offset, source.Count).CopyTo(destination); + } + } + + public static void Copy(byte[] dst, int dstOffset, ReadOnlySpan source) + { + if (source.Length > 0) + { + source.CopyTo(new Span(dst).Slice(dstOffset)); + } + } + + public static void Copy(Slice destination, ReadOnlySpan source) + { + if (source.Length > 0) + { + source.CopyTo(new Span(destination.Array, destination.Offset, destination.Count)); + } + } +#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 + /// 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! + [DebuggerStepThrough] + public static void CopyUnsafe([NotNull] byte[] dst, int dstOffset, [NotNull] byte[] src, int srcOffset, int count) + { + //Contract.Requires(count >= 0); + if (count > 0) + { + //Contract.Requires(dst != null && dstOffset >= 0 && src != null && srcOffset >= 0); + + fixed (byte* pDst = &dst[dstOffset]) + fixed (byte* pSrc = &src[srcOffset]) + { + Buffer.MemoryCopy(pSrc, pDst, count, count); + } + } + } + +#if ENABLE_SPAN + /// Copy the content of a native byte segment into a managed segment, without validating the arguments. + /// Destination buffer + /// Offset in destination buffer + /// Point to the 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! + [DebuggerStepThrough] + public static void CopyUnsafe([NotNull] byte[] dst, int dstOffset, ReadOnlySpan src) + { + //Contract.Requires(dst != null && dstOffset >= 0 && src.Length >= 0); + + fixed (byte* pDst = &dst[dstOffset]) + fixed (byte* pSrc = &MemoryMarshal.GetReference(src)) + { + Buffer.MemoryCopy(pSrc, pDst, src.Length, src.Length); + } + } +#endif + + /// Copy the content of a native byte segment into a managed segment, without validating the arguments. + /// Destination buffer + /// Offset in destination buffer + /// Point to the 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! + [DebuggerStepThrough] + public static void CopyUnsafe([NotNull] byte[] dst, int dstOffset, byte* src, int count) + { + //Contract.Requires(dst != null && src != null && dstOffset >= 0 && count >= 0); + + fixed (byte* pDst = &dst[dstOffset]) + { + Buffer.MemoryCopy(src, pDst, count, count); + } + } + + /// Copy the content of a native byte segment into a managed segment, without validating the arguments. + /// Destination buffer + /// Offset in destination buffer + /// Point to the 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! + [DebuggerStepThrough] + public static void CopyUnsafe([NotNull] byte[] dst, int dstOffset, byte* src, uint count) + { + //Contact.Requires(dst != null && src != null && dstOffset >= 0); + + fixed (byte* pDst = &dst[dstOffset]) + { + Buffer.MemoryCopy(src, pDst, count, count); + } + } + + /// Copy a managed slice to the specified memory location + /// Where to copy the bytes + /// Reference to the first byte to copy + /// Number of bytes to copy + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyUnsafe(byte* dest, ref byte src, int count) + { + if (count > 0) + { + Contract.Requires(dest != null); + fixed (byte* ptr = &src) + { + Buffer.MemoryCopy(ptr, dest, count, count); + } + } + } + + /// Copy a managed slice to the specified memory location + /// Where to copy the bytes + /// Slice of managed memory that will be copied to the destination + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyUnsafe(byte* dest, Slice src) + { + int count = src.Count; + if (count > 0) + { + Contract.Requires(dest != null && src.Array != null && src.Offset >= 0 && src.Count >= 0); + fixed (byte* ptr = &src.DangerousGetPinnableReference()) + { + Buffer.MemoryCopy(ptr, dest, count, count); + } + } + } + + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyUnsafe(Slice dest, byte* src, uint count) + { + if (count > 0) + { + Contract.Requires(dest.Array != null && dest.Offset >= 0 && dest.Count >= 0 && src != null); + fixed (byte* ptr = &dest.DangerousGetPinnableReference()) + { + Buffer.MemoryCopy(src, ptr, dest.Count, 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)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void CopyUnsafe([NotNull] byte* dest, [NotNull] byte* src, uint count) + { + Contract.Requires(dest != null && src != null); + Buffer.MemoryCopy(src, dest, count, count); + } + + /// Compare two buffers in memory, using the lexicographical order, without checking the arguments + /// Pointer to the first buffer + /// Size (in bytes) of the first buffer + /// Pointer to the second buffer + /// Size (in bytes) of the second buffer + /// The returned value will be < 0 if is "before" , 0 if is the same as , and < 0 if is "after" right. + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int CompareUnsafe(byte* left, uint leftCount, byte* right, uint rightCount) + { + Contract.Requires((left != null || leftCount == 0) && (right != null || rightCount == 0)); + + int c = CompareUnsafe(left, right, Math.Min(leftCount, rightCount)); + return c != 0 ? c : (int) (leftCount - rightCount); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int CompareUnsafe(byte* left, byte* right, uint count) + { + // the most frequent case is to compare keys that are natural or GUIDs, + // in which case there is a very high probability that the first byte is different already + // => we check for that case immediately + if (count != 0 && *left != *right) return *left - *right; + //REVIEW: we could special case count==4 or count==8 because they are probably frequent (FreeSpace map uses 4, indexes may use 8, ...) + return CompareUnsafeInternal(left, right, count); + } + + /// Compare two buffers in memory, using the lexicographical order, without checking the arguments + /// Pointer to the first buffer + /// Pointer to the second buffer + /// Size (in bytes) of both buffers + /// The returned value will be < 0 if is "before" , 0 if is the same as , and < 0 if is "after" right. + [Pure, MethodImpl(MethodImplOptions.NoInlining)] + private static int CompareUnsafeInternal(byte* left, byte* right, uint count) + { + Contract.Requires(count == 0 || (left != null && right != null)); + + // We would like to always use memcmp (fastest), but the overhead of PInvoke makes it slower for small keys (<= 256) + // For these, we will use a custom implementation which is a bit slower than memcmp but faster than the overhead of PInvoke. + + if (count == 0) return 0; + + // the minimum size to amortize the cost of P/Invoke seems to be over 256 bytes, On My Machine(tm) + if (count > 256) + { + return _memcmp(left, right, count); + } + + // we will scan the strings by XORing together segments of 8 bytes (then 4, then 2, ...) looking for the first segment that contains at least one difference (ie: at least one bit set after XORing) + // then, if we find a difference, we will "fine tune" the pointers to locate the first byte that is different + // then, we will return the difference between the bytes at this location + + // Sample scenario: + // __ cursor ___ first difference is at byte (cursor + 4) + // v v + // LEFT : .. AA AA AA AA AA AA AA AA .. + // RIGHT: .. AA AA AA AA BB AA AA AA .. + // XOR : ( 00 00 00 00 11 00 00 00 ) + // + // The result of the XOR is 0x11000000 and is not equal to 0, so the first difference is within these 8 bytes + // The first 4 bytes of the result are 0, which means that the difference is at offset 4 (ie: we needed to SHR 8 the result 4 times before having at least one bit set in 0..7 + // + // L XOR R: 00 00 00 00 11 00 00 00 + // offset : +0 +1 +2 +3 +4 +5 +6 +7 + // ^^__ first non-zero byte + + // number of 16-bytes segments to scan + long x; + if (count >= 16) + { + long y; + byte* end = left + (count & ~0xF); + while (left < end) + { + // parallelize the reads + x = *(long*) left ^ *(long*) right; + y = *(long*) (left + 8) ^ *(long*) (right + 8); + if (x != 0) + { + goto fine_tune_8; + } + if (y != 0) + { + x = y; + goto fine_tune_8_with_offset; + } + left += 16; + right += 16; + } + + if ((count & 0xF) == 0) + { // size is multiple of 16 with no differences => equal + return 0; // fast path for Guid keys + } + } + + // use the last 4 bits in the count to parse the tail + + if ((count & 8) != 0) + { // at least 8 bytes remaining + x = *(long*) left ^ *(long*) right; + if (x != 0) goto fine_tune_8; + if ((count & 7) == 0) return 0; // fast path for long keys + left += 8; + right += 8; + } + if ((count & 4) != 0) + { // at least 4 bytes remaining + x = *(int*) left ^ *(int*) right; + if (x != 0) goto fine_tune_4; + if ((count & 3) == 0) return 0; // fast path for int keys + left += 4; + right += 4; + } + if ((count & 2) != 0) + { // at least 2 bytes remaining + x = *(short*) left ^ *(short*) right; + if (x != 0) goto fine_tune_2; + left += 2; + right += 2; + } + if ((count & 1) != 0) + { // at least one byte remaining + return left[0] - right[0]; + } + // both strings are equal + return 0; + + fine_tune_8_with_offset: + // adjust the pointers (we were looking at the upper 8 bytes in a 16-bytes segment + left += 8; + right += 8; + + fine_tune_8: + // the difference is somewhere in the last 8 bytes + if ((uint)x == 0) + { // it is not in the first 4 bytes + x >>= 32; + left += 4; + right += 4; + } + fine_tune_4: + // the difference is somewhere in the last 4 bytes + if ((ushort) x == 0) + { // if is not in the first 2 bytes + // the difference is either at +2 or +3 + return (x & 0xFF0000) == 0 + ? left[3] - right[3] + : left[2] - right[2]; + } + + fine_tune_2: + // the difference is somewhere in the last 2 bytes + return (x & 0xFF) == 0 + ? left[1] - right[1] + : left[0] - right[0]; + } + + [Pure, MethodImpl(MethodImplOptions.NoInlining)] + private static int _memcmp([NotNull] byte* left, byte* right, uint count) + { + return NativeMethods.memcmp(left, right, (UIntPtr) count); + } + + /// Fill the content of a managed segment with zeroes + public static void Clear([NotNull] byte[] bytes, int offset, int count) + { + if (count > 0) + { + EnsureBufferIsValidNotNull(bytes, offset, count); + fixed (byte* ptr = &bytes[offset]) + { + ClearUnsafe(ptr, (uint) count); + } + } + } + + /// Fill the content of a managed segment with zeroes + public static void Clear([NotNull] byte[] bytes, uint offset, uint count) + { + if (count > 0) + { + EnsureBufferIsValidNotNull(bytes, offset, count); + fixed (byte* ptr = &bytes[offset]) + { + ClearUnsafe(ptr, count); + } + } + } + + /// Fill the content of a managed slice with zeroes + public static void Clear(Slice buffer) + { + Clear(buffer.Array, buffer.Offset, buffer.Count); + } + + /// Fill the content of an unmanaged buffer with zeroes, without checking the arguments + /// WARNING: invalid use of this method WILL corrupt the heap! + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + public static void ClearUnsafe([NotNull] byte* ptr, uint length) + { + Contract.Requires(ptr != null); + switch (length) + { + case 0: + return; + case 1: + *ptr = 0; + return; + case 2: + *(short*) ptr = 0; + return; + case 3: + *(short*) ptr = 0; + *(ptr + 2) = 0; + return; + case 4: + *(int*) ptr = 0; + return; + case 5: + ((int*) ptr)[0] = 0; + *(ptr + 4) = 0; + return; + case 6: + *(int*) ptr = 0; + *(short*) (ptr + 4) = 0; + return; + case 7: + *(int*)ptr = 0; + *(short*)(ptr + 4) = 0; + *(ptr + 6) = 0; + return; + case 8: + *(long*)ptr = 0; + return; + } + + if (length >= 512) + { // PInvoke into the native memset + _memset(ptr, 0, length); + return; + } + + while (length >= 16) + { + ((long*) ptr)[0] = 0; + ((long*) ptr)[1] = 0; + ptr += 16; + length -= 16; + } + if ((length & 8) != 0) + { + ((long*)ptr)[0] = 0; + ptr += 8; + } + if ((length & 4) != 0) + { + ((uint*) ptr)[0] = 0; + ptr += 4; + } + if ((length & 2) != 0) + { + ((short*)ptr)[0] = 0; + ptr += 2; + } + if ((length & 1) != 0) + { + *ptr = 0; + } + } + + /// Fill the content of an unmanaged buffer with zeroes, without checking the arguments + /// WARNING: invalid use of this method WILL corrupt the heap! + public static void ClearUnsafe([NotNull] byte* ptr, ulong length) + { + //pre-check incase of uint overflow + if (length >= 512) + { + Contract.Requires(ptr != null); + _memset(ptr, 0, length); + } + else + { + ClearUnsafe(ptr, (uint) length); + } + } + + /// Fill the content of an unamanged array with zeroes, without checking the arguments + /// Pointer to the start of the array + /// Number of items to clear + /// Size (in bytes) of one item + /// Will clear * elements in the array + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ClearUnsafe([NotNull] void* ptr, [Positive] int count, uint sizeOfItem) + { + ClearUnsafe((byte*) ptr, checked((uint) count * sizeOfItem)); + } + + /// Fill the content of a managed segment with the same byte repeated + public static void Fill([NotNull] byte[] bytes, int offset, int count, byte filler) + { + if (count > 0) + { + EnsureBufferIsValidNotNull(bytes, offset, count); + fixed (byte* ptr = &bytes[offset]) + { + if (filler == 0) + { + ClearUnsafe(ptr, (uint)count); + } + else + { + _memset(ptr, filler, (uint)count); + } + } + } + } + + [SecurityCritical, ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void FillUnsafe([NotNull] byte* ptr, uint count, byte filler) + { + if (count != 0) + { + Contract.Requires(ptr != null); + _memset(ptr, filler, count); + } + } + + public static void FillUnsafe([NotNull] byte* ptr, ulong count, byte filler) + { + if (count != 0) + { + Contract.Requires(ptr != null); + _memset(ptr, filler, count); + } + } + + [SecurityCritical] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.NoInlining)] + private static void _memset([NotNull] byte* ptr, byte filler, uint count) + { + NativeMethods.memset(ptr, filler, (UIntPtr) count); + } + + [SecurityCritical] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [MethodImpl(MethodImplOptions.NoInlining)] + private static void _memset([NotNull] byte* ptr, byte filler, ulong count) + { + NativeMethods.memset(ptr, filler, (UIntPtr) count); + } + + /// Add padding bytes to the end of buffer if it is not aligned to a specific value, and advance the cursor + /// Start of a buffer that may need padding + /// Size of the buffer + /// Required alignement of the buffer size, which MUST be a power of two. If the buffer is not aligned, additional 0 bytes are added at the end. + /// Address of the next byte after the buffer, with padding included + [NotNull] + public static byte* PadBuffer([NotNull] byte* buffer, uint size, uint alignment) + { + Contract.PointerNotNull(buffer, nameof(buffer)); + Contract.PowerOfTwo(alignment, nameof(alignment)); + uint pad = size % (alignment - 1); + byte* ptr = buffer + size; + if (pad != 0) + { + ClearUnsafe(ptr, pad); + ptr += alignment - pad; + } + return ptr; + } + + /// Compute the hash code of a byte segment + /// Buffer + /// Offset of the start of the segment in the buffer + /// Number of bytes in the segment + /// A 32-bit signed hash code calculated from all the bytes in the segment. + /// This should only be used for dictionaries or hashset that reside in memory only! The hashcode could change at any time in future versions. + public static int ComputeHashCode(byte[] bytes, int offset, int count) + { + if (count == 0) return unchecked((int) 2166136261); + EnsureBufferIsValidNotNull(bytes, offset, count); + fixed (byte* ptr = &bytes[offset]) + { + return ComputeHashCodeUnsafe(ptr, (uint) count); + } + } + + /// Compute the hash code of a byte buffer + /// This should only be used for dictionaries or hashset that reside in memory only! The hashcode could change at any time in future versions. + public static int ComputeHashCode(byte* bytes, uint count) + { + if (count == 0) return unchecked((int) 2166136261); + EnsureBufferIsValidNotNull(bytes, count); + return ComputeHashCodeUnsafe(bytes, count); + } + + /// Compute the hash code of a byte buffer + /// Array that contains the byte buffer (ignored if count == 0) + /// Offset of the first byte in the buffer (ignored if count == 0) + /// Number of bytes in the buffer + /// A 32-bit signed hash code calculated from all the bytes in the segment. + /// + /// If count == 0, then the value of is ignored. + /// This should only be used for dictionaries or hashset that reside in memory only! The hashcode could change at any time in future versions. + /// + internal static int ComputeHashCodeUnsafe([NotNull] byte[] bytes, int offset, int count) + { + if (count == 0) return unchecked((int) 2166136261); + fixed (byte* ptr = &bytes[offset]) + { + return ComputeHashCodeUnsafe(ptr, (uint) count); + } + } + + /// Compute the hash code of a byte buffer + /// Pointer to the first byte of the buffer (ignored if count == 0) + /// Number of bytes in the buffer + /// A 32-bit signed hash code calculated from all the bytes in the segment. + /// This should only be used for dictionaries or hashset that reside in memory only! The hashcode could change at any time in future versions. + internal static int ComputeHashCodeUnsafe([NotNull] byte* bytes, uint count) + { + //note: callers should have handled the case where bytes == null, but they can call us with count == 0 + Contract.Requires(bytes != null); + + //TODO: use a better hash algorithm? (xxHash, CityHash, SipHash, ...?) + // => will be called a lot when Slices are used as keys in an hash-based dictionary (like Dictionary) + // => won't matter much for *ordered* dictionary that will probably use IComparer.Compare(..) instead of the IEqalityComparer.GetHashCode()/Equals() combo + // => we don't need a cryptographic hash, just something fast and suitable for use with hashtables... + // => probably best to select an algorithm that works on 32-bit or 64-bit chunks + + // : unoptimized 32 bits FNV-1a implementation + uint h = 2166136261; // FNV1 32 bits offset basis + uint n = count; + while (n > 0) + { + h = unchecked ((h ^ *bytes++) * 16777619); // FNV1 32 prime + --n; + } + return unchecked((int) h); + // + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteBytesUnsafe([NotNull] byte* cursor, [NotNull] byte* data, uint count) + { + Contract.Requires(cursor != null && data != null); + if (count > 0) System.Buffer.MemoryCopy(data, cursor, count, count); + return cursor + count; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteBytes([NotNull] byte* cursor, [NotNull] byte* stop, [NotNull] byte* data, uint count) + { + Contract.Requires(cursor != null && stop != null && data != null); + if (count > 0) + { + if (cursor + count > stop) throw Errors.BufferOutOfBound(); + System.Buffer.MemoryCopy(data, cursor, count, count); + } + return cursor + count; + } + + #region VarInt Encoding... + + // VarInt encoding uses 7-bit per byte for the value, and uses the 8th bit as a "continue" (1) or "stop" (0) bit. + // The values is stored in Little Endian, ie: first the 7 lowest bits, then the next 7 lowest bits, until the 7 highest bits. + // + // ex: 0xxxxxxx = 1 byte (<= 127) + // 1xxxxxxx 0xxxxxxx = 2 bytes (<= 16383) + // 1xxxxxxx 1xxxxxxx 0xxxxxxx = 3 bytes (<= 2097151) + // + // The number of bytes required to store uint.MaxValue is 5 bytes, and for ulong.MaxValue is 9 bytes. + + /// Return the size (in bytes) that a 32-bit number would need when encoded as a VarInt + /// Number that needs to be encoded + /// Number of bytes needed (1-5) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfVarInt(uint value) + { + return value < (1U << 7) ? 1 : SizeOfVarIntSlow(value); + } + + private static uint SizeOfVarIntSlow(uint value) + { + // count is already known to be >= 128 + if (value < (1U << 14)) return 2; + if (value < (1U << 21)) return 3; + if (value < (1U << 28)) return 4; + return 5; + } + + /// Return the size (in bytes) that a 64-bit number would need when encoded as a VarInt + /// Number that needs to be encoded + /// Number of bytes needed (1-10) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfVarInt(ulong value) + { + return value < (1UL << 7) ? 1 : SizeOfVarIntSlow(value); + } + + private static uint SizeOfVarIntSlow(ulong value) + { + // value is already known to be >= 128 + if (value < (1UL << 14)) return 2; + if (value < (1UL << 21)) return 3; + if (value < (1UL << 28)) return 4; + if (value < (1UL << 35)) return 5; + if (value < (1UL << 42)) return 6; + if (value < (1UL << 49)) return 7; + if (value < (1UL << 56)) return 8; + if (value < (1UL << 63)) return 9; + return 10; + } + + /// Return the size (in bytes) that a variable-size array of bytes would need when encoded as a VarBytes + /// Size (in bytes) of the array + /// Number of bytes needed to encoded the size of the array, and the array itself (1 + N <= size <= 5 + N) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfVarBytes(uint size) + { + return checked(size + SizeOfVarInt(size)); + } + /// Return the size (in bytes) that a variable-size array of bytes would need when encoded as a VarBytes + /// Size (in bytes) of the array + /// Number of bytes needed to encoded the size of the array, and the array itself (1 + N <= size <= 5 + N) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int SizeOfVarBytes(int size) + { + return checked(size + (int) SizeOfVarInt((uint) size)); + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 3 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt16Unsafe([NotNull] byte* cursor, uint value) + { + Contract.Requires(cursor != null); + //note: use of '&' is intentional (prevent a branch in the generated code) + if (value < 0x80) + { + *cursor = (byte) value; + return cursor + 1; + } + return WriteVarInt32UnsafeSlow(cursor, value); + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 3 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt16([NotNull] byte* cursor, [NotNull] byte* stop, ushort value) + { + Contract.Requires(cursor != null && stop != null); + //note: use of '&' is intentional (prevent a branch in the generated code) + if (cursor < stop & value < 0x80) + { + *cursor = (byte) value; + return cursor + 1; + } + return WriteVarInt32Slow(cursor, stop, value); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint16') from the buffer, and advances the cursor + /// Can read up to 3 bytes from the input + [NotNull] + public static byte* ReadVarint16([NotNull] byte* cursor, [NotNull] byte* stop, out ushort value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor < stop && (value = *cursor) < 0x80) + { + return cursor + 1; + } + return ReadVarint16Slow(cursor, stop, out value); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor + /// Can read up to 5 bytes from the input + [NotNull] + private static byte* ReadVarint16Slow([NotNull] byte* cursor, [NotNull] byte* stop, out ushort value) + { + uint n; + + // unless cursor >= stop, we already know that the first byte has the MSB set + if (cursor >= stop) goto overflow; + uint b = cursor[0]; + Contract.Assert(b >= 0x80); + uint res = b & 0x7F; + + if (cursor + 1 >= stop) goto overflow; + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + if (cursor + 2 >= stop) goto overflow; + b = cursor[2]; + // third should only have 2 bits worth of data + if (b >= 0x04) throw Errors.VarIntOverflow(); + res |= (b & 0x3) << 14; + n = 3; + //TODO: check overflow bits? + + done: + value = (ushort) res; + return cursor + n; + + overflow: + value = 0; + throw Errors.VarIntTruncated(); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint16') from the buffer, and advances the cursor + /// Can read up to 3 bytes from the input + [NotNull] + public static byte* ReadVarint16Unsafe([NotNull] byte* cursor, out ushort value) + { + Contract.Requires(cursor != null); + uint n = 1; + + //TODO: we expect most values to be small (count or array length), so we should optimize for single byte varints where byte[0] <= 127 should be inlined, and defer to a slower method if >= 128. + + uint b = cursor[0]; + uint res = b & 0x7F; + if (b < 0x80) + { + goto done; + } + + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + b = cursor[2]; + // third should only have 2 bits worth of data + if (b >= 0x04) throw Errors.VarIntOverflow(); + res |= (b & 0x3) << 14; + n = 3; + + done: + value = (ushort) res; + return cursor + n; + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 5 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt32Unsafe([NotNull] byte* cursor, uint value) + { + Contract.Requires(cursor != null); + if (value < 0x80) + { + *cursor = (byte) value; + return cursor + 1; + } + return WriteVarInt32UnsafeSlow(cursor, value); + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 5 bytes + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteVarInt32UnsafeSlow([NotNull] byte* cursor, uint value) + { + byte* ptr = cursor; + while (value >= 0x80) + { + *ptr = (byte)(value | 0x80); + value >>= 7; + ++ptr; + } + *ptr = (byte)value; + return ptr + 1; + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 5 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt32([NotNull] byte* cursor, [NotNull] byte* stop, uint value) + { + Contract.Requires(cursor != null && stop != null); + //note: use of '&' is intentional (prevent a branch in the generated code) + if (cursor < stop & value < 0x80) + { + *cursor = (byte)value; + return cursor + 1; + } + return WriteVarInt32Slow(cursor, stop, value); + } + + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteVarInt32Slow([NotNull] byte* cursor, [NotNull] byte* stop, uint value) + { + //note: we know that value >= 128 (or that cursor is >= stop, in which case we will immediately fail below) + byte* ptr = cursor; + do + { + if (ptr >= stop) throw Errors.BufferOutOfBound(); + *ptr = (byte) (value | 0x80); + value >>= 7; + ++ptr; + } while (value >= 0x80); + + if (ptr >= stop) throw Errors.BufferOutOfBound(); + *ptr = (byte) value; + return ptr + 1; + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor + /// Can read up to 5 bytes from the input + [NotNull] + public static byte* ReadVarint32Unsafe([NotNull] byte* cursor, out uint value) + { + Contract.Requires(cursor != null); + uint n = 1; + + //TODO: we expect most values to be small (count or array length), so we should optimize for single byte varints where byte[0] <= 127 should be inlined, and defer to a slower method if >= 128. + + uint b = cursor[0]; + uint res = b & 0x7F; + if (b < 0x80) + { + goto done; + } + + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + b = cursor[2]; + res |= (b & 0x7F) << 14; + if (b < 0x80) + { + n = 3; + goto done; + } + + b = cursor[3]; + res |= (b & 0x7F) << 21; + if (b < 0x80) + { + n = 4; + goto done; + } + + // the fifth byte should only have 4 bits worth of data + b = cursor[4]; + if (b >= 0x20) throw Errors.VarIntOverflow(); + res |= (b & 0x1F) << 28; + n = 5; + + done: + value = res; + return cursor + n; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadVarint32([NotNull] byte* cursor, [NotNull] byte* stop, out uint value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor < stop && (value = *cursor) < 0x80) + { + return cursor + 1; + } + return ReadVarint32Slow(cursor, stop, out value); + } + + /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor + /// Can read up to 5 bytes from the input + [NotNull] + private static byte* ReadVarint32Slow([NotNull] byte* cursor, [NotNull] byte* stop, out uint value) + { + uint n; + + // unless cursor >= stop, we already know that the first byte has the MSB set + if (cursor >= stop) goto overflow; + uint b = cursor[0]; + Contract.Assert(b >= 0x80); + uint res = b & 0x7F; + + if (cursor + 1 >= stop) goto overflow; + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + if (cursor + 2 >= stop) goto overflow; + b = cursor[2]; + res |= (b & 0x7F) << 14; + if (b < 0x80) + { + n = 3; + goto done; + } + + if (cursor + 3 >= stop) goto overflow; + b = cursor[3]; + res |= (b & 0x7F) << 21; + if (b < 0x80) + { + n = 4; + goto done; + } + + // the fifth byte should only have 4 bits worth of data + if (cursor + 4 >= stop) goto overflow; + b = cursor[4]; + if (b >= 0x20) throw Errors.VarIntOverflow(); + res |= (b & 0x1F) << 28; + n = 5; + + done: + value = res; + return cursor + n; + + overflow: + value = 0; + throw Errors.VarIntTruncated(); + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 10 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt64Unsafe([NotNull] byte* cursor, ulong value) + { + Contract.Requires(cursor != null); + if (value < 0x80) + { + *cursor = (byte)value; + return cursor + 1; + } + return WriteVarInt64UnsafeSlow(cursor, value); + } + + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteVarInt64UnsafeSlow([NotNull] byte* cursor, ulong value) + { + //note: we know that value >= 128 + byte* ptr = cursor; + do + { + *ptr = (byte) (value | 0x80); + value >>= 7; + ++ptr; + } while (value >= 0x80); + *ptr = (byte)value; + return ptr + 1; + } + + /// Append a variable sized number to the output buffer + /// Pointer to the next free byte in the buffer + /// Stop address (to prevent overflow) + /// Value of the number to output + /// Pointer updated with the number of bytes written + /// Will write between 1 and 10 bytes + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteVarInt64([NotNull] byte* cursor, byte* stop, ulong value) + { + Contract.Requires(cursor != null && stop != null); + //note: use of '&' is intentional (prevent a branch in the generated code) + if (cursor < stop & value < 0x80) + { + *cursor = (byte) value; + return cursor + 1; + } + return WriteVarInt64Slow(cursor, stop, value); + } + + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteVarInt64Slow([NotNull] byte* cursor, byte* stop, ulong value) + { + //note: we know that value >= 128 (or that cursor is >= stop, in which case we will immediately fail below) + byte* ptr = cursor; + do + { + if (ptr >= stop) throw Errors.BufferOutOfBound(); + *ptr = (byte) (value | 0x80); + value >>= 7; + ++ptr; + } while (value >= 0x80); + + if (ptr >= stop) throw Errors.BufferOutOfBound(); + *ptr = (byte)value; + return ptr + 1; + } + + /// Reads a 7-bit encoded unsigned long (aka 'Varint32') from the buffer, and advances the cursor + /// Can read up to 10 bytes from the input + [NotNull] + public static byte* ReadVarint64Unsafe([NotNull] byte* cursor, out ulong value) + { + Contract.Requires(cursor != null); + uint n = 1; + + //note: we expect the value to be large (most frequent use it to decode a Sequence Number), so there is no point in optimizing for single byte varints... + + ulong b = cursor[0]; + ulong res = b & 0x7F; + if (b < 0x80) + { + goto done; + } + + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + b = cursor[2]; + res |= (b & 0x7F) << 14; + if (b < 0x80) + { + n = 3; + goto done; + } + + b = cursor[3]; + res |= (b & 0x7F) << 21; + if (b < 0x80) + { + n = 4; + goto done; + } + + b = cursor[4]; + res |= (b & 0x7F) << 28; + if (b < 0x80) + { + n = 5; + goto done; + } + + b = cursor[5]; + res |= (b & 0x7F) << 35; + if (b < 0x80) + { + n = 6; + goto done; + } + + b = cursor[6]; + res |= (b & 0x7F) << 42; + if (b < 0x80) + { + n = 7; + goto done; + } + + b = cursor[7]; + res |= (b & 0x7F) << 49; + if (b < 0x80) + { + n = 8; + goto done; + } + + b = cursor[8]; + res |= (b & 0x7F) << 56; + if (b < 0x80) + { + n = 9; + goto done; + } + + // the tenth byte should only have 1 bit worth of data + b = cursor[9]; + if (b > 1) throw Errors.VarIntOverflow(); + res |= (b & 0x1) << 63; + n = 10; + + done: + value = res; + return cursor + n; + } + + /// Reads a 7-bit encoded unsigned long (aka 'Varint32') from the buffer, and advances the cursor + /// Can read up to 10 bytes from the input + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadVarint64([NotNull] byte* cursor, [NotNull] byte* stop, out ulong value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor < stop && (value = *cursor) < 0x80) + { + return cursor + 1; + } + else + { + return ReadVarint64Slow(cursor, stop, out value); + } + } + + [NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static byte* ReadVarint64Slow([NotNull] byte* cursor, [NotNull] byte* stop, out ulong value) + { + uint n; + + // unless cursor >= stop, we already know that the first byte has the MSB set + if (cursor >= stop) goto overflow; + ulong b = cursor[0]; + Contract.Assert(b >= 0x80); + ulong res = b & 0x7F; + + if (cursor >= stop) goto overflow; + b = cursor[1]; + res |= (b & 0x7F) << 7; + if (b < 0x80) + { + n = 2; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[2]; + res |= (b & 0x7F) << 14; + if (b < 0x80) + { + n = 3; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[3]; + res |= (b & 0x7F) << 21; + if (b < 0x80) + { + n = 4; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[4]; + res |= (b & 0x7F) << 28; + if (b < 0x80) + { + n = 5; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[5]; + res |= (b & 0x7F) << 35; + if (b < 0x80) + { + n = 6; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[6]; + res |= (b & 0x7F) << 42; + if (b < 0x80) + { + n = 7; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[7]; + res |= (b & 0x7F) << 49; + if (b < 0x80) + { + n = 8; + goto done; + } + + if (cursor >= stop) goto overflow; + b = cursor[8]; + res |= (b & 0x7F) << 56; + if (b < 0x80) + { + n = 9; + goto done; + } + + // the tenth byte should only have 1 bit worth of data + if (cursor >= stop) goto overflow; + b = cursor[9]; + if (b > 1) throw Errors.VarIntOverflow(); + res |= (b & 0x1) << 63; + n = 10; + + done: + value = res; + return cursor + n; + + overflow: + value = 0; + throw Errors.VarIntTruncated(); + } + + /// Append a variable size byte sequence, using the VarInt encoding + /// This method performs bound checking. + [NotNull] + public static byte* WriteVarBytes([NotNull] byte* ptr, [NotNull] byte* stop, byte* data, uint count) + { + if (count == 0) + { // "Nil" + if (ptr >= stop) throw Errors.BufferOutOfBound(); + *ptr = 0; + return ptr + 1; + } + var cursor = WriteVarInt32(ptr, stop, count); + return WriteBytes(cursor, stop, data, count); + } + + /// Append a variable size byte sequence with an extra 0 at the end, using the VarInt encoding + /// This method performs bound checking. + [NotNull] + public static byte* WriteZeroTerminatedVarBytes([NotNull] byte* ptr, [NotNull] byte* stop, byte* data, uint count) + { + var cursor = WriteVarInt32(ptr, stop, count + 1); + cursor = WriteBytes(cursor, stop, data, count); + if (cursor >= stop) throw Errors.BufferOutOfBound(); + *cursor = 0; + return cursor + 1; + } + + #endregion + + #region Endianness... + +#if EXPECT_LITTLE_ENDIAN_HOST + // ReSharper disable ConditionIsAlwaysTrueOrFalse + // ReSharper disable UnreachableCode +#pragma warning disable 162 +#endif + + #region 16-bits + + /// Swap the order of the bytes in a 16-bit word + /// 0x0123 + /// 0x2301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ushort ByteSwap16(ushort value) + { + return (ushort) ((value << 8) | (value >> 8)); + } + + /// Swap the order of the bytes in a 16-bit word + /// 0x0123 + /// 0x2301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static short ByteSwap16(short value) + { + //note: masking is required to get rid of the sign bit + return (short) ((value << 8) | ((value >> 8) & 0xFF)); + } + + /// Load a 16-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x34, 0x12) => 0x1234 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static short LoadInt16LE([NotNull] void* ptr) + { + return IsLittleEndian ? *(short*)ptr : ByteSwap16(*(short*)ptr); + } + + /// Load a 16-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x34, 0x12) => 0x1234 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ushort LoadUInt16LE([NotNull] void* ptr) + { + return IsLittleEndian ? *(ushort*) ptr : ByteSwap16(*(ushort*) ptr); + } + + /// Store a 16-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value to store in the buffer + /// (ptr, 0x1234) => ptr[0] == 0x34, ptr[1] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt16LE([NotNull] void* ptr, short value) + { + *(short*)ptr = IsLittleEndian ? value : ByteSwap16(value); + } + + /// Store a 16-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value to store in the buffer + /// (ptr, 0x1234) => ptr[0] == 0x34, ptr[1] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt16LE([NotNull] void* ptr, ushort value) + { + *(ushort*) ptr = IsLittleEndian ? value : ByteSwap16(value); + } + + /// Load a 16-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x34, 0x12) => 0x1234 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static short LoadInt16BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap16(*(short*) ptr) : *(short*) ptr; + } + + /// Load a 16-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x12, 0x34) => 0x1234 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ushort LoadUInt16BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap16(*(ushort*) ptr) : *(ushort*) ptr; + } + + /// Store a 16-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 2-byte location + /// Logical value to store in the buffer + /// (ptr, 0x1234) => ptr[0] == 0x12, ptr[1] == 0x34 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt16BE([NotNull] void* ptr, short value) + { + *(short*) ptr = IsLittleEndian ? ByteSwap16(value) : value; + } + + /// Store a 16-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 2-byte location + /// Logical value to store in the buffer + /// (ptr, 0x1234) => ptr[0] == 0x12, ptr[1] == 0x34 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt16BE([NotNull] void* ptr, ushort value) + { + *(ushort*) ptr = IsLittleEndian ? ByteSwap16(value) : value; + } + + #endregion + + #region 24-bits + + /// Swap the order of the bytes in a 24-bit word + /// 0x012345 + /// 0x452301 + /// Bits 24-31 are ignored + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint ByteSwap24(uint value) + { + return (value & 0xFF) << 16 | (value & 0x00FF00) | ((value & 0xFF0000) >> 16); + } + + /// Swap the order of the bytes in a 24-bit word + /// 0x0123 + /// 0x2301 + /// Bits 24-31 are ignored + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int ByteSwap24(int value) + { + //note: masking is required to get rid of the sign bit + return (value & 0xFF) << 16 | (value & 0x00FF00) | ((value & 0xFF0000) >> 16); + } + + /// Load a 24-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x56, 0x34, 0x12 ]) => 0x123456 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LoadInt24LE([NotNull] void* ptr) + { + uint x = *(ushort*) ptr; + x |= (uint) ((byte*) ptr)[2] << 16; + return IsLittleEndian ? (int) x : (int) ByteSwap24(x); + } + + /// Load a 24-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 2-byte location + /// Logical value in host order + /// ([ 0x56, 0x34, 0x12 ]) => 0x123456 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint LoadUInt24LE([NotNull] void* ptr) + { + uint x = *(ushort*)ptr; + x |= (uint) ((byte*) ptr)[2] << 16; + return IsLittleEndian ? x : ByteSwap24(x); + } + + /// Store a 24-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 3-byte location + /// Logical value to store in the buffer. Bits 24-31 are ignored + /// (ptr, 0x123456) => ptr[0] == 0x56, ptr[1] == 0x34, ptr[2] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt24LE([NotNull] void* ptr, int value) + { + int x = IsLittleEndian ? value : ByteSwap24(value); + *(short*) ptr = (short) x; + ((byte*) ptr)[2] = (byte) (x >> 16); + } + + /// Store a 24-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 3-byte location + /// Logical value to store in the buffer. Bits 24-31 are ignored + /// (ptr, 0x123456) => ptr[0] == 0x56, ptr[1] == 0x34, ptr[2] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt24LE([NotNull] void* ptr, uint value) + { + uint x = IsLittleEndian ? value : ByteSwap24(value); + *(ushort*)ptr = (ushort)x; + ((byte*)ptr)[2] = (byte)(x >> 16); + } + + /// Load a 24-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 3-byte location + /// Logical value in host order + /// ([ 0x12, 0x34, 0x56 ]) => 0x123456 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LoadInt24BE([NotNull] void* ptr) + { + uint x = *(ushort*) ptr | ((uint) ((byte*) ptr)[2] << 16); + return IsLittleEndian ? ByteSwap24((int) x) : (int) x; + } + + /// Load a 24-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 3-byte location + /// Logical value in host order + /// ([ 0x12, 0x34, 0x56 ]) => 0x123456 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint LoadUInt24BE([NotNull] void* ptr) + { + uint x = *(ushort*) ptr | ((uint) ((byte*) ptr)[2] << 16); + return IsLittleEndian ? ByteSwap24(x) : x; + } + + /// Store a 24-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 3-byte location + /// Logical value to store in the buffer. Bits 24-31 are ignored + /// (ptr, 0x123456) => ptr[0] == 0x12, ptr[1] == 0x34, ptr[2] = 0x56 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt24BE([NotNull] void* ptr, int value) + { + int x = IsLittleEndian ? ByteSwap24(value) : value; + *(short*) ptr = (short) x; + ((byte*) ptr)[2] = (byte) (x >> 16); + } + + /// Store a 24-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 3-byte location + /// Logical value to store in the buffer. Bits 24-31 are ignored + /// (ptr, 0x123456) => ptr[0] == 0x12, ptr[1] == 0x34, ptr[2] = 0x56 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt24BE([NotNull] void* ptr, uint value) + { + uint x = IsLittleEndian ? ByteSwap24(value) : value; + *(ushort*)ptr = (ushort)x; + ((byte*)ptr)[2] = (byte)(x >> 16); + } + + #endregion + + #region 32-bits + + /// Swap the order of the bytes in a 32-bit word + /// 0x01234567 + /// 0x67452301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint ByteSwap32(uint value) + { + const uint MASK1_HI = 0xFF00FF00; + const uint MASK1_LO = 0x00FF00FF; + //PERF: do not remove the local 'tmp' variable (reusing 'value' is 4X slower with RyuJit64 than introducing a tmp variable) + uint tmp = ((value << 8) & MASK1_HI) | ((value >> 8) & MASK1_LO); + return (tmp << 16) | (tmp >> 16); + } + + /// Swap the order of the bytes in a 32-bit word + /// 0x01234567 + /// 0x67452301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int ByteSwap32(int value) + { + const int MASK1_HI = unchecked((int) 0xFF00FF00); + const int MASK1_LO = 0x00FF00FF; + //PERF: do not remove the local 'tmp' variable! Reusing 'value' is 4X slower with RyuJit64 than introducing a tmp variable + int tmp = ((value << 8) & MASK1_HI) | ((value >> 8) & MASK1_LO); + return (tmp << 16) | ((tmp >> 16) & 0xFFFF); + } + + /// Load a 32-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 4-byte location + /// Logical value in host order + /// ([ 0x78, 0x56, 0x34, 0x12) => 0x12345678 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LoadInt32LE([NotNull] void* ptr) + { + return IsLittleEndian ? *(int*) ptr : ByteSwap32(*(int*) ptr); + } + + /// Load a 32-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 4-byte location + /// Logical value in host order + /// ([ 0x78, 0x56, 0x34, 0x12) => 0x12345678 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint LoadUInt32LE([NotNull] void* ptr) + { + return IsLittleEndian ? * (uint*) ptr : ByteSwap32(* (uint*) ptr); + } + + /// Store a 32-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 4-byte location + /// Logical value to store in the buffer + /// (0x12345678) => ptr[0] == 0x78, ptr[1] == 0x56, ptr[2] == 0x34, ptr[3] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt32LE([NotNull] void* ptr, int value) + { + *(int*) ptr = IsLittleEndian ? value : ByteSwap32(value); + } + + /// Store a 32-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of a 4-byte location + /// Logical value to store in the buffer + /// (0x12345678) => ptr[0] == 0x78, ptr[1] == 0x56, ptr[2] == 0x34, ptr[3] == 0x12 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt32LE([NotNull] void* ptr, uint value) + { + *(uint*) ptr = IsLittleEndian ? value : ByteSwap32(value); + } + + /// Load a 32-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 4-byte location + /// Logical value in host order + /// ([ 0x12, 0x34, 0x56, 0x78) => 0x12345678 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int LoadInt32BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap32(*(int*) ptr) : *(int*) ptr; + } + + /// Load a 32-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 4-byte location + /// Logical value in host order + /// ([ 0x12, 0x34, 0x56, 0x78) => 0x12345678 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint LoadUInt32BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap32(*(uint*) ptr) : *(uint*) ptr; + } + + /// Store a 32-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 4-byte location + /// Logical value to store in the buffer + /// (ptr, 0x12345678) => ptr[0] == 0x12, ptr[1] == 0x34, ptr[2] == 0x56, ptr[3] == 0x78 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt32BE([NotNull] void* ptr, int value) + { + *(int*) ptr = IsLittleEndian ? ByteSwap32(value) : value; + } + + /// Store a 32-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of a 4-byte location + /// Logical value to store in the buffer + /// (ptr, 0x12345678) => ptr[0] == 0x12, ptr[1] == 0x34, ptr[2] == 0x56, ptr[3] == 0x78 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt32BE([NotNull] void* ptr, uint value) + { + *(uint*) ptr = IsLittleEndian ? ByteSwap32(value) : value; + } + + #endregion + + #region 64-bits + + /// Swap the order of the bytes in a 64-bit word + /// 0x0123456789ABCDEF + /// 0xEFCDAB8967452301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong ByteSwap64(ulong value) + { + const ulong MASK1_HI = 0xFF00FF00FF00FF00UL; + const ulong MASK1_LO = 0x00FF00FF00FF00FFUL; + const ulong MASK2_HI = 0xFFFF0000FFFF0000UL; + const ulong MASK2_LO = 0x0000FFFF0000FFFFUL; + + //PERF: do not remove the local 'tmp' variable! Reusing 'value' is 4X slower with RyuJit64 than introducing a tmp variable + ulong tmp = ((value << 8) & MASK1_HI) | ((value >> 8) & MASK1_LO); // swap pairs of 1 byte + tmp = ((tmp << 16) & MASK2_HI) | ((tmp >> 16) & MASK2_LO); // swap pairs of 2 bytes + return (tmp << 32) | (tmp >> 32); // swap pairs of 4 bytes + } + + /// Swap the order of the bytes in a 64-bit word + /// 0x0123456789ABCDEF + /// 0xEFCDAB8967452301 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static long ByteSwap64(long value) + { + const long MASK1_HI = unchecked((long) 0xFF00FF00FF00FF00L); + const long MASK1_LO = 0x00FF00FF00FF00FFL; + const long MASK2_HI = unchecked((long) 0xFFFF0000FFFF0000L); + const long MASK2_LO = 0x0000FFFF0000FFFFL; + + //PERF: do not remove the local 'tmp' variable! Reusing 'value' is 4X slower with RyuJit64 than introducing a tmp variable + long tmp = ((value << 8) & MASK1_HI) | ((value >> 8) & MASK1_LO); // swap pairs of 1 byte + tmp = ((tmp << 16) & MASK2_HI) | ((tmp >> 16) & MASK2_LO); // swap pairs of 2 bytes + return (tmp << 32) | ((tmp >> 32) & 0xFFFFFFFFL); // swap pairs of 4 bytes + } + + /// Load a 64-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of an 8-byte location + /// Logical value in host order + /// ([ 0xEF, 0xCD, 0xAB, 0x89, 0x67, 0x456, 0x23, 0x01) => 0x0123456789ABCDEF + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static long LoadInt64LE([NotNull] void* ptr) + { + return IsLittleEndian ? *(long*) ptr : ByteSwap64(*(long*) ptr); + } + + /// Load a 64-bit integer from an in-memory buffer that holds a value in Little-Endian ordering (also known as Host Order) + /// Memory address of an 8-byte location + /// Logical value in host order + /// ([ 0xEF, 0xCD, 0xAB, 0x89, 0x67, 0x456, 0x23, 0x01) => 0x0123456789ABCDEF + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong LoadUInt64LE([NotNull] void* ptr) + { + return IsLittleEndian ? *(ulong*) ptr : ByteSwap64(*(ulong*) ptr); + } + + /// Store a 64-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of an 8-byte location + /// Logical value to store in the buffer + /// (0x0123456789ABCDEF) => ptr[0] == 0xEF, ptr[1] == 0xCD, ptr[2] == 0xAB, ptr[3] == 0x89, ..., ptr[7] == 0x01 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt64LE([NotNull] void* ptr, long value) + { + *(long*) ptr = IsLittleEndian ? value : ByteSwap64(value); + } + + /// Store a 64-bit integer in an in-memory buffer that must hold a value in Little-Endian ordering (also known as Host Order) + /// Memory address of an 8-byte location + /// Logical value to store in the buffer + /// (0x0123456789ABCDEF) => ptr[0] == 0xEF, ptr[1] == 0xCD, ptr[2] == 0xAB, ptr[3] == 0x89, ..., ptr[7] == 0x01 + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt64LE([NotNull] void* ptr, ulong value) + { + *(ulong*) ptr = IsLittleEndian ? value : ByteSwap64(value); + } + + /// Load a 64-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of an 8-byte location + /// Logical value in host order + /// ([ 0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF) => 0x0123456789ABCDEF + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static long LoadInt64BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap64(*(long*) ptr) : *(long*) ptr; + } + + /// Load a 64-bit integer from an in-memory buffer that holds a value in Big-Endian ordering (also known as Network Order) + /// Memory address of an 8-byte location + /// Logical value in host order + /// ([ 0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF) => 0x0123456789ABCDEF + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong LoadUInt64BE([NotNull] void* ptr) + { + return IsLittleEndian ? ByteSwap64(*(ulong*) ptr) : *(ulong*) ptr; + } + + /// Store a 64-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of an 8-byte location + /// Logical value to store in the buffer + /// (ptr, 0x0123456789ABCDEF) => ptr[0] == 0x01, ptr[1] == 0x23, ptr[2] == 0x45, ptr[3] == 0x67, ..., ptr[7] == 0xEF + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreInt64BE([NotNull] void* ptr, long value) + { + *(long*) ptr = IsLittleEndian ? ByteSwap64(value) : value; + } + + /// Store a 64-bit integer in an in-memory buffer that must hold a value in Big-Endian ordering (also known as Network Order) + /// Memory address of an 8-byte location + /// Logical value to store in the buffer + /// (ptr, 0x0123456789ABCDEF) => ptr[0] == 0x01, ptr[1] == 0x23, ptr[2] == 0x45, ptr[3] == 0x67, ..., ptr[7] == 0xEF + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void StoreUInt64BE([NotNull] void* ptr, ulong value) + { + *(ulong*) ptr = IsLittleEndian ? ByteSwap64(value) : value; + } + + #endregion + +#if EXPECT_LITTLE_ENDIAN_HOST + #pragma warning restore 162 + // ReSharper restore UnreachableCode + // ReSharper restore ConditionIsAlwaysTrueOrFalse +#endif + + #endregion + + #region Fixed-Size Encoding + + // Plain old encoding where 32-bit values are stored using 4 bytes, 64-bit values are stored using 8 bytes, etc... + // Methods without suffix use Little-Endian, while methods with 'BE' suffix uses Big Endian. + + #region 16-bit + + /// Append a fixed size 16-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed16Unsafe([NotNull] byte* cursor, ushort value) + { + Contract.Requires(cursor != null); + StoreUInt16LE((ushort*) cursor, value); + return cursor + 2; + } + + /// Append a fixed size 16-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed16([NotNull] byte* cursor, [NotNull] byte* stop, ushort value) + { + Contract.Requires(cursor != null & stop != null); + if (cursor + 2 > stop) throw Errors.BufferOutOfBound(); + StoreUInt16LE((ushort*) cursor, value); + return cursor + 2; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ushort ReadFixed16([NotNull] byte* p) + { + return LoadUInt16LE((ushort*) p); + } + + [NotNull, Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed16([NotNull] byte* p, out ushort value) + { + value = LoadUInt16LE((ushort*) p); + return p + 2; + } + + /// Append a fixed size 16-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed16BEUnsafe([NotNull] byte* cursor, ushort value) + { + Contract.Requires(cursor != null); + StoreUInt16BE((ushort*) cursor, value); + return cursor + 2; + } + + /// Append a fixed size 16-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed16BE([NotNull] byte* cursor, [NotNull] byte* stop, ushort value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor + 2 > stop) throw Errors.BufferOutOfBound(); + StoreUInt16BE((ushort*) cursor, value); + return cursor + 2; + } + + /// Write a 16-bit zero + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteZeroFixed16([NotNull] byte* cursor) + { + // this does not care about LE or BE + *((ushort*)cursor) = 0; + return cursor + 2; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ushort ReadFixed16BE([NotNull] byte* p) + { + return LoadUInt16BE((ushort*) p); + } + + [NotNull, Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed16BE([NotNull] byte* p, out ushort value) + { + value = LoadUInt16BE((ushort*) p); + return p + 2; + } + + #endregion + + #region 32-bits + + /// Append a fixed size 32-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed32Unsafe([NotNull] byte* cursor, uint value) + { + Contract.Requires(cursor != null); + StoreUInt32LE((uint*) cursor, value); + return cursor + 4; + } + + /// Append a fixed size 32-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed32([NotNull] byte* cursor, [NotNull] byte* stop, uint value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor + 4 > stop) throw Errors.BufferOutOfBound(); + StoreUInt32LE((uint*) cursor, value); + return cursor + 4; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint ReadFixed32([NotNull] byte* p) + { + return LoadUInt32LE((uint*) p); + } + + [NotNull, Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed32([NotNull] byte* p, out uint value) + { + value = LoadUInt32LE((uint*) p); + return p + 4; + } + + /// Append a fixed size 32-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed32BEUnsafe([NotNull] byte* cursor, uint value) + { + Contract.Requires(cursor != null); + StoreUInt32BE((uint*) cursor, value); + return cursor + 4; + } + + /// Append a fixed size 32-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed32BE([NotNull] byte* cursor, [NotNull] byte* stop, uint value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor + 4 > stop) throw Errors.BufferOutOfBound(); + StoreUInt32BE((uint*) cursor, value); + return cursor + 4; + } + + /// Write a 32-bit zero + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteZeroFixed32([NotNull] byte* cursor) + { + // this does not care about LE or BE + *((uint*)cursor) = 0; + return cursor + 4; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint ReadFixed32BE([NotNull] byte* p) + { + return LoadUInt32BE((uint*) p); + } + + [NotNull, Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed32BE([NotNull] byte* p, out uint value) + { + value = LoadUInt32BE((uint*) p); + return p + 4; + } + + #endregion + + #region 64-bits + + /// Append a fixed size 64-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed64Unsafe([NotNull] byte* cursor, ulong value) + { + Contract.Requires(cursor != null); + StoreUInt64LE((ulong*) cursor, value); + return cursor + 8; + } + + /// Append a fixed size 64-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed64([NotNull] byte* cursor, [NotNull] byte* stop, ulong value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor + 8 > stop) throw Errors.BufferOutOfBound(); + StoreUInt64LE((ulong*) cursor, value); + return cursor + 8; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong ReadFixed64([NotNull] byte* p) + { + return LoadUInt64LE((ulong*) p); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed64([NotNull] byte* p, out ulong value) + { + value = LoadUInt64LE((ulong*) p); + return p + 8; + } + + /// Append a fixed size 64-bit number to the output buffer, using little-endian ordering + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed64BEUnsafe([NotNull] byte* cursor, ulong value) + { + Contract.Requires(cursor != null); + StoreUInt64BE((ulong*) cursor, value); + return cursor + 8; + } + + /// Append a fixed size 64-bit number to the output buffer, using little-endian ordering + /// This method DOES perform bound checking! Caller must ensure that the buffer has enough capacity + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteFixed64BE([NotNull] byte* cursor, [NotNull] byte* stop, ulong value) + { + Contract.Requires(cursor != null && stop != null); + if (cursor + 8 > stop) throw Errors.BufferOutOfBound(); + StoreUInt64BE((ulong*) cursor, value); + return cursor + 8; + } + + /// Write a 64-bit zero + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteZeroFixed64([NotNull] byte* cursor) + { + // this does not care about LE or BE + *((ulong*)cursor) = 0; + return cursor + 8; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static ulong ReadFixed64BE([NotNull] byte* p) + { + return LoadUInt64BE((ulong*) p); + } + + [NotNull, Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* ReadFixed64BE([NotNull] byte* p, out ulong value) + { + value = LoadUInt64BE((ulong*) p); + return p + 8; + } + + #endregion + + #endregion + + #region Compact Unordered Encoding... + + // Simple encoding where each integer is stored using the smallest number of bytes possible. + // The encoded result does preserve the value ordering, and the caller needs to remember the result size in order to decode the value from a stream. + // Values from 0 to 0xFF will use 1 byte, values from 0x100 for 0xFFFF will use two bytes, and so on. + + /// Return the minimum number of bytes that hold the bits set (1) in a 32-bit unsigned integer + /// Number that needs to be encoded + /// Number of bytes needed (1-4) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfCompact16(ushort value) + { + return value <= 0xFF ? 1U : 2U; + } + + /// Return the minimum number of bytes that hold the bits set (1) in a 32-bit unsigned integer + /// Number that needs to be encoded + /// Number of bytes needed (1-4) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfCompact32(uint value) + { + return value <= 0xFF ? 1U : SizeOfCompact32Slow(value); + } + + [Pure, MethodImpl(MethodImplOptions.NoInlining)] + private static uint SizeOfCompact32Slow(uint value) + { + // value is already known to be >= 256 + if (value < (1U << 16)) return 2; + if (value < (1U << 24)) return 3; + return 4; + } + + /// Return the minimum number of bytes that hold the bits set (1) in a 64-bit unsigned integer + /// Number that needs to be encoded + /// Number of bytes needed (1-8) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static uint SizeOfCompact64(ulong value) + { + return value <= 0xFF ? 1U : SizeOfCompact64Slow(value); + } + + [Pure, MethodImpl(MethodImplOptions.NoInlining)] + private static uint SizeOfCompact64Slow(ulong value) + { + // value is already known to be >= 256 + if (value < (1UL << 16)) return 2; + if (value < (1UL << 24)) return 3; + if (value < (1UL << 32)) return 4; + if (value < (1UL << 40)) return 5; + if (value < (1UL << 48)) return 6; + if (value < (1UL << 56)) return 7; + return 8; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact16Unsafe([NotNull] byte* ptr, ushort value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { + *ptr = (byte) value; + return ptr + 1; + } + + StoreUInt16LE((ushort*) ptr, value); + return ptr + 2; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact16BEUnsafe([NotNull] byte* ptr, ushort value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { + *ptr = (byte) value; + return ptr + 1; + } + + StoreUInt16BE((ushort*) ptr, value); + return ptr + 2; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact32Unsafe([NotNull] byte* ptr, uint value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { + ptr[0] = (byte) value; + return ptr + 1; + } + return WriteCompact32UnsafeSlow(ptr, value); + } + + [NotNull] + private static byte* WriteCompact32UnsafeSlow([NotNull] byte* ptr, uint value) + { + if (value <= 0xFFFF) + { + StoreUInt16LE((ushort*) ptr, (ushort) value); + return ptr + 2; + } + + if (value <= 0xFFFFFF) + { + StoreUInt16LE((ushort*) ptr, (ushort) value); + ptr[2] = (byte) (value >> 16); + return ptr + 3; + } + + StoreUInt32LE((uint*) ptr, value); + return ptr + 4; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact32BEUnsafe([NotNull] byte* ptr, uint value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { + ptr[0] = (byte) value; + return ptr + 1; + } + return WriteCompact32BEUnsafeSlow(ptr, value); + } + + [NotNull] + private static byte* WriteCompact32BEUnsafeSlow([NotNull] byte* ptr, uint value) + { + if (value <= 0xFFFF) + { + StoreUInt16BE((ushort*) ptr, (ushort) value); + return ptr + 2; + } + + if (value <= 0xFFFFFF) + { + ptr[0] = (byte) (value >> 16); + StoreUInt16BE((ushort*) (ptr + 1), (ushort) value); + return ptr + 3; + } + + StoreUInt32BE((uint*) ptr, value); + return ptr + 4; + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact64Unsafe([NotNull] byte* ptr, ulong value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { // 1 byte + ptr[0] = (byte) value; + return ptr + 1; + } + + if (value >= 0x100000000000000) + { // 8 bytes + StoreUInt64LE((ulong*) ptr, value); + return ptr + 8; + } + + return WriteCompact64UnsafeSlow(ptr, value); + } + + [NotNull] + private static byte* WriteCompact64UnsafeSlow([NotNull] byte* ptr, ulong value) + { + if (value <= 0xFFFFFFFF) + { // 2 .. 4 bytes + + if (value >= 0x1000000) + { + // 4 bytes + StoreUInt32LE((uint*) ptr, (uint) value); + return ptr + 4; + } + + StoreUInt16LE((ushort*) ptr, (ushort) value); + + if (value <= 0xFFFF) + { // 2 bytes + return ptr + 2; + } + + // 3 bytes + ptr[2] = (byte) (value >> 16); + return ptr + 3; + } + else + { // 5 .. 7 bytes + StoreUInt32LE((uint*) ptr, (uint) value); + + if (value <= 0xFFFFFFFFFF) + { // 5 bytes + ptr[4] = (byte) (value >> 32); + return ptr + 5; + } + + if (value <= 0xFFFFFFFFFFFF) + { // 6 bytes + StoreUInt16LE((ushort*) (ptr + 4), (ushort) (value >> 32)); + return ptr + 6; + } + + // 7 bytes + Contract.Assert(value <= 0xFFFFFFFFFFFFFF); + StoreUInt16LE((ushort*) (ptr + 4), (ushort) (value >> 32)); + ptr[6] = (byte) (value >> 48); + return ptr + 7; + } + } + + [NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteCompact64BEUnsafe([NotNull] byte* ptr, ulong value) + { + Contract.Requires(ptr != null); + if (value <= 0xFF) + { // 1 byte + ptr[0] = (byte) value; + return ptr + 1; + } + + if (value >= 0x100000000000000) + { // 8 bytes + StoreUInt64BE((ulong*) ptr, value); + return ptr + 8; + } + + return WriteCompact64BEUnsafeSlow(ptr, value); + } + + [NotNull] + private static byte* WriteCompact64BEUnsafeSlow([NotNull] byte* ptr, ulong value) + { + if (value <= 0xFFFFFFFF) + { // 2 .. 4 bytes + + if (value >= 0x1000000) + { + // 4 bytes + StoreUInt32BE((uint*) ptr, (uint) value); + return ptr + 4; + } + + + if (value <= 0xFFFF) + { // 2 bytes + StoreUInt16BE((ushort*) ptr, (ushort) value); + return ptr + 2; + } + + // 3 bytes + StoreUInt16BE((ushort*) ptr, (ushort) (value >> 8)); + ptr[2] = (byte) value; + return ptr + 3; + } + else + { // 5 .. 7 bytes + + if (value <= 0xFFFFFFFFFF) + { // 5 bytes + StoreUInt32BE((uint*) ptr, (uint) (value >> 8)); + ptr[4] = (byte) value; + return ptr + 5; + } + + if (value <= 0xFFFFFFFFFFFF) + { // 6 bytes + StoreUInt32BE((uint*) ptr, (uint) (value >> 16)); + StoreUInt16BE((ushort*) (ptr + 4), (ushort) value); + return ptr + 6; + } + + // 7 bytes + Contract.Assert(value <= 0xFFFFFFFFFFFFFF); + StoreUInt32BE((uint*) ptr, (uint) (value >> 24)); + StoreUInt16BE((ushort*) (ptr + 4), (ushort) (value >> 8)); + ptr[6] = (byte) value; + return ptr + 7; + } + } + + #endregion + + #region Compact Ordered Encoding... + + // Specialized encoding to store counters (integers) using as few bytes as possible, but with the ordering preserved when using lexicographical order, i.e: Encoded(-1) < Encoded(0) < Encoded(42) < Encoded(12345678) + // + // There are two variantes: Unsigned and Signed which encodes either positive values (ie: sizes, count, ...) or negatives/values (integers, deltas, coordinates, ...) + + #region Unsigned + + // The signed variant uses the 3 highest bits to encode the number of extra bytes needed to store the value. + // - The 5 lowest bits of the start byte are the 5 highest bits of the encoded value + // - Each additional byte stores the next 8 bits until the last byte that stores the lowest 8 bits. + // - To prevent multiple ways of encoding the same value (ex: 0 can be stored as '00' or '20 00' or '04 00 00'), and preserve the ordering guarantees, only the smallest form is legal + // - Only values between 0 and 2^61 -1 can be encoded that way! (values >= 2^60 are NOT SUPPORTED). + // - 4 bytes can encode up to 2^29-1 (~ sizes up to 512 MB), 8 bytes up to 2^61-1 (~ sizes up to 2 Exabytes) + // + // WIRE FORMAT: BBBNNNNN (NNNNNNNN ...) + // + // MIN MAX SIZE WIRE FORMAT = VALUE + // 0 31 1 byte 000AAAAA = b_AAAAA (5 bits) + // 32 (1<<13)-1 2 bytes 001AAAAA BBBBBBBB = b_AAAAA_BBBBBBBB (13 bits) + // (1<<13) (1<<21)-1 3 bytes 010AAAAA BBBBBBBB CCCCCCCC = b_AAAAA_BBBBBBBB_CCCCCCCC (21 bits) + // ... + // (1<<53) (1<<61)-1 8 bytes 111AAAAA BBBBBBBB CCCCCCCC DDDDDDDD EEEEEEEE FFFFFFFF GGGGGGGG = b_AAAAA_BBBBBBBB_CCCCCCCC_DDDDDDDD_EEEEEEEE_FFFFFFFF_GGGGGGGG (61 bits) + // + // Examples: + // - 0 => b_000_00000 => (1) '00' + // - 1 => b_000_00001 => (1) '01' + // - 31 => b_000_11111 => (1) '1F' + // - 32 => b_001_00000_00100000 => (2) '20 20' + // - 123 => b_001_00000_01111011 => (2) '20 7B' + // - 1234 => b_001_00100_11010010 => (2) '24 D2' + // - 12345 => b_010_00000_00110000_00111001 => (3) '40 30 39' + // - 2^16-1 => b_010_00000_11111111_11111111 => (3) '40 FF FF' + // - 2^16 => b_010_00001_00000000_00000000 => (3) '41 00 00' + // - 2^21-1 => b_010_11111_11111111_11111111 => (3) '5F FF FF' + // - 2^21 => b_011_00000_00100000_00000000_00000000 => (4) '60 20 00 00' + // - 2^29-1 => b_011_11111_11111111_11111111_11111111 => (4) '7F FF FF FF' + // - 2^29 => b_100_00000_00100000_00000000_00000000_00000000 => (5) '80 20 00 00 00' + // - 2^31-1 => b_100_00000_01111111_11111111_11111111_11111111 => (5) '80 7F FF FF FF' + // - 2^32-1 => b_100_00000_11111111_11111111_11111111_11111111 => (5) '80 FF FF FF FF' + // - 2^32 => b_100_00001_00000000_00000000_00000000_00000000 => (5) '81 00 00 00 00' + // - 2^61-1 => b_111_11111_11111111_11111111_11111111_11111111_11111111_11111111_11111111 => (8) 'FF FF FF FF FF FF FF FF' + + private const int OCU_LEN0 = 0 << 5; + private const int OCU_LEN1 = 1 << 5; + private const int OCU_LEN2 = 2 << 5; + private const int OCU_LEN3 = 3 << 5; + private const int OCU_LEN4 = 4 << 5; + private const int OCU_LEN5 = 5 << 5; + private const int OCU_LEN6 = 6 << 5; + private const int OCU_LEN7 = 7 << 5; + private const int OCU_BITMAK = (1 << 5) - 1; + private const uint OCU_MAX0 = (1U << 5) - 1; + private const uint OCU_MAX1 = (1U << (5 + 8)) - 1; + private const uint OCU_MAX2 = (1U << (5 + 8 * 2)) - 1; + private const uint OCU_MAX3 = (1U << (5 + 8 * 3)) - 1; + private const ulong OCU_MAX4 = (1UL << (5 + 8 * 4)) - 1; + private const ulong OCU_MAX5 = (1UL << (5 + 8 * 5)) - 1; + private const ulong OCU_MAX6 = (1UL << (5 + 8 * 6)) - 1; + private const ulong OCU_MAX7 = (1UL << (5 + 8 * 7)) - 1; + + + /// Return the size (in bytes) that a 32-bit counter value would need with the Compact Order Unsigned encoding + /// Number that needs to be encoded + /// Number of bytes needed (1-5) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int SizeOfOrderedUInt32(uint value) + { + return value <= OCU_MAX0 ? 1 + : value <= OCU_MAX1 ? 2 + : value <= OCU_MAX2 ? 3 + : value <= OCU_MAX3 ? 4 + : 5; + } + + /// Return the size (in bytes) that a 64-bit counter value would need with the Compact Order Unsigned encoding + /// Number that needs to be encoded, between 0 and 2^60-1 + /// Number of bytes needed (1-8), or 0 if the number would overflow (2^60 or greater) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int SizeOfOrderedUInt64(ulong value) + { + return value <= OCU_MAX0 ? 1 + : value <= OCU_MAX1 ? 2 + : value <= OCU_MAX2 ? 3 + : value <= OCU_MAX3 ? 4 + : value <= OCU_MAX4 ? 5 + : value <= OCU_MAX5 ? 6 + : value <= OCU_MAX6 ? 7 + : value <= OCU_MAX7 ? 8 + : 0; // this would throw! + } + + /// Append an unsigned 32-bit counter value using a compact ordered encoding + /// Pointer to the next free byte in the buffer + /// Positive counter value + /// Pointer updated with the number of bytes written + /// Will write between 1 and 5 bytes + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteOrderedUInt32Unsafe([NotNull] byte* cursor, uint value) + { + if (value <= OCU_MAX0) + { // < 32 + *cursor = (byte) (OCU_LEN0 | value); + return cursor + 1; + } + if (value <= OCU_MAX1) + { // < 8 KB + cursor[0] = (byte) (OCU_LEN1 | (value >> 8)); + cursor[1] = (byte) (value); + return cursor + 2; + } + return WriteOrderedUInt32UnsafeSlow(cursor, value); + } + + [MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteOrderedUInt32UnsafeSlow([NotNull] byte* cursor, uint value) + { + if (value <= OCU_MAX2) + { // < 2 MB + cursor[0] = (byte)(OCU_LEN2 | (value >> 16)); + cursor[1] = (byte)(value >> 8); + cursor[2] = (byte)(value); + return cursor + 3; + } + if (value <= OCU_MAX3) + { // < 512 MB + cursor[0] = (byte)(OCU_LEN3 | (value >> 24)); + cursor[1] = (byte)(value >> 16); + cursor[2] = (byte)(value >> 8); + cursor[3] = (byte)(value); + return cursor + 4; + } + cursor[0] = OCU_LEN4; // we waste a byte for values >= 512MB, which is unfortunate... + cursor[1] = (byte)(value >> 24); + cursor[2] = (byte)(value >> 16); + cursor[3] = (byte)(value >> 8); + cursor[4] = (byte)(value); + return cursor + 5; + } + + /// Append an unsigned 64-bit counter value (up to 2^61-1) using the Compact Ordered Unsigned encoding + /// Pointer to the next free byte in the buffer + /// Positive counter value that must be between 0 and 2^61 - 1 (2,305,843,009,213,693,951 or 0x1FFFFFFFFFFFFFFF) + /// Pointer updated with the number of bytes written + /// Will write between 1 and 8 bytes + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static byte* WriteOrderedUInt64Unsafe([NotNull] byte* cursor, ulong value) + { + return value <= uint.MaxValue ? WriteOrderedUInt32Unsafe(cursor, (uint) value) : WriteOrderedUInt64UnsafeSlow(cursor, value); + } + + [MethodImpl(MethodImplOptions.NoInlining)] + private static byte* WriteOrderedUInt64UnsafeSlow([NotNull] byte* cursor, ulong value) + { + if (value <= OCU_MAX4) + { + cursor[0] = (byte)(OCU_LEN4 | (value >> 32)); + cursor[1] = (byte)(value >> 24); + cursor[2] = (byte)(value >> 16); + cursor[3] = (byte)(value >> 8); + cursor[4] = (byte)(value); + return cursor + 5; + } + if (value <= OCU_MAX5) + { + cursor[0] = (byte)(OCU_LEN5 | (value >> 40)); + cursor[1] = (byte)(value >> 32); + cursor[2] = (byte)(value >> 24); + cursor[3] = (byte)(value >> 16); + cursor[4] = (byte)(value >> 8); + cursor[5] = (byte)(value); + return cursor + 6; + } + if (value <= OCU_MAX6) + { + cursor[0] = (byte)(OCU_LEN6 | (value >> 48)); + cursor[1] = (byte)(value >> 40); + cursor[2] = (byte)(value >> 32); + cursor[3] = (byte)(value >> 24); + cursor[4] = (byte)(value >> 16); + cursor[5] = (byte)(value >> 8); + cursor[6] = (byte)(value); + return cursor + 7; + } + + if (value <= OCU_MAX7) + { + cursor[0] = (byte) (OCU_LEN7 | (value >> 56)); + cursor[1] = (byte) (value >> 48); + cursor[2] = (byte) (value >> 40); + cursor[3] = (byte) (value >> 32); + cursor[4] = (byte) (value >> 24); + cursor[5] = (byte) (value >> 16); + cursor[6] = (byte) (value >> 8); + cursor[7] = (byte) (value); + return cursor + 8; + } + + throw new ArgumentOutOfRangeException(nameof(value), value, "Value must be less then 2^60"); + } + + /// Read an unsigned 32-bit counter value encoded using the Compact Ordered Unsigned encoding + /// + /// + /// + public static byte* ReadOrderedUInt32Unsafe(byte* cursor, out uint value) + { + uint start = cursor[0]; + switch (start >> 5) + { + case 0: + value = (start & OCU_BITMAK); + return cursor + 1; + case 1: + value = ((start & OCU_BITMAK) << 8) | ((uint) cursor[1]); + return cursor + 2; + case 2: + value = ((start & OCU_BITMAK) << 16) | ((uint) cursor[1] << 8) | ((uint) cursor[2]); + return cursor + 3; + case 3: + value = ((start & OCU_BITMAK) << 24) | ((uint)cursor[1] << 16) | ((uint)cursor[2] << 8) | (uint)cursor[3]; + return cursor + 4; + case 4: + // start bits MUST be 0 (else, there is an overflow) + if ((start & OCU_BITMAK) != 0) throw new InvalidDataException(); //TODO: message? + value = ((uint)cursor[1] << 24) | ((uint)cursor[2] << 16) | ((uint)cursor[3] << 8) | (uint)cursor[4]; + return cursor + 5; + default: + // overflow? + throw new InvalidDataException(); //TODO: message? + } + } + + /// Read an unsigned 64-bit counter value encoded using the Compact Ordered Unsigned encoding + /// + /// + /// + public static byte* ReadOrderedUInt64Unsafe(byte* cursor, out ulong value) + { + ulong start = cursor[0]; + switch (start >> 5) + { + case 0: + value = (start & OCU_BITMAK); + return cursor + 1; + case 1: + value = ((start & OCU_BITMAK) << 8) | ((ulong)cursor[1]); + return cursor + 2; + case 2: + value = ((start & OCU_BITMAK) << 16) | ((ulong)cursor[1] << 8) | ((ulong)cursor[2]); + return cursor + 3; + case 3: + value = ((start & OCU_BITMAK) << 24) | ((ulong)cursor[1] << 16) | ((ulong)cursor[2] << 8) | ((ulong)cursor[3]); + return cursor + 4; + case 4: + value = ((start & OCU_BITMAK) << 32) | ((ulong)cursor[1] << 24) | ((ulong)cursor[2] << 16) | ((ulong)cursor[3] << 8) | ((ulong)cursor[4]); + return cursor + 5; + case 5: + value = ((start & OCU_BITMAK) << 40) | ((ulong)cursor[1] << 32) | ((ulong)cursor[2] << 24) | ((ulong)cursor[3] << 16) | ((ulong)cursor[4] << 8) | ((ulong)cursor[5]); + return cursor + 6; + case 6: + value = ((start & OCU_BITMAK) << 48) | ((ulong)cursor[1] << 40) | ((ulong)cursor[2] << 32) | ((ulong)cursor[3] << 24) | ((ulong)cursor[4] << 16) | ((ulong)cursor[5] << 8) | ((ulong)cursor[6]); + return cursor + 7; + default: // 7 + value = ((start & OCU_BITMAK) << 56) | ((ulong)cursor[1] << 48) | ((ulong)cursor[2] << 40) | ((ulong)cursor[3] << 32) | ((ulong)cursor[4] << 24) | ((ulong)cursor[5] << 16) | ((ulong)cursor[6] << 8) | ((ulong)cursor[7]); + return cursor + 8; + } + } + + #endregion + + #region Signed + + // The signed variant is very similar, except that the start byte uses an additional "Sign" bit (inverted) + // - The hight bit (bit 7) of the start byte is 0 for negative numbers, and 1 for positive numbers + // - The next 3 bits (bits 6-4) of the start byte encode the number of extra bytes following + // - The last 4 bits (bit 3-0) contain the 4 highest bits of the encoded value + // - Each additional byte stores the next 8 bits until the last byte that stores the lowest 8 bits. + // - For negative values, the number of bytes required is computed by using Abs(X)-1, but the original negative value is used (after masking) + // i.e.: -1 becomes -(-1)-1 = 0 (which fits in 4 bits), and will be encoded as (-1) & 0xF = b_0_000_1111 = '0F', and 0 will be encoded as b_1_000_0000 = '10' (which is indeeded sorted after '0F') + // - Only values between -2^60 and 2^60-1 can be encoded that way! (values < -2^60 or >= 2^60 are NOT SUPPORTED) + + // WIRE FORMAT: SBBBNNNN (NNNNNNNN ...) + // - if S = 0, X is negative: BBB = 7 - exta bytes, NNN...N = 2's complement of X + // - if S = 1, X is positive: BBB = exta bytes, NNN...N = X + // + // MIN MAX SIZE WIRE FORMAT = VALUE + // -(1<<60) -(1<<52)-1 8 bytes 1111AAAA BBBBBBBB CCCCCCCC DDDDDDDD EEEEEEEE FFFFFFFF GGGGGGGG = b_AAAA_BBBBBBBB_CCCCCCCC_DDDDDDDD_EEEEEEEE_FFFFFFFF_GGGGGGGG (60 bits) + // ... + // -(1<<12) -17 2 bytes 1001AAAA BBBBBBBB = ~(b_AAAA_BBBBBBBB - 1) (12 bits) + // -16 -1 1 byte 0000AAAA = ~(b_AAAA - 1) (4 bits) + // 0 +15 1 byte 1000AAAA = b_AAAA (4 bits) + // +16 (1<<12)-1 2 bytes 1001AAAA BBBBBBBB = b_AAAA_BBBBBBBB (12 bits) + // ... + // (1<<52) (1<<60)-1 8 bytes 1111AAAA BBBBBBBB CCCCCCCC DDDDDDDD EEEEEEEE FFFFFFFF GGGGGGGG = b_AAAA_BBBBBBBB_CCCCCCCC_DDDDDDDD_EEEEEEEE_FFFFFFFF_GGGGGGGG (60 bits) + // + // Examples: + // - 0 => b_1_000_0000 => (1) '80' + // - 1 => b_1_000_0001 => (1) '81' + // - 15 => b_1_000_1111 => (1) '8F' + // - 16 => b_1_001_0000_00010000 => (2) '90 10' + // - 123 => b_1_001_0000_01111011 => (2) '90 7B' + // - 1234 => b_1_001_0100_11010010 => (2) '94 D2' + // - 12345 => b_1_010_0000_00110000_00111001 => (3) 'A0 30 39' + // - 2^16-1 => b_1_010_0001_00000000_00000000 => (3) 'A1 00 00' + // - 2^20-1 => b_1_010_1111_11111111_11111111 => (3) 'AF FF FF' + // - 2^21 => b_1_011_0000_00100000_00000000_00000000 => (4) 'B0 20 00 00' + // - 2^28-1 => b_1_011_1111_11111111_11111111_11111111 => (4) 'BF FF FF FF' + // - 2^32-1 => b_1_100_0000_11111111_11111111_11111111_11111111 => (4) 'C0 FF FF FF FF' + // - 2^32 => b_1_100_0001_00000000_00000000_00000000_00000000 => (4) 'C1 00 00 00 00' + // - 2^60-1 => b_1_111_1111_11111111_11111111_11111111_11111111_11111111_11111111_11111111 => (8) 'FF FF FF FF FF FF FF FF' + + //TODO! + + #endregion + + #endregion + + /// Convert a char in range '0-9A-Fa-f' into a value between 0 and 15 + /// Result is unspecified if char is not in the valid range! + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static int Nibble(char c) + { + // The lowest 4 bits almost give us the result we want: + // - '0'..'9': (c & 15) = 0..9; need to add 0 to get correct result + // - 'A'..'F': (c & 15) = 1..6; need to add 9 to get correct reuslt + // - 'a'..'f': (c & 15) = 1..6; need to add 9 to get correct reuslt + // We just need to tweak the value to have a bit that is different between digits and letters, and use that bit to compute the final offset of 0 or 9 + return (c & 15) + (((((c + 16) & ~64) >> 4) & 1) * 9); + } + + /// Convert values between 0 and 15 into a character from in range '0-9A-F' + /// Only the lower 4 bits are used, so the caller does not need to mask out the upper bits! + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static char Nibble(int x) + { + // We first tweak the value in order to have a bit that is different between 0-9 and 10-15. + // Then, we use that bit to compute the final offset that will end up adding +48 or +55 + // 0-9 : X + 54 + 1 - (1 x 7) = X + 48 = '0'-'9' + // 10-15 : X + 54 + 1 - (0 x 7) = X + 55 = 'A'-'F' + int tmp = ((x & 0xF) + 54); + return (char)(tmp + 1 - ((tmp & 32) >> 5) * 7); + //REVIEW: '* 7' could probably be replaced with some shift/add trickery... (but maybe the JIT will do it for us?) + } + + #region String Helpers... + + /// Check if a string only contains characters between 0 and 127 (ASCII) + [Pure] + public static bool IsAsciiString([NotNull] string value) + { + Contract.Requires(value != null); + fixed (char* pChars = value) + { + return IsAsciiString(pChars, value.Length); + } + } + + /// Check if a section of a string only contains characters between 0 and 127 (ASCII) + [Pure] + public static bool IsAsciiString([NotNull] string value, int offset, int count) + { + Contract.Requires(value != null && offset >= 0 && count <= 0 && offset + count <= value.Length); + if (count == 0) return true; + fixed (char* pChars = value) + { + return IsAsciiString(pChars + offset, count); + } + } + +#if ENABLE_SPAN + /// Check if a section of a string only contains characters between 0 and 127 (ASCII) + [Pure] + public static bool IsAsciiString(ReadOnlySpan value) + { + if (value.Length == 0) return true; + fixed (char* pChars = &MemoryMarshal.GetReference(value)) + { + return IsAsciiString(pChars, value.Length); + } + } +#endif + + /// Check if a string only contains characters between 0 and 127 (ASCII) + [Pure] + public static bool IsAsciiString([NotNull] char* pChars, int numChars) + { + Contract.Requires(pChars != null); + // we test if each char has at least one bit set above bit 7, ie: (char & 0xFF80) != 0 + // to speed things up, we check multiple chars at a time + + #region Performance Notes... + /* + The following loop is optimized to produce the best x64 code with Deskop CLR RyuJitJIT (x64) that is currently in 4.6.2 (preview) + => if the JIT changes, we may need to revisit! + + Currently, the x64 code generated for the main unrolled loop looks like this: + + MAIN_LOOP: + // rax = ptr + // rcx = end + (01) cmp rax,rcx // while (ptr < end) + (02) jae TAIL // => bypass for small strings <= 7 chars + + LOOP: + (03) mov r8,qword ptr [rax] // ulong x1 = *(ulong*) (ptr + 0); + (04) mov r9,qword ptr [rax+8] // ulong x2 = *(ulong*) (ptr + 8); + (05) mov r10,qword ptr [rax+10h] // ulong x3 = *(ulong*) (ptr + 8); + (06) mov r11,qword ptr [rax+18h] // ulong x4 = *(ulong*) (ptr + 12); + (07) mov rsi,0FF80FF80FF80FF80h + (08) and r8,rsi // x1 &= MASK4; + (09) and r9,rsi // x2 &= MASK4; + (10) and r10,rsi // x3 &= MASK4; + (11) and r11,rsi // x4 &= MASK4; + (12) add rax,20h // ptr += 16; + (13) or r8,r9 // (x1 != 0 || x2 != 0) + (14) mov r9,r10 + (15) or r9,r11 // (x3 != 0 || x4 != 0) + (16) or r8,r9 // (...) || (...) + (17) test r8,r8 // if (...) ... + (18) jne INVALID // ... goto INVALID; + (19) cmp rax,rcx // while (ptr < end) + (20) jb LOOP // ... (continue) + + TAIL: + // continue for size <= 7 + + Commentary: + - At 3 to 6 we parallelize the reads from memory into 4 register + - At 8 to 11 we perform the ANDs again in a way that can be //ized by the CPU + - At 12, we pre-increment the pointer, so that the value is ready at 19 + - At 13 to 16, the whole if expression is optimized into a 3 or in cascade. + - note: doing "(... || ...) || (... || ...)" is ~5% faster than "(... || ... || ... || ...)" on my CPU + - At 18, we jump to the "INVALID" case, instead of doing "return false", because current JIT produce better code that way + - note: if we "return false" here, the JIT adds an additional JMP inside the loop, which if ~15% slower on my CPU + */ + #endregion + + const ulong MASK_4_CHARS = 0xFF80FF80FF80FF80UL; + const uint MASK_2_CHARS = 0xFF80FF80U; + const ushort MASK_1_CHAR = 0xFF80; + + char* ptr = pChars; + char* end = ptr + (numChars & ~15); + while (ptr < end) + { + ulong x1 = *(ulong*) (ptr + 0); + ulong x2 = *(ulong*) (ptr + 4); + ulong x3 = *(ulong*) (ptr + 8); + ulong x4 = *(ulong*) (ptr + 12); + // combine all the bits together in stages + x1 |= x2; + x3 |= x4; + x1 |= x3; + // drop the LS 7 bits + x1 &= MASK_4_CHARS; + ptr += 16; + if (x1 != 0) goto INVALID; + } + + if ((numChars & 8) != 0) + { + ulong x1 = *(ulong*) (ptr + 0); + ulong x2 = *(ulong*) (ptr + 4); + x1 = x1 | x2; + x1 &= MASK_4_CHARS; + ptr += 8; + if (x1 != 0) goto INVALID; + } + + if ((numChars & 4) != 0) + { + ulong x1 = *(ulong*) ptr & MASK_4_CHARS; + if (x1 != 0) goto INVALID; + ptr += 4; + } + if ((numChars & 2) != 0) + { + uint x1 = *(uint*) ptr & MASK_2_CHARS; + if (x1 != 0) goto INVALID; + ptr += 2; + } + // check the last character, if present + return (numChars & 1) == 0 || (*ptr & MASK_1_CHAR) == 0; + + INVALID: + // there is one character that is >= 0x80 in the string + return false; + } + + /// Check if a section of byte array only contains bytes between 0 and 127 (7-bit ASCII) + /// False if at least one byte has bit 7 set to 1; otherwise, True. + [Pure] + public static bool IsAsciiBytes([NotNull] byte[] array, int offset, int count) + { + Contract.Requires(array != null); + fixed (byte* pBytes = &array[offset]) + { + return IsAsciiBytes(pBytes, checked((uint) count)); + } + } + + /// Check if a memory region only contains bytes between 0 and 127 (7-bit ASCII) + /// False if at least one byte has bit 7 set to 1; otherwise, True. + [Pure] + public static bool IsAsciiBytes([NotNull] byte* buffer, uint count) + { + Contract.Requires(buffer != null); + + // we test if each byte has at least one bit set above bit 7, ie: (byte & 0x80) != 0 + // to speed things up, we check multiple bytes at a time + + const ulong MASK_8 = 0x8080808080808080UL; + const uint MASK_4 = 0x80808080U; + const int MASK_2 = 0x8080; + const int MASK_1 = 0x80; + + byte* end = buffer + (count & ~31); + byte* ptr = buffer; + while (ptr < end) + { + ulong x1 = *((ulong*) ptr + 0); + ulong x2 = *((ulong*) ptr + 1); + ulong x3 = *((ulong*) ptr + 2); + ulong x4 = *((ulong*) ptr + 3); + x1 |= x2; + x3 |= x4; + x1 |= x3; + x1 &= MASK_8; + ptr += 32; + if (x1 != 0) goto INVALID; + } + + if ((count & 16) != 0) + { + ulong x1 = *((ulong*) ptr + 0); + ulong x2 = *((ulong*) ptr + 1); + x1 |= x2; + x1 &= MASK_8; + ptr += 16; + if (x1 != 0) goto INVALID; + } + if ((count & 8) != 0) + { + if ((*((ulong*) ptr) & MASK_8) != 0) goto INVALID; + ptr += 8; + } + if ((count & 4) != 0) + { + if ((*((uint*) ptr) & MASK_4) != 0) goto INVALID; + ptr += 4; + } + if ((count & 2) != 0) + { + if ((*((ushort*) ptr) & MASK_2) != 0) goto INVALID; + ptr += 2; + } + if ((count & 1) != 0) + { + return *ptr < MASK_1; + } + // there is one character that is >= 0x80 in the string + return true; + INVALID: + return false; + } + + /// Convert a byte stream into a .NET string by expanding each byte to 16 bits characters + /// Equivalent .NET string + /// + /// This is safe to use with 7-bit ASCII strings. + /// You should *NOT* use this if the buffer contains ANSI or UTF-8 encoded strings! + /// If the bufer contains bytes that are >= 0x80, they will be mapped to the equivalent Unicode code points (0x80..0xFF), WITHOUT converting them using current ANSI code page. + /// + /// + /// ConvertToByteString(new byte[] { 'A', 'B', 'C' }, 0, 3) => "ABC" + /// ConvertToByteString(new byte[] { 255, 'A', 'B', 'C' }, 0, 4) => "\xffABC" + /// ConvertToByteString(UTF8("é"), ...) => "é" (len=2, 'C3 A9') + /// + [Pure, NotNull] + public static string ConvertToByteString([NotNull] byte[] array, int offset, int count) + { + Contract.Requires(array != null && offset >= 0 && count >= 0 && offset + count <= array.Length); + + // fast allocate a new empty string that will be mutated in-place. + //note: this calls String::CtorCharCount() which in turn calls FastAllocateString(..), but will not fill the buffer with 0s if 'char' == '\0' + string str = new string('\0', count); + + fixed (byte* ptr = &array[offset]) + fixed (char* pChars = str) + { + ConvertToByteStringUnsafe(pChars, ptr, (uint) count); + return str; + } + } + + /// Convert a byte stream into a .NET string by expanding each byte to 16 bits characters + /// Equivalent .NET string + /// + /// This is safe to use with 7-bit ASCII strings. + /// You should *NOT* use this if the buffer contains ANSI or UTF-8 encoded strings! + /// If the bufer contains bytes that are >= 0x80, they will be mapped to the equivalent Unicode code points (0x80..0xFF), WITHOUT converting them using current ANSI code page. + /// + [Pure, NotNull] + public static string ConvertToByteString(byte* pBytes, uint count) + { + Contract.Requires(pBytes != null); + + if (count == 0) return String.Empty; + + // fast allocate a new empty string that will be mutated in-place. + //note: this calls String::CtorCharCount() which in turn calls FastAllocateString(..), but will not fill the buffer with 0s if 'char' == '\0' + string str = new string('\0', checked((int) count)); + fixed (char* pChars = str) + { + ConvertToByteStringUnsafe(pChars, pBytes, count); + return str; + } + } + + internal static void ConvertToByteStringUnsafe(char* pChars, byte* pBytes, uint count) + { + byte* inp = pBytes; + char* outp = pChars; + + // unroll 4 characters at a time + byte* inend = pBytes + (count & ~3); + while (inp < inend) + { + //this loop has been verified to produce the best x64 code I could get out from the DesktopCLR JIT (4.6.x) + long x = *(long*) inp; + // split + long y1 = x & 0xFF; + long y2 = x & 0xFF00; + long y3 = x & 0xFF0000; + long y4 = x & 0xFF000000; + // shift + y2 <<= 8; + y3 <<= 16; + y4 <<= 24; + // merge + y1 |= y2; + y3 |= y4; + y1 |= y3; + // output + *(long*) outp = y1; + inp += 4; + outp += 4; + } + // complete the tail + + if ((count & 2) != 0) + { // two chars + int x = *(ushort*) inp; + // split + int y1 = x & 0xFF; + int y2 = x & 0xFF00; + // shift + y2 <<= 8; + // merge + y2 |= y1; + // output + *(int*) outp = y2; + inp += 2; + outp += 2; + } + + if ((count & 1) != 0) + { // one char + *outp = (char) *inp; + } + } + #endregion + + [SuppressUnmanagedCodeSecurity] + [SecurityCritical] + internal static class NativeMethods + { + // C/C++ .NET + // --------------------------------- + // void* byte* (or IntPtr) + // size_t UIntPtr (or IntPtr) + // int int + // char byte + + /// Compare characters in two buffers. + /// First buffer. + /// Second buffer. + /// Number of bytes to compare. + /// The return value indicates the relationship between the buffers. + [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + public static extern int memcmp(byte* buf1, byte* buf2, UIntPtr count); + + /// Moves one buffer to another. + /// Destination object. + /// Source object. + /// Number of bytes to copy. + /// The value of dest. + /// Copies count bytes from src to dest. If some regions of the source area and the destination overlap, both functions ensure that the original source bytes in the overlapping region are copied before being overwritten. + [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + public static extern byte* memmove(byte* dest, byte* src, UIntPtr count); + + /// Sets buffers to a specified character. + /// Pointer to destination + /// Character to set + /// Number of characters + /// memset returns the value of dest. + /// The memset function sets the first count bytes of dest to the character c. + [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] + public static extern byte* memset(byte* dest, int ch, UIntPtr count); + + } + + [DebuggerNonUserCode] + internal static class Errors + { + + /// Reject an invalid slice by throw an error with the appropriate diagnostic message. + /// If the corresponding slice is invalid (offset or count out of bounds, array is null, ...) + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception MalformedBuffer(byte* bytes, long count) + { + if (count < 0) return BufferCountNotNeg(); + if (count > 0) + { + if (bytes == null) return BufferArrayNotNull(); + } + // maybe it's Lupus ? + return BufferInvalid(); + } + + /// Reject an invalid slice by throw an error with the appropriate diagnostic message. + /// If the corresponding slice is invalid (offset or count out of bounds, array is null, ...) + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception MalformedBuffer(byte[] array, long offset, long count) + { + if (offset < 0) return BufferOffsetNotNeg(); + if (count < 0) return BufferCountNotNeg(); + if (count > 0) + { + if (array == null) return BufferArrayNotNull(); + if (offset + count > array.Length) return BufferArrayToSmall(); + } + // maybe it's Lupus ? + return BufferInvalid(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static OverflowException PowerOfTwoOverflow() + { + return new OverflowException("Cannot compute the next power of two because the value would overflow."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static OverflowException PowerOfTwoNegative() + { + return new OverflowException("Cannot compute the next power of two for negative numbers."); + } + + /// Reject an attempt to write past the end of a buffer + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static InvalidOperationException BufferOutOfBound() + { + return new InvalidOperationException("Attempt to write outside of the buffer, or at a position that would overflow past the end."); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ThrowOffsetOutsideSlice() + { + throw OffsetOutsideSlice(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static Exception OffsetOutsideSlice() + { + // ReSharper disable once NotResolvedInText + return ThrowHelper.ArgumentOutOfRangeException("offset", "Offset is outside the bounds of the slice."); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ThrowIndexOutOfBound(int index) + { + throw IndexOutOfBound(index); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static IndexOutOfRangeException IndexOutOfBound(int index) + { + return new IndexOutOfRangeException("Index is outside the slice"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException SliceOffsetNotNeg() + { + return new FormatException("The specified slice has a negative offset, which is not legal. This may be a side effect of memory corruption."); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ThrowSliceCountNotNeg() + { + throw SliceCountNotNeg(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException SliceCountNotNeg() + { + return new FormatException("The specified slice has a negative size, which is not legal. This may be a side effect of memory corruption."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException SliceBufferNotNull() + { + return new FormatException("The specified slice is missing its underlying buffer."); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ThrowSliceBufferTooSmall() + { + throw SliceBufferTooSmall(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException SliceBufferTooSmall() + { + return new FormatException("The specified slice is larger than its underlying buffer."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException SliceInvalid() + { + return new FormatException("The specified slice is invalid."); + } + + [ContractAnnotation("=>halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowSliceTooLargeForConversion(int size) + { + throw new FormatException($"Cannot convert slice to value of type {typeof(T).Name} because it is larger than {size} bytes."); + } + + [ContractAnnotation("=>halt"), MethodImpl(MethodImplOptions.NoInlining)] + public static T ThrowSliceSizeInvalidForConversion(int size) + { + throw new FormatException($"Cannot convert slice of size {size} to value of type {typeof(T).Name}."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException BufferOffsetNotNeg() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("The specified segment has a negative offset, which is not legal. This may be a side effect of memory corruption.", "offset"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException BufferCountNotNeg() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("The specified segment has a negative size, which is not legal. This may be a side effect of memory corruption.", "count"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException BufferArrayNotNull() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("The specified segment is missing its underlying buffer.", "array"); + } + + [ContractAnnotation("=> halt"), MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void ThrowBufferArrayToSmall() + { + throw BufferArrayToSmall(); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException BufferArrayToSmall() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("The specified segment is larger than its underlying buffer.", "count"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static ArgumentException BufferInvalid() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("The specified segment is invalid."); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException VarIntOverflow() + { + return new FormatException("Malformed Varint would overflow the expected range"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException VarIntTruncated() + { + return new FormatException("Malformed Varint seems to be truncated"); + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + public static FormatException VarBytesTruncated() + { + return new FormatException("Malformed VarBytes seems to be truncated"); + } + + } + } + +} diff --git a/FoundationDB.Client/Utils/Slice.cs b/FoundationDB.Client/Utils/Slice.cs deleted file mode 100644 index 0fadf8990..000000000 --- a/FoundationDB.Client/Utils/Slice.cs +++ /dev/null @@ -1,2508 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB -{ - using JetBrains.Annotations; - using System; - using System.Collections.Generic; - using System.ComponentModel; - using System.Diagnostics; - using System.Globalization; - using System.IO; - using System.Linq; - using System.Text; - using System.Threading; - using System.Threading.Tasks; - using Doxense.Diagnostics.Contracts; - - /// Delimits a section of a byte array - [ImmutableObject(true), PublicAPI, DebuggerDisplay("Count={Count}, Offset={Offset}"), DebuggerTypeProxy(typeof(Slice.DebugView))] - public struct Slice : IEquatable, IEquatable>, IEquatable, IComparable - { - #region Static Members... - - /// Cached empty array of bytes - internal static readonly byte[] EmptyArray = new byte[0]; - - /// Cached empty array of slices - internal static readonly Slice[] EmptySliceArray = new Slice[0]; - - /// Cached array of bytes from 0 to 255 - internal static readonly byte[] ByteSprite; - - /// Null slice ("no segment") - public static readonly Slice Nil = default(Slice); - - /// Empty slice ("segment of 0 bytes") - public static readonly Slice Empty = new Slice(EmptyArray, 0, 0); - - [System.Diagnostics.CodeAnalysis.SuppressMessage("Microsoft.Usage", "CA2207:InitializeValueTypeStaticFieldsInline")] - static Slice() - { - var tmp = new byte[256]; - for (int i = 0; i < tmp.Length; i++) tmp[i] = (byte)i; - ByteSprite = tmp; - } - - #endregion - - /// Pointer to the buffer (or null for ) - public readonly byte[] Array; - - /// Offset of the first byte of the slice in the parent buffer - public readonly int Offset; - - /// Number of bytes in the slice - public readonly int Count; - - internal Slice(byte[] array, int offset, int count) - { - Contract.Requires(array != null && offset >= 0 && offset <= array.Length && count >= 0 && offset + count <= array.Length); - this.Array = array; - this.Offset = offset; - this.Count = count; - } - - /// Creates a slice mapping an entire buffer - /// - /// - public static Slice Create(byte[] bytes) - { - return - bytes == null ? Slice.Nil : - bytes.Length == 0 ? Slice.Empty : - new Slice(bytes, 0, bytes.Length); - } - - /// Creates a slice from an Array Segment - /// Segment of buffer to convert - public static Slice Create(ArraySegment arraySegment) - { - return Create(arraySegment.Array, arraySegment.Offset, arraySegment.Count); - } - - /// Creates a slice mapping a section of a buffer - /// Original buffer - /// Offset into buffer - /// Number of bytes - public static Slice Create(byte[] buffer, int offset, int count) - { - SliceHelpers.EnsureBufferIsValid(buffer, offset, count); - if (count == 0) - { - if (offset != 0) throw new ArgumentException("offset"); - return buffer == null ? Nil : Empty; - } - return new Slice(buffer, offset, count); - } - - /// Creates a slice mapping a section of a buffer, either directly or by making a copy. - /// Original buffer - /// Offset into buffer - /// Number of bytes - /// If true, creates a copy of the buffer. If false, maps directly into the buffer. - /// If is false, any change made to will also be visible in this slice. - public static Slice Create(byte[] buffer, int offset, int count, bool copy) - { - SliceHelpers.EnsureBufferIsValid(buffer, offset, count); - if (count == 0) - { - if (offset != 0) throw new ArgumentException("offset"); - return buffer == null ? Nil : Empty; - } - else if (copy) - { - var tmp = new byte[count]; - SliceHelpers.CopyBytesUnsafe(tmp, 0, buffer, offset, count); - return new Slice(tmp, 0, count); - } - else - { - return new Slice(buffer, offset, count); - } - } - - /// Create a new empty slice of a specified size containing all zeroes - /// - /// - public static Slice Create(int size) - { - if (size < 0) throw new ArgumentException("size"); - return size == 0 ? Slice.Empty : new Slice(new byte[size], 0, size); - } - - /// Creates a new slice with a copy of an unmanaged memory buffer - /// Pointer to unmanaged buffer - /// Number of bytes in the buffer - /// Slice with a managed copy of the data - internal static unsafe Slice Create(byte* ptr, int count) - { - if (count == 0) - { - return ptr == null ? Slice.Nil : Slice.Empty; - } - if (ptr == null) throw new ArgumentNullException(nameof(ptr)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count)); - - if (count == 1) - { - return Slice.FromByte(*ptr); - } - - var bytes = new byte[count]; - SliceHelpers.CopyBytesUnsafe(bytes, 0, ptr, count); - return new Slice(bytes, 0, count); - } - - /// Creates a new slice that contains the same byte repeated - /// Byte that will fill the slice - /// Number of bytes - /// New slice that contains times the byte . - public static Slice Repeat(byte value, int count) - { - if (count < 0) throw new ArgumentException("count"); - if (count == 0) return Slice.Empty; - - var res = new byte[count]; - SliceHelpers.SetBytes(res, 0, count, value); - return new Slice(res, 0, res.Length); - } - - /// Create a new slice filled with random bytes taken from a random number generator - /// Pseudo random generator to use (needs locking if instance is shared) - /// Number of random bytes to generate - /// Slice of bytes taken from - /// Warning: is not thread-safe ! If the instance is shared between threads, then it needs to be locked before calling this method. - public static Slice Random([NotNull] Random prng, int count) - { - if (prng == null) throw new ArgumentNullException(nameof(prng)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); - if (count == 0) return Slice.Empty; - - var bytes = new byte[count]; - prng.NextBytes(bytes); - return new Slice(bytes, 0, count); - } - - /// Create a new slice filled with random bytes taken from a cryptographic random number generator - /// Random generator to use (needs locking if instance is shared) - /// Number of random bytes to generate - /// If true, produce a sequence of non-zero bytes. - /// Slice of bytes taken from - /// Warning: All RNG implementations may not be thread-safe ! If the instance is shared between threads, then it may need to be locked before calling this method. - public static Slice Random([NotNull] System.Security.Cryptography.RandomNumberGenerator rng, int count, bool nonZeroBytes = false) - { - if (rng == null) throw new ArgumentNullException(nameof(rng)); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count cannot be negative"); - if (count == 0) return Slice.Empty; - - var bytes = new byte[count]; - - if (nonZeroBytes) - rng.GetNonZeroBytes(bytes); - else - rng.GetBytes(bytes); - - return new Slice(bytes, 0, count); - } - - /// Reports the zero-based index of the first occurrence of the specified slice in this source. - /// The slice Input slice - /// The slice to seek - /// - public static int Find(Slice source, Slice value) - { - const int NOT_FOUND = -1; - - SliceHelpers.EnsureSliceIsValid(ref source); - SliceHelpers.EnsureSliceIsValid(ref value); - - int m = value.Count; - if (m == 0) return 0; - - int n = source.Count; - if (n == 0) return NOT_FOUND; - - if (m == n) return source.Equals(value) ? 0 : NOT_FOUND; - if (m <= n) - { - byte[] src = source.Array; - int p = source.Offset; - byte firstByte = value[0]; - - // note: this is a very simplistic way to find a value, and is optimized for the case where the separator is only one byte (most common) - while (n-- > 0) - { - if (src[p++] == firstByte) - { // possible match ? - if (m == 1 || SliceHelpers.SameBytesUnsafe(src, p, value.Array, value.Offset + 1, m - 1)) - { - return p - source.Offset - 1; - } - } - } - } - - return NOT_FOUND; - } - - /// Concatenates all the elements of a slice array, using the specified separator between each element. - /// The slice to use as a separator. Can be empty. - /// An array that contains the elements to concatenate. - /// A slice that consists of the elements in a value delimited by the slice. If is an empty array, the method returns . - /// If is null. - public static Slice Join(Slice separator, [NotNull] Slice[] values) - { - if (values == null) throw new ArgumentNullException(nameof(values)); - - int count = values.Length; - if (count == 0) return Slice.Empty; - if (count == 1) return values[0]; - return Join(separator, values, 0, count); - } - - /// Concatenates the specified elements of a slice array, using the specified separator between each element. - /// The slice to use as a separator. Can be empty. - /// An array that contains the elements to concatenate. - /// The first element in to use. - /// The number of elements of to use. - /// A slice that consists of the slices in delimited by the slice. -or- if is zero, has no elements, or and all the elements of are . - /// If is null. - /// If or is less than zero. -or- plus is greater than the number of elements in . - public static Slice Join(Slice separator, [NotNull] Slice[] values, int startIndex, int count) - { - // Note: this method is modeled after String.Join() and should behave the same - // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) - - if (values == null) throw new ArgumentNullException(nameof(values)); - //REVIEW: support negative indexing ? - if (startIndex < 0) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); - if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); - - if (count == 0) return Slice.Empty; - if (count == 1) return values[startIndex]; - - int size = 0; - for (int i = 0; i < values.Length; i++) size += values[i].Count; - size += (values.Length - 1) * separator.Count; - - // if the size overflows, that means that the resulting buffer would need to be >= 2 GB, which is not possible! - if (size < 0) throw new OutOfMemoryException(); - - //note: we want to make sure the buffer of the writer will be the exact size (so that we can use the result as a byte[] without copying again) - var tmp = new byte[size]; - var writer = new SliceWriter(tmp); - for (int i = 0; i < values.Length; i++) - { - if (i > 0) writer.WriteBytes(separator); - writer.WriteBytes(values[i]); - } - Contract.Assert(writer.Buffer.Length == size); - return writer.ToSlice(); - } - - /// Concatenates the specified elements of a slice sequence, using the specified separator between each element. - /// The slice to use as a separator. Can be empty. - /// A sequence will return the elements to concatenate. - /// A slice that consists of the slices in delimited by the slice. -or- if has no elements, or and all the elements of are . - /// If is null. - public static Slice Join(Slice separator, [NotNull] IEnumerable values) - { - if (values == null) throw new ArgumentNullException(nameof(values)); - var array = (values as Slice[]) ?? values.ToArray(); - return Join(separator, array, 0, array.Length); - } - - /// Concatenates the specified elements of a slice array, using the specified separator between each element. - /// The slice to use as a separator. Can be empty. - /// An array that contains the elements to concatenate. - /// The first element in to use. - /// The number of elements of to use. - /// A byte array that consists of the slices in delimited by the slice. -or- an emtpy array if is zero, has no elements, or and all the elements of are . - /// If is null. - /// If or is less than zero. -or- plus is greater than the number of elements in . - [NotNull] - public static byte[] JoinBytes(Slice separator, [NotNull] Slice[] values, int startIndex, int count) - { - // Note: this method is modeled after String.Join() and should behave the same - // - Only difference is that Slice.Nil and Slice.Empty are equivalent (either for separator, or for the elements of the array) - - if (values == null) throw new ArgumentNullException(nameof(values)); - //REVIEW: support negative indexing ? - if (startIndex < 0) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be a positive integer"); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), count, "Count must be a positive integer"); - if (startIndex > values.Length - count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must fit within the array"); - - if (count == 0) return Slice.EmptyArray; - if (count == 1) return values[startIndex].GetBytes() ?? Slice.EmptyArray; - - int size = 0; - for (int i = 0; i < count; i++) size = checked(size + values[startIndex + i].Count); - size = checked(size + (count - 1) * separator.Count); - - // if the size overflows, that means that the resulting buffer would need to be >= 2 GB, which is not possible! - if (size < 0) throw new OutOfMemoryException(); - - //note: we want to make sure the buffer of the writer will be the exact size (so that we can use the result as a byte[] without copying again) - var tmp = new byte[size]; - int p = 0; - for (int i = 0; i < count; i++) - { - if (i > 0) separator.WriteTo(tmp, ref p); - values[startIndex + i].WriteTo(tmp, ref p); - } - Contract.Assert(p == tmp.Length); - return tmp; - } - - /// Concatenates the specified elements of a slice sequence, using the specified separator between each element. - /// The slice to use as a separator. Can be empty. - /// A sequence will return the elements to concatenate. - /// A byte array that consists of the slices in delimited by the slice. -or- an empty array if has no elements, or and all the elements of are . - /// If is null. - [NotNull] - public static byte[] JoinBytes(Slice separator, [NotNull] IEnumerable values) - { - if (values == null) throw new ArgumentNullException(nameof(values)); - var array = (values as Slice[]) ?? values.ToArray(); - return JoinBytes(separator, array, 0, array.Length); - } - - /// Returns a slice array that contains the sub-slices in that are delimited by . A parameter specifies whether to return empty array elements. - /// Input slice that must be split into sub-slices - /// Separator that delimits the sub-slices in . Cannot be empty or nil - /// to omit empty array alements from the array returned; or to include empty array elements in the array returned. - /// An array whose elements contain the sub-slices that are delimited by . - /// If is empty, or if is not one of the values. - /// If does not contain the delimiter, the returned array consists of a single element that repeats the input, or an empty array if input is itself empty. - /// To reduce memory usage, the sub-slices returned in the array will all share the same underlying buffer of the input slice. - [NotNull] - public static Slice[] Split(Slice input, Slice separator, StringSplitOptions options = StringSplitOptions.None) - { - // this method is made to behave the same way as String.Split(), especially the following edge cases - // - Empty.Split(..., StringSplitOptions.None) => { Empty } - // - Empty.Split(..., StringSplitOptions.RemoveEmptyEntries) => { } - // differences: - // - If input is Nil, it is considered equivalent to Empty - // - If separator is Nil or Empty, the method throws - - var list = new List(); - - if (separator.Count <= 0) throw new ArgumentException("Separator must have at least one byte", nameof(separator)); - if (options < StringSplitOptions.None || options > StringSplitOptions.RemoveEmptyEntries) throw new ArgumentException("options"); - - bool skipEmpty = options.HasFlag(StringSplitOptions.RemoveEmptyEntries); - if (input.Count == 0) - { - return skipEmpty ? Slice.EmptySliceArray : new [] { Slice.Empty }; - } - - while (input.Count > 0) - { - int p = Find(input, separator); - if (p < 0) - { // last chunk - break; - } - if (p == 0) - { // empty chunk - if (!skipEmpty) list.Add(Slice.Empty); - } - else - { - list.Add(input.Substring(0, p)); - } - // note: we checked earlier that separator.Count > 0, so we are guaranteed to advance the cursor - input = input.Substring(p + separator.Count); - } - - if (input.Count > 0 || !skipEmpty) - { - list.Add(input); - } - - return list.ToArray(); - } - - /// Decode a Base64 encoded string into a slice - public static Slice FromBase64(string base64String) - { - return base64String == null ? Slice.Nil : base64String.Length == 0 ? Slice.Empty : Slice.Create(Convert.FromBase64String(base64String)); - } - - /// Encode an unsigned 8-bit integer into a slice - public static Slice FromByte(byte value) - { - return new Slice(ByteSprite, value, 1); - } - - #region 16-bit integers - - /// Encode a signed 16-bit integer into a variable size slice (1 or 2 bytes) in little-endian - public static Slice FromInt16(short value) - { - if (value >= 0) - { - if (value <= 255) - { - return Slice.FromByte((byte)value); - } - else - { - return new Slice(new byte[] { (byte)value, (byte)(value >> 8) }, 0, 2); - } - } - - return FromFixed16(value); - } - - /// Encode a signed 16-bit integer into a 2-byte slice in little-endian - public static Slice FromFixed16(short value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8) - }, - 0, - 2 - ); - } - - /// Encode an unsigned 16-bit integer into a variable size slice (1 or 2 bytes) in little-endian - public static Slice FromUInt16(ushort value) - { - if (value <= 255) - { - return Slice.FromByte((byte)value); - } - else - { - return FromFixedU16(value); - } - } - - /// Encode an unsigned 16-bit integer into a 2-byte slice in little-endian - /// 0x1122 => 11 22 - public static Slice FromFixedU16(ushort value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8) - }, - 0, - 2 - ); - } - - /// Encode an unsigned 16-bit integer into a 2-byte slice in big-endian - /// 0x1122 => 22 11 - public static Slice FromFixedU16BE(ushort value) - { - return new Slice( - new byte[] - { - (byte)(value >> 8), - (byte)value - }, - 0, - 4 - ); - } - - /// Encode an unsigned 16-bit integer into 7-bit encoded unsigned int (aka 'Varint16') - public static Slice FromVarint16(ushort value) - { - if (value < 128) - { - return FromByte((byte)value); - } - else - { - var writer = new SliceWriter(3); - writer.WriteVarint16(value); - return writer.ToSlice(); - } - } - - #endregion - - #region 32-bit integers - - /// Encode a signed 32-bit integer into a variable size slice (1, 2 or 4 bytes) in little-endian - public static Slice FromInt32(int value) - { - if (value >= 0) - { - if (value <= 255) - { - return Slice.FromByte((byte)value); - } - if (value <= 65535) - { - //TODO: possible micro optimization is for values like 0x100, 0x201, 0x1413 or 0x4342, where we could use 2 consecutive bytes in the ByteSprite, - return new Slice(new byte[] { (byte)value, (byte)(value >> 8) }, 0, 2); - } - } - - return FromFixed32(value); - } - - /// Encode a signed 32-bit integer into a 4-byte slice in little-endian - public static Slice FromFixed32(int value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8), - (byte)(value >> 16), - (byte)(value >> 24) - }, - 0, - 4 - ); - } - - /// Encode an unsigned 32-bit integer into a variable size slice (1, 2 or 4 bytes) in little-endian - public static Slice FromUInt32(uint value) - { - if (value <= 255) - { - return Slice.FromByte((byte)value); - } - if (value <= 65535) - { - return new Slice(new byte[] { (byte)value, (byte)(value >> 8) }, 0, 2); - } - - return FromFixedU32(value); - } - - /// Encode an unsigned 32-bit integer into a 4-byte slice in little-endian - /// 0x11223344 => 11 22 33 44 - public static Slice FromFixedU32(uint value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8), - (byte)(value >> 16), - (byte)(value >> 24) - }, - 0, - 4 - ); - } - - /// Encode an unsigned 32-bit integer into a 4-byte slice in big-endian - /// 0x11223344 => 44 33 22 11 - public static Slice FromFixedU32BE(uint value) - { - return new Slice( - new byte[] - { - (byte)(value >> 24), - (byte)(value >> 16), - (byte)(value >> 8), - (byte)value - }, - 0, - 4 - ); - } - - /// Encode an unsigned 32-bit integer into 7-bit encoded unsigned int (aka 'Varint32') - public static Slice FromVarint32(uint value) - { - if (value < 128) - { - return FromByte((byte)value); - } - else - { - var writer = new SliceWriter(5); - writer.WriteVarint32(value); - return writer.ToSlice(); - } - } - - #endregion - - #region 64-bit integers - - /// Encode a signed 64-bit integer into a variable size slice (1, 2, 4 or 8 bytes) in little-endian - public static Slice FromInt64(long value) - { - if (value >= 0 && value <= int.MaxValue) - { - return FromInt32((int)value); - } - return FromFixed64(value); - } - - /// Encode a signed 64-bit integer into a 8-byte slice in little-endian - public static Slice FromFixed64(long value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8), - (byte)(value >> 16), - (byte)(value >> 24), - (byte)(value >> 32), - (byte)(value >> 40), - (byte)(value >> 48), - (byte)(value >> 56) - }, - 0, - 8 - ); - } - - /// Encode an unsigned 64-bit integer into a variable size slice (1, 2, 4 or 8 bytes) in little-endian - public static Slice FromUInt64(ulong value) - { - if (value <= 255) - { - return Slice.FromByte((byte)value); - } - if (value <= 65535) - { - return new Slice(new byte[] { (byte)value, (byte)(value >> 8) }, 0, 2); - } - - if (value <= uint.MaxValue) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8), - (byte)(value >> 16), - (byte)(value >> 24) - }, - 0, - 4 - ); - } - - return FromFixedU64(value); - } - - /// Encode an unsigned 64-bit integer into a 8-byte slice in little-endian - /// 0x1122334455667788 => 11 22 33 44 55 66 77 88 - public static Slice FromFixedU64(ulong value) - { - return new Slice( - new byte[] - { - (byte)value, - (byte)(value >> 8), - (byte)(value >> 16), - (byte)(value >> 24), - (byte)(value >> 32), - (byte)(value >> 40), - (byte)(value >> 48), - (byte)(value >> 56) - }, - 0, - 8 - ); - } - - /// Encode an unsigned 64-bit integer into a 8-byte slice in big-endian - /// 0x1122334455667788 => 88 77 66 55 44 33 22 11 - public static Slice FromFixedU64BE(ulong value) - { - return new Slice( - new byte[] - { - (byte)(value >> 56), - (byte)(value >> 48), - (byte)(value >> 40), - (byte)(value >> 32), - (byte)(value >> 24), - (byte)(value >> 16), - (byte)(value >> 8), - (byte)value - }, - 0, - 8 - ); - } - - /// Encode an unsigned 64-bit integer into 7-bit encoded unsigned int (aka 'Varint64') - public static Slice FromVarint64(ulong value) - { - if (value < 128) - { - return FromByte((byte)value); - } - else - { - var writer = new SliceWriter(10); - writer.WriteVarint64(value); - return writer.ToSlice(); - } - } - - #endregion - - #region decimals - - /// Encode a 32-bit decimal into an 4-byte slice - public static Slice FromSingle(float value) - { - //TODO: may not work on BE platforms? - uint bits; - unsafe { bits = *(uint*)(&value); } - return FromFixedU32(bits); - } - - /// Encode a 64-bit decimal into an 8-byte slice - public static Slice FromDouble(double value) - { - //TODO: may not work on BE platforms? - ulong bits; - unsafe { bits = *(ulong*)(&value); } - return FromFixedU64(bits); - } - - #endregion - - - /// Create a 16-byte slice containing a System.Guid encoding according to RFC 4122 (Big Endian) - /// WARNING: Slice.FromGuid(guid).GetBytes() will not produce the same result as guid.ToByteArray() ! - /// If you need to produce Microsoft compatible byte arrays, use Slice.Create(guid.ToByteArray()) but then you shoud NEVER use Slice.ToGuid() to decode such a value ! - public static Slice FromGuid(Guid value) - { - // UUID are stored using the RFC4122 format (Big Endian), while .NET's System.GUID use Little Endian - // => we will convert the GUID into a UUID under the hood, and hope that it gets converted back when read from the db - - return new Uuid128(value).ToSlice(); - } - - /// Create a 16-byte slice containing an RFC 4122 compliant 128-bit UUID - /// You should never call this method on a slice created from the result of calling System.Guid.ToByteArray() ! - public static Slice FromUuid128(Uuid128 value) - { - // UUID should already be in the RFC 4122 ordering - return value.ToSlice(); - } - - /// Create an 8-byte slice containing an 64-bit UUID - public static Slice FromUuid64(Uuid64 value) - { - return value.ToSlice(); - } - - internal static readonly Encoding DefaultEncoding = -#if CORE_CLR - Encoding.GetEncoding(0); -#else - Encoding.Default; -#endif - - /// Dangerously create a slice containing string converted to ASCII. 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) - { - return text == null ? Slice.Nil : text.Length == 0 ? Slice.Empty : Slice.Create(DefaultEncoding.GetBytes(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)); - } - - /// Create a slice that holds the UTF-8 encoded representation of - /// - /// 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) - { - if (value < 128) - { // ASCII - return Slice.FromByte((byte)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); - return n == 1 ? FromByte(tmp[0]) : new Slice(tmp, 0, n); - } - - /// Convert an hexadecimal digit (0-9A-Fa-f) into the corresponding decimal value - /// Hexadecimal digit (case insensitive) - /// Decimal value between 0 and 15, or an exception - private static int NibbleToDecimal(char c) - { - int x = c - 48; - if (x < 10) return x; - if (x >= 17 && x <= 42) return x - 7; - if (x >= 49 && x <= 74) return x - 39; - throw new FormatException("Input is not a valid hexadecimal digit"); - } - - /// Convert an hexadecimal encoded string ("1234AA7F") into a slice - /// String contains a sequence of pairs of hexadecimal digits with no separating spaces. - /// Slice containing the decoded byte array, or an exeception if the string is empty or has an odd length - public static Slice FromHexa(string hexaString) - { - if (string.IsNullOrEmpty(hexaString)) return hexaString == null ? Slice.Nil : Slice.Empty; - - if (hexaString.IndexOf(' ') > 0) - { // remove spaces - hexaString = hexaString.Replace(" ", ""); - } - - if ((hexaString.Length & 1) != 0) throw new ArgumentException("Hexadecimal string must be of even length", nameof(hexaString)); - - var buffer = new byte[hexaString.Length >> 1]; - for (int i = 0; i < hexaString.Length; i += 2) - { - buffer[i >> 1] = (byte) ((NibbleToDecimal(hexaString[i]) << 4) | NibbleToDecimal(hexaString[i + 1])); - } - return new Slice(buffer, 0, buffer.Length); - } - - /// Returns true is the slice is not null - /// An empty slice is NOT considered null - public bool HasValue { get { return this.Array != null; } } - - /// Returns true if the slice is null - /// An empty slice is NOT considered null - public bool IsNull { get { return this.Array == null; } } - - /// Return true if the slice is not null but contains 0 bytes - /// A null slice is NOT empty - public bool IsEmpty { get { return this.Count == 0 && this.Array != null; } } - - /// Returns true if the slice is null or empty, or false if it contains at least one byte - public bool IsNullOrEmpty { get { return this.Count == 0; } } - - /// Returns true if the slice contains at least one byte, or false if it is null or empty - public bool IsPresent { get { return this.Count > 0; } } - - /// Return a byte array containing all the bytes of the slice, or null if the slice is null - /// Byte array with a copy of the slice, or null - [Pure, CanBeNull] - public byte[] GetBytes() - { - if (this.Count == 0) return this.Array == null ? null : Slice.EmptyArray; - SliceHelpers.EnsureSliceIsValid(ref this); - - var tmp = new byte[this.Count]; - SliceHelpers.CopyBytesUnsafe(tmp, 0, this.Array, this.Offset, this.Count); - return tmp; - } - - /// Return a byte array containing a subset of the bytes of the slice, or null if the slice is null - /// Byte array with a copy of a subset of the slice, or null - [Pure, CanBeNull] - public byte[] GetBytes(int offset, int count) - { - //TODO: throw if this.Array == null ? (what does "Slice.Nil.GetBytes(..., 0)" mean ?) - - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset)); - if (count < 0 || offset + count > this.Count) throw new ArgumentOutOfRangeException(nameof(count)); - - if (count == 0) return this.Array == null ? null : Slice.EmptyArray; - SliceHelpers.EnsureSliceIsValid(ref this); - - var tmp = new byte[count]; - SliceHelpers.CopyBytesUnsafe(tmp, 0, this.Array, this.Offset + offset, count); - return tmp; - } - - /// Return a stream that wraps this slice - /// Stream that will read the slice from the start. - /// - /// You can use this method to convert text into specific encodings, load bitmaps (JPEG, PNG, ...), or any serialization format that requires a Stream or TextReader instance. - /// Disposing this stream will have no effect on the slice. - /// - [Pure, NotNull] - public SliceStream AsStream() - { - SliceHelpers.EnsureSliceIsValid(ref this); - return new SliceStream(this); - } - - /// Stringify a slice containing only ASCII chars - /// ASCII string, or null if the slice is null - [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); - } - - /// Stringify a slice containing an UTF-8 encoded string - /// Unicode string, or null if the slice is null - [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); - } - - /// Converts a slice using Base64 encoding - [Pure, CanBeNull] - public string ToBase64() - { - if (this.Count == 0) return this.Array == null ? null : String.Empty; - SliceHelpers.EnsureSliceIsValid(ref this); - return Convert.ToBase64String(this.Array, this.Offset, this.Count); - } - - /// Converts a slice into a string with each byte encoded into hexadecimal (lowercase) - /// "0123456789abcdef" - [Pure, CanBeNull] - public string ToHexaString() - { - if (this.Count == 0) return this.Array == null ? null : String.Empty; - var buffer = this.Array; - int p = this.Offset; - int n = this.Count; - var sb = new StringBuilder(n * 2); - while (n-- > 0) - { - byte b = buffer[p++]; - int x = b >> 4; - sb.Append((char)(x + (x < 10 ? 48 : 87))); - x = b & 0xF; - sb.Append((char)(x + (x < 10 ? 48 : 87))); - } - return sb.ToString(); - } - - /// Converts a slice into a string with each byte encoded into hexadecimal (uppercase) separated by a char - /// Character used to separate the hexadecimal pairs (ex: ' ') - /// "01 23 45 67 89 ab cd ef" - [Pure, CanBeNull] - public string ToHexaString(char sep) - { - if (this.Count == 0) return this.Array == null ? null : String.Empty; - var buffer = this.Array; - int p = this.Offset; - int n = this.Count; - var sb = new StringBuilder(n * (sep == '\0' ? 2 : 3)); - while (n-- > 0) - { - if (sep != '\0' && sb.Length > 0) sb.Append(sep); - byte b = buffer[p++]; - int x = b >> 4; - sb.Append((char)(x + (x < 10 ? 48 : 55))); - x = b & 0xF; - sb.Append((char)(x + (x < 10 ? 48 : 55))); - } - return sb.ToString(); - } - - [NotNull] - private static StringBuilder EscapeString(StringBuilder sb, [NotNull] byte[] buffer, int offset, int count, [NotNull] Encoding encoding) - { - if (sb == null) sb = new StringBuilder(count + 16); - foreach(var c in encoding.GetChars(buffer, offset, count)) - { - if ((c >= ' ' && c <= '~') || (c >= 880 && c <= 2047) || (c >= 12352 && c <= 12591)) - sb.Append(c); - else if (c == '\n') - sb.Append(@"\n"); - else if (c == '\r') - sb.Append(@"\r"); - else if (c == '\t') - sb.Append(@"\t"); - else if (c > 127) - sb.Append(@"\u").Append(((int)c).ToString("x4", CultureInfo.InvariantCulture)); - else // pas clean! - sb.Append(@"\x").Append(((int)c).ToString("x2", CultureInfo.InvariantCulture)); - } - return sb; - } - - /// Helper method that dumps the slice as a string (if it contains only printable ascii chars) or an hex array if it contains non printable chars. It should only be used for logging and troubleshooting ! - /// Returns either "'abc'", "<00 42 7F>", or "{ ...JSON... }". Returns "''" for Slice.Empty, and "" for - [Pure, NotNull] - public string ToAsciiOrHexaString() //REVIEW: rename this to ToPrintableString() ? - { - //REVIEW: rename this to ToFriendlyString() ? or ToLoggableString() ? - if (this.Count == 0) return this.Array != null ? "''" : String.Empty; - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset; - - // 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(); - } - - if (n >= 2) - { - // 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(); - } - } - - // do a first path on the slice to look for binary of possible text - bool mustEscape = false; - while (n-- > 0) - { - byte b = buffer[p++]; - if (b >= 32 && b < 127) continue; - - // we accept via escaping the following special chars: CR, LF, TAB - if (b == 10 || b == 13 || b == 9) - { - mustEscape = true; - continue; - } - - //TODO: are there any chars above 128 that could be accepted ? - - // this looks like binary - return "<" + ToHexaString(' ') + ">"; - } - - if (!mustEscape) - { // only printable chars found - return new StringBuilder(n + 2).Append('\'').Append(Encoding.ASCII.GetString(buffer, this.Offset, this.Count)).Append('\'').ToString(); - } - else - { // some escaping required - return EscapeString(new StringBuilder(n + 2).Append('\''), buffer, this.Offset, this.Count, Encoding.UTF8).Append('\'').ToString(); - } - } - - /// Converts a slice into a byte - /// Value of the first and only byte of the slice, or 0 if the slice is null or empty. - /// If the slice has more than one byte - [Pure] - public byte ToByte() - { - if (this.Count == 0) return 0; - if (this.Count > 1) throw new FormatException("Cannot convert slice into a Byte because it is larger than 1 byte"); - SliceHelpers.EnsureSliceIsValid(ref this); - return this.Array[this.Offset]; - } - - /// Converts a slice into a signed byte (-128..+127) - /// Value of the first and only byte of the slice, or 0 if the slice is null or empty. - /// If the slice has more than one byte - [Pure] - public sbyte ToSByte() - { - if (this.Count == 0) return 0; - if (this.Count > 1) throw new FormatException("Cannot convert slice into an SByte because it is larger than 1 byte"); - SliceHelpers.EnsureSliceIsValid(ref this); - return (sbyte)this.Array[this.Offset]; - } - - /// Converts a slice into a boolean. - /// False if the slice is empty, or is equal to the byte 0; otherwise, true. - [Pure] - public bool ToBool() - { - SliceHelpers.EnsureSliceIsValid(ref this); - // Anything appart from nil/empty, or the byte 0 itself is considered truthy. - return this.Count > 1 || (this.Count == 1 && this.Array[this.Offset] != 0); - //TODO: consider checking if the slice consist of only zeroes ? (ex: Slice.FromFixed32(0) could be considered falsy ...) - } - - #region 16 bits... - - /// Converts a slice into a little-endian encoded, signed 16-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 2 bytes - /// If there are more than 2 bytes in the slice - [Pure] - public short ToInt16() - { - SliceHelpers.EnsureSliceIsValid(ref this); - switch (this.Count) - { - case 0: return 0; - case 1: return this.Array[this.Offset]; - case 2: return (short) (this.Array[this.Offset] | (this.Array[this.Offset + 1] << 8)); - default: throw new FormatException("Cannot convert slice into an Int16 because it is larger than 2 bytes"); - } - } - - /// Converts a slice into a big-endian encoded, signed 16-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 2 bytes - /// If there are more than 2 bytes in the slice - [Pure] - public short ToInt16BE() - { - SliceHelpers.EnsureSliceIsValid(ref this); - switch (this.Count) - { - case 0: return 0; - case 1: return this.Array[this.Offset]; - case 2: return (short)(this.Array[this.Offset + 1] | (this.Array[this.Offset] << 8)); - default: throw new FormatException("Cannot convert slice into an Int16 because it is larger than 2 bytes"); - } - } - - /// Converts a slice into a little-endian encoded, unsigned 16-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 2 bytes - /// If there are more than 2 bytes in the slice - [Pure] - public ushort ToUInt16() - { - SliceHelpers.EnsureSliceIsValid(ref this); - switch (this.Count) - { - case 0: return 0; - case 1: return this.Array[this.Offset]; - case 2: return (ushort)(this.Array[this.Offset] | (this.Array[this.Offset + 1] << 8)); - default: throw new FormatException("Cannot convert slice into an UInt16 because it is larger than 2 bytes"); - } - } - - /// Converts a slice into a little-endian encoded, unsigned 16-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 2 bytes - /// If there are more than 2 bytes in the slice - [Pure] - public ushort ToUInt16BE() - { - SliceHelpers.EnsureSliceIsValid(ref this); - switch (this.Count) - { - case 0: return 0; - case 1: return this.Array[this.Offset]; - case 2: return (ushort)(this.Array[this.Offset + 1] | (this.Array[this.Offset] << 8)); - default: throw new FormatException("Cannot convert slice into an UInt16 because it is larger than 2 bytes"); - } - } - - /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 2) - /// Decoded unsigned short. - /// If is less than zero, or more than 2. - [Pure] - public ushort ReadUInt16(int offset, int bytes) - { - if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException(nameof(bytes)); - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset); - switch(bytes) - { - case 0: return 0; - case 1: return buffer[p]; - default: return (ushort)(buffer[p] | (buffer[p + 1] << 8)); - } - } - - /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 2) - /// Decoded unsigned short. - /// If is less than zero, or more than 2. - [Pure] - public ushort ReadUInt16BE(int offset, int bytes) - { - if (bytes < 0 || bytes > 2) throw new ArgumentOutOfRangeException(nameof(bytes)); - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset); - switch (bytes) - { - case 0: return 0; - case 1: return buffer[p]; - default: return (ushort)(buffer[p + 1] | (buffer[p] << 8)); - } - } - - #endregion - - #region 32 bits... - - /// Converts a slice into a little-endian encoded, signed 32-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 4 bytes - /// If there are more than 4 bytes in the slice - [Pure] - public int ToInt32() - { - if (this.Count == 0) return 0; - if (this.Count > 4) throw new FormatException("Cannot convert slice into an Int32 because it is larger than 4 bytes"); - SliceHelpers.EnsureSliceIsValid(ref this); - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset + n - 1; - - int value = buffer[p--]; - while (--n > 0) - { - value = (value << 8) | buffer[p--]; - } - return value; - } - - /// Converts a slice into a big-endian encoded, signed 32-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 4 bytes - /// If there are more than 4 bytes in the slice - [Pure] - public int ToInt32BE() - { - SliceHelpers.EnsureSliceIsValid(ref this); - - int n = this.Count; - if (n == 0) return 0; - if (n > 4) throw new FormatException("Cannot convert slice into an Int32 because it is larger than 4 bytes"); - - var buffer = this.Array; - int p = this.Offset; - - int value = buffer[p++]; - while (--n > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - /// Converts a slice into a little-endian encoded, unsigned 32-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes - /// If there are more than 4 bytes in the slice - [Pure] - public uint ToUInt32() - { - SliceHelpers.EnsureSliceIsValid(ref this); - - int n = this.Count; - if (n == 0) return 0; - if (n > 4) throw new FormatException("Cannot convert slice into an UInt32 because it is larger than 4 bytes"); - - var buffer = this.Array; - int p = this.Offset + n - 1; - - uint value = buffer[p--]; - while (--n > 0) - { - value = (value << 8) | buffer[p--]; - } - return value; - } - - /// Converts a slice into a big-endian encoded, unsigned 32-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 4 bytes - /// If there are more than 4 bytes in the slice - [Pure] - public uint ToUInt32BE() - { - SliceHelpers.EnsureSliceIsValid(ref this); - - int n = this.Count; - if (n == 0) return 0; - if (n > 4) throw new FormatException("Cannot convert slice into an UInt32 because it is larger than 4 bytes"); - - var buffer = this.Array; - int p = this.Offset; - - uint value = buffer[p++]; - while (--n > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 4) - /// Decoded unsigned integer. - /// If is less than zero, or more than 4. - [Pure] - public uint ReadUInt32(int offset, int bytes) - { - if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException(nameof(bytes)); - if (bytes == 0) return 0; - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset) + bytes - 1; - - uint value = buffer[p--]; - while (--bytes > 0) - { - value = (value << 8) | buffer[p--]; - } - return value; - } - - /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 4) - /// Decoded unsigned integer. - /// If is less than zero, or more than 4. - [Pure] - public uint ReadUInt32BE(int offset, int bytes) - { - if (bytes < 0 || bytes > 4) throw new ArgumentOutOfRangeException(nameof(bytes)); - if (bytes == 0) return 0; - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset); - - uint value = buffer[p++]; - while (--bytes > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - #endregion - - #region 64 bits... - - /// Converts a slice into a little-endian encoded, signed 64-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 8 bytes - /// If there are more than 8 bytes in the slice - [Pure] - public long ToInt64() - { - if (this.Count == 0) return 0L; - if (this.Count > 8) throw new FormatException("Cannot convert slice into an Int64 because it is larger than 8 bytes"); - SliceHelpers.EnsureSliceIsValid(ref this); - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset + n - 1; - - long value = buffer[p--]; - while (--n > 0) - { - value = (value << 8) | buffer[p--]; - } - - return value; - } - - /// Converts a slice into a big-endian encoded, signed 64-bit integer. - /// 0 of the slice is null or empty, a signed integer, or an error if the slice has more than 8 bytes - /// If there are more than 8 bytes in the slice - [Pure] - public long ToInt64BE() - { - if (this.Count == 0) return 0L; - if (this.Count > 8) throw new FormatException("Cannot convert slice into an Int64 because it is larger than 8 bytes"); - SliceHelpers.EnsureSliceIsValid(ref this); - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset; - - long value = buffer[p++]; - while (--n > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - /// Converts a slice into a little-endian encoded, unsigned 64-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes - /// If there are more than 8 bytes in the slice - [Pure] - public ulong ToUInt64() - { - if (this.Count == 0) return 0L; - if (this.Count > 8) throw new FormatException("Cannot convert slice into an UInt64 because it is larger than 8 bytes"); - SliceHelpers.EnsureSliceIsValid(ref this); - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset + n - 1; - - ulong value = buffer[p--]; - while (--n > 0) - { - value = (value << 8) | buffer[p--]; - } - return value; - } - - /// Converts a slice into a little-endian encoded, unsigned 64-bit integer. - /// 0 of the slice is null or empty, an unsigned integer, or an error if the slice has more than 8 bytes - /// If there are more than 8 bytes in the slice - [Pure] - public ulong ToUInt64BE() - { - if (this.Count == 0) return 0L; - if (this.Count > 8) throw new FormatException("Cannot convert slice into an UInt64 because it is larger than 8 bytes"); - SliceHelpers.EnsureSliceIsValid(ref this); - - var buffer = this.Array; - int n = this.Count; - int p = this.Offset; - - ulong value = buffer[p++]; - while (--n > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - /// Read a variable-length, little-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 8) - /// Decoded unsigned integer. - /// If is less than zero, or more than 8. - [Pure] - public ulong ReadUInt64(int offset, int bytes) - { - if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException(nameof(bytes)); - if (bytes == 0) return 0UL; - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset) + bytes - 1; - - ulong value = buffer[p--]; - while (--bytes > 0) - { - value = (value << 8) | buffer[p--]; - } - return value; - } - - /// Read a variable-length, big-endian encoded, unsigned integer from a specific location in the slice - /// Relative offset of the first byte - /// Number of bytes to read (up to 8) - /// Decoded unsigned integer. - /// If is less than zero, or more than 8. - [Pure] - public ulong ReadUInt64BE(int offset, int bytes) - { - if (bytes < 0 || bytes > 8) throw new ArgumentOutOfRangeException(nameof(bytes)); - if (bytes == 0) return 0UL; - - var buffer = this.Array; - int p = UnsafeMapToOffset(offset); - - ulong value = buffer[p++]; - while (--bytes > 0) - { - value = (value << 8) | buffer[p++]; - } - return value; - } - - /// Converts a slice into a 64-bit UUID. - /// Uuid decoded from the Slice. - /// The slice can either be an 8-byte array, or an ASCII string of 16, 17 or 19 chars - [Pure] - public Uuid64 ToUuid64() - { - if (this.Count == 0) return default(Uuid64); - SliceHelpers.EnsureSliceIsValid(ref this); - - switch (this.Count) - { - case 8: - { // binary (8 bytes) - return new Uuid64(this); - } - - case 16: // hex16 - case 17: // hex8-hex8 - case 19: // {hex8-hex8} - { - return Uuid64.Parse(this.ToAscii()); - } - } - - throw new FormatException("Cannot convert slice into an Uuid64 because it has an incorrect size"); - } - - #endregion - - #region 128 bits... - - /// Converts a slice into a Guid. - /// Native Guid decoded from the Slice. - /// The slice can either be a 16-byte RFC4122 GUID, or an ASCII string of 36 chars - [Pure] - public Guid ToGuid() - { - if (this.Count == 0) return default(Guid); - SliceHelpers.EnsureSliceIsValid(ref this); - - if (this.Count == 16) - { // direct byte array - - // UUID are stored using the RFC4122 format (Big Endian), while .NET's System.GUID use Little Endian - // we need to swap the byte order of the Data1, Data2 and Data3 chunks, to ensure that Guid.ToString() will return the proper value. - - return new Uuid128(this).ToGuid(); - } - - if (this.Count == 36) - { // string representation (ex: "da846709-616d-4e82-bf55-d1d3e9cde9b1") - return Guid.Parse(this.ToAscii()); - } - - throw new FormatException("Cannot convert slice into a Guid because it has an incorrect size"); - } - - /// Converts a slice into a 128-bit UUID. - /// Uuid decoded from the Slice. - /// The slice can either be a 16-byte RFC4122 GUID, or an ASCII string of 36 chars - [Pure] - public Uuid128 ToUuid128() - { - if (this.Count == 0) return default(Uuid128); - SliceHelpers.EnsureSliceIsValid(ref this); - - if (this.Count == 16) - { - return new Uuid128(this); - } - - if (this.Count == 36) - { - return Uuid128.Parse(this.ToAscii()); - } - - throw new FormatException("Cannot convert slice into an Uuid128 because it has an incorrect size"); - } - - #endregion - - /// Returns a new slice that contains an isolated copy of the buffer - /// Slice that is equivalent, but is isolated from any changes to the buffer - [Pure] - public Slice Memoize() - { - if (this.Count == 0) return this.Array == null ? Slice.Nil : Slice.Empty; - return new Slice(GetBytes(), 0, this.Count); - } - - /// Map an offset in the slice into the absolute offset in the buffer, without any bound checking - /// Relative offset (negative values mean from the end) - /// Absolute offset in the buffer - private int UnsafeMapToOffset(int index) - { - int p = NormalizeIndex(index); - Contract.Requires(p >= 0 & p < this.Count, "Index is outside the slice buffer"); - return this.Offset + p; - } - - /// Map an offset in the slice into the absolute offset in the buffer - /// Relative offset (negative values mean from the end) - /// Absolute offset in the buffer - /// If the index is outside the slice - private int MapToOffset(int index) - { - int p = NormalizeIndex(index); - if (p < 0 || p >= this.Count) FailIndexOutOfBound(index); - checked { return this.Offset + p; } - } - - /// Normalize negative index values into offset from the start - /// Relative offset (negative values mean from the end) - /// Relative offset from the start of the slice - private int NormalizeIndex(int index) - { - checked { return index < 0 ? index + this.Count : index; } - } - - /// Returns the value of one byte in the slice - /// Offset of the byte (negative values means start from the end) - public byte this[int index] - { - get { return this.Array[MapToOffset(index)]; } - } - - /// Returns a substring of the current slice that fits withing the specified index range - /// The starting position of the substring. Positive values means from the start, negative values means from the end - /// The end position (excluded) of the substring. Positive values means from the start, negative values means from the end - /// Subslice - public Slice this[int start, int end] - { - get - { - start = NormalizeIndex(start); - end = NormalizeIndex(end); - - // bound check - if (start < 0) start = 0; - if (end > this.Count) end = this.Count; - - if (start >= end) return Slice.Empty; - if (start == 0 && end == this.Count) return this; - - checked { return new Slice(this.Array, this.Offset + start, end - start); } - } - } - - [ContractAnnotation("=> halt")] - private static void FailIndexOutOfBound(int index) - { - throw new IndexOutOfRangeException("Index is outside the slice"); - } - - /// Copy this slice into another buffer, and move the cursor - /// Buffer where to copy this slice - /// Offset into the destination buffer - public void WriteTo(byte[] buffer, ref int cursor) - { - SliceHelpers.EnsureBufferIsValid(buffer, cursor, this.Count); - SliceHelpers.EnsureSliceIsValid(ref this); - - if (this.Count > 0) - { - SliceHelpers.CopyBytes(buffer, cursor, this.Array, this.Offset, this.Count); - cursor += this.Count; - } - } - - /// Copy this slice into another buffer - /// Buffer where to copy this slice - /// Offset into the destination buffer - public void CopyTo(byte[] buffer, int offset) - { - SliceHelpers.EnsureBufferIsValid(buffer, offset, this.Count); - SliceHelpers.EnsureSliceIsValid(ref this); - - SliceHelpers.CopyBytesUnsafe(buffer, offset, this.Array, this.Offset, this.Count); - } - - /// Retrieves a substring from this instance. The substring starts at a specified character position. - /// The starting position of the substring. Positive values mmeans from the start, negative values means from the end - /// A slice that is equivalent to the substring that begins at (from the start or the end depending on the sign) in this instance, or Slice.Empty if is equal to the length of the slice. - /// The substring does not copy the original data, and refers to the same buffer as the original slice. Any change to the parent slice's buffer will be seen by the substring. You must call Memoize() on the resulting substring if you want a copy - /// {"ABCDE"}.Substring(0) => {"ABC"} - /// {"ABCDE"}.Substring(1} => {"BCDE"} - /// {"ABCDE"}.Substring(-2} => {"DE"} - /// {"ABCDE"}.Substring(5} => Slice.Empty - /// Slice.Empty.Substring(0) => Slice.Empty - /// Slice.Nil.Substring(0) => Slice.Emtpy - /// - /// indicates a position not within this instance, or is less than zero - [Pure] - public Slice Substring(int offset) - { - if (offset == 0) return this; - - // negative values means from the end - if (offset < 0) offset = this.Count + offset; - - if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less then start of the slice"); - if (offset > this.Count) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be larger than end of slice"); - - return this.Count == offset ? Slice.Empty : new Slice(this.Array, this.Offset + offset, this.Count - offset); - } - - /// Retrieves a substring from this instance. The substring starts at a specified character position and has a specified length. - /// The starting position of the substring. Positive values means from the start, negative values means from the end - /// Number of bytes in the substring - /// A slice that is equivalent to the substring of length that begins at (from the start or the end depending on the sign) in this instance, or Slice.Empty if count is zero. - /// The substring does not copy the original data, and refers to the same buffer as the original slice. Any change to the parent slice's buffer will be seen by the substring. You must call Memoize() on the resulting substring if you want a copy - /// {"ABCDE"}.Substring(0, 3) => {"ABC"} - /// {"ABCDE"}.Substring(1, 3} => {"BCD"} - /// {"ABCDE"}.Substring(-2, 2} => {"DE"} - /// Slice.Empty.Substring(0, 0) => Slice.Empty - /// Slice.Nil.Substring(0, 0) => Slice.Emtpy - /// - /// plus indicates a position not within this instance, or or is less than zero - [Pure] - public Slice Substring(int offset, int count) - { - if (count == 0) return Slice.Empty; - - // negative values means from the end - if (offset < 0) offset = this.Count + offset; - - if (offset < 0 || offset >= this.Count) throw new ArgumentOutOfRangeException(nameof(offset), "Offset must be inside the slice"); - if (count < 0) throw new ArgumentOutOfRangeException(nameof(count), "Count must be a positive integer"); - if (offset > this.Count - count) throw new ArgumentOutOfRangeException(nameof(count), "Offset and count must refer to a location within the slice"); - - return new Slice(this.Array, this.Offset + offset, count); - } - - /// Returns a slice array that contains the sub-slices in this instance that are delimited by the specified separator - /// The slice that delimits the sub-slices in this instance. - /// to omit empty array elements from the array returned; or to include empty array elements in the array returned. - /// An array whose elements contains the sub-slices in this instance that are delimited by the value of . - [Pure] - public Slice[] Split(Slice separator, StringSplitOptions options = StringSplitOptions.None) - { - return Split(this, separator, options); - } - - /// Reports the zero-based index of the first occurence of the specified slice in this instance. - /// The slice to seek - /// The zero-based index of if that slice is found, or -1 if it is not. If is , then the return value is -1. - [Pure] - public int IndexOf(Slice value) - { - return Find(this, value); - } - - /// Reports the zero-based index of the first occurence of the specified slice in this instance. The search starts at a specified position. - /// The slice to seek - /// The search starting position - /// The zero-based index of if that slice is found, or -1 if it is not. If is , then the return value is startIndex - [Pure] - public int IndexOf(Slice value, int startIndex) - { - //REVIEW: support negative indexing ? - if (startIndex < 0 || startIndex > this.Count) throw new ArgumentOutOfRangeException(nameof(startIndex), startIndex, "Start index must be inside the buffer"); - if (this.Count == 0) - { - return value.Count == 0 ? startIndex : -1; - } - var tmp = startIndex == 0 ? this : new Slice(this.Array, this.Offset + startIndex, this.Count - startIndex); - return Find(tmp, value); - } - - /// Determines whether the beginning of this slice instance matches a specified slice. - /// The slice to compare - /// true if matches the beginning of this slice; otherwise, false - [Pure] - public bool StartsWith(Slice value) - { - if (!value.HasValue) throw new ArgumentNullException(nameof(value)); - - // any strings starts with the empty string - if (value.Count == 0) return true; - - // prefix cannot be bigger - if (value.Count > this.Count) return false; - - return SliceHelpers.SameBytes(this.Array, this.Offset, value.Array, value.Offset, value.Count); - } - - /// Determines whether the end of this slice instance matches a specified slice. - /// The slice to compare to the substring at the end of this instance. - /// true if matches the end of this slice; otherwise, false - [Pure] - public bool EndsWith(Slice value) - { - if (!value.HasValue) throw new ArgumentNullException(nameof(value)); - - // any strings ends with the empty string - if (value.Count == 0) return true; - - // suffix cannot be bigger - if (value.Count > this.Count) return false; - - return SliceHelpers.SameBytes(this.Array, this.Offset + this.Count - value.Count, value.Array, value.Offset, value.Count); - } - - /// Equivalent of StartsWith, but the returns false if both slices are identical - [Pure] - public bool PrefixedBy(Slice parent) - { - // empty is a parent of everyone - if (parent.Count == 0) return true; - - // we must have at least one more byte then the parent - if (this.Count <= parent.Count) return false; - - // must start with the same bytes - return SliceHelpers.SameBytes(parent.Array, parent.Offset, this.Array, this.Offset, parent.Count); - } - - /// Equivalent of EndsWith, but the returns false if both slices are identical - public bool SuffixedBy(Slice parent) - { - // empty is a parent of everyone - if (parent.IsNullOrEmpty) return true; - // empty is not a child of anything - if (this.IsNullOrEmpty) return false; - - // we must have at least one more byte then the parent - if (this.Count <= parent.Count) return false; - - // must start with the same bytes - return SliceHelpers.SameBytes(parent.Array, parent.Offset + this.Count - parent.Count, this.Array, this.Offset, parent.Count); - } - - /// Append/Merge a slice at the end of the current slice - /// Slice that must be appended - /// Merged slice if both slices are contigous, or a new slice containg the content of the current slice, followed by the tail slice. Or Slice.Empty if both parts are nil or empty - [Pure] - public Slice Concat(Slice tail) - { - if (tail.Count == 0) return this.Count > 0 ? this: Slice.Empty; - if (this.Count == 0) return tail; - - SliceHelpers.EnsureSliceIsValid(ref tail); - SliceHelpers.EnsureSliceIsValid(ref this); - - // special case: adjacent segments ? - if (object.ReferenceEquals(this.Array, tail.Array) && this.Offset + this.Count == tail.Offset) - { - return new Slice(this.Array, this.Offset, this.Count + tail.Count); - } - - byte[] tmp = new byte[this.Count + tail.Count]; - SliceHelpers.CopyBytesUnsafe(tmp, 0, this.Array, this.Offset, this.Count); - SliceHelpers.CopyBytesUnsafe(tmp, this.Count, tail.Array, tail.Offset, tail.Count); - return new Slice(tmp, 0, tmp.Length); - } - - /// Append an array of slice at the end of the current slice, all sharing the same buffer - /// Slices that must be appended - /// Array of slices (for all keys) that share the same underlying buffer - [Pure, NotNull] - public Slice[] ConcatRange([NotNull] Slice[] slices) - { - if (slices == null) throw new ArgumentNullException(nameof(slices)); - SliceHelpers.EnsureSliceIsValid(ref this); - - // pre-allocate by computing final buffer capacity - var prefixSize = this.Count; - var capacity = slices.Sum((slice) => prefixSize + slice.Count); - var writer = new SliceWriter(capacity); - var next = new List(slices.Length); - - //TODO: use multiple buffers if item count is huge ? - - foreach (var slice in slices) - { - writer.WriteBytes(this); - writer.WriteBytes(slice); - next.Add(writer.Position); - } - - return FdbKey.SplitIntoSegments(writer.Buffer, 0, next); - } - - /// Append a sequence of slice at the end of the current slice, all sharing the same buffer - /// Slices that must be appended - /// Array of slices (for all keys) that share the same underlying buffer - [Pure, NotNull] - public Slice[] ConcatRange([NotNull] IEnumerable slices) - { - if (slices == null) throw new ArgumentNullException(nameof(slices)); - - // use optimized version for arrays - var array = slices as Slice[]; - if (array != null) return ConcatRange(array); - - var next = new List(); - var writer = SliceWriter.Empty; - - //TODO: use multiple buffers if item count is huge ? - - foreach (var slice in slices) - { - writer.WriteBytes(this); - writer.WriteBytes(slice); - next.Add(writer.Position); - } - - return FdbKey.SplitIntoSegments(writer.Buffer, 0, next); - - } - - /// Concatenate two slices together - public static Slice Concat(Slice a, Slice b) - { - return a.Concat(b); - } - - /// Concatenate three slices together - public static Slice Concat(Slice a, Slice b, Slice c) - { - int count = a.Count + b.Count + c.Count; - if (count == 0) return Slice.Empty; - var writer = new SliceWriter(count); - writer.WriteBytes(a); - writer.WriteBytes(b); - writer.WriteBytes(c); - return writer.ToSlice(); - } - - /// Concatenate an array of slices into a single slice - public static Slice Concat(params Slice[] args) - { - int count = 0; - for (int i = 0; i < args.Length; i++) count += args[i].Count; - if (count == 0) return Slice.Empty; - var writer = new SliceWriter(count); - for (int i = 0; i < args.Length; i++) writer.WriteBytes(args[i]); - return writer.ToSlice(); - } - - /// Adds a prefix to a list of slices - /// Prefix to add to all the slices - /// List of slices to process - /// Array of slice that all start with and followed by the corresponding entry in - /// This method is optmized to reduce the amount of memory allocated - public static Slice[] ConcatRange(Slice prefix, IEnumerable slices) - { - if (slices == null) throw new ArgumentNullException(nameof(slices)); - - if (prefix.IsNullOrEmpty) - { // nothing to do, but we still need to copy the array - return slices.ToArray(); - } - - Slice[] res; - Slice[] arr; - ICollection coll; - - if ((arr = slices as Slice[]) != null) - { // fast-path for arrays (most frequent with range reads) - - // we wil use a SliceBuffer to store all the keys produced in as few byte[] arrays as needed - - // precompute the exact size needed - int totalSize = prefix.Count * arr.Length; - for (int i = 0; i < arr.Length; i++) totalSize += arr[i].Count; - var buf = new SliceBuffer(Math.Min(totalSize, 64 * 1024)); - - res = new Slice[arr.Length]; - for (int i = 0; i < arr.Length; i++) - { - res[i] = buf.Intern(prefix, arr[i], aligned: false); - } - } - else if ((coll = slices as ICollection) != null) - { // collection (size known) - - //TODO: also use a SliceBuffer since we could precompute the total size... - - res = new Slice[coll.Count]; - int p = 0; - foreach (var suffix in coll) - { - res[p++] = prefix.Concat(suffix); - } - } - else - { // streaming sequence (size unknown) - - //note: we can only scan the list once, so would be no way to get a sensible value for the buffer's page size - var list = new List(); - foreach (var suffix in slices) - { - list.Add(prefix.Concat(suffix)); - } - res = list.ToArray(); - } - - return res; - } - - /// Implicitly converts a Slice into an ArraySegment<byte> - public static implicit operator ArraySegment(Slice value) - { - if (!value.HasValue) return default(ArraySegment); - return new ArraySegment(value.Array, value.Offset, value.Count); - } - - /// Implicitly converts an ArraySegment<byte> into a Slice - public static implicit operator Slice(ArraySegment value) - { - return new Slice(value.Array, value.Offset, value.Count); - } - - #region Slice arithmetics... - - /// Compare two slices for equality - /// True if the slices contains the same bytes - public static bool operator ==(Slice a, Slice b) - { - return a.Equals(b); - } - - /// Compare two slices for inequality - /// True if the slice do not contain the same bytes - public static bool operator !=(Slice a, Slice b) - { - return !a.Equals(b); - } - - /// Compare two slices - /// True if is lexicographically less than ; otherwise, false. - public static bool operator <(Slice a, Slice b) - { - return a.CompareTo(b) < 0; - } - - /// Compare two slices - /// True if is lexicographically less than or equal to ; otherwise, false. - public static bool operator <=(Slice a, Slice b) - { - return a.CompareTo(b) <= 0; - } - - /// Compare two slices - /// True if is lexicographically greater than ; otherwise, false. - public static bool operator >(Slice a, Slice b) - { - return a.CompareTo(b) > 0; - } - - /// Compare two slices - /// True if is lexicographically greater than or equal to ; otherwise, false. - public static bool operator >=(Slice a, Slice b) - { - return a.CompareTo(b) >= 0; - } - - /// Append/Merge two slices together - /// First slice - /// Second slice - /// Merged slices if both slices are contigous, or a new slice containg the content of the first slice, followed by the second - public static Slice operator +(Slice a, Slice b) - { - return a.Concat(b); - } - - /// Appends a byte at the end of the slice - /// First slice - /// Byte to append at the end - /// New slice with the byte appended - public static Slice operator +(Slice a, byte b) - { - if (a.Count == 0) return Slice.FromByte(b); - var tmp = new byte[a.Count + 1]; - SliceHelpers.CopyBytesUnsafe(tmp, 0, a.Array, a.Offset, a.Count); - tmp[a.Count] = b; - return new Slice(tmp, 0, tmp.Length); - } - - /// Remove bytes at the end of slice - /// Smaller slice - public static Slice operator -(Slice s, int n) - { - if (n < 0) throw new ArgumentOutOfRangeException(nameof(n), "Cannot subtract a negative number from a slice"); - if (n > s.Count) throw new ArgumentOutOfRangeException(nameof(n), "Cannout substract more bytes than the slice contains"); - - if (n == 0) return s; - if (n == s.Count) return Slice.Empty; - - return new Slice(s.Array, s.Offset, s.Count - n); - } - - // note: We also need overloads with Nullable's to be able to do things like "if (slice == null)", "if (slice != null)" or "if (null != slice)". - // For structs that have "==" / "!=" operators, the compiler will think that when you write "slice == null", you really mean "(Slice?)slice == default(Slice?)", and that would ALWAYS false if you don't have specialized overloads to intercept. - - /// Determines whether two specified instances of are equal - public static bool operator ==(Slice? a, Slice? b) - { - return a.GetValueOrDefault().Equals(b.GetValueOrDefault()); - } - - /// Determines whether two specified instances of are not equal - public static bool operator !=(Slice? a, Slice? b) - { - return !a.GetValueOrDefault().Equals(b.GetValueOrDefault()); - } - - /// Determines whether one specified is less than another specified . - public static bool operator <(Slice? a, Slice? b) - { - return a.GetValueOrDefault() < b.GetValueOrDefault(); - } - - /// Determines whether one specified is less than or equal to another specified . - public static bool operator <=(Slice? a, Slice? b) - { - return a.GetValueOrDefault() <= b.GetValueOrDefault(); - } - - /// Determines whether one specified is greater than another specified . - public static bool operator >(Slice? a, Slice? b) - { - return a.GetValueOrDefault() > b.GetValueOrDefault(); - } - - /// Determines whether one specified is greater than or equal to another specified . - public static bool operator >=(Slice? a, Slice? b) - { - return a.GetValueOrDefault() >= b.GetValueOrDefault(); - } - - /// Concatenates two together. - public static Slice operator +(Slice? a, Slice? b) - { - // note: makes "slice + null" work! - return a.GetValueOrDefault().Concat(b.GetValueOrDefault()); - } - - #endregion - - /// Returns a printable representation of the key - /// You can roundtrip the result of calling slice.ToString() by passing it to (string) and get back the original slice. - public override string ToString() - { - return Slice.Dump(this); - } - - /// Returns a printable representation of a key - /// This may not be efficient, so it should only be use for testing/logging/troubleshooting - [NotNull] - public static string Dump(Slice value) - { - const int MAX_SIZE = 1024; - - if (value.Count == 0) return value.HasValue ? "" : ""; - - SliceHelpers.EnsureSliceIsValid(ref value); - - var buffer = value.Array; - int count = Math.Min(value.Count, MAX_SIZE); - int pos = value.Offset; - - var sb = new StringBuilder(count + 16); - while (count-- > 0) - { - int c = buffer[pos++]; - if (c < 32 || c >= 127 || c == 60) - { - sb.Append('<'); - int x = c >> 4; - sb.Append((char)(x + (x < 10 ? 48 : 55))); - x = c & 0xF; - sb.Append((char)(x + (x < 10 ? 48 : 55))); - sb.Append('>'); - } - else - { - sb.Append((char)c); - } - } - if (value.Count > MAX_SIZE) sb.Append("[...]"); - return sb.ToString(); - } - - /// Decode the string that was generated by slice.ToString() or Slice.Dump(), back into the original slice - /// This may not be efficient, so it should only be use for testing/logging/troubleshooting - public static Slice Unescape(string value) - { - var writer = SliceWriter.Empty; - for (int i = 0; i < value.Length; i++) - { - char c = value[i]; - if (c == '<') - { - if (value[i + 3] != '>') throw new FormatException(String.Format("Invalid escape character at offset {0}", i)); - c = (char)(NibbleToDecimal(value[i + 1]) << 4 | NibbleToDecimal(value[i + 2])); - i += 3; - } - writer.WriteByte((byte)c); - } - return writer.ToSlice(); - } - - #region Streams... - - /// Read the content of a stream into a slice - /// Source stream, that must be in a readable state - /// Slice containing the stream content (or if the stream is ) - /// If is null. - /// If the size of the stream exceeds or if it does not support reading. - public static Slice FromStream([NotNull] Stream data) - { - if (data == null) throw new ArgumentNullException(nameof(data)); - - // special case for empty values - if (data == Stream.Null) return Slice.Nil; - if (!data.CanRead) throw new InvalidOperationException("Cannot read from provided stream"); - - if (data.Length == 0) return Slice.Empty; - if (data.Length > int.MaxValue) throw new InvalidOperationException("Streams of more than 2GB are not supported"); - //TODO: other checks? - - int length; - checked { length = (int)data.Length; } - - if (data is MemoryStream || data is UnmanagedMemoryStream) // other types of already completed streams ? - { // read synchronously - return LoadFromNonBlockingStream(data, length); - } - - // read asynchronoulsy - return LoadFromBlockingStream(data, length); - } - - /// Asynchronously read the content of a stream into a slice - /// Source stream, that must be in a readable state - /// Optional cancellation token for this operation - /// Slice containing the stream content (or if the stream is ) - /// If is null. - /// If the size of the stream exceeds or if it does not support reading. - public static Task FromStreamAsync([NotNull] Stream data, CancellationToken ct) - { - if (data == null) throw new ArgumentNullException(nameof(data)); - - // special case for empty values - if (data == Stream.Null) return Task.FromResult(Slice.Nil); - if (!data.CanRead) throw new InvalidOperationException("Cannot read from provided stream"); - - if (data.Length == 0) return Task.FromResult(Slice.Empty); - if (data.Length > int.MaxValue) throw new InvalidOperationException("Streams of more than 2GB are not supported"); - //TODO: other checks? - - if (ct.IsCancellationRequested) return Task.FromCanceled(ct); - - int length; - checked { length = (int)data.Length; } - - if (data is MemoryStream || data is UnmanagedMemoryStream) // other types of already completed streams ? - { // read synchronously - return Task.FromResult(LoadFromNonBlockingStream(data, length)); - } - - // read asynchronoulsy - return LoadFromBlockingStreamAsync(data, length, 0, ct); - } - - /// Read from a non-blocking stream that already contains all the data in memory (MemoryStream, UnmanagedStream, ...) - /// Source stream - /// Number of bytes to read from the stream - /// Slice containing the loaded data - private static Slice LoadFromNonBlockingStream([NotNull] Stream source, int length) - { - Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue); - - var ms = source as MemoryStream; - if (ms != null) - { // Already holds onto a byte[] - - //note: should be use GetBuffer() ? It can throws and is dangerous (could mutate) - return Slice.Create(ms.ToArray()); - } - - // read it in bulk, without buffering - - var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? - - // note: reading should usually complete with only one big read, but loop until completed, just to be sure - int p = 0; - int r = length; - while (r > 0) - { - int n = source.Read(buffer, p, r); - if (n <= 0) throw new InvalidOperationException(String.Format("Unexpected end of stream at {0} / {1} bytes", p, length)); - p += n; - r -= n; - } - Contract.Assert(r == 0 && p == length); - - return Slice.Create(buffer); - } - - /// Synchronously read from a blocking stream (FileStream, NetworkStream, ...) - /// Source stream - /// Number of bytes to read from the stream - /// If non zero, max amount of bytes to read in one chunk. If zero, tries to read everything at once - /// Slice containing the loaded data - private static Slice LoadFromBlockingStream([NotNull] Stream source, int length, int chunkSize = 0) - { - Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue && chunkSize >= 0); - - if (chunkSize == 0) chunkSize = int.MaxValue; - - var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? - - // note: reading should usually complete with only one big read, but loop until completed, just to be sure - int p = 0; - int r = length; - while (r > 0) - { - int c = Math.Max(r, chunkSize); - int n = source.Read(buffer, p, c); - if (n <= 0) throw new InvalidOperationException(String.Format("Unexpected end of stream at {0} / {1} bytes", p, length)); - p += n; - r -= n; - } - Contract.Assert(r == 0 && p == length); - - return Slice.Create(buffer); - } - - /// Asynchronously read from a blocking stream (FileStream, NetworkStream, ...) - /// Source stream - /// Number of bytes to read from the stream - /// If non zero, max amount of bytes to read in one chunk. If zero, tries to read everything at once - /// Optional cancellation token for this operation - /// Slice containing the loaded data - private static async Task LoadFromBlockingStreamAsync([NotNull] Stream source, int length, int chunkSize, CancellationToken ct) - { - Contract.Requires(source != null && source.CanRead && source.Length <= int.MaxValue && chunkSize >= 0); - - if (chunkSize == 0) chunkSize = int.MaxValue; - - var buffer = new byte[length]; //TODO: round up to avoid fragmentation ? - - // note: reading should usually complete with only one big read, but loop until completed, just to be sure - int p = 0; - int r = length; - while (r > 0) - { - int c = Math.Min(r, chunkSize); - int n = await source.ReadAsync(buffer, p, c, ct); - if (n <= 0) throw new InvalidOperationException(String.Format("Unexpected end of stream at {0} / {1} bytes", p, length)); - p += n; - r -= n; - } - Contract.Assert(r == 0 && p == length); - - return Slice.Create(buffer); - } - - #endregion - - #region Equality, Comparison... - - /// Checks if an object is equal to the current slice - /// Object that can be either another slice, a byte array, or a byte array segment. - /// true if the object represents a sequence of bytes that has the same size and same content as the current slice. - public override bool Equals(object obj) - { - if (obj == null) return this.Array == null; - if (obj is Slice) return Equals((Slice)obj); - if (obj is ArraySegment) return Equals((ArraySegment)obj); - if (obj is byte[]) return Equals((byte[])obj); - return false; - } - - /// Gets the hash code for this slice - /// A 32-bit signed hash code calculated from all the bytes in the slice. - public override int GetHashCode() - { - SliceHelpers.EnsureSliceIsValid(ref this); - if (this.Array == null) return 0; - return SliceHelpers.ComputeHashCodeUnsafe(this.Array, this.Offset, this.Count); - } - - /// Checks if another slice is equal to the current slice. - /// Slice compared with the current instance - /// true if both slices have the same size and contain the same sequence of bytes; otherwise, false. - public bool Equals(Slice other) - { - SliceHelpers.EnsureSliceIsValid(ref other); - SliceHelpers.EnsureSliceIsValid(ref this); - - // note: Slice.Nil != Slice.Empty - if (this.Array == null) return other.Array == null; - if (other.Array == null) return false; - - return this.Count == other.Count && SliceHelpers.SameBytesUnsafe(this.Array, this.Offset, other.Array, other.Offset, this.Count); - } - - /// Lexicographically compare this slice with another one, and return an indication of their relative sort order - /// Slice to compare with this instance - /// Returns a NEGATIVE value if the current slice is LESS THAN , ZERO if it is EQUAL TO , and a POSITIVE value if it is GREATER THAN . - /// If both this instance and are Nil or Empty, the comparison will return ZERO. If only is Nil or Empty, it will return a NEGATIVE value. If only this instance is Nil or Empty, it will return a POSITIVE value. - public int CompareTo(Slice other) - { - if (this.Count == 0) return other.Count == 0 ? 0 : -1; - if (other.Count == 0) return +1; - SliceHelpers.EnsureSliceIsValid(ref other); - SliceHelpers.EnsureSliceIsValid(ref this); - return SliceHelpers.CompareBytesUnsafe(this.Array, this.Offset, this.Count, other.Array, other.Offset, other.Count); - } - - /// Checks if the content of a byte array segment matches the current slice. - /// Byte array segment compared with the current instance - /// true if both segment and slice have the same size and contain the same sequence of bytes; otherwise, false. - public bool Equals(ArraySegment other) - { - return this.Count == other.Count && SliceHelpers.SameBytes(this.Array, this.Offset, other.Array, other.Offset, this.Count); - } - - /// Checks if the content of a byte array matches the current slice. - /// Byte array compared with the current instance - /// true if the both array and slice have the same size and contain the same sequence of bytes; otherwise, false. - public bool Equals(byte[] other) - { - if (other == null) return this.Array == null; - return this.Count == other.Length && SliceHelpers.SameBytes(this.Array, this.Offset, other, 0, this.Count); - } - - #endregion - - [UsedImplicitly(ImplicitUseTargetFlags.WithMembers)] - private sealed class DebugView - { - private readonly Slice m_slice; - - public DebugView(Slice slice) - { - m_slice = slice; - } - - public byte[] Data - { - get - { - if (m_slice.Count == 0) return m_slice.Array == null ? null : EmptyArray; - if (m_slice.Offset == 0 && m_slice.Count == m_slice.Array.Length) return m_slice.Array; - var tmp = new byte[m_slice.Count]; - System.Array.Copy(m_slice.Array, m_slice.Offset, tmp, 0, m_slice.Count); - return tmp; - } - } - - public string Text - { - get - { - if (m_slice.Count == 0) return m_slice.Array == null ? null : String.Empty; - return m_slice.ToAsciiOrHexaString(); - } - } - - public int Count - { - get { return m_slice.Count; } - } - - } - - } - -} diff --git a/FoundationDB.Client/Utils/SliceComparer.cs b/FoundationDB.Client/Utils/SliceComparer.cs deleted file mode 100644 index d640bc6cb..000000000 --- a/FoundationDB.Client/Utils/SliceComparer.cs +++ /dev/null @@ -1,79 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB -{ - using System; - using System.Collections.Generic; - - /// Performs optimized equality and comparison checks on Slices - public sealed class SliceComparer : IComparer, IEqualityComparer - { - /// Default instance of the slice comparator - public static readonly SliceComparer Default = new SliceComparer(); - - private SliceComparer() - { } - - /// Lexicographically compare two slices and returns an indication of their relative sort order. - /// Slice compared with - /// Slice compared with - /// Returns a NEGATIVE value if is LESS THAN , ZERO if is EQUAL TO , and a POSITIVE value if is GREATER THAN . - /// - /// If both and are nil or empty, the comparison will return ZERO. If only is nil or empty, it will return a NEGATIVE value. If only is nil or empty, it will return a POSITIVE value. - /// There are no guarantees that non-zero results will be exactly -1 or +1. You should always use comparison operators or the sign of the returned value, instead of testing for equality with -1 or +1. - /// - public int Compare(Slice x, Slice 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); - } - - /// Checks if two slices are equal. - /// Slice compared with - /// Slice compared with - /// true if and have the same size and contain the same sequence of bytes; otherwise, false. - public bool Equals(Slice x, Slice y) - { - return x.Count == y.Count && SliceHelpers.SameBytes(x.Array, x.Offset, y.Array, y.Offset, y.Count); - } - - /// Computes the hash code of a slice - /// A slice - /// A 32-bit signed hash coded calculated from all the bytes in the slice - public int GetHashCode(Slice obj) - { - if (obj.Array == null) return 0; - return SliceHelpers.ComputeHashCode(obj.Array, obj.Offset, obj.Count); - } - - } - -} diff --git a/FoundationDB.Client/Utils/SliceHelpers.cs b/FoundationDB.Client/Utils/SliceHelpers.cs deleted file mode 100644 index 8148c001d..000000000 --- a/FoundationDB.Client/Utils/SliceHelpers.cs +++ /dev/null @@ -1,734 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -#define USE_NATIVE_MEMORY_OPERATORS - -namespace FoundationDB -{ - using System; - using System.Runtime.CompilerServices; - using System.Runtime.ConstrainedExecution; - using System.Runtime.InteropServices; - using System.Security; - using Doxense.Diagnostics.Contracts; - using JetBrains.Annotations; - - internal static class SliceHelpers - { - - public static void EnsureSliceIsValid(ref Slice slice) - { - 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) - { - ThrowMalformedSlice(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) - { -#if DEBUG - // If you break here, that means that a slice is invalid (negative count, offset, ...), which may be a sign of memory corruption! - // You should walk up the stack to see what is going on ! - if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); -#endif - - if (slice.Offset < 0) throw new FormatException("The specified slice has a negative offset, which is not legal. This may be a side effect of memory corruption."); - if (slice.Count < 0) throw new FormatException("The specified slice has a negative size, which is not legal. This may be a side effect of memory corruption."); - if (slice.Count > 0) - { - if (slice.Array == null) throw new FormatException("The specified slice is missing its underlying buffer."); - if (slice.Offset + slice.Count > slice.Array.Length) throw new FormatException("The specified slice is larger than its underlying buffer."); - } - // maybe it's Lupus ? - throw new FormatException("The specified slice is invalid."); - } - - public static void EnsureBufferIsValid(byte[] array, int offset, int count) - { - if (count == 0 && offset >= 0) return; - if (count < 0 || offset < 0 || array == null || offset + count > array.Length) - { - ThrowMalformedBuffer(array, offset, 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) - { - 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"); - if (count > 0) - { - if (array == null) throw new ArgumentException("The specified segment is missing its underlying buffer.", "array"); - if (offset + count > array.Length) throw new ArgumentException("The specified segment is larger than its underlying buffer.", "count"); - } - // maybe it's Lupus ? - throw new ArgumentException("The specified segment is invalid."); - } - - /// Round a size to a multiple of 16 - /// Minimum size required - /// Size rounded up to the next multiple of 16 - /// If the rounded size overflows over 2 GB - public static int Align(int size) - { - const int ALIGNMENT = 16; // MUST BE A POWER OF TWO! - const int MASK = (-ALIGNMENT) & int.MaxValue; - - if (size <= ALIGNMENT) - { - if (size < 0) throw new ArgumentOutOfRangeException("size", "Size cannot be negative"); - return ALIGNMENT; - } - // force an exception if we overflow above 2GB - checked { return (size + (ALIGNMENT - 1)) & MASK; } - } - - /// Round a number to the next power of 2 - /// Positive integer that will be rounded up (if not already a power of 2) - /// Smallest power of 2 that is greater then or equal to - /// Will return 1 for = 0 (because 0 is not a power 2 !), and will throws for < 0 - /// If is a negative number - public static int NextPowerOfTwo(int x) - { - // cf http://en.wikipedia.org/wiki/Power_of_two#Algorithm_to_round_up_to_power_of_two - - // special case - if (x == 0) return 1; - if (x < 0) throw new ArgumentOutOfRangeException("x", x, "Cannot compute the next power of two for negative numbers"); - //TODO: check for overflow at if x > 2^30 ? - - --x; - x |= (x >> 1); - x |= (x >> 2); - x |= (x >> 4); - x |= (x >> 8); - x |= (x >> 16); - return x + 1; - } - - /// Compute the hash code of a byte segment - /// Buffer - /// Offset of the start of the segment in the buffer - /// Number of bytes in the segment - /// A 32-bit signed hash code calculated from all the bytes in the segment. - public static int ComputeHashCode([NotNull] byte[] bytes, int offset, int count) - { - if (bytes == null || offset < 0 || count < 0 || offset + count > bytes.Length) SliceHelpers.ThrowMalformedBuffer(bytes, offset, count); - - return ComputeHashCodeUnsafe(bytes, offset, count); - } - - /// Compute the hash code of a byte segment, without validating the arguments - /// Buffer - /// Offset of the start of the segment in the buffer - /// Number of bytes in the segment - /// A 32-bit signed hash code calculated from all the bytes in the segment. - public static int ComputeHashCodeUnsafe([NotNull] byte[] bytes, int offset, int count) - { - Contract.Requires(bytes != null && offset >= 0 && count >= 0); - - //TODO: use a better hash algorithm? (xxHash, CityHash, SipHash, ...?) - // => will be called a lot when Slices are used as keys in an hash-based dictionary (like Dictionary) - // => won't matter much for *ordered* dictionary that will probably use IComparer.Compare(..) instead of the IEqalityComparer.GetHashCode()/Equals() combo - // => we don't need a cryptographic hash, just something fast and suitable for use with hashtables... - // => probably best to select an algorithm that works on 32-bit or 64-bit chunks - - // : unoptimized 32 bits FNV-1a implementation - uint h = 2166136261; // FNV1 32 bits offset basis - int p = offset; - int n = count; - while (n-- > 0) - { - h = (h ^ bytes[p++]) * 16777619; // FNV1 32 prime - } - return (int)h; - // - } - - /// Compare two byte segments for equality - /// Left buffer - /// Start offset in left buffer - /// Right buffer - /// Start offset in right buffer - /// Number of bytes to compare - /// true if all bytes are the same in both segments - public static bool SameBytes(byte[] left, int leftOffset, byte[] right, int rightOffset, int count) - { - SliceHelpers.EnsureBufferIsValid(left, leftOffset, count); - SliceHelpers.EnsureBufferIsValid(right, rightOffset, count); - - if (left == null || right == null) return left == right; - return SameBytesUnsafe(left, leftOffset, right, rightOffset, count); - } - - /// Compare two byte segments for equality, without validating the arguments - /// Left buffer - /// Start offset in left buffer - /// Right buffer - /// Start offset in right buffer - /// Number of bytes to compare - /// true if all bytes are the same in both segments - 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 - { - unsafe - { - fixed (byte* pLeft = left) - fixed (byte* pRight = right) - { - return 0 == CompareMemoryUnsafe(pLeft + leftOffset, pRight + rightOffset, 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); - } - - /// Compare two byte segments lexicographically, without validating the arguments - /// 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 CompareBytesUnsafe([NotNull] byte[] left, int leftOffset, int leftCount, [NotNull] byte[] right, int rightOffset, int rightCount) - { - 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 - { - unsafe - { - 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; - } - } - } - } - - /// 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); - } - - /// 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 - /// 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 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 - - unsafe - { - fixed (byte* ptr = dst) - { - MoveMemoryUnsafe(ptr + dstOffset, ptr + srcOffset, count); - } - } - } - else - { - unsafe - { - fixed (byte* pDst = dst) - fixed (byte* pSrc = src) - { - MoveMemoryUnsafe(pDst + dstOffset, pSrc + srcOffset, count); - } - } - } - } - - /// Copy the content of a native byte segment into a managed segment, without validating the arguments. - /// Destination buffer - /// Offset in destination buffer - /// Point to the 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 unsafe void CopyBytesUnsafe([NotNull] byte[] dst, int dstOffset, byte* src, int count) - { - Contract.Requires(dst != null && src != null && dstOffset >= 0 && count >= 0); - - if (count <= 8) - { - while (count-- > 0) - { - dst[dstOffset++] = *src++; - } - } - else - { - fixed (byte* ptr = dst) - { - MoveMemoryUnsafe(ptr + dstOffset, src, 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); - } - - /// Fill the content of a managed segment with the same byte repeated - 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 - { - fixed (byte* ptr = bytes) - { - SetMemoryUnsafe(ptr + offset, value, count); - } - } - } - } - - /// Fill the content of a native byte segment with the same byte repeated - 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); - } - } - - /// 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 - { - - /// Compare characters in two buffers. - /// First buffer. - /// Second buffer. - /// Number of bytes to compare. - /// The return value indicates the relationship between the buffers. - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern int memcmp(byte* buf1, byte* buf2, IntPtr count); - - /// Moves one buffer to another. - /// Destination object. - /// Source object. - /// Number of bytes to copy. - /// The value of dest. - /// Copies count bytes from src to dest. If some regions of the source area and the destination overlap, both functions ensure that the original source bytes in the overlapping region are copied before being overwritten. - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern byte* memmove(byte* dest, byte* src, IntPtr count); - - /// Sets the first bytes of to the byte . - /// Pointer to destination - /// Byte to set - /// Number of bytes - /// The value of - [DllImport("msvcrt.dll", CallingConvention = CallingConvention.Cdecl, SetLastError = false)] - public static extern byte* memset(byte* dest, int c, IntPtr count); - - } - -#endif - - } - -} diff --git a/FoundationDB.Client/Utils/SliceReader.cs b/FoundationDB.Client/Utils/SliceReader.cs deleted file mode 100644 index 057056895..000000000 --- a/FoundationDB.Client/Utils/SliceReader.cs +++ /dev/null @@ -1,282 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB -{ - using JetBrains.Annotations; - using System; - - /// Helper class that holds the internal state used to parse tuples from slices - public struct SliceReader - { - - /// Creates a reader on a byte array - public static SliceReader FromBuffer(byte[] buffer) - { - return new SliceReader(Slice.Create(buffer)); - } - - /// Creates a reader on a segment of a byte array - public static SliceReader FromBuffer(byte[] buffer, int offset, int count) - { - return new SliceReader(Slice.Create(buffer, offset, count)); - } - - /// Buffer containing the tuple being parsed - public Slice Buffer { get { return m_buffer; } } - private Slice m_buffer; //PERF: readonly struct - - /// Current position inside the buffer - public int Position; - - /// Creates a new reader over a slice - /// Slice that will be used as the underlying buffer - public SliceReader(Slice buffer) - { - m_buffer = buffer; - this.Position = 0; - } - - public SliceReader(byte[] buffer) - { - m_buffer = new Slice(buffer, 0, buffer.Length); - this.Position = 0; - } - - public SliceReader(byte[] buffer, int offset, int count) - { - m_buffer = new Slice(buffer, offset, count); - this.Position = 0; - } - - /// Returns true if there are more bytes to parse - public bool HasMore { get { return this.Position < m_buffer.Count; } } - - /// Returns the number of bytes remaining - public int Remaining { get { return Math.Max(0, m_buffer.Count - this.Position); } } - - /// Returns a slice with all the bytes read so far in the buffer - public Slice Head - { - get { return m_buffer.Substring(0, this.Position); } - } - - /// Returns a slice with all the remaining bytes in the buffer - public Slice Tail - { - get { return m_buffer.Substring(this.Position); } - } - - /// Ensure that there are at least bytes remaining in the buffer - public void EnsureBytes(int count) - { - if (count < 0 || checked(this.Position + count) > m_buffer.Count) throw new ArgumentOutOfRangeException("count"); - } - - /// Return the value of the next byte in the buffer, or -1 if we reached the end - [Pure] - public int PeekByte() - { - int p = this.Position; - return p < m_buffer.Count ? m_buffer[p] : -1; - } - - /// Return the value of the byte at a specified offset from the current position, or -1 if this is after the end, or before the start - [Pure] - public int PeekByteAt(int offset) - { - int p = this.Position + offset; - return p < m_buffer.Count && p >= 0 ? m_buffer[p] : -1; - } - - /// Skip the next bytes of the buffer - public void Skip(int count) - { - EnsureBytes(count); - - this.Position += count; - } - - /// Read the next byte from the buffer - public byte ReadByte() - { - EnsureBytes(1); - - int p = this.Position; - byte b = m_buffer[p]; - this.Position = p + 1; - return b; - } - - /// Read the next bytes from the buffer - public Slice ReadBytes(int count) - { - EnsureBytes(count); - - int p = this.Position; - this.Position = p + count; - return m_buffer.Substring(p, count); - } - - /// Read the next 2 bytes as an unsigned 16-bit integer, encoded in little-endian - public ushort ReadFixed16() - { - return ReadBytes(2).ToUInt16(); - } - - /// Read the next 4 bytes as an unsigned 32-bit integer, encoded in little-endian - public uint ReadFixed32() - { - return ReadBytes(4).ToUInt32(); - } - - /// Read the next 8 bytes as an unsigned 64-bit integer, encoded in little-endian - public ulong ReadFixed64() - { - return ReadBytes(8).ToUInt64(); - } - - /// Read the next 2 bytes as an unsigned 16-bit integer, encoded in big-endian - public ushort ReadFixed16BE() - { - return ReadBytes(2).ToUInt16BE(); - } - - /// Read the next 4 bytes as an unsigned 32-bit integer, encoded in big-endian - public uint ReadFixed32BE() - { - return ReadBytes(4).ToUInt32BE(); - } - - /// Read the next 8 bytes as an unsigned 64-bit integer, encoded in big-endian - public ulong ReadFixed64BE() - { - return ReadBytes(8).ToUInt64BE(); - } - - /// Read an encoded nul-terminated byte array from the buffer - public Slice ReadByteString() - { - var buffer = m_buffer.Array; - int start = m_buffer.Offset + this.Position; - int p = start; - int end = m_buffer.Offset + m_buffer.Count; - - while (p < end) - { - byte b = buffer[p++]; - if (b == 0) - { - //TODO: decode \0\xFF ? - if (p < end && buffer[p] == 0xFF) - { - // skip the next byte and continue - p++; - continue; - } - - this.Position = p - m_buffer.Offset; - return new Slice(buffer, start, p - start); - } - } - - throw new FormatException("Truncated byte string (expected terminal NUL not found)"); - } - - /// Reads a 7-bit encoded unsigned int (aka 'Varint16') from the buffer, and advances the cursor - /// Can Read up to 3 bytes from the input - public ushort ReadVarint16() - { - //note: this could read up to 21 bits of data, so we check for overflow - return checked((ushort)ReadVarint(3)); - } - - /// Reads a 7-bit encoded unsigned int (aka 'Varint32') from the buffer, and advances the cursor - /// Can Read up to 5 bytes from the input - public uint ReadVarint32() - { - //note: this could read up to 35 bits of data, so we check for overflow - return checked((uint)ReadVarint(5)); - } - - /// Reads a 7-bit encoded unsigned long (aka 'Varint32') from the buffer, and advances the cursor - /// Can Read up to 10 bytes from the input - public ulong ReadVarint64() - { - return ReadVarint(10); - } - - /// Reads a Base 128 Varint from the input - /// Maximum number of bytes allowed (5 for 32 bits, 10 for 64 bits) - private ulong ReadVarint(int count) - { - var buffer = m_buffer.Array; - int p = m_buffer.Offset + this.Position; - int end = m_buffer.Offset + m_buffer.Count; - - ulong x = 0; - int s = 0; - - // read bytes until the MSB is unset - while (count-- > 0) - { - if (p > end) throw new FormatException("Truncated Varint"); - byte b = buffer[p++]; - - x |= (b & 0x7FUL) << s; - if (b < 0x80) - { - this.Position = p - m_buffer.Offset; - return x; - } - s += 7; - } - throw new FormatException("Malformed Varint"); - } - - /// Reads a variable sized slice, by first reading its size (stored as a Varint32) and then the data - public Slice ReadVarbytes() - { - uint size = ReadVarint32(); - if (size > int.MaxValue) throw new FormatException("Malformed variable size"); - if (size == 0) return Slice.Empty; - return ReadBytes((int)size); - } - - public Uuid128 ReadUuid128() - { - return ReadBytes(16).ToUuid128(); - } - - public Uuid64 ReadUuid64() - { - return ReadBytes(8).ToUuid64(); - } - } - -} diff --git a/FoundationDB.Client/Utils/SliceWriter.cs b/FoundationDB.Client/Utils/SliceWriter.cs deleted file mode 100644 index 4b35a8fe0..000000000 --- a/FoundationDB.Client/Utils/SliceWriter.cs +++ /dev/null @@ -1,879 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB -{ - using System; - using System.Diagnostics; - using System.Runtime.CompilerServices; - using Doxense.Diagnostics.Contracts; - using JetBrains.Annotations; - - /// Slice buffer that emulates a pseudo-stream using a byte array that will automatically grow in size, if necessary - /// IMPORTANT: This struct does not extensively check the parameters! The caller should ensure that everything is valid (this is to get the max performance when serializing keys and values) - [DebuggerDisplay("Position={Position}, Capacity={Buffer == null ? -1 : Buffer.Length}"), DebuggerTypeProxy(typeof(SliceWriter.DebugView))] - public struct SliceWriter - { - // Invariant - // * Valid data always start at offset 0 - // * 'this.Position' is equal to the current size as well as the offset of the next available free spot - // * 'this.Buffer' is either null (meaning newly created stream), or is at least as big as this.Position - - #region Private Members... - - /// Buffer holding the data - public byte[] Buffer; - - /// Position in the buffer ( == number of already written bytes) - public int Position; - - #endregion - - #region Constructors... - - /// Returns a new, empty, slice writer - public static SliceWriter Empty { get { return default(SliceWriter); } } - - /// Create a new empty binary buffer with an initial allocated size - /// Initial capacity of the buffer - public SliceWriter(int capacity) - { - if (capacity < 0) throw new ArgumentOutOfRangeException("capacity"); - - this.Buffer = new byte[capacity]; - this.Position = 0; - } - - /// Create a new binary writer using an existing buffer - /// Initial buffer - /// Since the content of the will be modified, only a temporary or scratch buffer should be used. If the writer needs to grow, a new buffer will be allocated. - public SliceWriter([NotNull] byte[] buffer) - : this(buffer, 0) - { } - - /// Create a new binary buffer using an existing buffer and with the cursor to a specific location - /// Since the content of the will be modified, only a temporary or scratch buffer should be used. If the writer needs to grow, a new buffer will be allocated. - public SliceWriter([NotNull] byte[] buffer, int index) - { - if (buffer == null) throw new ArgumentNullException("buffer"); - if (index < 0 || index > buffer.Length) throw new ArgumentOutOfRangeException("index"); - - this.Buffer = buffer; - this.Position = index; - } - - /// Creates a new binary buffer, initialized by copying pre-existing data - /// Data that will be copied at the start of the buffer - /// Optional initial capacity of the buffer - /// The cursor will already be placed at the end of the prefix - public SliceWriter(Slice prefix, int capacity = 0) - { - if (capacity < 0) throw new ArgumentException("Capacity must be a positive integer.", "capacity"); - - int n = prefix.Count; - Contract.Assert(n >= 0); - - if (capacity == 0) - { // most frequent usage is to add a packed integer at the end of a prefix - capacity = SliceHelpers.Align(n + 8); - } - else - { - capacity = Math.Max(capacity, n); - } - - var buffer = new byte[capacity]; - if (n > 0) prefix.CopyTo(buffer, 0); - - this.Buffer = buffer; - this.Position = n; - } - - #endregion - - #region Public Properties... - - /// Returns true if the buffer contains at least some data - public bool HasData - { - get { return this.Position > 0; } - } - - /// Return the byte at the specified index - /// Index in the buffer (0-based if positive, from the end if negative) - public byte this[int index] - { - [Pure] - get - { - Contract.Assert(this.Buffer != null && this.Position >= 0); - //note: we will get bound checking for free in release builds - if (index < 0) index += this.Position; - if (index < 0 || index >= this.Position) throw new IndexOutOfRangeException(); - return this.Buffer[index]; - } - } - - /// Returns a slice pointing to a segment inside the buffer - /// The starting position of the substring. Positive values means from the start, negative values means from the end - /// The end position (excluded) of the substring. Positive values means from the start, negative values means from the end - /// Slice that corresponds to the section selected. If the if equal to or greater than then an empty Slice is returned - /// If either or is outside of the currently allocated buffer. - public Slice this[int? beginInclusive, int? endExclusive] - { - [Pure] - get - { - int from = beginInclusive ?? 0; - int until = endExclusive ?? this.Position; - - // remap negative indexes - if (from < 0) from += this.Position; - if (until < 0) until += this.Position; - - // bound check - if (from < 0 || from >= this.Position) throw new ArgumentOutOfRangeException("beginInclusive", "The start index must be inside the bounds of the buffer."); - if (until < 0 || until > this.Position) throw new ArgumentOutOfRangeException("endExclusive", "The end index must be inside the bounds of the buffer."); - - // chop chop - int count = until - from; - return count > 0 ? new Slice(this.Buffer, from, count) : Slice.Empty; - } - } - - #endregion - - /// Returns a byte array filled with the contents of the buffer - /// The buffer is copied in the byte array. And change to one will not impact the other - [Pure, NotNull] - public byte[] GetBytes() - { - Contract.Requires(this.Position >= 0); - - var bytes = new byte[this.Position]; - if (this.Position > 0) - { - Contract.Assert(this.Buffer != null && this.Buffer.Length >= this.Position); - SliceHelpers.CopyBytesUnsafe(bytes, 0, this.Buffer, 0, bytes.Length); - } - return bytes; - } - - /// Returns a slice pointing to the content of the buffer - /// Any change to the slice will change the buffer ! - [Pure] - public Slice ToSlice() - { - if (this.Buffer == null || this.Position == 0) - { - return Slice.Empty; - } - else - { - Contract.Assert(this.Buffer.Length >= this.Position); - return new Slice(this.Buffer, 0, this.Position); - } - } - - /// Returns a slice pointing to the first bytes of the buffer - /// Size of the segment - /// Any change to the slice will change the buffer ! - /// If is less than zero, or larger than the current buffer size - [Pure] - public Slice ToSlice(int count) - { - if (count < 0 || count > this.Position) throw new ArgumentException("count"); - - return count > 0 ? new Slice(this.Buffer, 0, count) : Slice.Empty; - } - - /// Returns a slice pointing to a segment inside the buffer - /// Offset of the segment from the start of the buffer - /// Any change to the slice will change the buffer ! - /// If is less then zero, or after the current position - [Pure] - public Slice Substring(int offset) - { - if (offset < 0 || offset > this.Position) throw new ArgumentException("Offset must be inside the buffer", "offset"); - - int count = this.Position - offset; - return count > 0 ? new Slice(this.Buffer, offset, this.Position - offset) : Slice.Empty; - } - - /// Returns a slice pointing to a segment inside the buffer - /// Offset of the segment from the start of the buffer - /// Size of the segment - /// Any change to the slice will change the buffer ! - /// If either or are less then zero, or do not fit inside the current buffer - [Pure] - public Slice Substring(int offset, int count) - { - if (offset < 0 || offset >= this.Position) throw new ArgumentException("Offset must be inside the buffer", "offset"); - if (count < 0 || offset + count > this.Position) throw new ArgumentException("The buffer is too small", "count"); - - return count > 0 ? new Slice(this.Buffer, offset, count) : Slice.Empty; - } - - /// Truncate the buffer by setting the cursor to the specified position. - /// New size of the buffer - /// If the buffer was smaller, it will be resized and filled with zeroes. If it was biffer, the cursor will be set to the specified position, but previous data will not be deleted. - public void SetLength(int position) - { - Contract.Requires(position >= 0); - - if (this.Position < position) - { - int missing = position - this.Position; - EnsureBytes(missing); - //TODO: native memset() ? - Array.Clear(this.Buffer, this.Position, missing); - } - this.Position = position; - } - - /// Delete the first N bytes of the buffer, and shift the remaining to the front - /// Number of bytes to remove at the head of the buffer - /// New size of the buffer (or 0 if it is empty) - /// This should be called after every successfull write to the underlying stream, to update the buffer. - public int Flush(int bytes) - { - if (bytes == 0) return this.Position; - if (bytes < 0) throw new ArgumentOutOfRangeException("bytes"); - - if (bytes < this.Position) - { // copy the left over data to the start of the buffer - int remaining = this.Position - bytes; - SliceHelpers.CopyBytesUnsafe(this.Buffer, 0, this.Buffer, bytes, remaining); - this.Position = remaining; - return remaining; - } - else - { - //REVIEW: should we throw if there are less bytes in the buffer than we want to flush ? - this.Position = 0; - return 0; - } - } - - /// Empties the current buffer after a succesfull write - /// Shrink the buffer if a lot of memory is wated - public void Reset() - { - if (this.Position > 0) - { - // reduce size ? - // If the buffer exceeds 4K and we used less than 1/8 of it the last time, we will "shrink" the buffer - if (this.Buffer.Length > 4096 && (this.Position << 3) <= Buffer.Length) - { // Shrink it - Buffer = new byte[SliceHelpers.NextPowerOfTwo(this.Position)]; - } - else - { // Clear it - //TODO: native memset() ? - Array.Clear(Buffer, 0, this.Position); - } - this.Position = 0; - } - } - - /// Advance the cursor of the buffer without writing anything, and return the previous position - /// Number of bytes to skip - /// Pad value (0xFF by default) - /// Position of the cursor BEFORE moving it. Can be used as a marker to go back later and fill some value - /// Will fill the skipped bytes with - public int Skip(int skip, byte pad = 0xFF) - { - Contract.Requires(skip > 0); - - EnsureBytes(skip); - var buffer = this.Buffer; - int p = this.Position; - for (int i = 0; i < skip; i++) - { - buffer[p + i] = pad; - } - this.Position = p + skip; - return p; - } - - /// Rewinds the cursor to a previous position in the buffer, while saving the current position - /// Will receive the current cursor position - /// Previous position in the buffer - public void Rewind(out int cursor, int position) - { - Contract.Requires(position >= 0 && position <= this.Position); - cursor = this.Position; - this.Position = position; - } - - /// Add a byte to the end of the buffer, and advance the cursor - /// Byte, 8 bits -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - public void WriteByte(byte value) - { - EnsureBytes(1); - this.Buffer[this.Position] = value; - ++this.Position; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void UnsafeWriteByte(byte value) - { - Contract.Requires(this.Buffer != null && this.Position < this.Buffer.Length); - this.Buffer[this.Position++] = value; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void WriteByte2(byte value1, byte value2) - { - EnsureBytes(2); - - int p = this.Position; - this.Buffer[p] = value1; - this.Buffer[p + 1] = value2; - this.Position = p + 2; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void UnsafeWriteByte2(byte value1, byte value2) - { - Contract.Requires(this.Buffer != null && this.Position + 1 < this.Buffer.Length); - int p = this.Position; - this.Buffer[p] = value1; - this.Buffer[p + 1] = value2; - this.Position = p + 2; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void WriteByte3(byte value1, byte value2, byte value3) - { - EnsureBytes(3); - - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - this.Position = p + 3; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void UnsafeWriteByte3(byte value1, byte value2, byte value3) - { - Contract.Requires(this.Buffer != null && this.Position + 2 < this.Buffer.Length); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - this.Position = p + 3; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void WriteByte4(byte value1, byte value2, byte value3, byte value4) - { - EnsureBytes(4); - - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - buffer[p + 3] = value4; - this.Position = p + 4; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void UnsafeWriteByte4(byte value1, byte value2, byte value3, byte value4) - { - Contract.Requires(this.Buffer != null && this.Position + 3 < this.Buffer.Length); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - buffer[p + 3] = value4; - this.Position = p + 4; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void WriteByte5(byte value1, byte value2, byte value3, byte value4, byte value5) - { - EnsureBytes(5); - - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - buffer[p + 3] = value4; - buffer[p + 4] = value5; - this.Position = p + 5; - } - -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - internal void UnsafeWriteByte5(byte value1, byte value2, byte value3, byte value4, byte value5) - { - Contract.Requires(this.Buffer != null && this.Position + 4 < this.Buffer.Length); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = value1; - buffer[p + 1] = value2; - buffer[p + 2] = value3; - buffer[p + 3] = value4; - buffer[p + 4] = value5; - this.Position = p + 5; - } - - /// Append a byte array to the end of the buffer - /// - public void WriteBytes(byte[] data) - { - if (data != null) - { - WriteBytes(data, 0, data.Length); - } - } - - /// Append a chunk of a byte array to the end of the buffer - /// - /// - /// - public void WriteBytes(byte[] data, int offset, int count) - { - SliceHelpers.EnsureBufferIsValid(data, offset, count); - - if (count > 0) - { - EnsureBytes(count); - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data, offset, count); - this.Position += count; - } - } - - /// Append a chunk of memory to the end of the buffer - public unsafe void WriteBytesUnsafe(byte* data, int count) - { - if (data == null) throw new ArgumentNullException("data"); - if (count < 0) throw new ArgumentOutOfRangeException("count"); - - if (count > 0) - { - EnsureBytes(count); - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data, count); - this.Position += count; - } - } - - internal void UnsafeWriteBytes(byte[] data, int offset, int count) - { - Contract.Requires(this.Buffer != null && this.Position >= 0 && data != null && count >= 0 && this.Position + count <= this.Buffer.Length && offset >= 0 && offset + count <= data.Length); - - if (count > 0) - { - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data, offset, count); - this.Position += count; - } - } - - /// Append a segment of bytes to the end of the buffer - public void WriteBytes(Slice data) - { - SliceHelpers.EnsureSliceIsValid(ref data); - - int n = data.Count; - if (n > 0) - { - EnsureBytes(n); - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data.Array, data.Offset, n); - this.Position += n; - } - } - - internal unsafe void WriteBytes(byte* data, int count) - { - if (count == 0) return; - if (data == null) throw new ArgumentNullException("data"); - if (count < 0) throw new ArgumentException("count"); - - EnsureBytes(count); - Contract.Assert(this.Buffer != null && this.Position >= 0 && this.Position + count <= this.Buffer.Length); - - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data, count); - this.Position += count; - } - - internal unsafe void UnsafeWriteBytes(byte* data, int count) - { - if (count <= 0) return; - - Contract.Requires(this.Buffer != null && this.Position >= 0 && data != null && count >= 0 && this.Position + count <= this.Buffer.Length); - - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, data, count); - this.Position += count; - } - - #region Fixed, Little-Endian - - /// Writes a 16-bit unsigned integer, using little-endian encoding - /// Advances the cursor by 2 bytes - public void WriteFixed16(uint value) - { - EnsureBytes(2); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)value; - buffer[p + 1] = (byte)(value >> 8); - this.Position = p + 2; - } - - /// Writes a 32-bit unsigned integer, using little-endian encoding - /// Advances the cursor by 4 bytes - public void WriteFixed32(uint value) - { - EnsureBytes(4); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)value; - buffer[p + 1] = (byte)(value >> 8); - buffer[p + 2] = (byte)(value >> 16); - buffer[p + 3] = (byte)(value >> 24); - this.Position = p + 4; - } - - /// Writes a 64-bit unsigned integer, using little-endian encoding - /// Advances the cursor by 8 bytes - public void WriteFixed64(ulong value) - { - EnsureBytes(8); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)value; - buffer[p + 1] = (byte)(value >> 8); - buffer[p + 2] = (byte)(value >> 16); - buffer[p + 3] = (byte)(value >> 24); - buffer[p + 4] = (byte)(value >> 32); - buffer[p + 5] = (byte)(value >> 40); - buffer[p + 6] = (byte)(value >> 48); - buffer[p + 7] = (byte)(value >> 56); - this.Position = p + 8; - } - - #endregion - - #region Fixed, Big-Endian - - /// Writes a 16-bit unsigned integer, using big-endian encoding - /// Advances the cursor by 2 bytes - public void WriteFixed16BE(uint value) - { - EnsureBytes(2); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)(value >> 8); - buffer[p + 1] = (byte)value; - this.Position = p + 2; - } - - /// Writes a 32-bit unsigned integer, using big-endian encoding - /// Advances the cursor by 4 bytes - public void WriteFixed32BE(uint value) - { - EnsureBytes(4); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)(value >> 24); - buffer[p + 1] = (byte)(value >> 16); - buffer[p + 2] = (byte)(value >> 8); - buffer[p + 3] = (byte)(value); - this.Position = p + 4; - } - - /// Writes a 64-bit unsigned integer, using big-endian encoding - /// Advances the cursor by 8 bytes - public void WriteFixed64BE(ulong value) - { - EnsureBytes(8); - var buffer = this.Buffer; - int p = this.Position; - buffer[p] = (byte)(value >> 56); - buffer[p + 1] = (byte)(value >> 48); - buffer[p + 2] = (byte)(value >> 40); - buffer[p + 3] = (byte)(value >> 32); - buffer[p + 4] = (byte)(value >> 24); - buffer[p + 5] = (byte)(value >> 16); - buffer[p + 6] = (byte)(value >> 8); - buffer[p + 7] = (byte)(value); - this.Position = p + 8; - } - - #endregion - - #region Variable size - - /// Writes a 7-bit encoded unsigned int (aka 'Varint16') at the end, and advances the cursor - public void WriteVarint16(ushort value) - { - const uint MASK = 128; - - if (value < (1 << 7)) - { - WriteByte((byte)value); - } - else if (value < (1 << 14)) - { - WriteByte2( - (byte)(value | MASK), - (byte)(value >> 7) - ); - } - else - { - WriteByte3( - (byte)(value | MASK), - (byte)((value >> 7) | MASK), - (byte)(value >> 14) - ); - } - } - - /// Writes a 7-bit encoded unsigned int (aka 'Varint32') at the end, and advances the cursor - public void WriteVarint32(uint value) - { - const uint MASK = 128; - - if (value < (1 << 7)) - { - WriteByte((byte)value); - } - else if (value < (1 << 14)) - { - WriteByte2( - (byte)(value | MASK), - (byte)(value >> 7) - ); - } - else if (value < (1 << 21)) - { - WriteByte3( - (byte)(value | MASK), - (byte)((value >> 7) | MASK), - (byte)(value >> 14) - ); - } - else if (value < (1 << 28)) - { - WriteByte4( - (byte)(value | MASK), - (byte)((value >> 7) | MASK), - (byte)((value >> 14) | MASK), - (byte)(value >> 21) - ); - } - else - { - WriteByte5( - (byte)(value | MASK), - (byte)((value >> 7) | MASK), - (byte)((value >> 14) | MASK), - (byte)((value >> 21) | MASK), - (byte)(value >> 28) - ); - } - } - - /// Writes a 7-bit encoded unsigned long (aka 'Varint64') at the end, and advances the cursor - public void WriteVarint64(ulong value) - { - const uint MASK = 128; - - // max size is 5 - EnsureBytes(value < (1 << 7) ? 1 : value < (1 << 14) ? 2 : value < (1 << 21) ? 3 : 10); - - var buffer = this.Buffer; - int p = this.Position; - while (value >= MASK) - { - buffer[p++] = (byte)((value & (MASK - 1)) | MASK); - value >>= 7; - } - - buffer[p++] = (byte)value; - this.Position = p; - } - - /// Writes a length-prefixed byte array, and advances the cursor - public void WriteVarbytes(Slice value) - { - //REVIEW: what should we do for Slice.Nil ? - - SliceHelpers.EnsureSliceIsValid(ref value); - int n = value.Count; - if (n < 128) - { - EnsureBytes(n + 1); - var buffer = this.Buffer; - int p = this.Position; - // write the count (single byte) - buffer[p] = (byte)n; - // write the bytes - if (n > 0) - { - SliceHelpers.CopyBytesUnsafe(buffer, p + 1, value.Array, value.Offset, n); - } - this.Position = p + n + 1; - } - else - { - // write the count - WriteVarint32((uint)value.Count); - // write the bytes - SliceHelpers.CopyBytesUnsafe(this.Buffer, this.Position, value.Array, value.Offset, n); - this.Position += n; - } - } - - #endregion - - /// Ensures that we can fit a specific amount of data at the end of the buffer - /// Number of bytes that will be written - /// If the buffer is too small, it will be resized -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - public void EnsureBytes(int count) - { - Contract.Requires(count >= 0); - if (Buffer == null || Position + count > Buffer.Length) - { - GrowBuffer(ref Buffer, Position + count); - } - Contract.Ensures(this.Buffer != null && this.Buffer.Length >= this.Position + count); - } - - /// Ensures that we can fit data at a specifc offset in the buffer - /// Offset into the buffer (from the start) - /// Number of bytes that will be written at this offset - /// If the buffer is too small, it will be resized -#if !NET_4_0 - [MethodImpl(MethodImplOptions.AggressiveInlining)] -#endif - public void EnsureOffsetAndSize(int offset, int count) - { - Contract.Requires(offset >= 0); - Contract.Requires(count >= 0); - - if (this.Buffer == null || offset + count > this.Buffer.Length) - { - GrowBuffer(ref this.Buffer, offset + count); - } - } - - /// Resize a buffer by doubling its capacity - /// Reference to the variable holding the buffer to create/resize. If null, a new buffer will be allocated. If not, the content of the buffer will be copied into the new buffer. - /// Mininum guaranteed buffer size after resizing. - /// The buffer will be resized to the maximum betweeb the previous size multiplied by 2, and . The capacity will always be rounded to a multiple of 16 to reduce memory fragmentation - public static void GrowBuffer(ref byte[] buffer, int minimumCapacity = 0) - { - Contract.Requires(minimumCapacity >= 0); - - // double the size of the buffer, or use the minimum required - long newSize = Math.Max(buffer == null ? 0 : (((long)buffer.Length) << 1), minimumCapacity); - - // .NET (as of 4.5) cannot allocate an array with more than 2^31 - 1 items... - if (newSize > 0x7fffffffL) FailCannotGrowBuffer(); - - // round up to 16 bytes, to reduce fragmentation - int size = SliceHelpers.Align((int)newSize); - - Array.Resize(ref buffer, size); - } - - [ContractAnnotation("=> halt")] - private static void FailCannotGrowBuffer() - { -#if DEBUG - // If you breakpoint here, that means that you probably have an uncheked maximum buffer size, or a runaway while(..) { append(..) } code in your layer code ! - // => you should ALWAYS ensure a reasonable maximum size of your allocations ! - if (Debugger.IsAttached) Debugger.Break(); -#endif - // note: some methods in the BCL do throw an OutOfMemoryException when attempting to allocated more than 2^31 - throw new OutOfMemoryException("Buffer cannot be resized, because it would exceed the maximum allowed size"); - } - - [UsedImplicitly(ImplicitUseTargetFlags.WithMembers)] - private sealed class DebugView - { - private readonly SliceWriter m_writer; - - public DebugView(SliceWriter writer) - { - m_writer = writer; - } - - public byte[] Data - { - get - { - if (m_writer.Buffer.Length == m_writer.Position) return m_writer.Buffer; - var tmp = new byte[m_writer.Position]; - System.Array.Copy(m_writer.Buffer, tmp, tmp.Length); - return tmp; - } - } - - public int Position - { - get { return m_writer.Position; } - } - - } - - } - -} diff --git a/FoundationDB.Client/Utils/StringConverters.cs b/FoundationDB.Client/Utils/StringConverters.cs new file mode 100644 index 000000000..073a7d7de --- /dev/null +++ b/FoundationDB.Client/Utils/StringConverters.cs @@ -0,0 +1,736 @@ + +namespace Doxense +{ + using System; + using System.Globalization; + using System.Runtime.CompilerServices; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + + internal static class StringConverters + { + #region Numbers... + + //NOTE: ces mthodes ont t importes de KTL/Sioux + //REVIEW: je ne sais pas si c'est la meilleure place pour ce code? + + /// Table de lookup pour les nombres entre 0 et 99, afin d'viter d'allouer une string inutilement + //note: vu que ce sont des literals, ils sont interned automatiquement + private static readonly string[] SmallNumbers = new string[100] + { + "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", + "10", "11", "12", "13", "14", "15", "16", "17", "18", "19", + "20", "21", "22", "23", "24", "25", "26", "27", "28", "29", + "30", "31", "32", "33", "34", "35", "36", "37", "38", "39", + "40", "41", "42", "43", "44", "45", "46", "47", "48", "49", + "50", "51", "52", "53", "54", "55", "56", "57", "58", "59", + "60", "61", "62", "63", "64", "65", "66", "67", "68", "69", + "70", "71", "72", "73", "74", "75", "76", "77", "78", "79", + "80", "81", "82", "83", "84", "85", "86", "87", "88", "89", + "90", "91", "92", "93", "94", "95", "96", "97", "98", "99", + }; + + /// Convertit un entier en chane, de manire optimise + /// Valeure entire convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(int value) + { + var cache = StringConverters.SmallNumbers; + return value >= 0 && value < cache.Length ? cache[value] : value.ToString(NumberFormatInfo.InvariantInfo); + } + + /// Convertit un entier en chane, de manire optimise + /// Valeure entire convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(uint value) + { + var cache = StringConverters.SmallNumbers; + return value < cache.Length ? cache[value] : value.ToString(NumberFormatInfo.InvariantInfo); + } + + /// Convertit un entier en chane, de manire optimise + /// Valeure entire convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(long value) + { + var cache = StringConverters.SmallNumbers; + return value >= 0 && value < cache.Length ? cache[(int) value] : value.ToString(NumberFormatInfo.InvariantInfo); + } + + /// Convertit un entier en chane, de manire optimise + /// Valeure entire convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(ulong value) + { + var cache = StringConverters.SmallNumbers; + return value < (ulong) cache.Length ? cache[(int) value] : value.ToString(NumberFormatInfo.InvariantInfo); + } + + /// Convertit un dcimal en chane, de manire optimise + /// Valeure dcimale convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(float value) + { + long x = unchecked((long) value); + // ReSharper disable once CompareOfFloatsByEqualityOperator + return x != value + ? value.ToString("R", CultureInfo.InvariantCulture) + : (x >= 0 && x < StringConverters.SmallNumbers.Length ? StringConverters.SmallNumbers[(int) x] : x.ToString(NumberFormatInfo.InvariantInfo)); + } + + /// Convertit un dcimal en chane, de manire optimise + /// Valeure dcimale convertir + /// Version chane + /// Cette fonction essaye d'vite le plus possibles des allocations mmoire + [Pure, NotNull] + public static string ToString(double value) + { + long x = unchecked((long)value); + // ReSharper disable once CompareOfFloatsByEqualityOperator + return x != value + ? value.ToString("R", CultureInfo.InvariantCulture) + : (x >= 0 && x < StringConverters.SmallNumbers.Length ? StringConverters.SmallNumbers[(int)x] : x.ToString(NumberFormatInfo.InvariantInfo)); + } + + /// Convertit une chane en boolen + /// Chane de texte (ex: "true") + /// Valeur par dfaut si vide ou invalide + /// Valeur boolenne correspondant (ex: true) ou valeur par dfaut + /// Les valeurs pour true sont "true", "yes", "on", "1". + /// Les valeurs pour false sont "false", "no", "off", "0", ou tout le reste + /// null et chane vide sont considrs comme false + /// + [Pure] + public static bool ToBoolean(string value, bool dflt) + { + if (string.IsNullOrEmpty(value)) return dflt; + char c = value[0]; + if (c == 't' || c == 'T') return true; + if (c == 'f' || c == 'F') return false; + if (c == 'y' || c == 'Y') return true; + if (c == 'n' || c == 'N') return false; + if ((c == 'o' || c == 'O') && value.Length > 1) { c = value[1]; return c == 'n' || c == 'N'; } + if (c == '1') return true; + if (c == '0') return false; + return dflt; + } + + /// Convertit une chane en boolen + /// Chane de texte (ex: "true") + /// Valeur boolenne correspondant (ex: true) ou null + /// Les valeurs pour true sont "true", "yes", "on", "1". + /// Les valeurs pour false sont "false", "no", "off", "0" + /// + [Pure] + public static bool? ToBoolean(string value) + { + if (string.IsNullOrEmpty(value)) return null; + char c = value[0]; + if (c == 't' || c == 'T') return true; + if (c == 'f' || c == 'F') return false; + if (c == 'y' || c == 'Y') return true; + if (c == 'n' || c == 'N') return false; + if ((c == 'o' || c == 'O') && value.Length > 1) { c = value[1]; return c == 'n' || c == 'N'; } + if (c == '1') return true; + if (c == '0') return false; + return null; + } + + /// Convertit un entier jusqu'au prochain sparateur (ou fin de buffer). A utilis pour simuler un Split + /// Buffer de caractres + /// Offset courant dans le buffer + /// + /// Sparateur attendu entre les ints + /// Valeur par dfaut retourne si erreur + /// Rcupre le rsultat de la conversion + /// Rcupre la nouvelle position (aprs le sparateur) + /// true si int charg, false si erreur (plus de place, incorrect, ...) + /// Si buffer est null + public static unsafe bool FastTryGetInt([NotNull] char* buffer, int offset, int length, char separator, int defaultValue, out int result, out int newpos) + { + Contract.PointerNotNull(buffer, nameof(buffer)); + result = defaultValue; + newpos = offset; + if (offset < 0 || offset >= length) return false; // deja a la fin !! + + char c = buffer[offset]; + if (c == separator) { newpos = offset + 1; return false; } // avance quand mme le curseur + if (!char.IsDigit(c)) + { // c'est pas un nombre, va jusqu'au prochain sparateur + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + } + newpos = offset; + return false; // deja le separateur, ou pas un digit == WARNING: le curseur ne sera pas avanc! + } + int res = c - 48; + offset++; + // il y a au moins 1 digit, parcourt les suivants + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + if (!char.IsDigit(c)) + { // va jusqu'au prochain sparator + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + } + newpos = offset; + return false; + } + // accumule le digit + res = res * 10 + (c - 48); + } + + result = res; + newpos = offset; + return true; + } + + /// Convertit un entier jusqu'au prochain sparateur (ou fin de buffer). A utilis pour simuler un Split + /// Buffer de caractres + /// Offset courant dans le buffer + /// + /// Sparateur attendu entre les ints + /// Valeur par dfaut retourne si erreur + /// Rcupre le rsultat de la conversion + /// Rcupre la nouvelle position (aprs le sparateur) + /// true si int charg, false si erreur (plus de place, incorrect, ...) + /// Si buffer est null + public static unsafe bool FastTryGetLong([NotNull] char* buffer, int offset, int length, char separator, long defaultValue, out long result, out int newpos) + { + Contract.PointerNotNull(buffer, nameof(buffer)); + result = defaultValue; + newpos = offset; + if (offset < 0 || offset >= length) return false; // deja a la fin !! + + char c = buffer[offset]; + if (c == separator) { newpos = offset + 1; return false; } // avance quand mme le curseur + if (!char.IsDigit(c)) + { // c'est pas un nombre, va jusqu'au prochain sparateur + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + } + newpos = offset; + return false; // deja le separateur, ou pas un digit == WARNING: le curseur ne sera pas avanc! + } + int res = c - 48; + offset++; + // il y a au moins 1 digit, parcourt les suivants + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + if (!char.IsDigit(c)) + { // va jusqu'au prochain sparator + while (offset < length) + { + c = buffer[offset++]; + if (c == separator) break; + } + newpos = offset; + return false; + } + // accumule le digit + res = res * 10 + (c - 48); + } + + result = res; + newpos = offset; + return true; + } + + /// Convertit une chane en entier (int) + /// Chane de caractre (ex: "1234") + /// Valeur par dfaut si vide ou invalide + /// Entier correspondant ou valeur par dfaut si pb (ex: 1234) + [Pure] + public static int ToInt32(string value, int defaultValue) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + // optimisation: si premier carac pas chiffre, exit + char c = value[0]; + if (value.Length == 1) return char.IsDigit(c) ? c - 48 : defaultValue; + if (!char.IsDigit(c) && c != '-' && c != '+' && c != ' ') return defaultValue; + return int.TryParse(value, NumberStyles.Integer, CultureInfo.InvariantCulture, out int res) ? res : defaultValue; + } + + /// Convertit une chane en entier (int) + /// Chane de caractre (ex: "1234") + /// Entier correspondant ou null si pb (ex: 1234) + [Pure] + public static int? ToInt32(string value) + { + if (string.IsNullOrEmpty(value)) return default(int?); + // optimisation: si premier carac pas chiffre, exit + char c = value[0]; + if (value.Length == 1) return char.IsDigit(c) ? (c - 48) : default(int?); + if (!char.IsDigit(c) && c != '-' && c != '+' && c != ' ') return default(int?); + return int.TryParse(value, NumberStyles.Integer, CultureInfo.InvariantCulture, out int res) ? res : default(int?); + } + + /// Convertit une chane en entier (long) + /// Chane de caractre (ex: "1234") + /// Valeur par dfaut si vide ou invalide + /// Entier correspondant ou valeur par dfaut si pb (ex: 1234) + [Pure] + public static long ToInt64(string value, long defaultValue) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + // optimisation: si premier carac pas chiffre, exit + char c = value[0]; + if (value.Length == 1) return char.IsDigit(c) ? ((long) c - 48) : defaultValue; + if (!char.IsDigit(c) && c != '-' && c != '+' && c != ' ') return defaultValue; + return long.TryParse(value, NumberStyles.Integer, CultureInfo.InvariantCulture, out long res) ? res : defaultValue; + } + + /// Convertit une chane en entier (long) + /// Chane de caractre (ex: "1234") + /// Entier correspondant ou null si pb (ex: 1234) + [Pure] + public static long? ToInt64(string value) + { + if (string.IsNullOrEmpty(value)) return default(long?); + // optimisation: si premier carac pas chiffre, exit + char c = value[0]; + if (value.Length == 1) return char.IsDigit(c) ? ((long) c - 48) : default(long?); + if (!char.IsDigit(c) && c != '-' && c != '+' && c != ' ') return default(long?); + return long.TryParse(value, NumberStyles.Integer, CultureInfo.InvariantCulture, out long res) ? res : default(long?); + } + + /// Convertit une chaine de caractre en double, quelque soit la langue locale (utilise le '.' comme sparateur dcimal) + /// Chaine (ex: "1.0", "123.456e7") + /// Valeur par dfaut si problme de conversion ou null + /// Culture (par dfaut InvariantCulture) + /// Double correspondant + [Pure] + public static double ToDouble(string value, double defaultValue, IFormatProvider culture = null) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + char c = value[0]; + if (!char.IsDigit(c) && c != '+' && c != '-' && c != '.' && c != ' ') return defaultValue; + if (culture == null) culture = CultureInfo.InvariantCulture; + if (culture == CultureInfo.InvariantCulture && value.IndexOf(',') >= 0) value = value.Replace(',', '.'); + return double.TryParse(value, NumberStyles.Float | NumberStyles.AllowThousands, culture, out double result) ? result : defaultValue; + } + + [Pure] + public static double? ToDouble(string value, IFormatProvider culture = null) + { + if (value == null) return default(double?); + double result = ToDouble(value, double.NaN, culture); + return double.IsNaN(result) ? default(double?) : result; + } + + /// Convertit une chaine de caractre en float, quelque soit la langue locale (utilise le '.' comme sparateur dcimal) + /// Chaine (ex: "1.0", "123.456e7") + /// Valeur par dfaut si problme de conversion ou null + /// Culture (par dfaut InvariantCulture) + /// Float correspondant + [Pure] + public static float ToSingle(string value, float defaultValue, IFormatProvider culture = null) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + char c = value[0]; + if (!char.IsDigit(c) && c != '+' && c != '-' && c != '.' && c != ' ') return defaultValue; + if (culture == null) culture = CultureInfo.InvariantCulture; + if (culture == CultureInfo.InvariantCulture && value.IndexOf(',') >= 0) value = value.Replace(',', '.'); + return float.TryParse(value, NumberStyles.Float | NumberStyles.AllowThousands, culture, out float result) ? result : defaultValue; + } + + [Pure] + public static float? ToSingle(string value, IFormatProvider culture = null) + { + if (value == null) return default(float?); + float result = ToSingle(value, float.NaN, culture); + return double.IsNaN(result) ? default(float?) : result; + } + + /// Convertit une chaine de caractre en double, quelque soit la langue locale (utilise le '.' comme sparateur dcimal) + /// Chaine (ex: "1.0", "123.456e7") + /// Valeur par dfaut si problme de conversion ou null + /// Culture (par dfaut InvariantCulture) + /// Dcimal correspondant + [Pure] + public static decimal ToDecimal(string value, decimal defaultValue, IFormatProvider culture = null) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + char c = value[0]; + if (!char.IsDigit(c) && c != '+' && c != '-' && c != '.' && c != ' ') return defaultValue; + if (culture == null) culture = CultureInfo.InvariantCulture; + if (culture == CultureInfo.InvariantCulture && value.IndexOf(',') >= 0) value = value.Replace(',', '.'); + return decimal.TryParse(value, NumberStyles.Float | NumberStyles.AllowThousands, culture, out decimal result) ? result : defaultValue; + } + + [Pure] + public static decimal? ToDecimal(string value, IFormatProvider culture = null) + { + if (string.IsNullOrEmpty(value)) return default(decimal?); + char c = value[0]; + if (!char.IsDigit(c) && c != '+' && c != '-' && c != '.' && c != ' ') return default(decimal?); + if (culture == null) culture = CultureInfo.InvariantCulture; + if (culture == CultureInfo.InvariantCulture && value.IndexOf(',') >= 0) value = value.Replace(',', '.'); + return decimal.TryParse(value, NumberStyles.Float | NumberStyles.AllowThousands, culture, out decimal result) ? result : default(decimal?); + } + + /// Convertit une chaine en DateTime + /// Date convertir + /// Valeur par dfaut + /// + /// Voir StringConverters.ParseDateTime() + [Pure] + public static DateTime ToDateTime(string value, DateTime defaultValue, CultureInfo culture = null) + { + return ParseDateTime(value, defaultValue, culture); + } + + /// Convertit une chaine en DateTime + /// Date convertir + /// + /// Voir StringConverters.ParseDateTime() + [Pure] + public static DateTime? ToDateTime(string value, CultureInfo culture = null) + { + if (string.IsNullOrEmpty(value)) return default(DateTime?); + DateTime result = ParseDateTime(value, DateTime.MaxValue, culture); + return result == DateTime.MaxValue ? default(DateTime?) : result; + } + + /// Convertit une chaine de caractres en GUID + /// Chaine (ex: "123456-789") + /// Valeur par dfaut si problme de conversion ou null + /// GUID correspondant + [Pure] + public static Guid ToGuid(string value, Guid defaultValue) + { + if (string.IsNullOrEmpty(value)) return defaultValue; + return Guid.TryParse(value, out Guid result) ? result : defaultValue; + } + + [Pure] + public static Guid? ToGuid(string value) + { + if (string.IsNullOrEmpty(value)) return default(Guid?); + return Guid.TryParse(value, out Guid result) ? result : default(Guid?); + } + + /// Convertit une chaine de caractres en Enum + /// Type de l'Enum + /// Chaine (ex: "Red", "2", ...) + /// Valeur par dfaut si problme de conversion ou null + /// Valeur de l'enum correspondante + /// Accepte les valeures sous forme textuelle ou numrique, case insensitive + [Pure] + public static TEnum ToEnum(string value, TEnum defaultValue) + where TEnum : struct, IComparable, IConvertible, IFormattable + { + if (string.IsNullOrEmpty(value)) return defaultValue; + return Enum.TryParse(value, true, out TEnum result) ? result : defaultValue; + } + + [Pure] + public static TEnum? ToEnum(string value) + where TEnum : struct, IComparable, IConvertible, IFormattable + { + if (string.IsNullOrEmpty(value)) return default(TEnum?); + return Enum.TryParse(value, true, out TEnum result) ? result : default(TEnum?); + } + + #endregion + + #region Dates... + + /// Convertit une date en une chaine de caractres au format "YYYYMMDDHHMMSS" + /// Date formater + /// Date formate sur 14 caractres au format YYYYMMDDHHMMSS + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string ToDateTimeString(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("yyyyMMddHHmmss", CultureInfo.InvariantCulture); + } + + /// Convertit une date en une chaine de caractres au format "AAAAMMJJ" + /// Date formater + /// Date formate sur 8 caractres au format AAAAMMJJ + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string ToDateString(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("yyyyMMdd", CultureInfo.InvariantCulture); + } + + /// Convertit un heure en une chaine de caractres au format "HHMMSS" + /// Date formater + /// Heure formate sur 6 caractres au format HHMMSS + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string ToTimeString(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("HHmmss", CultureInfo.InvariantCulture); + } + + /// Convertit une date en une chaine de caractres au format "yyyy-MM-dd HH:mm:ss" + /// Date convertir + /// Chaine au format "yyyy-MM-dd HH:mm:ss" + [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static string FormatDateTime(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("yyyy-MM-dd HH:mm:ss", CultureInfo.InvariantCulture); + } + + /// Convertit une date en une chaine de caractres au format "yyyy-MM-dd" + /// Date convertir + /// Chaine au format "yyyy-MM-dd" + [Pure, NotNull] + public static string FormatDate(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("yyyy-MM-dd", CultureInfo.InvariantCulture); + } + + /// Convertit une heure en une chaine de caractres au format "hh:mm:ss" + /// Heure convertir + /// Chaine au format "hh:mm:ss" + [Pure, NotNull] + public static string FormatTime(DateTime date) + { + //REVIEW: PERF: faire une version optimise? + return date.ToString("HH:mm:ss", CultureInfo.InvariantCulture); + } + + /// Convertit une chaine de caractre au format "YYYY", "YYYYMM", "YYYYMMDD" ou "YYYYMMDDHHMMSS" en DateTime + /// Chaine de caractres convertir + /// Objet DateTime correspondant, ou exception si incorrect + /// Si la date est incorrecte + [Pure] + public static DateTime ParseDateTime(string date) + { + return ParseDateTime(date, null); + } + + /// Convertit une chaine de caractre au format "YYYY", "YYYYMM", "YYYYMMDD" ou "YYYYMMDDHHMMSS" en DateTime + /// Chaine de caractres convertir + /// Culture (pour le format attendu) ou null + /// Objet DateTime correspondant, ou exception si incorrect + /// Si la date est incorrecte + [Pure] + public static DateTime ParseDateTime(string date, CultureInfo culture) + { + if (!TryParseDateTime(date, culture, out DateTime result, true)) throw FailInvalidDateFormat(); + return result; + } + + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailInvalidDateFormat() + { + // ReSharper disable once NotResolvedInText + return new ArgumentException("Invalid date format", "date"); + } + + /// Convertit une chaine de caractre au format "YYYY", "YYYYMM", "YYYYMMDD" ou "YYYYMMDDHHMMSS" en DateTime + /// Chaine de caractres convertir + /// Valeur par dfaut + /// Objet DateTime correspondant, ou dflt si date est null ou vide + [Pure] + public static DateTime ParseDateTime(string date, DateTime dflt) + { + if (string.IsNullOrEmpty(date)) return dflt; + if (!TryParseDateTime(date, null, out DateTime result, false)) return dflt; + return result; + } + + /// Convertit une chaine de caractre au format "YYYY", "YYYYMM", "YYYYMMDD" ou "YYYYMMDDHHMMSS" en DateTime + /// Chaine de caractres convertir + /// Valeur par dfaut + /// Culture (pour le format attendu) ou null + /// Objet DateTime correspondant, ou dflt si date est null ou vide + [Pure] + public static DateTime ParseDateTime(string date, DateTime dflt, CultureInfo culture) + { + if (!TryParseDateTime(date, culture, out DateTime result, false)) return dflt; + return result; + } + private static int ParseDateSegmentUnsafe(string source, int offset, int size) + { + // note: normalement le caller a dja valid les paramtres + int sum = source[offset++] - '0'; + if (sum < 0 || sum >= 10) return -1; // invalid first digit + while (--size > 0) + { + int d = source[offset++] - '0'; + if (d < 0 || d >= 10) return -1; // invalid digit! + sum = (sum * 10) + d; + } + return sum; + } + + /// Essayes de convertir une chaine de caratres au format "YYYY", "YYYYMM", "YYYYMMDD" ou "YYYYMMDDHHMMSS" en DateTime + /// Chaine de caractres convertir + /// Culture (pour le format attendu) ou null + /// Date convertie (ou DateTime.MinValue en cas de problme) + /// Si false, absorbe les exceptions ventuelles. Si true, laisse les s'chaper + /// True si la date est correcte, false dans les autres cas + [Pure] + public static bool TryParseDateTime(string date, CultureInfo culture, out DateTime result, bool throwsFail) + { + result = DateTime.MinValue; + + if (date == null) { if (throwsFail) throw new ArgumentNullException(nameof(date)); else return false; } + if (date.Length < 4) { if (throwsFail) throw new FormatException("Date '" + date + "' must be at least 4 characters long"); else return false; } + //if (throwsFail) throw new FormatException("Date '"+date+"' must contains only digits"); else return false; + try + { + if (char.IsDigit(date[0])) + { // commence par un chiffre, c'est peut etre un timestamp? + switch (date.Length) + { + case 4: + { // YYYY -> YYYY/01/01 00:00:00.000 + int y = ParseDateSegmentUnsafe(date, 0, 4); + if (y < 1 || y > 9999) break; + result = new DateTime(y, 1, 1); + return true; + } + case 6: + { // YYYYMM -> YYYY/MM/01 00:00:00.000 + int y = ParseDateSegmentUnsafe(date, 0, 4); + if (y < 1 || y > 9999) break; + int m = ParseDateSegmentUnsafe(date, 4, 2); + if (m < 1 || m > 12) break; + result = new DateTime(y, m, 1); + return true; + } + case 8: + { // YYYYMMDD -> YYYY/MM/DD 00:00:00.000 + int y = ParseDateSegmentUnsafe(date, 0, 4); + if (y < 1 || y > 9999) break; + int m = ParseDateSegmentUnsafe(date, 4, 2); + if (m < 1 || m > 12) break; + int d = ParseDateSegmentUnsafe(date, 6, 2); + if (d < 1 || d > 31) break; + result = new DateTime(y, m, d); + return true; + } + case 14: + { // YYYYMMDDHHMMSS -> YYYY/MM/DD HH:MM:SS.000 + int y = ParseDateSegmentUnsafe(date, 0, 4); + if (y < 1 || y > 9999) break; + int m = ParseDateSegmentUnsafe(date, 4, 2); + if (m < 1 || m > 12) break; + int d = ParseDateSegmentUnsafe(date, 6, 2); + if (d < 1 || d > 31) break; + int h = ParseDateSegmentUnsafe(date, 8, 2); + if (h < 0 || h > 23) break; + int n = ParseDateSegmentUnsafe(date, 10, 2); + if (n < 0 || n > 59) break; + int s = ParseDateSegmentUnsafe(date, 12, 2); + if (s < 0 || s > 59) break; + result = new DateTime(y, m, d, h, n, s); + return true; + } + case 17: + { // YYYYMMDDHHMMSSFFF -> YYYY/MM/DD HH:MM:SS.FFF + int y = ParseDateSegmentUnsafe(date, 0, 4); + if (y < 1 || y > 9999) break; + int m = ParseDateSegmentUnsafe(date, 4, 2); + if (m < 1 || m > 12) break; + int d = ParseDateSegmentUnsafe(date, 6, 2); + if (d < 1 || d > 31) break; + int h = ParseDateSegmentUnsafe(date, 8, 2); + if (h < 0 || h > 23) break; + int n = ParseDateSegmentUnsafe(date, 10, 2); + if (n < 0 || n > 59) break; + int s = ParseDateSegmentUnsafe(date, 12, 2); + if (s < 0 || s > 59) break; + int f = ParseDateSegmentUnsafe(date, 14, 3); + result = new DateTime(y, m, d, h, n, s, f); + return true; + } + } + } + else if (char.IsLetter(date[0])) + { // on va tenter un ParseExact ("Vendredi, 37 Trumaire 1789 3 heures moint le quart") + result = DateTime.ParseExact(date, new[] { "D", "F", "f" }, culture ?? CultureInfo.InvariantCulture, DateTimeStyles.None); + return true; + } + + // Je vais tenter le jackpot, mon cher Julien! + result = DateTime.Parse(date, culture ?? CultureInfo.InvariantCulture); + return true; + } + catch (FormatException) + { // Dommage! La cagnote est remise la fois prochaine... + if (throwsFail) throw; + return false; + } + catch (ArgumentOutOfRangeException) + { // Pb sur un DateTime avec des dates invalides (31 fvrier, ...) + if (throwsFail) throw; + return false; + } + } + + /// Convertit une heure "human friendly" en DateTime: "11","11h","11h00","11:00" -> {11:00:00.000} + /// Chaine contenant l'heure convertir + /// Object DateTime contenant l'heure. La partie "date" est fixe aujourd'hui + [Pure] + public static DateTime ParseTime([NotNull] string time) + { + Contract.NotNullOrEmpty(time, nameof(time)); + + time = time.ToLowerInvariant(); + + int hour; + int minute = 0; + int second = 0; + + int p = time.IndexOf('h'); + if (p > 0) + { + hour = System.Convert.ToInt16(time.Substring(0, p)); + if (p + 1 >= time.Length) + minute = 0; + else + minute = System.Convert.ToInt16(time.Substring(p + 1)); + } + else + { + p = time.IndexOf(':'); + if (p > 0) + { + hour = System.Convert.ToInt16(time.Substring(0, p)); + if (p + 1 >= time.Length) + minute = 0; + else + minute = System.Convert.ToInt16(time.Substring(p + 1)); + } + else + { + hour = System.Convert.ToInt16(time); + } + } + var d = DateTime.Today; + return new DateTime(d.Year, d.Month, d.Day, hour, minute, second, 0); + } + + #endregion + } +} diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Utils/Uuid128.cs index ab273b5d3..2211b7a1c 100644 --- a/FoundationDB.Client/Utils/Uuid128.cs +++ b/FoundationDB.Client/Utils/Uuid128.cs @@ -26,16 +26,22 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace System { - using JetBrains.Annotations; using System; + using System.Collections.Generic; using System.ComponentModel; + using System.Diagnostics; + using System.Runtime.CompilerServices; using System.Runtime.InteropServices; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; - /// RFC 4122 compliant 128-bit UUID - /// You should use this type if you are primarily exchanged UUIDs with non-.NET platforms, that use the RFC 4122 byte ordering (big endian). The type System.Guid uses the Microsoft encoding (little endian) and is not compatible. - [ImmutableObject(true), PublicAPI, StructLayout(LayoutKind.Explicit), Serializable] + /// Represents an RFC 4122 compliant 128-bit UUID + /// You should use this type if you are primarily exchanging UUIDs with non-.NET platforms, that use the RFC 4122 byte ordering (big endian). The type System.Guid uses the Microsoft encoding (little endian) and is not compatible. + [DebuggerDisplay("[{ToString(),nq}]")] + [ImmutableObject(true), StructLayout(LayoutKind.Explicit), Serializable] public struct Uuid128 : IFormattable, IComparable, IEquatable, IComparable, IEquatable { // This is just a wrapper struct on System.Guid that makes sure that ToByteArray() and Parse(byte[]) and new(byte[]) will parse according to RFC 4122 (http://www.ietf.org/rfc/rfc4122.txt) @@ -93,6 +99,7 @@ public struct Uuid128 : IFormattable, IComparable, IEquatable, ICompara #region Constructors... + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(Guid guid) : this() { @@ -103,33 +110,56 @@ public Uuid128(string value) : this(new Guid(value)) { } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(Slice slice) : this() { m_packed = Convert(slice); } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(byte[] bytes) - : this(Slice.Create(bytes)) - { } + : this() + { + m_packed = Convert(bytes.AsSlice()); + } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(int a, short b, short c, byte[] d) : this(new Guid(a, b, c, d)) { } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(int a, short b, short c, byte d, byte e, byte f, byte g, byte h, byte i, byte j, byte k) : this(new Guid(a, b, c, d, e, f, g, h, i, j, k)) { } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid128(uint a, ushort b, ushort c, byte d, byte e, byte f, byte g, byte h, byte i, byte j, byte k) : this(new Guid(a, b, c, d, e, f, g, h, i, j, k)) { } + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid128(Uuid64 a, Uuid64 b) + : this() + { + m_packed = Convert(a, b); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid128(Uuid64 a, uint b, uint c) + : this() + { + m_packed = Convert(a, b, c); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static explicit operator Guid(Uuid128 uuid) { return uuid.m_packed; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static explicit operator Uuid128(Guid guid) { return new Uuid128(guid); @@ -137,30 +167,67 @@ public static explicit operator Uuid128(Guid guid) public static readonly Uuid128 Empty = default(Uuid128); - public static readonly Uuid128 MaxValue = new Uuid128(uint.MaxValue, ushort.MaxValue, ushort.MaxValue, 255, 255, 255, 255, 255, 255, 255, 255); + /// Size is 16 bytes + public const int SizeOf = 16; public static Uuid128 NewUuid() { return new Uuid128(Guid.NewGuid()); } - internal static Guid Convert(Slice input) + public static Guid Convert(Slice input) { - if (input.Count <= 0) return default(Guid); + input.EnsureSliceIsValid(); + if (input.Count == 0) return default(Guid); + if (input.Count != 16) throw new ArgumentException("Slice for UUID must be exactly 16 bytes long"); - if (input.Array == null) throw new ArgumentNullException("input"); - if (input.Count == 16) + unsafe { - unsafe + fixed (byte* buf = &input.DangerousGetPinnableReference()) { - fixed (byte* buf = input.Array) - { - return Read(buf + input.Offset); - } + return ReadUnsafe(buf); } } + } + + public static unsafe Guid Convert(byte* buffer, int count) + { + if (count == 0) return default(Guid); + if (count != 16) throw new ArgumentException("Slice for UUID must be exactly 16 bytes long"); + + return ReadUnsafe(buffer); + } + + public static Guid Convert(Uuid64 a, Uuid64 b) + { + unsafe + { + byte* buf = stackalloc byte[16]; + a.WriteToUnsafe(buf); + b.WriteToUnsafe(buf + 8); + return ReadUnsafe(buf); + } + } + + public static Guid Convert(Uuid64 a, uint b, uint c) + { + unsafe + { + byte* buf = stackalloc byte[16]; + a.WriteToUnsafe(buf); + + buf[8] = (byte) b; + buf[9] = (byte)(b >> 8); + buf[10] = (byte)(b >> 16); + buf[11] = (byte)(b >> 24); - throw new ArgumentException("Slice for UUID must be exactly 16 bytes long"); + buf[12] = (byte) c; + buf[13] = (byte)(c >> 8); + buf[14] = (byte)(c >> 16); + buf[15] = (byte)(c >> 24); + + return ReadUnsafe(buf); + } } public static Uuid128 Parse([NotNull] string input) @@ -175,8 +242,7 @@ public static Uuid128 ParseExact([NotNull] string input, string format) public static bool TryParse(string input, out Uuid128 result) { - Guid guid; - if (!Guid.TryParse(input, out guid)) + if (!Guid.TryParse(input, out Guid guid)) { result = default(Uuid128); return false; @@ -187,8 +253,7 @@ public static bool TryParse(string input, out Uuid128 result) public static bool TryParseExact(string input, string format, out Uuid128 result) { - Guid guid; - if (!Guid.TryParseExact(input, format, out guid)) + if (!Guid.TryParseExact(input, format, out Guid guid)) { result = default(Uuid128); return false; @@ -201,6 +266,7 @@ public static bool TryParseExact(string input, string format, out Uuid128 result public long Timestamp { + [Pure] get { long ts = m_timeLow; @@ -212,6 +278,7 @@ public long Timestamp public int Version { + [Pure] get { return m_timeHiAndVersion >> 12; @@ -220,6 +287,7 @@ public int Version public int ClockSequence { + [Pure] get { int clk = m_clkSeqLow; @@ -230,6 +298,7 @@ public int ClockSequence public long Node { + [Pure] get { long node; @@ -238,15 +307,17 @@ public long Node node |= ((long)m_node2) << 24; node |= ((long)m_node3) << 16; node |= ((long)m_node4) << 8; - node |= (long)m_node5; + node |= m_node5; return node; } } - #region Conversion... + #region Unsafe I/O... - internal unsafe static Guid Read(byte* src) + [Pure] + public static unsafe Guid ReadUnsafe([NotNull] byte* src) { + Contract.Requires(src != null); Guid tmp; if (BitConverter.IsLittleEndian) @@ -277,11 +348,30 @@ internal unsafe static Guid Read(byte* src) return tmp; } - internal unsafe static void Write(Guid value, byte* ptr) + public static Guid ReadUnsafe([NotNull] byte[] buffer, int offset) + { + Contract.Requires(buffer != null && offset >= 0 && offset + 15 < buffer.Length); + unsafe + { + fixed (byte* ptr = &buffer[offset]) + { + return ReadUnsafe(ptr); + } + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static unsafe void WriteUnsafe(Guid value, [NotNull] byte* ptr) + { + WriteUnsafe(&value, ptr); + } + + internal static unsafe void WriteUnsafe([NotNull] Guid* value, [NotNull] byte* ptr) { + Contract.Requires(value != null && ptr != null); if (BitConverter.IsLittleEndian) { - byte* src = (byte*)&value; + byte* src = (byte*) value; // Data1: 32 bits, must swap ptr[0] = src[3]; @@ -299,19 +389,91 @@ internal unsafe static void Write(Guid value, byte* ptr) } else { - long* src = (long*)&value; + long* src = (long*) value; *(long*)(ptr) = src[0]; *(long*)(ptr + 8) = src[1]; } } - internal unsafe void WriteTo(byte* ptr) + public static void WriteUnsafe(Guid value, [NotNull] byte[] buffer, int offset) { - Write(m_packed, ptr); + Contract.Requires(buffer != null && offset >= 0 && offset + 15 < buffer.Length); + unsafe + { + fixed (byte* ptr = &buffer[offset]) + { + WriteUnsafe(value, ptr); + } + } } - [Pure] + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public unsafe void WriteToUnsafe([NotNull] byte* ptr) + { + WriteUnsafe(m_packed, ptr); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteToUnsafe([NotNull] byte[] buffer, int offset) + { + WriteUnsafe(m_packed, buffer, offset); + } + + #endregion + + #region Decomposition... + + /// Split this 128-bit UUID into two 64-bit UUIDs + /// Receives the first 8 bytes (in network order) of this UUID + /// Receives the last 8 bytes (in network order) of this UUID + public void Split(out Uuid64 high, out Uuid64 low) + { + unsafe + { + byte* buffer = stackalloc byte[16]; + WriteUnsafe(m_packed, buffer); + high = new Uuid64(Uuid64.ReadUnsafe(buffer)); + low = new Uuid64(Uuid64.ReadUnsafe(buffer + 8)); + } + } + + /// Split this 128-bit UUID into two 64-bit numbers + /// Receives the first 8 bytes (in network order) of this UUID + /// Receives the last 8 bytes (in network order) of this UUID + public void Split(out ulong high, out ulong low) + { + unsafe + { + byte* buffer = stackalloc byte[16]; + WriteUnsafe(m_packed, buffer); + high = Uuid64.ReadUnsafe(buffer); + low = Uuid64.ReadUnsafe(buffer + 8); + } + } + + /// Split this 128-bit UUID into two 64-bit numbers + /// Receives the first 8 bytes (in network order) of this UUID + /// Receives the middle 4 bytes (in network order) of this UUID + /// Receives the last 4 bytes (in network order) of this UUID + public void Split(out ulong high, out uint mid, out uint low) + { + unsafe + { + byte* buffer = stackalloc byte[16]; + WriteUnsafe(m_packed, buffer); + high = Uuid64.ReadUnsafe(buffer); + var id = Uuid64.ReadUnsafe(buffer + 8); + mid = (uint) (id >> 32); + low = (uint) id; + } + } + + #endregion + + #region Conversion... + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Guid ToGuid() { return m_packed; @@ -326,14 +488,15 @@ public byte[] ToByteArray() unsafe { fixed (byte* ptr = res) + fixed (Uuid128* self = &this) { - Write(m_packed, ptr); + WriteUnsafe((Guid*) self, ptr); } } return res; } - [Pure] + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice ToSlice() { //TODO: optimize this ? @@ -355,6 +518,57 @@ public string ToString(string format, IFormatProvider provider) return m_packed.ToString(format, provider); } + /// Increment the value of this UUID + /// Positive value + /// Incremented UUID + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid128 Increment([Positive] int value) + { + Contract.Requires(value >= 0); + return Increment(checked((ulong)value)); + } + + /// Increment the value of this UUID + /// Positive value + /// Incremented UUID + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid128 Increment([Positive] long value) + { + Contract.Requires(value >= 0); + return Increment(checked((ulong)value)); + } + + /// Increment the value of this UUID + /// Value to add to this UUID + /// Incremented UUID + [Pure] + public Uuid128 Increment(ulong value) + { + unsafe + { + fixed (Uuid128* self = &this) + { + // serialize GUID into High Endian format + byte* buf = stackalloc byte[16]; + WriteUnsafe((Guid*)self, buf); + + // Add the low 64 bits (in HE) + ulong lo = UnsafeHelpers.LoadUInt64BE(buf + 8); + ulong sum = lo + value; + if (sum < value) + { // overflow occured, we must carry to the high 64 bits (in HE) + ulong hi = UnsafeHelpers.LoadUInt64BE(buf); + UnsafeHelpers.StoreUInt64BE(buf, unchecked(hi + 1)); + } + UnsafeHelpers.StoreUInt64BE(buf + 8, sum); + // deserialize back to GUID + return new Uuid128(ReadUnsafe(buf)); + } + } + } + + //TODO: Decrement + #endregion #region Equality / Comparison ... @@ -362,46 +576,55 @@ public string ToString(string format, IFormatProvider provider) public override bool Equals(object obj) { if (obj == null) return false; - if (obj is Uuid128) return m_packed == ((Uuid128)obj); - if (obj is Guid) return m_packed == ((Guid)obj); + if (obj is Uuid128 u128) return m_packed == u128.m_packed; + if (obj is Guid g) return m_packed == g; + //TODO: Slice? string? return false; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public bool Equals(Uuid128 other) { return m_packed == other.m_packed; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public bool Equals(Guid other) { return m_packed == other; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator ==(Uuid128 a, Uuid128 b) { return a.m_packed == b.m_packed; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator !=(Uuid128 a, Uuid128 b) { return a.m_packed != b.m_packed; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator ==(Uuid128 a, Guid b) { return a.m_packed == b; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator !=(Uuid128 a, Guid b) { return a.m_packed != b; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator ==(Guid a, Uuid128 b) { return a == b.m_packed; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public static bool operator !=(Guid a, Uuid128 b) { return a != b.m_packed; @@ -412,6 +635,7 @@ public override int GetHashCode() return m_packed.GetHashCode(); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public int CompareTo(Uuid128 other) { return m_packed.CompareTo(other.m_packed); @@ -419,16 +643,42 @@ public int CompareTo(Uuid128 other) public int CompareTo(object obj) { - if (obj == null) return 1; - - if (obj is Uuid128) - return m_packed.CompareTo(((Uuid128)obj).m_packed); - else - return m_packed.CompareTo(obj); + switch (obj) + { + case null: return 1; + case Uuid128 u128: return m_packed.CompareTo(u128.m_packed); + case Guid g: return m_packed.CompareTo(g); + } + return m_packed.CompareTo(obj); } #endregion + /// Instance of this times can be used to test Uuid128 for equality and ordering + public sealed class Comparer : IEqualityComparer, IComparer + { + + public static readonly Comparer Default = new Comparer(); + + private Comparer() + { } + + public bool Equals(Uuid128 x, Uuid128 y) + { + return x.m_packed.Equals(y.m_packed); + } + + public int GetHashCode(Uuid128 obj) + { + return obj.m_packed.GetHashCode(); + } + + public int Compare(Uuid128 x, Uuid128 y) + { + return x.m_packed.CompareTo(y.m_packed); + } + } + } } diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index 5e08d1b44..55a485da4 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -26,55 +26,78 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +using System.Security.Cryptography; + +namespace System { using System; + using System.Collections.Generic; using System.ComponentModel; using System.Diagnostics; using System.Globalization; + using System.Runtime.CompilerServices; + using Doxense; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using JetBrains.Annotations; - [DebuggerDisplay("[{ToString()}]")] - [ImmutableObject(true), PublicAPI, Serializable] + /// Represents a 64-bit UUID that is stored in high-endian format on the wire + [DebuggerDisplay("[{ToString(),nq}]")] + [ImmutableObject(true), Serializable] public struct Uuid64 : IFormattable, IEquatable, IComparable { public static readonly Uuid64 Empty = default(Uuid64); + /// Size is 8 bytes + public const int SizeOf = 8; + private readonly ulong m_value; + //note: this will be in host order (so probably Little-Endian) in order to simplify parsing and ordering + + #region Constructors... + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid64(ulong value) { m_value = value; } + [MethodImpl(MethodImplOptions.AggressiveInlining)] public Uuid64(long value) { m_value = (ulong)value; } - public Uuid64(byte[] value) + /// Pack two 32-bits components into a 64-bit UUID + /// Upper 32 bits (XXXXXXXX-........) + /// Lower 32 bits (........-XXXXXXXX) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid64(uint a, uint b) { - if (value == null) throw new ArgumentNullException("value"); - if (value.Length != 8) throw new ArgumentException("Value must be 8 bytes long", "value"); - - m_value = Read(value, 0); + //Contract.Requires((ulong) b < (1UL << 48)); + m_value = ((ulong) a << 32) | b; } - public Uuid64(Slice value) + /// Pack two components into a 64-bit UUID + /// Upper 16 bits (XXXX....-........) + /// Lower 48 bits (....XXXX-XXXXXXXX) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public Uuid64(ushort a, long b) { - if (value == null) throw new ArgumentNullException("value"); - if (value.Count != 8) throw new ArgumentException("Value must be 8 bytes long", "value"); + //Contract.Requires((ulong) b < (1UL << 48)); + m_value = ((ulong) a << 48) | ((ulong) b & ((1UL << 48) - 1)); + } - m_value = Read(value.Array, value.Offset); + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailInvalidBufferSize([InvokerParameterName] string arg) + { + return ThrowHelper.ArgumentException(arg, "Value must be 8 bytes long"); } - public Uuid64(string value) + [Pure, NotNull, MethodImpl(MethodImplOptions.NoInlining)] + private static Exception FailInvalidFormat() { - if (!TryParse(value, out m_value)) - { - throw new FormatException("Invalid Uuid64 format"); - } + return ThrowHelper.FormatException("Invalid " + nameof(Uuid64) + " format"); } /// Generate a new random 64-bit UUID, using a global source of randomness. @@ -83,131 +106,402 @@ public Uuid64(string value) ///

If you need sequential uuids, you should use a different generator (ex: FlakeID, ...)

///

This method uses a cryptographic RNG under a lock to generate 8 bytes of randomness, which can be slow. If you must generate a large number of unique ids, you should use a different source.

/// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Uuid64 NewUuid() { //Note: we chould use Guid.NewGuid() as a source of randomness, but even though a guid is "guaranteed" to be unique, a substring of a guid is not.. or is it? return Uuid64RandomGenerator.Default.NewUuid(); } + #endregion + + #region Decomposition... + + /// Split into two 32-bit halves + /// Most significant 32 bits + /// Least significant 32 bits + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Split(out uint a, out uint b) + { + a = (uint) (m_value >> 32); + b = (uint) m_value; + } + + /// Split into two halves + /// Most significant 16 bits + /// Least significant 48 bits + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void Split(out ushort a, out long b) + { + a = (ushort) (m_value >> 48); + b = (long) (m_value & ~((1UL << 48) - 1)); + } + + #endregion + + #region Reading... + + /// Read a 64-bit UUID from a byte array + /// Array of exactly 0 or 8 bytes + [Pure] + public static Uuid64 Read(byte[] value) + { + Contract.NotNull(value, nameof(value)); + if (value.Length == 0) return default(Uuid64); + if (value.Length == 8) return new Uuid64(ReadUnsafe(value, 0)); + throw FailInvalidBufferSize(nameof(value)); + } + + /// Read a 64-bit UUID from part of a byte array + [Pure] + [Obsolete("Use Uuid64.Read(ReadOnlySpan) instead!")] + public static Uuid64 Read(byte[] value, int offset, int count) + { + Contract.DoesNotOverflow(value, offset, count, nameof(value)); + if (count == 0) return default(Uuid64); + if (count == 8) return new Uuid64(ReadUnsafe(value, 0)); + throw FailInvalidBufferSize(nameof(count)); + } + + /// Read a 64-bit UUID from slice of memory + /// slice of exactly 0 or 8 bytes + [Pure] + public static Uuid64 Read(Slice value) + { + Contract.NotNull(value.Array, nameof(value)); + if (value.Count == 0) return default(Uuid64); + if (value.Count == 8) return new Uuid64(ReadUnsafe(value.Array, value.Offset)); + throw FailInvalidBufferSize(nameof(value)); + } + + /// Read a 64-bit UUID from slice of memory + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe Uuid64 Read(byte* ptr, uint count) + { + if (count == 0) return default(Uuid64); + if (count == 8) return new Uuid64(ReadUnsafe(ptr)); + throw FailInvalidBufferSize(nameof(count)); + } + + #endregion + #region Parsing... - public static Uuid64 Parse([NotNull] string s) +#if ENABLED_SPAN + + /// Parse a string representation of an UUid64 + /// String in either formats: "", "badc0ffe-e0ddf00d", "badc0ffee0ddf00d", "{badc0ffe-e0ddf00d}", "{badc0ffee0ddf00d}" + /// Parsing is case-insensitive. The empty string is mapped to Uuid64.Empty. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Uuid64 Parse([NotNull] string buffer) { - if (s == null) throw new ArgumentNullException("s"); - ulong value; - if (!TryParse(s, out value)) + Contract.NotNull(buffer, nameof(buffer)); + if (!TryParse(buffer.AsSpan(), out var value)) { - throw new FormatException("Invalid Uuid64 format"); + throw FailInvalidFormat(); } - return new Uuid64(value); + return value; } - public static bool TryParse([NotNull] string s, out Uuid64 result) + /// Parse a string representation of an UUid64 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Uuid64 Parse(ReadOnlySpan buffer) { - if (s == null) throw new ArgumentNullException("s"); - ulong value; - if (!TryParse(s, out value)) + if (!TryParse(buffer, out var value)) { - result = default(Uuid64); - return false; + throw FailInvalidFormat(); } - result = new Uuid64(value); - return true; + return value; } - private static bool TryParse(string s, out ulong result) + /// Parse a string representation of an UUid64 + [Pure] + [Obsolete("Use Uuid64.Parse(ReadOnlySpan) instead", error: true)] //TODO: remove me! + public static unsafe Uuid64 Parse(char* buffer, int count) + { + if (count == 0) return default(Uuid64); + if (!TryParse(new ReadOnlySpan(buffer, count), out var value)) + { + throw FailInvalidFormat(); + } + return value; + } + + /// Parse a Base62 encoded string representation of an UUid64 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Uuid64 FromBase62([NotNull] string buffer) + { + Contract.NotNull(buffer, nameof(buffer)); + if (!TryParseBase62(buffer.AsSpan(), out var value)) + { + throw FailInvalidFormat(); + } + return value; + } + + /// Try parsing a string representation of an UUid64 + public static bool TryParse([NotNull] string buffer, out Uuid64 result) + { + Contract.NotNull(buffer, nameof(buffer)); + return TryParse(buffer.AsSpan(), out result); + } + + /// Try parsing a string representation of an UUid64 + public static bool TryParse(ReadOnlySpan s, out Uuid64 result) { Contract.Requires(s != null); // we support the following formats: "{hex8-hex8}", "{hex16}", "hex8-hex8", "hex16" and "base62" // we don't support base10 format, because there is no way to differentiate from hex or base62 - result = 0; + result = default(Uuid64); switch (s.Length) { + case 0: + { // empty + return true; + } + case 16: + { // xxxxxxxxxxxxxxxx + return TryDecode16Unsafe(s, separator: false, out result); + } + case 17: + { // xxxxxxxx-xxxxxxxx + if (s[8] != '-') return false; + return TryDecode16Unsafe(s, separator: true, out result); + } + case 18: + { // {xxxxxxxxxxxxxxxx} + if (s[0] != '{' || s[17] != '}') + { + return false; + } + return TryDecode16Unsafe(s.Slice(1, s.Length - 2), separator: false, out result); + } case 19: { // {xxxxxxxx-xxxxxxxx} if (s[0] != '{' || s[18] != '}') { return false; } - return TryDecode16(s.ToCharArray(), 1, true, out result); + return TryDecode16Unsafe(s.Slice(1, s.Length - 2), separator: true, out result); + } + default: + { + return false; + } + } + } + + public static bool TryParseBase62(ReadOnlySpan s, out Uuid64 result) + { + if (s.Length == 0) + { + result = default(Uuid64); + return true; + } + + if (s.Length <= 11 && Base62.TryDecode(s, out ulong x)) + { + result = new Uuid64(x); + return true; + } + + result = default(Uuid64); + return false; + } + +#else + + /// Parse a string representation of an UUid64 + /// String in either formats: "", "badc0ffe-e0ddf00d", "badc0ffee0ddf00d", "{badc0ffe-e0ddf00d}", "{badc0ffee0ddf00d}" + /// Parsing is case-insensitive. The empty string is mapped to Uuid64.Empty. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Uuid64 Parse([NotNull] string buffer) + { + Contract.NotNull(buffer, nameof(buffer)); + unsafe + { + fixed (char* chars = buffer) + { + if (!TryParse(chars, buffer.Length, out var value)) + { + throw FailInvalidFormat(); + } + + return value; + } + } + } + + /// Parse a string representation of an UUid64 + [Pure] + [Obsolete("Use Uuid64.Parse(ReadOnlySpan) instead", error: true)] //TODO: remove me! + public static unsafe Uuid64 Parse(char* chars, int numChars) + { + if (numChars == 0) return default(Uuid64); + if (!TryParse(chars, numChars, out var value)) + { + throw FailInvalidFormat(); + } + return value; + } + + /// Parse a Base62 encoded string representation of an UUid64 + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static Uuid64 FromBase62([NotNull] string buffer) + { + Contract.NotNull(buffer, nameof(buffer)); + unsafe + { + fixed (char* chars = buffer) + { + if (!TryParseBase62(chars, buffer.Length, out var value)) + { + throw FailInvalidFormat(); + } + + return value; + } + } + } + + /// Try parsing a string representation of an UUid64 + public static bool TryParse([NotNull] string buffer, out Uuid64 result) + { + Contract.NotNull(buffer, nameof(buffer)); + unsafe + { + fixed (char* chars = buffer) + { + return TryParse(chars, buffer.Length, out result); + } + } + } + + /// Try parsing a string representation of an UUid64 + public static unsafe bool TryParse(char* chars, int numChars, out Uuid64 result) + { + Contract.Requires(chars != null && numChars >= 0); + + // we support the following formats: "{hex8-hex8}", "{hex16}", "hex8-hex8", "hex16" and "base62" + // we don't support base10 format, because there is no way to differentiate from hex or base62 + + result = default(Uuid64); + switch (numChars) + { + case 0: + { // empty + return true; + } + case 16: + { // xxxxxxxxxxxxxxxx + return TryDecode16Unsafe(chars, numChars, false, out result); + } + case 17: + { // xxxxxxxx-xxxxxxxx + if (chars[8] != '-') return false; + return TryDecode16Unsafe(chars, numChars, true, out result); } case 18: { // {xxxxxxxxxxxxxxxx} - if (s[0] != '{' || s[17] != '}') + if (chars[0] != '{' || chars[17] != '}') { return false; } - return TryDecode16(s.ToCharArray(), 1, false, out result); + return TryDecode16Unsafe(chars + 1, numChars - 2, false, out result); } - case 17: - { // xxxxxxxx-xxxxxxxx - if (s[8] != '-') return false; - return TryDecode16(s.ToCharArray(), 0, true, out result); + case 19: + { // {xxxxxxxx-xxxxxxxx} + if (chars[0] != '{' || chars[18] != '}') + { + return false; + } + return TryDecode16Unsafe(chars + 1, numChars - 2, true, out result); } - case 16: - { // xxxxxxxxxxxxxxxx - return TryDecode16(s.ToCharArray(), 0, false, out result); + default: + { + return false; } } + } - // only base62 is allowed - if (s.Length <= 11) + public static unsafe bool TryParseBase62(char* chars, int numChars, out Uuid64 result) + { + if (numChars == 0) { - return TryDecode62(s.ToCharArray(), out result); + result = default(Uuid64); + return true; } + if (numChars <= 11 && Base62.TryDecode(chars, numChars, out ulong x)) + { + result = new Uuid64(x); + return true; + } + + result = default(Uuid64); return false; + } +#endif + #endregion #region Casting... + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static implicit operator Uuid64(ulong value) { return new Uuid64(value); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static explicit operator ulong(Uuid64 value) { return value.m_value; } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static implicit operator Uuid64(long value) { return new Uuid64(value); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static explicit operator long(Uuid64 value) { - return (long)value.m_value; + return (long) value.m_value; } #endregion #region IFormattable... + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public long ToInt64() { - return (long)m_value; + return (long) m_value; } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public ulong ToUInt64() { return m_value; } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice ToSlice() { return Slice.FromFixedU64BE(m_value); } + [Pure, NotNull] public byte[] ToByteArray() { var bytes = Slice.FromFixedU64BE(m_value).Array; - Contract.Assert(bytes != null && bytes.Length == 8); // HACKHACK: for perf reasons, we rely on the fact that Slice.FromFixedU64BE() allocates a new 8-byte array that we can return without copying + Contract.Ensures(bytes != null && bytes.Length == 8); // HACKHACK: for perf reasons, we rely on the fact that Slice.FromFixedU64BE() allocates a new 8-byte array that we can return without copying return bytes; } @@ -216,27 +510,19 @@ public byte[] ToByteArray() /// Strings returned by this method will always to 17 characters long. public override string ToString() { - return ToString(null, null); + return ToString("D", null); } /// Returns a string representation of the value of this instance, according to the provided format specifier. /// A single format specifier that indicates how to format the value of this Guid. The format parameter can be "D", "B", "X", "G", "Z" or "N". If format is null or an empty string (""), "D" is used. /// The value of this , using the specified format. /// See for a description of the different formats + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public string ToString(string format) { return ToString(format, null); } - /// Returns a string representation of the value of this instance. - /// This argument is ignored - /// String using the format "xxxxxxxx-xxxxxxxx", where 'x' is a lower-case hexadecimal digit - /// Strings returned by this method will always to 17 characters long. - public string ToString(IFormatProvider formatProvider) - { - return ToString("D", null); - } - /// Returns a string representation of the value of this instance of the class, according to the provided format specifier and culture-specific format information. /// A single format specifier that indicates how to format the value of this Guid. The format parameter can be "D", "N", "Z", "R", "X" or "B". If format is null or an empty string (""), "D" is used. /// An object that supplies culture-specific formatting information. Only used for the "R" format. @@ -267,12 +553,12 @@ public string ToString(string format, IFormatProvider formatProvider) case "C": case "c": { // base 62, compact, no padding - return Encode62(m_value, padded: false); + return Base62.Encode(m_value, padded: false); } case "Z": case "z": { // base 62, padded with '0' up to 11 chars - return Encode62(m_value, padded: true); + return Base62.Encode(m_value, padded: true); } case "R": @@ -300,8 +586,11 @@ public string ToString(string format, IFormatProvider formatProvider) { // "{xxxxxxxx-xxxxxxxx}" return Encode16(m_value, separator: true, quotes: true, upper: false); } + default: + { + throw new FormatException("Invalid " + nameof(Uuid64) + " format specification."); + } } - throw new FormatException("Invalid Uuid64 format specification."); } #endregion @@ -310,16 +599,19 @@ public string ToString(string format, IFormatProvider formatProvider) public override bool Equals(object obj) { - if (obj is Uuid64) return Equals((Uuid64)obj); - if (obj is ulong) return m_value == (ulong)obj; - if (obj is long) return m_value == (ulong)(long)obj; - //TODO: string format ? Slice ? + switch (obj) + { + case Uuid64 u64: return Equals(u64); + case ulong ul: return m_value == ul; + case long l: return m_value == (ulong) l; + //TODO: string format ? Slice ? + } return false; } public override int GetHashCode() { - return ((int)m_value) ^ (int)(m_value >> 32); + return ((int) m_value) ^ (int) (m_value >> 32); } public bool Equals(Uuid64 other) @@ -336,13 +628,15 @@ public int CompareTo(Uuid64 other) #region Base16 encoding... + [Pure] private static char HexToLowerChar(int a) { a &= 0xF; return a > 9 ? (char)(a - 10 + 'a') : (char)(a + '0'); } - private static unsafe char* HexsToLowerChars(char* ptr, int a) + [NotNull] + private static unsafe char* HexsToLowerChars([NotNull] char* ptr, int a) { Contract.Requires(ptr != null); ptr[0] = HexToLowerChar(a >> 28); @@ -356,13 +650,15 @@ private static char HexToLowerChar(int a) return ptr + 8; } + [Pure] private static char HexToUpperChar(int a) { a &= 0xF; return a > 9 ? (char)(a - 10 + 'A') : (char)(a + '0'); } - private static unsafe char* HexsToUpperChars(char* ptr, int a) + [NotNull] + private static unsafe char* HexsToUpperChars([NotNull] char* ptr, int a) { Contract.Requires(ptr != null); ptr[0] = HexToUpperChar(a >> 28); @@ -376,14 +672,15 @@ private static char HexToUpperChar(int a) return ptr + 8; } - private unsafe static string Encode16(ulong value, bool separator, bool quotes, bool upper) + [Pure, NotNull] + private static unsafe string Encode16(ulong value, bool separator, bool quotes, bool upper) { int size = 16 + (separator ? 1 : 0) + (quotes ? 2 : 0); char* buffer = stackalloc char[24]; // max 19 mais on arrondi a 24 char* ptr = buffer; if (quotes) *ptr++ = '{'; - ptr = upper + ptr = upper ? HexsToUpperChars(ptr, (int)(value >> 32)) : HexsToLowerChars(ptr, (int)(value >> 32)); if (separator) *ptr++ = '-'; @@ -392,12 +689,13 @@ private unsafe static string Encode16(ulong value, bool separator, bool quotes, : HexsToLowerChars(ptr, (int)(value & 0xFFFFFFFF)); if (quotes) *ptr++ = '}'; - Contract.Assert(ptr == buffer + size); + Contract.Ensures(ptr == buffer + size); return new string(buffer, 0, size); } private const int INVALID_CHAR = -1; + [Pure] private static int CharToHex(char c) { if (c <= '9') @@ -415,12 +713,14 @@ private static int CharToHex(char c) return INVALID_CHAR; } - private static bool TryCharsToHexs(char[] chars, int offset, out uint result) +#if ENABLE_SPAN + + private static bool TryCharsToHexsUnsafe(ReadOnlySpan chars, out uint result) { int word = 0; for (int i = 0; i < 8; i++) { - int a = CharToHex(chars[offset++]); + int a = CharToHex(chars[i]); if (a == INVALID_CHAR) { result = 0; @@ -432,22 +732,53 @@ private static bool TryCharsToHexs(char[] chars, int offset, out uint result) return true; } - private static bool TryDecode16(char[] chars, int offset, bool separator, out ulong result) + private static bool TryDecode16Unsafe(ReadOnlySpan chars, bool separator, out Uuid64 result) { - uint a, b; - - if ((!separator || chars[offset + 8] == '-') - && TryCharsToHexs(chars, offset, out a) - && TryCharsToHexs(chars, offset + (separator ? 9 : 8), out b)) + if ((!separator || chars[8] == '-') + && TryCharsToHexsUnsafe(chars, out uint hi) + && TryCharsToHexsUnsafe(chars.Slice(separator ? 9 : 8), out uint lo)) { - result = ((ulong)a << 32) | (ulong)b; + result = new Uuid64(((ulong)hi << 32) | lo); return true; } + result = default(Uuid64); + return false; + } + +#else - result = 0; + private static unsafe bool TryCharsToHexsUnsafe(char* chars, int numChars, out uint result) + { + int word = 0; + for (int i = 0; i < 8; i++) + { + int a = CharToHex(chars[i]); + if (a == INVALID_CHAR) + { + result = 0; + return false; + } + word = (word << 4) | a; + } + result = (uint)word; + return true; + } + + private static unsafe bool TryDecode16Unsafe(char* chars, int numChars, bool separator, out Uuid64 result) + { + if ((!separator || chars[8] == '-') + && TryCharsToHexsUnsafe(chars, numChars, out uint hi) + && TryCharsToHexsUnsafe(chars + (separator ? 9 : 8), numChars - (separator ? 9 : 8), out uint lo)) + { + result = new Uuid64(((ulong)hi << 32) | lo); + return true; + } + result = default(Uuid64); return false; } +#endif + #endregion #region Base62 encoding... @@ -455,159 +786,283 @@ private static bool TryDecode16(char[] chars, int offset, bool separator, out ul //NOTE: this version of base62 encoding puts the digits BEFORE the letters, to ensure that the string representation of a UUID64 is in the same order as its byte[] or ulong version. // => This scheme use the "0-9A-Za-z" ordering, while most other base62 encoder use "a-zA-Z0-9" - private static readonly char[] Base62LexicographicChars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz".ToCharArray(); - private static readonly int[] Base62Values = new int[3 * 32] - { - /* 32.. 63 */ -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, -1, -1, -1, -1, -1, -1, - /* 64.. 95 */ -1, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, -1, -1, -1, -1, -1, - /* 96..127 */ -1, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, -1, -1, -1, -1, -1, - }; - - /// Encode a 64-bit value into a base-62 string - /// 64-bit value to encode - /// If true, keep the leading '0' to return a string of length 11. If false, discards all extra leading '0' digits. - /// String that contains only digits, lower and upper case letters. The string will be lexicographically ordered, which means that sorting by string will give the same order as sorting by value. - /// - /// Encode62(0, false) => "0" - /// Encode62(0, true) => "00000000000" - /// Encode62(0xDEADBEEF) => "" - /// - private static string Encode62(ulong value, bool padded) - { - // special case for default(Uuid64) which may be more frequent than others - if (value == 0) return padded ? "00000000000" : "0"; - - // encoding a 64 bits value in Base62 yields 10.75 "digits", which is rounded up to 11 chars. - const int MAX_SIZE = 11; + private static class Base62 + { + //note: nested static class, so that we only allocate the internal buffers if Base62 encoding is actually used + + private static readonly char[] Base62LexicographicChars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz".ToCharArray(); - unsafe + private static readonly int[] Base62Values = new int[3 * 32] + { + /* 32.. 63 */ -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, -1, -1, -1, -1, -1, -1, + /* 64.. 95 */ -1, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, -1, -1, -1, -1, -1, + /* 96..127 */ -1, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, -1, -1, -1, -1, -1, + }; + + /// Encode a 64-bit value into a base-62 string + /// 64-bit value to encode + /// If true, keep the leading '0' to return a string of length 11. If false, discards all extra leading '0' digits. + /// String that contains only digits, lower and upper case letters. The string will be lexicographically ordered, which means that sorting by string will give the same order as sorting by value. + /// + /// Encode62(0, false) => "0" + /// Encode62(0, true) => "00000000000" + /// Encode62(0xDEADBEEF) => "" + /// + public static string Encode(ulong value, bool padded) { - // The maximum size is 11 chars, but we will allocate 64 bytes on the stack to keep alignment. - char* chars = stackalloc char[16]; - char[] bc = Base62LexicographicChars; + // special case for default(Uuid64) which may be more frequent than others + if (value == 0) return padded ? "00000000000" : "0"; - // start from the last "digit" - char* pc = chars + (MAX_SIZE - 1); + // encoding a 64 bits value in Base62 yields 10.75 "digits", which is rounded up to 11 chars. + const int MAX_SIZE = 11; - while (pc >= chars) + unsafe { - ulong r = value % 62L; - value /= 62L; - *pc-- = bc[(int)r]; - if (!padded && value == 0) - { // the rest will be all zeroes - break; + // The maximum size is 11 chars, but we will allocate 64 bytes on the stack to keep alignment. + char* chars = stackalloc char[16]; + char[] bc = Base62LexicographicChars; + + // start from the last "digit" + char* pc = chars + (MAX_SIZE - 1); + + while (pc >= chars) + { + ulong r = value % 62L; + value /= 62L; + *pc-- = bc[(int) r]; + if (!padded && value == 0) + { // the rest will be all zeroes + break; + } } + + ++pc; + int count = MAX_SIZE - (int) (pc - chars); + Contract.Assert(count > 0 && count <= 11); + return count <= 0 ? String.Empty : new string(pc, 0, count); } + } - ++pc; - int count = MAX_SIZE - (int)(pc - chars); - Contract.Assert(count > 0 && count <= 11); - return count <= 0 ? String.Empty : new string(pc, 0, count); +#if ENABLE_SPAN + + public static bool TryDecode(char[] s, out ulong value) + { + if (s == null) { value = 0; return false; } + return TryDecode(new ReadOnlySpan(s), out value); } - } - private static bool TryDecode62(char[] s, out ulong value) - { - if (s == null || s.Length == 0 || s.Length > 11) - { // fail: too small/too big - value = 0; - return false; + public static bool TryDecode(ReadOnlySpan s, out ulong value) + { + if (s == null || s.Length == 0 || s.Length > 11) + { // fail: too small/too big + value = 0; + return false; + } + + // we know that the original value is exactly 64bits, and any missing digit is '0' + ulong factor = 1UL; + ulong acc = 0UL; + int p = s.Length - 1; + int[] bv = Base62Values; + while (p >= 0) + { + // read digit + int a = s[p]; + // decode base62 digit + a = a >= 32 && a < 128 ? bv[a - 32] : -1; + if (a == -1) + { // fail: invalid character + value = 0; + return false; + } + // accumulate, while checking for overflow + acc = checked(acc + ((ulong) a * factor)); + if (p-- > 0) factor *= 62; + } + value = acc; + return true; } - // we know that the original value is exactly 64bits, and any missing digit is '0' - ulong factor = 1UL; - ulong acc = 0UL; - int p = s.Length - 1; - int[] bv = Base62Values; - while (p >= 0) +#else + + + public static bool TryDecode(char[] s, out ulong value) { - // read digit - int a = s[p]; - // decode base62 digit - a = a >= 32 && a < 128 ? bv[a - 32] : -1; - if (a == -1) - { // fail: invalid character + if (s == null) { value = 0; return false; } + + unsafe + { + fixed (char* chars = s) + { + return TryDecode(chars, s.Length, out value); + } + } + } + + public static unsafe bool TryDecode(char* chars, int numChars, out ulong value) + { + if (chars == null || numChars == 0 || numChars > 11) + { // fail: too small/too big value = 0; return false; } - // accumulate, while checking for overflow - acc = checked(acc + ((ulong)a * factor)); - if (p-- > 0) factor *= 62; + + // we know that the original value is exactly 64bits, and any missing digit is '0' + ulong factor = 1UL; + ulong acc = 0UL; + int p = numChars - 1; + int[] bv = Base62Values; + while (p >= 0) + { + // read digit + int a = chars[p]; + // decode base62 digit + a = a >= 32 && a < 128 ? bv[a - 32] : -1; + if (a == -1) + { // fail: invalid character + value = 0; + return false; + } + // accumulate, while checking for overflow + acc = checked(acc + ((ulong) a * factor)); + if (p-- > 0) factor *= 62; + } + value = acc; + return true; } - value = acc; - return true; + +#endif + } #endregion - #region Fast I/O... + #region Unsafe I/O... - internal static ulong Read(byte[] buffer, int offset) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe ulong ReadUnsafe([NotNull] byte* src) { - Contract.Requires(buffer != null && offset >= 0 && offset + 7 < buffer.Length); - // buffer contains the bytes in Big Endian - ulong res = buffer[offset + 7]; - res |= ((ulong)buffer[offset + 6]) << 8; - res |= ((ulong)buffer[offset + 5]) << 16; - res |= ((ulong)buffer[offset + 4]) << 24; - res |= ((ulong)buffer[offset + 3]) << 32; - res |= ((ulong)buffer[offset + 2]) << 40; - res |= ((ulong)buffer[offset + 1]) << 48; - res |= ((ulong)buffer[offset + 0]) << 56; - return res; + //Contract.Requires(src != null); + return UnsafeHelpers.LoadUInt64BE(src); } - internal unsafe static ulong Read(byte* src) +#if ENABLE_SPAN + internal static unsafe ulong ReadUnsafe(ReadOnlySpan src) { - ulong tmp; + //Contract.Requires(src.Length >= 0); + fixed (byte* ptr = &MemoryMarshal.GetReference(src)) + { + return UnsafeHelpers.LoadUInt64BE(ptr); + } + } +#endif - if (BitConverter.IsLittleEndian) - { // Intel ? - byte* ptr = (byte*)&tmp; - // big endian - ptr[0] = src[7]; - ptr[1] = src[6]; - ptr[2] = src[5]; - ptr[3] = src[4]; - ptr[4] = src[3]; - ptr[5] = src[2]; - ptr[6] = src[1]; - ptr[7] = src[0]; + [Pure] + public static ulong ReadUnsafe([NotNull] byte[] buffer, int offset) + { + //Contract.Requires(buffer != null && offset >= 0 && offset + 7 < buffer.Length); + // buffer contains the bytes in Big Endian + unsafe + { + fixed (byte* ptr = &buffer[offset]) + { + return UnsafeHelpers.LoadUInt64BE(ptr); + } } - else - { // ARM ? - tmp = *((ulong*)src); + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static unsafe void WriteUnsafe(ulong value, byte* ptr) + { + //Contract.Requires(ptr != null); + UnsafeHelpers.StoreUInt64BE(ptr, value); + } + + public static void WriteUnsafe(ulong value, [NotNull] byte[] buffer, int offset) + { + //Contract.Requires(buffer != null && offset >= 0 && offset + 7 < buffer.Length); + unsafe + { + fixed (byte* ptr = &buffer[offset]) + { + UnsafeHelpers.StoreUInt64BE(ptr, value); + } } + } - return tmp; + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public unsafe void WriteToUnsafe([NotNull] byte* ptr) + { + WriteUnsafe(m_value, ptr); } - internal unsafe static void Write(ulong value, byte* ptr) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteToUnsafe([NotNull] byte[] buffer, int offset) { - if (BitConverter.IsLittleEndian) - { // Intel ? - byte* src = (byte*)&value; - ptr[0] = src[7]; - ptr[1] = src[6]; - ptr[2] = src[5]; - ptr[3] = src[4]; - ptr[4] = src[3]; - ptr[5] = src[2]; - ptr[6] = src[1]; - ptr[7] = src[0]; + WriteUnsafe(m_value, buffer, offset); + } + +#if ENABLE_SPAN + public void WriteTo(byte[] buffer, int offset) + { + WriteTo(buffer.AsSpan(offset)); + } + + public void WriteTo(Span destination) + { + if (destination.Length < 8) throw FailInvalidBufferSize(nameof(destination)); + unsafe + { + fixed (byte* ptr = &MemoryMarshal.GetReference(destination)) + { + WriteUnsafe(m_value, ptr); + } } - else - { // ARM ? - *((ulong*)ptr) = value; + } + + public bool TryWriteTo(Span destination) + { + if (destination.Length < 8) return false; + unsafe + { + fixed (byte* ptr = &MemoryMarshal.GetReference(destination)) + { + WriteUnsafe(m_value, ptr); + return true; + } } + } +#else + public void WriteTo(byte[] buffer, int offset) + { + WriteTo(buffer.AsSlice(offset)); + } + public void WriteTo(Slice destination) + { + if (destination.Count < 8) throw FailInvalidBufferSize(nameof(destination)); + unsafe + { + fixed (byte* ptr = &destination.DangerousGetPinnableReference()) + { + WriteUnsafe(m_value, ptr); + } + } } - internal unsafe void WriteTo(byte* ptr) + public bool TryWriteTo(Slice destination) { - Write(m_value, ptr); + if (destination.Count < 8) return false; + unsafe + { + fixed (byte* ptr = &destination.DangerousGetPinnableReference()) + { + WriteUnsafe(m_value, ptr); + return true; + } + } } +#endif #endregion @@ -707,18 +1162,48 @@ internal unsafe void WriteTo(byte* ptr) #endregion + /// Instance of this times can be used to test Uuid64 for equality and ordering + public sealed class Comparer : IEqualityComparer, IComparer + { + + public static readonly Comparer Default = new Comparer(); + + private Comparer() + { } + + public bool Equals(Uuid64 x, Uuid64 y) + { + return x.m_value == y.m_value; + } + + public int GetHashCode(Uuid64 obj) + { + return obj.m_value.GetHashCode(); + } + + public int Compare(Uuid64 x, Uuid64 y) + { + return x.m_value.CompareTo(y.m_value); + } + } + } - /// Helper class for generating 64-bit UUIDs from a secure random number generator + /// Generates 64-bit UUIDs using a secure random number generator + /// Methods of this type are thread-safe. public sealed class Uuid64RandomGenerator { /// Default instance of a random generator /// Using this instance will introduce a global lock in your application. You can create specific instances for worker threads, if you require concurrency. + [NotNull] public static readonly Uuid64RandomGenerator Default = new Uuid64RandomGenerator(); - private readonly System.Security.Cryptography.RandomNumberGenerator m_rng; - private readonly byte[] m_scratch = new byte[8]; + [NotNull] + private RandomNumberGenerator Rng { get; } + + [NotNull] + private readonly byte[] Scratch = new byte[8]; /// Create a new instance of a random UUID generator public Uuid64RandomGenerator() @@ -726,9 +1211,9 @@ public Uuid64RandomGenerator() { } /// Create a new instance of a random UUID generator, using a specific random number generator - public Uuid64RandomGenerator(System.Security.Cryptography.RandomNumberGenerator generator) + public Uuid64RandomGenerator(RandomNumberGenerator generator) { - m_rng = generator ?? System.Security.Cryptography.RandomNumberGenerator.Create(); + this.Rng = generator ?? RandomNumberGenerator.Create(); } /// Return a new random 64-bit UUID @@ -737,13 +1222,18 @@ public Uuid64RandomGenerator(System.Security.Cryptography.RandomNumberGenerator ///

This methods needs to acquire a lock. If multiple threads needs to generate ids concurrently, you may need to create an instance of this class for each threads.

///

The uniqueness of the generated uuids depends on the quality of the random number generator. If you cannot tolerate collisions, you either have to check if a newly generated uid already exists, or use a different kind of generator.

/// + [Pure] public Uuid64 NewUuid() { - lock (m_rng) + //REVIEW: OPTIMIZE: use a per-thread instance of the rng and scratch buffer? + // => right now, NewUuid() is a Global Lock for the whole process! + lock (this.Rng) { // get 8 bytes of randomness (0 allowed) - m_rng.GetBytes(m_scratch); - return new Uuid64(m_scratch); + this.Rng.GetBytes(this.Scratch); + //note: do *NOT* call GetBytes(byte[], int, int) because it creates creates a temp buffer, calls GetBytes(byte[]) and copy the result back! (as of .NET 4.7.1) + //TODO: PERF: use Span APIs once (if?) they become available! + return Uuid64.Read(this.Scratch); } } diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 2cf43d9be..8ed5fc450 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -218,7 +218,7 @@ public void Delete([NotNull] IFdbTransaction trans) if (value.IsNullOrEmpty) return default(long?); //note: python code stores the size as a string - long size = Int64.Parse(value.ToAscii()); + long size = Int64.Parse(value.ToString()); if (size < 0) throw new InvalidOperationException("The internal blob size cannot be negative"); return size; } diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 740012f7a..01192c4ca 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -76,9 +76,9 @@ public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] this.Encoder = encoder; //TODO: rewrite this, using FdbEncoderSubpsace<..> ! - this.ConflictedPop = this.Subspace.Partition.ByKey(Slice.FromAscii("pop")); - this.ConflictedItem = this.Subspace.Partition.ByKey(Slice.FromAscii("conflict")); - this.QueueItem = this.Subspace.Partition.ByKey(Slice.FromAscii("item")); + this.ConflictedPop = this.Subspace.Partition.ByKey(Slice.FromStringAscii("pop")); + this.ConflictedItem = this.Subspace.Partition.ByKey(Slice.FromStringAscii("conflict")); + this.QueueItem = this.Subspace.Partition.ByKey(Slice.FromStringAscii("item")); } /// Subspace used as a prefix for all items in this table diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index fa76eb686..fd69098f4 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -28,14 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Indexing { - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Diagnostics; - using System.Globalization; using System.Threading.Tasks; + using FoundationDB.Client; + using JetBrains.Annotations; /// Simple index that maps values of type into lists of ids of type /// Type of the unique id of each document or entity diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index a112db88b..b646a694a 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -30,14 +30,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Interning { - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using System; using System.Collections.Generic; using System.Diagnostics; using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; /// Provides a class for interning (aka normalizing, aliasing) commonly-used long strings into shorter representations. [DebuggerDisplay("Subspace={Subspace}")] diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs index 747a8c2e5..979f37d30 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs @@ -31,8 +31,7 @@ namespace FoundationDB.Layers.Documents using System; using System.Collections.Generic; using System.Linq; - using FoundationDB.Client; - using FoundationDB.Layers.Tuples; + using Doxense.Collections.Tuples; /// Interface that defines a class that knows of to chop instances of into slices /// Type of documents @@ -103,7 +102,7 @@ public KeyValuePair[] Split(List> do // convert into tuples .Select(kvp => new KeyValuePair( STuple.Create(kvp.Key), - STuple.Create(kvp.Value).ToSlice() + TuPack.Pack(kvp.Value) )) .ToArray(); } @@ -117,7 +116,7 @@ public List> Build(KeyValuePair[] pa { list.Add(new KeyValuePair( part.Key.Last(), - STuple.Unpack(part.Value) + TuPack.Unpack(part.Value) )); } return list; diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 200ee0b41..4113c952c 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -31,9 +31,9 @@ namespace FoundationDB.Layers.Blobs using System; using System.Collections.Generic; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; // THIS IS NOT AN OFFICIAL LAYER, JUST A PROTOTYPE TO TEST A FEW THINGS ! @@ -127,7 +127,7 @@ public async Task> GetAsync([NotNull] IFdbReadOnlyTra if (id == null) throw new ArgumentNullException(nameof(id)); if (fields == null) throw new ArgumentNullException(nameof(fields)); - var keys = STuple.EncodePrefixedKeys(GetKey(id), fields); + var keys = TuPack.EncodePrefixedKeys(GetKey(id), fields); var values = await trans.GetValuesAsync(keys).ConfigureAwait(false); Contract.Assert(values != null && values.Length == fields.Length); @@ -232,7 +232,7 @@ public Task> GetKeys(IFdbReadOnlyTransaction trans, ITuple id) return trans .GetRange(KeyRange.StartsWith(prefix)) - .Select((kvp) => ParseFieldKey(STuple.Unpack(kvp.Key))) + .Select((kvp) => ParseFieldKey(TuPack.Unpack(kvp.Key))) .ToListAsync(); } diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs index 632b67762..325467032 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapBuilder.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Layers.Experimental.Indexing using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; using JetBrains.Annotations; @@ -94,7 +95,7 @@ internal static CompressedWord[] DecodeWords(Slice data, int size, BitRange boun { Contract.Requires(size >= 0 && data.Count >= 4 && (data.Count & 3) == 0); - int capacity = SliceHelpers.NextPowerOfTwo(size); + int capacity = BitHelpers.NextPowerOfTwo(size); if (capacity < 0) capacity = size; var words = new CompressedWord[capacity]; @@ -154,7 +155,7 @@ public void EnsureCapacity(int minSize) { if (minSize > m_size) { - int newSize = SliceHelpers.NextPowerOfTwo(minSize); + int newSize = BitHelpers.NextPowerOfTwo(minSize); if (newSize < 0) newSize = minSize; if (newSize < 8) newSize = 8; Array.Resize(ref m_words, newSize); diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs index 9474919e1..5a77a644c 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWordIterator.cs @@ -31,7 +31,7 @@ namespace FoundationDB.Layers.Experimental.Indexing using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; + using Doxense.Memory; /// Iterator that reads 32-bit compressed words from a compressed bitmap public struct CompressedBitmapWordIterator : IEnumerator diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs index 3a812b954..75406e89e 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmapWriter.cs @@ -30,6 +30,7 @@ namespace FoundationDB.Layers.Experimental.Indexing { using System; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; using JetBrains.Annotations; @@ -64,7 +65,7 @@ public sealed class CompressedBitmapWriter /// Create a new compressed bitmap writer public CompressedBitmapWriter() - : this(SliceWriter.Empty, true) + : this(default(SliceWriter), true) { } /// Create a new compressed bitmap writer, with a hint for the initial capacity diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs index 4d501db3d..59d676d34 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/WordAlignHybridCoding.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Layers.Experimental.Indexing using System; using System.Text; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; using JetBrains.Annotations; diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 3660bdb13..3ac58d757 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -169,7 +169,7 @@ private async Task PushQueueAsync(IFdbTransaction tr, IDynamicKeySubspace queue, private void StoreTask(IFdbTransaction tr, Slice taskId, DateTime scheduledUtc, Slice taskBody) { - tr.Annotate("Writing task {0}", taskId.ToAsciiOrHexaString()); + tr.Annotate("Writing task {0:P}", taskId); var prefix = this.TaskStore.Partition.ByKey(taskId); @@ -183,7 +183,7 @@ private void StoreTask(IFdbTransaction tr, Slice taskId, DateTime scheduledUtc, private void ClearTask(IFdbTransaction tr, Slice taskId) { - tr.Annotate("Deleting task {0}", taskId.ToAsciiOrHexaString()); + tr.Annotate("Deleting task {0:P}", taskId); // clear all metadata about the task tr.ClearRange(KeyRange.StartsWith(this.TaskStore.Keys.Encode(taskId))); @@ -206,9 +206,9 @@ await db.ReadWriteAsync(async (tr) => { Interlocked.Increment(ref m_schedulingAttempts); #if DEBUG - if (tr.Context.Retries > 0) Console.WriteLine("# retry n°" + tr.Context.Retries + " for task " + taskId.ToAsciiOrHexaString()); + if (tr.Context.Retries > 0) Console.WriteLine($"# retry n°{tr.Context.Retries} for task {taskId:P}"); #endif - tr.Annotate("I want to schedule {0}", taskId.ToAsciiOrHexaString()); + tr.Annotate("I want to schedule {0:P}", taskId); // find a random worker from the idle ring var randomWorkerKey = await FindRandomItem(tr, this.IdleRing).ConfigureAwait(false); @@ -217,7 +217,7 @@ await db.ReadWriteAsync(async (tr) => { Slice workerId = this.IdleRing.Keys.Decode(randomWorkerKey.Key); - tr.Annotate("Assigning {0} to {1}", taskId.ToAsciiOrHexaString(), workerId.ToAsciiOrHexaString()); + tr.Annotate("Assigning {0:P} to {1:P}", taskId, workerId); // remove worker from the idle ring tr.Clear(this.IdleRing.Keys.Encode(workerId)); @@ -229,7 +229,7 @@ await db.ReadWriteAsync(async (tr) => } else { - tr.Annotate("Queueing {0}", taskId.ToAsciiOrHexaString()); + tr.Annotate("Queueing {0:P}", taskId); await PushQueueAsync(tr, this.UnassignedTaskRing, taskId).ConfigureAwait(false); } @@ -268,7 +268,7 @@ public async Task RunWorkerAsync(IFdbDatabase db, Func { - tr.Annotate("I'm worker #{0} with id {1}", num, workerId.ToAsciiOrHexaString()); + tr.Annotate("I'm worker #{0} with id {1:P}", num, workerId); myId = workerId; watch = default(FdbWatch); @@ -302,13 +302,13 @@ await db.ReadWriteAsync( { // mark this worker as busy // note: we need a random id so generate one if it is the first time... if (!myId.IsPresent) myId = GetRandomId(); - tr.Annotate("Found {0}, switch to busy with id {1}", msg.Id.ToAsciiOrHexaString(), myId.ToAsciiOrHexaString()); + tr.Annotate("Found {0:P}, switch to busy with id {1:P}", msg.Id, myId); tr.Set(this.BusyRing.Keys.Encode(myId), msg.Id); this.Counters.Increment(tr, COUNTER_BUSY); } else if (myId.IsPresent) { // remove ourselves from the busy ring - tr.Annotate("Found nothing, switch to idle with id {0}", myId.ToAsciiOrHexaString()); + tr.Annotate("Found nothing, switch to idle with id {0:P}", myId); //tr.Clear(this.BusyRing.Pack(myId)); } } @@ -316,7 +316,7 @@ await db.ReadWriteAsync( if (msg.Id.IsPresent) { // get the task body - tr.Annotate("Fetching body for task {0}", msg.Id.ToAsciiOrHexaString()); + tr.Annotate("Fetching body for task {0:P}", msg.Id); var prefix = this.TaskStore.Partition.ByKey(msg.Id); //TODO: replace this with a get_range ? var data = await tr.GetValuesAsync(new [] { @@ -343,7 +343,7 @@ await db.ReadWriteAsync( // the idle key will also be used as the watch key to wake us up var watchKey = this.IdleRing.Keys.Encode(myId); - tr.Annotate("Will start watching on key {0} with id {1}", watchKey.ToAsciiOrHexaString(), myId.ToAsciiOrHexaString()); + tr.Annotate("Will start watching on key {0:P} with id {1:P}", watchKey, myId); tr.Set(watchKey, Slice.Empty); this.Counters.Increment(tr, COUNTER_IDLE); @@ -381,7 +381,7 @@ await db.ReadWriteAsync( { // the task has been dropped? // TODO: loggin? #if DEBUG - Console.WriteLine("[####] Task[" + msg.Id.ToAsciiOrHexaString() + "] has vanished?"); + Console.WriteLine($"[####] Task[{msg.Id:P}] has vanished?"); #endif } else @@ -394,7 +394,7 @@ await db.ReadWriteAsync( { //TODO: logging? #if DEBUG - Console.Error.WriteLine("Task[" + msg.Id.ToAsciiOrHexaString() + "] failed: " + e.ToString()); + Console.Error.WriteLine($"Task[{msg.Id:P}] failed: {e}"); #endif } } diff --git a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs index 9297761e7..63d8ec3d6 100644 --- a/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs +++ b/FoundationDB.Layers.Experimental/Messaging/WorkerPoolTest.cs @@ -72,7 +72,7 @@ private async Task RunAsync(IFdbDatabase db, IDynamicKeySubspace location, Cance { var workerPool = new FdbWorkerPool(location); - Console.WriteLine("workerPool at " + location.GetPrefix().ToAsciiOrHexaString()); + Console.WriteLine($"workerPool at {location.GetPrefix():P}"); var workerSignal = new AsyncCancelableMutex(ct); var clientSignal = new AsyncCancelableMutex(ct); @@ -133,14 +133,14 @@ private async Task RunAsync(IFdbDatabase db, IDynamicKeySubspace location, Cance Func dump = async (label) => { - Console.WriteLine(""); + Console.WriteLine($""); using (var tr = db.BeginTransaction(ct)) { await tr.Snapshot .GetRange(KeyRange.StartsWith(location.GetPrefix())) .ForEachAsync((kvp) => { - Console.WriteLine(" - " + location.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); + Console.WriteLine($" - {location.Keys.Unpack(kvp.Key)} = {kvp.Value:V}"); }).ConfigureAwait(false); } Console.WriteLine(""); diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index bdae87da6..892e85292 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -26,16 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense.Linq; - namespace FoundationDB.Linq.Expressions { using System; using System.Linq.Expressions; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using JetBrains.Annotations; /// Helper class to construct Query Expressions @@ -89,7 +88,7 @@ public static FdbQueryRangeExpression RangeStartsWith(Slice prefix, FdbRangeOpti [NotNull] public static FdbQueryRangeExpression RangeStartsWith(ITuple tuple, FdbRangeOptions options = null) { - return Range(tuple.ToSelectorPair(), options); + return RangeStartsWith(TuPack.Pack(tuple), options); } /// Return the intersection between one of more sequences of results diff --git a/FoundationDB.Samples/Benchmarks/BenchRunner.cs b/FoundationDB.Samples/Benchmarks/BenchRunner.cs index 3b4ec443d..5fddc44d0 100644 --- a/FoundationDB.Samples/Benchmarks/BenchRunner.cs +++ b/FoundationDB.Samples/Benchmarks/BenchRunner.cs @@ -9,9 +9,9 @@ namespace FoundationDB.Samples.Benchmarks using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Mathematics.Statistics; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; public class BenchRunner : IAsyncTest { @@ -111,7 +111,7 @@ public async Task Run(IFdbDatabase db, TextWriter log, CancellationToken ct) } else { - var foos = STuple.EncodePrefixedKeys(foo, Enumerable.Range(1, this.Value).ToArray()); + var foos = TuPack.EncodePrefixedKeys(foo, Enumerable.Range(1, this.Value).ToArray()); await db.ReadAsync(tr => tr.GetValuesAsync(foos), ct); } break; diff --git a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs index 261a83ba5..1ae9df56a 100644 --- a/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs +++ b/FoundationDB.Samples/MessageQueue/MessageQueueRunner.cs @@ -2,10 +2,6 @@ namespace FoundationDB.Samples.Tutorials { - using Doxense.Mathematics.Statistics; - using FoundationDB.Client; - using FoundationDB.Layers.Messaging; - using FoundationDB.Layers.Tuples; using System; using System.Collections.Generic; using System.Diagnostics; @@ -13,6 +9,10 @@ namespace FoundationDB.Samples.Tutorials using System.IO; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using Doxense.Mathematics.Statistics; + using FoundationDB.Client; + using FoundationDB.Layers.Messaging; public class MessageQueueRunner : IAsyncTest { @@ -84,7 +84,7 @@ public async Task RunProducer(IFdbDatabase db, CancellationToken ct) while (!ct.IsCancellationRequested) { int k = cnt++; - Slice taskId = STuple.EncodeKey(this.Id.GetHashCode(), k); + Slice taskId = TuPack.EncodeKey(this.Id.GetHashCode(), k); string msg = "Message #" + k + " from producer " + this.Id + " (" + DateTime.UtcNow.ToString("O") + ")"; @@ -116,7 +116,7 @@ await this.WorkerPool.RunWorkerAsync(db, async (msg, _ct) => var latency = msg.Received - msg.Scheduled; Interlocked.Increment(ref received); - Console.Write("[" + received.ToString("N0") + " msg, ~" + latency.TotalMilliseconds.ToString("N3") + " ms] " + msg.Id.ToAsciiOrHexaString() + " \r"); + Console.Write($"[{received:N0} msg, ~{latency.TotalMilliseconds:N3} ms] {msg.Id:P} \r"); this.TimeLine.Add(latency.TotalMilliseconds); @@ -155,22 +155,22 @@ public async Task RunStatus(IFdbDatabase db, CancellationToken ct) Console.WriteLine("> Idle"); await tr.Snapshot.GetRange(idleLocation.Keys.ToRange()).ForEachAsync((kvp) => { - Console.WriteLine("- Idle." + idleLocation.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); + Console.WriteLine($"- Idle.{idleLocation.Keys.Unpack(kvp.Key)} = {kvp.Value:V}"); }); Console.WriteLine("> Busy"); await tr.Snapshot.GetRange(busyLocation.Keys.ToRange()).ForEachAsync((kvp) => { - Console.WriteLine("- Busy." + busyLocation.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); + Console.WriteLine($"- Busy.{busyLocation.Keys.Unpack(kvp.Key)} = {kvp.Value:V}"); }); Console.WriteLine("> Unassigned"); await tr.Snapshot.GetRange(unassignedLocation.Keys.ToRange()).ForEachAsync((kvp) => { - Console.WriteLine("- Unassigned." + unassignedLocation.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); + Console.WriteLine($"- Unassigned.{unassignedLocation.Keys.Unpack(kvp.Key)} = {kvp.Value:V}"); }); Console.WriteLine("> Tasks"); await tr.Snapshot.GetRange(tasksLocation.Keys.ToRange()).ForEachAsync((kvp) => { - Console.WriteLine("- Tasks." + tasksLocation.Keys.Unpack(kvp.Key) + " = " + kvp.Value.ToAsciiOrHexaString()); + Console.WriteLine($"- Tasks.{tasksLocation.Keys.Unpack(kvp.Key)} = {kvp.Value:V}"); }); Console.WriteLine("<"); } diff --git a/FoundationDB.Samples/Tutorials/ClassScheduling.cs b/FoundationDB.Samples/Tutorials/ClassScheduling.cs index fe7cfe862..cfdceb91f 100644 --- a/FoundationDB.Samples/Tutorials/ClassScheduling.cs +++ b/FoundationDB.Samples/Tutorials/ClassScheduling.cs @@ -8,9 +8,9 @@ namespace FoundationDB.Samples.Tutorials using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; public class ClassScheduling : IAsyncTest { @@ -62,7 +62,7 @@ await db.WriteAsync((tr) => { foreach (var c in this.ClassNames) { - tr.Set(ClassKey(c), Slice.FromAscii("100")); + tr.Set(ClassKey(c), Slice.FromStringAscii("100")); } }, ct); @@ -75,7 +75,7 @@ await db.WriteAsync((tr) => public Task> AvailableClasses(IFdbReadOnlyTransaction tr) { return tr.GetRange(this.Subspace.Keys.ToRange(STuple.Create("class"))) - .Where(kvp => { int _; return Int32.TryParse(kvp.Value.ToAscii(), out _); }) // (step 3) + .Where(kvp => { int _; return Int32.TryParse(kvp.Value.ToStringAscii(), out _); }) // (step 3) .Select(kvp => this.Subspace.Keys.Decode(kvp.Key)) .ToListAsync(); } @@ -91,7 +91,7 @@ public async Task Signup(IFdbTransaction tr, string s, string c) { // already signed up return; } - int seatsLeft = Int32.Parse((await tr.GetAsync(ClassKey(c))).ToAscii()); + int seatsLeft = Int32.Parse((await tr.GetAsync(ClassKey(c))).ToStringAscii()); if (seatsLeft <= 0) { throw new InvalidOperationException("No remaining seats"); @@ -100,7 +100,7 @@ public async Task Signup(IFdbTransaction tr, string s, string c) var classes = await tr.GetRange(AttendsKeys(s)).ToListAsync(); if (classes.Count >= 5) throw new InvalidOperationException("Too many classes"); - tr.Set(ClassKey(c), Slice.FromAscii((seatsLeft - 1).ToString())); + tr.Set(ClassKey(c), Slice.FromStringAscii((seatsLeft - 1).ToString())); tr.Set(rec, Slice.Empty); } @@ -115,8 +115,8 @@ public async Task Drop(IFdbTransaction tr, string s, string c) return; } - var students = Int32.Parse((await tr.GetAsync(ClassKey(c))).ToAscii()); - tr.Set(ClassKey(c), Slice.FromAscii((students + 1).ToString())); + var students = Int32.Parse((await tr.GetAsync(ClassKey(c))).ToStringAscii()); + tr.Set(ClassKey(c), Slice.FromStringAscii((students + 1).ToString())); tr.Clear(rec); } diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index 2af7b729c..c9292d81c 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Tests.Sandbox { using System; @@ -37,9 +36,9 @@ namespace FoundationDB.Tests.Sandbox using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Linq; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; class Program { @@ -180,7 +179,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.Create(STuple.EncodeKey(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.Create(TuPack.EncodeKey(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); @@ -266,8 +265,8 @@ private static async Task HelloWorld(CancellationToken ct) // Writes some data in to the database using (var tr = db.BeginTransaction(ct)) { - tr.Set(STuple.EncodeKey("Test", 123), Slice.FromString("Hello World!")); - tr.Set(STuple.EncodeKey("Test", 456), Slice.FromInt64(DateTime.UtcNow.Ticks)); + tr.Set(TuPack.EncodeKey("Test", 123), Slice.FromString("Hello World!")); + tr.Set(TuPack.EncodeKey("Test", 456), Slice.FromInt64(DateTime.UtcNow.Ticks)); } } @@ -301,7 +300,7 @@ private static async Task TestSimpleTransactionAsync(IFdbDatabase db, Cancellati Console.WriteLine("Setting 'TopSecret' = rnd(512)"); var data = new byte[512]; new Random(1234).NextBytes(data); - trans.Set(location.Keys.Encode("TopSecret"), Slice.Create(data)); + trans.Set(location.Keys.Encode("TopSecret"), data.AsSlice()); Console.WriteLine("Committing transaction..."); await trans.CommitAsync(); @@ -335,7 +334,7 @@ private static async Task BenchInsertSmallKeysAsync(IFdbDatabase db, int N, int tmp[1] = (byte)(i >> 8); // (Batch, 1) = [......] // (Batch, 2) = [......] - trans.Set(subspace.Keys.Encode(k * N + i), Slice.Create(tmp)); + trans.Set(subspace.Keys.Encode(k * N + i), tmp.AsSlice()); } await trans.CommitAsync(); } @@ -395,7 +394,7 @@ private static async Task BenchConcurrentInsert(IFdbDatabase db, int k, int N, i tmp[1] = (byte)(i >> 8); // ("Batch", batch_index, i) = [..random..] - trans.Set(subspace.Keys.Encode(i), Slice.Create(tmp)); + trans.Set(subspace.Keys.Encode(i), tmp.AsSlice()); } x.Stop(); Console.WriteLine("> [" + offset + "] packaged " + n + " keys (" + trans.Size.ToString("N0", CultureInfo.InvariantCulture) + " bytes) in " + FormatTimeMilli(x.Elapsed.TotalMilliseconds)); @@ -517,7 +516,7 @@ private static async Task BenchClearAsync(IFdbDatabase db, int N, CancellationTo { // clear a lot of small keys, in a single transaction - var location = db.Partition.ByKey(Slice.FromAscii("hello")); + var location = db.Partition.ByKey(Slice.FromStringAscii("hello")); var sw = Stopwatch.StartNew(); using (var trans = db.BeginTransaction(ct)) @@ -547,7 +546,7 @@ private static async Task BenchUpdateSameKeyLotsOfTimesAsync(IFdbDatabase db, in list[i] = (byte)i; using (var trans = db.BeginTransaction(ct)) { - trans.Set(key, Slice.Create(list)); + trans.Set(key, list.AsSlice()); await trans.CommitAsync(); } if (i % 100 == 0) Console.Write("\r> " + i + " / " + N); @@ -576,7 +575,7 @@ private static async Task BenchUpdateLotsOfKeysAsync(IFdbDatabase db, int N, Can { for (int k = i; k < i + 1000 && k < N; k++) { - trans.Set(keys[k], Slice.Create(segment)); + trans.Set(keys[k], segment.AsSlice()); } await trans.CommitAsync(); Console.Write("\r" + i + " / " + N); @@ -597,7 +596,7 @@ private static async Task BenchUpdateLotsOfKeysAsync(IFdbDatabase db, int N, Can { var list = data[i].Value.GetBytes(); list[(list.Length >> 1) + 1] = (byte) rnd.Next(256); - trans.Set(data[i].Key, Slice.Create(list)); + trans.Set(data[i].Key, list.AsSlice()); } Console.WriteLine("COMMIT"); @@ -646,7 +645,7 @@ private static async Task BenchBulkInsertThenBulkReadAsync(IFdbDatabase db, int int z = 0; foreach (int i in Enumerable.Range(chunk.Key, chunk.Value)) { - tr.Set(subspace.Keys.Encode(i), Slice.Create(new byte[256])); + tr.Set(subspace.Keys.Encode(i), Slice.Create(256)); z++; } diff --git a/FoundationDB.Tests/Async/AsyncBufferFacts.cs b/FoundationDB.Tests/Async/AsyncBufferFacts.cs index d69f114f8..9905af813 100644 --- a/FoundationDB.Tests/Async/AsyncBufferFacts.cs +++ b/FoundationDB.Tests/Async/AsyncBufferFacts.cs @@ -49,67 +49,79 @@ public async Task Test_AsyncTaskBuffer_In_Arrival_Order() // Test that we can queue N async tasks in a buffer that will only accept K tasks at a time, // and pump them into a list that will received in arrival order + const int ITER = 10; const int N = 20; const int K = 5; - // since this can lock up, we need a global timeout ! - using (var go = new CancellationTokenSource(TimeSpan.FromSeconds(10))) + for (int r = 0; r < ITER; r++) { - var token = go.Token; - token.Register(() => Console.WriteLine("### TIMEOUT EXPIRED!")); - - var list = new List(); - bool didComplete = false; - Exception error = null; - var target = AsyncHelpers.CreateTarget( - (x, ct) => - { - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + "] received " + x); - list.Add(x); - }, - () => - { - didComplete = true; - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + "] completed"); - }, - (e) => + // since this can lock up, we need a global timeout ! + using (var go = new CancellationTokenSource(TimeSpan.FromSeconds(10))) + { + var token = go.Token; + token.Register(() => Log("### TIMEOUT EXPIRED!")); + + var list = new List(); + bool didComplete = false; + Exception error = null; + var target = AsyncHelpers.CreateTarget( + (x, ct) => + { + Log("[target#" + Thread.CurrentThread.ManagedThreadId + "] received " + x); + list.Add(x); + }, + () => + { + didComplete = true; + Log("[target#" + Thread.CurrentThread.ManagedThreadId + "] completed"); + }, + (e) => + { + error = e.SourceException; + Log("[target#" + Thread.CurrentThread.ManagedThreadId + "] error " + e.SourceException.ToString()); + } + ); + + var buffer = AsyncHelpers.CreateOrderPreservingAsyncBuffer(K); + + Log("### Starting pumping"); + var pumpTask = buffer.PumpToAsync(target, token); + + var rnd = new Random(0x1337); + for (int i = 0; i < N; i++) { - error = e.SourceException; - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + "] error " + e.SourceException.ToString()); + int x = i; + var task = Task.Run( + async () => + { + await Task.Delay(10 + rnd.Next(50), token); + Log("[source#" + Thread.CurrentThread.ManagedThreadId + "] produced " + x); + return x; + }, + token); + + await buffer.OnNextAsync(task, token); } - ); + // signal the end + buffer.OnCompleted(); - var buffer = AsyncHelpers.CreateOrderPreservingAsyncBuffer(K); + Log("### Finished producing"); - Console.WriteLine("starting pumping"); - var pumpTask = buffer.PumpToAsync(target, token); - - var rnd = new Random(0x1337); - for (int i = 0; i < N; i++) - { - int x = i; - var task = Task.Run(async () => + await Task.WhenAny(pumpTask, Task.Delay(15 * 1000, go.Token)); + if (!pumpTask.IsCompleted) { - await Task.Delay(10 + rnd.Next(50), token); - Console.WriteLine("[source#" + Thread.CurrentThread.ManagedThreadId + "] produced " + x); - return x; - }, token); - - await buffer.OnNextAsync(task, token); - } - // signal the end - buffer.OnCompleted(); - - Console.WriteLine("finished producing"); - - await pumpTask; + Log("FAILED: HARD TIMEOUT! PumpTask did not complete in time :("); + Assert.Fail("The PumpTask did not complete in time"); + } + Assert.That(async () => await pumpTask, Throws.Nothing, "PumpTask failed"); - Console.WriteLine("finished pumping"); + Log("### Finished pumping"); - Console.WriteLine("Result: " + String.Join(", ", list)); - Assert.That(didComplete, Is.True); - Assert.That(error, Is.Null); - Assert.That(list, Is.EqualTo(Enumerable.Range(0, N).ToArray())); + Log("Result: " + String.Join(", ", list)); + Assert.That(didComplete, Is.True); + Assert.That(error, Is.Null); + Assert.That(list, Is.EqualTo(Enumerable.Range(0, N).ToArray())); + } } } @@ -118,80 +130,112 @@ public async Task Test_AsyncTaskBuffer_In_Arrival_Order() public async Task Test_AsyncTaskBuffer_In_Completion_Order() { // Test that we can queue N async tasks in a buffer that will only accept K tasks at a time, - // and pump them into a list that will received in completion order + // and pump them into a list that will be received in completion order + const int ITER = 10; const int N = 20; const int K = 5; - - // since this can lock up, we need a global timeout ! - using (var go = new CancellationTokenSource(TimeSpan.FromSeconds(10))) + for (int r = 0; r < ITER; r++) { - var token = go.Token; - token.Register(() => Console.WriteLine("### TIMEOUT EXPIRED!")); + Log(); + Log($"######### RUN {r}"); + Log(); - var list = new List(); - bool didComplete = false; - ExceptionDispatchInfo error = null; - var clock = Stopwatch.StartNew(); + int completeCount = 0; + string[] stacks = new string[16]; - var target = AsyncHelpers.CreateTarget( - (x, ct) => - { - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + " @ " + clock.ElapsedTicks + "] received " + x); - list.Add(x); - }, - () => - { - didComplete = true; - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + " @ " + clock.ElapsedTicks + "] completed"); - }, - (e) => + // since this can lock up, we need a global timeout ! + using (var go = new CancellationTokenSource(TimeSpan.FromSeconds(10))) + { + var token = go.Token; + token.Register(() => Log("### TIMEOUT EXPIRED!")); + + var list = new List(); + bool didComplete = false; + ExceptionDispatchInfo error = null; + var clock = Stopwatch.StartNew(); + + var target = AsyncHelpers.CreateTarget( + onNext: (x, ct) => + { + Log($"[target#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] onNext {x}"); + list.Add(x); + }, + onCompleted: () => + { + int n = Interlocked.Increment(ref completeCount); + stacks[n - 1] = Environment.StackTrace; + if (n > 1) + { + Log("*** OnComplete() CALLED MULTIPLE TIMES!! :("); + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); + } + didComplete = true; + Log($"[target#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] onCompleted"); + }, + onError: (e) => + { + error = e; + Log($"[target#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] onError {e.SourceException}"); + } + ); + + var buffer = AsyncHelpers.CreateUnorderedAsyncBuffer(K); + + Log("### Starting pumping"); + var pumpTask = buffer.PumpToAsync(target, token); + + var rnd = new Random(0x1337); + for (int i = 0; i < N; i++) { - error = e; - Console.WriteLine("[target#" + Thread.CurrentThread.ManagedThreadId + " @ " + clock.ElapsedTicks + "] error " + e.SourceException.ToString()); + int x = i; + var task = Task.Run( + async () => + { + Log($"[source#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] thinking {x} on task {Task.CurrentId}"); + // simulate random workload + await Task.Delay(10 + rnd.Next(50), token); + Log($"[source#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] produced {x} on task {Task.CurrentId}"); + return x; + }, + token); + + //Log($"[parent#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] calling OnNextAsync({task.Id}) for {x}..."); + var t = buffer.OnNextAsync(task, token); + //Log($"[parent#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] called OnNextAsync({task.Id}) for {x} : {t.Status}"); + await t; + //Log($"[parent#{Thread.CurrentThread.ManagedThreadId,-2} @ {clock.Elapsed.TotalMilliseconds:N3}] awaited OnNextAsync({task.Id}) for {x}"); } - ); - - var buffer = AsyncHelpers.CreateUnorderedAsyncBuffer(K); + // signal the end + buffer.OnCompleted(); - Console.WriteLine("starting pumping"); - var pumpTask = buffer.PumpToAsync(target, token); + Log("### Finished producing!"); - var rnd = new Random(0x1337); - for (int i = 0; i < N; i++) - { - int x = i; - var task = Task.Run(async () => + await Task.WhenAny(pumpTask, Task.Delay(15 * 1000, go.Token)); + if (!pumpTask.IsCompleted) { - Console.WriteLine("[source#" + Thread.CurrentThread.ManagedThreadId + " @ " + clock.ElapsedTicks + "] thinking " + x); - await Task.Delay(10 + rnd.Next(50), token); - Console.WriteLine("[source#" + Thread.CurrentThread.ManagedThreadId + " @ " + clock.ElapsedTicks + "] produced " + x); - return x; - }, token); - - await buffer.OnNextAsync(task, token); - } - // signal the end - buffer.OnCompleted(); - - Console.WriteLine("finished producing"); - - await pumpTask; + Log("FAILED: HARD TIMEOUT! PumpTask did not complete in time :("); + Log($"DidComplete: {didComplete}"); + Log($"Error: {error?.SourceException}"); + Assert.Fail("The PumpTask did not complete in time"); + } + Assert.That(async () => await pumpTask, Throws.Nothing, "PumpTask failed"); - Console.WriteLine("finished pumping"); + Log("### Finished pumping"); - Console.WriteLine("Result: " + String.Join(", ", list)); - Assert.That(didComplete, Is.True); - Assert.That(error, Is.Null); - //note: order doesn't matter, but all should be there - Assert.That(list, Is.EquivalentTo(Enumerable.Range(0, N).ToArray())); + Log($"Result: {String.Join(", ", list)}"); + Assert.That(didComplete, Is.True); + Assert.That(error, Is.Null); + //note: order doesn't matter, but all should be there + Assert.That(list, Is.EquivalentTo(Enumerable.Range(0, N).ToArray())); + } } } [Test] - public async Task Test_FdbAsyncTransform() + public async Task Test_AsyncTransform() { // async transform start concurrent tasks for all source items @@ -211,35 +255,41 @@ public async Task Test_FdbAsyncTransform() async (x, _) => { // each element takes a random time to compute - await Task.Delay(5 + rnd1.Next(25)); + await Task.Delay(5 + rnd1.Next(25), this.Cancellation); return Math.Sqrt(x); }, queue, TaskScheduler.Default ); - var pumpTask = AsyncHelpers.PumpToListAsync(queue, token); - - var rnd2 = new Random(5678); + var pumpTask = queue.PumpToListAsync(token); for (int i = 0; i < N; i++) { // emulate a batched source - if (i % 10 == 0) await Task.Delay(100); + if (i % 10 == 0) await Task.Delay(100, this.Cancellation); await transform.OnNextAsync(i, token); } transform.OnCompleted(); + + await Task.WhenAny(pumpTask, Task.Delay(10 * 1000, go.Token)); + if (!pumpTask.IsCompleted) + { + Log("FAILED: HARD TIMEOUT! PumpTask did not complete in time :("); + Assert.Fail("The PumpTask did not complete in time"); + } + var list = await pumpTask; - Console.WriteLine("results: " + String.Join(", ", list)); + Log($"results: {String.Join(", ", list)}"); Assert.That(list, Is.EqualTo(Enumerable.Range(0, N).Select(x => Math.Sqrt(x)).ToArray())); } } [Test] - public async Task Test_FdbAsyncPump_Stops_On_First_Error() + public async Task Test_AsyncPump_Stops_On_First_Error() { const int MAX_CAPACITY = 5; @@ -249,25 +299,29 @@ public async Task Test_FdbAsyncPump_Stops_On_First_Error() { var token = go.Token; - var rnd1 = new Random(1234); - var queue = AsyncHelpers.CreateOrderPreservingAsyncBuffer(MAX_CAPACITY); - var pumpTask = AsyncHelpers.PumpToListAsync(queue, token); - - var rnd2 = new Random(5678); + var pumpTask = queue.PumpToListAsync(token); #pragma warning disable 162 await queue.OnNextAsync(Task.FromResult(0), token); await queue.OnNextAsync(Task.FromResult(1), token); - await queue.OnNextAsync(Task.Run(() => { throw new InvalidOperationException("Oops"); return 123; }), token); + await queue.OnNextAsync(Task.Run(() => { throw new InvalidOperationException("Oops"); return 123; }, this.Cancellation), token); await queue.OnNextAsync(Task.FromResult(3), token); - await queue.OnNextAsync(Task.Run(() => { throw new InvalidOperationException("Epic Fail"); return 456; }), token); + await queue.OnNextAsync(Task.Run(() => { throw new InvalidOperationException("Epic Fail"); return 456; }, this.Cancellation), token); queue.OnCompleted(); #pragma warning restore 162 - var x = Assert.Throws(async () => await pumpTask, "Pump should throw the last exception encountered"); - Assert.That(x.Message, Is.EqualTo("Oops")); + await Task.WhenAny(pumpTask, Task.Delay(10 * 1000, go.Token)); + if (!pumpTask.IsCompleted) + { + Log("FAILED: HARD TIMEOUT! PumpTask did not complete in time :("); + Assert.Fail("The PumpTask did not complete in time"); + } + + Assert.That(async () => await pumpTask, Throws.InvalidOperationException.With.Message.EqualTo("Oops"), "Pump should rethrow the first exception encountered"); + //REVIEW: should we instead use AggregateException if multiple errors are pushed? + // => AggregateException is a pain to use :( } diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index e7cc6156a..b1534e192 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -28,20 +28,19 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Client; - using FoundationDB.Filters.Logging; - using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Diagnostics; - using System.Globalization; using System.IO; using System.Linq; using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + using FoundationDB.Filters.Logging; + using FoundationDB.Layers.Directories; + using NUnit.Framework; [TestFixture] public class DatabaseBulkFacts : FdbTest diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index 745b09ebc..752edb950 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -28,14 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Status; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.IO; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + using NUnit.Framework; [TestFixture] public class DatabaseFacts : FdbTest @@ -205,7 +204,15 @@ public async Task Test_Can_Get_Coordinators() { var coordinators = await Fdb.System.GetCoordinatorsAsync(db, this.Cancellation); Assert.That(coordinators, Is.Not.Null); - Assert.That(coordinators.Description, Is.EqualTo("local")); + Log("raw : " + coordinators.RawValue); + Log("id : "+ coordinators.Id); + Log("desc:" + coordinators.Description); + Log("coordinators:"); + foreach (var x in coordinators.Coordinators) + { + Log($"- {x.Address}:{x.Port}{(x.Tls ? " (TLS)" : "")}"); + } + Assert.That(coordinators.Description, Is.Not.Null.Or.Empty); //note: it should be a long numerical string, but it changes for each installation Assert.That(coordinators.Id, Is.Not.Null.And.Length.GreaterThan(0)); Assert.That(coordinators.Coordinators, Is.Not.Null.And.Length.GreaterThan(0)); @@ -231,16 +238,16 @@ public async Task Test_Can_Get_Storage_Engine() Slice actual; using (var tr = db.BeginReadOnlyTransaction(this.Cancellation).WithReadAccessToSystemKeys()) { - actual = await tr.GetAsync(Slice.FromAscii("\xFF/conf/storage_engine")); + actual = await tr.GetAsync(Slice.FromByteString("\xFF/conf/storage_engine")); } if (mode == "ssd") { // ssd = '0' - Assert.That(actual, Is.EqualTo(Slice.FromAscii("0"))); + Assert.That(actual, Is.EqualTo(Slice.FromStringAscii("0"))); } else { // memory = '1' - Assert.That(actual, Is.EqualTo(Slice.FromAscii("1"))); + Assert.That(actual, Is.EqualTo(Slice.FromStringAscii("1"))); } } } @@ -270,7 +277,7 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.Create(STuple.EncodeKey("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.Create(TuPack.EncodeKey("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); @@ -280,14 +287,14 @@ public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>test<00><02>hello<00>")); // keys inside the global space are valid - Assert.That(db.IsKeyValid(STuple.EncodeKey("test", 123)), Is.True); + Assert.That(db.IsKeyValid(TuPack.EncodeKey("test", 123)), Is.True); // keys outside the global space are invalid - Assert.That(db.IsKeyValid(Slice.Create(new byte[] { 42 })), Is.False); + Assert.That(db.IsKeyValid(Slice.FromByte(42)), Is.False); } // using a random binary prefix - using (var db = await Fdb.OpenAsync(null, "DB", new KeySubspace(Slice.Create(new byte[] { 42, 255, 0, 90 })), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", new KeySubspace(new byte[] { 42, 255, 0, 90 }.AsSlice()), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); @@ -300,7 +307,7 @@ public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() Assert.That(db.IsKeyValid(Slice.Unescape("*<00>Z123")), Is.True); // keys outside the global space are invalid - Assert.That(db.IsKeyValid(Slice.Create(new byte[] { 123 })), Is.False); + Assert.That(db.IsKeyValid(Slice.FromByte(123)), Is.False); Assert.That(db.IsKeyValid(Slice.Unescape("*")), Is.False); } diff --git a/FoundationDB.Tests/Encoders/EncoderFacts.cs b/FoundationDB.Tests/Encoders/EncoderFacts.cs index ffc7d748b..73ba2c1c8 100644 --- a/FoundationDB.Tests/Encoders/EncoderFacts.cs +++ b/FoundationDB.Tests/Encoders/EncoderFacts.cs @@ -28,18 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Converters.Tests { + using System; + using Doxense.Collections.Tuples; using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; + using FoundationDB.Client.Tests; using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Globalization; - using System.Linq; - using System.Text; [TestFixture] - public class EncoderFacts + public class EncoderFacts : FdbTest { [Test] @@ -89,7 +85,7 @@ public void Test_Ordered_BinaryEncoder() Assert.That(encoder, Is.Not.Null); Assert.That(encoder.EncodeKey(Slice.FromString("hello world")), Is.EqualTo(Slice.Unescape("<01>hello world<00>"))); - Assert.That(encoder.EncodeKey(Slice.Create(new byte[] { 0, 0xFF, 0 })), Is.EqualTo(Slice.Unescape("<01><00><00><00>"))); + Assert.That(encoder.EncodeKey(new byte[] { 0, 0xFF, 0 }.AsSlice()), Is.EqualTo(Slice.Unescape("<01><00><00><00>"))); Assert.That(encoder.EncodeKey(Slice.Empty), Is.EqualTo(Slice.Unescape("<01><00>"))); Assert.That(encoder.EncodeKey(Slice.Nil), Is.EqualTo(Slice.Unescape("<00>"))); @@ -115,7 +111,7 @@ public void Test_Tuple_Composite_Encoder() // note: EncodeKey(...) is just a shortcurt for packing all items in a tuple, and EncodeComposite(..., count = 3) var data = encoder.EncodeKey(x, y, z); - Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y, z))); + Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x, y, z))); var items = encoder.DecodeKey(data); Assert.That(items.Item1, Is.EqualTo(x)); @@ -125,15 +121,15 @@ public void Test_Tuple_Composite_Encoder() // partial key encoding data = encoder.EncodeComposite(items, 2); - Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y))); - items = encoder.DecodeComposite(STuple.EncodeKey(x, y), 2); + Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x, y))); + items = encoder.DecodeComposite(TuPack.EncodeKey(x, y), 2); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(y)); Assert.That(items.Item3, Is.EqualTo(default(Guid))); data = encoder.EncodeComposite(items, 1); - Assert.That(data, Is.EqualTo(STuple.EncodeKey(x))); - items = encoder.DecodeComposite(STuple.EncodeKey(x), 1); + Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x))); + items = encoder.DecodeComposite(TuPack.EncodeKey(x), 1); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(default(long))); Assert.That(items.Item3, Is.EqualTo(default(Guid))); diff --git a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs index 67f88ca4e..3374858d0 100644 --- a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs +++ b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs @@ -28,18 +28,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Converters.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; - using System.Collections.Generic; - using System.Globalization; - using System.Linq; - using System.Text; + using Doxense.Collections.Tuples; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Memory; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] - public class TypeCodecFacts + public class TypeCodecFacts : FdbTest { [Test] @@ -48,13 +45,13 @@ public void Test_Simple_Integer_Codec() var codec = TupleCodec.Default; Assert.That(codec, Is.Not.Null); - Assert.That(codec.EncodeOrdered(0), Is.EqualTo(STuple.EncodeKey(0))); - Assert.That(codec.EncodeOrdered(123), Is.EqualTo(STuple.EncodeKey(123))); - Assert.That(codec.EncodeOrdered(123456), Is.EqualTo(STuple.EncodeKey(123456))); + Assert.That(codec.EncodeOrdered(0), Is.EqualTo(TuPack.EncodeKey(0))); + Assert.That(codec.EncodeOrdered(123), Is.EqualTo(TuPack.EncodeKey(123))); + Assert.That(codec.EncodeOrdered(123456), Is.EqualTo(TuPack.EncodeKey(123456))); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey(0)), Is.EqualTo(0)); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey(123)), Is.EqualTo(123)); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey(123456)), Is.EqualTo(123456)); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey(0)), Is.EqualTo(0)); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey(123)), Is.EqualTo(123)); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey(123456)), Is.EqualTo(123456)); } [Test] @@ -63,13 +60,13 @@ public void Test_Simple_String_Codec() var codec = TupleCodec.Default; Assert.That(codec, Is.Not.Null); - Assert.That(codec.EncodeOrdered("héllø Wörld"), Is.EqualTo(STuple.EncodeKey("héllø Wörld"))); - Assert.That(codec.EncodeOrdered(String.Empty), Is.EqualTo(STuple.EncodeKey(""))); - Assert.That(codec.EncodeOrdered(null), Is.EqualTo(STuple.EncodeKey(default(string)))); + Assert.That(codec.EncodeOrdered("héllø Wörld"), Is.EqualTo(TuPack.EncodeKey("héllø Wörld"))); + Assert.That(codec.EncodeOrdered(String.Empty), Is.EqualTo(TuPack.EncodeKey(""))); + Assert.That(codec.EncodeOrdered(null), Is.EqualTo(TuPack.EncodeKey(default(string)))); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey("héllø Wörld")), Is.EqualTo("héllø Wörld")); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey(String.Empty)), Is.EqualTo("")); - Assert.That(codec.DecodeOrdered(STuple.EncodeKey(default(string))), Is.Null); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey("héllø Wörld")), Is.EqualTo("héllø Wörld")); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey(String.Empty)), Is.EqualTo("")); + Assert.That(codec.DecodeOrdered(TuPack.EncodeKey(default(string))), Is.Null); } [Test] @@ -85,12 +82,12 @@ public void Test_Simple_SelfTerms_Codecs() var second = TupleCodec.Default; var third = TupleCodec.Default; - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); first.EncodeOrderedSelfTerm(ref writer, x); second.EncodeOrderedSelfTerm(ref writer, y); third.EncodeOrderedSelfTerm(ref writer, z); var data = writer.ToSlice(); - Assert.That(data, Is.EqualTo(STuple.EncodeKey(x, y, z))); + Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x, y, z))); var reader = new SliceReader(data); Assert.That(first.DecodeOrderedSelfTerm(ref reader), Is.EqualTo(x)); diff --git a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs index 033b20c6a..779b8ab3e 100644 --- a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs +++ b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs @@ -28,17 +28,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing.Tests { - using FoundationDB.Client; - using NUnit.Framework; using System; using System.Collections.Generic; - using System.Linq; - using System.Text; - using MathNet.Numerics.Distributions; using System.Diagnostics; using System.Globalization; - using FoundationDB.Layers.Tuples; using System.IO; + using System.Linq; + using System.Text; + using Doxense.Collections.Tuples; + using MathNet.Numerics.Distributions; + using NUnit.Framework; [TestFixture] [Category("LongRunning")] @@ -290,10 +289,10 @@ public void TestFoo() return buf; }; - compress(Slice.Create(mostlyZeroes(1))); - compress(Slice.Create(mostlyZeroes(10))); - compress(Slice.Create(mostlyZeroes(42))); - compress(Slice.Create(mostlyZeroes(100))); + compress(mostlyZeroes(1).AsSlice()); + compress(mostlyZeroes(10).AsSlice()); + compress(mostlyZeroes(42).AsSlice()); + compress(mostlyZeroes(100).AsSlice()); // mostly ones @@ -310,10 +309,10 @@ public void TestFoo() return buf; }; - compress(Slice.Create(mostlyOnes(1))); - compress(Slice.Create(mostlyOnes(10))); - compress(Slice.Create(mostlyOnes(42))); - compress(Slice.Create(mostlyOnes(100))); + compress(mostlyOnes(1).AsSlice()); + compress(mostlyOnes(10).AsSlice()); + compress(mostlyOnes(42).AsSlice()); + compress(mostlyOnes(100).AsSlice()); // progressive Func testBit = (b, p) => (b[p >> 3] & (1 << (p & 7))) != 0; @@ -321,7 +320,7 @@ public void TestFoo() const int VALUES = 8192; var buffer = new byte[VALUES / 8]; var output = new CompressedBitmapWriter(); - WordAlignHybridEncoder.CompressTo(Slice.Create(buffer), output); + WordAlignHybridEncoder.CompressTo(buffer.AsSlice(), output); Console.WriteLine("{0}\t{1}\t1024", 0, output.Length); for (int i = 0; i < VALUES / 8; i++) { @@ -335,7 +334,7 @@ public void TestFoo() setBit(buffer, p); output.Reset(); - WordAlignHybridEncoder.CompressTo(Slice.Create(buffer), output); + WordAlignHybridEncoder.CompressTo(buffer.AsSlice(), output); Console.WriteLine("{0}\t{1}\t1024", 1.0d * (i + 1) / VALUES, output.Length); } @@ -366,14 +365,12 @@ public MemoryIndex(IEqualityComparer comparer = null) public CompressedBitmap Lookup(TKey value) { - CompressedBitmap bmp; - return this.Values.TryGetValue(value, out bmp) ? bmp : null; + return this.Values.TryGetValue(value, out CompressedBitmap bmp) ? bmp : null; } public int Count(TKey value) { - int cnt; - return this.Statistics.TryGetValue(value, out cnt) ? cnt : 0; + return this.Statistics.TryGetValue(value, out int cnt) ? cnt : 0; } public double Frequency(TKey value) @@ -388,9 +385,8 @@ private static Action MakeInserter(MemoryIndex index, Fu { int docId = idFunc(doc); TKey indexedValue = keyFunc(doc); - CompressedBitmap bmp; int count; - if (!index.Values.TryGetValue(indexedValue, out bmp)) + if (!index.Values.TryGetValue(indexedValue, out CompressedBitmap bmp)) { bmp = CompressedBitmap.Empty; count = 0; @@ -410,15 +406,15 @@ private static Action MakeInserter(MemoryIndex index, Fu private static string MakeHeatMap(int[] map) { int max = map.Max(); - string scale = "`.:;+=xX$&#"; - double r = (double)(scale.Length - 1) / max; + const string SCALE = "`.:;+=xX$&#"; + double r = (double)(SCALE.Length - 1) / max; var chars = new char[map.Length]; for (int i = 0; i < map.Length; i++) { if (map[i] == 0) chars[i] = '\xA0'; else - chars[i] = scale[(int)Math.Round(r * map[i], MidpointRounding.AwayFromZero)]; + chars[i] = SCALE[(int)Math.Round(r * map[i], MidpointRounding.AwayFromZero)]; } return new string(chars); } @@ -437,11 +433,9 @@ private static void DumpIndex(string label, MemoryIndex index, foreach (var kv in index.Values.OrderBy((kv) => orderBy(kv.Key, index.Count(kv.Key)), comparer)) { var t = STuple.Create(kv.Key); - var tk = t.ToSlice(); + var tk = TuPack.Pack(t); - int bits, words, literals, fillers; - double ratio; - kv.Value.GetStatistics(out bits, out words, out literals, out fillers, out ratio); + kv.Value.GetStatistics(out int bits, out int words, out int literals, out int _, out double ratio); long legacyIndexSize = 0; // size estimate of a regular FDB index (..., "Value", GUID) = "" Array.Clear(map, 0, map.Length); @@ -487,8 +481,7 @@ private static List DumpIndexQueryResult(Dictionary c var results = new List(); foreach (var docId in bitmap.GetView()) { - Character charac; - Assert.That(characters.TryGetValue(docId, out charac), Is.True); + Assert.That(characters.TryGetValue(docId, out Character charac), Is.True); results.Add(charac); Console.WriteLine("- {0}: {1} {2}{3}", docId, charac.Name, charac.Gender == "Male" ? "\u2642" : charac.Gender == "Female" ? "\u2640" : charac.Gender, charac.Dead ? " (\u271D)" : ""); @@ -773,6 +766,7 @@ public void Test_Randomized_Data() #endregion + [Test] public void Test_BigBadIndexOfTheDead() { // simulate a dataset where 50,000 users create a stream of 10,000,000 events, with a non uniform distribution, ie: few users making the bulk, and a long tail of mostly inactive users @@ -905,12 +899,9 @@ public void Test_BigBadIndexOfTheDead() j = 0; foreach (var kv in controlStats) { - CompressedBitmapBuilder builder; - Assert.That(index.TryGetValue(kv.Value, out builder), Is.True, "{0} is missing from index", kv.Value); + Assert.That(index.TryGetValue(kv.Value, out CompressedBitmapBuilder builder), Is.True, "{0} is missing from index", kv.Value); var bmp = builder.ToBitmap(); - int bits, words, a, b; - double ratio; - bmp.GetStatistics(out bits, out words, out a, out b, out ratio); + bmp.GetStatistics(out int bits, out int words, out int a, out int b, out _); Assert.That(bits, Is.EqualTo(kv.Count), "{0} has invalid count", kv.Value); int sz = bmp.ToSlice().Count; log.WriteLine("{0,8} : {1,5} bits, {2} words ({3} lit. / {4} fil.), {5:N0} bytes, {6:N3} bytes/doc, {7:N2}% compression", kv.Value, bits, words, a, b, sz, 1.0 * sz / bits, 100.0 * (4 + 17 + sz) / (17 + (4 + 17) * bits)); diff --git a/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs b/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs index 41e5e5141..a284db52d 100644 --- a/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs +++ b/FoundationDB.Tests/Experimental/Indexing/SuperSlowUncompressedBitmap.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Experimental.Indexing.Tests { - using NUnit.Framework; using System; using System.Collections.Generic; using System.Diagnostics; using System.Text; + using NUnit.Framework; /// Super Naive and Slow reference implementation of a 'Compressed' Bitmap /// This is basically a bool[] that is used to verify other implementations diff --git a/FoundationDB.Tests/Experimental/JsonNetCodec.cs b/FoundationDB.Tests/Experimental/JsonNetCodec.cs index b5b530473..09e127d0b 100644 --- a/FoundationDB.Tests/Experimental/JsonNetCodec.cs +++ b/FoundationDB.Tests/Experimental/JsonNetCodec.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Types.Json using System.IO; using System.Text; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; using Newtonsoft.Json; @@ -70,7 +71,7 @@ protected virtual Slice EncodeInternal(TDocument document) int size = checked((int)ms.Length); Contract.Assert(tmp != null && size >= 0 && size <= tmp.Length); - return Slice.Create(tmp, 0, size); + return tmp.AsSlice(0, size); } } @@ -97,13 +98,13 @@ void IUnorderedTypeCodec.EncodeUnorderedSelfTerm(ref SliceWriter outp { var packed = EncodeInternal(value); Contract.Assert(packed.Count >= 0); - output.WriteVarint32((uint)packed.Count); + output.WriteVarInt32((uint)packed.Count); output.WriteBytes(packed); } TDocument IUnorderedTypeCodec.DecodeUnorderedSelfTerm(ref SliceReader input) { - uint size = input.ReadVarint32(); + uint size = input.ReadVarInt32(); if (size > int.MaxValue) throw new FormatException("Malformed data size"); var packed = input.ReadBytes((int)size); diff --git a/FoundationDB.Tests/Experimental/ProtobufCodec.cs b/FoundationDB.Tests/Experimental/ProtobufCodec.cs index 8ae2d14fc..b9d05ae55 100644 --- a/FoundationDB.Tests/Experimental/ProtobufCodec.cs +++ b/FoundationDB.Tests/Experimental/ProtobufCodec.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Types.ProtocolBuffers using System; using System.IO; using Doxense.Diagnostics.Contracts; + using Doxense.Memory; using FoundationDB.Client; public class ProtobufCodec : IValueEncoder, IUnorderedTypeCodec @@ -55,7 +56,7 @@ protected virtual Slice EncodeInternal(TDocument document) int size = checked((int)ms.Length); Contract.Assert(tmp != null && size >= 0 && size <= tmp.Length); - return Slice.Create(tmp, 0, size); + return tmp.AsSlice(0, size); } } @@ -78,13 +79,13 @@ void IUnorderedTypeCodec.EncodeUnorderedSelfTerm(ref SliceWriter outp { var packed = EncodeInternal(value); Contract.Assert(packed.Count >= 0); - output.WriteVarint32((uint)packed.Count); + output.WriteVarInt32((uint)packed.Count); output.WriteBytes(packed); } TDocument IUnorderedTypeCodec.DecodeUnorderedSelfTerm(ref SliceReader input) { - uint size = input.ReadVarint32(); + uint size = input.ReadVarInt32(); if (size > int.MaxValue) throw new FormatException("Malformed data size"); var packed = input.ReadBytes((int)size); @@ -102,4 +103,4 @@ TDocument IUnorderedTypeCodec.DecodeUnordered(Slice input) } } -} \ No newline at end of file +} diff --git a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs index 49747bc3f..405687e15 100644 --- a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs +++ b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs @@ -28,13 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Filters.Logging.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using NUnit.Framework; using System; using System.Linq; - using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] public class LoggingFilterFacts : FdbTest diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index de2513e18..a88afe825 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -90,12 +90,11 @@ - + - - - + + @@ -103,7 +102,7 @@ - + @@ -112,14 +111,14 @@ - + - + diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index 1099f30a6..ae5b22a76 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -33,8 +33,8 @@ namespace FoundationDB.Client.Tests using System.Linq; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; using NUnit.Framework; [TestFixture] @@ -66,25 +66,25 @@ public void Test_FdbKey_Constants() public void Test_FdbKey_Increment() { - var key = FdbKey.Increment(Slice.FromAscii("Hello")); - Assert.That(key.ToAscii(), Is.EqualTo("Hellp")); + var key = FdbKey.Increment(Slice.FromByteString("Hello")); + Assert.That(key.ToStringAscii(), Is.EqualTo("Hellp")); - key = FdbKey.Increment(Slice.FromAscii("Hello\x00")); - Assert.That(key.ToAscii(), Is.EqualTo("Hello\x01")); + key = FdbKey.Increment(Slice.FromByteString("Hello\x00")); + Assert.That(key.ToStringAscii(), Is.EqualTo("Hello\x01")); - key = FdbKey.Increment(Slice.FromAscii("Hello\xFE")); - Assert.That(key.ToAscii(), Is.EqualTo("Hello\xFF")); + key = FdbKey.Increment(Slice.FromByteString("Hello\xFE")); + Assert.That(key.ToStringAscii(), Is.EqualTo("Hello\xFF")); - key = FdbKey.Increment(Slice.FromAscii("Hello\xFF")); - Assert.That(key.ToAscii(), Is.EqualTo("Hellp"), "Should remove training \\xFF"); + key = FdbKey.Increment(Slice.FromByteString("Hello\xFF")); + Assert.That(key.ToStringAscii(), Is.EqualTo("Hellp"), "Should remove training \\xFF"); - key = FdbKey.Increment(Slice.FromAscii("A\xFF\xFF\xFF")); - Assert.That(key.ToAscii(), Is.EqualTo("B"), "Should truncate all trailing \\xFFs"); + key = FdbKey.Increment(Slice.FromByteString("A\xFF\xFF\xFF")); + Assert.That(key.ToStringAscii(), Is.EqualTo("B"), "Should truncate all trailing \\xFFs"); // corner cases Assert.That(() => FdbKey.Increment(Slice.Nil), Throws.InstanceOf().With.Property("ParamName").EqualTo("slice")); Assert.That(() => FdbKey.Increment(Slice.Empty), Throws.InstanceOf()); - Assert.That(() => FdbKey.Increment(Slice.FromAscii("\xFF")), Throws.InstanceOf()); + Assert.That(() => FdbKey.Increment(Slice.FromByteString("\xFF")), Throws.InstanceOf()); } @@ -206,26 +206,26 @@ public void Test_KeyRange_Contains() // ["", "") range = KeyRange.Empty; Assert.That(range.Contains(Slice.Empty), Is.False); - Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.False); - Assert.That(range.Contains(Slice.FromAscii("hello")), Is.False); - Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); + Assert.That(range.Contains(Slice.FromByteString("\x00")), Is.False); + Assert.That(range.Contains(Slice.FromByteString("hello")), Is.False); + Assert.That(range.Contains(Slice.FromByteString("\xFF")), Is.False); // ["", "\xFF" ) - range = KeyRange.Create(Slice.Empty, Slice.FromAscii("\xFF")); + range = KeyRange.Create(Slice.Empty, Slice.FromByteString("\xFF")); Assert.That(range.Contains(Slice.Empty), Is.True); - Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.True); - Assert.That(range.Contains(Slice.FromAscii("hello")), Is.True); - Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); + Assert.That(range.Contains(Slice.FromByteString("\x00")), Is.True); + Assert.That(range.Contains(Slice.FromByteString("hello")), Is.True); + Assert.That(range.Contains(Slice.FromByteString("\xFF")), Is.False); // ["\x00", "\xFF" ) - range = KeyRange.Create(Slice.FromAscii("\x00"), Slice.FromAscii("\xFF")); + range = KeyRange.Create(Slice.FromByteString("\x00"), Slice.FromByteString("\xFF")); Assert.That(range.Contains(Slice.Empty), Is.False); - Assert.That(range.Contains(Slice.FromAscii("\x00")), Is.True); - Assert.That(range.Contains(Slice.FromAscii("hello")), Is.True); - Assert.That(range.Contains(Slice.FromAscii("\xFF")), Is.False); + Assert.That(range.Contains(Slice.FromByteString("\x00")), Is.True); + Assert.That(range.Contains(Slice.FromByteString("hello")), Is.True); + Assert.That(range.Contains(Slice.FromByteString("\xFF")), Is.False); // corner cases - Assert.That(KeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("A")).Contains(Slice.FromAscii("A")), Is.False, "Equal bounds"); + Assert.That(KeyRange.Create(Slice.FromByteString("A"), Slice.FromByteString("A")).Contains(Slice.FromByteString("A")), Is.False, "Equal bounds"); } [Test] @@ -236,29 +236,29 @@ public void Test_KeyRange_Test() KeyRange range; // range: [ "A", "Z" ) - range = KeyRange.Create(Slice.FromAscii("A"), Slice.FromAscii("Z")); + range = KeyRange.Create(Slice.FromByteString("A"), Slice.FromByteString("Z")); // Excluding the end: < "Z" - Assert.That(range.Test(Slice.FromAscii("\x00"), endIncluded: false), Is.EqualTo(BEFORE)); - Assert.That(range.Test(Slice.FromAscii("@"), endIncluded: false), Is.EqualTo(BEFORE)); - Assert.That(range.Test(Slice.FromAscii("A"), endIncluded: false), Is.EqualTo(INSIDE)); - Assert.That(range.Test(Slice.FromAscii("Z"), endIncluded: false), Is.EqualTo(AFTER)); - Assert.That(range.Test(Slice.FromAscii("Z\x00"), endIncluded: false), Is.EqualTo(AFTER)); - Assert.That(range.Test(Slice.FromAscii("\xFF"), endIncluded: false), Is.EqualTo(AFTER)); + Assert.That(range.Test(Slice.FromByteString("\x00"), endIncluded: false), Is.EqualTo(BEFORE)); + Assert.That(range.Test(Slice.FromByteString("@"), endIncluded: false), Is.EqualTo(BEFORE)); + Assert.That(range.Test(Slice.FromByteString("A"), endIncluded: false), Is.EqualTo(INSIDE)); + Assert.That(range.Test(Slice.FromByteString("Z"), endIncluded: false), Is.EqualTo(AFTER)); + Assert.That(range.Test(Slice.FromByteString("Z\x00"), endIncluded: false), Is.EqualTo(AFTER)); + Assert.That(range.Test(Slice.FromByteString("\xFF"), endIncluded: false), Is.EqualTo(AFTER)); // Including the end: <= "Z" - Assert.That(range.Test(Slice.FromAscii("\x00"), endIncluded: true), Is.EqualTo(BEFORE)); - Assert.That(range.Test(Slice.FromAscii("@"), endIncluded: true), Is.EqualTo(BEFORE)); - Assert.That(range.Test(Slice.FromAscii("A"), endIncluded: true), Is.EqualTo(INSIDE)); - Assert.That(range.Test(Slice.FromAscii("Z"), endIncluded: true), Is.EqualTo(INSIDE)); - Assert.That(range.Test(Slice.FromAscii("Z\x00"), endIncluded: true), Is.EqualTo(AFTER)); - Assert.That(range.Test(Slice.FromAscii("\xFF"), endIncluded: true), Is.EqualTo(AFTER)); - - range = KeyRange.Create(STuple.EncodeKey("A"), STuple.EncodeKey("Z")); - Assert.That(range.Test(STuple.EncodeKey("@")), Is.EqualTo((BEFORE))); - Assert.That(range.Test(STuple.EncodeKey("A")), Is.EqualTo((INSIDE))); - Assert.That(range.Test(STuple.EncodeKey("Z")), Is.EqualTo((AFTER))); - Assert.That(range.Test(STuple.EncodeKey("Z"), endIncluded: true), Is.EqualTo(INSIDE)); + Assert.That(range.Test(Slice.FromByteString("\x00"), endIncluded: true), Is.EqualTo(BEFORE)); + Assert.That(range.Test(Slice.FromByteString("@"), endIncluded: true), Is.EqualTo(BEFORE)); + Assert.That(range.Test(Slice.FromByteString("A"), endIncluded: true), Is.EqualTo(INSIDE)); + Assert.That(range.Test(Slice.FromByteString("Z"), endIncluded: true), Is.EqualTo(INSIDE)); + Assert.That(range.Test(Slice.FromByteString("Z\x00"), endIncluded: true), Is.EqualTo(AFTER)); + Assert.That(range.Test(Slice.FromByteString("\xFF"), endIncluded: true), Is.EqualTo(AFTER)); + + range = KeyRange.Create(TuPack.EncodeKey("A"), TuPack.EncodeKey("Z")); + Assert.That(range.Test(TuPack.EncodeKey("@")), Is.EqualTo((BEFORE))); + Assert.That(range.Test(TuPack.EncodeKey("A")), Is.EqualTo((INSIDE))); + Assert.That(range.Test(TuPack.EncodeKey("Z")), Is.EqualTo((AFTER))); + Assert.That(range.Test(TuPack.EncodeKey("Z"), endIncluded: true), Is.EqualTo(INSIDE)); } [Test] @@ -267,15 +267,15 @@ public void Test_KeyRange_StartsWith() KeyRange range; // "abc" => [ "abc", "abd" ) - range = KeyRange.StartsWith(Slice.FromAscii("abc")); - Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc"))); - Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abd"))); + range = KeyRange.StartsWith(Slice.FromByteString("abc")); + Assert.That(range.Begin, Is.EqualTo(Slice.FromByteString("abc"))); + Assert.That(range.End, Is.EqualTo(Slice.FromByteString("abd"))); // "" => ArgumentException Assert.That(() => KeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); // "\xFF" => ArgumentException - Assert.That(() => KeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.FromByteString("\xFF")), Throws.InstanceOf()); // null => ArgumentException Assert.That(() => KeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); @@ -287,15 +287,15 @@ public void Test_KeyRange_PrefixedBy() KeyRange range; // "abc" => [ "abc\x00", "abd" ) - range = KeyRange.PrefixedBy(Slice.FromAscii("abc")); - Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc\x00"))); - Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abd"))); + range = KeyRange.PrefixedBy(Slice.FromByteString("abc")); + Assert.That(range.Begin, Is.EqualTo(Slice.FromByteString("abc\x00"))); + Assert.That(range.End, Is.EqualTo(Slice.FromByteString("abd"))); // "" => ArgumentException Assert.That(() => KeyRange.PrefixedBy(Slice.Empty), Throws.InstanceOf()); // "\xFF" => ArgumentException - Assert.That(() => KeyRange.PrefixedBy(Slice.FromAscii("\xFF")), Throws.InstanceOf()); + Assert.That(() => KeyRange.PrefixedBy(Slice.FromByteString("\xFF")), Throws.InstanceOf()); // null => ArgumentException Assert.That(() => KeyRange.PrefixedBy(Slice.Nil), Throws.InstanceOf()); @@ -309,17 +309,17 @@ public void Test_KeyRange_FromKey() // "" => [ "", "\x00" ) range = KeyRange.FromKey(Slice.Empty); Assert.That(range.Begin, Is.EqualTo(Slice.Empty)); - Assert.That(range.End, Is.EqualTo(Slice.FromAscii("\x00"))); + Assert.That(range.End, Is.EqualTo(Slice.FromByteString("\x00"))); // "abc" => [ "abc", "abc\x00" ) - range = KeyRange.FromKey(Slice.FromAscii("abc")); - Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("abc"))); - Assert.That(range.End, Is.EqualTo(Slice.FromAscii("abc\x00"))); + range = KeyRange.FromKey(Slice.FromByteString("abc")); + Assert.That(range.Begin, Is.EqualTo(Slice.FromByteString("abc"))); + Assert.That(range.End, Is.EqualTo(Slice.FromByteString("abc\x00"))); // "\xFF" => [ "\xFF", "\xFF\x00" ) - range = KeyRange.FromKey(Slice.FromAscii("\xFF")); - Assert.That(range.Begin, Is.EqualTo(Slice.FromAscii("\xFF"))); - Assert.That(range.End, Is.EqualTo(Slice.FromAscii("\xFF\x00"))); + range = KeyRange.FromKey(Slice.FromByteString("\xFF")); + Assert.That(range.Begin, Is.EqualTo(Slice.FromByteString("\xFF"))); + Assert.That(range.End, Is.EqualTo(Slice.FromByteString("\xFF\x00"))); Assert.That(() => KeyRange.FromKey(Slice.Nil), Throws.InstanceOf()); } @@ -335,44 +335,44 @@ public void Test_FdbKey_PrettyPrint() Assert.That(FdbKey.Dump(Slice.FromByte(0)), Is.EqualTo("<00>")); Assert.That(FdbKey.Dump(Slice.FromByte(255)), Is.EqualTo("")); - Assert.That(FdbKey.Dump(Slice.Create(new byte[] { 0, 1, 2, 3, 4, 5, 6, 7 })), Is.EqualTo("<00><01><02><03><04><05><06><07>")); - Assert.That(FdbKey.Dump(Slice.Create(new byte[] { 255, 254, 253, 252, 251, 250, 249, 248 })), Is.EqualTo("")); + Assert.That(FdbKey.Dump(new byte[] { 0, 1, 2, 3, 4, 5, 6, 7 }.AsSlice()), Is.EqualTo("<00><01><02><03><04><05><06><07>")); + Assert.That(FdbKey.Dump(new byte[] { 255, 254, 253, 252, 251, 250, 249, 248 }.AsSlice()), Is.EqualTo("")); Assert.That(FdbKey.Dump(Slice.FromString("hello")), Is.EqualTo("hello")); Assert.That(FdbKey.Dump(Slice.FromString("héllø")), Is.EqualTo("hll")); // tuples should be decoded properly - Assert.That(FdbKey.Dump(STuple.EncodeKey(123)), Is.EqualTo("(123,)"), "Singleton tuples should end with a ','"); - Assert.That(FdbKey.Dump(STuple.EncodeKey(Slice.FromAscii("hello"))), Is.EqualTo("('hello',)"), "ASCII strings should use single quotes"); - Assert.That(FdbKey.Dump(STuple.EncodeKey("héllø")), Is.EqualTo("(\"héllø\",)"), "Unicode strings should use double quotes"); - Assert.That(FdbKey.Dump(STuple.EncodeKey(Slice.Create(new byte[] { 1, 2, 3 }))), Is.EqualTo("(<01 02 03>,)")); - Assert.That(FdbKey.Dump(STuple.EncodeKey(123, 456)), Is.EqualTo("(123, 456)"), "Elements should be separated with a space, and not end up with ','"); - Assert.That(FdbKey.Dump(STuple.EncodeKey(true, false, default(object))), Is.EqualTo("(1, 0, null)"), "Booleans should be displayed as numbers, and null should be in lowercase"); //note: even though it's tempting to using Python's "Nil", it's not very ".NETty" - Assert.That(FdbKey.Dump(STuple.EncodeKey(1.0d, Math.PI, Math.E)), Is.EqualTo("(1, 3.1415926535897931, 2.7182818284590451)"), "Doubles should used dot and have full precision (17 digits)"); - Assert.That(FdbKey.Dump(STuple.EncodeKey(1.0f, (float)Math.PI, (float)Math.E)), Is.EqualTo("(1, 3.14159274, 2.71828175)"), "Singles should used dot and have full precision (10 digits)"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(123)), Is.EqualTo("(123,)"), "Singleton tuples should end with a ','"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(Slice.FromByteString("hello"))), Is.EqualTo("('hello',)"), "ASCII strings should use single quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey("héllø")), Is.EqualTo("(\"héllø\",)"), "Unicode strings should use double quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(new byte[] { 1, 2, 3 }.AsSlice())), Is.EqualTo("(<01 02 03>,)")); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(123, 456)), Is.EqualTo("(123, 456)"), "Elements should be separated with a space, and not end up with ','"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(true, false, default(object))), Is.EqualTo("(1, 0, null)"), "Booleans should be displayed as numbers, and null should be in lowercase"); //note: even though it's tempting to using Python's "Nil", it's not very ".NETty" + Assert.That(FdbKey.Dump(TuPack.EncodeKey(1.0d, Math.PI, Math.E)), Is.EqualTo("(1, 3.1415926535897931, 2.7182818284590451)"), "Doubles should used dot and have full precision (17 digits)"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(1.0f, (float)Math.PI, (float)Math.E)), Is.EqualTo("(1, 3.14159274, 2.71828175)"), "Singles should used dot and have full precision (10 digits)"); var guid = Guid.NewGuid(); - Assert.That(FdbKey.Dump(STuple.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {...}, and without quotes"); var uuid128 = Uuid128.NewUuid(); - Assert.That(FdbKey.Dump(STuple.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {...}, and without quotes"); var uuid64 = Uuid64.NewUuid(); - Assert.That(FdbKey.Dump(STuple.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {...}, and without quotes"); // ranges should be decoded when possible - var key = STuple.ToRange(STuple.Create("hello")); + var key = TuPack.ToRange(STuple.Create("hello")); // "<02>hello<00><00>" .. "<02>hello<00>" Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",).<00>")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",).")); - key = KeyRange.StartsWith(STuple.EncodeKey("hello")); + key = KeyRange.StartsWith(TuPack.EncodeKey("hello")); // "<02>hello<00>" .. "<02>hello<01>" Assert.That(FdbKey.PrettyPrint(key.Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(\"hello\",)")); Assert.That(FdbKey.PrettyPrint(key.End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(\"hello\",) + 1")); - var t = STuple.EncodeKey(123); + var t = TuPack.EncodeKey(123); Assert.That(FdbKey.PrettyPrint(t, FdbKey.PrettyPrintMode.Single), Is.EqualTo("(123,)")); - Assert.That(FdbKey.PrettyPrint(STuple.ToRange(t).Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(123,).<00>")); - Assert.That(FdbKey.PrettyPrint(STuple.ToRange(t).End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(123,).")); + Assert.That(FdbKey.PrettyPrint(TuPack.ToRange(t).Begin, FdbKey.PrettyPrintMode.Begin), Is.EqualTo("(123,).<00>")); + Assert.That(FdbKey.PrettyPrint(TuPack.ToRange(t).End, FdbKey.PrettyPrintMode.End), Is.EqualTo("(123,).")); } diff --git a/FoundationDB.Tests/Layers/BlobFacts.cs b/FoundationDB.Tests/Layers/BlobFacts.cs index 1451328fd..4f02a5707 100644 --- a/FoundationDB.Tests/Layers/BlobFacts.cs +++ b/FoundationDB.Tests/Layers/BlobFacts.cs @@ -28,12 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Blobs.Tests { + using System; + using System.Threading.Tasks; using FoundationDB.Client; using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; using NUnit.Framework; - using System; - using System.Threading.Tasks; [TestFixture] public class BlobFacts : FdbTest @@ -121,7 +120,7 @@ public async Task Test_FdbBlob_CanAppendLargeChunks() { using (var tr = db.BeginTransaction(this.Cancellation)) { - await blob.AppendAsync(tr, Slice.Create(data)); + await blob.AppendAsync(tr, data.AsSlice()); await tr.CommitAsync(); } } diff --git a/FoundationDB.Tests/Layers/CounterFacts.cs b/FoundationDB.Tests/Layers/CounterFacts.cs index 50c079cc7..26d7c0453 100644 --- a/FoundationDB.Tests/Layers/CounterFacts.cs +++ b/FoundationDB.Tests/Layers/CounterFacts.cs @@ -28,13 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Counters.Tests { - using FoundationDB.Client.Tests; - using NUnit.Framework; using System; using System.Diagnostics; using System.Linq; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] [Obsolete] diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index bf2dc887e..b8315aea4 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -30,15 +30,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Directories { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Filters.Logging; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using FoundationDB.Filters.Logging; + using NUnit.Framework; [TestFixture] public class DirectoryFacts : FdbTest @@ -567,12 +567,12 @@ public async Task Test_Directory_Partitions() var directory = FdbDirectoryLayer.Create(location); Console.WriteLine(directory); - var partition = await directory.CreateAsync(db, "Foo", Slice.FromAscii("partition"), this.Cancellation); + var partition = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); // we can't get the partition key directory (because it's a root directory) so we need to cheat a little bit var partitionKey = KeySubspace.Copy(partition).GetPrefix(); Console.WriteLine(partition); Assert.That(partition, Is.InstanceOf()); - Assert.That(partition.Layer, Is.EqualTo(Slice.FromAscii("partition"))); + Assert.That(partition.Layer, Is.EqualTo(Slice.FromStringAscii("partition"))); Assert.That(partition.FullName, Is.EqualTo("Foo")); Assert.That(partition.Path, Is.EqualTo(new[] { "Foo" }), "Partition's path should be absolute"); Assert.That(partition.DirectoryLayer, Is.Not.SameAs(directory), "Partitions should have their own DL"); @@ -617,7 +617,7 @@ public async Task Test_Directory_Cannot_Move_To_Another_Partition() var directory = FdbDirectoryLayer.Create(location); Console.WriteLine(directory); - var foo = await directory.CreateAsync(db, "Foo", Slice.FromAscii("partition"), this.Cancellation); + var foo = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); Console.WriteLine(foo); // create a 'Bar' under the 'Foo' partition @@ -646,7 +646,7 @@ public async Task Test_Directory_Cannot_Move_To_A_Sub_Partition() var directory = FdbDirectoryLayer.Create(location); Console.WriteLine(directory); - var outer = await directory.CreateAsync(db, "Outer", Slice.FromAscii("partition"), this.Cancellation); + var outer = await directory.CreateAsync(db, "Outer", Slice.FromStringAscii("partition"), this.Cancellation); Console.WriteLine(outer); // create a 'Inner' subpartition under the 'Outer' partition @@ -846,7 +846,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K var directory = FdbDirectoryLayer.Create(location); Console.WriteLine(directory); - var partition = await directory.CreateAsync(db, "Foo", Slice.FromAscii("partition"), this.Cancellation); + var partition = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); //note: if we want a testable key INSIDE the partition, we have to get it from a sub-directory var subdir = await partition.CreateOrOpenAsync(db, "Bar", this.Cancellation); var barKey = subdir.GetPrefix(); @@ -899,7 +899,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => partition.ToRange()); shouldFail(() => partition.ToRange(Slice.FromString("hello"))); - shouldFail(() => partition.ToRange(STuple.EncodeKey("hello"))); + shouldFail(() => partition.ToRange(TuPack.EncodeKey("hello"))); // Tuples @@ -915,13 +915,13 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K shouldFail(() => partition.Keys.EncodeMany((IEnumerable)new object[] { 123, "hello", true })); shouldFail(() => partition.Keys.Unpack(barKey)); - shouldFail(() => partition.Keys.UnpackMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.UnpackMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); shouldFail(() => partition.Keys.Decode(barKey)); shouldFail(() => partition.Keys.DecodeMany(new[] { barKey, barKey })); shouldFail(() => partition.Keys.DecodeLast(barKey)); - shouldFail(() => partition.Keys.DecodeLastMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.DecodeLastMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); shouldFail(() => partition.Keys.DecodeFirst(barKey)); - shouldFail(() => partition.Keys.DecodeFirstMany(new[] { barKey, barKey + STuple.EncodeKey(123) })); + shouldFail(() => partition.Keys.DecodeFirstMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); //FIXME: need to re-enable this code! #if REFACTORING_IN_PROGRESS diff --git a/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs b/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs index 43505b7af..692ea391a 100644 --- a/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs +++ b/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs @@ -28,12 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Documents.Tests { + using System; + using System.Threading.Tasks; using FoundationDB.Client.Tests; using FoundationDB.Types.Json; using FoundationDB.Types.ProtocolBuffers; using NUnit.Framework; - using System; - using System.Threading.Tasks; [TestFixture] public class DocumentCollectionFacts : FdbTest diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index 430843a94..08530dc3f 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -28,14 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Net; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] public class MapFacts : FdbTest @@ -168,11 +168,11 @@ public async Task Test_FdbMap_With_Custom_Key_Encoder() // Encode IPEndPoint as the (IP, Port,) encoded with the Tuple codec // note: there is a much simpler way or creating composite keys, this is just a quick and dirty test! var keyEncoder = KeyValueEncoders.Bind( - (ipe) => ipe == null ? Slice.Empty : STuple.EncodeKey(ipe.Address, ipe.Port), + (ipe) => ipe == null ? Slice.Empty : TuPack.EncodeKey(ipe.Address, ipe.Port), (packed) => { if (packed.IsNullOrEmpty) return default(IPEndPoint); - var t = STuple.Unpack(packed); + var t = TuPack.Unpack(packed); return new IPEndPoint(t.Get(0), t.Get(1)); } ); diff --git a/FoundationDB.Tests/Layers/MultiMapFacts.cs b/FoundationDB.Tests/Layers/MultiMapFacts.cs index 9d81dc565..ec1009945 100644 --- a/FoundationDB.Tests/Layers/MultiMapFacts.cs +++ b/FoundationDB.Tests/Layers/MultiMapFacts.cs @@ -28,14 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections.Tests { + using System; + using System.Threading.Tasks; using FoundationDB.Client; using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Net; - using System.Threading.Tasks; [TestFixture] public class MultiMapFacts : FdbTest diff --git a/FoundationDB.Tests/Layers/RankedSetFacts.cs b/FoundationDB.Tests/Layers/RankedSetFacts.cs index 38d22a8c4..89608f6e5 100644 --- a/FoundationDB.Tests/Layers/RankedSetFacts.cs +++ b/FoundationDB.Tests/Layers/RankedSetFacts.cs @@ -28,17 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; - using System.Collections.Generic; using System.Diagnostics; - using System.Linq; using System.Text; - using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] [Obsolete] @@ -65,7 +62,7 @@ await db.ReadWriteAsync(async (tr) => for (int i = 0; i < 100; i++) { Console.Write("\rInserting " + i); - await db.ReadWriteAsync((tr) => vector.InsertAsync(tr, STuple.EncodeKey(rnd.Next())), this.Cancellation); + await db.ReadWriteAsync((tr) => vector.InsertAsync(tr, TuPack.EncodeKey(rnd.Next())), this.Cancellation); } sw.Stop(); Console.WriteLine("\rDone in {0:N3} sec", sw.Elapsed.TotalSeconds); diff --git a/FoundationDB.Tests/Layers/StringInternFacts.cs b/FoundationDB.Tests/Layers/StringInternFacts.cs index ba9396502..f11442ca7 100644 --- a/FoundationDB.Tests/Layers/StringInternFacts.cs +++ b/FoundationDB.Tests/Layers/StringInternFacts.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Interning.Tests { + using System; + using System.Threading.Tasks; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; - using System; - using System.Threading.Tasks; [TestFixture] public class StringInternFacts : FdbTest diff --git a/FoundationDB.Tests/Layers/TupleFacts.cs b/FoundationDB.Tests/Layers/TupleFacts.cs deleted file mode 100644 index 610fc9004..000000000 --- a/FoundationDB.Tests/Layers/TupleFacts.cs +++ /dev/null @@ -1,3041 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013, Doxense SARL -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 - -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; - using System.Diagnostics; - using System.Linq; - using System.Net; - using System.Text; - - [TestFixture] - public class TupleFacts : FdbTest - { - - #region General Use... - - [Test] - public void Test_STuple_Create() - { - var t1 = STuple.Create("hello world"); - Assert.That(t1.Count, Is.EqualTo(1)); - Assert.That(t1.Item1, Is.EqualTo("hello world")); - Assert.That(t1.Get(0), Is.EqualTo("hello world")); - Assert.That(t1[0], Is.EqualTo("hello world")); - - var t2 = STuple.Create("hello world", 123); - Assert.That(t2.Count, Is.EqualTo(2)); - Assert.That(t2.Item1, Is.EqualTo("hello world")); - Assert.That(t2.Item2, Is.EqualTo(123)); - Assert.That(t2.Get(0), Is.EqualTo("hello world")); - Assert.That(t2.Get(1), Is.EqualTo(123)); - Assert.That(t2[0], Is.EqualTo("hello world")); - Assert.That(t2[1], Is.EqualTo(123)); - - var t3 = STuple.Create("hello world", 123, false); - Assert.That(t3.Count, Is.EqualTo(3)); - Assert.That(t3.Item1, Is.EqualTo("hello world")); - Assert.That(t3.Item2, Is.EqualTo(123)); - Assert.That(t3.Item3, Is.EqualTo(false)); - Assert.That(t3.Get(0), Is.EqualTo("hello world")); - Assert.That(t3.Get(1), Is.EqualTo(123)); - Assert.That(t3.Get(2), Is.EqualTo(false)); - Assert.That(t3[0], Is.EqualTo("hello world")); - Assert.That(t3[1], Is.EqualTo(123)); - Assert.That(t3[2], Is.EqualTo(false)); - - var t4 = STuple.Create("hello world", 123, false, 1234L); - Assert.That(t4.Count, Is.EqualTo(4)); - Assert.That(t4.Item1, Is.EqualTo("hello world")); - Assert.That(t4.Item2, Is.EqualTo(123)); - Assert.That(t4.Item3, Is.EqualTo(false)); - Assert.That(t4.Item4, Is.EqualTo(1234L)); - Assert.That(t4.Get(0), Is.EqualTo("hello world")); - Assert.That(t4.Get(1), Is.EqualTo(123)); - Assert.That(t4.Get(2), Is.EqualTo(false)); - Assert.That(t4.Get(3), Is.EqualTo(1234L)); - Assert.That(t4[0], Is.EqualTo("hello world")); - Assert.That(t4[1], Is.EqualTo(123)); - Assert.That(t4[2], Is.EqualTo(false)); - Assert.That(t4[3], Is.EqualTo(1234L)); - - var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); - Assert.That(t5.Count, Is.EqualTo(5)); - Assert.That(t5.Item1, Is.EqualTo("hello world")); - Assert.That(t5.Item2, Is.EqualTo(123)); - Assert.That(t5.Item3, Is.EqualTo(false)); - Assert.That(t5.Item4, Is.EqualTo(1234L)); - Assert.That(t5.Item5, Is.EqualTo(-1234)); - Assert.That(t5.Get(0), Is.EqualTo("hello world")); - Assert.That(t5.Get(1), Is.EqualTo(123)); - Assert.That(t5.Get(2), Is.EqualTo(false)); - Assert.That(t5.Get(3), Is.EqualTo(1234L)); - Assert.That(t5.Get(4), Is.EqualTo(-1234)); - Assert.That(t5[0], Is.EqualTo("hello world")); - Assert.That(t5[1], Is.EqualTo(123)); - Assert.That(t5[2], Is.EqualTo(false)); - Assert.That(t5[3], Is.EqualTo(1234L)); - Assert.That(t5[4], Is.EqualTo(-1234)); - - var tn = STuple.Create(new object[] { "hello world", 123, false, 1234L, -1234, "six" }); - Assert.That(tn.Count, Is.EqualTo(6)); - Assert.That(tn.Get(0), Is.EqualTo("hello world")); - Assert.That(tn.Get(1), Is.EqualTo(123)); - Assert.That(tn.Get(2), Is.EqualTo(false)); - Assert.That(tn.Get(3), Is.EqualTo(1234)); - Assert.That(tn.Get(4), Is.EqualTo(-1234)); - Assert.That(tn.Get(5), Is.EqualTo("six")); - } - - [Test] - public void Test_STuple_Wrap() - { - // STuple.Wrap(...) does not copy the items of the array - - var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; - - var t = STuple.Wrap(arr); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(4)); - Assert.That(t[0], Is.EqualTo("Hello")); - Assert.That(t[1], Is.EqualTo(123)); - Assert.That(t[2], Is.EqualTo(false)); - Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); - - t = STuple.Wrap(arr, 1, 2); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(2)); - Assert.That(t[0], Is.EqualTo(123)); - Assert.That(t[1], Is.EqualTo(false)); - - // changing the underyling array should change the tuple - // DON'T DO THIS IN ACTUAL CODE!!! - - arr[1] = 456; - arr[2] = true; - Log("t = {0}", t); - - Assert.That(t[0], Is.EqualTo(456)); - Assert.That(t[1], Is.EqualTo(true)); - } - - [Test] - public void Test_STuple_FromObjects() - { - // STuple.FromObjects(...) does a copy of the items of the array - - var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; - - var t = STuple.FromObjects(arr); - Log("t = {0}", t); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(4)); - Assert.That(t[0], Is.EqualTo("Hello")); - Assert.That(t[1], Is.EqualTo(123)); - Assert.That(t[2], Is.EqualTo(false)); - Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); - - t = STuple.FromObjects(arr, 1, 2); - Log("t = {0}", t); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(2)); - Assert.That(t[0], Is.EqualTo(123)); - Assert.That(t[1], Is.EqualTo(false)); - - // changing the underyling array should NOT change the tuple - - arr[1] = 456; - arr[2] = true; - Log("t = {0}", t); - - Assert.That(t[0], Is.EqualTo(123)); - Assert.That(t[1], Is.EqualTo(false)); - } - - [Test] - public void Test_STuple_FromArray() - { - var items = new string[] { "Bonjour", "le", "Monde" }; - - var t = STuple.FromArray(items); - Log("t = {0}", t); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(3)); - Assert.That(t[0], Is.EqualTo("Bonjour")); - Assert.That(t[1], Is.EqualTo("le")); - Assert.That(t[2], Is.EqualTo("Monde")); - - t = STuple.FromArray(items, 1, 2); - Log("t = {0}", t); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(2)); - Assert.That(t[0], Is.EqualTo("le")); - Assert.That(t[1], Is.EqualTo("Monde")); - - // changing the underlying array should NOT change the tuple - items[1] = "ze"; - Log("t = {0}", t); - - Assert.That(t[0], Is.EqualTo("le")); - } - - [Test] - public void Test_STuple_Negative_Indexing() - { - var t1 = STuple.Create("hello world"); - Assert.That(t1.Get(-1), Is.EqualTo("hello world")); - Assert.That(t1[-1], Is.EqualTo("hello world")); - - var t2 = STuple.Create("hello world", 123); - Assert.That(t2.Get(-1), Is.EqualTo(123)); - Assert.That(t2.Get(-2), Is.EqualTo("hello world")); - Assert.That(t2[-1], Is.EqualTo(123)); - Assert.That(t2[-2], Is.EqualTo("hello world")); - - var t3 = STuple.Create("hello world", 123, false); - Assert.That(t3.Get(-1), Is.EqualTo(false)); - Assert.That(t3.Get(-2), Is.EqualTo(123)); - Assert.That(t3.Get(-3), Is.EqualTo("hello world")); - Assert.That(t3[-1], Is.EqualTo(false)); - Assert.That(t3[-2], Is.EqualTo(123)); - Assert.That(t3[-3], Is.EqualTo("hello world")); - - var t4 = STuple.Create("hello world", 123, false, 1234L); - Assert.That(t4.Get(-1), Is.EqualTo(1234L)); - Assert.That(t4.Get(-2), Is.EqualTo(false)); - Assert.That(t4.Get(-3), Is.EqualTo(123)); - Assert.That(t4.Get(-4), Is.EqualTo("hello world")); - Assert.That(t4[-1], Is.EqualTo(1234L)); - Assert.That(t4[-2], Is.EqualTo(false)); - Assert.That(t4[-3], Is.EqualTo(123)); - Assert.That(t4[-4], Is.EqualTo("hello world")); - - var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); - Assert.That(t5.Get(-1), Is.EqualTo(-1234)); - Assert.That(t5.Get(-2), Is.EqualTo(1234L)); - Assert.That(t5.Get(-3), Is.EqualTo(false)); - Assert.That(t5.Get(-4), Is.EqualTo(123)); - Assert.That(t5.Get(-5), Is.EqualTo("hello world")); - Assert.That(t5[-1], Is.EqualTo(-1234)); - Assert.That(t5[-2], Is.EqualTo(1234L)); - Assert.That(t5[-3], Is.EqualTo(false)); - Assert.That(t5[-4], Is.EqualTo(123)); - Assert.That(t5[-5], Is.EqualTo("hello world")); - - var tn = STuple.Create(new object[] { "hello world", 123, false, 1234, -1234, "six" }); - Assert.That(tn.Get(-1), Is.EqualTo("six")); - Assert.That(tn.Get(-2), Is.EqualTo(-1234)); - Assert.That(tn.Get(-3), Is.EqualTo(1234)); - Assert.That(tn.Get(-4), Is.EqualTo(false)); - Assert.That(tn.Get(-5), Is.EqualTo(123)); - Assert.That(tn.Get(-6), Is.EqualTo("hello world")); - Assert.That(tn[-1], Is.EqualTo("six")); - Assert.That(tn[-2], Is.EqualTo(-1234)); - Assert.That(tn[-3], Is.EqualTo(1234)); - Assert.That(tn[-4], Is.EqualTo(false)); - Assert.That(tn[-5], Is.EqualTo(123)); - Assert.That(tn[-6], Is.EqualTo("hello world")); - } - - [Test] - public void Test_STuple_First_And_Last() - { - // tuple.First() should be equivalent to tuple.Get(0) - // tuple.Last() should be equivalent to tuple.Get(-1) - - var t1 = STuple.Create(1); - Assert.That(t1.First(), Is.EqualTo(1)); - Assert.That(t1.First(), Is.EqualTo("1")); - Assert.That(((ITuple)t1).Last(), Is.EqualTo(1)); - Assert.That(((ITuple)t1).Last(), Is.EqualTo("1")); - - var t2 = STuple.Create(1, 2); - Assert.That(t2.First(), Is.EqualTo(1)); - Assert.That(t2.First(), Is.EqualTo("1")); - Assert.That(t2.Last, Is.EqualTo(2)); - Assert.That(((ITuple)t2).Last(), Is.EqualTo(2)); - Assert.That(((ITuple)t2).Last(), Is.EqualTo("2")); - - var t3 = STuple.Create(1, 2, 3); - Assert.That(t3.First(), Is.EqualTo(1)); - Assert.That(t3.First(), Is.EqualTo("1")); - Assert.That(t3.Last, Is.EqualTo(3)); - Assert.That(((ITuple)t3).Last(), Is.EqualTo(3)); - Assert.That(((ITuple)t3).Last(), Is.EqualTo("3")); - - var t4 = STuple.Create(1, 2, 3, 4); - Assert.That(t4.First(), Is.EqualTo(1)); - Assert.That(t4.First(), Is.EqualTo("1")); - Assert.That(t4.Last, Is.EqualTo(4)); - Assert.That(((ITuple)t4).Last(), Is.EqualTo(4)); - Assert.That(((ITuple)t4).Last(), Is.EqualTo("4")); - - var t5 = STuple.Create(1, 2, 3, 4, 5); - Assert.That(t5.First(), Is.EqualTo(1)); - Assert.That(t5.First(), Is.EqualTo("1")); - Assert.That(t5.Last, Is.EqualTo(5)); - Assert.That(((ITuple)t5).Last(), Is.EqualTo(5)); - Assert.That(((ITuple)t5).Last(), Is.EqualTo("5")); - - var tn = STuple.Create(1, 2, 3, 4, 5, 6); - Assert.That(tn.First(), Is.EqualTo(1)); - Assert.That(tn.First(), Is.EqualTo("1")); - Assert.That(tn.Last(), Is.EqualTo(6)); - Assert.That(tn.Last(), Is.EqualTo("6")); - - Assert.That(() => STuple.Empty.First(), Throws.InstanceOf()); - Assert.That(() => STuple.Empty.Last(), Throws.InstanceOf()); - } - - [Test] - public void Test_STuple_Unpack_First_And_Last() - { - // should only work with tuples having at least one element - - Slice packed; - - packed = STuple.EncodeKey(1); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("1")); - - packed = STuple.EncodeKey(1, 2); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(2)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("2")); - - packed = STuple.EncodeKey(1, 2, 3); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(3)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("3")); - - packed = STuple.EncodeKey(1, 2, 3, 4); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(4)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("4")); - - packed = STuple.EncodeKey(1, 2, 3, 4, 5); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(5)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("5")); - - packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(6)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("6")); - - packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(7)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("7")); - - packed = STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeFirst(packed), Is.EqualTo("1")); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo(8)); - Assert.That(STuple.DecodeLast(packed), Is.EqualTo("8")); - - Assert.That(() => STuple.DecodeFirst(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeFirst(Slice.Empty), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeLast(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeLast(Slice.Empty), Throws.InstanceOf()); - - } - - [Test] - public void Test_STuple_UnpackSingle() - { - // should only work with tuples having exactly one element - - Slice packed; - - packed = STuple.EncodeKey(1); - Assert.That(STuple.DecodeKey(packed), Is.EqualTo(1)); - Assert.That(STuple.DecodeKey(packed), Is.EqualTo("1")); - - packed = STuple.EncodeKey("Hello\0World"); - Assert.That(STuple.DecodeKey(packed), Is.EqualTo("Hello\0World")); - - Assert.That(() => STuple.DecodeKey(Slice.Nil), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(Slice.Empty), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7)), Throws.InstanceOf()); - Assert.That(() => STuple.DecodeKey(STuple.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8)), Throws.InstanceOf()); - - } - - [Test] - public void Test_STuple_Embedded_Tuples() - { - // (A,B).Append((C,D)) should return (A,B,(C,D)) (length 3) and not (A,B,C,D) (length 4) - - STuple x = STuple.Create("A", "B"); - STuple y = STuple.Create("C", "D"); - - // using the instance method that returns an STuple - ITuple z = x.Append(y); - Log(z); - Assert.That(z, Is.Not.Null); - Assert.That(z.Count, Is.EqualTo(3)); - Assert.That(z[0], Is.EqualTo("A")); - Assert.That(z[1], Is.EqualTo("B")); - Assert.That(z[2], Is.EqualTo(y)); - var t = z.Get(2); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(2)); - Assert.That(t[0], Is.EqualTo("C")); - Assert.That(t[1], Is.EqualTo("D")); - - // casted down to the interface ITuple - z = ((ITuple)x).Append((ITuple)y); - Log(z); - Assert.That(z, Is.Not.Null); - Assert.That(z.Count, Is.EqualTo(3)); - Assert.That(z[0], Is.EqualTo("A")); - Assert.That(z[1], Is.EqualTo("B")); - Assert.That(z[2], Is.EqualTo(y)); - t = z.Get(2); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(2)); - Assert.That(t[0], Is.EqualTo("C")); - Assert.That(t[1], Is.EqualTo("D")); - - // composite index key "(prefix, value, id)" - ITuple subspace = STuple.Create(123, 42); - ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) - string id = "Doc123"; - z = subspace.Append(value, id); - Log(z); - Assert.That(z.Count, Is.EqualTo(4)); - } - - [Test] - public void Test_STuple_With() - { - //note: important to always cast to (ITuple) to be sure that we don't call specialized instance methods (tested elsewhere) - ITuple t; - - // Size 1 - - t = STuple.Create(123); - t.With((int a) => - { - Assert.That(a, Is.EqualTo(123)); - }); - Assert.That(t.With((int a) => - { - Assert.That(a, Is.EqualTo(123)); - return 42; - }), Is.EqualTo(42)); - - // Size 2 - - t = t.Append("abc"); - t.With((int a, string b) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - }); - Assert.That(t.With((int a, string b) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - return 42; - }), Is.EqualTo(42)); - - // Size 3 - - t = t.Append(3.14f); - t.With((int a, string b, float c) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - }); - Assert.That(t.With((int a, string b, float c) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - return 42; - }), Is.EqualTo(42)); - - // Size 4 - - t = t.Append(true); - t.With((int a, string b, float c, bool d) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - }); - Assert.That(t.With((int a, string b, float c, bool d) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - return 42; - }), Is.EqualTo(42)); - - // Size 5 - - t = t.Append('z'); - t.With((int a, string b, float c, bool d, char e) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - }); - Assert.That(t.With((int a, string b, float c, bool d, char e) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - return 42; - }), Is.EqualTo(42)); - - // Size 6 - - t = t.Append(Math.PI); - t.With((int a, string b, float c, bool d, char e, double f) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - }); - Assert.That(t.With((int a, string b, float c, bool d, char e, double f) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - return 42; - }), Is.EqualTo(42)); - - // Size 7 - - t = t.Append(IPAddress.Loopback); - t.With((int a, string b, float c, bool d, char e, double f, IPAddress g) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - Assert.That(g, Is.EqualTo(IPAddress.Loopback)); - }); - Assert.That(t.With((int a, string b, float c, bool d, char e, double f, IPAddress g) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - Assert.That(g, Is.EqualTo(IPAddress.Loopback)); - return 42; - }), Is.EqualTo(42)); - - // Size 8 - - t = t.Append(DateTime.MaxValue); - t.With((int a, string b, float c, bool d, char e, double f, IPAddress g, DateTime h) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - Assert.That(g, Is.EqualTo(IPAddress.Loopback)); - Assert.That(h, Is.EqualTo(DateTime.MaxValue)); - }); - Assert.That(t.With((int a, string b, float c, bool d, char e, double f, IPAddress g, DateTime h) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - Assert.That(f, Is.EqualTo(Math.PI)); - Assert.That(g, Is.EqualTo(IPAddress.Loopback)); - Assert.That(h, Is.EqualTo(DateTime.MaxValue)); - return 42; - }), Is.EqualTo(42)); - - } - - [Test] - public void Test_STuple_With_Struct() - { - // calling With() on the structs is faster - - STuple t1 = STuple.Create(123); - t1.With((a) => - { - Assert.That(a, Is.EqualTo(123)); - }); - Assert.That(t1.With((a) => - { - Assert.That(a, Is.EqualTo(123)); - return 42; - }), Is.EqualTo(42)); - - STuple t2 = STuple.Create(123, "abc"); - t2.With((a, b) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - }); - Assert.That(t2.With((a, b) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - return 42; - }), Is.EqualTo(42)); - - STuple t3 = STuple.Create(123, "abc", 3.14f); - t3.With((a, b, c) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - }); - Assert.That(t3.With((a, b, c) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - return 42; - }), Is.EqualTo(42)); - - STuple t4 = STuple.Create(123, "abc", 3.14f, true); - t4.With((a, b, c, d) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - }); - Assert.That(t4.With((a, b, c, d) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - return 42; - }), Is.EqualTo(42)); - - STuple t5 = STuple.Create(123, "abc", 3.14f, true, 'z'); - t5.With((a, b, c, d, e) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - }); - Assert.That(t5.With((a, b, c, d, e) => - { - Assert.That(a, Is.EqualTo(123)); - Assert.That(b, Is.EqualTo("abc")); - Assert.That(c, Is.EqualTo(3.14f)); - Assert.That(d, Is.True); - Assert.That(e, Is.EqualTo('z')); - return 42; - }), Is.EqualTo(42)); - - //TODO: add more if we ever add struct tuples with 6 or more items - } - - [Test] - public void Test_STuple_Of_Size() - { - // OfSize(n) check the size and return the tuple if it passed - // VerifySize(n) only check the size - // Both should throw if tuple is null, or not the expected size - - Action verify = (t) => - { - for (int i = 0; i <= 10; i++) - { - if (t.Count > i) - { - Assert.That(() => t.OfSize(i), Throws.InstanceOf()); - Assert.That(t.OfSizeAtLeast(i), Is.SameAs(t)); - Assert.That(() => t.OfSizeAtMost(i), Throws.InstanceOf()); - } - else if (t.Count < i) - { - Assert.That(() => t.OfSize(i), Throws.InstanceOf()); - Assert.That(() => t.OfSizeAtLeast(i), Throws.InstanceOf()); - Assert.That(t.OfSizeAtMost(i), Is.SameAs(t)); - } - else - { - Assert.That(t.OfSize(i), Is.SameAs(t)); - Assert.That(t.OfSizeAtLeast(i), Is.SameAs(t)); - Assert.That(t.OfSizeAtMost(i), Is.SameAs(t)); - } - } - }; - - verify(STuple.Empty); - verify(STuple.Create(123)); - verify(STuple.Create(123, "abc")); - verify(STuple.Create(123, "abc", 3.14f)); - verify(STuple.Create(123, "abc", 3.14f, true)); - verify(STuple.Create(123, "abc", 3.14f, true, 'z')); - verify(STuple.FromArray(new[] { "hello", "world", "!" })); - verify(STuple.FromEnumerable(Enumerable.Range(0, 10))); - - verify(STuple.Create(123, "abc", 3.14f, true, 'z')[0, 2]); - verify(STuple.Create(123, "abc", 3.14f, true, 'z')[1, 4]); - verify(STuple.FromEnumerable(Enumerable.Range(0, 50)).Substring(15, 6)); - - ITuple none = null; - Assert.That(() => none.OfSize(0), Throws.InstanceOf()); - Assert.That(() => none.OfSizeAtLeast(0), Throws.InstanceOf()); - Assert.That(() => none.OfSizeAtMost(0), Throws.InstanceOf()); - } - - [Test] - public void Test_STuple_Truncate() - { - ITuple t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); - - var head = t.Truncate(1); - Assert.That(head, Is.Not.Null); - Assert.That(head.Count, Is.EqualTo(1)); - Assert.That(head[0], Is.EqualTo("Hello")); - - head = t.Truncate(2); - Assert.That(head, Is.Not.Null); - Assert.That(head.Count, Is.EqualTo(2)); - Assert.That(head[0], Is.EqualTo("Hello")); - Assert.That(head[1], Is.EqualTo(123)); - - head = t.Truncate(5); - Assert.That(head, Is.EqualTo(t)); - - var tail = t.Truncate(-1); - Assert.That(tail, Is.Not.Null); - Assert.That(tail.Count, Is.EqualTo(1)); - Assert.That(tail[0], Is.EqualTo("World")); - - tail = t.Truncate(-2); - Assert.That(tail, Is.Not.Null); - Assert.That(tail.Count, Is.EqualTo(2)); - Assert.That(tail[0], Is.EqualTo(TimeSpan.FromSeconds(5))); - Assert.That(tail[1], Is.EqualTo("World")); - - tail = t.Truncate(-5); - Assert.That(tail, Is.EqualTo(t)); - - Assert.That(t.Truncate(0), Is.EqualTo(STuple.Empty)); - Assert.That(() => t.Truncate(6), Throws.InstanceOf()); - Assert.That(() => t.Truncate(-6), Throws.InstanceOf()); - - Assert.That(() => STuple.Empty.Truncate(1), Throws.InstanceOf()); - Assert.That(() => STuple.Create("Hello", "World").Truncate(3), Throws.InstanceOf()); - Assert.That(() => STuple.Create("Hello", "World").Truncate(-3), Throws.InstanceOf()); - } - - [Test] - public void Test_STuple_As() - { - // ITuple.As<...>() adds types to an untyped ITuple - ITuple t; - - t = STuple.Create("Hello"); - var t1 = t.As(); - Assert.That(t1.Item1, Is.EqualTo("Hello")); - - t = STuple.Create("Hello", 123); - var t2 = t.As(); - Assert.That(t2.Item1, Is.EqualTo("Hello")); - Assert.That(t2.Item2, Is.EqualTo(123)); - - t = STuple.Create("Hello", 123, false); - var t3 = t.As(); - Assert.That(t3.Item1, Is.EqualTo("Hello")); - Assert.That(t3.Item2, Is.EqualTo(123)); - Assert.That(t3.Item3, Is.EqualTo(false)); - - var t4 = STuple - .Create("Hello", 123, false, TimeSpan.FromSeconds(5)) - .As(); - Assert.That(t4.Item1, Is.EqualTo("Hello")); - Assert.That(t4.Item2, Is.EqualTo(123)); - Assert.That(t4.Item3, Is.EqualTo(false)); - Assert.That(t4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - - t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); - var t5 = t.As(); - Assert.That(t5.Item1, Is.EqualTo("Hello")); - Assert.That(t5.Item2, Is.EqualTo(123)); - Assert.That(t5.Item3, Is.EqualTo(false)); - Assert.That(t5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - Assert.That(t5.Item5, Is.EqualTo("World")); - } - - [Test] - public void Test_Cast_To_BCL_Tuples() - { - // implicit: Tuple => STuple - // explicit: STuple => Tuple - - var t1 = STuple.Create("Hello"); - var b1 = (Tuple) t1; // explicit - Assert.That(b1, Is.Not.Null); - Assert.That(b1.Item1, Is.EqualTo("Hello")); - STuple r1 = t1; // implicit - Assert.That(r1.Item1, Is.EqualTo("Hello")); - - var t2 = STuple.Create("Hello", 123); - var b2 = (Tuple)t2; // explicit - Assert.That(b2, Is.Not.Null); - Assert.That(b2.Item1, Is.EqualTo("Hello")); - Assert.That(b2.Item2, Is.EqualTo(123)); - STuple r2 = t2; // implicit - Assert.That(r2.Item1, Is.EqualTo("Hello")); - Assert.That(r2.Item2, Is.EqualTo(123)); - - var t3 = STuple.Create("Hello", 123, false); - var b3 = (Tuple)t3; // explicit - Assert.That(b3, Is.Not.Null); - Assert.That(b3.Item1, Is.EqualTo("Hello")); - Assert.That(b3.Item2, Is.EqualTo(123)); - Assert.That(b3.Item3, Is.EqualTo(false)); - STuple r3 = t3; // implicit - Assert.That(r3.Item1, Is.EqualTo("Hello")); - Assert.That(r3.Item2, Is.EqualTo(123)); - Assert.That(r3.Item3, Is.EqualTo(false)); - - var t4 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5)); - var b4 = (Tuple)t4; // explicit - Assert.That(b4, Is.Not.Null); - Assert.That(b4.Item1, Is.EqualTo("Hello")); - Assert.That(b4.Item2, Is.EqualTo(123)); - Assert.That(b4.Item3, Is.EqualTo(false)); - Assert.That(b4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - STuple r4 = t4; // implicit - Assert.That(r4.Item1, Is.EqualTo("Hello")); - Assert.That(r4.Item2, Is.EqualTo(123)); - Assert.That(r4.Item3, Is.EqualTo(false)); - Assert.That(r4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - - var t5 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); - var b5 = (Tuple)t5; // explicit - Assert.That(b5, Is.Not.Null); - Assert.That(b5.Item1, Is.EqualTo("Hello")); - Assert.That(b5.Item2, Is.EqualTo(123)); - Assert.That(b5.Item3, Is.EqualTo(false)); - Assert.That(b5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - Assert.That(b5.Item5, Is.EqualTo("World")); - STuple r5 = t5; // implicit - Assert.That(r5.Item1, Is.EqualTo("Hello")); - Assert.That(r5.Item2, Is.EqualTo(123)); - Assert.That(r5.Item3, Is.EqualTo(false)); - Assert.That(r5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); - Assert.That(r5.Item5, Is.EqualTo("World")); - - } - - #endregion - - #region Splicing... - - private static void VerifyTuple(string message, ITuple t, object[] expected) - { - // count - if (t.Count != expected.Length) - { -#if DEBUG - if (Debugger.IsAttached) Debugger.Break(); -#endif - Assert.Fail("{0}: Count mismatch between observed {1} and expected {2} for tuple of type {3}", message, t, STuple.ToString(expected), t.GetType().Name); - } - - // direct access - for (int i = 0; i < expected.Length; i++) - { - Assert.That(ComparisonHelper.AreSimilar(t[i], expected[i]), Is.True, "{0}: t[{1}] != expected[{1}]", message, i); - } - - // iterator - int p = 0; - foreach (var obj in t) - { - if (p >= expected.Length) Assert.Fail("Spliced iterator overshoot at t[{0}] = {1}", p, obj); - Assert.That(ComparisonHelper.AreSimilar(obj, expected[p]), Is.True, "{0}: Iterator[{1}], {2} ~= {3}", message, p, obj, expected[p]); - ++p; - } - Assert.That(p, Is.EqualTo(expected.Length), "{0}: t.GetEnumerator() returned only {1} elements out of {2} exected", message, p, expected.Length); - - // CopyTo - var tmp = new object[expected.Length]; - t.CopyTo(tmp, 0); - for (int i = 0; i < tmp.Length; i++) - { - Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: CopyTo[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); - } - - // Memoize - tmp = t.Memoize().ToArray(); - for (int i = 0; i < tmp.Length; i++) - { - Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: Memoize.Items[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); - } - - // Append - if (!(t is SlicedTuple)) - { - var u = t.Append("last"); - Assert.That(u.Get(-1), Is.EqualTo("last")); - tmp = u.ToArray(); - for (int i = 0; i < tmp.Length - 1; i++) - { - Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: Appended[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); - } - } - } - - [Test] - public void Test_Can_Splice_FdbListTuple() - { - var items = new object[] { "hello", "world", 123, "foo", 456, "bar" }; - // 0 1 2 3 4 5 - // -6 -5 -4 -3 -2 -1 - - var tuple = new ListTuple(items); - Assert.That(tuple.Count, Is.EqualTo(6)); - - // get all - VerifyTuple("[:]", tuple[null, null], items); - VerifyTuple("[:]", tuple[null, 6], items); - VerifyTuple("[:]", tuple[0, null], items); - VerifyTuple("[:]", tuple[0, 6], items); - VerifyTuple("[:]", tuple[0, null], items); - VerifyTuple("[:]", tuple[-6, null], items); - VerifyTuple("[:]", tuple[-6, 6], items); - - // tail - VerifyTuple("[n:]", tuple[4, null], new object[] { 456, "bar" }); - VerifyTuple("[n:+]", tuple[4, 6], new object[] { 456, "bar" }); - VerifyTuple("[-n:+]", tuple[-2, 6], new object[] { 456, "bar" }); - VerifyTuple("[-n:-]", tuple[-2, null], new object[] { 456, "bar" }); - - // head - VerifyTuple("[:n]", tuple[null, 3], new object[] { "hello", "world", 123 }); - VerifyTuple("[0:n]", tuple[0, 3], new object[] { "hello", "world", 123 }); - VerifyTuple("[0:-n]", tuple[0, -3], new object[] { "hello", "world", 123 }); - VerifyTuple("[-:n]", tuple[-6, 3], new object[] { "hello", "world", 123 }); - VerifyTuple("[-:-n]", tuple[-6, -3], new object[] { "hello", "world", 123 }); - - // single - VerifyTuple("[0:1]", tuple[0, 1], new object[] { "hello" }); - VerifyTuple("[-6:-5]", tuple[-6, -5], new object[] { "hello" }); - VerifyTuple("[1:2]", tuple[1, 2], new object[] { "world" }); - VerifyTuple("[-5:-4]", tuple[-5, -4], new object[] { "world" }); - VerifyTuple("[5:6]", tuple[5, 6], new object[] { "bar" }); - VerifyTuple("[-1:]", tuple[-1, null], new object[] { "bar" }); - - // chunk - VerifyTuple("[2:4]", tuple[2, 4], new object[] { 123, "foo" }); - VerifyTuple("[2:-2]", tuple[2, -2], new object[] { 123, "foo" }); - VerifyTuple("[-4:4]", tuple[-4, 4], new object[] { 123, "foo" }); - VerifyTuple("[-4:-2]", tuple[-4, -2], new object[] { 123, "foo" }); - - // remove first - VerifyTuple("[1:]", tuple[1, null], new object[] { "world", 123, "foo", 456, "bar" }); - VerifyTuple("[1:+]", tuple[1, 6], new object[] { "world", 123, "foo", 456, "bar" }); - VerifyTuple("[-5:]", tuple[-5, null], new object[] { "world", 123, "foo", 456, "bar" }); - VerifyTuple("[-5:+]", tuple[-5, 6], new object[] { "world", 123, "foo", 456, "bar" }); - - // remove last - VerifyTuple("[:5]", tuple[null, 5], new object[] { "hello", "world", 123, "foo", 456 }); - VerifyTuple("[:-1]", tuple[null, -1], new object[] { "hello", "world", 123, "foo", 456 }); - VerifyTuple("[0:5]", tuple[0, 5], new object[] { "hello", "world", 123, "foo", 456 }); - VerifyTuple("[0:-1]", tuple[0, -1], new object[] { "hello", "world", 123, "foo", 456 }); - - // out of range - VerifyTuple("[2:7]", tuple[2, 7], new object[] { 123, "foo", 456, "bar" }); - VerifyTuple("[2:42]", tuple[2, 42], new object[] { 123, "foo", 456, "bar" }); - VerifyTuple("[2:123456]", tuple[2, 123456], new object[] { 123, "foo", 456, "bar" }); - VerifyTuple("[-7:2]", tuple[-7, 2], new object[] { "hello", "world" }); - VerifyTuple("[-42:2]", tuple[-42, 2], new object[] { "hello", "world" }); - } - - private static object[] GetRange(int fromIncluded, int toExcluded, int count) - { - if (count == 0) return new object[0]; - - if (fromIncluded < 0) fromIncluded += count; - if (toExcluded < 0) toExcluded += count; - - if (toExcluded > count) toExcluded = count; - var tmp = new object[toExcluded - fromIncluded]; - for (int i = 0; i < tmp.Length; i++) tmp[i] = new string((char) (65 + fromIncluded + i), 1); - return tmp; - } - - [Test] - public void Test_Randomized_Splices() - { - // Test a random mix of sizes, and indexes... - - const int N = 100 * 1000; - - var tuples = new ITuple[14]; - tuples[0] = STuple.Empty; - tuples[1] = STuple.Create("A"); - tuples[2] = STuple.Create("A", "B"); - tuples[3] = STuple.Create("A", "B", "C"); - tuples[4] = STuple.Create("A", "B", "C", "D"); - tuples[5] = STuple.Create("A", "B", "C", "D", "E"); - tuples[6] = STuple.Create("A", "B", "C", "D", "E", "F"); - tuples[7] = STuple.Create("A", "B", "C", "D", "E", "F", "G"); - tuples[8] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H"); - tuples[9] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I"); - tuples[10]= STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J"); - tuples[11] = new JoinedTuple(tuples[6], STuple.Create("G", "H", "I", "J", "K")); - tuples[12] = new LinkedTuple(tuples[11], "L"); - tuples[13] = new LinkedTuple(STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L"), "M"); - -#if false - Console.Write("Checking tuples"); - - foreach (var tuple in tuples) - { - var t = FdbTuple.Unpack(tuple.ToSlice()); - Assert.That(t.Equals(tuple), Is.True, t.ToString() + " != unpack(" + tuple.ToString() + ")"); - } -#endif - - var rnd = new Random(123456); - - for (int i = 0; i < N; i++) - { - if (i % 500 == 0) Console.Write("."); - var len = rnd.Next(tuples.Length); - var tuple = tuples[len]; - Assert.That(tuple.Count, Is.EqualTo(len)); - - string prefix = tuple.ToString(); - - if (rnd.Next(5) == 0) - { // randomly pack/unpack - tuple = STuple.Unpack(tuple.ToSlice()); - prefix = "unpacked:" + prefix; - } - else if (rnd.Next(5) == 0) - { // randomly memoize - tuple = tuple.Memoize(); - prefix = "memoized:" + prefix; - } - - switch (rnd.Next(6)) - { - case 0: - { // [:+rnd] - int x = rnd.Next(len); - VerifyTuple(prefix + "[:" + x.ToString() + "]", tuple[null, x], GetRange(0, x, len)); - break; - } - case 1: - { // [+rnd:] - int x = rnd.Next(len); - VerifyTuple(prefix + "[" + x.ToString() + ":]", tuple[x, null], GetRange(x, int.MaxValue, len)); - break; - } - case 2: - { // [:-rnd] - int x = -1 - rnd.Next(len); - VerifyTuple(prefix + "[:" + x.ToString() + "]", tuple[null, x], GetRange(0, len + x, len)); - break; - } - case 3: - { // [-rnd:] - int x = -1 - rnd.Next(len); - VerifyTuple(prefix + "[" + x.ToString() + ":]", tuple[x, null], GetRange(len + x, int.MaxValue, len)); - break; - } - case 4: - { // [rnd:rnd] - int x = rnd.Next(len); - int y; - do { y = rnd.Next(len); } while (y < x); - VerifyTuple(prefix + " [" + x.ToString() + ":" + y.ToString() + "]", tuple[x, y], GetRange(x, y, len)); - break; - } - case 5: - { // [-rnd:-rnd] - int x = -1 - rnd.Next(len); - int y; - do { y = -1 - rnd.Next(len); } while (y < x); - VerifyTuple(prefix + " [" + x.ToString() + ":" + y.ToString() + "]", tuple[x, y], GetRange(len + x, len + y, len)); - break; - } - } - - } - Console.WriteLine(" done"); - - } - - #endregion - - #region Serialization... - - [Test] - public void Test_STuple_Serialize_Bytes() - { - // Byte arrays are stored with prefix '01' followed by the bytes, and terminated by '00'. All occurences of '00' in the byte array are escaped with '00 FF' - // - Best case: packed_size = 2 + array_len - // - Worst case: packed_size = 2 + array_len * 2 - - Slice packed; - - packed = STuple.EncodeKey(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }); - Assert.That(packed.ToString(), Is.EqualTo("<01><12>4Vx<9A><00>")); - packed = STuple.EncodeKey(new byte[] { 0x00, 0x42 }); - Assert.That(packed.ToString(), Is.EqualTo("<01><00>B<00>")); - packed = STuple.EncodeKey(new byte[] { 0x42, 0x00 }); - Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>")); - packed = STuple.EncodeKey(new byte[] { 0x42, 0x00, 0x42 }); - Assert.That(packed.ToString(), Is.EqualTo("<01>B<00>B<00>")); - packed = STuple.EncodeKey(new byte[] { 0x42, 0x00, 0x00, 0x42 }); - Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>B<00>")); - } - - [Test] - public void Test_STuple_Deserialize_Bytes() - { - ITuple t; - - t = STuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); - Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF })); - Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF")); - - t = STuple.Unpack(Slice.Unescape("<01><42><00><00>")); - Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00 })); - Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00")); - - t = STuple.Unpack(Slice.Unescape("<01><00><42><00>")); - Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x00, 0x42 })); - Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("00 42")); - - t = STuple.Unpack(Slice.Unescape("<01><42><00><42><00>")); - Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00, 0x42 })); - Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 42")); - - t = STuple.Unpack(Slice.Unescape("<01><42><00><00><42><00>")); - Assert.That(t.Get(0), Is.EqualTo(new byte[] { 0x42, 0x00, 0x00, 0x42 })); - Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 00 42")); - } - - [Test] - public void Test_STuple_Serialize_Unicode_Strings() - { - // Unicode strings are stored with prefix '02' followed by the utf8 bytes, and terminated by '00'. All occurences of '00' in the UTF8 bytes are escaped with '00 FF' - - Slice packed; - - // simple string - packed = STuple.Create("hello world").ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("<02>hello world<00>")); - - // empty - packed = STuple.Create(String.Empty).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("<02><00>")); - - // null - packed = STuple.Create(default(string)).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("<00>")); - - // unicode - packed = STuple.Create("こんにちは世界").ToSlice(); - // note: Encoding.UTF8.GetBytes("こんにちは世界") => { e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c } - Assert.That(packed.ToString(), Is.EqualTo("<02><81><93><82><93><81><81><81><96><95><8C><00>")); - } - - [Test] - public void Test_STuple_Deserialize_Unicode_Strings() - { - ITuple t; - - // simple string - t = STuple.Unpack(Slice.Unescape("<02>hello world<00>")); - Assert.That(t.Get(0), Is.EqualTo("hello world")); - Assert.That(t[0], Is.EqualTo("hello world")); - - // empty - t = STuple.Unpack(Slice.Unescape("<02><00>")); - Assert.That(t.Get(0), Is.EqualTo(String.Empty)); - Assert.That(t[0], Is.EqualTo(String.Empty)); - - // null - t = STuple.Unpack(Slice.Unescape("<00>")); - Assert.That(t.Get(0), Is.EqualTo(default(string))); - Assert.That(t[0], Is.Null); - - // unicode - t = STuple.Unpack(Slice.Unescape("<02><81><93><82><93><81><81><81><96><95><8C><00>")); - // note: Encoding.UTF8.GetString({ e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c }) => "こんにちは世界" - Assert.That(t.Get(0), Is.EqualTo("こんにちは世界")); - Assert.That(t[0], Is.EqualTo("こんにちは世界")); - } - - [Test] - public void Test_STuple_Serialize_Guids() - { - // 128-bit Guids are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 - - // System.Guid are stored in Little-Endian, but RFC 4122's UUIDs are stored in Big Endian, so per convention we will swap them - - Slice packed; - - // note: new Guid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; - packed = STuple.Create(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - - packed = STuple.Create(Guid.Empty).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); - - } - - [Test] - public void Test_STuple_Deserialize_Guids() - { - // 128-bit Guids are stored with prefix '30' followed by 16 bytes - // we also accept byte arrays (prefix '01') if they are of length 16 - - ITuple packed; - - packed = STuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - Assert.That(packed[0], Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - - packed = STuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); - Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); - Assert.That(packed[0], Is.EqualTo(Guid.Empty)); - - // unicode string - packed = STuple.Unpack(Slice.Unescape("<02>03020100-0504-0706-0809-0a0b0c0d0e0f<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("03020100-0504-0706-0809-0a0b0c0d0e0f"))); - //note: t[0] returns a string, not a GUID - - // null maps to Guid.Empty - packed = STuple.Unpack(Slice.Unescape("<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); - //note: t[0] returns null, not a GUID - - } - - [Test] - public void Test_STuple_Serialize_Uuid128s() - { - // UUID128s are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 - - Slice packed; - - // note: new Uuid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; - packed = STuple.Create(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - - packed = STuple.Create(Uuid128.Empty).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); - } - - [Test] - public void Test_STuple_Deserialize_Uuid128s() - { - // UUID128s are stored with prefix '30' followed by 16 bytes (the result of uuid.ToByteArray()) - // we also accept byte arrays (prefix '01') if they are of length 16 - - ITuple packed; - - // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; - packed = STuple.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - Assert.That(packed[0], Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - - packed = STuple.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); - Assert.That(packed[0], Is.EqualTo(Uuid128.Empty)); - - // unicode string - packed = STuple.Unpack(Slice.Unescape("<02>00010203-0405-0607-0809-0a0b0c0d0e0f<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); - //note: t[0] returns a string, not a UUID - - // null maps to Uuid.Empty - packed = STuple.Unpack(Slice.Unescape("<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); - //note: t[0] returns null, not a UUID - - } - - [Test] - public void Test_STuple_Serialize_Uuid64s() - { - // UUID64s are stored with prefix '31' followed by 8 bytes formatted according to RFC 4122 - - Slice packed; - - // note: new Uuid(bytes from 0 to 7) => "00010203-04050607"; - packed = STuple.Create(Uuid64.Parse("00010203-04050607")).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("1<00><01><02><03><04><05><06><07>")); - - packed = STuple.Create(Uuid64.Parse("01234567-89ABCDEF")).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("1<01>#Eg<89>")); - - packed = STuple.Create(Uuid64.Empty).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00><00><00><00><00>")); - - packed = STuple.Create(new Uuid64(0xBADC0FFEE0DDF00DUL)).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("1<0F>
<0D>")); - - packed = STuple.Create(new Uuid64(0xDEADBEEFL)).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00>")); - } - - [Test] - public void Test_STuple_Deserialize_Uuid64s() - { - // UUID64s are stored with prefix '31' followed by 8 bytes (the result of uuid.ToByteArray()) - // we also accept byte arrays (prefix '01') if they are of length 8, and unicode strings (prefix '02') - - ITuple packed; - - // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; - packed = STuple.Unpack(Slice.Unescape("<31><01><23><45><67><89>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); - Assert.That(packed[0], Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); - - packed = STuple.Unpack(Slice.Unescape("<31><00><00><00><00><00><00><00><00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); - Assert.That(packed[0], Is.EqualTo(Uuid64.Empty)); - - // 8 bytes - packed = STuple.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); - //note: t[0] returns a string, not a UUID - - // unicode string - packed = STuple.Unpack(Slice.Unescape("<02>01234567-89abcdef<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); - //note: t[0] returns a string, not a UUID - - // null maps to Uuid.Empty - packed = STuple.Unpack(Slice.Unescape("<00>")); - Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); - //note: t[0] returns null, not a UUID - - } - - [Test] - public void Test_STuple_Serialize_Integers() - { - // Positive integers are stored with a variable-length encoding. - // - The prefix is 0x14 + the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x14 to 0x1C - // - The bytes are stored in High-Endian (ie: the upper bits first) - // Examples: - // - 0 => <14> - // - 1..255 => <15><##> - // - 256..65535 .. => <16> - // - ulong.MaxValue => <1C> - - Assert.That( - STuple.Create(0).ToSlice().ToString(), - Is.EqualTo("<14>") - ); - - Assert.That( - STuple.Create(1).ToSlice().ToString(), - Is.EqualTo("<15><01>") - ); - - Assert.That( - STuple.Create(255).ToSlice().ToString(), - Is.EqualTo("<15>") - ); - - Assert.That( - STuple.Create(256).ToSlice().ToString(), - Is.EqualTo("<16><01><00>") - ); - - Assert.That( - STuple.Create(65535).ToSlice().ToString(), - Is.EqualTo("<16>") - ); - - Assert.That( - STuple.Create(65536).ToSlice().ToString(), - Is.EqualTo("<17><01><00><00>") - ); - - Assert.That( - STuple.Create(int.MaxValue).ToSlice().ToString(), - Is.EqualTo("<18><7F>") - ); - - // signed max - Assert.That( - STuple.Create(long.MaxValue).ToSlice().ToString(), - Is.EqualTo("<1C><7F>") - ); - - // unsigned max - Assert.That( - STuple.Create(ulong.MaxValue).ToSlice().ToString(), - Is.EqualTo("<1C>") - ); - } - - [Test] - public void Test_STuple_Deserialize_Integers() - { - - Action verify = (encoded, value) => - { - var slice = Slice.Unescape(encoded); - Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(value), "DeserializeBoxed({0})", encoded); - - // int64 - Assert.That(TuplePackers.DeserializeInt64(slice), Is.EqualTo(value), "DeserializeInt64({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo(value), "Deserialize({0})", encoded); - - // uint64 - if (value >= 0) - { - Assert.That(TuplePackers.DeserializeUInt64(slice), Is.EqualTo((ulong)value), "DeserializeUInt64({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ulong)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeUInt64(slice), Throws.InstanceOf(), "DeserializeUInt64({0})", encoded); - } - - // int32 - if (value <= int.MaxValue && value >= int.MinValue) - { - Assert.That(TuplePackers.DeserializeInt32(slice), Is.EqualTo((int)value), "DeserializeInt32({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((int)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeInt32(slice), Throws.InstanceOf(), "DeserializeInt32({0})", encoded); - } - - // uint32 - if (value <= uint.MaxValue && value >= 0) - { - Assert.That(TuplePackers.DeserializeUInt32(slice), Is.EqualTo((uint)value), "DeserializeUInt32({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((uint)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeUInt32(slice), Throws.InstanceOf(), "DeserializeUInt32({0})", encoded); - } - - // int16 - if (value <= short.MaxValue && value >= short.MinValue) - { - Assert.That(TuplePackers.DeserializeInt16(slice), Is.EqualTo((short)value), "DeserializeInt16({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((short)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeInt16(slice), Throws.InstanceOf(), "DeserializeInt16({0})", encoded); - } - - // uint16 - if (value <= ushort.MaxValue && value >= 0) - { - Assert.That(TuplePackers.DeserializeUInt16(slice), Is.EqualTo((ushort)value), "DeserializeUInt16({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ushort)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeUInt16(slice), Throws.InstanceOf(), "DeserializeUInt16({0})", encoded); - } - - // sbyte - if (value <= sbyte.MaxValue && value >= sbyte.MinValue) - { - Assert.That(TuplePackers.DeserializeSByte(slice), Is.EqualTo((sbyte)value), "DeserializeSByte({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((sbyte)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeSByte(slice), Throws.InstanceOf(), "DeserializeSByte({0})", encoded); - } - - // byte - if (value <= 255 && value >= 0) - { - Assert.That(TuplePackers.DeserializeByte(slice), Is.EqualTo((byte)value), "DeserializeByte({0})", encoded); - Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((byte)value), "Deserialize({0})", encoded); - } - else - { - Assert.That(() => TuplePackers.DeserializeByte(slice), Throws.InstanceOf(), "DeserializeByte({0})", encoded); - } - - }; - verify("<14>", 0); - verify("<15>{", 123); - verify("<15><80>", 128); - verify("<15>", 255); - verify("<16><01><00>", 256); - verify("<16><04>", 1234); - verify("<16><80><00>", 32768); - verify("<16>", 65535); - verify("<17><01><00><00>", 65536); - verify("<13>", -1); - verify("<13><00>", -255); - verify("<12>", -256); - verify("<12><00><00>", -65535); - verify("<11>", -65536); - verify("<18><7F>", int.MaxValue); - verify("<10><7F>", int.MinValue); - verify("<1C><7F>", long.MaxValue); - verify("<0C><7F>", long.MinValue); - } - - [Test] - public void Test_STuple_Serialize_Negative_Integers() - { - // Negative integers are stored with a variable-length encoding. - // - The prefix is 0x14 - the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x0C to 0x13 - // - The value is encoded as the one's complement, and stored in High-Endian (ie: the upper bits first) - // - There is no way to encode '-0', it will be encoded as '0' (<14>) - // Examples: - // - -255..-1 => <13><00> .. <13> - // - -65535..-256 => <12><00>00> .. <12> - // - long.MinValue => <0C><7F> - - Assert.That( - STuple.Create(-1).ToSlice().ToString(), - Is.EqualTo("<13>") - ); - - Assert.That( - STuple.Create(-255).ToSlice().ToString(), - Is.EqualTo("<13><00>") - ); - - Assert.That( - STuple.Create(-256).ToSlice().ToString(), - Is.EqualTo("<12>") - ); - Assert.That( - STuple.Create(-257).ToSlice().ToString(), - Is.EqualTo("<12>") - ); - - Assert.That( - STuple.Create(-65535).ToSlice().ToString(), - Is.EqualTo("<12><00><00>") - ); - Assert.That( - STuple.Create(-65536).ToSlice().ToString(), - Is.EqualTo("<11>") - ); - - Assert.That( - STuple.Create(int.MinValue).ToSlice().ToString(), - Is.EqualTo("<10><7F>") - ); - - Assert.That( - STuple.Create(long.MinValue).ToSlice().ToString(), - Is.EqualTo("<0C><7F>") - ); - } - - [Test] - public void Test_STuple_Serialize_Singles() - { - // 32-bit floats are stored in 5 bytes, using the prefix 0x20 followed by the High-Endian representation of their normalized form - - Assert.That(STuple.Create(0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 00")); - Assert.That(STuple.Create(42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 C2 28 00 00")); - Assert.That(STuple.Create(-42f).ToSlice().ToHexaString(' '), Is.EqualTo("20 3D D7 FF FF")); - - Assert.That(STuple.Create((float)Math.Sqrt(2)).ToSlice().ToHexaString(' '), Is.EqualTo("20 BF B5 04 F3")); - - Assert.That(STuple.Create(float.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 80 00 00"), "float.MinValue"); - Assert.That(STuple.Create(float.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 7F FF FF"), "float.MaxValue"); - Assert.That(STuple.Create(-0f).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FF"), "-0f"); - Assert.That(STuple.Create(float.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 7F FF FF"), "float.NegativeInfinity"); - Assert.That(STuple.Create(float.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("20 FF 80 00 00"), "float.PositiveInfinity"); - Assert.That(STuple.Create(float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 80 00 00 01"), "+float.Epsilon"); - Assert.That(STuple.Create(-float.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("20 7F FF FF FE"), "-float.Epsilon"); - - // all possible variants of NaN should all be equal - Assert.That(STuple.Create(float.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("20 00 3F FF FF"), "float.NaN"); - - // cook up a non standard NaN (with some bits set in the fraction) - float f = float.NaN; // defined as 1f / 0f - uint nan; - unsafe { nan = *((uint*)&f); } - nan += 123; - unsafe { f = *((float*)&nan); } - Assert.That(float.IsNaN(f), Is.True); - Assert.That( - STuple.Create(f).ToSlice().ToHexaString(' '), - Is.EqualTo("20 00 3F FF FF"), - "All variants of NaN must be normalized" - //note: if we have 20 00 3F FF 84, that means that the NaN was not normalized - ); - - } - - [Test] - public void Test_STuple_Deserialize_Singles() - { - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 80 00 00 00")), Is.EqualTo(0f), "0f"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 C2 28 00 00")), Is.EqualTo(42f), "42f"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 3D D7 FF FF")), Is.EqualTo(-42f), "-42f"); - - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 BF B5 04 F3")), Is.EqualTo((float)Math.Sqrt(2)), "Sqrt(2)"); - - // well known values - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.MinValue"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 FF 7F FF FF")), Is.EqualTo(float.MaxValue), "float.MaxValue"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 7F FF FF FF")), Is.EqualTo(-0f), "-0f"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 7F FF FF")), Is.EqualTo(float.NegativeInfinity), "float.NegativeInfinity"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 FF 80 00 00")), Is.EqualTo(float.PositiveInfinity), "float.PositiveInfinity"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.Epsilon"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 80 00 00 01")), Is.EqualTo(float.Epsilon), "+float.Epsilon"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 7F FF FF FE")), Is.EqualTo(-float.Epsilon), "-float.Epsilon"); - - // all possible variants of NaN should end up equal and normalized to float.NaN - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); - } - - [Test] - public void Test_STuple_Serialize_Doubles() - { - // 64-bit floats are stored in 9 bytes, using the prefix 0x21 followed by the High-Endian representation of their normalized form - - Assert.That(STuple.Create(0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 00")); - Assert.That(STuple.Create(42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 45 00 00 00 00 00 00")); - Assert.That(STuple.Create(-42d).ToSlice().ToHexaString(' '), Is.EqualTo("21 3F BA FF FF FF FF FF FF")); - - Assert.That(STuple.Create(Math.PI).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 09 21 FB 54 44 2D 18")); - Assert.That(STuple.Create(Math.E).ToSlice().ToHexaString(' '), Is.EqualTo("21 C0 05 BF 0A 8B 14 57 69")); - - Assert.That(STuple.Create(double.MinValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 10 00 00 00 00 00 00"), "double.MinValue"); - Assert.That(STuple.Create(double.MaxValue).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF EF FF FF FF FF FF FF"), "double.MaxValue"); - Assert.That(STuple.Create(-0d).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FF"), "-0d"); - Assert.That(STuple.Create(double.NegativeInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 0F FF FF FF FF FF FF"), "double.NegativeInfinity"); - Assert.That(STuple.Create(double.PositiveInfinity).ToSlice().ToHexaString(' '), Is.EqualTo("21 FF F0 00 00 00 00 00 00"), "double.PositiveInfinity"); - Assert.That(STuple.Create(double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 01"), "+double.Epsilon"); - Assert.That(STuple.Create(-double.Epsilon).ToSlice().ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FE"), "-double.Epsilon"); - - // all possible variants of NaN should all be equal - - Assert.That(STuple.Create(double.NaN).ToSlice().ToHexaString(' '), Is.EqualTo("21 00 07 FF FF FF FF FF FF"), "double.NaN"); - - // cook up a non standard NaN (with some bits set in the fraction) - double d = double.NaN; // defined as 1d / 0d - ulong nan; - unsafe { nan = *((ulong*)&d); } - nan += 123; - unsafe { d = *((double*)&nan); } - Assert.That(double.IsNaN(d), Is.True); - Assert.That( - STuple.Create(d).ToSlice().ToHexaString(' '), - Is.EqualTo("21 00 07 FF FF FF FF FF FF") - //note: if we have 21 00 07 FF FF FF FF FF 84, that means that the NaN was not normalized - ); - - // roundtripping vectors of doubles - var tuple = STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)); - Assert.That(STuple.Unpack(STuple.EncodeKey(Math.PI, Math.E, Math.Log(1), Math.Log(2))), Is.EqualTo(tuple)); - Assert.That(STuple.Unpack(STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); - Assert.That(STuple.Unpack(STuple.Empty.Append(Math.PI).Append(Math.E).Append(Math.Log(1)).Append(Math.Log(2)).ToSlice()), Is.EqualTo(tuple)); - } - - [Test] - public void Test_STuple_Deserialize_Doubles() - { - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 00")), Is.EqualTo(0d), "0d"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 45 00 00 00 00 00 00")), Is.EqualTo(42d), "42d"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 3F BA FF FF FF FF FF FF")), Is.EqualTo(-42d), "-42d"); - - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 09 21 FB 54 44 2D 18")), Is.EqualTo(Math.PI), "Math.PI"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 C0 05 BF 0A 8B 14 57 69")), Is.EqualTo(Math.E), "Math.E"); - - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 10 00 00 00 00 00 00")), Is.EqualTo(double.MinValue), "double.MinValue"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 FF EF FF FF FF FF FF FF")), Is.EqualTo(double.MaxValue), "double.MaxValue"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FF")), Is.EqualTo(-0d), "-0d"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 0F FF FF FF FF FF FF")), Is.EqualTo(double.NegativeInfinity), "double.NegativeInfinity"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 FF F0 00 00 00 00 00 00")), Is.EqualTo(double.PositiveInfinity), "double.PositiveInfinity"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 01")), Is.EqualTo(double.Epsilon), "+double.Epsilon"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FE")), Is.EqualTo(-double.Epsilon), "-double.Epsilon"); - - // all possible variants of NaN should end up equal and normalized to double.NaN - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF FF")), Is.EqualTo(double.NaN), "double.NaN"); - Assert.That(STuple.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF 84")), Is.EqualTo(double.NaN), "double.NaN"); - } - - [Test] - public void Test_STuple_Serialize_Booleans() - { - // Booleans are stored as interger 0 (<14>) for false, and integer 1 (<15><01>) for true - - Slice packed; - - // bool - packed = STuple.EncodeKey(false); - Assert.That(packed.ToString(), Is.EqualTo("<14>")); - packed = STuple.EncodeKey(true); - Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); - - // bool? - packed = STuple.EncodeKey(default(bool?)); - Assert.That(packed.ToString(), Is.EqualTo("<00>")); - packed = STuple.EncodeKey((bool?)false); - Assert.That(packed.ToString(), Is.EqualTo("<14>")); - packed = STuple.EncodeKey((bool?)true); - Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); - - // tuple containing bools - packed = STuple.Create(true).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); - packed = STuple.Create(true, null, false).ToSlice(); - Assert.That(packed.ToString(), Is.EqualTo("<15><01><00><14>")); - } - - [Test] - public void Test_STuple_Deserialize_Booleans() - { - // Null, 0, and empty byte[]/strings are equivalent to False. All others are equivalent to True - - // Falsy... - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.EqualTo(false), "Null => False"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(false), "0 => False"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<01><00>")), Is.EqualTo(false), "byte[0] => False"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<02><00>")), Is.EqualTo(false), "String.Empty => False"); - - // Truthy - Assert.That(STuple.DecodeKey(Slice.Unescape("<15><01>")), Is.EqualTo(true), "1 => True"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<13>")), Is.EqualTo(true), "-1 => True"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<01>Hello<00>")), Is.EqualTo(true), "'Hello' => True"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<02>Hello<00>")), Is.EqualTo(true), "\"Hello\" => True"); - Assert.That(STuple.DecodeKey(STuple.EncodeKey(123456789)), Is.EqualTo(true), "random int => True"); - - Assert.That(STuple.DecodeKey(Slice.Unescape("<02>True<00>")), Is.EqualTo(true), "\"True\" => True"); - Assert.That(STuple.DecodeKey(Slice.Unescape("<02>False<00>")), Is.EqualTo(true), "\"False\" => True ***"); - // note: even though it would be tempting to convert the string "false" to False, it is not a standard behavior accross all bindings - - // When decoded to object, though, they should return 0 and 1 - Assert.That(TuplePackers.DeserializeBoxed(STuple.EncodeKey(false)), Is.EqualTo(0)); - Assert.That(TuplePackers.DeserializeBoxed(STuple.EncodeKey(true)), Is.EqualTo(1)); - } - - [Test] - public void Test_STuple_Serialize_IPAddress() - { - // IP Addresses are stored as a byte array (<01>..<00>), in network order (big-endian) - // They will take from 6 to 10 bytes, depending on the number of '.0' in them. - - Assert.That( - STuple.Create(IPAddress.Loopback).ToSlice().ToHexaString(' '), - Is.EqualTo("01 7F 00 FF 00 FF 01 00") - ); - - Assert.That( - STuple.Create(IPAddress.Any).ToSlice().ToHexaString(' '), - Is.EqualTo("01 00 FF 00 FF 00 FF 00 FF 00") - ); - - Assert.That( - STuple.Create(IPAddress.Parse("1.2.3.4")).ToSlice().ToHexaString(' '), - Is.EqualTo("01 01 02 03 04 00") - ); - - } - - - [Test] - public void Test_STuple_Deserialize_IPAddress() - { - Assert.That(STuple.DecodeKey(Slice.Unescape("<01><7F><00><00><01><00>")), Is.EqualTo(IPAddress.Parse("127.0.0.1"))); - Assert.That(STuple.DecodeKey(Slice.Unescape("<01><00><00><00><00><00>")), Is.EqualTo(IPAddress.Parse("0.0.0.0"))); - Assert.That(STuple.DecodeKey(Slice.Unescape("<01><01><02><03><04><00>")), Is.EqualTo(IPAddress.Parse("1.2.3.4"))); - - Assert.That(STuple.DecodeKey(STuple.EncodeKey("127.0.0.1")), Is.EqualTo(IPAddress.Loopback)); - - var ip = IPAddress.Parse("192.168.0.1"); - Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.ToString())), Is.EqualTo(ip)); - Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.GetAddressBytes())), Is.EqualTo(ip)); - Assert.That(STuple.DecodeKey(STuple.EncodeKey(ip.Address)), Is.EqualTo(ip)); - } - - [Test] - public void Test_STuple_NullableTypes() - { - // Nullable types will either be encoded as <14> for null, or their regular encoding if not null - - // serialize - - Assert.That(STuple.EncodeKey(0), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(STuple.EncodeKey(123), Is.EqualTo(Slice.Unescape("<15>{"))); - Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - - Assert.That(STuple.EncodeKey(0L), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(STuple.EncodeKey(123L), Is.EqualTo(Slice.Unescape("<15>{"))); - Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - - Assert.That(STuple.EncodeKey(true), Is.EqualTo(Slice.Unescape("<15><01>"))); - Assert.That(STuple.EncodeKey(false), Is.EqualTo(Slice.Unescape("<14>"))); - Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>")), "Maybe it was File Not Found?"); - - Assert.That(STuple.EncodeKey(Guid.Empty), Is.EqualTo(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>"))); - Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - - Assert.That(STuple.EncodeKey(TimeSpan.Zero), Is.EqualTo(Slice.Unescape("!<80><00><00><00><00><00><00><00>"))); - Assert.That(STuple.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); - - // deserialize - - Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - - Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0L)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123L)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - - Assert.That(STuple.DecodeKey(Slice.Unescape("<15><01>")), Is.True); - Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.False); - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - - Assert.That(STuple.DecodeKey(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")), Is.EqualTo(Guid.Empty)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - - Assert.That(STuple.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(TimeSpan.Zero)); - Assert.That(STuple.DecodeKey(Slice.Unescape("<00>")), Is.Null); - - } - - [Test] - public void Test_STuple_Serialize_Alias() - { - Assert.That( - STuple.EncodeKey(FdbTupleAlias.System).ToString(), - Is.EqualTo("") - ); - - Assert.That( - STuple.EncodeKey(FdbTupleAlias.Directory).ToString(), - Is.EqualTo("") - ); - - Assert.That( - STuple.EncodeKey(FdbTupleAlias.Zero).ToString(), - Is.EqualTo("<00>") - ); - - } - - [Test] - public void Test_STuple_Deserialize_Alias() - { - Slice slice; - - slice = Slice.Unescape(""); - Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.System)); - - slice = Slice.Unescape(""); - Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(FdbTupleAlias.Directory)); - - //note: FdbTupleAlias.Start is <00> and will be deserialized as null - } - - [Test] - public void Test_STuple_Serialize_Embedded_Tuples() - { - Action verify = (t, expected) => - { - var key = t.ToSlice(); - Assert.That(key.ToHexaString(' '), Is.EqualTo(expected)); - Assert.That(STuple.Pack(t), Is.EqualTo(key)); - var t2 = STuple.Unpack(key); - Assert.That(t2, Is.Not.Null); - Assert.That(t2.Count, Is.EqualTo(t.Count), "{0}", t2); - Assert.That(t2, Is.EqualTo(t)); - }; - - // Index composite key - ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) - string docId = "Doc123"; - // key would be "(..., value, id)" - - verify( - STuple.Create(42, value, docId), - "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" - ); - verify( - STuple.Create(new object[] { 42, value, docId }), - "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" - ); - verify( - STuple.Create(42).Append(value).Append(docId), - "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" - ); - verify( - STuple.Create(42).Append(value, docId), - "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" - ); - - // multiple depth - verify( - STuple.Create(1, STuple.Create(2, 3), STuple.Create(STuple.Create(4, 5, 6)), 7), - "15 01 03 15 02 15 03 00 03 03 15 04 15 05 15 06 00 00 15 07" - ); - - // corner cases - verify( - STuple.Create(STuple.Empty), - "03 00" // empty tumple should have header and footer - ); - verify( - STuple.Create(STuple.Empty, default(string)), - "03 00 00" // outer null should not be escaped - ); - verify( - STuple.Create(STuple.Create(default(string)), default(string)), - "03 00 FF 00 00" // inner null should be escaped, but not outer - ); - verify( - STuple.Create(STuple.Create(0x100, 0x10000, 0x1000000)), - "03 16 01 00 17 01 00 00 18 01 00 00 00 00" - ); - verify( - STuple.Create(default(string), STuple.Empty, default(string), STuple.Create(default(string)), default(string)), - "00 03 00 00 03 00 FF 00 00" - ); - - } - - [Test] - public void Test_STuple_SameBytes() - { - ITuple t1 = STuple.Create("hello world"); - ITuple t2 = STuple.Create(new object[] { "hello world" }); - - Assert.That(t1.ToSlice(), Is.EqualTo(t2.ToSlice())); - - t1 = STuple.Create("hello world", 1234); - t2 = STuple.Create("hello world").Append(1234); - - Assert.That(t1.ToSlice(), Is.EqualTo(t2.ToSlice())); - - } - - [Test] - public void Test_STuple_Create_ToSlice() - { - Assert.That( - STuple.Create("hello world").ToSlice().ToString(), - Is.EqualTo("<02>hello world<00>") - ); - - Assert.That( - STuple.Create("hello", "world").ToSlice().ToString(), - Is.EqualTo("<02>hello<00><02>world<00>") - ); - - Assert.That( - STuple.Create("hello world", 123).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><15>{") - ); - - Assert.That( - STuple.Create("hello world", 1234, -1234).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><16><04><12>-") - ); - - Assert.That( - STuple.Create("hello world", 123, false).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><15>{<14>") - ); - - Assert.That( - STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") - ); - - Assert.That( - STuple.Create(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") - ); - - Assert.That( - STuple.FromArray(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }, 1, 2).ToSlice().ToString(), - Is.EqualTo("<15>{<14>") - ); - - Assert.That( - STuple.FromEnumerable(new List { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }).ToSlice().ToString(), - Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") - ); - - } - - [Test] - public void Test_STuple_EncodeKey() - { - Assert.That( - STuple.EncodeKey("hello world").ToString(), - Is.EqualTo("<02>hello world<00>") - ); - - Assert.That( - STuple.EncodeKey("hello", "world").ToString(), - Is.EqualTo("<02>hello<00><02>world<00>") - ); - - Assert.That( - STuple.EncodeKey("hello world", 123).ToString(), - Is.EqualTo("<02>hello world<00><15>{") - ); - - Assert.That( - STuple.EncodeKey("hello world", 1234, -1234).ToString(), - Is.EqualTo("<02>hello world<00><16><04><12>-") - ); - - Assert.That( - STuple.EncodeKey("hello world", 123, false).ToString(), - Is.EqualTo("<02>hello world<00><15>{<14>") - ); - - Assert.That( - STuple.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToString(), - Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") - ); - } - - [Test] - public void Test_STuple_Unpack() - { - - var packed = STuple.Create("hello world").ToSlice(); - Log(packed); - - var tuple = STuple.Unpack(packed); - Assert.That(tuple, Is.Not.Null); - Log(tuple); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple.Get(0), Is.EqualTo("hello world")); - - packed = STuple.Create("hello world", 123).ToSlice(); - Log(packed); - - tuple = STuple.Unpack(packed); - Assert.That(tuple, Is.Not.Null); - Log(tuple); - Assert.That(tuple.Count, Is.EqualTo(2)); - Assert.That(tuple.Get(0), Is.EqualTo("hello world")); - Assert.That(tuple.Get(1), Is.EqualTo(123)); - - packed = STuple.Create(1, 256, 257, 65536, int.MaxValue, long.MaxValue).ToSlice(); - Log(packed); - - tuple = STuple.Unpack(packed); - Assert.That(tuple, Is.Not.Null); - Assert.That(tuple.Count, Is.EqualTo(6)); - Assert.That(tuple.Get(0), Is.EqualTo(1)); - Assert.That(tuple.Get(1), Is.EqualTo(256)); - Assert.That(tuple.Get(2), Is.EqualTo(257), ((SlicedTuple)tuple).GetSlice(2).ToString()); - Assert.That(tuple.Get(3), Is.EqualTo(65536)); - Assert.That(tuple.Get(4), Is.EqualTo(int.MaxValue)); - Assert.That(tuple.Get(5), Is.EqualTo(long.MaxValue)); - - packed = STuple.Create(-1, -256, -257, -65536, int.MinValue, long.MinValue).ToSlice(); - Log(packed); - - tuple = STuple.Unpack(packed); - Assert.That(tuple, Is.Not.Null); - Assert.That(tuple, Is.InstanceOf()); - Log(tuple); - Assert.That(tuple.Count, Is.EqualTo(6)); - Assert.That(tuple.Get(0), Is.EqualTo(-1)); - Assert.That(tuple.Get(1), Is.EqualTo(-256)); - Assert.That(tuple.Get(2), Is.EqualTo(-257), "Slice is " + ((SlicedTuple)tuple).GetSlice(2).ToString()); - Assert.That(tuple.Get(3), Is.EqualTo(-65536)); - Assert.That(tuple.Get(4), Is.EqualTo(int.MinValue)); - Assert.That(tuple.Get(5), Is.EqualTo(long.MinValue)); - } - - [Test] - public void Test_STuple_CreateBoxed() - { - ITuple tuple; - - tuple = STuple.CreateBoxed(default(object)); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.Null); - - tuple = STuple.CreateBoxed(1); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.EqualTo(1)); - - tuple = STuple.CreateBoxed(1L); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.EqualTo(1L)); - - tuple = STuple.CreateBoxed(false); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.EqualTo(false)); - - tuple = STuple.CreateBoxed("hello"); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.EqualTo("hello")); - - tuple = STuple.CreateBoxed(new byte[] { 1, 2, 3 }); - Assert.That(tuple.Count, Is.EqualTo(1)); - Assert.That(tuple[0], Is.EqualTo(Slice.Create(new byte[] { 1, 2, 3 }))); - } - - [Test] - public void Test_STuple_EncodeKey_Boxed() - { - Slice slice; - - slice = STuple.EncodeKey(default(object)); - Assert.That(slice.ToString(), Is.EqualTo("<00>")); - - slice = STuple.EncodeKey(1); - Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - - slice = STuple.EncodeKey(1L); - Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - - slice = STuple.EncodeKey(1U); - Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - - slice = STuple.EncodeKey(1UL); - Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); - - slice = STuple.EncodeKey(false); - Assert.That(slice.ToString(), Is.EqualTo("<14>")); - - slice = STuple.EncodeKey(new byte[] { 4, 5, 6 }); - Assert.That(slice.ToString(), Is.EqualTo("<01><04><05><06><00>")); - - slice = STuple.EncodeKey("hello"); - Assert.That(slice.ToString(), Is.EqualTo("<02>hello<00>")); - } - - [Test] - public void Test_STuple_Numbers_Are_Sorted_Lexicographically() - { - // pick two numbers 'x' and 'y' at random, and check that the order of 'x' compared to 'y' is the same as 'pack(tuple(x))' compared to 'pack(tuple(y))' - - // ie: ensure that x.CompareTo(y) always has the same sign as Tuple(x).CompareTo(Tuple(y)) - - const int N = 1 * 1000 * 1000; - var rnd = new Random(); - var sw = Stopwatch.StartNew(); - - for (int i = 0; i < N; i++) - { - int x = rnd.Next() - 1073741824; - int y = x; - while (y == x) - { - y = rnd.Next() - 1073741824; - } - - var t1 = STuple.Create(x).ToSlice(); - var t2 = STuple.Create(y).ToSlice(); - - int dint = x.CompareTo(y); - int dtup = t1.CompareTo(t2); - - if (dtup == 0) Assert.Fail("Tuples for x={0} and y={1} should not have the same packed value", x, y); - - // compare signs - if (Math.Sign(dint) != Math.Sign(dtup)) - { - Assert.Fail("Tuples for x={0} and y={1} are not sorted properly ({2} / {3}): t(x)='{4}' and t(y)='{5}'", x, y, dint, dtup, t1.ToString(), t2.ToString()); - } - } - sw.Stop(); - Log("Checked {0:N0} tuples in {1:N1} ms", N, sw.ElapsedMilliseconds); - - } - - [Test] - public void Test_STuple_Serialize_ITupleFormattable() - { - // types that implement ITupleFormattable should be packed by calling ToTuple() and then packing the returned tuple - - Slice packed; - - packed = TuplePacker.Serialize(new Thing { Foo = 123, Bar = "hello" }); - Assert.That(packed.ToString(), Is.EqualTo("<03><15>{<02>hello<00><00>")); - - packed = TuplePacker.Serialize(new Thing()); - Assert.That(packed.ToString(), Is.EqualTo("<03><14><00><00>")); - - packed = TuplePacker.Serialize(default(Thing)); - Assert.That(packed.ToString(), Is.EqualTo("<00>")); - - } - - [Test] - public void Test_STuple_Deserialize_ITupleFormattable() - { - Slice slice; - Thing thing; - - slice = Slice.Unescape("<03><16><01><02>world<00><00>"); - thing = TuplePackers.DeserializeFormattable(slice); - Assert.That(thing, Is.Not.Null); - Assert.That(thing.Foo, Is.EqualTo(456)); - Assert.That(thing.Bar, Is.EqualTo("world")); - - slice = Slice.Unescape("<03><14><00><00>"); - thing = TuplePackers.DeserializeFormattable(slice); - Assert.That(thing, Is.Not.Null); - Assert.That(thing.Foo, Is.EqualTo(0)); - Assert.That(thing.Bar, Is.EqualTo(null)); - - slice = Slice.Unescape("<00>"); - thing = TuplePackers.DeserializeFormattable(slice); - Assert.That(thing, Is.Null); - } - - [Test] - public void Test_STuple_BatchPack_Of_Tuples() - { - Slice[] slices; - var tuples = new ITuple[] { - STuple.Create("hello"), - STuple.Create(123), - STuple.Create(false), - STuple.Create("world", 456, true) - }; - - // array version - slices = STuple.Pack(tuples); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(tuples.Length)); - Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); - - // IEnumerable version that is passed an array - slices = STuple.Pack((IEnumerable)tuples); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(tuples.Length)); - Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); - - // IEnumerable version but with a "real" enumerable - slices = STuple.Pack(tuples.Select(t => t)); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(tuples.Length)); - Assert.That(slices, Is.EqualTo(tuples.Select(t => t.ToSlice()).ToArray())); - } - - [Test] - public void Test_STuple_EncodeKeys_Of_T() - { - Slice[] slices; - - #region PackRange(Tuple, ...) - - var tuple = STuple.Create("hello"); - int[] items = new int[] { 1, 2, 3, 123, -1, int.MaxValue }; - - // array version - slices = STuple.EncodePrefixedKeys(tuple, items); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - - // IEnumerable version that is passed an array - slices = STuple.EncodePrefixedKeys(tuple, (IEnumerable)items); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - - // IEnumerable version but with a "real" enumerable - slices = STuple.EncodePrefixedKeys(tuple, items.Select(t => t)); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - - #endregion - - #region PackRange(Slice, ...) - - string[] words = new string[] { "hello", "world", "très bien", "断トツ", "abc\0def", null, String.Empty }; - - var merged = STuple.EncodePrefixedKeys(Slice.FromByte(42), words); - Assert.That(merged, Is.Not.Null); - Assert.That(merged.Length, Is.EqualTo(words.Length)); - - for (int i = 0; i < words.Length; i++) - { - var expected = Slice.FromByte(42) + STuple.EncodeKey(words[i]); - Assert.That(merged[i], Is.EqualTo(expected)); - - Assert.That(merged[i].Array, Is.SameAs(merged[0].Array), "All slices should be stored in the same buffer"); - if (i > 0) Assert.That(merged[i].Offset, Is.EqualTo(merged[i - 1].Offset + merged[i - 1].Count), "All slices should be contiguous"); - } - - // corner cases - Assert.That(() => STuple.EncodePrefixedKeys(Slice.Empty, default(int[])), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); - Assert.That(() => STuple.EncodePrefixedKeys(Slice.Empty, default(IEnumerable)), Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); - - #endregion - } - - [Test] - public void Test_STuple_EncodeKeys_Boxed() - { - Slice[] slices; - var tuple = STuple.Create("hello"); - object[] items = new object[] { "world", 123, false, Guid.NewGuid(), long.MinValue }; - - // array version - slices = STuple.EncodePrefixedKeys(tuple, items); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - - // IEnumerable version that is passed an array - slices = STuple.EncodePrefixedKeys(tuple, (IEnumerable)items); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - - // IEnumerable version but with a "real" enumerable - slices = STuple.EncodePrefixedKeys(tuple, items.Select(t => t)); - Assert.That(slices, Is.Not.Null); - Assert.That(slices.Length, Is.EqualTo(items.Length)); - Assert.That(slices, Is.EqualTo(items.Select(x => tuple.Append(x).ToSlice()).ToArray())); - } - - #endregion - - #region TupleParser - - private static string Clean(string value) - { - var sb = new StringBuilder(value.Length + 8); - foreach (var c in value) - { - if (c < ' ') sb.Append("\\x").Append(((int)c).ToString("x2")); else sb.Append(c); - } - return sb.ToString(); - } - - private static void PerformWriterTest(TuplePackers.Encoder action, T value, string expectedResult, string message = null) - { - var writer = new TupleWriter(); - action(ref writer, value); - - Assert.That( - writer.Output.ToSlice().ToHexaString(' '), - Is.EqualTo(expectedResult), - message != null ? "Value {0} ({1}) was not properly packed: {2}" : "Value {0} ({1}) was not properly packed", value == null ? "" : value is string ? Clean(value as string) : value.ToString(), (value == null ? "null" : value.GetType().Name), message); - } - - [Test] - public void Test_TupleParser_WriteInt64() - { - var test = new TuplePackers.Encoder(TupleParser.WriteInt64); - - PerformWriterTest(test, 0L, "14"); - - PerformWriterTest(test, 1L, "15 01"); - PerformWriterTest(test, 2L, "15 02"); - PerformWriterTest(test, 123L, "15 7B"); - PerformWriterTest(test, 255L, "15 FF"); - PerformWriterTest(test, 256L, "16 01 00"); - PerformWriterTest(test, 257L, "16 01 01"); - PerformWriterTest(test, 65535L, "16 FF FF"); - PerformWriterTest(test, 65536L, "17 01 00 00"); - PerformWriterTest(test, 65537L, "17 01 00 01"); - - PerformWriterTest(test, -1L, "13 FE"); - PerformWriterTest(test, -123L, "13 84"); - PerformWriterTest(test, -255L, "13 00"); - PerformWriterTest(test, -256L, "12 FE FF"); - PerformWriterTest(test, -65535L, "12 00 00"); - PerformWriterTest(test, -65536L, "11 FE FF FF"); - - PerformWriterTest(test, (1L << 24) - 1, "17 FF FF FF"); - PerformWriterTest(test, 1L << 24, "18 01 00 00 00"); - - PerformWriterTest(test, (1L << 32) - 1, "18 FF FF FF FF"); - PerformWriterTest(test, (1L << 32), "19 01 00 00 00 00"); - - PerformWriterTest(test, long.MaxValue, "1C 7F FF FF FF FF FF FF FF"); - PerformWriterTest(test, long.MinValue, "0C 7F FF FF FF FF FF FF FF"); - PerformWriterTest(test, long.MaxValue - 1, "1C 7F FF FF FF FF FF FF FE"); - PerformWriterTest(test, long.MinValue + 1, "0C 80 00 00 00 00 00 00 00"); - - } - - [Test] - public void Test_TupleParser_WriteInt64_Respects_Ordering() - { - var list = new List>(); - - Action test = (x) => - { - var writer = new TupleWriter(); - TupleParser.WriteInt64(ref writer, x); - var res = new KeyValuePair(x, writer.Output.ToSlice()); - list.Add(res); - Log("{0,20} : {0:x16} {1}", res.Key, res.Value.ToString()); - }; - - // We can't test 2^64 values, be we are interested at what happens around powers of two (were size can change) - - // negatives - for (int i = 63; i >= 3; i--) - { - long x = -(1L << i); - - if (i < 63) - { - test(x - 2); - test(x - 1); - } - test(x + 0); - test(x + 1); - test(x + 2); - } - - test(-2); - test(0); - test(+1); - test(+2); - - // positives - for (int i = 3; i <= 63; i++) - { - long x = (1L << i); - - test(x - 2); - test(x - 1); - if (i < 63) - { - test(x + 0); - test(x + 1); - test(x + 2); - } - } - - KeyValuePair previous = list[0]; - for (int i = 1; i < list.Count; i++) - { - KeyValuePair current = list[i]; - - Assert.That(current.Key, Is.GreaterThan(previous.Key)); - Assert.That(current.Value, Is.GreaterThan(previous.Value), "Expect {0} > {1}", current.Key, previous.Key); - - previous = current; - } - } - - [Test] - public void Test_TupleParser_WriteUInt64() - { - var test = new TuplePackers.Encoder(TupleParser.WriteUInt64); - - PerformWriterTest(test, 0UL, "14"); - - PerformWriterTest(test, 1UL, "15 01"); - PerformWriterTest(test, 123UL, "15 7B"); - PerformWriterTest(test, 255UL, "15 FF"); - PerformWriterTest(test, 256UL, "16 01 00"); - PerformWriterTest(test, 257UL, "16 01 01"); - PerformWriterTest(test, 65535UL, "16 FF FF"); - PerformWriterTest(test, 65536UL, "17 01 00 00"); - PerformWriterTest(test, 65537UL, "17 01 00 01"); - - PerformWriterTest(test, (1UL << 24) - 1, "17 FF FF FF"); - PerformWriterTest(test, 1UL << 24, "18 01 00 00 00"); - - PerformWriterTest(test, (1UL << 32) - 1, "18 FF FF FF FF"); - PerformWriterTest(test, (1UL << 32), "19 01 00 00 00 00"); - - PerformWriterTest(test, ulong.MaxValue, "1C FF FF FF FF FF FF FF FF"); - PerformWriterTest(test, ulong.MaxValue-1, "1C FF FF FF FF FF FF FF FE"); - - } - - [Test] - public void Test_TupleParser_WriteUInt64_Respects_Ordering() - { - var list = new List>(); - - Action test = (x) => - { - var writer = new TupleWriter(); - TupleParser.WriteUInt64(ref writer, x); - var res = new KeyValuePair(x, writer.Output.ToSlice()); - list.Add(res); -#if DEBUG - Log("{0,20} : {0:x16} {1}", res.Key, res.Value); -#endif - }; - - // We can't test 2^64 values, be we are interested at what happens around powers of two (were size can change) - - test(0); - test(1); - - // positives - for (int i = 3; i <= 63; i++) - { - ulong x = (1UL << i); - - test(x - 2); - test(x - 1); - test(x + 0); - test(x + 1); - test(x + 2); - } - test(ulong.MaxValue - 2); - test(ulong.MaxValue - 1); - test(ulong.MaxValue); - - KeyValuePair previous = list[0]; - for (int i = 1; i < list.Count; i++) - { - KeyValuePair current = list[i]; - - Assert.That(current.Key, Is.GreaterThan(previous.Key)); - Assert.That(current.Value, Is.GreaterThan(previous.Value), "Expect {0} > {1}", current.Key, previous.Key); - - previous = current; - } - } - - [Test] - public void Test_TupleParser_WriteString() - { - string s; - var test = new TuplePackers.Encoder(TupleParser.WriteString); - Func encodeSimple = (value) => "02 " + Slice.Create(Encoding.UTF8.GetBytes(value)).ToHexaString(' ') + " 00"; - Func encodeWithZeroes = (value) => "02 " + Slice.Create(Encoding.UTF8.GetBytes(value)).ToHexaString(' ').Replace("00", "00 FF") + " 00"; - - PerformWriterTest(test, null, "00"); - PerformWriterTest(test, String.Empty, "02 00"); - PerformWriterTest(test, "A", "02 41 00"); - PerformWriterTest(test, "\x80", "02 C2 80 00"); - PerformWriterTest(test, "\xFF", "02 C3 BF 00"); - PerformWriterTest(test, "\xFFFE", "02 EF BF BE 00"); // UTF-8 BOM - - PerformWriterTest(test, "ASCII", "02 41 53 43 49 49 00"); - PerformWriterTest(test, "héllø le 世界", "02 68 C3 A9 6C 6C C3 B8 20 6C 65 20 E4 B8 96 E7 95 8C 00"); - - // Must escape '\0' contained in the string as '\x00\xFF' - PerformWriterTest(test, "\0", "02 00 FF 00"); - PerformWriterTest(test, "A\0", "02 41 00 FF 00"); - PerformWriterTest(test, "\0A", "02 00 FF 41 00"); - PerformWriterTest(test, "A\0\0A", "02 41 00 FF 00 FF 41 00"); - PerformWriterTest(test, "A\0B\0\xFF", "02 41 00 FF 42 00 FF C3 BF 00"); - - // random human text samples - - s = "This is a long string that has more than 1024 chars to force the encoder to use multiple chunks, and with some random UNICODE at the end so that it can not be optimized as ASCII-only." + new string('A', 1024) + "ಠ_ಠ"; - PerformWriterTest(test, s, encodeSimple(s)); - - s = "String of exactly 1024 ASCII chars !"; s += new string('A', 1024 - s.Length); - PerformWriterTest(test, s, encodeSimple(s)); - - s = "Ceci est une chaîne de texte qui contient des caractères UNICODE supérieurs à 0x7F mais inférieurs à 0x800"; // n'est-il pas ? - PerformWriterTest(test, s, encodeSimple(s)); - - s = "色は匂へど 散りぬるを 我が世誰そ 常ならむ 有為の奥山 今日越えて 浅き夢見じ 酔ひもせず"; // iroha! - PerformWriterTest(test, s, encodeSimple(s)); - - s = "String that ends with funny UTF-32 chars like \xDFFF\xDBFF"; // supposed to be 0x10FFFF encoded in UTF-16 - PerformWriterTest(test, s, encodeSimple(s)); - - // strings with random non-zero UNICODE chars - var rnd = new Random(); - for (int k = 0; k < 100; k++) - { - int size = 1 + rnd.Next(10000); - var chars = new char[size]; - for (int i = 0; i < chars.Length; i++) - { - // 1..0xFFFF - switch (rnd.Next(3)) - { - case 0: chars[i] = (char)rnd.Next(1, 0x80); break; - case 1: chars[i] = (char)rnd.Next(0x80, 0x800); break; - case 2: chars[i] = (char)rnd.Next(0x800, 0xFFFF); break; - } - } - s = new string(chars); - PerformWriterTest(test, s, encodeSimple(s), "Random string with non-zero unicode chars (from 1 to 0xFFFF)"); - } - - // random strings with zeroes - for (int k = 0; k < 100; k++) - { - int size = 1 + rnd.Next(10000); - var chars = new char[size]; - for (int i = 0; i < chars.Length; i++) - { - switch(rnd.Next(4)) - { - case 0: chars[i] = '\0'; break; - case 1: chars[i] = (char)rnd.Next(1, 0x80); break; - case 2: chars[i] = (char)rnd.Next(0x80, 0x800); break; - case 3: chars[i] = (char)rnd.Next(0x800, 0xFFFF); break; - } - } - s = new string(chars); - PerformWriterTest(test, s, encodeWithZeroes(s), "Random string with zeros "); - } - - } - - [Test] - public void Test_TupleParser_WriteChar() - { - var test = new TuplePackers.Encoder(TupleParser.WriteChar); - - // 1 bytes - PerformWriterTest(test, 'A', "02 41 00", "Unicode chars in the ASCII table take only one byte in UTF-8"); - PerformWriterTest(test, '\0', "02 00 FF 00", "\\0 must be escaped as 00 FF"); - PerformWriterTest(test, '\x7F', "02 7F 00", "1..127 take ony 1 bytes"); - // 2 bytes - PerformWriterTest(test, '\x80', "02 C2 80 00", "128 needs 2 bytes"); - PerformWriterTest(test, '\xFF', "02 C3 BF 00", "ASCII chars above 128 take at least 2 bytes in UTF-8"); - PerformWriterTest(test, 'é', "02 C3 A9 00", "0x00E9, LATIN SMALL LETTER E WITH ACUTE"); - PerformWriterTest(test, 'ø', "02 C3 B8 00", "0x00F8, LATIN SMALL LETTER O WITH STROKE"); - PerformWriterTest(test, '\x07FF', "02 DF BF 00"); - // 3 bytes - PerformWriterTest(test, '\x0800', "02 E0 A0 80 00", "0x800 takes at least 3 bytes"); - PerformWriterTest(test, 'ಠ', "02 E0 B2 A0 00", "KANNADA LETTER TTHA"); - PerformWriterTest(test, '世', "02 E4 B8 96 00", "0x4E16, CJK Ideograph"); - PerformWriterTest(test, '界', "02 E7 95 8C 00", "0x754C, CJK Ideoghaph"); - PerformWriterTest(test, '\xFFFE', "02 EF BF BE 00", "Unicode BOM becomes EF BF BE in UTF-8"); - PerformWriterTest(test, '\xFFFF', "02 EF BF BF 00", "Maximum UTF-16 character"); - - // check all the unicode chars - for (int i = 1; i <= 65535; i++) - { - char c = (char)i; - var writer = new TupleWriter(); - TupleParser.WriteChar(ref writer, c); - string s = new string(c, 1); - Assert.That(writer.Output.ToSlice().ToString(), Is.EqualTo("<02>" + Slice.Create(Encoding.UTF8.GetBytes(s)).ToString() + "<00>"), "{0} '{1}'", i, c); - } - } - - #endregion - - #region Equality / Comparison - - private static void AssertEquality(ITuple x, ITuple y) - { - Assert.That(x.Equals(y), Is.True, "x.Equals(y)"); - Assert.That(x.Equals((object)y), Is.True, "x.Equals((object)y)"); - Assert.That(y.Equals(x), Is.True, "y.Equals(x)"); - Assert.That(y.Equals((object)x), Is.True, "y.Equals((object)y"); - } - - private static void AssertInequality(ITuple x, ITuple y) - { - Assert.That(x.Equals(y), Is.False, "!x.Equals(y)"); - Assert.That(x.Equals((object)y), Is.False, "!x.Equals((object)y)"); - Assert.That(y.Equals(x), Is.False, "!y.Equals(x)"); - Assert.That(y.Equals((object)x), Is.False, "!y.Equals((object)y"); - } - - [Test] - public void Test_STuple_Equals() - { - var t1 = STuple.Create(1, 2); - // self equality - AssertEquality(t1, t1); - - var t2 = STuple.Create(1, 2); - // same type equality - AssertEquality(t1, t2); - - var t3 = STuple.Create(new object[] { 1, 2 }); - // other tuple type equality - AssertEquality(t1, t3); - - var t4 = STuple.Create(1).Append(2); - // multi step - AssertEquality(t1, t4); - } - - [Test] - public void Test_STuple_Similar() - { - var t1 = STuple.Create(1, 2); - var t2 = STuple.Create((long)1, (short)2); - var t3 = STuple.Create("1", "2"); - var t4 = STuple.Create(new object[] { 1, 2L }); - var t5 = STuple.Unpack(Slice.Unescape("<02>1<00><15><02>")); - - AssertEquality(t1, t1); - AssertEquality(t1, t2); - AssertEquality(t1, t3); - AssertEquality(t1, t4); - AssertEquality(t1, t5); - AssertEquality(t2, t2); - AssertEquality(t2, t3); - AssertEquality(t2, t4); - AssertEquality(t2, t5); - AssertEquality(t3, t3); - AssertEquality(t3, t4); - AssertEquality(t3, t5); - AssertEquality(t4, t4); - AssertEquality(t4, t5); - AssertEquality(t5, t5); - } - - [Test] - public void Test_STuple_Not_Equal() - { - var t1 = STuple.Create(1, 2); - - var x1 = STuple.Create(2, 1); - var x2 = STuple.Create("11", "22"); - var x3 = STuple.Create(1, 2, 3); - var x4 = STuple.Unpack(Slice.Unescape("<15><01>")); - - AssertInequality(t1, x1); - AssertInequality(t1, x2); - AssertInequality(t1, x3); - AssertInequality(t1, x4); - - AssertInequality(x1, x2); - AssertInequality(x1, x3); - AssertInequality(x1, x4); - AssertInequality(x2, x3); - AssertInequality(x2, x4); - AssertInequality(x3, x4); - } - - [Test] - public void Test_STuple_Substring_Equality() - { - var x = STuple.FromArray(new [] { "A", "C" }); - var y = STuple.FromArray(new[] { "A", "B", "C" }); - - Assert.That(x.Substring(0, 1), Is.EqualTo(y.Substring(0, 1))); - Assert.That(x.Substring(1, 1), Is.EqualTo(y.Substring(2, 1))); - - var aa = STuple.Create("A"); - var bb = STuple.Create("A"); - Assert.That(aa == bb, Is.True); - - var a = x.Substring(0, 1); - var b = y.Substring(0, 1); - Assert.That(a.Equals((ITuple)b), Is.True); - Assert.That(a.Equals((object)b), Is.True); - Assert.That(object.Equals(a, b), Is.True); - Assert.That(STuple.Equals(a, b), Is.True); - Assert.That(STuple.Equivalent(a, b), Is.True); - - // this is very unfortunate, but 'a == b' does NOT work because ITuple is an interface, and there is no known way to make it work :( - //Assert.That(a == b, Is.True); - } - - [Test] - public void Test_STuple_String_AutoCast() - { - // 'a' ~= "A" - AssertEquality(STuple.Create("A"), STuple.Create('A')); - AssertInequality(STuple.Create("A"), STuple.Create('B')); - AssertInequality(STuple.Create("A"), STuple.Create('a')); - - // ASCII ~= Unicode - AssertEquality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("ABC"))); - AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("DEF"))); - AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromAscii("abc"))); - - // 'a' ~= ASCII 'a' - AssertEquality(STuple.Create(Slice.FromAscii("A")), STuple.Create('A')); - AssertInequality(STuple.Create(Slice.FromAscii("A")), STuple.Create('B')); - AssertInequality(STuple.Create(Slice.FromAscii("A")), STuple.Create('a')); - } - - #endregion - - #region Formatters - - [Test] - public void Test_Default_TupleFormatter_For_Common_Types() - { - - // common simple types - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - - // corner cases - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - - // ITupleFormattable types - Assert.That(TupleFormatter.Default, Is.InstanceOf>()); - } - - [Test] - public void Test_Format_Common_Types() - { - Assert.That(TupleFormatter.Default.ToTuple(123), Is.EqualTo(STuple.Create(123))); - Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(123)), Is.EqualTo(123)); - - Assert.That(TupleFormatter.Default.ToTuple(true), Is.EqualTo(STuple.Create(true))); - Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(true)), Is.True); - - Assert.That(TupleFormatter.Default.ToTuple("hello"), Is.EqualTo(STuple.Create("hello"))); - Assert.That(TupleFormatter.Default.FromTuple(STuple.Create("hello")), Is.EqualTo("hello")); - - var t = STuple.Create(new object[] { "hello", 123, false }); - Assert.That(TupleFormatter.Default.ToTuple(t), Is.SameAs(t)); - Assert.That(TupleFormatter.Default.FromTuple(t), Is.SameAs(t)); - - var thing = new Thing { Foo = 123, Bar = "hello" }; - Assert.That(TupleFormatter.Default.ToTuple(thing), Is.EqualTo(STuple.Create(123, "hello"))); - - var thing2 = TupleFormatter.Default.FromTuple(STuple.Create(456, "world")); - Assert.That(thing2, Is.Not.Null); - Assert.That(thing2.Foo, Is.EqualTo(456)); - Assert.That(thing2.Bar, Is.EqualTo("world")); - - } - - [Test] - public void Test_Create_Appender_Formatter() - { - // create an appender formatter that will always add the values after the same prefix - - var fmtr = TupleFormatter.CreateAppender(STuple.Create("hello", "world")); - Assert.That(fmtr, Is.InstanceOf>()); - - Assert.That(fmtr.ToTuple(123), Is.EqualTo(STuple.Create("hello", "world", 123))); - Assert.That(fmtr.ToTuple(456), Is.EqualTo(STuple.Create("hello", "world", 456))); - Assert.That(fmtr.ToTuple(-1), Is.EqualTo(STuple.Create("hello", "world", -1))); - - Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", 42)), Is.EqualTo(42)); - Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", -1)), Is.EqualTo(-1)); - - Assert.That(() => fmtr.FromTuple(null), Throws.InstanceOf()); - Assert.That(() => fmtr.FromTuple(STuple.Empty), Throws.InstanceOf()); - Assert.That(() => fmtr.FromTuple(STuple.Create("hello", "world", 42, 77)), Throws.InstanceOf(), "Too many values"); - Assert.That(() => fmtr.FromTuple(STuple.Create("hello_world", 42)), Throws.InstanceOf(), "not enough values"); - Assert.That(() => fmtr.FromTuple(STuple.Create("world", "hello", "42")), Throws.InstanceOf(), "incorrect type"); - Assert.That(() => fmtr.FromTuple(STuple.Create(42)), Throws.InstanceOf(), "missing prefix"); - Assert.That(() => fmtr.FromTuple(STuple.Create("extra", "hello", "world", 42)), Throws.InstanceOf(), "prefix must match exactly"); - Assert.That(() => fmtr.FromTuple(STuple.Create("Hello", "World", 42)), Throws.InstanceOf(), "case sensitive"); - } - - #endregion - - #region Bench.... - - [Test] - public void Bench_STuple_Unpack_Random() - { - const int N = 100 * 1000; - - Slice FUNKY_ASCII = Slice.FromAscii("bonjour\x00le\x00\xFFmonde"); - string FUNKY_STRING = "hello\x00world"; - string UNICODE_STRING = "héllø 世界"; - - Console.Write("Creating {0:N0} random tuples", N); - var tuples = new List(N); - var rnd = new Random(777); - var guids = Enumerable.Range(0, 10).Select(_ => Guid.NewGuid()).ToArray(); - var uuid128s = Enumerable.Range(0, 10).Select(_ => Uuid128.NewUuid()).ToArray(); - var uuid64s = Enumerable.Range(0, 10).Select(_ => Uuid64.NewUuid()).ToArray(); - var fuzz = new byte[1024 + 1000]; rnd.NextBytes(fuzz); - var sw = Stopwatch.StartNew(); - for (int i = 0; i < N; i++) - { - ITuple tuple = STuple.Empty; - int s = 1 + (int)Math.Sqrt(rnd.Next(128)); - if (i % (N / 100) == 0) Console.Write("."); - for (int j = 0; j < s; j++) - { - switch (rnd.Next(17)) - { - case 0: tuple = tuple.Append(rnd.Next(255)); break; - case 1: tuple = tuple.Append(-1 - rnd.Next(255)); break; - case 2: tuple = tuple.Append(256 + rnd.Next(65536 - 256)); break; - case 3: tuple = tuple.Append(rnd.Next(int.MaxValue)); break; - case 4: tuple = tuple.Append((rnd.Next(int.MaxValue) << 32) | rnd.Next(int.MaxValue)); break; - case 5: tuple = tuple.Append(new string('A', 1 + rnd.Next(16))); break; - case 6: tuple = tuple.Append(new string('B', 8 + (int)Math.Sqrt(rnd.Next(1024)))); break; - case 7: tuple = tuple.Append(UNICODE_STRING); break; - case 8: tuple = tuple.Append(FUNKY_STRING); break; - case 9: tuple = tuple.Append(FUNKY_ASCII); break; - case 10: tuple = tuple.Append(guids[rnd.Next(10)]); break; - case 11: tuple = tuple.Append(uuid128s[rnd.Next(10)]); break; - case 12: tuple = tuple.Append(uuid64s[rnd.Next(10)]); break; - case 13: tuple = tuple.Append(Slice.Create(fuzz, rnd.Next(1000), 1 + (int)Math.Sqrt(rnd.Next(1024)))); break; - case 14: tuple = tuple.Append(default(string)); break; - case 15: tuple = tuple.Append("hello"); break; - case 16: tuple = tuple.Append(rnd.Next(2) == 0); break; - } - } - tuples.Add(tuple); - } - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(" > {0:N0} items", tuples.Sum(x => x.Count)); - Log(" > {0}", tuples[42]); - Log(); - - Console.Write("Packing tuples..."); - sw.Restart(); - var slices = STuple.Pack(tuples); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); - Log(" > {0:N0} bytes", slices.Sum(x => x.Count)); - Log(" > {0}", slices[42]); - Log(); - - Console.Write("Unpacking tuples..."); - sw.Restart(); - var unpacked = slices.Select(slice => STuple.Unpack(slice)).ToList(); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); - Log(" > {0}", unpacked[42]); - Log(); - - Console.Write("Comparing ..."); - sw.Restart(); - tuples.Zip(unpacked, (x, y) => x.Equals(y)).All(b => b); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(); - - Console.Write("Tuples.ToString ..."); - sw.Restart(); - var strings = tuples.Select(x => x.ToString()).ToList(); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(" > {0:N0} chars", strings.Sum(x => x.Length)); - Log(" > {0}", strings[42]); - Log(); - - Console.Write("Unpacked.ToString ..."); - sw.Restart(); - strings = unpacked.Select(x => x.ToString()).ToList(); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - Log(" > {0:N0} chars", strings.Sum(x => x.Length)); - Log(" > {0}", strings[42]); - Log(); - - Console.Write("Memoizing ..."); - sw.Restart(); - var memoized = tuples.Select(x => x.Memoize()).ToList(); - sw.Stop(); - Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); - } - - #endregion - - private class Thing : ITupleFormattable - { - public Thing() - { } - - public int Foo { get; set; } - public string Bar { get; set; } - - ITuple ITupleFormattable.ToTuple() - { - return STuple.Create(this.Foo, this.Bar); - } - - void ITupleFormattable.FromTuple(ITuple tuple) - { - this.Foo = tuple.Get(0); - this.Bar = tuple.Get(1); - } - } - - } - - -} diff --git a/FoundationDB.Tests/Layers/VectorFacts.cs b/FoundationDB.Tests/Layers/VectorFacts.cs index 4d8aaf839..8fd0d3392 100644 --- a/FoundationDB.Tests/Layers/VectorFacts.cs +++ b/FoundationDB.Tests/Layers/VectorFacts.cs @@ -28,17 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Collections.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; - using NUnit.Framework; using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Linq; using System.Text; - using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] [Obsolete] @@ -82,7 +77,7 @@ public async Task Test_Vector_Fast() Console.WriteLine("> Pop empty: " + await vector.PopAsync(tr)); await PrintVector(vector, tr); - await vector.PushAsync(tr, Slice.FromAscii("foo")); + await vector.PushAsync(tr, Slice.FromString("foo")); Console.WriteLine("> Pop size 1: " + await vector.PopAsync(tr)); await PrintVector(vector, tr); @@ -98,12 +93,12 @@ public async Task Test_Vector_Fast() Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); Console.WriteLine("Settings values"); - vector.Set(tr, 0, Slice.FromAscii("Portez")); - vector.Set(tr, 1, Slice.FromAscii("ce vieux")); - vector.Set(tr, 2, Slice.FromAscii("whisky")); - vector.Set(tr, 3, Slice.FromAscii("au juge")); - vector.Set(tr, 4, Slice.FromAscii("blond qui")); - vector.Set(tr, 5, Slice.FromAscii("fume")); + vector.Set(tr, 0, Slice.FromString("Portez")); + vector.Set(tr, 1, Slice.FromString("ce vieux")); + vector.Set(tr, 2, Slice.FromString("whisky")); + vector.Set(tr, 3, Slice.FromString("au juge")); + vector.Set(tr, 4, Slice.FromString("blond qui")); + vector.Set(tr, 5, Slice.FromString("fume")); await PrintVector(vector, tr); Console.WriteLine("FRONT"); @@ -149,7 +144,7 @@ public async Task Test_Vector_Fast() Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); Console.WriteLine("> Adding 'word' to index 10, resize to 25"); - vector.Set(tr, 10, Slice.FromAscii("word")); + vector.Set(tr, 10, Slice.FromString("word")); await vector.ResizeAsync(tr, 25); await PrintVector(vector, tr); Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); @@ -175,7 +170,7 @@ private static async Task PrintVector(FdbVector vector, IFdbReadOnlyTransa await tr.GetRange(vector.Subspace.Keys.ToRange()).ForEachAsync((kvp) => { if (!first) sb.Append(", "); else first = false; - sb.Append(vector.Subspace.Keys.DecodeLast(kvp.Key) + ":" + kvp.Value.ToAsciiOrHexaString()); + sb.Append($"{vector.Subspace.Keys.DecodeLast(kvp.Key)}:{kvp.Value:P}"); }); Console.WriteLine("> Vector: (" + sb.ToString() + ")"); diff --git a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs similarity index 99% rename from FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs rename to FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs index 6c7585046..30d65cc4c 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Linq.Tests { using System; @@ -38,14 +37,14 @@ namespace FoundationDB.Linq.Tests using System.Threading.Tasks; using Doxense; using Doxense.Async; + using Doxense.Collections.Tuples; using Doxense.Linq; using Doxense.Linq.Async.Iterators; using FoundationDB.Client.Tests; - using FoundationDB.Layers.Tuples; using NUnit.Framework; [TestFixture] - public class FdbAsyncEnumerableFacts : FdbTest + public class AsyncEnumerableFacts : FdbTest { [Test] diff --git a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs b/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs similarity index 97% rename from FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs rename to FoundationDB.Tests/Linq/AsyncQueryableFacts.cs index 65d550649..742d19573 100644 --- a/FoundationDB.Tests/Linq/FdbAsyncQueryableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs @@ -28,21 +28,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Linq.Tests { + using System; + using System.Collections.Generic; + using System.Threading.Tasks; using FoundationDB.Client; using FoundationDB.Client.Tests; using FoundationDB.Layers.Indexing; - using FoundationDB.Layers.Tuples; using FoundationDB.Linq.Expressions; using FoundationDB.Linq.Providers; using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Linq.Expressions; - using System.Threading; - using System.Threading.Tasks; [TestFixture] - public class FdbAsyncQueryableFacts : FdbTest + public class AsyncQueryableFacts : FdbTest { [Test] diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index c8f07de2f..b483e6118 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -26,24 +26,23 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB.Linq.Expressions.Tests + namespace FoundationDB.Linq.Expressions.Tests { using System; using System.Collections.Generic; using System.Linq.Expressions; + using Doxense.Collections.Tuples; using Doxense.Linq; using FoundationDB.Client; using FoundationDB.Layers.Indexing; - using FoundationDB.Layers.Tuples; using NUnit.Framework; [TestFixture] public class FdbQueryExpressionFacts { - private FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(STuple.EncodeKey("Foos", 1))); - private FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(STuple.EncodeKey("Foos", 2))); + private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(TuPack.EncodeKey("Foos", 1))); + private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(TuPack.EncodeKey("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() @@ -94,7 +93,7 @@ public void Test_FdbQueryIndexLookupExpression_From_Lambda() public void Test_FdbQueryRangeExpression() { var expr = FdbQueryExpressions.Range( - STuple.Create("Foo").ToSelectorPair() + KeySelectorPair.Create(TuPack.ToRange(STuple.Create("Foo"))) ); Console.WriteLine(expr); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 930a7aea7..b51369a16 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -25,7 +25,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. */ #endregion - + namespace FoundationDB.Client.Tests { using System; @@ -33,10 +33,10 @@ namespace FoundationDB.Client.Tests using System.Diagnostics; using System.Linq; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Linq; using Doxense.Linq.Async.Iterators; using FoundationDB.Layers.Directories; - using FoundationDB.Layers.Tuples; using NUnit.Framework; [TestFixture] @@ -501,7 +501,7 @@ public async Task Test_Can_MergeSort() { for (int i = 0; i < N; i++) { - tr.Set(lists[k].Keys.Encode((i * K) + k), STuple.EncodeKey(k, i)); + tr.Set(lists[k].Keys.Encode((i * K) + k), TuPack.EncodeKey(k, i)); } await tr.CommitAsync(); } @@ -526,8 +526,8 @@ public async Task Test_Can_MergeSort() for (int i = 0; i < K * N; i++) { - Assert.That(location.ExtractKey(results[i].Key), Is.EqualTo(STuple.EncodeKey(i % K, i))); - Assert.That(results[i].Value, Is.EqualTo(STuple.EncodeKey(i % K, i / K))); + Assert.That(location.ExtractKey(results[i].Key), Is.EqualTo(TuPack.EncodeKey(i % K, i))); + Assert.That(results[i].Value, Is.EqualTo(TuPack.EncodeKey(i % K, i / K))); } } } @@ -566,7 +566,7 @@ public async Task Test_Range_Intersect() for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); - var value = STuple.EncodeKey(k, i); + var value = TuPack.EncodeKey(k, i); //Console.WriteLine("> " + key + " = " + value); tr.Set(key, value); } @@ -638,7 +638,7 @@ public async Task Test_Range_Except() for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); - var value = STuple.EncodeKey(k, i); + var value = TuPack.EncodeKey(k, i); //Console.WriteLine("> " + key + " = " + value); tr.Set(key, value); } @@ -711,7 +711,7 @@ await db.WriteAsync((tr) => { var query = tr.Except( new[] { locItems.Keys.ToRange(), locProcessed.Keys.ToRange() }, - (kv) => STuple.Unpack(kv.Key).Substring(-2), // note: keys come from any of the two ranges, so we must only keep the last 2 elements of the tuple + (kv) => TuPack.Unpack(kv.Key).Substring(-2), // note: keys come from any of the two ranges, so we must only keep the last 2 elements of the tuple TupleComparisons.Composite() // compares t[0] as a string, and t[1] as an int ); diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 84b36f0c6..f4e7b6dbd 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -28,13 +28,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Layers.Tuples.Tests { + using System; + using Doxense.Collections.Tuples; using FoundationDB.Client; - using FoundationDB.Layers.Tuples; + using FoundationDB.Client.Tests; using NUnit.Framework; - using System; [TestFixture] - public class SubspaceFacts + public class SubspaceFacts : FdbTest { [Test] @@ -54,7 +55,7 @@ public void Test_Empty_Subspace_Is_Empty() [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = KeySubspace.CreateDynamic(Slice.Create(new byte[] { 42, 255, 0, 127 })); + var subspace = KeySubspace.CreateDynamic(new byte[] { 42, 255, 0, 127 }.AsSlice()); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -62,12 +63,12 @@ public void Test_Subspace_With_Binary_Prefix() // concat(Slice) should append the slice to the binary prefix directly Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("*<00><7F><04><03><02><01>")); - Assert.That(subspace.ConcatKey(Slice.FromAscii("hello")).ToString(), Is.EqualTo("*<00><7F>hello")); + Assert.That(subspace.ConcatKey(Slice.FromStringAscii("hello")).ToString(), Is.EqualTo("*<00><7F>hello")); // pack(...) should use tuple serialization Assert.That(subspace.Keys.Encode(123).ToString(), Is.EqualTo("*<00><7F><15>{")); Assert.That(subspace.Keys.Encode("hello").ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); - Assert.That(subspace.Keys.Encode(Slice.FromAscii("world")).ToString(), Is.EqualTo("*<00><7F><01>world<00>")); + Assert.That(subspace.Keys.Encode(Slice.FromStringAscii("world")).ToString(), Is.EqualTo("*<00><7F><01>world<00>")); Assert.That(subspace.Keys.Pack(STuple.Create("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); // if we derive a tuple from this subspace, it should keep the binary prefix when converted to a key @@ -77,7 +78,7 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(t.Get(0), Is.EqualTo("world")); Assert.That(t.Get(1), Is.EqualTo(123)); Assert.That(t.Get(2), Is.False); - var k = t.ToSlice(); + var k = TuPack.Pack(t); Assert.That(k.ToString(), Is.EqualTo("*<00><7F><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -126,7 +127,7 @@ public void Test_Subspace_With_Tuple_Prefix() // concat(Slice) should append the slice to the tuple prefix directly Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("<02>hello<00><04><03><02><01>")); - Assert.That(subspace.ConcatKey(Slice.FromAscii("world")).ToString(), Is.EqualTo("<02>hello<00>world")); + Assert.That(subspace.ConcatKey(Slice.FromStringAscii("world")).ToString(), Is.EqualTo("<02>hello<00>world")); // pack(...) should use tuple serialization Assert.That(subspace.Keys.Encode(123).ToString(), Is.EqualTo("<02>hello<00><15>{")); @@ -140,7 +141,7 @@ public void Test_Subspace_With_Tuple_Prefix() Assert.That(t.Get(1), Is.EqualTo(123)); Assert.That(t.Get(2), Is.False); // but ToSlice() should include the prefix - var k = t.ToSlice(); + var k = TuPack.Pack(t); Assert.That(k.ToString(), Is.EqualTo("<02>hello<00><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -170,7 +171,7 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() Assert.That(key.ToString(), Is.EqualTo("<04><03><02><01>")); // create another child - var grandChild = child.Partition[Slice.FromAscii("hello")]; + var grandChild = child.Partition[Slice.FromStringAscii("hello")]; Assert.That(grandChild, Is.Not.Null); Assert.That(grandChild.GetPrefix().ToString(), Is.EqualTo("hello")); @@ -197,15 +198,15 @@ public void Test_Subspace_Partitioning_With_Tuple_Suffix() // create a tuple from this child subspace var tuple = child.Keys.Append(123); Assert.That(tuple, Is.Not.Null); - Assert.That(tuple.ToSlice().ToString(), Is.EqualTo("<02>hca<00><15>{")); + Assert.That(TuPack.Pack(tuple).ToString(), Is.EqualTo("<02>hca<00><15>{")); // derive another tuple from this one var t1 = tuple.Append(false); - Assert.That(t1.ToSlice().ToString(), Is.EqualTo("<02>hca<00><15>{<14>")); + Assert.That(TuPack.Pack(t1).ToString(), Is.EqualTo("<02>hca<00><15>{<14>")); // check that we could also create the same tuple starting from the parent subspace var t2 = parent.Keys.Append("hca", 123, false); - Assert.That(t2.ToSlice(), Is.EqualTo(t1.ToSlice())); + Assert.That(TuPack.Pack(t2), Is.EqualTo(TuPack.Pack(t1))); // cornercase Assert.That(child.Partition[STuple.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index 75fa63bf8..009ea39a7 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client.Tests using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; internal static class TestHelpers { @@ -43,7 +44,7 @@ internal static class TestHelpers public static readonly string TestClusterFile = null; public static readonly string TestDbName = "DB"; - public static readonly Slice TestGlobalPrefix = Slice.FromAscii("T"); + public static readonly Slice TestGlobalPrefix = Slice.FromStringAscii("T"); public static readonly string[] TestPartition = new string[] { "Tests", Environment.MachineName }; public static readonly int DefaultTimeout = 15 * 1000; @@ -112,7 +113,7 @@ await tr try { // attemps decoding it as a tuple - keyDump = key.ToTuple().ToString(); + keyDump = TuPack.Unpack(key).ToString(); } catch (Exception) { diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 41af6be75..6e8e545b7 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -// ReSharper disable ConsiderUsingConfigureAwait namespace FoundationDB.Client.Tests { using NUnit.Framework; @@ -154,8 +153,8 @@ public async Task Test_Creating_Concurrent_Transactions_Are_Independent() } finally { - if (tr1 != null) tr1.Dispose(); - if (tr2 != null) tr2.Dispose(); + tr1?.Dispose(); + tr2?.Dispose(); } } @@ -353,7 +352,7 @@ public async Task Test_Write_And_Read_Simple_Keys() { tr.Set(location.Keys.Encode("hello"), Slice.FromString("World!")); tr.Set(location.Keys.Encode("timestamp"), Slice.FromInt64(ticks)); - tr.Set(location.Keys.Encode("blob"), Slice.Create(new byte[] { 42, 123, 7 })); + tr.Set(location.Keys.Encode("blob"), new byte[] { 42, 123, 7 }.AsSlice()); await tr.CommitAsync(); @@ -959,7 +958,7 @@ public async Task Test_GetRange_With_Concurrent_Change_Should_Conflict() await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); }, this.Cancellation); // we will read the first key from [0, 100), expected 50 @@ -977,7 +976,7 @@ await db.WriteAsync((tr) => // 42 < 50 > conflict !!! using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 42), Slice.FromAscii("forty-two")); + tr2.Set(loc.Keys.Encode("foo", 42), Slice.FromString("forty-two")); await tr2.CommitAsync(); } @@ -994,7 +993,7 @@ await db.WriteAsync((tr) => await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); }, this.Cancellation); using (var tr1 = db.BeginTransaction(this.Cancellation)) @@ -1008,7 +1007,7 @@ await db.WriteAsync((tr) => // 77 > 50 => no conflict using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromAscii("docm")); + tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromString("docm")); await tr2.CommitAsync(); } @@ -1033,7 +1032,7 @@ public async Task Test_GetKey_With_Concurrent_Change_Should_Conflict() await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); }, this.Cancellation); // we will ask for the first key from >= 0, expecting 50, but if another transaction inserts something BEFORE 50, our key selector would have returned a different result, causing a conflict @@ -1047,7 +1046,7 @@ await db.WriteAsync((tr) => // 42 < 50 => conflict !!! using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 42), Slice.FromAscii("forty-two")); + tr2.Set(loc.Keys.Encode("foo", 42), Slice.FromString("forty-two")); await tr2.CommitAsync(); } @@ -1062,7 +1061,7 @@ await db.WriteAsync((tr) => await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); }, this.Cancellation); using (var tr1 = db.BeginTransaction(this.Cancellation)) @@ -1074,7 +1073,7 @@ await db.WriteAsync((tr) => // 77 > 50 => no conflict using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromAscii("docm")); + tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromString("docm")); await tr2.CommitAsync(); } @@ -1090,8 +1089,8 @@ await db.WriteAsync((tr) => await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); - tr.Set(loc.Keys.Encode("foo", 100), Slice.FromAscii("one hundred")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); + tr.Set(loc.Keys.Encode("foo", 100), Slice.FromString("one hundred")); }, this.Cancellation); using (var tr1 = db.BeginTransaction(this.Cancellation)) @@ -1103,7 +1102,7 @@ await db.WriteAsync((tr) => // 77 between 50 and 100 => conflict !!! using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromAscii("docm")); + tr2.Set(loc.Keys.Encode("foo", 77), Slice.FromString("docm")); await tr2.CommitAsync(); } @@ -1119,8 +1118,8 @@ await db.WriteAsync((tr) => await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); - tr.Set(loc.Keys.Encode("foo", 100), Slice.FromAscii("one hundred")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); + tr.Set(loc.Keys.Encode("foo", 100), Slice.FromString("one hundred")); }, this.Cancellation); using (var tr1 = db.BeginTransaction(this.Cancellation)) @@ -1132,7 +1131,7 @@ await db.WriteAsync((tr) => // another transaction changes the VALUE of 50 and 100 (but does not change the fact that they exist nor add keys in between) using (var tr2 = db.BeginTransaction(this.Cancellation)) { - tr2.Set(loc.Keys.Encode("foo", 100), Slice.FromAscii("cent")); + tr2.Set(loc.Keys.Encode("foo", 100), Slice.FromString("cent")); await tr2.CommitAsync(); } @@ -1148,8 +1147,8 @@ await db.WriteAsync((tr) => await db.WriteAsync((tr) => { tr.ClearRange(loc); - tr.Set(loc.Keys.Encode("foo", 50), Slice.FromAscii("fifty")); - tr.Set(loc.Keys.Encode("foo", 100), Slice.FromAscii("one hundred")); + tr.Set(loc.Keys.Encode("foo", 50), Slice.FromString("fifty")); + tr.Set(loc.Keys.Encode("foo", 100), Slice.FromString("one hundred")); }, this.Cancellation); using (var tr1 = db.BeginTransaction(this.Cancellation)) @@ -1255,15 +1254,18 @@ public async Task Test_Read_Isolation_From_Writes() // - Regular reads see the writes made by the transaction itself, but not the writes made by other transactions that committed in between // - Snapshot reads never see the writes made since the transaction read version, including the writes made by the transaction itself + //Fdb.Start(200); // <-- the test passes + //Fdb.Start(300); // <-- the test fails + using (var db = await OpenTestPartitionAsync()) { var location = db.Partition.ByKey("test"); await db.ClearRangeAsync(location, this.Cancellation); - var a = location.Keys.Encode("A"); - var b = location.Keys.Encode("B"); - var c = location.Keys.Encode("C"); - var d = location.Keys.Encode("D"); + var A = location.Keys.Encode("A"); + var B = location.Keys.Encode("B"); + var C = location.Keys.Encode("C"); + var D = location.Keys.Encode("D"); // Reads (before and after): // - A and B will use regular reads @@ -1274,39 +1276,38 @@ public async Task Test_Read_Isolation_From_Writes() await db.WriteAsync((tr) => { - tr.Set(a, Slice.FromString("a")); - tr.Set(b, Slice.FromString("b")); - tr.Set(c, Slice.FromString("c")); - tr.Set(d, Slice.FromString("d")); + tr.Set(A, Slice.FromString("a")); + tr.Set(B, Slice.FromString("b")); + tr.Set(C, Slice.FromString("c")); + tr.Set(D, Slice.FromString("d")); }, this.Cancellation); + Log("Initial db state:"); + await DumpSubspace(db, location); + using (var tr = db.BeginTransaction(this.Cancellation)) { - var aval = await tr.GetAsync(a); - var bval = await tr.GetAsync(b); - var cval = await tr.Snapshot.GetAsync(c); - var dval = await tr.Snapshot.GetAsync(d); - Assert.That(aval.ToUnicode(), Is.EqualTo("a")); - Assert.That(bval.ToUnicode(), Is.EqualTo("b")); - Assert.That(cval.ToUnicode(), Is.EqualTo("c")); - Assert.That(dval.ToUnicode(), Is.EqualTo("d")); + // check initial state + Assert.That((await tr.GetAsync(A)).ToStringUtf8(), Is.EqualTo("a")); + Assert.That((await tr.GetAsync(B)).ToStringUtf8(), Is.EqualTo("b")); + Assert.That((await tr.Snapshot.GetAsync(C)).ToStringUtf8(), Is.EqualTo("c")); + Assert.That((await tr.Snapshot.GetAsync(D)).ToStringUtf8(), Is.EqualTo("d")); - tr.Set(a, Slice.FromString("aa")); - tr.Set(c, Slice.FromString("cc")); + // mutate (not yet comitted) + tr.Set(A, Slice.FromString("aa")); + tr.Set(C, Slice.FromString("cc")); await db.WriteAsync((tr2) => { - tr2.Set(b, Slice.FromString("bb")); - tr2.Set(d, Slice.FromString("dd")); + tr2.Set(B, Slice.FromString("bb")); + tr2.Set(D, Slice.FromString("dd")); }, this.Cancellation); - aval = await tr.GetAsync(a); - bval = await tr.GetAsync(b); - cval = await tr.Snapshot.GetAsync(c); - dval = await tr.Snapshot.GetAsync(d); - Assert.That(aval.ToUnicode(), Is.EqualTo("aa"), "The transaction own writes should change the value of regular reads"); - Assert.That(bval.ToUnicode(), Is.EqualTo("b"), "Other transaction writes should not change the value of regular reads"); - Assert.That(cval.ToUnicode(), Is.EqualTo("c"), "The transaction own writes should not change the value of snapshot reads"); - Assert.That(dval.ToUnicode(), Is.EqualTo("d"), "Other transaction writes should not change the value of snapshot reads"); + // check what the transaction sees + Assert.That((await tr.GetAsync(A)).ToStringUtf8(), Is.EqualTo("aa"), "The transaction own writes should change the value of regular reads"); + Assert.That((await tr.GetAsync(B)).ToStringUtf8(), Is.EqualTo("b"), "Other transaction writes should not change the value of regular reads"); + //FAIL: test fails here because we read "CC" ?? + Assert.That((await tr.Snapshot.GetAsync(C)).ToStringUtf8(), Is.EqualTo("c"), "The transaction own writes should not change the value of snapshot reads"); + Assert.That((await tr.Snapshot.GetAsync(D)).ToStringUtf8(), Is.EqualTo("d"), "Other transaction writes should not change the value of snapshot reads"); //note: committing here would conflict } @@ -1443,7 +1444,7 @@ public async Task Test_Has_Access_To_System_Keys() // should fail if access to system keys has not been requested await TestHelpers.AssertThrowsFdbErrorAsync( - () => tr.GetRange(Slice.FromAscii("\xFF"), Slice.FromAscii("\xFF\xFF"), new FdbRangeOptions { Limit = 10 }).ToListAsync(), + () => tr.GetRange(Slice.FromByteString("\xFF"), Slice.FromByteString("\xFF\xFF"), new FdbRangeOptions { Limit = 10 }).ToListAsync(), FdbError.KeyOutsideLegalRange, "Should not have access to system keys by default" ); @@ -1451,7 +1452,7 @@ await TestHelpers.AssertThrowsFdbErrorAsync( // should succeed once system access has been requested tr.WithReadAccessToSystemKeys(); - var keys = await tr.GetRange(Slice.FromAscii("\xFF"), Slice.FromAscii("\xFF\xFF"), new FdbRangeOptions { Limit = 10 }).ToListAsync(); + var keys = await tr.GetRange(Slice.FromByteString("\xFF"), Slice.FromByteString("\xFF\xFF"), new FdbRangeOptions { Limit = 10 }).ToListAsync(); Assert.That(keys, Is.Not.Null); } @@ -1826,7 +1827,7 @@ public async Task Test_Can_Get_Boundary_Keys() // the datacenter id seems to be at offset 40 var dataCenterId = key.Value.Substring(40, 16).ToHexaString(); - Log("- {0} : ({1}) {2}", key.Key.ToHexaString(), key.Value.Count, key.Value.ToAsciiOrHexaString()); + Log("- {0:X} : ({1}) {2:P}", key.Key, key.Value.Count, key.Value); Log(" > node = {0}", nodeId); Log(" > machine = {0}", machineId); Log(" > datacenter = {0}", dataCenterId); diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index f3a55e4d3..8a0497fe1 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -141,6 +141,7 @@ public async Task Test_Transactionals_Retries_On_Transient_Errors() } [Test][Category("LongRunning")] + [Ignore("This tests a bug in an old version (v2.0.7) and takes a long time to run!")] public async Task Test_Transactionals_Retries_Do_Not_Leak_When_Reading_Too_Much() { // we have a transaction that tries to read too much data, and will always take more than 5 seconds to execute diff --git a/FoundationDB.Tests/Utils/ConversionFacts.cs b/FoundationDB.Tests/Utils/ConversionFacts.cs index b964b8f8d..3a660a38e 100644 --- a/FoundationDB.Tests/Utils/ConversionFacts.cs +++ b/FoundationDB.Tests/Utils/ConversionFacts.cs @@ -28,17 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Converters.Tests { - using FoundationDB.Client; - using FoundationDB.Client.Utils; - using NUnit.Framework; using System; - using System.Collections.Generic; - using System.Globalization; - using System.Linq; - using System.Text; + using Doxense.Runtime.Converters; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] - public class ConversionFacts + public class ConversionFacts : FdbTest { [Test] diff --git a/FoundationDB.Tests/Utils/FdbConvertersFacts.cs b/FoundationDB.Tests/Utils/FdbConvertersFacts.cs deleted file mode 100644 index 66da1b771..000000000 --- a/FoundationDB.Tests/Utils/FdbConvertersFacts.cs +++ /dev/null @@ -1,174 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client.Converters.Tests -{ - using NUnit.Framework; - using System; - - [TestFixture] - public class FdbConvertersFacts - { - - [Test] - public void Test_Can_Convert_Numbers_To_Bool() - { - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0), Is.False); - Assert.That(FdbConverters.Convert(0.0f), Is.False); - Assert.That(FdbConverters.Convert(float.NaN), Is.False); - Assert.That(FdbConverters.Convert(0.0d), Is.False); - Assert.That(FdbConverters.Convert(double.NaN), Is.False); - - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123), Is.True); - Assert.That(FdbConverters.Convert(123.0f), Is.True); - Assert.That(FdbConverters.Convert(123.0d), Is.True); - } - - [Test] - public void Test_Can_Convert_Numbers_To_Int32() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_UInt32() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123U)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123U)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_Int64() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123L)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123L)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_UInt64() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123UL)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123UL)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_Single() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123f)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123f)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_Double() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo(123d)); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo(123d)); - } - - [Test] - public void Test_Can_Convert_Numbers_To_String() - { - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123.0f), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123.4f), Is.EqualTo("123.4")); - Assert.That(FdbConverters.Convert(123.0d), Is.EqualTo("123")); - Assert.That(FdbConverters.Convert(123.4d), Is.EqualTo("123.4")); - } - - } -} diff --git a/FoundationDB.Tests/Utils/SliceComparerFacts.cs b/FoundationDB.Tests/Utils/SliceComparerFacts.cs index b54c7fcb5..9ee9084e8 100644 --- a/FoundationDB.Tests/Utils/SliceComparerFacts.cs +++ b/FoundationDB.Tests/Utils/SliceComparerFacts.cs @@ -28,10 +28,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Client; - using NUnit.Framework; using System; using System.Text; + using NUnit.Framework; [TestFixture] public class SliceComparerFacts : FdbTest @@ -42,9 +41,9 @@ public class SliceComparerFacts : FdbTest [Test] public void Test_SliceComparer_Equals() { - var cmp = SliceComparer.Default; + var cmp = Slice.Comparer.Default; Assert.That(cmp, Is.Not.Null); - Assert.That(SliceComparer.Default, Is.SameAs(cmp)); + Assert.That(Slice.Comparer.Default, Is.SameAs(cmp)); Assert.That(cmp.Equals(Slice.Nil, Slice.Nil), Is.True); Assert.That(cmp.Equals(Slice.Empty, Slice.Empty), Is.True); @@ -52,26 +51,26 @@ public void Test_SliceComparer_Equals() Assert.That(cmp.Equals(Slice.Empty, Slice.Nil), Is.False); Assert.That(cmp.Equals(Slice.FromByte(42), Slice.FromByte(42)), Is.True); - Assert.That(cmp.Equals(Slice.FromByte(42), Slice.Create(new byte[] { 42 })), Is.True); + Assert.That(cmp.Equals(Slice.FromByte(42), new byte[] { 42 }.AsSlice()), Is.True); Assert.That(cmp.Equals(Slice.FromByte(42), Slice.FromByte(77)), Is.False); - Assert.That(cmp.Equals(Slice.Create(new byte[] { 65, 66, 67 }), Slice.FromString("ABC")), Is.True); - Assert.That(cmp.Equals(Slice.Create(new byte[] { 65, 66, 67, 68 }), Slice.FromString("ABC")), Is.False); + Assert.That(cmp.Equals(new byte[] { 65, 66, 67 }.AsSlice(), Slice.FromString("ABC")), Is.True); + Assert.That(cmp.Equals(new byte[] { 65, 66, 67, 68 }.AsSlice(), Slice.FromString("ABC")), Is.False); var buf1 = Encoding.ASCII.GetBytes("ABBAABA"); var buf2 = Encoding.ASCII.GetBytes("ABBAABA"); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 2), Slice.Create(buf1, 0, 2)), Is.True); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 2), Slice.Create(buf1, 0, 3)), Is.False); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 2), Slice.Create(buf1, 4, 2)), Is.True); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 3), Slice.Create(buf1, 4, 3)), Is.False); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 2), Slice.Create(buf2, 4, 2)), Is.True); - Assert.That(cmp.Equals(Slice.Create(buf1, 0, 3), Slice.Create(buf2, 4, 3)), Is.False); + Assert.That(cmp.Equals(buf1.AsSlice(0, 2), buf1.AsSlice(0, 2)), Is.True); + Assert.That(cmp.Equals(buf1.AsSlice(0, 2), buf1.AsSlice(0, 3)), Is.False); + Assert.That(cmp.Equals(buf1.AsSlice(0, 2), buf1.AsSlice(4, 2)), Is.True); + Assert.That(cmp.Equals(buf1.AsSlice(0, 3), buf1.AsSlice(4, 3)), Is.False); + Assert.That(cmp.Equals(buf1.AsSlice(0, 2), buf2.AsSlice(4, 2)), Is.True); + Assert.That(cmp.Equals(buf1.AsSlice(0, 3), buf2.AsSlice(4, 3)), Is.False); } [Test] public void Test_SliceComparer_GetHashCode_Should_Return_Same_As_Slice() { - var cmp = SliceComparer.Default; + var cmp = Slice.Comparer.Default; Assert.That(cmp, Is.Not.Null); Assert.That(cmp.GetHashCode(Slice.Nil), Is.EqualTo(Slice.Nil.GetHashCode())); @@ -89,7 +88,7 @@ public void Test_SliceComparer_GetHashCode_Should_Return_Same_As_Slice() [Test] public void Test_SliceComparer_Compare() { - var cmp = SliceComparer.Default; + var cmp = Slice.Comparer.Default; Assert.That(cmp, Is.Not.Null); Assert.That(cmp.Compare(Slice.Nil, Slice.Nil), Is.EqualTo(0)); diff --git a/FoundationDB.Tests/Utils/SliceFacts.cs b/FoundationDB.Tests/Utils/SliceFacts.cs index fef7ed960..e15eaf731 100644 --- a/FoundationDB.Tests/Utils/SliceFacts.cs +++ b/FoundationDB.Tests/Utils/SliceFacts.cs @@ -26,16 +26,18 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Tests +namespace Doxense.Memory.Tests { - using FoundationDB.Client; + //README:IMPORTANT! This source file is expected to be stored as UTF-8! If the encoding is changed, some tests below may fail because they rely on specific code points! + using NUnit.Framework; using System; using System.Collections.Generic; + using System.Diagnostics.CodeAnalysis; using System.IO; using System.Linq; using System.Text; - using System.Threading.Tasks; + using FoundationDB.Client.Tests; [TestFixture] public class SliceFacts : FdbTest @@ -57,9 +59,10 @@ public void Test_Slice_Nil() Assert.That(Slice.Nil.IsPresent, Is.False); Assert.That(Slice.Nil.GetBytes(), Is.Null); - Assert.That(Slice.Nil.ToAscii(), Is.Null); + Assert.That(Slice.Nil.GetBytesOrEmpty(), Is.Not.Null.And.Length.EqualTo(0)); + Assert.That(Slice.Nil.ToByteString(), Is.Null); Assert.That(Slice.Nil.ToUnicode(), Is.Null); - Assert.That(Slice.Nil.ToAsciiOrHexaString(), Is.EqualTo(String.Empty)); + Assert.That(Slice.Nil.PrettyPrint(), Is.EqualTo(String.Empty)); } [Test] @@ -70,7 +73,7 @@ public void Test_Slice_Empty() Assert.That(Slice.Empty.Count, Is.EqualTo(0)); Assert.That(Slice.Empty.Offset, Is.EqualTo(0)); Assert.That(Slice.Empty.Array, Is.Not.Null); - Assert.That(Slice.Empty.Array.Length, Is.EqualTo(0)); + Assert.That(Slice.Empty.Array.Length, Is.GreaterThan(0), "The backing array for Slice.Empty should not be empty, in order to work properly with the fixed() operator!"); Assert.That(Slice.Empty.IsNull, Is.False); Assert.That(Slice.Empty.HasValue, Is.True); @@ -78,16 +81,17 @@ public void Test_Slice_Empty() Assert.That(Slice.Empty.IsNullOrEmpty, Is.True); Assert.That(Slice.Empty.IsPresent, Is.False); - Assert.That(Slice.Empty.GetBytes(), Is.EqualTo(new byte[0])); - Assert.That(Slice.Empty.ToAscii(), Is.EqualTo(String.Empty)); + Assert.That(Slice.Empty.GetBytes(), Is.Not.Null.And.Length.EqualTo(0)); + Assert.That(Slice.Empty.GetBytesOrEmpty(), Is.Not.Null.And.Length.EqualTo(0)); + Assert.That(Slice.Empty.ToByteString(), Is.EqualTo(String.Empty)); Assert.That(Slice.Empty.ToUnicode(), Is.EqualTo(String.Empty)); - Assert.That(Slice.Empty.ToAsciiOrHexaString(), Is.EqualTo("''")); + Assert.That(Slice.Empty.PrettyPrint(), Is.EqualTo("''")); } [Test] public void Test_Slice_With_Content() { - Slice slice = Slice.FromAscii("ABC"); + Slice slice = Slice.FromStringAscii("ABC"); Assert.That(slice.Count, Is.EqualTo(3)); Assert.That(slice.Offset, Is.EqualTo(0)); @@ -101,9 +105,10 @@ public void Test_Slice_With_Content() Assert.That(slice.IsPresent, Is.True); Assert.That(slice.GetBytes(), Is.EqualTo(new byte[3] { 65, 66, 67 })); - Assert.That(slice.ToAscii(), Is.EqualTo("ABC")); + Assert.That(slice.GetBytesOrEmpty(), Is.EqualTo(new byte[3] { 65, 66, 67 })); + Assert.That(slice.ToByteString(), Is.EqualTo("ABC")); Assert.That(slice.ToUnicode(), Is.EqualTo("ABC")); - Assert.That(slice.ToAsciiOrHexaString(), Is.EqualTo("'ABC'")); + Assert.That(slice.PrettyPrint(), Is.EqualTo("'ABC'")); } [Test] @@ -118,13 +123,13 @@ public void Test_Slice_Create_With_Capacity() [Test] public void Test_Slice_Create_With_Byte_Array() { - Assert.That(Slice.Create(default(byte[])).GetBytes(), Is.EqualTo(null)); - Assert.That(Slice.Create(new byte[0]).GetBytes(), Is.EqualTo(new byte[0])); - Assert.That(Slice.Create(new byte[] { 1, 2, 3 }).GetBytes(), Is.EqualTo(new byte[] { 1, 2, 3 })); + Assert.That(default(byte[]).AsSlice().GetBytes(), Is.EqualTo(null)); + Assert.That(new byte[0].AsSlice().GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(new byte[] { 1, 2, 3 }.AsSlice().GetBytes(), Is.EqualTo(new byte[] { 1, 2, 3 })); // the array return by GetBytes() should not be the same array that was passed to Create ! byte[] tmp = Guid.NewGuid().ToByteArray(); // create a 16-byte array - var slice = Slice.Create(tmp); + var slice = tmp.AsSlice(); Assert.That(slice.Array, Is.SameAs(tmp)); Assert.That(slice.Offset, Is.EqualTo(0)); Assert.That(slice.Count, Is.EqualTo(tmp.Length)); @@ -133,7 +138,7 @@ public void Test_Slice_Create_With_Byte_Array() Assert.That(slice.GetBytes(), Is.Not.SameAs(tmp)); // create from a slice of the array - slice = Slice.Create(tmp, 4, 7); + slice = tmp.AsSlice(4, 7); Assert.That(slice.Array, Is.SameAs(tmp)); Assert.That(slice.Offset, Is.EqualTo(4)); Assert.That(slice.Count, Is.EqualTo(7)); @@ -141,42 +146,42 @@ public void Test_Slice_Create_With_Byte_Array() Array.Copy(tmp, 4, buf, 0, 7); Assert.That(slice.GetBytes(), Is.EqualTo(buf)); - Assert.That(Slice.Create(default(byte[])), Is.EqualTo(Slice.Nil)); - Assert.That(Slice.Create(new byte[0]), Is.EqualTo(Slice.Empty)); + Assert.That(default(byte[]).AsSlice(), Is.EqualTo(Slice.Nil)); + Assert.That(new byte[0].AsSlice(), Is.EqualTo(Slice.Empty)); } [Test] public void Test_Slice_Create_Validates_Arguments() { // null array only allowed with offset=0 and count=0 - Assert.That(() => Slice.Create(null, 0, 1), Throws.InstanceOf()); - Assert.That(() => Slice.Create(null, 1, 0), Throws.InstanceOf()); - Assert.That(() => Slice.Create(null, 1, 1), Throws.InstanceOf()); + // ReSharper disable AssignNullToNotNullAttribute + Assert.That(() => default(byte[]).AsSlice(0, 1), Throws.InstanceOf()); + Assert.That(() => default(byte[]).AsSlice(1, 0), Throws.Nothing, "Count 0 ignores offset"); + Assert.That(() => default(byte[]).AsSlice(1, 1), Throws.InstanceOf()); + // ReSharper restore AssignNullToNotNullAttribute // empty array only allowed with offset=0 and count=0 - Assert.That(() => Slice.Create(new byte[0], 0, 1), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[0], 1, 0), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[0], 1, 1), Throws.InstanceOf()); + Assert.That(() => new byte[0].AsSlice(0, 1), Throws.InstanceOf()); + Assert.That(() => new byte[0].AsSlice(1, 0), Throws.Nothing, "Count 0 ignores offset"); + Assert.That(() => new byte[0].AsSlice(1, 1), Throws.InstanceOf()); // last item must fit in the buffer - Assert.That(() => Slice.Create(new byte[3], 0, 4), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[3], 1, 3), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[3], 3, 1), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(0, 4), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(1, 3), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(3, 1), Throws.InstanceOf()); // negative arguments - //TODO: should we allow negative indexing where Slice.Create(..., -1, 1) would mean "the last byte" ? - Assert.That(() => Slice.Create(new byte[3], -1, 1), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[3], 0, -1), Throws.InstanceOf()); - Assert.That(() => Slice.Create(new byte[3], -1, -1), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(-1, 1), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(0, -1), Throws.InstanceOf()); + Assert.That(() => new byte[3].AsSlice(-1, -1), Throws.InstanceOf()); } [Test] public void Test_Slice_Create_With_ArraySegment() { - Slice slice; byte[] tmp = Guid.NewGuid().ToByteArray(); - slice = Slice.Create(new ArraySegment(tmp)); + Slice slice = new ArraySegment(tmp).AsSlice(); Assert.That(slice.Array, Is.SameAs(tmp)); Assert.That(slice.Offset, Is.EqualTo(0)); Assert.That(slice.Count, Is.EqualTo(tmp.Length)); @@ -184,7 +189,7 @@ public void Test_Slice_Create_With_ArraySegment() Assert.That(slice.GetBytes(), Is.EqualTo(tmp)); Assert.That(slice.GetBytes(), Is.Not.SameAs(tmp)); - slice = Slice.Create(new ArraySegment(tmp, 4, 7)); + slice = new ArraySegment(tmp, 4, 7).AsSlice(); Assert.That(slice.Array, Is.SameAs(tmp)); Assert.That(slice.Offset, Is.EqualTo(4)); Assert.That(slice.Count, Is.EqualTo(7)); @@ -192,17 +197,16 @@ public void Test_Slice_Create_With_ArraySegment() Array.Copy(tmp, 4, buf, 0, 7); Assert.That(slice.GetBytes(), Is.EqualTo(buf)); - Assert.That(Slice.Create(default(ArraySegment)), Is.EqualTo(Slice.Nil)); - Assert.That(Slice.Create(new ArraySegment(new byte[0])), Is.EqualTo(Slice.Empty)); + Assert.That(default(ArraySegment).AsSlice(), Is.EqualTo(Slice.Nil)); + Assert.That(new ArraySegment(new byte[0]).AsSlice(), Is.EqualTo(Slice.Empty)); } [Test] public void Test_Slice_Pseudo_Random() { - Slice slice; var rng = new Random(); - slice = Slice.Random(rng, 16); + Slice slice = Slice.Random(rng, 16); Assert.That(slice.Array, Is.Not.Null); Assert.That(slice.Array.Length, Is.GreaterThanOrEqualTo(16)); Assert.That(slice.Offset, Is.EqualTo(0)); @@ -212,18 +216,18 @@ public void Test_Slice_Pseudo_Random() Assert.That(Slice.Random(rng, 0), Is.EqualTo(Slice.Empty)); - Assert.That(() => Slice.Random(default(System.Random), 16), Throws.InstanceOf()); + // ReSharper disable once AssignNullToNotNullAttribute + Assert.That(() => Slice.Random(default(Random), 16), Throws.InstanceOf()); Assert.That(() => Slice.Random(rng, -1), Throws.InstanceOf()); } [Test] public void Test_Slice_Cryptographic_Random() { - Slice slice; var rng = System.Security.Cryptography.RandomNumberGenerator.Create(); // normal - slice = Slice.Random(rng, 16); + Slice slice = Slice.Random(rng, 16); Assert.That(slice.Array, Is.Not.Null); Assert.That(slice.Array.Length, Is.GreaterThanOrEqualTo(16)); Assert.That(slice.Offset, Is.EqualTo(0)); @@ -242,32 +246,102 @@ public void Test_Slice_Cryptographic_Random() } Assert.That(Slice.Random(rng, 0), Is.EqualTo(Slice.Empty)); + // ReSharper disable once AssignNullToNotNullAttribute Assert.That(() => Slice.Random(default(System.Security.Cryptography.RandomNumberGenerator), 16), Throws.InstanceOf()); Assert.That(() => Slice.Random(rng, -1), Throws.InstanceOf()); } [Test] - public void Test_Slice_FromAscii() + public void Test_Slice_FromStringAscii() + { + Assert.That(Slice.FromStringAscii(default(string)).GetBytes(), Is.Null); + Assert.That(Slice.FromStringAscii(string.Empty).GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(Slice.FromStringAscii("A").GetBytes(), Is.EqualTo(new byte[] { 0x41 })); + Assert.That(Slice.FromStringAscii("AB").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42 })); + Assert.That(Slice.FromStringAscii("ABC").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringAscii("ABCD").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43, 0x44 })); + Assert.That(Slice.FromStringAscii("\xFF/ABC").GetBytes(), Is.EqualTo(new byte[] { 0xFF, 0x2F, 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringAscii("héllô").GetBytes(), Is.EqualTo(new byte[] { (byte)'h', 0xE9, (byte)'l', (byte)'l', 0xF4 })); + Assert.That(Slice.FromStringAscii("This is a test of the emergency encoding system").GetBytes(), Is.EqualTo(Encoding.ASCII.GetBytes("This is a test of the emergency encoding system"))); + + // if the string contains non-ASCII chars, it would be corrupted so FromAscii() should throw + // note: the line below should contain two kanjis. If your editor displays '??' or squares, it is probably not able to display unicode chars properly + Assert.That(() => Slice.FromStringAscii("hello 世界"), Throws.Exception, "String that contains code points >= 0x80 should throw"); + } + + [Test] + public void Test_Slice_ToStringAscii() + { + Assert.That(Slice.Nil.ToStringAscii(), Is.Null); + Assert.That(Slice.Empty.ToStringAscii(), Is.EqualTo(String.Empty)); + Assert.That(new byte[] { 0x41 }.AsSlice().ToStringAscii(), Is.EqualTo("A")); + Assert.That(new byte[] { 0x41, 0x42 }.AsSlice().ToStringAscii(), Is.EqualTo("AB")); + Assert.That(new byte[] { 0x41, 0x42, 0x43 }.AsSlice().ToStringAscii(), Is.EqualTo("ABC")); + Assert.That(new byte[] { 0x41, 0x42, 0x43, 0x44 }.AsSlice().ToStringAscii(), Is.EqualTo("ABCD")); + Assert.That(new byte[] { 0x7F, 0x00, 0x1F }.AsSlice().ToStringAscii(), Is.EqualTo("\x7F\x00\x1F")); + Assert.That(new byte[] { 0x41, 0x42, 0x43, 0x44, 0x45, 0x46 }.AsSlice(2, 3).ToStringAscii(), Is.EqualTo("CDE")); + Assert.That(Encoding.ASCII.GetBytes("This is a test of the emergency encoding system").AsSlice().ToStringAscii(), Is.EqualTo("This is a test of the emergency encoding system")); + + // If the slice contain anything other than 7+bit ASCII, it should throw! + Assert.That(() => new byte[] { 0xFF, 0x41, 0x42, 0x43 }.AsSlice().ToStringAscii(), Throws.Exception, "\\xFF is not valid in 7-bit ASCII strings!"); + Assert.That(() => Encoding.Default.GetBytes("héllô").AsSlice().ToStringAscii(), Throws.Exception, "String that contain code points >= 0x80 should trow"); + Assert.That(() => Encoding.UTF8.GetBytes("héllo 世界").AsSlice().ToStringAscii(), Throws.Exception, "String that contains code points >= 0x80 should throw"); + } + + [Test] + public void Test_Slice_FromByteString() + { + Assert.That(Slice.FromByteString(default(string)).GetBytes(), Is.Null); + Assert.That(Slice.FromByteString(string.Empty).GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(Slice.FromByteString("ABC").GetBytes(), Is.EqualTo(new [] { (byte) 'A', (byte) 'B', (byte) 'C' })); + Assert.That(Slice.FromByteString("\xFF/ABC").GetBytes(), Is.EqualTo(new [] { (byte) 0xFF, (byte) '/', (byte) 'A', (byte) 'B', (byte) 'C' })); + Assert.That(Slice.FromByteString("héllô").GetBytes(), Is.EqualTo(new byte[] { (byte)'h', 0xE9, (byte)'l', (byte)'l', 0xF4 })); + + // if the caller likes to live dangerously and call, then the data should be corrupted + var slice = Slice.FromByteString("hello 世界"); // DON'T EVER DO THAT! + Assume.That('世' & 0xFF, Is.EqualTo(0x16)); + Assume.That('界' & 0xFF, Is.EqualTo(0x4C)); + Assert.That(slice, Is.EqualTo(Slice.Unescape("hello <16><4C>"))); + Assert.That(slice.ToByteString(), Is.EqualTo("hello \x16L"), "non-ASCII chars should be corrupted after decoding"); + Assert.That(slice.Count, Is.EqualTo(8)); + + } + + [Test] + public void Test_Slice_FromStringAnsi() { - Assert.That(Slice.FromAscii(default(string)).GetBytes(), Is.Null); - Assert.That(Slice.FromAscii(String.Empty).GetBytes(), Is.EqualTo(new byte[0])); - Assert.That(Slice.FromAscii("ABC").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringAnsi(default(string)).GetBytes(), Is.Null); + Assert.That(Slice.FromStringAnsi(string.Empty).GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(Slice.FromStringAnsi("ABC").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringAnsi("\xFF/ABC").GetBytes(), Is.EqualTo(new[] { (byte)0xFF, (byte)'/', (byte)'A', (byte)'B', (byte)'C' })); + Assert.That(Slice.FromStringAnsi("héllô").GetBytes(), Is.EqualTo(Encoding.Default.GetBytes("héllô"))); //note: this depends on your OS locale! // if the string contains non-ASCII chars, it will be corrupted // note: the line below should contain two kanjis. If your editor displays '??' or squares, it is probably not able to display unicode chars properly - var slice = Slice.FromAscii("hello 世界"); // 8 'letters' - Assert.That(slice.GetBytes(), Is.EqualTo(Encoding.Default.GetBytes("hello 世界"))); - Assert.That(slice.ToAscii(), Is.EqualTo("hello ??"), "non-ASCII chars should be converted to '?'"); - Assert.That(slice.Count, Is.EqualTo(8)); + var slice = Slice.FromStringAnsi("hello 世界"); // 8 'letters' + Assert.That(slice.GetBytes(), Is.EqualTo(Encoding.Default.GetBytes("hello 世界"))); //note: this depends on your OS locale! + Assert.That(slice.ToStringAnsi(), Is.EqualTo("hello ??"), "non-ANSI chars should be converted to '?'"); + } - //REVIEW: should FromAscii() throw an exception on non-ASCII chars? It will silently corrupt strings if nobody checks the value.... + [Test] + public void Test_Slice_ToStringAnsi() + { + Assert.That(Slice.Nil.ToStringAnsi(), Is.Null); + Assert.That(Slice.Empty.ToStringAnsi(), Is.EqualTo(String.Empty)); + Assert.That(new[] { (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringAnsi(), Is.EqualTo("ABC")); + Assert.That(Encoding.Default.GetBytes("héllô").AsSlice().ToStringAnsi(), Is.EqualTo("héllô")); //note: this depends on your OS locale! + Assert.That(new[] { (byte) 0xFF, (byte) '/', (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringAnsi(), Is.EqualTo("\xFF/ABC")); + + // if the string contains non-ANSI chars, it will be corrupted + // note: the line below should contain two kanjis. If your editor displays '??' or squares, it is probably not able to display unicode chars properly + Assert.That(Encoding.UTF8.GetBytes("héllô 世界").AsSlice().ToStringAnsi(), Is.EqualTo("h\xC3\xA9ll\xC3\xB4 \xE4\xB8\u2013\xE7\u2022\u0152")); //note: this may change depending on your locale! } [Test] public void Test_Slice_FromString_Uses_UTF8() { Assert.That(Slice.FromString(default(string)).GetBytes(), Is.Null); - Assert.That(Slice.FromString(String.Empty).GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(Slice.FromString(string.Empty).GetBytes(), Is.EqualTo(new byte[0])); Assert.That(Slice.FromString("ABC").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43 })); Assert.That(Slice.FromString("é").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xA9 })); @@ -277,364 +351,1179 @@ public void Test_Slice_FromString_Uses_UTF8() Assert.That(slice.GetBytes(), Is.EqualTo(Encoding.UTF8.GetBytes("héllø 世界"))); Assert.That(slice.ToUnicode(), Is.EqualTo("héllø 世界"), "non-ASCII chars should not be corrupted"); Assert.That(slice.Count, Is.EqualTo(14)); + + // UTF8 does not map \xFF or \xFE directly to a single byte (but at least it should round-trip) + Assert.That(Slice.FromString("\xFF").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xBF })); + Assert.That(Slice.FromString("\xFE").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xBE })); + Assert.That(new byte[] { 0xC3, 0xBF }.AsSlice().ToUnicode(), Is.EqualTo("\xFF")); + Assert.That(new byte[] { 0xC3, 0xBE }.AsSlice().ToUnicode(), Is.EqualTo("\xFE")); + } + + [Test] + public void Test_Slice_FromStringUtf8() + { + Assert.That(Slice.FromStringUtf8(default(string)).GetBytes(), Is.Null); + Assert.That(Slice.FromStringUtf8(string.Empty).GetBytes(), Is.EqualTo(new byte[0])); + Assert.That(Slice.FromStringUtf8("ABC").GetBytes(), Is.EqualTo(new byte[] { 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringUtf8("é").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xA9 })); + + // if the string contains UTF-8 characters, it should be encoded properly + // note: the line below should contain two kanjis. If your editor displays '??' or squares, it is probably not able to display unicode chars properly + var slice = Slice.FromStringUtf8("héllø 世界"); // 8 'letters' + Assert.That(slice.GetBytes(), Is.EqualTo(Encoding.UTF8.GetBytes("héllø 世界"))); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("héllø 世界"), "non-ASCII chars should not be corrupted"); + Assert.That(slice.ToUnicode(), Is.EqualTo("héllø 世界"), "non-ASCII chars should not be corrupted"); + Assert.That(slice.Count, Is.EqualTo(14)); + + // UTF8 does not map \xFF or \xFE directly to a single byte (but at least it should round-trip) + Assert.That(Slice.FromStringUtf8("\xFF").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xBF })); + Assert.That(Slice.FromStringUtf8("\xFE").GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xBE })); + Assert.That(new byte[] { 0xC3, 0xBF }.AsSlice().ToStringUtf8(), Is.EqualTo("\xFF")); + Assert.That(new byte[] { 0xC3, 0xBF }.AsSlice().ToUnicode(), Is.EqualTo("\xFF")); + Assert.That(new byte[] { 0xC3, 0xBE }.AsSlice().ToStringUtf8(), Is.EqualTo("\xFE")); + Assert.That(new byte[] { 0xC3, 0xBE }.AsSlice().ToUnicode(), Is.EqualTo("\xFE")); + } + + [Test] + public void Test_Slice_ToStringUtf8() + { + Assert.That(Slice.Nil.ToStringUtf8(), Is.Null); + Assert.That(Slice.Empty.ToStringUtf8(), Is.EqualTo(String.Empty)); + Assert.That(new[] { (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringUtf8(), Is.EqualTo("ABC")); + Assert.That(Encoding.UTF8.GetBytes("héllô").AsSlice().ToStringUtf8(), Is.EqualTo("héllô")); //note: this depends on your OS locale! + Assert.That(Encoding.UTF8.GetBytes("世界").AsSlice().ToStringUtf8(), Is.EqualTo("世界")); + + // should remove the bom! + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringUtf8(), Is.EqualTo("ABC"), "BOM should be removed"); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF }.AsSlice().ToStringUtf8(), Is.EqualTo(String.Empty), "BOM should also be removed for empty string"); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, 0xEF, 0xBB, 0xBF, (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringUtf8(), Is.EqualTo("\uFEFFABC"), "Only one BOM should be removed"); + + // custom case for 0xFF and 0xFE + Assert.That(new byte[] { 0xFF, (byte) '/', (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringAnsi(), Is.EqualTo("\xFF/ABC")); + Assert.That(new byte[] { 0xFE, (byte) '/', (byte) 'A', (byte) 'B', (byte) 'C' }.AsSlice().ToStringAnsi(), Is.EqualTo("\xFE/ABC")); + + // corrupted UTF-8 + Assert.That(() => new byte[] { 0xEF, 0xBB }.AsSlice().ToStringUtf8(), Throws.Exception, "Partial BOM should fail to decode"); + Assert.That(() => new byte[] { (byte) 'A', 0xc3, 0x28, (byte) 'B' }.AsSlice().ToStringUtf8(), Throws.Exception, "Invalid 2-byte sequence"); + Assert.That(() => new byte[] { (byte) 'A', 0xe2, 0x28, 0xa1, (byte) 'B' }.AsSlice().ToStringUtf8(), Throws.Exception, "Invalid 3-byte sequence"); + Assert.That(() => new byte[] { (byte) 'A', 0xf0, 0x28, 0x8c, 0x28, (byte) 'B' }.AsSlice().ToStringUtf8(), Throws.Exception, "Invalid 4-byte sequence"); + Assert.That(() => new byte[] { (byte) 'A', 0xf0, 0x28, /*..SNIP..*/ }.AsSlice().ToStringUtf8(), Throws.Exception, "Truncated 4-byte sequence"); + } + + [Test] + public void Test_Slice_FromStringUtf8WithBom() + { + Assert.That(Slice.FromStringUtf8WithBom(default(string)).GetBytes(), Is.Null); + Assert.That(Slice.FromStringUtf8WithBom(string.Empty).GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF })); + Assert.That(Slice.FromStringUtf8WithBom("ABC").GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF, 0x41, 0x42, 0x43 })); + Assert.That(Slice.FromStringUtf8WithBom("é").GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xA9 })); + + // if the string contains UTF-8 characters, it should be encoded properly + // note: the line below should contain two kanjis. If your editor displays '??' or squares, it is probably not able to display unicode chars properly + var slice = Slice.FromStringUtf8WithBom("héllø 世界"); // 8 'letters' + Assert.That(slice.GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF }.Concat(Encoding.UTF8.GetBytes("héllø 世界")).ToArray())); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("héllø 世界"), "The BOM should be removed"); + Assert.That(slice.ToUnicode(), Is.EqualTo("\xFEFFhéllø 世界"), "The BOM should be preserved"); + Assert.That(slice.Count, Is.EqualTo(3 + 14)); + + // UTF8 does not map \xFF or \xFE directly to a single byte (but at least it should round-trip) + Assert.That(Slice.FromStringUtf8WithBom("\xFF").GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBF })); + Assert.That(Slice.FromStringUtf8WithBom("\xFE").GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBE })); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBF }.AsSlice().ToStringUtf8(), Is.EqualTo("\xFF")); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBF }.AsSlice().ToUnicode(), Is.EqualTo("\uFEFF\xFF")); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBE }.AsSlice().ToStringUtf8(), Is.EqualTo("\xFE")); + Assert.That(new byte[] { 0xEF, 0xBB, 0xBF, 0xC3, 0xBE }.AsSlice().ToUnicode(), Is.EqualTo("\uFEFF\xFE")); + } + + [Test] + public void Test_Slice_FromChar_Uses_UTF8() + { + // from 0 to 127 is regular single-byte ASCII + Assert.That(Slice.FromChar('\0').GetBytes(), Is.EqualTo(new byte[] { 0 })); + Assert.That(Slice.FromChar('\x01').GetBytes(), Is.EqualTo(new byte[] { 1 })); + Assert.That(Slice.FromChar('0').GetBytes(), Is.EqualTo(new byte[] { 48 })); + Assert.That(Slice.FromChar('A').GetBytes(), Is.EqualTo(new byte[] { 65 })); + Assert.That(Slice.FromChar('a').GetBytes(), Is.EqualTo(new byte[] { 97 })); + Assert.That(Slice.FromChar('~').GetBytes(), Is.EqualTo(new byte[] { 126 })); + Assert.That(Slice.FromChar('\x7F').GetBytes(), Is.EqualTo(new byte[] { 127 })); + + // 128 and above is multi-byte UTF-8 + Assert.That(Slice.FromChar('\x80').GetBytes(), Is.EqualTo(new byte[] { 0xC2, 0x80 })); + Assert.That(Slice.FromChar('é').GetBytes(), Is.EqualTo(new byte[] { 0xC3, 0xA9 })); + Assert.That(Slice.FromChar('\u221E').GetBytes(), Is.EqualTo(new byte[] { 0xE2, 0x88, 0x9E })); + Assert.That(Slice.FromChar('\uFFFE').GetBytes(), Is.EqualTo(new byte[] { 0xEF, 0xBF, 0xBE})); + } + + #region Signed... + + #region 24-bits + + #region Little-Endian + + [Test] + public void Test_Slice_ToInt24() + { + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt24(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x34, 0x12 }.AsSlice().ToInt24(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x34, 0x12, 0x00 }.AsSlice().ToInt24(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x56, 0x34, 0x12 }.AsSlice().ToInt24(), Is.EqualTo(0x123456)); + + Assert.That(new byte[] { }.AsSlice().ToInt24(), Is.EqualTo(0)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt24(), Is.EqualTo(0)); + Assert.That(new byte[] { 127 }.AsSlice().ToInt24(), Is.EqualTo(127)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt24(), Is.EqualTo(255)); + Assert.That(new byte[] { 0, 1 }.AsSlice().ToInt24(), Is.EqualTo(256)); + Assert.That(new byte[] { 255, 127 }.AsSlice().ToInt24(), Is.EqualTo(32767)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt24(), Is.EqualTo(65535)); + Assert.That(new byte[] { 0, 0, 1 }.AsSlice().ToInt24(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 255, 255, 127 }.AsSlice().ToInt24(), Is.EqualTo((1 << 23) - 1)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt24(), Is.EqualTo((1 << 24) - 1)); + + Assert.That(() => Slice.Create(4).ToInt24(), Throws.InstanceOf()); } + #endregion + + #region Big Endian + + [Test] + public void Test_Slice_ToInt24BE() + { + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt24BE(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x12, 0x34 }.AsSlice().ToInt24BE(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x12, 0x34, 0x56 }.AsSlice().ToInt24BE(), Is.EqualTo(0x123456)); + + Assert.That(new byte[] { }.AsSlice().ToInt24BE(), Is.EqualTo(0)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt24BE(), Is.EqualTo(0)); + Assert.That(new byte[] { 127 }.AsSlice().ToInt24BE(), Is.EqualTo(127)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt24BE(), Is.EqualTo(255)); + Assert.That(new byte[] { 1, 0 }.AsSlice().ToInt24BE(), Is.EqualTo(256)); + Assert.That(new byte[] { 127, 255 }.AsSlice().ToInt24BE(), Is.EqualTo(32767)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt24BE(), Is.EqualTo(65535)); + Assert.That(new byte[] { 1, 0, 0 }.AsSlice().ToInt24BE(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 127, 255, 255 }.AsSlice().ToInt24BE(), Is.EqualTo((1 << 23) - 1)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt24BE(), Is.EqualTo((1 << 24) - 1)); + + Assert.That(() => Slice.Create(4).ToInt24BE(), Throws.InstanceOf()); + } + + #endregion + + #endregion + + #region 32-bits + + #region Little-Endian + [Test] public void Test_Slice_FromInt32() { // 32-bit integers should be encoded in little endian, and with 1, 2 or 4 bytes - // 0x12 -> { 12 } - // 0x1234 -> { 34 12 } - // 0x123456 -> { 56 34 12 00 } - // 0x12345678 -> { 78 56 34 12 } - Assert.That(Slice.FromInt32(0x12).ToHexaString(), Is.EqualTo("12")); - Assert.That(Slice.FromInt32(0x1234).ToHexaString(), Is.EqualTo("3412")); - Assert.That(Slice.FromInt32(0x123456).ToHexaString(), Is.EqualTo("56341200")); - Assert.That(Slice.FromInt32(0x12345678).ToHexaString(), Is.EqualTo("78563412")); + void Verify(int value, string expected) + { + Assert.That(Slice.FromInt32(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } - Assert.That(Slice.FromInt32(0).ToHexaString(), Is.EqualTo("00")); - Assert.That(Slice.FromInt32(1).ToHexaString(), Is.EqualTo("01")); - Assert.That(Slice.FromInt32(255).ToHexaString(), Is.EqualTo("ff")); - Assert.That(Slice.FromInt32(256).ToHexaString(), Is.EqualTo("0001")); - Assert.That(Slice.FromInt32(65535).ToHexaString(), Is.EqualTo("ffff")); - Assert.That(Slice.FromInt32(65536).ToHexaString(), Is.EqualTo("00000100")); - Assert.That(Slice.FromInt32(int.MaxValue).ToHexaString(), Is.EqualTo("ffffff7f")); - Assert.That(Slice.FromInt32(int.MinValue).ToHexaString(), Is.EqualTo("00000080")); + Verify(0x12, "12"); + Verify(0x1234, "3412"); + Verify(0x123456, "563412"); + Verify(0x12345678, "78563412"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(255, "FF"); + Verify(256, "0001"); + Verify(65535, "FFFF"); + Verify(65536, "000001"); + Verify(16777215, "FFFFFF"); + Verify(16777216, "00000001"); + Verify(int.MaxValue, "FFFFFF7F"); + Verify(int.MinValue, "00000080"); + } + + [Test] + public void Test_Slice_FromFixed32() + { + // FromFixed32 always produce 4 bytes and uses Little Endian + + Assert.That(Slice.FromFixed32(0).GetBytes(), Is.EqualTo(new byte[4])); + Assert.That(Slice.FromFixed32(1).GetBytes(), Is.EqualTo(new byte[] { 1, 0, 0, 0 })); + Assert.That(Slice.FromFixed32(256).GetBytes(), Is.EqualTo(new byte[] { 0, 1, 0, 0 })); + Assert.That(Slice.FromFixed32(65536).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 1, 0 })); + Assert.That(Slice.FromFixed32(16777216).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 1 })); + Assert.That(Slice.FromFixed32(short.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 127, 0, 0 })); + Assert.That(Slice.FromFixed32(int.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 127 })); + + Assert.That(Slice.FromFixed32(-1).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255 })); + Assert.That(Slice.FromFixed32(-256).GetBytes(), Is.EqualTo(new byte[] { 0, 255, 255, 255 })); + Assert.That(Slice.FromFixed32(-65536).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 255, 255 })); + Assert.That(Slice.FromFixed32(-16777216).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 255 })); + Assert.That(Slice.FromFixed32(int.MinValue).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 128 })); + + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + int x = rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); + Slice s = Slice.FromFixed32(x); + Assert.That(s.Count, Is.EqualTo(4)); + Assert.That(s.ToInt32(), Is.EqualTo(x)); + } } [Test] public void Test_Slice_ToInt32() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToInt32(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x34, 0x12 }).ToInt32(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12 }).ToInt32(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12, 0x00 }).ToInt32(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x78, 0x56, 0x34, 0x12 }).ToInt32(), Is.EqualTo(0x12345678)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt32(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x34, 0x12 }.AsSlice().ToInt32(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x56, 0x34, 0x12 }.AsSlice().ToInt32(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x56, 0x34, 0x12, 0x00 }.AsSlice().ToInt32(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt32(), Is.EqualTo(0x12345678)); + + Assert.That(new byte[] { }.AsSlice().ToInt32(), Is.EqualTo(0)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt32(), Is.EqualTo(0)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt32(), Is.EqualTo(255)); + Assert.That(new byte[] { 0, 1 }.AsSlice().ToInt32(), Is.EqualTo(256)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt32(), Is.EqualTo(65535)); + Assert.That(new byte[] { 0, 0, 1 }.AsSlice().ToInt32(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 0, 0, 1, 0 }.AsSlice().ToInt32(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt32(), Is.EqualTo((1 << 24) - 1)); + Assert.That(new byte[] { 0, 0, 0, 1 }.AsSlice().ToInt32(), Is.EqualTo(1 << 24)); + Assert.That(new byte[] { 255, 255, 255, 127 }.AsSlice().ToInt32(), Is.EqualTo(int.MaxValue)); + + Assert.That(() => Slice.Create(5).ToInt32(), Throws.InstanceOf()); + } - Assert.That(Slice.Create(new byte[] { 0 }).ToInt32(), Is.EqualTo(0)); - Assert.That(Slice.Create(new byte[] { 255 }).ToInt32(), Is.EqualTo(255)); - Assert.That(Slice.Create(new byte[] { 0, 1 }).ToInt32(), Is.EqualTo(256)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToInt32(), Is.EqualTo(65535)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1 }).ToInt32(), Is.EqualTo(1 << 16)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1, 0 }).ToInt32(), Is.EqualTo(1 << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToInt32(), Is.EqualTo((1 << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 1 }).ToInt32(), Is.EqualTo(1 << 24)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 127 }).ToInt32(), Is.EqualTo(int.MaxValue)); + #endregion + + #region Big Endian + + [Test] + public void Test_Slice_FromInt32BE() + { + // 32-bit integers should be encoded in little endian, and with 1, 2 or 4 bytes + + void Verify(int value, string expected) + { + Assert.That(Slice.FromInt32BE(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12, "12"); + Verify(0x1234, "1234"); + Verify(0x123456, "123456"); + Verify(0x12345678, "12345678"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(255, "FF"); + Verify(256, "0100"); + Verify(65535, "FFFF"); + Verify(65536, "010000"); + Verify(16777215, "FFFFFF"); + Verify(16777216, "01000000"); + Verify(int.MaxValue, "7FFFFFFF"); + Verify(int.MinValue, "80000000"); } [Test] - public void Test_Slice_ToInt32BE() + public void Test_Slice_FromFixed32BE() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToInt32BE(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34 }).ToInt32BE(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56 }).ToInt32BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x00, 0x12, 0x34, 0x56 }).ToInt32BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78 }).ToInt32BE(), Is.EqualTo(0x12345678)); + // FromFixed32 always produce 4 bytes and uses Little Endian + + Assert.That(Slice.FromFixed32BE(0).GetBytes(), Is.EqualTo(new byte[4])); + Assert.That(Slice.FromFixed32BE(1).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 1 })); + Assert.That(Slice.FromFixed32BE(256).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 1, 0 })); + Assert.That(Slice.FromFixed32BE(65536).GetBytes(), Is.EqualTo(new byte[] { 0, 1, 0, 0 })); + Assert.That(Slice.FromFixed32BE(16777216).GetBytes(), Is.EqualTo(new byte[] { 1, 0, 0, 0 })); + Assert.That(Slice.FromFixed32BE(short.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 127, 255 })); + Assert.That(Slice.FromFixed32BE(int.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 127, 255, 255, 255 })); + + Assert.That(Slice.FromFixed32BE(-1).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255 })); + Assert.That(Slice.FromFixed32BE(-256).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 0 })); + Assert.That(Slice.FromFixed32BE(-65536).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 0, 0 })); + Assert.That(Slice.FromFixed32BE(-16777216).GetBytes(), Is.EqualTo(new byte[] { 255, 0, 0, 0 })); + Assert.That(Slice.FromFixed32BE(int.MinValue).GetBytes(), Is.EqualTo(new byte[] { 128, 0, 0, 0 })); - Assert.That(Slice.Create(new byte[] { 0 }).ToInt32BE(), Is.EqualTo(0)); - Assert.That(Slice.Create(new byte[] { 255 }).ToInt32BE(), Is.EqualTo(255)); - Assert.That(Slice.Create(new byte[] { 1, 0 }).ToInt32BE(), Is.EqualTo(256)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToInt32BE(), Is.EqualTo(65535)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0 }).ToInt32BE(), Is.EqualTo(1 << 16)); - Assert.That(Slice.Create(new byte[] { 0, 1, 0, 0 }).ToInt32BE(), Is.EqualTo(1 << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToInt32BE(), Is.EqualTo((1 << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0 }).ToInt32BE(), Is.EqualTo(1 << 24)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255 }).ToInt32BE(), Is.EqualTo(int.MaxValue)); + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + int x = rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); + Slice s = Slice.FromFixed32BE(x); + Assert.That(s.Count, Is.EqualTo(4)); + Assert.That(s.ToInt32BE(), Is.EqualTo(x)); + } } + [Test] + public void Test_Slice_ToInt32BE() + { + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt32BE(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x12, 0x34 }.AsSlice().ToInt32BE(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x12, 0x34, 0x56 }.AsSlice().ToInt32BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x00, 0x12, 0x34, 0x56 }.AsSlice().ToInt32BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78 }.AsSlice().ToInt32BE(), Is.EqualTo(0x12345678)); + + Assert.That(new byte[] { }.AsSlice().ToInt32BE(), Is.EqualTo(0)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt32BE(), Is.EqualTo(0)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt32BE(), Is.EqualTo(255)); + Assert.That(new byte[] { 1, 0 }.AsSlice().ToInt32BE(), Is.EqualTo(256)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt32BE(), Is.EqualTo(65535)); + Assert.That(new byte[] { 1, 0, 0 }.AsSlice().ToInt32BE(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 0, 1, 0, 0 }.AsSlice().ToInt32BE(), Is.EqualTo(1 << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt32BE(), Is.EqualTo((1 << 24) - 1)); + Assert.That(new byte[] { 1, 0, 0, 0 }.AsSlice().ToInt32BE(), Is.EqualTo(1 << 24)); + Assert.That(new byte[] { 127, 255, 255, 255 }.AsSlice().ToInt32BE(), Is.EqualTo(int.MaxValue)); + + Assert.That(() => Slice.Create(5).ToInt32BE(), Throws.InstanceOf()); + } + + #endregion + + #endregion + + #region 64-bits + + #region Little-Endian + [Test] public void Test_Slice_FromInt64() { // 64-bit integers should be encoded in little endian, and with 1, 2, 4 or 8 bytes - // 0x12 -> { 12 } - // 0x1234 -> { 34 12 } - // 0x123456 -> { 56 34 12 00 } - // 0x12345678 -> { 78 56 34 12 } - // 0x123456789A -> { 9A 78 56 34 12 00 00 00} - // 0x123456789ABC -> { BC 9A 78 56 34 12 00 00} - // 0x123456789ABCDE -> { DE BC 9A 78 56 34 12 00} - // 0x123456789ABCDEF0 -> { F0 DE BC 9A 78 56 34 12 } - - Assert.That(Slice.FromInt64(0x12).ToHexaString(), Is.EqualTo("12")); - Assert.That(Slice.FromInt64(0x1234).ToHexaString(), Is.EqualTo("3412")); - Assert.That(Slice.FromInt64(0x123456).ToHexaString(), Is.EqualTo("56341200")); - Assert.That(Slice.FromInt64(0x12345678).ToHexaString(), Is.EqualTo("78563412")); - Assert.That(Slice.FromInt64(0x123456789A).ToHexaString(), Is.EqualTo("9a78563412000000")); - Assert.That(Slice.FromInt64(0x123456789ABC).ToHexaString(), Is.EqualTo("bc9a785634120000")); - Assert.That(Slice.FromInt64(0x123456789ABCDE).ToHexaString(), Is.EqualTo("debc9a7856341200")); - Assert.That(Slice.FromInt64(0x123456789ABCDEF0).ToHexaString(), Is.EqualTo("f0debc9a78563412")); - - Assert.That(Slice.FromInt64(0).ToHexaString(), Is.EqualTo("00")); - Assert.That(Slice.FromInt64(1).ToHexaString(), Is.EqualTo("01")); - Assert.That(Slice.FromInt64(255).ToHexaString(), Is.EqualTo("ff")); - Assert.That(Slice.FromInt64(256).ToHexaString(), Is.EqualTo("0001")); - Assert.That(Slice.FromInt64(65535).ToHexaString(), Is.EqualTo("ffff")); - Assert.That(Slice.FromInt64(65536).ToHexaString(), Is.EqualTo("00000100")); - Assert.That(Slice.FromInt64(int.MaxValue).ToHexaString(), Is.EqualTo("ffffff7f")); - Assert.That(Slice.FromInt64(int.MinValue).ToHexaString(), Is.EqualTo("00000080ffffffff")); - Assert.That(Slice.FromInt64(1L + int.MaxValue).ToHexaString(), Is.EqualTo("0000008000000000")); - Assert.That(Slice.FromInt64(long.MaxValue).ToHexaString(), Is.EqualTo("ffffffffffffff7f")); - Assert.That(Slice.FromInt64(long.MinValue).ToHexaString(), Is.EqualTo("0000000000000080")); + + void Verify(long value, string expected) + { + Assert.That(Slice.FromInt64(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12, "12"); + Verify(0x1234, "3412"); + Verify(0x123456, "563412"); + Verify(0x12345678, "78563412"); + Verify(0x123456789A, "9A78563412"); + Verify(0x123456789ABC, "BC9A78563412"); + Verify(0x123456789ABCDE, "DEBC9A78563412"); + Verify(0x123456789ABCDEF0, "F0DEBC9A78563412"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(255, "FF"); + Verify(256, "0001"); + Verify(65535, "FFFF"); + Verify(65536, "000001"); + Verify(16777215, "FFFFFF"); + Verify(16777216, "00000001"); + Verify(int.MaxValue, "FFFFFF7F"); + Verify(int.MinValue, "00000080FFFFFFFF"); + Verify(1L + int.MaxValue, "00000080"); + Verify(long.MaxValue, "FFFFFFFFFFFFFF7F"); + Verify(long.MinValue, "0000000000000080"); } + [Test] + public void Test_Slice_FromFixed64() + { + // FromFixed64 always produce 8 bytes and uses Little Endian + + void Verify(long value, byte[] expected) + { + Assert.That(Slice.FromFixed64(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0L, new byte[8]); + Verify(1L, new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }); + Verify(1L << 8, new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 }); + Verify(1L << 16, new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 }); + Verify(1L << 24, new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 }); + Verify(1L << 32, new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 }); + Verify(1L << 40, new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 }); + Verify(1L << 48, new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 }); + Verify(1L << 56, new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }); + Verify(short.MaxValue, new byte[] { 255, 127, 0, 0, 0, 0, 0, 0 }); + Verify(int.MaxValue, new byte[] { 255, 255, 255, 127, 0, 0, 0, 0 }); + Verify(long.MaxValue, new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }); + + Verify(-1L, new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }); + Verify(-256L, new byte[] { 0, 255, 255, 255, 255, 255, 255, 255 }); + Verify(-65536L, new byte[] { 0, 0, 255, 255, 255, 255, 255, 255 }); + Verify(-16777216L, new byte[] { 0, 0, 0, 255, 255, 255, 255, 255 }); + Verify(-4294967296L, new byte[] { 0, 0, 0, 0, 255, 255, 255, 255 }); + Verify(long.MinValue, new byte[] { 0, 0, 0, 0, 0, 0, 0, 128 }); + + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + long x = (long)rnd.Next() * rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); + Slice s = Slice.FromFixed64(x); + Assert.That(s.Count, Is.EqualTo(8)); + Assert.That(s.ToInt64(), Is.EqualTo(x)); + } + } + [Test] public void Test_Slice_ToInt64() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToInt64(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12, 0x00 }).ToInt64(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x78, 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x12345678)); - Assert.That(Slice.Create(new byte[] { 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x123456789A)); - Assert.That(Slice.Create(new byte[] { 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x123456789ABC)); - Assert.That(Slice.Create(new byte[] { 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x123456789ABCDE)); - Assert.That(Slice.Create(new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToInt64(), Is.EqualTo(0x123456789ABCDEF0)); - - Assert.That(Slice.Create(new byte[] { 0 }).ToInt64(), Is.EqualTo(0L)); - Assert.That(Slice.Create(new byte[] { 255 }).ToInt64(), Is.EqualTo(255L)); - Assert.That(Slice.Create(new byte[] { 0, 1 }).ToInt64(), Is.EqualTo(256L)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToInt64(), Is.EqualTo(65535L)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1, 0 }).ToInt64(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToInt64(), Is.EqualTo((1L << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 24)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 32)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 40)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 48)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }).ToInt64(), Is.EqualTo(1L << 56)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 127 }).ToInt64(), Is.EqualTo(int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }).ToInt64(), Is.EqualTo(long.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }).ToInt64(), Is.EqualTo(-1L)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x56, 0x34, 0x12, 0x00 }.AsSlice().ToInt64(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x12345678)); + Assert.That(new byte[] { 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x123456789A)); + Assert.That(new byte[] { 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x123456789ABC)); + Assert.That(new byte[] { 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x123456789ABCDE)); + Assert.That(new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToInt64(), Is.EqualTo(0x123456789ABCDEF0)); + + Assert.That(new byte[] { }.AsSlice().ToInt64(), Is.EqualTo(0L)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt64(), Is.EqualTo(0L)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt64(), Is.EqualTo(255L)); + Assert.That(new byte[] { 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(256L)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt64(), Is.EqualTo(65535L)); + Assert.That(new byte[] { 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 0, 0, 1, 0 }.AsSlice().ToInt64(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt64(), Is.EqualTo((1L << 24) - 1)); + Assert.That(new byte[] { 0, 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 24)); + Assert.That(new byte[] { 0, 0, 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 32)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 40)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 48)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }.AsSlice().ToInt64(), Is.EqualTo(1L << 56)); + Assert.That(new byte[] { 255, 255, 255, 127 }.AsSlice().ToInt64(), Is.EqualTo(int.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }.AsSlice().ToInt64(), Is.EqualTo(long.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToInt64(), Is.EqualTo(-1L)); // should validate the arguments - var x = Slice.Create(new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }); + var x = new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice(); Assert.That(() => MutateOffset(x, -1).ToInt64(), Throws.InstanceOf()); Assert.That(() => MutateCount(x, 9).ToInt64(), Throws.InstanceOf()); Assert.That(() => MutateArray(x, null).ToInt64(), Throws.InstanceOf()); } + #endregion + + #region Big-Endian + + [Test] + public void Test_Slice_FromInt64BE() + { + // 64-bit integers should be encoded in little endian, and with 1, 2, 4 or 8 bytes + + void Verify(long value, string expected) + { + Assert.That(Slice.FromInt64BE(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12, "12"); + Verify(0x1234, "1234"); + Verify(0x123456, "123456"); + Verify(0x12345678, "12345678"); + Verify(0x123456789A, "123456789A"); + Verify(0x123456789ABC, "123456789ABC"); + Verify(0x123456789ABCDE, "123456789ABCDE"); + Verify(0x123456789ABCDEF0, "123456789ABCDEF0"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(127, "7F"); + Verify(128, "80"); + + Verify(1L << 8, "0100"); + Verify(1L << 16, "010000"); + Verify(1L << 24, "01000000"); + Verify(1L << 32, "0100000000"); + Verify(1L << 40, "010000000000"); + Verify(1L << 48, "01000000000000"); + Verify(1L << 56, "0100000000000000"); + + Verify((1L << 8) - 1, "FF"); + Verify((1L << 16) - 1, "FFFF"); + Verify((1L << 24) - 1, "FFFFFF"); + Verify((1L << 32) - 1, "FFFFFFFF"); + Verify((1L << 40) - 1, "FFFFFFFFFF"); + Verify((1L << 48) - 1, "FFFFFFFFFFFF"); + Verify((1L << 56) - 1, "FFFFFFFFFFFFFF"); + Verify(long.MaxValue, "7FFFFFFFFFFFFFFF"); + + Verify(-1, "FFFFFFFFFFFFFFFF"); + Verify(-2, "FFFFFFFFFFFFFFFE"); + Verify(-256, "FFFFFFFFFFFFFF00"); + Verify(-65536, "FFFFFFFFFFFF0000"); + Verify(-16777216, "FFFFFFFFFF000000"); + Verify(int.MinValue, "FFFFFFFF80000000"); + Verify(long.MinValue, "8000000000000000"); + + } + + [Test] + public void Test_Slice_FromFixed64BE() + { + // FromFixed64 always produce 8 bytes and uses Little Endian + + void Verify(long value, byte[] expected) + { + Assert.That(Slice.FromFixed64BE(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0L, new byte[8]); + Verify(1L, new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }); + Verify(1L << 8, new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 }); + Verify(1L << 16, new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 }); + Verify(1L << 24, new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 }); + Verify(1L << 32, new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 }); + Verify(1L << 40, new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 }); + Verify(1L << 48, new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 }); + Verify(1L << 56, new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }); + Verify(short.MaxValue, new byte[] { 0, 0, 0, 0, 0, 0, 127, 255 }); + Verify(int.MaxValue, new byte[] { 0, 0, 0, 0, 127, 255, 255, 255 }); + Verify(long.MaxValue, new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }); + + Verify(-1L, new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }); + Verify(-256L, new byte[] { 255, 255, 255, 255, 255, 255, 255, 0 }); + Verify(-65536L, new byte[] { 255, 255, 255, 255, 255, 255, 0, 0 }); + Verify(-16777216L, new byte[] { 255, 255, 255, 255, 255, 0, 0, 0 }); + Verify(-4294967296L, new byte[] { 255, 255, 255, 255, 0, 0, 0, 0 }); + Verify(long.MinValue, new byte[] { 128, 0, 0, 0, 0, 0, 0, 0 }); + + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + long x = (long)rnd.Next() * rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); + Slice s = Slice.FromFixed64BE(x); + Assert.That(s.Count, Is.EqualTo(8)); + Assert.That(s.ToInt64BE(), Is.EqualTo(x)); + } + } + [Test] public void Test_Slice_ToInt64BE() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToInt64BE(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34 }).ToInt64BE(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56 }).ToInt64BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x00, 0x12, 0x34, 0x56 }).ToInt64BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78 }).ToInt64BE(), Is.EqualTo(0x12345678)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A }).ToInt64BE(), Is.EqualTo(0x123456789A)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC }).ToInt64BE(), Is.EqualTo(0x123456789ABC)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE }).ToInt64BE(), Is.EqualTo(0x123456789ABCDE)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }).ToInt64BE(), Is.EqualTo(0x123456789ABCDEF0)); - - Assert.That(Slice.Create(new byte[] { 0 }).ToInt64BE(), Is.EqualTo(0L)); - Assert.That(Slice.Create(new byte[] { 255 }).ToInt64BE(), Is.EqualTo(255L)); - Assert.That(Slice.Create(new byte[] { 1, 0 }).ToInt64BE(), Is.EqualTo(256L)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToInt64BE(), Is.EqualTo(65535L)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 0, 1, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToInt64BE(), Is.EqualTo((1L << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 24)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 32)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 40)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 48)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }).ToInt64BE(), Is.EqualTo(1L << 56)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255 }).ToInt64BE(), Is.EqualTo(int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }).ToInt64BE(), Is.EqualTo(long.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }).ToInt64BE(), Is.EqualTo(-1L)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToInt64BE(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x12, 0x34 }.AsSlice().ToInt64BE(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x12, 0x34, 0x56 }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x00, 0x12, 0x34, 0x56 }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78 }.AsSlice().ToInt64BE(), Is.EqualTo(0x12345678)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456789A)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456789ABC)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456789ABCDE)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }.AsSlice().ToInt64BE(), Is.EqualTo(0x123456789ABCDEF0)); + + Assert.That(new byte[] { }.AsSlice().ToInt64BE(), Is.EqualTo(0L)); + Assert.That(new byte[] { 0 }.AsSlice().ToInt64BE(), Is.EqualTo(0L)); + Assert.That(new byte[] { 255 }.AsSlice().ToInt64BE(), Is.EqualTo(255L)); + Assert.That(new byte[] { 1, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(256L)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToInt64BE(), Is.EqualTo(65535L)); + Assert.That(new byte[] { 1, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 0, 1, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToInt64BE(), Is.EqualTo((1L << 24) - 1)); + Assert.That(new byte[] { 1, 0, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 24)); + Assert.That(new byte[] { 1, 0, 0, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 32)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 40)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 48)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }.AsSlice().ToInt64BE(), Is.EqualTo(1L << 56)); + Assert.That(new byte[] { 127, 255, 255, 255 }.AsSlice().ToInt64BE(), Is.EqualTo(int.MaxValue)); + Assert.That(new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToInt64BE(), Is.EqualTo(long.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToInt64BE(), Is.EqualTo(-1L)); // should validate the arguments - var x = Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }); + var x = new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }.AsSlice(); Assert.That(() => MutateOffset(x, -1).ToInt64BE(), Throws.InstanceOf()); Assert.That(() => MutateCount(x, 9).ToInt64BE(), Throws.InstanceOf()); Assert.That(() => MutateArray(x, null).ToInt64BE(), Throws.InstanceOf()); } + #endregion + + #endregion + + #endregion + + #region Unsigned... + + #region 32-bits + + #region Little-Endian + [Test] public void Test_Slice_FromUInt32() { // 32-bit integers should be encoded in little endian, and with 1, 2 or 4 bytes - // 0x12 -> { 12 } - // 0x1234 -> { 34 12 } - // 0x123456 -> { 56 34 12 00 } - // 0x12345678 -> { 78 56 34 12 } - Assert.That(Slice.FromUInt32(0x12).ToHexaString(), Is.EqualTo("12")); - Assert.That(Slice.FromUInt32(0x1234).ToHexaString(), Is.EqualTo("3412")); - Assert.That(Slice.FromUInt32(0x123456).ToHexaString(), Is.EqualTo("56341200")); - Assert.That(Slice.FromUInt32(0x12345678).ToHexaString(), Is.EqualTo("78563412")); + void Verify(uint value, string expected) + { + Assert.That(Slice.FromUInt32(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12, "12"); + Verify(0x1234, "3412"); + Verify(0x123456, "563412"); + Verify(0x12345678, "78563412"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(255, "FF"); + Verify(256, "0001"); + Verify(65535, "FFFF"); + Verify(65536, "000001"); + Verify(int.MaxValue, "FFFFFF7F"); + Verify(uint.MaxValue, "FFFFFFFF"); + } + + [Test] + public void Test_Slice_FromFixedU32() + { + // FromFixed32 always produce 4 bytes and uses Little Endian + + void Verify(uint value, byte[] expected) + { + Assert.That(Slice.FromFixedU32(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0, new byte[4]); + Verify(1, new byte[] { 1, 0, 0, 0 }); + Verify(256, new byte[] { 0, 1, 0, 0 }); + Verify(ushort.MaxValue, new byte[] { 255, 255, 0, 0 }); + Verify(65536, new byte[] { 0, 0, 1, 0 }); + Verify(16777216, new byte[] { 0, 0, 0, 1 }); + Verify(int.MaxValue, new byte[] { 255, 255, 255, 127 }); + Verify(uint.MaxValue, new byte[] { 255, 255, 255, 255 }); - Assert.That(Slice.FromUInt32(0).ToHexaString(), Is.EqualTo("00")); - Assert.That(Slice.FromUInt32(1).ToHexaString(), Is.EqualTo("01")); - Assert.That(Slice.FromUInt32(255).ToHexaString(), Is.EqualTo("ff")); - Assert.That(Slice.FromUInt32(256).ToHexaString(), Is.EqualTo("0001")); - Assert.That(Slice.FromUInt32(65535).ToHexaString(), Is.EqualTo("ffff")); - Assert.That(Slice.FromUInt32(65536).ToHexaString(), Is.EqualTo("00000100")); - Assert.That(Slice.FromUInt32(int.MaxValue).ToHexaString(), Is.EqualTo("ffffff7f")); - Assert.That(Slice.FromUInt32(uint.MaxValue).ToHexaString(), Is.EqualTo("ffffffff")); + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + uint x = (uint)rnd.Next() + (uint)rnd.Next(); + Slice s = Slice.FromFixedU32(x); + Assert.That(s.Count, Is.EqualTo(4)); + Assert.That(s.ToUInt32(), Is.EqualTo(x)); + } } [Test] public void Test_Slice_ToUInt32() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToUInt32(), Is.EqualTo(0x12U)); - Assert.That(Slice.Create(new byte[] { 0x34, 0x12 }).ToUInt32(), Is.EqualTo(0x1234U)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12 }).ToUInt32(), Is.EqualTo(0x123456U)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12, 0x00 }).ToUInt32(), Is.EqualTo(0x123456U)); - Assert.That(Slice.Create(new byte[] { 0x78, 0x56, 0x34, 0x12 }).ToUInt32(), Is.EqualTo(0x12345678U)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToUInt32(), Is.EqualTo(0x12U)); + Assert.That(new byte[] { 0x34, 0x12 }.AsSlice().ToUInt32(), Is.EqualTo(0x1234U)); + Assert.That(new byte[] { 0x56, 0x34, 0x12 }.AsSlice().ToUInt32(), Is.EqualTo(0x123456U)); + Assert.That(new byte[] { 0x56, 0x34, 0x12, 0x00 }.AsSlice().ToUInt32(), Is.EqualTo(0x123456U)); + Assert.That(new byte[] { 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt32(), Is.EqualTo(0x12345678U)); + + Assert.That(new byte[] { }.AsSlice().ToUInt32(), Is.EqualTo(0U)); + Assert.That(new byte[] { 0 }.AsSlice().ToUInt32(), Is.EqualTo(0U)); + Assert.That(new byte[] { 255 }.AsSlice().ToUInt32(), Is.EqualTo(255U)); + Assert.That(new byte[] { 0, 1 }.AsSlice().ToUInt32(), Is.EqualTo(256U)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToUInt32(), Is.EqualTo(65535U)); + Assert.That(new byte[] { 0, 0, 1 }.AsSlice().ToUInt32(), Is.EqualTo(1U << 16)); + Assert.That(new byte[] { 0, 0, 1, 0 }.AsSlice().ToUInt32(), Is.EqualTo(1U << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToUInt32(), Is.EqualTo((1U << 24) - 1U)); + Assert.That(new byte[] { 0, 0, 0, 1 }.AsSlice().ToUInt32(), Is.EqualTo(1U << 24)); + Assert.That(new byte[] { 255, 255, 255, 127 }.AsSlice().ToUInt32(), Is.EqualTo((uint)int.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255 }.AsSlice().ToUInt32(), Is.EqualTo(uint.MaxValue)); + + Assert.That(() => Slice.Create(5).ToUInt32(), Throws.InstanceOf()); + } - Assert.That(Slice.Create(new byte[] { 0 }).ToUInt32(), Is.EqualTo(0U)); - Assert.That(Slice.Create(new byte[] { 255 }).ToUInt32(), Is.EqualTo(255U)); - Assert.That(Slice.Create(new byte[] { 0, 1 }).ToUInt32(), Is.EqualTo(256U)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToUInt32(), Is.EqualTo(65535U)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1 }).ToUInt32(), Is.EqualTo(1U << 16)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1, 0 }).ToUInt32(), Is.EqualTo(1U << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToUInt32(), Is.EqualTo((1U << 24) - 1U)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 1 }).ToUInt32(), Is.EqualTo(1U << 24)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 127 }).ToUInt32(), Is.EqualTo((uint)int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255 }).ToUInt32(), Is.EqualTo(uint.MaxValue)); + #endregion + + #region Big-Endian + + [Test] + public void Test_Slice_FromUInt32BE() + { + // 32-bit integers should be encoded in big endian, and with 1, 2 or 4 bytes + + void Verify(uint value, string expected) + { + Assert.That(Slice.FromUInt32BE(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12, "12"); + Verify(0x1234, "1234"); + Verify(0x123456, "123456"); + Verify(0x12345678, "12345678"); + + Verify(0, "00"); + Verify(1, "01"); + Verify(255, "FF"); + Verify(256, "0100"); + Verify(65535, "FFFF"); + Verify(65536, "010000"); + Verify(int.MaxValue, "7FFFFFFF"); + Verify(uint.MaxValue, "FFFFFFFF"); } [Test] - public void Test_Slice_ToUInt32BE() + public void Test_Slice_FromFixedU32BE() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToUInt32BE(), Is.EqualTo(0x12U)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34 }).ToUInt32BE(), Is.EqualTo(0x1234U)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56 }).ToUInt32BE(), Is.EqualTo(0x123456U)); - Assert.That(Slice.Create(new byte[] { 0x00, 0x12, 0x34, 0x56 }).ToUInt32BE(), Is.EqualTo(0x123456U)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78 }).ToUInt32BE(), Is.EqualTo(0x12345678U)); + // FromFixedU32BE always produce 4 bytes and uses Big Endian + + void Verify(uint value, byte[] expected) + { + Assert.That(Slice.FromFixedU32BE(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0, new byte[4]); + Verify(1, new byte[] { 0, 0, 0, 1 }); + Verify(256, new byte[] { 0, 0, 1, 0 }); + Verify(ushort.MaxValue, new byte[] { 0, 0, 255, 255 }); + Verify(65536, new byte[] { 0, 1, 0, 0 }); + Verify(16777216, new byte[] { 1, 0, 0, 0 }); + Verify(int.MaxValue, new byte[] { 127, 255, 255, 255 }); + Verify(uint.MaxValue, new byte[] { 255, 255, 255, 255 }); - Assert.That(Slice.Create(new byte[] { 0 }).ToUInt32BE(), Is.EqualTo(0U)); - Assert.That(Slice.Create(new byte[] { 255 }).ToUInt32BE(), Is.EqualTo(255U)); - Assert.That(Slice.Create(new byte[] { 1, 0 }).ToUInt32BE(), Is.EqualTo(256U)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToUInt32BE(), Is.EqualTo(65535U)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0 }).ToUInt32BE(), Is.EqualTo(1U << 16)); - Assert.That(Slice.Create(new byte[] { 0, 1, 0, 0 }).ToUInt32BE(), Is.EqualTo(1U << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToUInt32BE(), Is.EqualTo((1U << 24) - 1U)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0 }).ToUInt32BE(), Is.EqualTo(1U << 24)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255 }).ToUInt32BE(), Is.EqualTo((uint)int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255 }).ToUInt32BE(), Is.EqualTo(uint.MaxValue)); + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + uint x = (uint)rnd.Next() + (uint)rnd.Next(); + Slice s = Slice.FromFixedU32BE(x); + Assert.That(s.Count, Is.EqualTo(4)); + Assert.That(s.ToUInt32BE(), Is.EqualTo(x)); + } } + [Test] + public void Test_Slice_ToUInt32BE() + { + Assert.That(new byte[] { 0x12 }.AsSlice().ToUInt32BE(), Is.EqualTo(0x12U)); + Assert.That(new byte[] { 0x12, 0x34 }.AsSlice().ToUInt32BE(), Is.EqualTo(0x1234U)); + Assert.That(new byte[] { 0x12, 0x34, 0x56 }.AsSlice().ToUInt32BE(), Is.EqualTo(0x123456U)); + Assert.That(new byte[] { 0x00, 0x12, 0x34, 0x56 }.AsSlice().ToUInt32BE(), Is.EqualTo(0x123456U)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78 }.AsSlice().ToUInt32BE(), Is.EqualTo(0x12345678U)); + + Assert.That(new byte[] { }.AsSlice().ToUInt32BE(), Is.EqualTo(0U)); + Assert.That(new byte[] { 0 }.AsSlice().ToUInt32BE(), Is.EqualTo(0U)); + Assert.That(new byte[] { 255 }.AsSlice().ToUInt32BE(), Is.EqualTo(255U)); + Assert.That(new byte[] { 1, 0 }.AsSlice().ToUInt32BE(), Is.EqualTo(256U)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToUInt32BE(), Is.EqualTo(65535U)); + Assert.That(new byte[] { 1, 0, 0 }.AsSlice().ToUInt32BE(), Is.EqualTo(1U << 16)); + Assert.That(new byte[] { 0, 1, 0, 0 }.AsSlice().ToUInt32BE(), Is.EqualTo(1U << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToUInt32BE(), Is.EqualTo((1U << 24) - 1U)); + Assert.That(new byte[] { 1, 0, 0, 0 }.AsSlice().ToUInt32BE(), Is.EqualTo(1U << 24)); + Assert.That(new byte[] { 127, 255, 255, 255 }.AsSlice().ToUInt32BE(), Is.EqualTo((uint)int.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255 }.AsSlice().ToUInt32BE(), Is.EqualTo(uint.MaxValue)); + + Assert.That(() => Slice.Create(5).ToUInt32BE(), Throws.InstanceOf()); + } + + #endregion + + #endregion + + #region 64-bits + [Test] public void Test_Slice_FromUInt64() { // 64-bit integers should be encoded in little endian, and with 1, 2, 4 or 8 bytes - // 0x12 -> { 12 } - // 0x1234 -> { 34 12 } - // 0x123456 -> { 56 34 12 00 } - // 0x12345678 -> { 78 56 34 12 } - // 0x123456789A -> { 9A 78 56 34 12 00 00 00} - // 0x123456789ABC -> { BC 9A 78 56 34 12 00 00} - // 0x123456789ABCDE -> { DE BC 9A 78 56 34 12 00} - // 0x123456789ABCDEF0 -> { F0 DE BC 9A 78 56 34 12 } - - Assert.That(Slice.FromUInt64(0x12UL).ToHexaString(), Is.EqualTo("12")); - Assert.That(Slice.FromUInt64(0x1234UL).ToHexaString(), Is.EqualTo("3412")); - Assert.That(Slice.FromUInt64(0x123456UL).ToHexaString(), Is.EqualTo("56341200")); - Assert.That(Slice.FromUInt64(0x12345678UL).ToHexaString(), Is.EqualTo("78563412")); - Assert.That(Slice.FromUInt64(0x123456789AUL).ToHexaString(), Is.EqualTo("9a78563412000000")); - Assert.That(Slice.FromUInt64(0x123456789ABCUL).ToHexaString(), Is.EqualTo("bc9a785634120000")); - Assert.That(Slice.FromUInt64(0x123456789ABCDEUL).ToHexaString(), Is.EqualTo("debc9a7856341200")); - Assert.That(Slice.FromUInt64(0x123456789ABCDEF0UL).ToHexaString(), Is.EqualTo("f0debc9a78563412")); - - Assert.That(Slice.FromUInt64(0UL).ToHexaString(), Is.EqualTo("00")); - Assert.That(Slice.FromUInt64(1UL).ToHexaString(), Is.EqualTo("01")); - Assert.That(Slice.FromUInt64(255UL).ToHexaString(), Is.EqualTo("ff")); - Assert.That(Slice.FromUInt64(256UL).ToHexaString(), Is.EqualTo("0001")); - Assert.That(Slice.FromUInt64(ushort.MaxValue).ToHexaString(), Is.EqualTo("ffff")); - Assert.That(Slice.FromUInt64(65536UL).ToHexaString(), Is.EqualTo("00000100")); - Assert.That(Slice.FromUInt64(int.MaxValue).ToHexaString(), Is.EqualTo("ffffff7f")); - Assert.That(Slice.FromUInt64(uint.MaxValue).ToHexaString(), Is.EqualTo("ffffffff")); - Assert.That(Slice.FromUInt64(long.MaxValue).ToHexaString(), Is.EqualTo("ffffffffffffff7f")); - Assert.That(Slice.FromUInt64(ulong.MaxValue).ToHexaString(), Is.EqualTo("ffffffffffffffff")); + + void Verify(ulong value, string expected) + { + Assert.That(Slice.FromUInt64(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12UL, "12"); + Verify(0x1234UL, "3412"); + Verify(0x123456UL, "563412"); + Verify(0x12345678UL, "78563412"); + Verify(0x123456789AUL, "9A78563412"); + Verify(0x123456789ABCUL, "BC9A78563412"); + Verify(0x123456789ABCDEUL, "DEBC9A78563412"); + Verify(0x123456789ABCDEF0UL, "F0DEBC9A78563412"); + + Verify(0UL, "00"); + Verify(1UL, "01"); + Verify(255UL, "FF"); + Verify(256UL, "0001"); + Verify(ushort.MaxValue, "FFFF"); + Verify(65536UL, "000001"); + Verify(16777215UL, "FFFFFF"); + Verify(16777216UL, "00000001"); + Verify(int.MaxValue, "FFFFFF7F"); + Verify(16777216UL, "00000001"); + Verify(uint.MaxValue + 1UL, "0000000001"); + Verify(long.MaxValue, "FFFFFFFFFFFFFF7F"); + Verify(ulong.MaxValue, "FFFFFFFFFFFFFFFF"); } + [Test] + public void Test_Slice_FromFixedU64() + { + // FromFixed64 always produce 8 bytes and uses Little Endian + + void Verify(ulong value, byte[] expected) + { + Assert.That(Slice.FromFixedU64(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0UL, new byte[8]); + Verify(1UL, new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }); + Verify(1UL << 8, new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 }); + Verify(1UL << 16, new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 }); + Verify(1UL << 24, new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 }); + Verify(1UL << 32, new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 }); + Verify(1UL << 40, new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 }); + Verify(1UL << 48, new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 }); + Verify(1UL << 56, new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }); + Verify(ushort.MaxValue, new byte[] { 255, 255, 0, 0, 0, 0, 0, 0 }); + Verify(int.MaxValue, new byte[] { 255, 255, 255, 127, 0, 0, 0, 0 }); + Verify(uint.MaxValue, new byte[] { 255, 255, 255, 255, 0, 0, 0, 0 }); + Verify(long.MaxValue, new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }); + Verify(ulong.MaxValue, new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }); + + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + ulong x = (ulong)rnd.Next() * (ulong)rnd.Next(); + Slice s = Slice.FromFixedU64(x); + Assert.That(s.Count, Is.EqualTo(8)); + Assert.That(s.ToUInt64(), Is.EqualTo(x)); + } + } + [Test] public void Test_Slice_ToUInt64() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToUInt64(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x56, 0x34, 0x12, 00 }).ToUInt64(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x78, 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x12345678)); - Assert.That(Slice.Create(new byte[] { 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x123456789A)); - Assert.That(Slice.Create(new byte[] { 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x123456789ABC)); - Assert.That(Slice.Create(new byte[] { 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x123456789ABCDE)); - Assert.That(Slice.Create(new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }).ToUInt64(), Is.EqualTo(0x123456789ABCDEF0)); - - Assert.That(Slice.Create(new byte[] { 0 }).ToUInt64(), Is.EqualTo(0UL)); - Assert.That(Slice.Create(new byte[] { 255 }).ToUInt64(), Is.EqualTo(255UL)); - Assert.That(Slice.Create(new byte[] { 0, 1 }).ToUInt64(), Is.EqualTo(256UL)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToUInt64(), Is.EqualTo(65535UL)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 16)); - Assert.That(Slice.Create(new byte[] { 0, 0, 1, 0 }).ToUInt64(), Is.EqualTo(1UL << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToUInt64(), Is.EqualTo((1UL << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 24)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 32)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 40)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 48)); - Assert.That(Slice.Create(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }).ToUInt64(), Is.EqualTo(1UL << 56)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 127 }).ToUInt64(), Is.EqualTo(int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255 }).ToUInt64(), Is.EqualTo(uint.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }).ToUInt64(), Is.EqualTo(long.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }).ToUInt64(), Is.EqualTo(ulong.MaxValue)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x56, 0x34, 0x12, 00 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x12345678)); + Assert.That(new byte[] { 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456789A)); + Assert.That(new byte[] { 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456789ABC)); + Assert.That(new byte[] { 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456789ABCDE)); + Assert.That(new byte[] { 0xF0, 0xDE, 0xBC, 0x9A, 0x78, 0x56, 0x34, 0x12 }.AsSlice().ToUInt64(), Is.EqualTo(0x123456789ABCDEF0)); + + Assert.That(new byte[] { }.AsSlice().ToUInt64(), Is.EqualTo(0UL)); + Assert.That(new byte[] { 0 }.AsSlice().ToUInt64(), Is.EqualTo(0UL)); + Assert.That(new byte[] { 255 }.AsSlice().ToUInt64(), Is.EqualTo(255UL)); + Assert.That(new byte[] { 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(256UL)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToUInt64(), Is.EqualTo(65535UL)); + Assert.That(new byte[] { 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 16)); + Assert.That(new byte[] { 0, 0, 1, 0 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToUInt64(), Is.EqualTo((1UL << 24) - 1)); + Assert.That(new byte[] { 0, 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 24)); + Assert.That(new byte[] { 0, 0, 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 32)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 40)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 48)); + Assert.That(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }.AsSlice().ToUInt64(), Is.EqualTo(1UL << 56)); + Assert.That(new byte[] { 255, 255, 255, 127 }.AsSlice().ToUInt64(), Is.EqualTo(int.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255 }.AsSlice().ToUInt64(), Is.EqualTo(uint.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 }.AsSlice().ToUInt64(), Is.EqualTo(long.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToUInt64(), Is.EqualTo(ulong.MaxValue)); // should validate the arguments - var x = Slice.Create(new byte[] { 0x78, 0x56, 0x34, 0x12 }); + var x = new byte[] { 0x78, 0x56, 0x34, 0x12 }.AsSlice(); Assert.That(() => MutateOffset(x, -1).ToUInt64(), Throws.InstanceOf()); Assert.That(() => MutateCount(x, 5).ToUInt64(), Throws.InstanceOf()); - Assert.That(() => MutateArray(x, null).ToUInt64(), Throws.InstanceOf()); + Assert.That(() => MutateArray(x, null).ToUInt64(), Throws.InstanceOf()); + } + + [Test] + public void Test_Slice_FromUInt64BE() + { + // 64-bit integers should be encoded in big endian, and using from 1 to 8 bytes + + void Verify(ulong value, string expected) + { + Assert.That(Slice.FromUInt64BE(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0x12UL, "12"); + Verify(0x1234UL, "1234"); + Verify(0x123456UL, "123456"); + Verify(0x12345678UL, "12345678"); + Verify(0x123456789AUL, "123456789A"); + Verify(0x123456789ABCUL, "123456789ABC"); + Verify(0x123456789ABCDEUL, "123456789ABCDE"); + Verify(0x123456789ABCDEF0UL, "123456789ABCDEF0"); + + Verify(0UL, "00"); + Verify(1UL, "01"); + Verify(255UL, "FF"); + Verify(256UL, "0100"); + Verify(ushort.MaxValue, "FFFF"); + Verify(65536UL, "010000"); + Verify(16777215UL, "FFFFFF"); + Verify(16777216UL, "01000000"); + Verify(int.MaxValue, "7FFFFFFF"); + Verify(16777216UL, "01000000"); + Verify(uint.MaxValue + 1UL, "0100000000"); + Verify(long.MaxValue, "7FFFFFFFFFFFFFFF"); + Verify(ulong.MaxValue, "FFFFFFFFFFFFFFFF"); + + } + + [Test] + public void Test_Slice_FromFixedU64BE() + { + // FromFixed64 always produce 8 bytes and uses Big Endian + + void Verify(ulong value, byte[] expected) + { + Assert.That(Slice.FromFixedU64BE(value).GetBytes(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0UL, new byte[8]); + Verify(1L, new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 }); + Verify(1L << 8, new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 }); + Verify(1L << 16, new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 }); + Verify(1L << 24, new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 }); + Verify(1L << 32, new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 }); + Verify(1L << 40, new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 }); + Verify(1L << 48, new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 }); + Verify(1L << 56, new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }); + Verify(ushort.MaxValue, new byte[] { 0, 0, 0, 0, 0, 0, 255, 255 }); + Verify(int.MaxValue, new byte[] { 0, 0, 0, 0, 127, 255, 255, 255 }); + Verify(uint.MaxValue, new byte[] { 0, 0, 0, 0, 255, 255, 255, 255 }); + Verify(long.MaxValue, new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }); + Verify(ulong.MaxValue, new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }); + + var rnd = new Random(); + for (int i = 0; i < 1000; i++) + { + ulong x = (ulong)rnd.Next() * (ulong)rnd.Next(); + Slice s = Slice.FromFixedU64BE(x); + Assert.That(s.Count, Is.EqualTo(8)); + Assert.That(s.ToUInt64BE(), Is.EqualTo(x)); + } } [Test] public void Test_Slice_ToUInt64BE() { - Assert.That(Slice.Create(new byte[] { 0x12 }).ToUInt64BE(), Is.EqualTo(0x12)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34 }).ToUInt64BE(), Is.EqualTo(0x1234)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56 }).ToUInt64BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x00, 0x12, 0x34, 0x56 }).ToUInt64BE(), Is.EqualTo(0x123456)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78 }).ToUInt64BE(), Is.EqualTo(0x12345678)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A }).ToUInt64BE(), Is.EqualTo(0x123456789A)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC }).ToUInt64BE(), Is.EqualTo(0x123456789ABC)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE }).ToUInt64BE(), Is.EqualTo(0x123456789ABCDE)); - Assert.That(Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }).ToUInt64BE(), Is.EqualTo(0x123456789ABCDEF0)); - - Assert.That(Slice.Create(new byte[] { 0 }).ToUInt64BE(), Is.EqualTo(0L)); - Assert.That(Slice.Create(new byte[] { 255 }).ToUInt64BE(), Is.EqualTo(255L)); - Assert.That(Slice.Create(new byte[] { 1, 0 }).ToUInt64BE(), Is.EqualTo(256L)); - Assert.That(Slice.Create(new byte[] { 255, 255 }).ToUInt64BE(), Is.EqualTo(65535L)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 0, 1, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 16)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255 }).ToUInt64BE(), Is.EqualTo((1L << 24) - 1)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 24)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 32)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 40)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 48)); - Assert.That(Slice.Create(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }).ToUInt64BE(), Is.EqualTo(1L << 56)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255 }).ToUInt64BE(), Is.EqualTo(int.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255 }).ToUInt64BE(), Is.EqualTo(uint.MaxValue)); - Assert.That(Slice.Create(new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }).ToUInt64BE(), Is.EqualTo(long.MaxValue)); - Assert.That(Slice.Create(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }).ToUInt64BE(), Is.EqualTo(ulong.MaxValue)); + Assert.That(new byte[] { 0x12 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x12)); + Assert.That(new byte[] { 0x12, 0x34 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x1234)); + Assert.That(new byte[] { 0x12, 0x34, 0x56 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x00, 0x12, 0x34, 0x56 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x12345678)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456789A)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456789ABC)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456789ABCDE)); + Assert.That(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }.AsSlice().ToUInt64BE(), Is.EqualTo(0x123456789ABCDEF0)); + + Assert.That(new byte[] { }.AsSlice().ToUInt64BE(), Is.EqualTo(0L)); + Assert.That(new byte[] { 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(0L)); + Assert.That(new byte[] { 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(255L)); + Assert.That(new byte[] { 1, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(256L)); + Assert.That(new byte[] { 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(65535L)); + Assert.That(new byte[] { 1, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 0, 1, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 16)); + Assert.That(new byte[] { 255, 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo((1L << 24) - 1)); + Assert.That(new byte[] { 1, 0, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 24)); + Assert.That(new byte[] { 1, 0, 0, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 32)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 40)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 48)); + Assert.That(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 }.AsSlice().ToUInt64BE(), Is.EqualTo(1L << 56)); + Assert.That(new byte[] { 127, 255, 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(int.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(uint.MaxValue)); + Assert.That(new byte[] { 127, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(long.MaxValue)); + Assert.That(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 }.AsSlice().ToUInt64BE(), Is.EqualTo(ulong.MaxValue)); // should validate the arguments - var x = Slice.Create(new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }); + var x = new byte[] { 0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0 }.AsSlice(); Assert.That(() => MutateOffset(x, -1).ToUInt64BE(), Throws.InstanceOf()); Assert.That(() => MutateCount(x, 9).ToUInt64BE(), Throws.InstanceOf()); Assert.That(() => MutateArray(x, null).ToUInt64BE(), Throws.InstanceOf()); } + #endregion + + #endregion + + #region Floating Point... + + private static string SwapHexa(string hexa) + { + char[] res = new char[hexa.Length]; + int p = 0; + for (int i = hexa.Length - 2; i >= 0; i -= 2, p += 2) + { + res[i + 0] = hexa[p + 0]; + res[i + 1] = hexa[p + 1]; + } + return new string(res); + } + + [Test] + public void Test_Slice_FromSingle() + { + void Verify(float value, string expected) + { + Assert.That(Slice.FromSingle(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0} (Little Endian)", value); + Assert.That(Slice.FromSingleBE(value).ToHexaString(), Is.EqualTo(SwapHexa(expected)), "Invalid encoding for {0} (Big Endian)", value); + } + + Verify(0f, "00000000"); + Verify(1f, "0000803F"); + Verify(-1f, "000080BF"); + Verify(10f, "00002041"); + Verify(0.1f, "CDCCCC3D"); + Verify(0.5f, "0000003F"); + + Verify(1f / 3f, "ABAAAA3E"); + Verify((float) Math.PI, "DB0F4940"); + Verify((float) Math.E, "54F82D40"); + + Verify(float.NaN, "0000C0FF"); + Verify(float.Epsilon, "01000000"); + Verify(float.MaxValue, "FFFF7F7F"); + Verify(float.MinValue, "FFFF7FFF"); + Verify(float.PositiveInfinity, "0000807F"); + Verify(float.NegativeInfinity, "000080FF"); + } + + [Test] + public void Test_Slice_ToSingle() + { + void Verify(string value, float expected) + { + Assert.That(Slice.FromHexa(value).ToSingle(), Is.EqualTo(expected), "Invalid decoding for '{0}' (Little Endian)", value); + Assert.That(Slice.FromHexa(SwapHexa(value)).ToSingleBE(), Is.EqualTo(expected), "Invalid decoding for '{0}' (Big Endian)", value); + } + + Assert.That(Slice.Empty.ToSingle(), Is.EqualTo(0d)); + Verify("00000000", 0f); + Verify("0000803F", 1f); + Verify("000080BF", -1f); + Verify("00002041", 10f); + Verify("CDCCCC3D", 0.1f); + Verify("0000003F", 0.5f); + + Verify("ABAAAA3E", 1f / 3f); + Verify("DB0F4940", (float) Math.PI); + Verify("54F82D40", (float) Math.E); + + Verify("0000C0FF", float.NaN); + Verify("01000000", float.Epsilon); + Verify("FFFF7F7F", float.MaxValue); + Verify("FFFF7FFF", float.MinValue); + Verify("0000807F", float.PositiveInfinity); + Verify("000080FF", float.NegativeInfinity); + + Assert.That(() => Slice.Create(5).ToSingle(), Throws.InstanceOf()); + Assert.That(() => Slice.Create(3).ToSingle(), Throws.InstanceOf()); + } + + [Test] + public void Test_Slice_FromDouble() + { + void Verify(double value, string expected) + { + Assert.That(Slice.FromDouble(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0} (Little Endian)", value); + Assert.That(Slice.FromDoubleBE(value).ToHexaString(), Is.EqualTo(SwapHexa(expected)), "Invalid encoding for {0} (Big Endian)", value); + } + + Verify(0d, "0000000000000000"); + Verify(1d, "000000000000F03F"); + Verify(-1d, "000000000000F0BF"); + Verify(10d, "0000000000002440"); + Verify(0.1d, "9A9999999999B93F"); + Verify(0.5d, "000000000000E03F"); + + Verify(1d / 3d, "555555555555D53F"); + Verify(Math.PI, "182D4454FB210940"); + Verify(Math.E, "6957148B0ABF0540"); + + Verify(double.NaN, "000000000000F8FF"); + Verify(double.Epsilon, "0100000000000000"); + Verify(double.MaxValue, "FFFFFFFFFFFFEF7F"); + Verify(double.MinValue, "FFFFFFFFFFFFEFFF"); + Verify(double.PositiveInfinity, "000000000000F07F"); + Verify(double.NegativeInfinity, "000000000000F0FF"); + + } + + [Test] + public void Test_Slice_ToDouble() + { + void Verify(string value, double expected) + { + Assert.That(Slice.FromHexa(value).ToDouble(), Is.EqualTo(expected), "Invalid decoding for '{0}' (Little Endian)", value); + Assert.That(Slice.FromHexa(SwapHexa(value)).ToDoubleBE(), Is.EqualTo(expected), "Invalid decoding for '{0}' (Big Endian)", value); + } + + Verify("", 0d); + Verify("0000000000000000", 0d); + Verify("000000000000F03F", 1d); + Verify("000000000000F0BF", -1d); + Verify("0000000000002440", 10d); + Verify("9A9999999999B93F", 0.1d); + Verify("000000000000E03F", 0.5d); + + Verify("555555555555D53F", 1d / 3d); + Verify("182D4454FB210940", Math.PI); + Verify("6957148B0ABF0540", Math.E); + + Verify("000000000000F8FF", double.NaN); + Verify("0100000000000000", double.Epsilon); + Verify("FFFFFFFFFFFFEF7F", double.MaxValue); + Verify("FFFFFFFFFFFFEFFF", double.MinValue); + Verify("000000000000F07F", double.PositiveInfinity); + Verify("000000000000F0FF", double.NegativeInfinity); + + Assert.That(() => Slice.Create(9).ToDouble(), Throws.InstanceOf()); + Assert.That(() => Slice.Create(7).ToDouble(), Throws.InstanceOf()); + } + + [Test] + public void Test_Slice_FromDecimal() + { + void Verify(decimal value, string expected) + { + Assert.That(Slice.FromDecimal(value).ToHexaString(), Is.EqualTo(expected), "Invalid encoding for {0}", value); + } + + Verify(0m, "00000000000000000000000000000000"); + Verify(1m, "00000000000000000100000000000000"); + Verify(-1m, "00000080000000000100000000000000"); + Verify(10m, "00000000000000000A00000000000000"); + Verify(0.1m, "00000100000000000100000000000000"); + Verify(0.5m, "00000100000000000500000000000000"); + + Verify(1m / 3m, "00001C00CA44C50A55555505CB00B714"); + Verify((decimal) Math.PI, "00000E000000000083246AE7B91D0100"); + Verify((decimal)Math.E, "00000E0000000000D04947EE39F70000"); + + Verify(decimal.MaxValue, "00000000FFFFFFFFFFFFFFFFFFFFFFFF"); + Verify(decimal.MinValue, "00000080FFFFFFFFFFFFFFFFFFFFFFFF"); + + } + + [Test] + public void Test_Slice_ToDecimal() + { + void Verify(string value, decimal expected) + { + Assert.That(Slice.FromHexa(value).ToDecimal(), Is.EqualTo(expected), "Invalid decoding for '{0}'", value); + } + + Verify("", 0m); + Verify("00000000000000000000000000000000", 0m); + Verify("00000000000000000100000000000000", 1m); + Verify("00000080000000000100000000000000", -1m); + Verify("00000000000000000A00000000000000", 10m); + Verify("00000100000000000100000000000000", 0.1m); + Verify("00000100000000000500000000000000", 0.5m); + + Verify("00001C00CA44C50A55555505CB00B714", 1m / 3m); + Verify("00000E000000000083246AE7B91D0100", (decimal) Math.PI); + Verify("00000E0000000000D04947EE39F70000", (decimal) Math.E); + + Verify("00000000FFFFFFFFFFFFFFFFFFFFFFFF", decimal.MaxValue); + Verify("00000080FFFFFFFFFFFFFFFFFFFFFFFF", decimal.MinValue); + + Assert.That(() => Slice.Create(15).ToDecimal(), Throws.InstanceOf()); + Assert.That(() => Slice.Create(17).ToDecimal(), Throws.InstanceOf()); + } + + #endregion + + #region UUIDs... + [Test] public void Test_Slice_FromGuid() { // Verify that System.GUID are stored as UUIDs using RFC 4122, and not their natural in-memory format - Slice slice; - // empty guid should be all zeroes - slice = Slice.FromGuid(Guid.Empty); + Slice slice = Slice.FromGuid(Guid.Empty); Assert.That(slice.ToHexaString(), Is.EqualTo("00000000000000000000000000000000")); // GUIDs should be stored using RFC 4122 (big endian) @@ -642,35 +1531,36 @@ public void Test_Slice_FromGuid() // byte order should follow the string! slice = Slice.FromGuid(guid); - Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899aabbccddeeff"), "Slice.FromGuid() should use the RFC 4122 encoding"); + Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899AABBCCDDEEFF"), "Slice.FromGuid() should use the RFC 4122 encoding"); // but guid in memory should follow MS format - slice = Slice.Create(guid.ToByteArray()); // <-- this is BAD, don't try this at home ! - Assert.That(slice.ToHexaString(), Is.EqualTo("33221100554477668899aabbccddeeff")); + slice = guid.ToByteArray().AsSlice(); // <-- this is BAD, don't try this at home ! + Assert.That(slice.ToHexaString(), Is.EqualTo("33221100554477668899AABBCCDDEEFF")); } [Test] public void Test_Slice_ToGuid() { - Slice slice; - Guid guid; + // nil or empty should return Guid.Empty + Assert.That(Slice.Nil.ToGuid(), Is.EqualTo(Guid.Empty)); + Assert.That(Slice.Empty.ToGuid(), Is.EqualTo(Guid.Empty)); - // all zeroes should return Guid.Empty - slice = Slice.Create(16); + // all zeroes should also return Guid.Empty + Slice slice = Slice.Create(16); Assert.That(slice.ToGuid(), Is.EqualTo(Guid.Empty)); // RFC 4122 encoded UUIDs should be properly reversed when converted to System.GUID slice = Slice.FromHexa("00112233445566778899aabbccddeeff"); - guid = slice.ToGuid(); + Guid guid = slice.ToGuid(); Assert.That(guid.ToString(), Is.EqualTo("00112233-4455-6677-8899-aabbccddeeff"), "slice.ToGuid() should convert RFC 4122 encoded UUIDs into native System.Guid"); // round-trip guid = Guid.NewGuid(); Assert.That(Slice.FromGuid(guid).ToGuid(), Is.EqualTo(guid)); - Assert.That(Slice.FromAscii(guid.ToString()).ToGuid(), Is.EqualTo(guid), "String literals should also be converted if they match the expected format"); + Assert.That(Slice.FromStringAscii(guid.ToString()).ToGuid(), Is.EqualTo(guid), "String literals should also be converted if they match the expected format"); - Assert.That(() => Slice.FromAscii("random text").ToGuid(), Throws.InstanceOf()); + Assert.That(() => Slice.FromStringAscii("random text").ToGuid(), Throws.InstanceOf()); // should validate the arguments var x = Slice.FromGuid(guid); @@ -685,10 +1575,8 @@ public void Test_Slice_FromUuid128() { // Verify that FoundationDb.Client.Uuid are stored as 128-bit UUIDs using RFC 4122 - Slice slice; - // empty guid should be all zeroes - slice = Slice.FromUuid128(Uuid128.Empty); + Slice slice = Slice.FromUuid128(Uuid128.Empty); Assert.That(slice.ToHexaString(), Is.EqualTo("00000000000000000000000000000000")); // UUIDs should be stored using RFC 4122 (big endian) @@ -696,21 +1584,24 @@ public void Test_Slice_FromUuid128() // byte order should follow the string! slice = Slice.FromUuid128(uuid); - Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899aabbccddeeff"), "Slice.FromUuid() should preserve RFC 4122 ordering"); + Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899AABBCCDDEEFF"), "Slice.FromUuid() should preserve RFC 4122 ordering"); // ToByteArray() should also be safe - slice = Slice.Create(uuid.ToByteArray()); - Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899aabbccddeeff")); + slice = uuid.ToByteArray().AsSlice(); + Assert.That(slice.ToHexaString(), Is.EqualTo("00112233445566778899AABBCCDDEEFF")); } [Test] public void Test_Slice_ToUuid128() { - Slice slice; - Uuid128 uuid; - - // all zeroes should return Uuid.Empty - slice = Slice.Create(16); + // nil or empty should return Uuid128.Empty + Uuid128 uuid = Slice.Nil.ToUuid128(); + Assert.That(uuid, Is.EqualTo(Uuid128.Empty)); + uuid = Slice.Empty.ToUuid128(); + Assert.That(uuid, Is.EqualTo(Uuid128.Empty)); + + // all zeroes should also return Uuid128.Empty + Slice slice = Slice.Create(16); Assert.That(slice.ToUuid128(), Is.EqualTo(Uuid128.Empty)); // RFC 4122 encoded UUIDs should not keep the byte ordering @@ -722,9 +1613,9 @@ public void Test_Slice_ToUuid128() uuid = Uuid128.NewUuid(); Assert.That(Slice.FromUuid128(uuid).ToUuid128(), Is.EqualTo(uuid)); - Assert.That(Slice.FromAscii(uuid.ToString()).ToUuid128(), Is.EqualTo(uuid), "String literals should also be converted if they match the expected format"); + Assert.That(Slice.FromStringAscii(uuid.ToString()).ToUuid128(), Is.EqualTo(uuid), "String literals should also be converted if they match the expected format"); - Assert.That(() => Slice.FromAscii("random text").ToUuid128(), Throws.InstanceOf()); + Assert.That(() => Slice.FromStringAscii("random text").ToUuid128(), Throws.InstanceOf()); // should validate the arguments var x = Slice.FromUuid128(uuid); @@ -738,30 +1629,32 @@ public void Test_Slice_FromUuid64() { // Verify that FoundationDb.Client.Uuid64 are stored as 64-bit UUIDs in big-endian - Slice slice; - // empty guid should be all zeroes - slice = Slice.FromUuid64(Uuid64.Empty); + Slice slice = Slice.FromUuid64(Uuid64.Empty); Assert.That(slice.ToHexaString(), Is.EqualTo("0000000000000000")); // UUIDs should be stored in lexicographical order - var uuid = new Uuid64("01234567-89abcdef"); + var uuid = Uuid64.Parse("01234567-89abcdef"); // byte order should follow the string! slice = Slice.FromUuid64(uuid); - Assert.That(slice.ToHexaString(), Is.EqualTo("0123456789abcdef"), "Slice.FromUuid64() should preserve ordering"); + Assert.That(slice.ToHexaString(), Is.EqualTo("0123456789ABCDEF"), "Slice.FromUuid64() should preserve ordering"); // ToByteArray() should also be safe - slice = Slice.Create(uuid.ToByteArray()); - Assert.That(slice.ToHexaString(), Is.EqualTo("0123456789abcdef")); + slice = uuid.ToByteArray().AsSlice(); + Assert.That(slice.ToHexaString(), Is.EqualTo("0123456789ABCDEF")); } [Test] public void Test_Slice_ToUuid64() { - Uuid64 uuid; + // nil or empty should return Uuid64.Empty + Uuid64 uuid = Slice.Nil.ToUuid64(); + Assert.That(uuid, Is.EqualTo(Uuid64.Empty)); + uuid = Slice.Empty.ToUuid64(); + Assert.That(uuid, Is.EqualTo(Uuid64.Empty)); - // all zeroes should return Uuid.Empty + // all zeroes should also return Uuid64.Empty uuid = Slice.Create(8).ToUuid64(); Assert.That(uuid, Is.EqualTo(Uuid64.Empty)); @@ -773,9 +1666,9 @@ public void Test_Slice_ToUuid64() uuid = Uuid64.NewUuid(); Assert.That(Slice.FromUuid64(uuid).ToUuid64(), Is.EqualTo(uuid)); - Assert.That(Slice.FromAscii(uuid.ToString()).ToUuid64(), Is.EqualTo(uuid), "String literals should also be converted if they match the expected format"); + Assert.That(Slice.FromStringAscii(uuid.ToString()).ToUuid64(), Is.EqualTo(uuid), "String literals should also be converted if they match the expected format"); - Assert.That(() => Slice.FromAscii("random text").ToUuid64(), Throws.InstanceOf()); + Assert.That(() => Slice.FromStringAscii("random text").ToUuid64(), Throws.InstanceOf()); // should validate the arguments var x = Slice.FromUuid64(uuid); @@ -784,111 +1677,44 @@ public void Test_Slice_ToUuid64() Assert.That(() => MutateArray(x, null).ToUuid64(), Throws.InstanceOf()); } - [Test] - public void Test_Slice_FromFixed32() - { - // FromFixed32 always produce 4 bytes and uses Little Endian - - Assert.That(Slice.FromFixed32(0).GetBytes(), Is.EqualTo(new byte[4])); - Assert.That(Slice.FromFixed32(1).GetBytes(), Is.EqualTo(new byte[] { 1, 0, 0, 0 })); - Assert.That(Slice.FromFixed32(256).GetBytes(), Is.EqualTo(new byte[] { 0, 1, 0, 0 })); - Assert.That(Slice.FromFixed32(65536).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 1, 0 })); - Assert.That(Slice.FromFixed32(16777216).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 1 })); - Assert.That(Slice.FromFixed32(short.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 127, 0, 0 })); - Assert.That(Slice.FromFixed32(int.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 127 })); - - Assert.That(Slice.FromFixed32(-1).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed32(-256).GetBytes(), Is.EqualTo(new byte[] { 0, 255, 255, 255 })); - Assert.That(Slice.FromFixed32(-65536).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 255, 255 })); - Assert.That(Slice.FromFixed32(-16777216).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 255 })); - Assert.That(Slice.FromFixed32(int.MinValue).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 128 })); - - var rnd = new Random(); - for (int i = 0; i < 1000; i++) - { - int x = rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); - Slice s = Slice.FromFixed32(x); - Assert.That(s.Count, Is.EqualTo(4)); - Assert.That(s.ToInt32(), Is.EqualTo(x)); - } - } + #endregion [Test] - public void Test_Slice_FromFixed64() + public void Test_Slice_FromBase64() { - // FromFixed64 always produce 8 bytes and uses Little Endian - - Assert.That(Slice.FromFixed64(0L).GetBytes(), Is.EqualTo(new byte[8])); - Assert.That(Slice.FromFixed64(1L).GetBytes(), Is.EqualTo(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 8).GetBytes(), Is.EqualTo(new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 16).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 24).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 32).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 40).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 })); - Assert.That(Slice.FromFixed64(1L << 48).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 })); - Assert.That(Slice.FromFixed64(1L << 56).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 })); - Assert.That(Slice.FromFixed64(short.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 127, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(int.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 127, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixed64(long.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 })); - - Assert.That(Slice.FromFixed64(-1L).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed64(-256L).GetBytes(), Is.EqualTo(new byte[] { 0, 255, 255, 255, 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed64(-65536L).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 255, 255, 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed64(-16777216L).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 255, 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed64(-4294967296L).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 255, 255, 255, 255 })); - Assert.That(Slice.FromFixed64(long.MinValue).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 0, 0, 128 })); + // numl string is Nil slice + Slice slice = Slice.FromBase64(default(string)); + Assert.That(slice, Is.EqualTo(Slice.Nil)); - var rnd = new Random(); - for (int i = 0; i < 1000; i++) - { - long x = (long)rnd.Next() * rnd.Next() * (rnd.Next(2) == 0 ? +1 : -1); - Slice s = Slice.FromFixed64(x); - Assert.That(s.Count, Is.EqualTo(8)); - Assert.That(s.ToInt64(), Is.EqualTo(x)); - } - } - - [Test] - public void Test_Slice_FromFixedU64() - { - // FromFixed64 always produce 8 bytes and uses Little Endian + // empty string is empty slice + slice = Slice.FromBase64(""); + Assert.That(slice, Is.EqualTo(Slice.Empty)); - Assert.That(Slice.FromFixedU64(0UL).GetBytes(), Is.EqualTo(new byte[8])); - Assert.That(Slice.FromFixedU64(1UL).GetBytes(), Is.EqualTo(new byte[] { 1, 0, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 8).GetBytes(), Is.EqualTo(new byte[] { 0, 1, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 16).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 1, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 24).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 1, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 32).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 1, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 40).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 1, 0, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 48).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 0, 1, 0 })); - Assert.That(Slice.FromFixedU64(1UL << 56).GetBytes(), Is.EqualTo(new byte[] { 0, 0, 0, 0, 0, 0, 0, 1 })); - Assert.That(Slice.FromFixedU64(ushort.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 0, 0, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(int.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 127, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(uint.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255, 0, 0, 0, 0 })); - Assert.That(Slice.FromFixedU64(long.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255, 255, 255, 255, 127 })); - Assert.That(Slice.FromFixedU64(ulong.MaxValue).GetBytes(), Is.EqualTo(new byte[] { 255, 255, 255, 255, 255, 255, 255, 255 })); + // UUIDs should be stored in lexicographical order + slice = Slice.FromBase64(Convert.ToBase64String(Encoding.UTF8.GetBytes("Hello, World!"))); + Assert.That(slice.ToUnicode(), Is.EqualTo("Hello, World!")); - var rnd = new Random(); - for (int i = 0; i < 1000; i++) - { - ulong x = (ulong)rnd.Next() * (ulong)rnd.Next(); - Slice s = Slice.FromFixedU64(x); - Assert.That(s.Count, Is.EqualTo(8)); - Assert.That(s.ToUInt64(), Is.EqualTo(x)); - } + // malformed + Assert.That(() => Slice.FromBase64(Convert.ToBase64String(Encoding.UTF8.GetBytes("Hello, World!")).Substring(1)), Throws.InstanceOf()); + Assert.That(() => Slice.FromBase64("This is not a base64 string!"), Throws.InstanceOf()); } + #region Equality / Comparison / HashCodes... + [Test] + [SuppressMessage("ReSharper", "EqualExpressionComparison")] public void Test_Slice_Equality() { - - var a = Slice.Create(new byte[] { 1, 2, 3 }); - var b = Slice.Create(new byte[] { 1, 2, 3 }); - var c = Slice.Create(new byte[] { 0, 1, 2, 3, 4 }, 1, 3); - var x = Slice.Create(new byte[] { 4, 5, 6 }); - var y = Slice.Create(new byte[] { 1, 2, 3 }, 0, 2); - var z = Slice.Create(new byte[] { 1, 2, 3, 4 }); - +#pragma warning disable 1718 + // a == b == c && x != y && a != x + var a = new byte[] { 1, 2, 3 }.AsSlice(); + var b = new byte[] { 1, 2, 3 }.AsSlice(); + var c = new byte[] { 0, 1, 2, 3, 4 }.AsSlice(1, 3); + var x = new byte[] { 4, 5, 6 }.AsSlice(); + var y = new byte[] { 1, 2, 3 }.AsSlice(0, 2); + var z = new byte[] { 1, 2, 3, 4 }.AsSlice(); + + // IEquatable // equals Assert.That(a, Is.EqualTo(a)); Assert.That(a, Is.EqualTo(b)); @@ -899,23 +1725,68 @@ public void Test_Slice_Equality() Assert.That(c, Is.EqualTo(a)); Assert.That(c, Is.EqualTo(b)); Assert.That(c, Is.EqualTo(c)); - // not equals Assert.That(a, Is.Not.EqualTo(x)); Assert.That(a, Is.Not.EqualTo(y)); Assert.That(a, Is.Not.EqualTo(z)); + + // Default Comparer + // equals + Assert.That(Slice.Comparer.Default.Equals(a, a), Is.True); + Assert.That(Slice.Comparer.Default.Equals(a, b), Is.True); + Assert.That(Slice.Comparer.Default.Equals(a, c), Is.True); + Assert.That(Slice.Comparer.Default.Equals(b, a), Is.True); + Assert.That(Slice.Comparer.Default.Equals(b, b), Is.True); + Assert.That(Slice.Comparer.Default.Equals(b, c), Is.True); + Assert.That(Slice.Comparer.Default.Equals(c, a), Is.True); + Assert.That(Slice.Comparer.Default.Equals(c, b), Is.True); + Assert.That(Slice.Comparer.Default.Equals(c, c), Is.True); + // not equals + Assert.That(Slice.Comparer.Default.Equals(a, x), Is.False); + Assert.That(Slice.Comparer.Default.Equals(a, y), Is.False); + Assert.That(Slice.Comparer.Default.Equals(a, z), Is.False); + + // Operators + // == + Assert.That(a == a, Is.True); + Assert.That(a == b, Is.True); + Assert.That(a == c, Is.True); + Assert.That(b == a, Is.True); + Assert.That(b == b, Is.True); + Assert.That(b == c, Is.True); + Assert.That(c == a, Is.True); + Assert.That(c == b, Is.True); + Assert.That(c == c, Is.True); + Assert.That(a == x, Is.False); + Assert.That(a == y, Is.False); + Assert.That(a == z, Is.False); + // != + Assert.That(a != a, Is.False); + Assert.That(a != b, Is.False); + Assert.That(a != c, Is.False); + Assert.That(b != a, Is.False); + Assert.That(b != b, Is.False); + Assert.That(b != c, Is.False); + Assert.That(c != a, Is.False); + Assert.That(c != b, Is.False); + Assert.That(c != c, Is.False); + Assert.That(a != x, Is.True); + Assert.That(a != y, Is.True); + Assert.That(a != z, Is.True); +#pragma warning restore 1718 + } [Test] public void Test_Slice_Equals_Slice() { - var a = Slice.Create(new byte[] { 1, 2, 3 }); - var b = Slice.Create(new byte[] { 1, 2, 3 }); - var c = Slice.Create(new byte[] { 0, 1, 2, 3, 4 }, 1, 3); - var x = Slice.Create(new byte[] { 4, 5, 6 }); - var y = Slice.Create(new byte[] { 1, 2, 3 }, 0, 2); - var z = Slice.Create(new byte[] { 1, 2, 3, 4 }); + var a = new byte[] { 1, 2, 3 }.AsSlice(); + var b = new byte[] { 1, 2, 3 }.AsSlice(); + var c = new byte[] { 0, 1, 2, 3, 4 }.AsSlice(1, 3); + var x = new byte[] { 4, 5, 6 }.AsSlice(); + var y = new byte[] { 1, 2, 3 }.AsSlice(0, 2); + var z = new byte[] { 1, 2, 3, 4 }.AsSlice(); // equals Assert.That(a.Equals(a), Is.True); @@ -943,37 +1814,37 @@ public void Test_Slice_Equals_Slice() [Test] public void Test_Slice_Equality_Corner_Cases() { - Assert.That(Slice.Create(null), Is.EqualTo(Slice.Nil)); - Assert.That(Slice.Create(new byte[0]), Is.EqualTo(Slice.Empty)); - - Assert.That(Slice.Create(null) == Slice.Nil, Is.True, "null == Nil"); - Assert.That(Slice.Create(null) == Slice.Empty, Is.False, "null != Empty"); - Assert.That(Slice.Create(new byte[0]) == Slice.Empty, Is.True, "[0] == Empty"); - Assert.That(Slice.Create(new byte[0]) == Slice.Nil, Is.False, "[0] != Nill"); + Assert.That(default(byte[]).AsSlice(), Is.EqualTo(Slice.Nil)); + Assert.That(new byte[0].AsSlice(), Is.EqualTo(Slice.Empty)); + + Assert.That(default(byte[]).AsSlice() == Slice.Nil, Is.True, "null == Nil"); + Assert.That(default(byte[]).AsSlice() == Slice.Empty, Is.False, "null != Empty"); + Assert.That(new byte[0].AsSlice() == Slice.Empty, Is.True, "[0] == Empty"); + Assert.That(new byte[0].AsSlice() == Slice.Nil, Is.False, "[0] != Nill"); // "slice == null" should be the equivalent to "slice.IsNull" so only true for Slice.Nil Assert.That(Slice.Nil == null, Is.True, "'Slice.Nil == null' is true"); Assert.That(Slice.Empty == null, Is.False, "'Slice.Empty == null' is false"); - Assert.That(Slice.FromByte(1) == null, Is.False, "'{1} == null' is false"); + Assert.That(Slice.FromByte(1) == null, Is.False, "'[1] == null' is false"); Assert.That(null == Slice.Nil, Is.True, "'Slice.Nil == null' is true"); Assert.That(null == Slice.Empty, Is.False, "'Slice.Empty == null' is false"); - Assert.That(null == Slice.FromByte(1), Is.False, "'{1} == null' is false"); + Assert.That(null == Slice.FromByte(1), Is.False, "'[1] == null' is false"); // "slice != null" should be the equivalent to "slice.HasValue" so only false for Slice.Nil Assert.That(Slice.Nil != null, Is.False, "'Slice.Nil != null' is false"); Assert.That(Slice.Empty != null, Is.True, "'Slice.Empty != null' is true"); - Assert.That(Slice.FromByte(1) != null, Is.True, "'{1} != null' is true"); + Assert.That(Slice.FromByte(1) != null, Is.True, "'[1] != null' is true"); Assert.That(null != Slice.Nil, Is.False, "'Slice.Nil != null' is false"); Assert.That(null != Slice.Empty, Is.True, "'Slice.Empty != null' is true"); - Assert.That(null != Slice.FromByte(1), Is.True, "'{1} != null' is true"); + Assert.That(null != Slice.FromByte(1), Is.True, "'[1] != null' is true"); } [Test] public void Test_Slice_Equality_TwoByteArrayWithSameContentShouldReturnTrue() { - var s1 = Slice.FromAscii("abcd"); - var s2 = Slice.FromAscii("abcd"); - Assert.IsTrue(s1.Equals(s2), "'abcd' should equals 'abcd'"); + var s1 = Slice.FromStringAscii("abcd"); + var s2 = Slice.FromStringAscii("abcd"); + Assert.That(s1.Equals(s2), Is.True, "'abcd' should equals 'abcd'"); } [Test] @@ -981,17 +1852,17 @@ public void Test_Slice_Equality_TwoByteArrayWithSameContentFromSameOriginalBuffe { var origin = System.Text.Encoding.ASCII.GetBytes("abcdabcd"); var a1 = new ArraySegment(origin, 0, 4); //"abcd", refer first part of origin buffer - var s1 = Slice.Create(a1); // + var s1 = a1.AsSlice(); // var a2 = new ArraySegment(origin, 4, 4);//"abcd", refer second part of origin buffer - var s2 = Slice.Create(a2); - Assert.IsTrue(s1.Equals(s2), "'abcd' should equals 'abcd'"); + var s2 = a2.AsSlice(); + Assert.That(s1.Equals(s2), Is.True, "'abcd' should equals 'abcd'"); } [Test] public void Test_Slice_Equality_Malformed() { - var good = Slice.FromAscii("good"); - var evil = Slice.FromAscii("evil"); + var good = Slice.FromStringAscii("good"); + var evil = Slice.FromStringAscii("evil"); // argument should be validated Assert.That(() => good.Equals(MutateOffset(evil, -1)), Throws.InstanceOf()); @@ -1019,6 +1890,12 @@ public void Test_Slice_Hash_Code() Assert.That(Slice.FromString("zabcz").Substring(1, 3).GetHashCode(), Is.EqualTo(Slice.FromString("abc").GetHashCode()), "Hashcode should not depend on the offset in the array"); Assert.That(Slice.FromString("abc").GetHashCode(), Is.Not.EqualTo(Slice.FromString("abcd").GetHashCode()), "Hashcode should include all the bytes"); + Assert.That(Slice.Comparer.Default.GetHashCode(Slice.Nil), Is.EqualTo(0), "Nil hashcode should always be 0"); + Assert.That(Slice.Comparer.Default.GetHashCode(Slice.Empty), Is.Not.EqualTo(0), "Empty hashcode should not be equal to 0"); + Assert.That(Slice.Comparer.Default.GetHashCode(Slice.FromString("abc")), Is.EqualTo(Slice.FromString("abc").GetHashCode()), "Hashcode should not depend on the backing array"); + Assert.That(Slice.Comparer.Default.GetHashCode(Slice.FromString("zabcz").Substring(1, 3)), Is.EqualTo(Slice.FromString("abc").GetHashCode()), "Hashcode should not depend on the offset in the array"); + Assert.That(Slice.Comparer.Default.GetHashCode(Slice.FromString("abc")), Is.Not.EqualTo(Slice.FromString("abcd").GetHashCode()), "Hashcode should include all the bytes"); + // should validate the arguments var x = Slice.FromString("evil"); Assert.That(() => MutateOffset(x, -1).GetHashCode(), Throws.InstanceOf()); @@ -1027,30 +1904,84 @@ public void Test_Slice_Hash_Code() } [Test] + [SuppressMessage("ReSharper", "EqualExpressionComparison")] public void Test_Slice_Comparison() { - var a = Slice.FromAscii("a"); - var ab = Slice.FromAscii("ab"); - var abc = Slice.FromAscii("abc"); - var abc2 = Slice.FromAscii("abc"); // same bytes but different buffer - var b = Slice.FromAscii("b"); - +#pragma warning disable 1718 + var a = Slice.FromStringAscii("a"); + var ab = Slice.FromStringAscii("ab"); + var abc = Slice.FromStringAscii("abc"); + var abc2 = Slice.FromStringAscii("abc"); // same bytes but different buffer + var b = Slice.FromStringAscii("b"); + + // CompateTo // a = b Assert.That(a.CompareTo(a), Is.EqualTo(0)); Assert.That(ab.CompareTo(ab), Is.EqualTo(0)); Assert.That(abc.CompareTo(abc), Is.EqualTo(0)); Assert.That(abc.CompareTo(abc2), Is.EqualTo(0)); - // a < b Assert.That(a.CompareTo(b), Is.LessThan(0)); Assert.That(a.CompareTo(ab), Is.LessThan(0)); Assert.That(a.CompareTo(abc), Is.LessThan(0)); - // a > b Assert.That(b.CompareTo(a), Is.GreaterThan(0)); Assert.That(b.CompareTo(ab), Is.GreaterThan(0)); Assert.That(b.CompareTo(abc), Is.GreaterThan(0)); - + + // Default Comparer + // a = b + Assert.That(Slice.Comparer.Default.Compare(a, a), Is.EqualTo(0)); + Assert.That(Slice.Comparer.Default.Compare(ab, ab), Is.EqualTo(0)); + Assert.That(Slice.Comparer.Default.Compare(abc, abc), Is.EqualTo(0)); + Assert.That(Slice.Comparer.Default.Compare(abc, abc2), Is.EqualTo(0)); + // a < b + Assert.That(Slice.Comparer.Default.Compare(a, b), Is.LessThan(0)); + Assert.That(Slice.Comparer.Default.Compare(a, ab), Is.LessThan(0)); + Assert.That(Slice.Comparer.Default.Compare(a, abc), Is.LessThan(0)); + // a > b + Assert.That(Slice.Comparer.Default.Compare(b, a), Is.GreaterThan(0)); + Assert.That(Slice.Comparer.Default.Compare(b, ab), Is.GreaterThan(0)); + Assert.That(Slice.Comparer.Default.Compare(b, abc), Is.GreaterThan(0)); + + // Operators + // < + Assert.That(a < a, Is.False); + Assert.That(a < ab, Is.True); + Assert.That(ab < b, Is.True); + Assert.That(a < b, Is.True); + Assert.That(ab < a, Is.False); + Assert.That(b < ab, Is.False); + Assert.That(b < a, Is.False); + Assert.That(abc < abc2, Is.False); + // <= + Assert.That(a <= a, Is.True); + Assert.That(a <= ab, Is.True); + Assert.That(ab <= b, Is.True); + Assert.That(a <= b, Is.True); + Assert.That(ab <= a, Is.False); + Assert.That(b <= ab, Is.False); + Assert.That(b <= a, Is.False); + Assert.That(abc <= abc2, Is.True); + // > + Assert.That(a > a, Is.False); + Assert.That(ab > a, Is.True); + Assert.That(b > ab, Is.True); + Assert.That(b > a, Is.True); + Assert.That(a > ab, Is.False); + Assert.That(ab > b, Is.False); + Assert.That(a > b, Is.False); + Assert.That(abc > abc2, Is.False); + // >= + Assert.That(a >= a, Is.True); + Assert.That(ab >= a, Is.True); + Assert.That(b >= ab, Is.True); + Assert.That(b >= a, Is.True); + Assert.That(a >= ab, Is.False); + Assert.That(ab >= b, Is.False); + Assert.That(a >= b, Is.False); + Assert.That(abc >= abc2, Is.True); +#pragma warning restore 1718 } [Test] @@ -1063,7 +1994,7 @@ public void Test_Slice_Comparison_Corner_Cases() Assert.That(Slice.Empty.CompareTo(Slice.Nil), Is.EqualTo(0)); // X > NULL, NULL < X - var abc = Slice.FromAscii("abc"); + var abc = Slice.FromStringAscii("abc"); Assert.That(abc.CompareTo(Slice.Nil), Is.GreaterThan(0)); Assert.That(abc.CompareTo(Slice.Empty), Is.GreaterThan(0)); Assert.That(Slice.Nil.CompareTo(abc), Is.LessThan(0)); @@ -1073,8 +2004,8 @@ public void Test_Slice_Comparison_Corner_Cases() [Test] public void Test_Slice_Comparison_Malformed() { - var good = Slice.FromAscii("good"); - var evil = Slice.FromAscii("evil"); + var good = Slice.FromStringAscii("good"); + var evil = Slice.FromStringAscii("evil"); // argument should be validated Assert.That(() => good.CompareTo(MutateOffset(evil, -1)), Throws.InstanceOf()); @@ -1089,6 +2020,8 @@ public void Test_Slice_Comparison_Malformed() Assert.That(() => MutateOffset(MutateCount(evil, 5), -1).CompareTo(good), Throws.InstanceOf()); } + #endregion + private static readonly string UNICODE_TEXT = "Thïs Ïs à strîng thât contaÎns somé ùnicodè charactêrs and should be encoded in UTF-8: よろしくお願いします"; private static readonly byte[] UNICODE_BYTES = Encoding.UTF8.GetBytes(UNICODE_TEXT); @@ -1105,6 +2038,7 @@ public void Test_Slice_FromStream() Assert.That(slice.GetBytes(), Is.EqualTo(UNICODE_BYTES)); Assert.That(slice.ToUnicode(), Is.EqualTo(UNICODE_TEXT)); + // ReSharper disable once AssignNullToNotNullAttribute Assert.That(() => Slice.FromStream(null), Throws.InstanceOf(), "Should throw if null"); Assert.That(Slice.FromStream(Stream.Null), Is.EqualTo(Slice.Nil), "Stream.Null should return Slice.Nil"); @@ -1116,52 +2050,65 @@ public void Test_Slice_FromStream() } [Test] - public async Task Test_Slice_FromStreamAsync() + public void Test_Slice_Substring() { - Slice slice; - - // Reading from a MemoryStream should use the non-async path - using (var ms = new MemoryStream(UNICODE_BYTES)) - { - slice = await Slice.FromStreamAsync(ms, this.Cancellation); - } - Assert.That(slice.Count, Is.EqualTo(UNICODE_BYTES.Length)); - Assert.That(slice.GetBytes(), Is.EqualTo(UNICODE_BYTES)); - Assert.That(slice.ToUnicode(), Is.EqualTo(UNICODE_TEXT)); - - // Reading from a FileStream should use the async path - var tmp = Path.GetTempFileName(); - try - { - File.WriteAllBytes(tmp, UNICODE_BYTES); - using(var fs = File.OpenRead(tmp)) - { - slice = await Slice.FromStreamAsync(fs, this.Cancellation); - } - } - finally - { - File.Delete(tmp); - } - - Assert.That(slice.Count, Is.EqualTo(UNICODE_BYTES.Length)); - Assert.That(slice.GetBytes(), Is.EqualTo(UNICODE_BYTES)); - Assert.That(slice.ToUnicode(), Is.EqualTo(UNICODE_TEXT)); + Assert.That(Slice.Empty.Substring(0), Is.EqualTo(Slice.Empty)); + Assert.That(Slice.Empty.Substring(0, 0), Is.EqualTo(Slice.Empty)); + Assert.That(() => Slice.Empty.Substring(0, 1), Throws.InstanceOf()); + Assert.That(() => Slice.Empty.Substring(1), Throws.InstanceOf()); + Assert.That(() => Slice.Empty.Substring(1, 0), Throws.Nothing, "We allow out of bound substring if count == 0"); + + // Substring(offset) + Assert.That(Value("Hello, World!").Substring(0), Is.EqualTo(Value("Hello, World!"))); + Assert.That(Value("Hello, World!").Substring(7), Is.EqualTo(Value("World!"))); + Assert.That(Value("Hello, World!").Substring(12), Is.EqualTo(Value("!"))); + Assert.That(Value("Hello, World!").Substring(13), Is.EqualTo(Slice.Empty)); + Assert.That(() => Value("Hello, World!").Substring(14), Throws.InstanceOf()); + + // Substring(offset, count) + Assert.That(Value("Hello, World!").Substring(0, 5), Is.EqualTo(Value("Hello"))); + Assert.That(Value("Hello, World!").Substring(7, 5), Is.EqualTo(Value("World"))); + Assert.That(Value("Hello, World!").Substring(7, 6), Is.EqualTo(Value("World!"))); + Assert.That(Value("Hello, World!").Substring(12, 1), Is.EqualTo(Value("!"))); + Assert.That(Value("Hello, World!").Substring(13, 0), Is.EqualTo(Slice.Empty)); + Assert.That(() => Value("Hello, World!").Substring(7, 7), Throws.InstanceOf()); + Assert.That(() => Value("Hello, World!").Substring(13, 1), Throws.InstanceOf()); + Assert.That(() => Value("Hello, World!").Substring(7, -1), Throws.InstanceOf()); + + // Substring(offset) negative indexing + Assert.That(Value("Hello, World!").Substring(-1), Is.EqualTo(Value("!"))); + Assert.That(Value("Hello, World!").Substring(-2), Is.EqualTo(Value("d!"))); + Assert.That(Value("Hello, World!").Substring(-6), Is.EqualTo(Value("World!"))); + Assert.That(Value("Hello, World!").Substring(-13), Is.EqualTo(Value("Hello, World!"))); + Assert.That(() => Value("Hello, World!").Substring(-14), Throws.InstanceOf()); + + // Slice - int + Assert.That(Value("Hello, World!") - 0, Is.EqualTo(Value("Hello, World!"))); + Assert.That(Value("Hello, World!") - 1, Is.EqualTo(Value("Hello, World"))); + Assert.That(Value("Hello, World!") - 8, Is.EqualTo(Value("Hello"))); + Assert.That(Value("Hello, World!") - 12, Is.EqualTo(Value("H"))); + Assert.That(Value("Hello, World!") - 13, Is.EqualTo(Slice.Empty)); + Assert.That(() => Value("Hello, World!") - 14, Throws.InstanceOf()); + Assert.That(() => Value("Hello, World!") - (-1), Throws.InstanceOf()); } [Test] public void Test_Slice_Concat() { - var a = Slice.FromString("a"); - var b = Slice.FromString("b"); - var c = Slice.FromString("c"); - var ab = Slice.FromString("ab"); - var bc = Slice.FromString("bc"); - var abc = Slice.FromString("abc"); + var a = Value("a"); + var b = Value("b"); + var c = Value("c"); + var ab = Value("ab"); + var bc = Value("bc"); + var abc = Value("abc"); + // Concat2 + + Assert.That(Slice.Concat(a, a).ToUnicode(), Is.EqualTo("aa")); Assert.That(Slice.Concat(a, b).ToUnicode(), Is.EqualTo("ab")); Assert.That(Slice.Concat(b, c).ToUnicode(), Is.EqualTo("bc")); + Assert.That(Slice.Concat(ab, ab).ToUnicode(), Is.EqualTo("abab")); Assert.That(Slice.Concat(ab, c).ToUnicode(), Is.EqualTo("abc")); Assert.That(Slice.Concat(a, bc).ToUnicode(), Is.EqualTo("abc")); Assert.That(Slice.Concat(a, b, c).ToUnicode(), Is.EqualTo("abc")); @@ -1180,6 +2127,8 @@ public void Test_Slice_Concat() Assert.That(Slice.Concat(Slice.Empty, abc), Is.EqualTo(abc)); Assert.That(Slice.Concat(Slice.Nil, abc), Is.EqualTo(abc)); + // Concat3 + Assert.That(Slice.Concat(Slice.Empty, b, c), Is.EqualTo(bc)); Assert.That(Slice.Concat(ab, Slice.Empty, c), Is.EqualTo(abc)); Assert.That(Slice.Concat(a, b, Slice.Empty), Is.EqualTo(ab)); @@ -1189,21 +2138,35 @@ public void Test_Slice_Concat() Assert.That(Slice.Concat(Slice.Nil, Slice.Nil, Slice.Nil), Is.EqualTo(Slice.Empty)); Assert.That(Slice.Concat(Slice.Empty, Slice.Empty, Slice.Empty), Is.EqualTo(Slice.Empty)); + + // Slice + Slice + Assert.That(a + a, Is.EqualTo(Value("aa"))); + Assert.That(a + b, Is.EqualTo(Value("ab"))); + Assert.That(b + c, Is.EqualTo(Value("bc"))); + Assert.That(ab + ab, Is.EqualTo(Value("abab"))); + Assert.That(ab + c, Is.EqualTo(Value("abc"))); + Assert.That(a + bc, Is.EqualTo(Value("abc"))); + + // Slice + byte + Assert.That(a + 0, Is.EqualTo(Key("a\x00"))); + Assert.That(a + 1, Is.EqualTo(Key("a\x01"))); + Assert.That(b + (byte)'A', Is.EqualTo(Key("bA"))); + Assert.That(abc + 255, Is.EqualTo(Key("abc\xff"))); } [Test] public void Test_Slice_Join_Array() { - var a = Slice.FromString("A"); - var b = Slice.FromString("BB"); - var c = Slice.FromString("CCC"); + var a = Value("A"); + var b = Value("BB"); + var c = Value("CCC"); // empty separator should just join all slices together Assert.That(Slice.Join(Slice.Empty, new Slice[0]), Is.EqualTo(Slice.Empty)); Assert.That(Slice.Join(Slice.Empty, new[] { Slice.Empty }), Is.EqualTo(Slice.Empty)); - Assert.That(Slice.Join(Slice.Empty, new[] { a }), Is.EqualTo(Slice.FromString("A"))); - Assert.That(Slice.Join(Slice.Empty, new[] { a, b }), Is.EqualTo(Slice.FromString("ABB"))); - Assert.That(Slice.Join(Slice.Empty, new[] { a, b, c }), Is.EqualTo(Slice.FromString("ABBCCC"))); + Assert.That(Slice.Join(Slice.Empty, new[] { a }), Is.EqualTo(Value("A"))); + Assert.That(Slice.Join(Slice.Empty, new[] { a, b }), Is.EqualTo(Value("ABB"))); + Assert.That(Slice.Join(Slice.Empty, new[] { a, b, c }), Is.EqualTo(Value("ABBCCC"))); Assert.That(Slice.Join(Slice.Empty, new[] { a, b, c }).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(Slice.Empty, new[] { a, b, c }).Count, Is.EqualTo(6)); @@ -1211,63 +2174,64 @@ public void Test_Slice_Join_Array() var sep = Slice.FromChar(','); Assert.That(Slice.Join(sep, new Slice[0]), Is.EqualTo(Slice.Empty)); Assert.That(Slice.Join(sep, new[] { Slice.Empty }), Is.EqualTo(Slice.Empty)); - Assert.That(Slice.Join(sep, new[] { a }), Is.EqualTo(Slice.FromString("A"))); - Assert.That(Slice.Join(sep, new[] { a, b }), Is.EqualTo(Slice.FromString("A,BB"))); - Assert.That(Slice.Join(sep, new[] { a, b, c }), Is.EqualTo(Slice.FromString("A,BB,CCC"))); + Assert.That(Slice.Join(sep, new[] { a }), Is.EqualTo(Value("A"))); + Assert.That(Slice.Join(sep, new[] { a, b }), Is.EqualTo(Value("A,BB"))); + Assert.That(Slice.Join(sep, new[] { a, b, c }), Is.EqualTo(Value("A,BB,CCC"))); Assert.That(Slice.Join(sep, new[] { a, b, c }).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(sep, new[] { a, b, c }).Count, Is.EqualTo(8)); - Assert.That(Slice.Join(sep, new[] { a, Slice.Empty, c }), Is.EqualTo(Slice.FromString("A,,CCC"))); - Assert.That(Slice.Join(sep, new[] { Slice.Empty, b, c }), Is.EqualTo(Slice.FromString(",BB,CCC"))); - Assert.That(Slice.Join(sep, new[] { Slice.Empty, Slice.Empty, Slice.Empty }), Is.EqualTo(Slice.FromString(",,"))); + Assert.That(Slice.Join(sep, new[] { a, Slice.Empty, c }), Is.EqualTo(Value("A,,CCC"))); + Assert.That(Slice.Join(sep, new[] { Slice.Empty, b, c }), Is.EqualTo(Value(",BB,CCC"))); + Assert.That(Slice.Join(sep, new[] { Slice.Empty, Slice.Empty, Slice.Empty }), Is.EqualTo(Value(",,"))); // multi byte separator, with a non-0 offset - sep = Slice.FromString("!<@>!").Substring(1, 3); + sep = Value("!<@>!").Substring(1, 3); Assert.That(sep.Offset, Is.EqualTo(1)); Assert.That(Slice.Join(sep, new Slice[0]), Is.EqualTo(Slice.Empty)); Assert.That(Slice.Join(sep, new[] { Slice.Empty }), Is.EqualTo(Slice.Empty)); - Assert.That(Slice.Join(sep, new[] { a }), Is.EqualTo(Slice.FromString("A"))); - Assert.That(Slice.Join(sep, new[] { a, b }), Is.EqualTo(Slice.FromString("A<@>BB"))); - Assert.That(Slice.Join(sep, new[] { a, b, c }), Is.EqualTo(Slice.FromString("A<@>BB<@>CCC"))); + Assert.That(Slice.Join(sep, new[] { a }), Is.EqualTo(Value("A"))); + Assert.That(Slice.Join(sep, new[] { a, b }), Is.EqualTo(Value("A<@>BB"))); + Assert.That(Slice.Join(sep, new[] { a, b, c }), Is.EqualTo(Value("A<@>BB<@>CCC"))); Assert.That(Slice.Join(sep, new[] { a, b, c }).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(sep, new[] { a, b, c }).Count, Is.EqualTo(12)); // join slices that use the same underlying buffer - string s = "hello world!!!"; - byte[] tmp = Encoding.UTF8.GetBytes(s); + const string HELLO_WORLD = "hello world!!!"; + byte[] tmp = Encoding.UTF8.GetBytes(HELLO_WORLD); var slices = new Slice[tmp.Length]; - for (int i = 0; i < tmp.Length; i++) slices[i] = Slice.Create(tmp, i, 1); - Assert.That(Slice.Join(Slice.Empty, slices), Is.EqualTo(Slice.FromString(s))); - Assert.That(Slice.Join(Slice.FromChar(':'), slices), Is.EqualTo(Slice.FromString("h:e:l:l:o: :w:o:r:l:d:!:!:!"))); + for (int i = 0; i < tmp.Length; i++) slices[i] = tmp.AsSlice(i, 1); + Assert.That(Slice.Join(Slice.Empty, slices), Is.EqualTo(Value(HELLO_WORLD))); + Assert.That(Slice.Join(Slice.FromChar(':'), slices), Is.EqualTo(Value("h:e:l:l:o: :w:o:r:l:d:!:!:!"))); } [Test] public void Test_Slice_Join_Enumerable() { - var query = Enumerable.Range(1, 3).Select(c => Slice.FromString(new string((char)(64 + c), c))); - Assert.That(Slice.Join(Slice.Empty, Enumerable.Empty()), Is.EqualTo(Slice.Empty)); - Assert.That(Slice.Join(Slice.Empty, query), Is.EqualTo(Slice.FromString("ABBCCC"))); + + // ReSharper disable PossibleMultipleEnumeration + var query = Enumerable.Range(1, 3).Select(c => Value(new string((char)(64 + c), c))); + Assert.That(Slice.Join(Slice.Empty, query), Is.EqualTo(Value("ABBCCC"))); Assert.That(Slice.Join(Slice.Empty, query).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(Slice.Empty, query).Count, Is.EqualTo(6)); var sep = Slice.FromChar(','); Assert.That(Slice.Join(sep, Enumerable.Empty()), Is.EqualTo(Slice.Empty)); - Assert.That(Slice.Join(sep, query), Is.EqualTo(Slice.FromString("A,BB,CCC"))); + Assert.That(Slice.Join(sep, query), Is.EqualTo(Value("A,BB,CCC"))); Assert.That(Slice.Join(sep, query).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(sep, query).Count, Is.EqualTo(8)); var arr = query.ToArray(); - Assert.That(Slice.Join(Slice.Empty, (IEnumerable)arr), Is.EqualTo(Slice.FromString("ABBCCC"))); + Assert.That(Slice.Join(Slice.Empty, (IEnumerable)arr), Is.EqualTo(Value("ABBCCC"))); Assert.That(Slice.Join(Slice.Empty, (IEnumerable)arr).Offset, Is.EqualTo(0)); Assert.That(Slice.Join(Slice.Empty, (IEnumerable)arr).Count, Is.EqualTo(6)); - + // ReSharper restore PossibleMultipleEnumeration } [Test] public void Test_Slice_JoinBytes() { var sep = Slice.FromChar(' '); - var tokens = new[] { Slice.FromString("hello"), Slice.FromString("world"), Slice.FromString("!") }; + var tokens = new[] { Value("hello"), Value("world"), Value("!") }; var joined = Slice.JoinBytes(sep, tokens); Assert.That(joined, Is.Not.Null); @@ -1301,8 +2265,10 @@ public void Test_Slice_JoinBytes() Assert.That(joined, Is.Not.Null); Assert.That(joined.Length, Is.EqualTo(0)); + // ReSharper disable AssignNullToNotNullAttribute Assert.That(() => Slice.JoinBytes(sep, default(Slice[]), 0, 0), Throws.InstanceOf()); Assert.That(() => Slice.JoinBytes(sep, default(IEnumerable)), Throws.InstanceOf()); + // ReSharper restore AssignNullToNotNullAttribute Assert.That(() => Slice.JoinBytes(sep, tokens, 0, 4), Throws.InstanceOf()); Assert.That(() => Slice.JoinBytes(sep, tokens, -1, 1), Throws.InstanceOf()); @@ -1313,31 +2279,31 @@ public void Test_Slice_JoinBytes() [Test] public void Test_Slice_Split() { - var a = Slice.FromString("A"); - var b = Slice.FromString("BB"); - var c = Slice.FromString("CCC"); + var a = Value("A"); + var b = Value("BB"); + var c = Value("CCC"); var comma = Slice.FromChar(','); - Assert.That(Slice.FromString("A").Split(comma), Is.EqualTo(new[] { a })); - Assert.That(Slice.FromString("A,BB").Split(comma), Is.EqualTo(new[] { a, b })); - Assert.That(Slice.FromString("A,BB,CCC").Split(comma), Is.EqualTo(new[] { a, b, c })); + Assert.That(Value("A").Split(comma), Is.EqualTo(new[] { a })); + Assert.That(Value("A,BB").Split(comma), Is.EqualTo(new[] { a, b })); + Assert.That(Value("A,BB,CCC").Split(comma), Is.EqualTo(new[] { a, b, c })); // empty values should be kept or discarded, depending on the option settings - Assert.That(Slice.FromString("A,,CCC").Split(comma, StringSplitOptions.None), Is.EqualTo(new[] { a, Slice.Empty, c })); - Assert.That(Slice.FromString("A,,CCC").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new[] { a, c })); + Assert.That(Value("A,,CCC").Split(comma, StringSplitOptions.None), Is.EqualTo(new[] { a, Slice.Empty, c })); + Assert.That(Value("A,,CCC").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new[] { a, c })); // edge cases // > should behave the same as String.Split() Assert.That(Slice.Empty.Split(comma, StringSplitOptions.None), Is.EqualTo(new [] { Slice.Empty })); Assert.That(Slice.Empty.Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new Slice[0])); - Assert.That(Slice.FromString("A,").Split(comma, StringSplitOptions.None), Is.EqualTo(new[] { a, Slice.Empty })); - Assert.That(Slice.FromString("A,").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new [] { a })); - Assert.That(Slice.FromString(",").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new Slice[0])); - Assert.That(Slice.FromString(",,,").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new Slice[0])); + Assert.That(Value("A,").Split(comma, StringSplitOptions.None), Is.EqualTo(new[] { a, Slice.Empty })); + Assert.That(Value("A,").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new [] { a })); + Assert.That(Value(",").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new Slice[0])); + Assert.That(Value(",,,").Split(comma, StringSplitOptions.RemoveEmptyEntries), Is.EqualTo(new Slice[0])); // multi-bytes separator with an offset - var sep = Slice.FromString("!<@>!").Substring(1, 3); - Assert.That(Slice.FromString("A<@>BB<@>CCC").Split(sep), Is.EqualTo(new[] { a, b, c })); + var sep = Value("!<@>!").Substring(1, 3); + Assert.That(Value("A<@>BB<@>CCC").Split(sep), Is.EqualTo(new[] { a, b, c })); } #region Black Magic Incantations... @@ -1371,5 +2337,21 @@ private static Slice MutateArray(Slice value, byte[] array) #endregion + #region Helpers... + + /// Create a key from a byte string + private static Slice Key(string byteString) + { + return Slice.FromByteString(byteString); + } + + /// Create a value from a unicode string + private static Slice Value(string text) + { + return Slice.FromString(text); + } + + #endregion } + } diff --git a/FoundationDB.Tests/Utils/SliceHelperFacts.cs b/FoundationDB.Tests/Utils/SliceHelperFacts.cs deleted file mode 100644 index 58306a129..000000000 --- a/FoundationDB.Tests/Utils/SliceHelperFacts.cs +++ /dev/null @@ -1,112 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client.Tests -{ - using FoundationDB.Client; - using NUnit.Framework; - using System; - using System.Text; - - [TestFixture] - public class SliceHelperFacts : FdbTest - { - - #region SliceHelpers... - - [Test] - public void Test_SliceHelpers_Align() - { - // Even though 0 is a multiple of 16, it is always rounded up to 16 to simplify buffer handling logic - Assert.That(SliceHelpers.Align(0), Is.EqualTo(16)); - // 1..16 => 16 - for (int i = 1; i <= 16; i++) { Assert.That(SliceHelpers.Align(i), Is.EqualTo(16), "Align({0}) => 16", i); } - // 17..32 => 32 - for (int i = 17; i <= 32; i++) { Assert.That(SliceHelpers.Align(i), Is.EqualTo(32), "Align({0}) => 32", i); } - // 33..48 => 48 - for (int i = 33; i <= 48; i++) { Assert.That(SliceHelpers.Align(i), Is.EqualTo(48), "Align({0}) => 48", i); } - - // 2^N-1 - for (int i = 6; i < 30; i++) - { - Assert.That(SliceHelpers.Align((1 << i) - 1), Is.EqualTo(1 << i)); - } - // largest non overflowing - Assert.That(() => SliceHelpers.Align(int.MaxValue - 15), Is.EqualTo((int.MaxValue - 15))); - - // overflow - Assert.That(() => SliceHelpers.Align(int.MaxValue), Throws.InstanceOf()); - Assert.That(() => SliceHelpers.Align(int.MaxValue - 14), Throws.InstanceOf()); - - // negative values - Assert.That(() => SliceHelpers.Align(-1), Throws.InstanceOf()); - Assert.That(() => SliceHelpers.Align(int.MinValue), Throws.InstanceOf()); - } - - [Test] - public void Test_SliceHelpers_NextPowerOfTwo() - { - // 0 is a special case, to simplify bugger handling logic - Assert.That(SliceHelpers.NextPowerOfTwo(0), Is.EqualTo(1), "Special case for 0"); - Assert.That(SliceHelpers.NextPowerOfTwo(1), Is.EqualTo(1)); - Assert.That(SliceHelpers.NextPowerOfTwo(2), Is.EqualTo(2)); - - for (int i = 2; i < 31; i++) - { - Assert.That(SliceHelpers.NextPowerOfTwo((1 << i) - 1), Is.EqualTo(1 << i)); - Assert.That(SliceHelpers.NextPowerOfTwo(1 << i), Is.EqualTo(1 << i)); - } - - Assert.That(() => SliceHelpers.NextPowerOfTwo(-1), Throws.InstanceOf()); - Assert.That(() => SliceHelpers.NextPowerOfTwo(-42), Throws.InstanceOf()); - } - - [Test] - public void Test_SliceHelpers_ComputeHashCode() - { - //note: if everything fails, check that the hashcode algorithm hasn't changed also ! - - Assert.That(SliceHelpers.ComputeHashCode(new byte[0], 0, 0), Is.EqualTo(-2128831035)); - Assert.That(SliceHelpers.ComputeHashCode(new byte[1], 0, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCode(new byte[2], 0, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCode(new byte[2], 1, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCode(new byte[2], 0, 2), Is.EqualTo(292984781)); - Assert.That(SliceHelpers.ComputeHashCode(Encoding.Default.GetBytes("hello"), 0, 5), Is.EqualTo(1335831723)); - - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(new byte[0], 0, 0), Is.EqualTo(-2128831035)); - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(new byte[1], 0, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(new byte[2], 0, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(new byte[2], 1, 1), Is.EqualTo(84696351)); - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(new byte[2], 0, 2), Is.EqualTo(292984781)); - Assert.That(SliceHelpers.ComputeHashCodeUnsafe(Encoding.Default.GetBytes("hello"), 0, 5), Is.EqualTo(1335831723)); - } - - #endregion - - } -} diff --git a/FoundationDB.Tests/Utils/SliceStreamFacts.cs b/FoundationDB.Tests/Utils/SliceStreamFacts.cs index 0bfe73560..e2a4a65d1 100644 --- a/FoundationDB.Tests/Utils/SliceStreamFacts.cs +++ b/FoundationDB.Tests/Utils/SliceStreamFacts.cs @@ -26,16 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB.Client.Tests +namespace Doxense.Memory.Tests { - using FoundationDB.Client; - using NUnit.Framework; using System; using System.Collections.Generic; using System.IO; using System.Linq; using System.Text; + using Doxense.Memory; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] public class SliceStreamFacts : FdbTest @@ -137,13 +137,12 @@ public void Test_SliceStream_CopyTo() } } - + [Test] public void Test_SliceListStream_Basics() { const int N = 65536; var rnd = new Random(); - Slice slice; // create a random buffer var bytes = new byte[N]; @@ -156,7 +155,7 @@ public void Test_SliceListStream_Basics() while(r > 0) { int sz = Math.Min(1 + rnd.Next(1024), r); - slice = Slice.Create(bytes, p, sz); + Slice slice = bytes.AsSlice(p, sz); if (rnd.Next(2) == 1) slice = slice.Memoize(); slices.Add(slice); diff --git a/FoundationDB.Tests/Utils/SliceWriterFacts.cs b/FoundationDB.Tests/Utils/SliceWriterFacts.cs index d7c6c9f75..21f52c9f8 100644 --- a/FoundationDB.Tests/Utils/SliceWriterFacts.cs +++ b/FoundationDB.Tests/Utils/SliceWriterFacts.cs @@ -26,15 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client.Utils.Tests +namespace Doxense.Memory.Tests { - using FoundationDB.Client; - using NUnit.Framework; using System; using System.Text; + using Doxense.Memory; + using FoundationDB.Client.Tests; + using NUnit.Framework; [TestFixture] - public class SliceWriterFacts + public class SliceWriterFacts : FdbTest { private static string Clean(string value) @@ -51,16 +52,16 @@ private static string Clean(string value) private static void PerformWriterTest(TestHandler action, T value, string expectedResult, string message = null) { - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); action(ref writer, value); - Assert.That(writer.ToSlice().ToHexaString(' '), Is.EqualTo(expectedResult), "Value {0} ({1}) was not properly packed", value == null ? "" : value is string ? Clean(value as string) : value.ToString(), (value == null ? "null" : value.GetType().Name)); + Assert.That(writer.ToSlice().ToHexaString(' '), Is.EqualTo(expectedResult), "Value {0} ({1}) was not properly packed. {2}", value == null ? "" : value is string ? Clean(value as string) : value.ToString(), (value == null ? "null" : value.GetType().Name), message); } [Test] public void Test_Empty_Writer() { - var writer = SliceWriter.Empty; + var writer = default(SliceWriter); Assert.That(writer.Position, Is.EqualTo(0)); Assert.That(writer.HasData, Is.False); Assert.That(writer.Buffer, Is.Null); @@ -70,64 +71,267 @@ public void Test_Empty_Writer() [Test] public void Test_WriteBytes() { - TestHandler test = (ref SliceWriter writer, byte[] value) => writer.WriteBytes(value); + { + TestHandler test = (ref SliceWriter writer, byte[] value) => writer.WriteBytes(value); - PerformWriterTest(test, null, ""); - PerformWriterTest(test, new byte[0], ""); - PerformWriterTest(test, new byte[] { 66 }, "42"); - PerformWriterTest(test, new byte[] { 65, 66, 67 }, "41 42 43"); + PerformWriterTest(test, null, ""); + PerformWriterTest(test, new byte[0], ""); + PerformWriterTest(test, new byte[] {66}, "42"); + PerformWriterTest(test, new byte[] {65, 66, 67}, "41 42 43"); + } + { + TestHandler test = (ref SliceWriter writer, Slice value) => writer.WriteBytes(value); + + PerformWriterTest(test, Slice.Nil, ""); + PerformWriterTest(test, Slice.Empty, ""); + PerformWriterTest(test, Slice.FromByte(66), "42"); + PerformWriterTest(test, new byte[] { 65, 66, 67 }.AsSlice(), "41 42 43"); + PerformWriterTest(test, new byte[] { 65, 66, 67, 68, 69 }.AsSlice(1, 3), "42 43 44"); + } } [Test] - public void Test_WriteByte() + public void Test_WriteByte_Unsigned() { TestHandler test = (ref SliceWriter writer, byte value) => writer.WriteByte(value); - PerformWriterTest(test, default(byte), "00"); - PerformWriterTest(test, (byte)1, "01"); - PerformWriterTest(test, (byte)42, "2A"); - PerformWriterTest(test, (byte)255, "FF"); + PerformWriterTest(test, 0, "00"); + PerformWriterTest(test, 1, "01"); + PerformWriterTest(test, 42, "2A"); + PerformWriterTest(test, 255, "FF"); + } + + [Test] + public void Test_WriteByte_Signed() + { + TestHandler test = (ref SliceWriter writer, sbyte value) => writer.WriteByte(value); + + PerformWriterTest(test, 0, "00"); + PerformWriterTest(test, 1, "01"); + PerformWriterTest(test, 42, "2A"); + PerformWriterTest(test, sbyte.MaxValue, "7F"); + PerformWriterTest(test, -1, "FF"); + PerformWriterTest(test, sbyte.MinValue, "80"); + } + + [Test] + public void Test_WriteFixed16_Unsigned() + { + TestHandler test = (ref SliceWriter writer, ushort value) => writer.WriteFixed16(value); + + PerformWriterTest(test, 0, "00 00"); + PerformWriterTest(test, 1, "01 00"); + PerformWriterTest(test, 0x12, "12 00"); + PerformWriterTest(test, 0x1234, "34 12"); + PerformWriterTest(test, ushort.MaxValue, "FF FF"); + } + + [Test] + public void Test_WriteFixed16_Signed() + { + TestHandler test = (ref SliceWriter writer, short value) => writer.WriteFixed16(value); + + PerformWriterTest(test, 0, "00 00"); + PerformWriterTest(test, 1, "01 00"); + PerformWriterTest(test, 0x12, "12 00"); + PerformWriterTest(test, 0x1234, "34 12"); + PerformWriterTest(test, short.MaxValue, "FF 7F"); + PerformWriterTest(test, -1, "FF FF"); + PerformWriterTest(test, short.MinValue, "00 80"); + } + + [Test] + public void Test_WriteFixed16BE_Unsigned() + { + TestHandler test = (ref SliceWriter writer, ushort value) => writer.WriteFixed16BE(value); + + PerformWriterTest(test, 0, "00 00"); + PerformWriterTest(test, 1, "00 01"); + PerformWriterTest(test, 0x12, "00 12"); + PerformWriterTest(test, 0x1234, "12 34"); + PerformWriterTest(test, ushort.MaxValue, "FF FF"); + } + + [Test] + public void Test_WriteFixed16BE_Signed() + { + TestHandler test = (ref SliceWriter writer, short value) => writer.WriteFixed16BE(value); + + PerformWriterTest(test, 0, "00 00"); + PerformWriterTest(test, 1, "00 01"); + PerformWriterTest(test, 0x12, "00 12"); + PerformWriterTest(test, 0x1234, "12 34"); + PerformWriterTest(test, short.MaxValue, "7F FF"); + PerformWriterTest(test, -1, "FF FF"); + PerformWriterTest(test, short.MinValue, "80 00"); } [Test] - public void Test_WriteFixed32() + public void Test_WriteFixed32_Unsigned() { TestHandler test = (ref SliceWriter writer, uint value) => writer.WriteFixed32(value); - PerformWriterTest(test, 0U, "00 00 00 00"); - PerformWriterTest(test, 1U, "01 00 00 00"); - PerformWriterTest(test, 0x12U, "12 00 00 00"); - PerformWriterTest(test, 0x1234U, "34 12 00 00"); - PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00"); - PerformWriterTest(test, 0x123456U, "56 34 12 00"); - PerformWriterTest(test, 0xDEADBEEF, "EF BE AD DE"); - PerformWriterTest(test, uint.MaxValue, "FF FF FF FF"); + PerformWriterTest(test, 0U, "00 00 00 00"); + PerformWriterTest(test, 1U, "01 00 00 00"); + PerformWriterTest(test, 0x12U, "12 00 00 00"); + PerformWriterTest(test, 0x1234U, "34 12 00 00"); + PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00"); + PerformWriterTest(test, 0x123456U, "56 34 12 00"); + PerformWriterTest(test, 0xDEADBEEF, "EF BE AD DE"); + PerformWriterTest(test, uint.MaxValue, "FF FF FF FF"); + } + + [Test] + public void Test_WriteFixed32_Signed() + { + TestHandler test = (ref SliceWriter writer, int value) => writer.WriteFixed32(value); + + PerformWriterTest(test, 0, "00 00 00 00"); + PerformWriterTest(test, 1, "01 00 00 00"); + PerformWriterTest(test, 0x12, "12 00 00 00"); + PerformWriterTest(test, 0x1234, "34 12 00 00"); + PerformWriterTest(test, short.MaxValue, "FF 7F 00 00"); + PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00"); + PerformWriterTest(test, 0x123456, "56 34 12 00"); + PerformWriterTest(test, unchecked((int)0xDEADBEEF), "EF BE AD DE"); + PerformWriterTest(test, int.MaxValue, "FF FF FF 7F"); + PerformWriterTest(test, -1, "FF FF FF FF"); + PerformWriterTest(test, short.MinValue, "00 80 FF FF"); + PerformWriterTest(test, int.MinValue, "00 00 00 80"); + + } + + [Test] + public void Test_WriteFixed32BE_Unsigned() + { + TestHandler test = (ref SliceWriter writer, uint value) => writer.WriteFixed32BE(value); + + PerformWriterTest(test, 0U, "00 00 00 00"); + PerformWriterTest(test, 1U, "00 00 00 01"); + PerformWriterTest(test, 0x12U, "00 00 00 12"); + PerformWriterTest(test, 0x1234U, "00 00 12 34"); + PerformWriterTest(test, ushort.MaxValue, "00 00 FF FF"); + PerformWriterTest(test, 0x123456U, "00 12 34 56"); + PerformWriterTest(test, 0xDEADBEEF, "DE AD BE EF"); + PerformWriterTest(test, uint.MaxValue, "FF FF FF FF"); + } + + [Test] + public void Test_WriteFixed32BE_Signed() + { + TestHandler test = (ref SliceWriter writer, int value) => writer.WriteFixed32BE(value); + + PerformWriterTest(test, 0, "00 00 00 00"); + PerformWriterTest(test, 1, "00 00 00 01"); + PerformWriterTest(test, 0x12, "00 00 00 12"); + PerformWriterTest(test, 0x1234, "00 00 12 34"); + PerformWriterTest(test, short.MaxValue, "00 00 7F FF"); + PerformWriterTest(test, ushort.MaxValue, "00 00 FF FF"); + PerformWriterTest(test, 0x123456, "00 12 34 56"); + PerformWriterTest(test, unchecked((int)0xDEADBEEF), "DE AD BE EF"); + PerformWriterTest(test, int.MaxValue, "7F FF FF FF"); + PerformWriterTest(test, -1, "FF FF FF FF"); + PerformWriterTest(test, short.MinValue, "FF FF 80 00"); + PerformWriterTest(test, int.MinValue, "80 00 00 00"); + } [Test] - public void Test_WriteFixed64() + public void Test_WriteFixed64_Unsigned() { TestHandler test = (ref SliceWriter writer, ulong value) => writer.WriteFixed64(value); - PerformWriterTest(test, 0UL, "00 00 00 00 00 00 00 00"); - PerformWriterTest(test, 1UL, "01 00 00 00 00 00 00 00"); - PerformWriterTest(test, 0x12UL, "12 00 00 00 00 00 00 00"); - PerformWriterTest(test, 0x1234UL, "34 12 00 00 00 00 00 00"); - PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00 00 00 00 00"); - PerformWriterTest(test, 0x123456UL, "56 34 12 00 00 00 00 00"); - PerformWriterTest(test, 0x12345678UL, "78 56 34 12 00 00 00 00"); - PerformWriterTest(test, uint.MaxValue, "FF FF FF FF 00 00 00 00"); - PerformWriterTest(test, 0x123456789AUL, "9A 78 56 34 12 00 00 00"); - PerformWriterTest(test, 0x123456789ABCUL, "BC 9A 78 56 34 12 00 00"); - PerformWriterTest(test, 0x123456789ABCDEUL, "DE BC 9A 78 56 34 12 00"); - PerformWriterTest(test, 0xBADC0FFEE0DDF00DUL, "0D F0 DD E0 FE 0F DC BA"); - PerformWriterTest(test, ulong.MaxValue, "FF FF FF FF FF FF FF FF"); + PerformWriterTest(test, 0UL, "00 00 00 00 00 00 00 00"); + PerformWriterTest(test, 1UL, "01 00 00 00 00 00 00 00"); + PerformWriterTest(test, 0x12UL, "12 00 00 00 00 00 00 00"); + PerformWriterTest(test, 0x1234UL, "34 12 00 00 00 00 00 00"); + PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00 00 00 00 00"); + PerformWriterTest(test, 0x123456UL, "56 34 12 00 00 00 00 00"); + PerformWriterTest(test, 0x12345678UL, "78 56 34 12 00 00 00 00"); + PerformWriterTest(test, uint.MaxValue, "FF FF FF FF 00 00 00 00"); + PerformWriterTest(test, 0x123456789AUL, "9A 78 56 34 12 00 00 00"); + PerformWriterTest(test, 0x123456789ABCUL, "BC 9A 78 56 34 12 00 00"); + PerformWriterTest(test, 0x123456789ABCDEUL, "DE BC 9A 78 56 34 12 00"); + PerformWriterTest(test, 0xBADC0FFEE0DDF00DUL, "0D F0 DD E0 FE 0F DC BA"); + PerformWriterTest(test, ulong.MaxValue, "FF FF FF FF FF FF FF FF"); + } + + [Test] + public void Test_WriteFixed64_Signed() + { + TestHandler test = (ref SliceWriter writer, long value) => writer.WriteFixed64(value); + + PerformWriterTest(test, 0L, "00 00 00 00 00 00 00 00"); + PerformWriterTest(test, 1L, "01 00 00 00 00 00 00 00"); + PerformWriterTest(test, 0x12L, "12 00 00 00 00 00 00 00"); + PerformWriterTest(test, 0x1234L, "34 12 00 00 00 00 00 00"); + PerformWriterTest(test, short.MaxValue, "FF 7F 00 00 00 00 00 00"); + PerformWriterTest(test, ushort.MaxValue, "FF FF 00 00 00 00 00 00"); + PerformWriterTest(test, 0x123456L, "56 34 12 00 00 00 00 00"); + PerformWriterTest(test, 0x12345678L, "78 56 34 12 00 00 00 00"); + PerformWriterTest(test, int.MaxValue, "FF FF FF 7F 00 00 00 00"); + PerformWriterTest(test, uint.MaxValue, "FF FF FF FF 00 00 00 00"); + PerformWriterTest(test, 0x123456789AL, "9A 78 56 34 12 00 00 00"); + PerformWriterTest(test, 0x123456789ABCL, "BC 9A 78 56 34 12 00 00"); + PerformWriterTest(test, 0x123456789ABCDEL, "DE BC 9A 78 56 34 12 00"); + PerformWriterTest(test, unchecked((long) 0xBADC0FFEE0DDF00D), "0D F0 DD E0 FE 0F DC BA"); + PerformWriterTest(test, long.MaxValue, "FF FF FF FF FF FF FF 7F"); + PerformWriterTest(test, -1L, "FF FF FF FF FF FF FF FF"); + PerformWriterTest(test, short.MinValue, "00 80 FF FF FF FF FF FF"); + PerformWriterTest(test, int.MinValue, "00 00 00 80 FF FF FF FF"); + PerformWriterTest(test, long.MinValue, "00 00 00 00 00 00 00 80"); + } + + [Test] + public void Test_WriteFixed64BE_Unsigned() + { + TestHandler test = (ref SliceWriter writer, ulong value) => writer.WriteFixed64BE(value); + + PerformWriterTest(test, 0UL, "00 00 00 00 00 00 00 00"); + PerformWriterTest(test, 1UL, "00 00 00 00 00 00 00 01"); + PerformWriterTest(test, 0x12UL, "00 00 00 00 00 00 00 12"); + PerformWriterTest(test, 0x1234UL, "00 00 00 00 00 00 12 34"); + PerformWriterTest(test, ushort.MaxValue, "00 00 00 00 00 00 FF FF"); + PerformWriterTest(test, 0x123456UL, "00 00 00 00 00 12 34 56"); + PerformWriterTest(test, 0x12345678UL, "00 00 00 00 12 34 56 78"); + PerformWriterTest(test, uint.MaxValue, "00 00 00 00 FF FF FF FF"); + PerformWriterTest(test, 0x123456789AUL, "00 00 00 12 34 56 78 9A"); + PerformWriterTest(test, 0x123456789ABCUL, "00 00 12 34 56 78 9A BC"); + PerformWriterTest(test, 0x123456789ABCDEUL, "00 12 34 56 78 9A BC DE"); + PerformWriterTest(test, 0xBADC0FFEE0DDF00DUL, "BA DC 0F FE E0 DD F0 0D"); + PerformWriterTest(test, ulong.MaxValue, "FF FF FF FF FF FF FF FF"); + } + + [Test] + public void Test_WriteFixed64BE_Signed() + { + TestHandler test = (ref SliceWriter writer, long value) => writer.WriteFixed64BE(value); + + PerformWriterTest(test, 0L, "00 00 00 00 00 00 00 00"); + PerformWriterTest(test, 1L, "00 00 00 00 00 00 00 01"); + PerformWriterTest(test, 0x12L, "00 00 00 00 00 00 00 12"); + PerformWriterTest(test, 0x1234L, "00 00 00 00 00 00 12 34"); + PerformWriterTest(test, short.MaxValue, "00 00 00 00 00 00 7F FF"); + PerformWriterTest(test, ushort.MaxValue, "00 00 00 00 00 00 FF FF"); + PerformWriterTest(test, 0x123456L, "00 00 00 00 00 12 34 56"); + PerformWriterTest(test, 0x12345678L, "00 00 00 00 12 34 56 78"); + PerformWriterTest(test, int.MaxValue, "00 00 00 00 7F FF FF FF"); + PerformWriterTest(test, uint.MaxValue, "00 00 00 00 FF FF FF FF"); + PerformWriterTest(test, 0x123456789AL, "00 00 00 12 34 56 78 9A"); + PerformWriterTest(test, 0x123456789ABCL, "00 00 12 34 56 78 9A BC"); + PerformWriterTest(test, 0x123456789ABCDEL, "00 12 34 56 78 9A BC DE"); + PerformWriterTest(test, unchecked((long)0xBADC0FFEE0DDF00D), "BA DC 0F FE E0 DD F0 0D"); + PerformWriterTest(test, long.MaxValue, "7F FF FF FF FF FF FF FF"); + PerformWriterTest(test, -1L, "FF FF FF FF FF FF FF FF"); + PerformWriterTest(test, short.MinValue, "FF FF FF FF FF FF 80 00"); + PerformWriterTest(test, int.MinValue, "FF FF FF FF 80 00 00 00"); + PerformWriterTest(test, long.MinValue, "80 00 00 00 00 00 00 00"); } [Test] public void Test_WriteVarint32() { - TestHandler test = (ref SliceWriter writer, uint value) => writer.WriteVarint32(value); + TestHandler test = (ref SliceWriter writer, uint value) => writer.WriteVarInt32(value); PerformWriterTest(test, 0U, "00"); PerformWriterTest(test, 1U, "01"); @@ -147,7 +351,7 @@ public void Test_WriteVarint32() [Test] public void Test_WriteVarint64() { - TestHandler test = (ref SliceWriter writer, ulong value) => writer.WriteVarint64(value); + TestHandler test = (ref SliceWriter writer, ulong value) => writer.WriteVarInt64(value); PerformWriterTest(test, 0UL, "00"); PerformWriterTest(test, 1UL, "01"); @@ -177,7 +381,7 @@ public void Test_WriteVarint64() [Test] public void Test_WriteVarBytes() { - TestHandler test = (ref SliceWriter writer, Slice value) => writer.WriteVarbytes(value); + TestHandler test = (ref SliceWriter writer, Slice value) => writer.WriteVarBytes(value); PerformWriterTest(test, Slice.Nil, "00"); PerformWriterTest(test, Slice.Empty, "00"); @@ -187,6 +391,108 @@ public void Test_WriteVarBytes() PerformWriterTest(test, Slice.FromFixedU32(0xDEADBEEF), "04 EF BE AD DE"); } + [Test] + public void Test_WriteBase10_Signed() + { + TestHandler test = (ref SliceWriter writer, int value) => writer.WriteBase10(value); + + // positive numbers + PerformWriterTest(test, 0, "30"); + PerformWriterTest(test, 1, "31"); + PerformWriterTest(test, 9, "39"); + PerformWriterTest(test, 10, "31 30"); + PerformWriterTest(test, 42, "34 32"); + PerformWriterTest(test, 99, "39 39"); + PerformWriterTest(test, 100, "31 30 30"); + PerformWriterTest(test, 123, "31 32 33"); + PerformWriterTest(test, 999, "39 39 39"); + PerformWriterTest(test, 1000, "31 30 30 30"); + PerformWriterTest(test, 1234, "31 32 33 34"); + PerformWriterTest(test, 9999, "39 39 39 39"); + PerformWriterTest(test, 10000, "31 30 30 30 30"); + PerformWriterTest(test, 12345, "31 32 33 34 35"); + PerformWriterTest(test, 99999, "39 39 39 39 39"); + PerformWriterTest(test, 100000, "31 30 30 30 30 30"); + PerformWriterTest(test, 123456, "31 32 33 34 35 36"); + PerformWriterTest(test, 999999, "39 39 39 39 39 39"); + PerformWriterTest(test, 1000000, "31 30 30 30 30 30 30"); + PerformWriterTest(test, 1234567, "31 32 33 34 35 36 37"); + PerformWriterTest(test, 9999999, "39 39 39 39 39 39 39"); + PerformWriterTest(test, 10000000, "31 30 30 30 30 30 30 30"); + PerformWriterTest(test, 12345678, "31 32 33 34 35 36 37 38"); + PerformWriterTest(test, 99999999, "39 39 39 39 39 39 39 39"); + PerformWriterTest(test, 100000000, "31 30 30 30 30 30 30 30 30"); + PerformWriterTest(test, 123456789, "31 32 33 34 35 36 37 38 39"); + PerformWriterTest(test, 999999999, "39 39 39 39 39 39 39 39 39"); + PerformWriterTest(test, int.MaxValue, "32 31 34 37 34 38 33 36 34 37"); + + // negative numbers + PerformWriterTest(test, -1, "2D 31"); + PerformWriterTest(test, -9, "2D 39"); + PerformWriterTest(test, -10, "2D 31 30"); + PerformWriterTest(test, -42, "2D 34 32"); + PerformWriterTest(test, -99, "2D 39 39"); + PerformWriterTest(test, -100, "2D 31 30 30"); + PerformWriterTest(test, -123, "2D 31 32 33"); + PerformWriterTest(test, -999, "2D 39 39 39"); + PerformWriterTest(test, -1000, "2D 31 30 30 30"); + PerformWriterTest(test, -1234, "2D 31 32 33 34"); + PerformWriterTest(test, -9999, "2D 39 39 39 39"); + PerformWriterTest(test, -10000, "2D 31 30 30 30 30"); + PerformWriterTest(test, -12345, "2D 31 32 33 34 35"); + PerformWriterTest(test, -99999, "2D 39 39 39 39 39"); + PerformWriterTest(test, -100000, "2D 31 30 30 30 30 30"); + PerformWriterTest(test, -123456, "2D 31 32 33 34 35 36"); + PerformWriterTest(test, -999999, "2D 39 39 39 39 39 39"); + PerformWriterTest(test, -1000000, "2D 31 30 30 30 30 30 30"); + PerformWriterTest(test, -1234567, "2D 31 32 33 34 35 36 37"); + PerformWriterTest(test, -9999999, "2D 39 39 39 39 39 39 39"); + PerformWriterTest(test, -10000000, "2D 31 30 30 30 30 30 30 30"); + PerformWriterTest(test, -12345678, "2D 31 32 33 34 35 36 37 38"); + PerformWriterTest(test, -99999999, "2D 39 39 39 39 39 39 39 39"); + PerformWriterTest(test, -100000000, "2D 31 30 30 30 30 30 30 30 30"); + PerformWriterTest(test, -123456789, "2D 31 32 33 34 35 36 37 38 39"); + PerformWriterTest(test, -999999999, "2D 39 39 39 39 39 39 39 39 39"); + PerformWriterTest(test, int.MinValue, "2D 32 31 34 37 34 38 33 36 34 38"); + } + + [Test] + public void Test_WriteBase10_Unsigned() + { + TestHandler test = (ref SliceWriter writer, uint value) => writer.WriteBase10(value); + + // positive numbers + PerformWriterTest(test, 0, "30"); + PerformWriterTest(test, 1, "31"); + PerformWriterTest(test, 9, "39"); + PerformWriterTest(test, 10, "31 30"); + PerformWriterTest(test, 42, "34 32"); + PerformWriterTest(test, 99, "39 39"); + PerformWriterTest(test, 100, "31 30 30"); + PerformWriterTest(test, 123, "31 32 33"); + PerformWriterTest(test, 999, "39 39 39"); + PerformWriterTest(test, 1000, "31 30 30 30"); + PerformWriterTest(test, 1234, "31 32 33 34"); + PerformWriterTest(test, 9999, "39 39 39 39"); + PerformWriterTest(test, 10000, "31 30 30 30 30"); + PerformWriterTest(test, 12345, "31 32 33 34 35"); + PerformWriterTest(test, 99999, "39 39 39 39 39"); + PerformWriterTest(test, 100000, "31 30 30 30 30 30"); + PerformWriterTest(test, 123456, "31 32 33 34 35 36"); + PerformWriterTest(test, 999999, "39 39 39 39 39 39"); + PerformWriterTest(test, 1000000, "31 30 30 30 30 30 30"); + PerformWriterTest(test, 1234567, "31 32 33 34 35 36 37"); + PerformWriterTest(test, 9999999, "39 39 39 39 39 39 39"); + PerformWriterTest(test, 10000000, "31 30 30 30 30 30 30 30"); + PerformWriterTest(test, 12345678, "31 32 33 34 35 36 37 38"); + PerformWriterTest(test, 99999999, "39 39 39 39 39 39 39 39"); + PerformWriterTest(test, 100000000, "31 30 30 30 30 30 30 30 30"); + PerformWriterTest(test, 123456789, "31 32 33 34 35 36 37 38 39"); + PerformWriterTest(test, 999999999, "39 39 39 39 39 39 39 39 39"); + PerformWriterTest(test, int.MaxValue, "32 31 34 37 34 38 33 36 34 37"); + PerformWriterTest(test, uint.MaxValue, "34 32 39 34 39 36 37 32 39 35"); + } + [Test] public void Test_Indexer() { @@ -263,5 +569,245 @@ public void Test_Skip() Assert.That(writer.ToSlice().ToString(), Is.EqualTo("helloworld*****")); } + [Test] + public void Test_ToSlice() + { + var writer = new SliceWriter(64); + var slice = writer.ToSlice(); + //note: slice.Array is not guaranteed to be equal to writer.Buffer + Assert.That(slice.Count, Is.EqualTo(0)); + Assert.That(slice.Offset, Is.EqualTo(0)); + + writer.WriteBytes(Slice.FromString("hello world!")); + slice = writer.ToSlice(); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(12)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!")); + + writer.WriteBytes(Slice.FromString("foo")); + slice = writer.ToSlice(); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(15)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!foo")); + } + + [Test] + public void Test_Head() + { + var writer = new SliceWriter(64); + var slice = writer.Head(0); + Assert.That(slice.Count, Is.EqualTo(0)); + Assert.That(slice.Offset, Is.EqualTo(0)); + //note: slice.Array is not guaranteed to be equal to writer.Buffer + Assert.That(() => writer.Head(1), Throws.InstanceOf()); + + writer.WriteBytes(Slice.FromString("hello world!")); + slice = writer.Head(5); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(5)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello")); + + slice = writer.Head(12); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(12)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!")); + + Assert.That(() => writer.Head(13), Throws.InstanceOf()); + Assert.That(() => writer.Head(-1), Throws.InstanceOf()); + + writer.WriteBytes(Slice.FromString("foo")); + slice = writer.Head(3); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(3)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hel")); + + slice = writer.Head(15); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(15)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!foo")); + + Assert.That(() => writer.Head(16), Throws.InstanceOf()); + + } + + [Test] + public void Test_Tail() + { + var writer = new SliceWriter(64); + var slice = writer.Tail(0); + Assert.That(slice.Count, Is.EqualTo(0)); + Assert.That(slice.Offset, Is.EqualTo(0)); + //note: slice.Array is not guaranteed to be equal to writer.Buffer + Assert.That(() => writer.Head(1), Throws.InstanceOf()); + + writer.WriteBytes(Slice.FromString("hello world!")); + slice = writer.Tail(6); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(6)); + Assert.That(slice.Count, Is.EqualTo(6)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("world!")); + + slice = writer.Tail(12); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(12)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!")); + + Assert.That(() => writer.Tail(13), Throws.InstanceOf()); + Assert.That(() => writer.Tail(-1), Throws.InstanceOf()); + + writer.WriteBytes(Slice.FromString("foo")); + slice = writer.Tail(3); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(12)); + Assert.That(slice.Count, Is.EqualTo(3)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("foo")); + + slice = writer.Tail(15); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(15)); + Assert.That(slice.ToStringAscii(), Is.EqualTo("hello world!foo")); + + Assert.That(() => writer.Tail(16), Throws.InstanceOf()); + + } + + [Test] + public void Test_AppendBytes() + { + var writer = new SliceWriter(64); + var slice = writer.AppendBytes(Slice.Empty); + //note: slice.Array is not guaranteed to be equal to writer.Buffer + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(0)); + + slice = writer.AppendBytes(Slice.FromString("hello world!")); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(0)); + Assert.That(slice.Count, Is.EqualTo(12)); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("hello world!")); + Assert.That(writer.ToSlice().ToStringUtf8(), Is.EqualTo("hello world!")); + + var foo = Slice.FromString("foo"); + slice = writer.AppendBytes(foo); + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(12)); + Assert.That(slice.Count, Is.EqualTo(3)); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("foo")); + Assert.That(writer.ToSlice().ToStringUtf8(), Is.EqualTo("hello world!foo")); + + var bar = Slice.FromString("bar"); + unsafe + { + fixed (byte* ptr = &bar.DangerousGetPinnableReference()) + { + slice = writer.AppendBytes(ptr, 3); + } + } + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(15)); + Assert.That(slice.Count, Is.EqualTo(3)); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("bar")); + Assert.That(writer.ToSlice().ToStringUtf8(), Is.EqualTo("hello world!foobar")); + + var baz = Slice.FromString("baz"); + unsafe + { + fixed (byte* ptr = &baz.DangerousGetPinnableReference()) + { + //TODO: this test was using ReadOnlySpan, update it once we enable support for these! + slice = writer.AppendBytes(ptr, 3); + } + } + Assert.That(slice.Array, Is.SameAs(writer.Buffer)); + Assert.That(slice.Offset, Is.EqualTo(18)); + Assert.That(slice.Count, Is.EqualTo(3)); + Assert.That(slice.ToStringUtf8(), Is.EqualTo("baz")); + Assert.That(writer.ToSlice().ToStringUtf8(), Is.EqualTo("hello world!foobarbaz")); + + unsafe + { + slice = writer.AppendBytes(null, 0); + } + //note: slice.Array is not guaranteed to be equal to writer.Buffer + Assert.That(slice.Offset, Is.EqualTo(0)); //REVIEW: should we return (Buffer, Position, 0) instead of (EmptyArray, 0, 0) ? + Assert.That(slice.Count, Is.EqualTo(0)); + } + + [Test] + public void Test_WriteBytes_Resize_Buffer() + { + + // check buffer resize occurs as intended + var original = new byte[32]; + var writer = new SliceWriter(original); + Assert.That(writer.Buffer, Is.SameAs(original)); + + // first write should not resize the buffer + writer.WriteBytes(Slice.Repeat((byte)'a', 24)); + Assert.That(writer.Buffer, Is.SameAs(original)); + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("aaaaaaaaaaaaaaaaaaaaaaaa")); + + // second write should resize the buffer + writer.WriteBytes(Slice.Repeat((byte)'b', 24)); + // buffer should have been replaced with larger one + Assert.That(writer.Buffer, Is.Not.SameAs(original)); + Assert.That(writer.Buffer.Length, Is.GreaterThanOrEqualTo(48)); + + //but the content should be unchanged + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("aaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbb")); + + // adding exactly what is missing should not resize the buffer + writer = new SliceWriter(original); + writer.WriteBytes(Slice.Repeat((byte)'c', original.Length)); + Assert.That(writer.Buffer, Is.SameAs(original)); + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("cccccccccccccccccccccccccccccccc")); + + // adding nothing should not resize the buffer + writer.WriteBytes(Slice.Empty); + Assert.That(writer.Buffer, Is.SameAs(original)); + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("cccccccccccccccccccccccccccccccc")); + + // adding a single byte should resize the buffer + writer.WriteBytes(Slice.FromChar('Z')); + Assert.That(writer.Buffer, Is.Not.SameAs(original)); + Assert.That(writer.Buffer.Length, Is.GreaterThanOrEqualTo(33)); + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("ccccccccccccccccccccccccccccccccZ")); + } + + [Test] + public void Test_AppendBytes_Resize_Buffer() + { + + // check buffer resize occurs as intended + var original = new byte[32]; + var writer = new SliceWriter(original); + Assert.That(writer.Buffer, Is.SameAs(original)); + + // first write should not resize the buffer + var aaa = writer.AppendBytes(Slice.Repeat((byte) 'a', 24)); + Assert.That(aaa.Array, Is.SameAs(original)); + + // second write should resize the buffer + var bbb = writer.AppendBytes(Slice.Repeat((byte) 'b', 24)); + Assert.That(bbb.Array, Is.SameAs(writer.Buffer)); + //note: buffer should have been copied between both calls, so 'aaa' should point to the OLD buffer + Assert.That(bbb.Array, Is.Not.SameAs(original)); + //but the content should be unchanged + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("aaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbb")); + // => mutating aaa should not change the buffer + aaa.Array[aaa.Offset] = (byte) 'Z'; + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("aaaaaaaaaaaaaaaaaaaaaaaabbbbbbbbbbbbbbbbbbbbbbbb")); + // => but mutating bbb should change the buffer + bbb.Array[bbb.Offset] = (byte)'Z'; + Assert.That(writer.ToSlice().ToStringAscii(), Is.EqualTo("aaaaaaaaaaaaaaaaaaaaaaaaZbbbbbbbbbbbbbbbbbbbbbbb")); + } } } diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs new file mode 100644 index 000000000..a639f6c28 --- /dev/null +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -0,0 +1,2144 @@ +#region BSD Licence +/* Copyright (c) 2013, Doxense SARL +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 + +//#define ENABLE_VALUETUPLE + +// ReSharper disable AccessToModifiedClosure +namespace Doxense.Collections.Tuples.Tests +{ + using System; + using System.Linq; + using System.Net; + using Doxense.Runtime.Converters; + using FoundationDB.Client.Tests; + using NUnit.Framework; + + [TestFixture] + public class TupleFacts : FdbTest + { + + #region General Use... + + [Test] + public void Test_Tuple_0() + { + var t0 = STuple.Create(); + Assert.That(t0.Count, Is.EqualTo(0)); + Assert.That(t0.ToArray(), Is.EqualTo(new object[0])); + Assert.That(t0.ToString(), Is.EqualTo("()")); + Assert.That(t0, Is.InstanceOf()); + } + + [Test] + public void Test_Tuple_1() + { + var t1 = STuple.Create("hello world"); + Assert.That(t1.Count, Is.EqualTo(1)); + Assert.That(t1.Item1, Is.EqualTo("hello world")); + Assert.That(t1.Get(0), Is.EqualTo("hello world")); + Assert.That(t1[0], Is.EqualTo("hello world")); + Assert.That(t1.ToArray(), Is.EqualTo(new object[] { "hello world" })); + Assert.That(t1.ToString(), Is.EqualTo("(\"hello world\",)")); + Assert.That(t1, Is.InstanceOf>()); + + Assert.That(STuple.Create(123).GetHashCode(), Is.EqualTo(STuple.Create("Hello", 123).Tail.GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123).GetHashCode(), Is.EqualTo(STuple.Create(123L).GetHashCode()), "Hashcode should be stable"); + + // ReSharper disable CannotApplyEqualityOperatorToType + // ReSharper disable EqualExpressionComparison + Assert.That(STuple.Create(123) == STuple.Create(123), Is.True, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123) != STuple.Create(123), Is.False, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123) == STuple.Create(456), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123) != STuple.Create(456), Is.True, "op_Inequality should work for struct tuples"); + // ReSharper restore EqualExpressionComparison + // ReSharper restore CannotApplyEqualityOperatorToType + + { // Deconstruct + t1.Deconstruct(out string item1); + Assert.That(item1, Is.EqualTo("hello world")); + } + } + + [Test] + public void Test_Tuple_2() + { + var t2 = STuple.Create("hello world", 123); + Assert.That(t2.Count, Is.EqualTo(2)); + Assert.That(t2.Item1, Is.EqualTo("hello world")); + Assert.That(t2.Item2, Is.EqualTo(123)); + Assert.That(t2.Get(0), Is.EqualTo("hello world")); + Assert.That(t2.Get(1), Is.EqualTo(123)); + Assert.That(t2[0], Is.EqualTo("hello world")); + Assert.That(t2[1], Is.EqualTo(123)); + Assert.That(t2.ToArray(), Is.EqualTo(new object[] { "hello world", 123 })); + Assert.That(t2.ToString(), Is.EqualTo("(\"hello world\", 123)")); + Assert.That(t2, Is.InstanceOf>()); + + Assert.That(t2.Tail.Count, Is.EqualTo(1)); + Assert.That(t2.Tail.Item1, Is.EqualTo(123)); + + Assert.That(STuple.Create(123, true).GetHashCode(), Is.EqualTo(STuple.Create("Hello", 123, true).Tail.GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123, true).GetHashCode(), Is.EqualTo(STuple.Create(123L, 1).GetHashCode()), "Hashcode should be stable"); + + // ReSharper disable CannotApplyEqualityOperatorToType + // ReSharper disable EqualExpressionComparison + Assert.That(STuple.Create(123, true) == STuple.Create(123, true), Is.True, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true) != STuple.Create(123, true), Is.False, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true) == STuple.Create(456, true), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true) != STuple.Create(456, true), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true) == STuple.Create(123, false), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true) != STuple.Create(123, false), Is.True, "op_Inequality should work for struct tuples"); + // ReSharper restore EqualExpressionComparison + // ReSharper restore CannotApplyEqualityOperatorToType + + { // Deconstruct + t2.Deconstruct(out string item1, out int item2); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + } +#if ENABLE_VALUETUPLE + { // Deconstruct + (string item1, int item2) = t2; + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + } +#endif + } + + [Test] + public void Test_Tuple_3() + { + var t3 = STuple.Create("hello world", 123, false); + Assert.That(t3.Count, Is.EqualTo(3)); + Assert.That(t3.Item1, Is.EqualTo("hello world")); + Assert.That(t3.Item2, Is.EqualTo(123)); + Assert.That(t3.Item3, Is.False); + Assert.That(t3.Get(0), Is.EqualTo("hello world")); + Assert.That(t3.Get(1), Is.EqualTo(123)); + Assert.That(t3.Get(2), Is.False); + Assert.That(t3[0], Is.EqualTo("hello world")); + Assert.That(t3[1], Is.EqualTo(123)); + Assert.That(t3[2], Is.False); + Assert.That(t3.ToArray(), Is.EqualTo(new object[] { "hello world", 123, false })); + Assert.That(t3.ToString(), Is.EqualTo(@"(""hello world"", 123, false)")); + Assert.That(t3, Is.InstanceOf>()); + + Assert.That(t3.Tail.Count, Is.EqualTo(2)); + Assert.That(t3.Tail.Item1, Is.EqualTo(123)); + Assert.That(t3.Tail.Item2, Is.False); + + Assert.That(STuple.Create(123, true, "foo").GetHashCode(), Is.EqualTo(STuple.Create("Hello", 123, true, "foo").Tail.GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123, true, "foo").GetHashCode(), Is.EqualTo(STuple.Create(123L, 1, "foo").GetHashCode()), "Hashcode should be stable"); + + // ReSharper disable CannotApplyEqualityOperatorToType + // ReSharper disable EqualExpressionComparison + Assert.That(STuple.Create(123, true, "foo") == STuple.Create(123, true, "foo"), Is.True, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") != STuple.Create(123, true, "foo"), Is.False, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") == STuple.Create(456, true, "foo"), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") != STuple.Create(456, true, "foo"), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") == STuple.Create(123, false, "foo"), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") != STuple.Create(123, false, "foo"), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") == STuple.Create(123, true, "bar"), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo") != STuple.Create(123, true, "bar"), Is.True, "op_Inequality should work for struct tuples"); + // ReSharper restore EqualExpressionComparison + // ReSharper restore CannotApplyEqualityOperatorToType + + { // Deconstruct + t3.Deconstruct(out string item1, out int item2, out bool item3); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + } +#if ENABLE_VALUETUPLE + { // Deconstruct + (string item1, int item2, bool item3) = t3; + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + } +#endif + } + + [Test] + public void Test_Tuple_4() + { + var t4 = STuple.Create("hello world", 123, false, 1234L); + Assert.That(t4.Count, Is.EqualTo(4)); + Assert.That(t4.Item1, Is.EqualTo("hello world")); + Assert.That(t4.Item2, Is.EqualTo(123)); + Assert.That(t4.Item3, Is.False); + Assert.That(t4.Item4, Is.EqualTo(1234L)); + Assert.That(t4.Get(0), Is.EqualTo("hello world")); + Assert.That(t4.Get(1), Is.EqualTo(123)); + Assert.That(t4.Get(2), Is.False); + Assert.That(t4.Get(3), Is.EqualTo(1234L)); + Assert.That(t4[0], Is.EqualTo("hello world")); + Assert.That(t4[1], Is.EqualTo(123)); + Assert.That(t4[2], Is.False); + Assert.That(t4[3], Is.EqualTo(1234L)); + Assert.That(t4.ToArray(), Is.EqualTo(new object[] { "hello world", 123, false, 1234L})); + Assert.That(t4.ToString(), Is.EqualTo(@"(""hello world"", 123, false, 1234)")); + Assert.That(t4, Is.InstanceOf>()); + + Assert.That(t4.Tail.Count, Is.EqualTo(3)); + Assert.That(t4.Tail.Item1, Is.EqualTo(123)); + Assert.That(t4.Tail.Item2, Is.False); + Assert.That(t4.Tail.Item3, Is.EqualTo(1234L)); + + Assert.That(STuple.Create(123, true, "foo", 666).GetHashCode(), Is.EqualTo(STuple.Create("Hello", 123, true, "foo", 666).Tail.GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123, true, "foo", 666).GetHashCode(), Is.EqualTo(STuple.Create(123L, 1, "foo", 666UL).GetHashCode()), "Hashcode should be stable"); + + // ReSharper disable CannotApplyEqualityOperatorToType + // ReSharper disable EqualExpressionComparison + Assert.That(STuple.Create(123, true, "foo", 666) == STuple.Create(123, true, "foo", 666), Is.True, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) != STuple.Create(123, true, "foo", 666), Is.False, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) == STuple.Create(456, true, "foo", 666), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) != STuple.Create(456, true, "foo", 666), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) == STuple.Create(123, false, "foo", 666), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) != STuple.Create(123, false, "foo", 666), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) == STuple.Create(123, true, "bar", 666), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) != STuple.Create(123, true, "bar", 666), Is.True, "op_Inequality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) == STuple.Create(123, true, "foo", 667), Is.False, "op_Equality should work for struct tuples"); + Assert.That(STuple.Create(123, true, "foo", 666) != STuple.Create(123, true, "foo", 667), Is.True, "op_Inequality should work for struct tuples"); + // ReSharper restore EqualExpressionComparison + // ReSharper restore CannotApplyEqualityOperatorToType + + { // Deconstruct + t4.Deconstruct(out string item1, out int item2, out bool item3, out long item4); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + } +#if ENABLE_VALUETUPLE + { // Deconstruct + (string item1, int item2, bool item3, long item4) = t4; + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + } +#endif + } + + [Test] + public void Test_Tuple_5() + { + var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); + Assert.That(t5.Count, Is.EqualTo(5)); + Assert.That(t5.Item1, Is.EqualTo("hello world")); + Assert.That(t5.Item2, Is.EqualTo(123)); + Assert.That(t5.Item3, Is.False); + Assert.That(t5.Item4, Is.EqualTo(1234L)); + Assert.That(t5.Item5, Is.EqualTo(-1234)); + Assert.That(t5.Get(0), Is.EqualTo("hello world")); + Assert.That(t5.Get(1), Is.EqualTo(123)); + Assert.That(t5.Get(2), Is.False); + Assert.That(t5.Get(3), Is.EqualTo(1234L)); + Assert.That(t5.Get(4), Is.EqualTo(-1234)); + Assert.That(t5[0], Is.EqualTo("hello world")); + Assert.That(t5[1], Is.EqualTo(123)); + Assert.That(t5[2], Is.False); + Assert.That(t5[3], Is.EqualTo(1234L)); + Assert.That(t5[4], Is.EqualTo(-1234)); + Assert.That(t5.ToArray(), Is.EqualTo(new object[] { "hello world", 123, false, 1234L, -1234 })); + Assert.That(t5.ToString(), Is.EqualTo(@"(""hello world"", 123, false, 1234, -1234)")); + Assert.That(t5, Is.InstanceOf>()); + + Assert.That(t5.Tail.Count, Is.EqualTo(4)); + Assert.That(t5.Tail.Item1, Is.EqualTo(123)); + Assert.That(t5.Tail.Item2, Is.False); + Assert.That(t5.Tail.Item3, Is.EqualTo(1234L)); + Assert.That(t5.Tail.Item4, Is.EqualTo(-1234L)); + + Assert.That(STuple.Create(123, true, "foo", 666, false).GetHashCode(), Is.EqualTo(STuple.Create("Hello", 123, true, "foo", 666, false).Tail.GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123, true, "foo", 666, false).GetHashCode(), Is.EqualTo(STuple.Create(123L, 1, "foo", 666UL, 0).GetHashCode()), "Hashcode should be stable"); + + { // Deconstruct + t5.Deconstruct(out string item1, out int item2, out bool item3, out long item4, out long item5); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + Assert.That(item5, Is.EqualTo(-1234L)); + } +#if ENABLE_VALUETUPLE + { // Deconstruct + (string item1, int item2, bool item3, long item4, long item5) = t5; + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + Assert.That(item5, Is.EqualTo(-1234L)); + } +#endif + } + + [Test] + public void Test_Tuple_6() + { + var t6 = STuple.Create("hello world", 123, false, 1234L, -1234, "six"); + Assert.That(t6.Count, Is.EqualTo(6)); + Assert.That(t6.Item1, Is.EqualTo("hello world")); + Assert.That(t6.Item2, Is.EqualTo(123)); + Assert.That(t6.Item3, Is.False); + Assert.That(t6.Item4, Is.EqualTo(1234L)); + Assert.That(t6.Item5, Is.EqualTo(-1234)); + Assert.That(t6.Get(0), Is.EqualTo("hello world")); + Assert.That(t6.Get(1), Is.EqualTo(123)); + Assert.That(t6.Get(2), Is.False); + Assert.That(t6.Get(3), Is.EqualTo(1234L)); + Assert.That(t6.Get(4), Is.EqualTo(-1234)); + Assert.That(t6.Get(5), Is.EqualTo("six")); + Assert.That(t6[0], Is.EqualTo("hello world")); + Assert.That(t6[1], Is.EqualTo(123)); + Assert.That(t6[2], Is.False); + Assert.That(t6[3], Is.EqualTo(1234L)); + Assert.That(t6[4], Is.EqualTo(-1234)); + Assert.That(t6[5], Is.EqualTo("six")); + Assert.That(t6.ToArray(), Is.EqualTo(new object[] { "hello world", 123, false, 1234L, -1234, "six" })); + Assert.That(t6.ToString(), Is.EqualTo(@"(""hello world"", 123, false, 1234, -1234, ""six"")")); + Assert.That(t6, Is.InstanceOf>()); + + Assert.That(t6.Tail.Count, Is.EqualTo(5)); + Assert.That(t6.Tail.Item1, Is.EqualTo(123)); + Assert.That(t6.Tail.Item2, Is.False); + Assert.That(t6.Tail.Item3, Is.EqualTo(1234L)); + Assert.That(t6.Tail.Item4, Is.EqualTo(-1234L)); + Assert.That(t6.Tail.Item5, Is.EqualTo("six")); + + Assert.That(STuple.Create(123, true, "foo", 666, false, "bar").GetHashCode(), Is.EqualTo(STuple.Create(123, true, "foo", 666, false).Append("bar").GetHashCode()), "Hashcode should be stable"); + Assert.That(STuple.Create(123, true, "foo", 666, false, "bar").GetHashCode(), Is.EqualTo(STuple.Create(123L, 1, "foo", 666UL, 0, "bar").GetHashCode()), "Hashcode should be stable"); + + { // Deconstruct + t6.Deconstruct(out string item1, out int item2, out bool item3, out long item4, out long item5, out string item6); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + Assert.That(item5, Is.EqualTo(-1234L)); + Assert.That(item6, Is.EqualTo("six")); + } +#if ENABLE_VALUETUPLE + { // Deconstruct + (string item1, int item2, bool item3, long item4, long item5, string item6) = t6; + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234L)); + Assert.That(item5, Is.EqualTo(-1234L)); + Assert.That(item6, Is.EqualTo("six")); + } +#endif + } + + [Test] + public void Test_Tuple_Many() + { + // ReSharper disable once RedundantExplicitParamsArrayCreation + ITuple tn = STuple.Create(new object[] { "hello world", 123, false, 1234L, -1234, "six", true, Math.PI }); + Assert.That(tn.Count, Is.EqualTo(8)); + Assert.That(tn.Get(0), Is.EqualTo("hello world")); + Assert.That(tn.Get(1), Is.EqualTo(123)); + Assert.That(tn.Get(2), Is.False); + Assert.That(tn.Get(3), Is.EqualTo(1234)); + Assert.That(tn.Get(4), Is.EqualTo(-1234)); + Assert.That(tn.Get(5), Is.EqualTo("six")); + Assert.That(tn.Get(6), Is.True); + Assert.That(tn.Get(7), Is.EqualTo(Math.PI)); + Assert.That(tn.ToArray(), Is.EqualTo(new object[] { "hello world", 123, false, 1234L, -1234, "six", true, Math.PI })); + Assert.That(tn.ToString(), Is.EqualTo("(\"hello world\", 123, false, 1234, -1234, \"six\", true, 3.1415926535897931)")); + Assert.That(tn, Is.InstanceOf()); + + { // Deconstruct + string item1; + int item2; + bool item3; + long item4; + long item5; + string item6; + bool item7; + double item8; + + Assert.That(() => tn.Deconstruct(out item1), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2, out item3), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2, out item3, out item4), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2, out item3, out item4, out item5), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2, out item3, out item4, out item5, out item6), Throws.InvalidOperationException); + Assert.That(() => tn.Deconstruct(out item1, out item2, out item3, out item4, out item5, out item6, out item7), Throws.InvalidOperationException); + + tn.Deconstruct(out item1, out item2, out item3, out item4, out item5, out item6, out item7, out item8); + Assert.That(item1, Is.EqualTo("hello world")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(1234)); + Assert.That(item5, Is.EqualTo(-1234)); + Assert.That(item6, Is.EqualTo("six")); + Assert.That(item7, Is.True); + Assert.That(item8, Is.EqualTo(Math.PI)); + } + } + + [Test] + public void Test_Tuple_Wrap() + { + // STuple.Wrap(...) does not copy the items of the array + + var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; + + var t = STuple.Wrap(arr); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(4)); + Assert.That(t[0], Is.EqualTo("Hello")); + Assert.That(t[1], Is.EqualTo(123)); + Assert.That(t[2], Is.False); + Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); + + { // Deconstruct + t.Deconstruct(out string item1, out int item2, out bool item3, out TimeSpan item4); + Assert.That(item1, Is.EqualTo("Hello")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + } + + t = STuple.Wrap(arr, 1, 2); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(2)); + Assert.That(t[0], Is.EqualTo(123)); + Assert.That(t[1], Is.False); + + // changing the underyling array should change the tuple + // DON'T DO THIS IN ACTUAL CODE!!! + + arr[1] = 456; + arr[2] = true; + Log($"t = {t}"); + + Assert.That(t[0], Is.EqualTo(456)); + Assert.That(t[1], Is.True); + + { // Deconstruct + t.Deconstruct(out int item1, out bool item2); + Assert.That(item1, Is.EqualTo(456)); + Assert.That(item2, Is.True); + } + + } + + [Test] + public void Test_Tuple_FromObjects() + { + // STuple.FromObjects(...) does a copy of the items of the array + + var arr = new object[] { "Hello", 123, false, TimeSpan.FromSeconds(5) }; + + var t = STuple.FromObjects(arr); + Log($"t = {t}"); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(4)); + Assert.That(t[0], Is.EqualTo("Hello")); + Assert.That(t[1], Is.EqualTo(123)); + Assert.That(t[2], Is.False); + Assert.That(t[3], Is.EqualTo(TimeSpan.FromSeconds(5))); + + { // Deconstruct + t.Deconstruct(out string item1, out int item2, out bool item3, out TimeSpan item4); + Assert.That(item1, Is.EqualTo("Hello")); + Assert.That(item2, Is.EqualTo(123)); + Assert.That(item3, Is.False); + Assert.That(item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + } + + t = STuple.FromObjects(arr, 1, 2); + Log($"t = {t}"); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(2)); + Assert.That(t[0], Is.EqualTo(123)); + Assert.That(t[1], Is.False); + + { // Deconstruct + t.Deconstruct(out int item1, out bool item2); + Assert.That(item1, Is.EqualTo(123)); + Assert.That(item2, Is.False); + } + + // changing the underyling array should NOT change the tuple + + arr[1] = 456; + arr[2] = true; + Log($"t = {t}"); + + Assert.That(t[0], Is.EqualTo(123)); + Assert.That(t[1], Is.False); + + } + + [Test] + public void Test_Tuple_FromArray() + { + var items = new[] { "Bonjour", "le", "Monde" }; + + var t = STuple.FromArray(items); + Log($"t = {t}"); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(3)); + Assert.That(t[0], Is.EqualTo("Bonjour")); + Assert.That(t[1], Is.EqualTo("le")); + Assert.That(t[2], Is.EqualTo("Monde")); + + { // Deconstruct + t.Deconstruct(out string item1, out string item2, out string item3); + Assert.That(item1, Is.EqualTo("Bonjour")); + Assert.That(item2, Is.EqualTo("le")); + Assert.That(item3, Is.EqualTo("Monde")); + } + + t = STuple.FromArray(items, 1, 2); + Log($"t = {t}"); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(2)); + Assert.That(t[0], Is.EqualTo("le")); + Assert.That(t[1], Is.EqualTo("Monde")); + { // Deconstruct + t.Deconstruct(out string item1, out string item2); + Assert.That(item1, Is.EqualTo("le")); + Assert.That(item2, Is.EqualTo("Monde")); + } + + // changing the underlying array should NOT change the tuple + items[1] = "ze"; + Log($"t = {t}"); + + Assert.That(t[0], Is.EqualTo("le")); + } + + [Test] + public void Test_Tuple_Negative_Indexing() + { + var t1 = STuple.Create("hello world"); + Assert.That(t1.Get(-1), Is.EqualTo("hello world")); + Assert.That(t1[-1], Is.EqualTo("hello world")); + + var t2 = STuple.Create("hello world", 123); + Assert.That(t2.Get(-1), Is.EqualTo(123)); + Assert.That(t2.Get(-2), Is.EqualTo("hello world")); + Assert.That(t2[-1], Is.EqualTo(123)); + Assert.That(t2[-2], Is.EqualTo("hello world")); + + var t3 = STuple.Create("hello world", 123, false); + Assert.That(t3.Get(-1), Is.False); + Assert.That(t3.Get(-2), Is.EqualTo(123)); + Assert.That(t3.Get(-3), Is.EqualTo("hello world")); + Assert.That(t3[-1], Is.False); + Assert.That(t3[-2], Is.EqualTo(123)); + Assert.That(t3[-3], Is.EqualTo("hello world")); + + var t4 = STuple.Create("hello world", 123, false, 1234L); + Assert.That(t4.Get(-1), Is.EqualTo(1234L)); + Assert.That(t4.Get(-2), Is.False); + Assert.That(t4.Get(-3), Is.EqualTo(123)); + Assert.That(t4.Get(-4), Is.EqualTo("hello world")); + Assert.That(t4[-1], Is.EqualTo(1234L)); + Assert.That(t4[-2], Is.False); + Assert.That(t4[-3], Is.EqualTo(123)); + Assert.That(t4[-4], Is.EqualTo("hello world")); + + var t5 = STuple.Create("hello world", 123, false, 1234L, -1234); + Assert.That(t5.Get(-1), Is.EqualTo(-1234)); + Assert.That(t5.Get(-2), Is.EqualTo(1234L)); + Assert.That(t5.Get(-3), Is.False); + Assert.That(t5.Get(-4), Is.EqualTo(123)); + Assert.That(t5.Get(-5), Is.EqualTo("hello world")); + Assert.That(t5[-1], Is.EqualTo(-1234)); + Assert.That(t5[-2], Is.EqualTo(1234L)); + Assert.That(t5[-3], Is.False); + Assert.That(t5[-4], Is.EqualTo(123)); + Assert.That(t5[-5], Is.EqualTo("hello world")); + + // ReSharper disable once RedundantExplicitParamsArrayCreation + var tn = STuple.Create(new object[] { "hello world", 123, false, 1234, -1234, "six" }); + Assert.That(tn.Get(-1), Is.EqualTo("six")); + Assert.That(tn.Get(-2), Is.EqualTo(-1234)); + Assert.That(tn.Get(-3), Is.EqualTo(1234)); + Assert.That(tn.Get(-4), Is.False); + Assert.That(tn.Get(-5), Is.EqualTo(123)); + Assert.That(tn.Get(-6), Is.EqualTo("hello world")); + Assert.That(tn[-1], Is.EqualTo("six")); + Assert.That(tn[-2], Is.EqualTo(-1234)); + Assert.That(tn[-3], Is.EqualTo(1234)); + Assert.That(tn[-4], Is.False); + Assert.That(tn[-5], Is.EqualTo(123)); + Assert.That(tn[-6], Is.EqualTo("hello world")); + } + + [Test] + public void Test_Tuple_First_And_Last() + { + // tuple.First() should be equivalent to tuple.Get(0) + // tuple.Last() should be equivalent to tuple.Get(-1) + + var t1 = STuple.Create(1); + Assert.That(t1.First(), Is.EqualTo(1)); + Assert.That(t1.First(), Is.EqualTo("1")); + Assert.That(((ITuple)t1).Last(), Is.EqualTo(1)); + Assert.That(((ITuple)t1).Last(), Is.EqualTo("1")); + + var t2 = STuple.Create(1, 2); + Assert.That(t2.First(), Is.EqualTo(1)); + Assert.That(t2.First(), Is.EqualTo("1")); + Assert.That(t2.Last, Is.EqualTo(2)); + Assert.That(((ITuple)t2).Last(), Is.EqualTo(2)); + Assert.That(((ITuple)t2).Last(), Is.EqualTo("2")); + + var t3 = STuple.Create(1, 2, 3); + Assert.That(t3.First(), Is.EqualTo(1)); + Assert.That(t3.First(), Is.EqualTo("1")); + Assert.That(t3.Last, Is.EqualTo(3)); + Assert.That(((ITuple)t3).Last(), Is.EqualTo(3)); + Assert.That(((ITuple)t3).Last(), Is.EqualTo("3")); + + var t4 = STuple.Create(1, 2, 3, 4); + Assert.That(t4.First(), Is.EqualTo(1)); + Assert.That(t4.First(), Is.EqualTo("1")); + Assert.That(t4.Last, Is.EqualTo(4)); + Assert.That(((ITuple)t4).Last(), Is.EqualTo(4)); + Assert.That(((ITuple)t4).Last(), Is.EqualTo("4")); + + var t5 = STuple.Create(1, 2, 3, 4, 5); + Assert.That(t5.First(), Is.EqualTo(1)); + Assert.That(t5.First(), Is.EqualTo("1")); + Assert.That(t5.Last, Is.EqualTo(5)); + Assert.That(((ITuple)t5).Last(), Is.EqualTo(5)); + Assert.That(((ITuple)t5).Last(), Is.EqualTo("5")); + + var tn = STuple.Create(1, 2, 3, 4, 5, 6); + Assert.That(tn.First(), Is.EqualTo(1)); + Assert.That(tn.First(), Is.EqualTo("1")); + Assert.That(tn.Last(), Is.EqualTo(6)); + Assert.That(tn.Last(), Is.EqualTo("6")); + + Assert.That(() => ((ITuple) STuple.Empty).First(), Throws.InstanceOf()); + Assert.That(() => ((ITuple) STuple.Empty).Last(), Throws.InstanceOf()); + } + + [Test] + public void Test_Tuple_CreateBoxed() + { + ITuple tuple; + + tuple = STuple.CreateBoxed(default(object)); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.Null); + + tuple = STuple.CreateBoxed(1); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.EqualTo(1)); + + tuple = STuple.CreateBoxed(1L); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.EqualTo(1L)); + + tuple = STuple.CreateBoxed(false); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.False); + + tuple = STuple.CreateBoxed("hello"); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.EqualTo("hello")); + + tuple = STuple.CreateBoxed(new byte[] { 1, 2, 3 }); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple[0], Is.EqualTo(new byte[] { 1, 2, 3 }.AsSlice())); + } + + [Test] + public void Test_Tuple_Embedded_Tuples() + { + // (A,B).Append((C,D)) should return (A,B,(C,D)) (length 3) and not (A,B,C,D) (length 4) + + STuple x = STuple.Create("A", "B"); + STuple y = STuple.Create("C", "D"); + + // using the instance method that returns a STuple + ITuple z = x.Append(y); + Log(z); + Assert.That(z, Is.Not.Null); + Assert.That(z.Count, Is.EqualTo(3)); + Assert.That(z[0], Is.EqualTo("A")); + Assert.That(z[1], Is.EqualTo("B")); + Assert.That(z[2], Is.EqualTo(y)); + var t = z.Get(2); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(2)); + Assert.That(t[0], Is.EqualTo("C")); + Assert.That(t[1], Is.EqualTo("D")); + + // casted down to the interface ITuple + z = ((ITuple)x).Append((ITuple)y); + Log(z); + Assert.That(z, Is.Not.Null); + Assert.That(z.Count, Is.EqualTo(3)); + Assert.That(z[0], Is.EqualTo("A")); + Assert.That(z[1], Is.EqualTo("B")); + Assert.That(z[2], Is.EqualTo(y)); + t = z.Get(2); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(2)); + Assert.That(t[0], Is.EqualTo("C")); + Assert.That(t[1], Is.EqualTo("D")); + + // composite index key "(prefix, value, id)" + ITuple subspace = STuple.Create(123, 42); + ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) + const string ID = "Doc123"; + z = subspace.Append(value, ID); + Log(z); + Assert.That(z.Count, Is.EqualTo(4)); + } + + [Test] + public void Test_Tuple_With() + { + //note: important to always cast to (ITuple) to be sure that we don't call specialized instance methods (tested elsewhere) + ITuple t; + bool called; + + // Size 1 + + t = STuple.Create(123); + called = false; + t.With((int a) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a) => + { + Assert.That(a, Is.EqualTo(123)); + return 42; + }), Is.EqualTo(42)); + Assert.That(() => t.With((int a) => throw new InvalidOperationException("BOOM")), Throws.InvalidOperationException.With.Message.EqualTo("BOOM")); + + // Size 2 + + t = t.Append("abc"); + called = false; + t.With((int a, string b) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + return 42; + }), Is.EqualTo(42)); + + // Size 3 + + t = t.Append(3.14f); + called = false; + t.With((int a, string b, float c) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + return 42; + }), Is.EqualTo(42)); + + // Size 4 + + t = t.Append(true); + called = false; + t.With((int a, string b, float c, bool d) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c, bool d) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + return 42; + }), Is.EqualTo(42)); + + // Size 5 + + t = t.Append('z'); + called = false; + t.With((int a, string b, float c, bool d, char e) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c, bool d, char e) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + return 42; + }), Is.EqualTo(42)); + + // Size 6 + + t = t.Append(Math.PI); + called = false; + t.With((int a, string b, float c, bool d, char e, double f) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c, bool d, char e, double f) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + return 42; + }), Is.EqualTo(42)); + + // Size 7 + + t = t.Append(IPAddress.Loopback); + called = false; + t.With((int a, string b, float c, bool d, char e, double f, IPAddress g) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + Assert.That(g, Is.EqualTo(IPAddress.Loopback)); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c, bool d, char e, double f, IPAddress g) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + Assert.That(g, Is.EqualTo(IPAddress.Loopback)); + return 42; + }), Is.EqualTo(42)); + + // Size 8 + + t = t.Append(DateTime.MaxValue); + called = false; + t.With((int a, string b, float c, bool d, char e, double f, IPAddress g, DateTime h) => + { + called = true; + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + Assert.That(g, Is.EqualTo(IPAddress.Loopback)); + Assert.That(h, Is.EqualTo(DateTime.MaxValue)); + }); + Assert.That(called, Is.True); + Assert.That(t.With((int a, string b, float c, bool d, char e, double f, IPAddress g, DateTime h) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + Assert.That(f, Is.EqualTo(Math.PI)); + Assert.That(g, Is.EqualTo(IPAddress.Loopback)); + Assert.That(h, Is.EqualTo(DateTime.MaxValue)); + return 42; + }), Is.EqualTo(42)); + } + + [Test] + public void Test_Tuple_With_Struct() + { + // calling With() on the structs is faster + + STuple t1 = STuple.Create(123); + t1.With((a) => + { + Assert.That(a, Is.EqualTo(123)); + }); + Assert.That(t1.With((a) => + { + Assert.That(a, Is.EqualTo(123)); + return 42; + }), Is.EqualTo(42)); + + STuple t2 = STuple.Create(123, "abc"); + t2.With((a, b) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + }); + Assert.That(t2.With((a, b) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + return 42; + }), Is.EqualTo(42)); + + STuple t3 = STuple.Create(123, "abc", 3.14f); + t3.With((a, b, c) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + }); + Assert.That(t3.With((a, b, c) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + return 42; + }), Is.EqualTo(42)); + + STuple t4 = STuple.Create(123, "abc", 3.14f, true); + t4.With((a, b, c, d) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + }); + Assert.That(t4.With((a, b, c, d) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + return 42; + }), Is.EqualTo(42)); + + STuple t5 = STuple.Create(123, "abc", 3.14f, true, 'z'); + t5.With((a, b, c, d, e) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + }); + Assert.That(t5.With((a, b, c, d, e) => + { + Assert.That(a, Is.EqualTo(123)); + Assert.That(b, Is.EqualTo("abc")); + Assert.That(c, Is.EqualTo(3.14f)); + Assert.That(d, Is.True); + Assert.That(e, Is.EqualTo('z')); + return 42; + }), Is.EqualTo(42)); + + //TODO: add more if we ever add struct tuples with 6 or more items + } + + [Test] + public void Test_Tuple_Of_Size() + { + // OfSize(n) check the size and return the tuple if it passed + // VerifySize(n) only check the size + // Both should throw if tuple is null, or not the expected size + + void Verify(ITuple t) + { + for (int i = 0; i <= 10; i++) + { + if (t.Count > i) + { + Assert.That(() => t.OfSize(i), Throws.InstanceOf()); + Assert.That(t.OfSizeAtLeast(i), Is.SameAs(t)); + Assert.That(() => t.OfSizeAtMost(i), Throws.InstanceOf()); + } + else if (t.Count < i) + { + Assert.That(() => t.OfSize(i), Throws.InstanceOf()); + Assert.That(() => t.OfSizeAtLeast(i), Throws.InstanceOf()); + Assert.That(t.OfSizeAtMost(i), Is.SameAs(t)); + } + else + { + Assert.That(t.OfSize(i), Is.SameAs(t)); + Assert.That(t.OfSizeAtLeast(i), Is.SameAs(t)); + Assert.That(t.OfSizeAtMost(i), Is.SameAs(t)); + } + } + } + + Verify(STuple.Empty); + Verify(STuple.Create(123)); + Verify(STuple.Create(123, "abc")); + Verify(STuple.Create(123, "abc", 3.14f)); + Verify(STuple.Create(123, "abc", 3.14f, true)); + Verify(STuple.Create(123, "abc", 3.14f, true, 'z')); + Verify(STuple.FromArray(new[] { "hello", "world", "!" })); + Verify(STuple.FromEnumerable(Enumerable.Range(0, 10))); + + Verify(STuple.Create(123, "abc", 3.14f, true, 'z')[0, 2]); + Verify(STuple.Create(123, "abc", 3.14f, true, 'z')[1, 4]); + Verify(STuple.FromEnumerable(Enumerable.Range(0, 50)).Substring(15, 6)); + + // ReSharper disable ExpressionIsAlwaysNull + ITuple none = null; + Assert.That(() => none.OfSize(0), Throws.InstanceOf()); + Assert.That(() => none.OfSizeAtLeast(0), Throws.InstanceOf()); + Assert.That(() => none.OfSizeAtMost(0), Throws.InstanceOf()); + // ReSharper restore ExpressionIsAlwaysNull + } + + [Test] + public void Test_Tuple_Truncate() + { + ITuple t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + + var head = t.Truncate(1); + Assert.That(head, Is.Not.Null); + Assert.That(head.Count, Is.EqualTo(1)); + Assert.That(head[0], Is.EqualTo("Hello")); + + head = t.Truncate(2); + Assert.That(head, Is.Not.Null); + Assert.That(head.Count, Is.EqualTo(2)); + Assert.That(head[0], Is.EqualTo("Hello")); + Assert.That(head[1], Is.EqualTo(123)); + + head = t.Truncate(5); + Assert.That(head, Is.EqualTo(t)); + + var tail = t.Truncate(-1); + Assert.That(tail, Is.Not.Null); + Assert.That(tail.Count, Is.EqualTo(1)); + Assert.That(tail[0], Is.EqualTo("World")); + + tail = t.Truncate(-2); + Assert.That(tail, Is.Not.Null); + Assert.That(tail.Count, Is.EqualTo(2)); + Assert.That(tail[0], Is.EqualTo(TimeSpan.FromSeconds(5))); + Assert.That(tail[1], Is.EqualTo("World")); + + tail = t.Truncate(-5); + Assert.That(tail, Is.EqualTo(t)); + + Assert.That(t.Truncate(0), Is.EqualTo(STuple.Empty)); + Assert.That(() => t.Truncate(6), Throws.InstanceOf()); + Assert.That(() => t.Truncate(-6), Throws.InstanceOf()); + + Assert.That(() => STuple.Empty.Truncate(1), Throws.InstanceOf()); + Assert.That(() => STuple.Create("Hello", "World").Truncate(3), Throws.InstanceOf()); + Assert.That(() => STuple.Create("Hello", "World").Truncate(-3), Throws.InstanceOf()); + } + + [Test] + public void Test_Tuple_As() + { + // ITuple.As<...>() adds types to an untyped ITuple + ITuple t; + + t = STuple.Create("Hello"); + var t1 = t.As(); + Assert.That(t1.Item1, Is.EqualTo("Hello")); + + t = STuple.Create("Hello", 123); + var t2 = t.As(); + Assert.That(t2.Item1, Is.EqualTo("Hello")); + Assert.That(t2.Item2, Is.EqualTo(123)); + + t = STuple.Create("Hello", 123, false); + var t3 = t.As(); + Assert.That(t3.Item1, Is.EqualTo("Hello")); + Assert.That(t3.Item2, Is.EqualTo(123)); + Assert.That(t3.Item3, Is.False); + + var t4 = STuple + .Create("Hello", 123, false, TimeSpan.FromSeconds(5)) + .As(); + Assert.That(t4.Item1, Is.EqualTo("Hello")); + Assert.That(t4.Item2, Is.EqualTo(123)); + Assert.That(t4.Item3, Is.False); + Assert.That(t4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + + t = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + var t5 = t.As(); + Assert.That(t5.Item1, Is.EqualTo("Hello")); + Assert.That(t5.Item2, Is.EqualTo(123)); + Assert.That(t5.Item3, Is.False); + Assert.That(t5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + Assert.That(t5.Item5, Is.EqualTo("World")); + } + + [Test] + public void Test_Cast_To_BCL_Tuples() + { + // implicit: Tuple => ITuple + // explicit: ITuple => Tuple + + var t1 = STuple.Create("Hello"); + var b1 = (Tuple) t1; // explicit + Assert.That(b1, Is.Not.Null); + Assert.That(b1.Item1, Is.EqualTo("Hello")); + STuple r1 = t1; // implicit + Assert.That(r1.Item1, Is.EqualTo("Hello")); + + var t2 = STuple.Create("Hello", 123); + var b2 = (Tuple)t2; // explicit + Assert.That(b2, Is.Not.Null); + Assert.That(b2.Item1, Is.EqualTo("Hello")); + Assert.That(b2.Item2, Is.EqualTo(123)); + STuple r2 = t2; // implicit + Assert.That(r2.Item1, Is.EqualTo("Hello")); + Assert.That(r2.Item2, Is.EqualTo(123)); + + var t3 = STuple.Create("Hello", 123, false); + var b3 = (Tuple)t3; // explicit + Assert.That(b3, Is.Not.Null); + Assert.That(b3.Item1, Is.EqualTo("Hello")); + Assert.That(b3.Item2, Is.EqualTo(123)); + Assert.That(b3.Item3, Is.False); + STuple r3 = t3; // implicit + Assert.That(r3.Item1, Is.EqualTo("Hello")); + Assert.That(r3.Item2, Is.EqualTo(123)); + Assert.That(r3.Item3, Is.False); + + var t4 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5)); + var b4 = (Tuple)t4; // explicit + Assert.That(b4, Is.Not.Null); + Assert.That(b4.Item1, Is.EqualTo("Hello")); + Assert.That(b4.Item2, Is.EqualTo(123)); + Assert.That(b4.Item3, Is.False); + Assert.That(b4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + STuple r4 = t4; // implicit + Assert.That(r4.Item1, Is.EqualTo("Hello")); + Assert.That(r4.Item2, Is.EqualTo(123)); + Assert.That(r4.Item3, Is.False); + Assert.That(r4.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + + var t5 = STuple.Create("Hello", 123, false, TimeSpan.FromSeconds(5), "World"); + var b5 = (Tuple)t5; // explicit + Assert.That(b5, Is.Not.Null); + Assert.That(b5.Item1, Is.EqualTo("Hello")); + Assert.That(b5.Item2, Is.EqualTo(123)); + Assert.That(b5.Item3, Is.False); + Assert.That(b5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + Assert.That(b5.Item5, Is.EqualTo("World")); + STuple r5 = t5; // implicit + Assert.That(r5.Item1, Is.EqualTo("Hello")); + Assert.That(r5.Item2, Is.EqualTo(123)); + Assert.That(r5.Item3, Is.False); + Assert.That(r5.Item4, Is.EqualTo(TimeSpan.FromSeconds(5))); + Assert.That(r5.Item5, Is.EqualTo("World")); + + } + + [Test] + public void Test_Tuple_Stringify() + { + // typed tuples + Assert.That(STuple.Empty.ToString(), Is.EqualTo("()")); + Assert.That(STuple.Create("hello world").ToString(), Is.EqualTo(@"(""hello world"",)")); + Assert.That(STuple.Create(true).ToString(), Is.EqualTo("(true,)")); + Assert.That(STuple.Create(123).ToString(), Is.EqualTo("(123,)")); + Assert.That(STuple.Create(123U).ToString(), Is.EqualTo("(123,)")); + Assert.That(STuple.Create(123L).ToString(), Is.EqualTo("(123,)")); + Assert.That(STuple.Create(123UL).ToString(), Is.EqualTo("(123,)")); + Assert.That(STuple.Create(123.4d).ToString(), Is.EqualTo("(123.4,)")); + Assert.That(STuple.Create(123.4f).ToString(), Is.EqualTo("(123.4,)")); + Assert.That(STuple.Create(Guid.Parse("102cb0aa-2151-4c72-9e9d-61cf2980cbd0")).ToString(), Is.EqualTo("({102cb0aa-2151-4c72-9e9d-61cf2980cbd0},)")); + Assert.That(STuple.Create(Uuid128.Parse("102cb0aa-2151-4c72-9e9d-61cf2980cbd0")).ToString(), Is.EqualTo("({102cb0aa-2151-4c72-9e9d-61cf2980cbd0},)")); + Assert.That(STuple.Create(Uuid64.Parse("102cb0aa-21514c72")).ToString(), Is.EqualTo("({102CB0AA-21514C72},)")); + Assert.That(STuple.Create(new byte[] { 0x02, 0x41, 0x42, 0x43, 0x00 }).ToString(), Is.EqualTo("(`<02>ABC<00>`,)")); + Assert.That(STuple.Create(new byte[] { 0x02, 0x41, 0x42, 0x43, 0x00 }.AsSlice()).ToString(), Is.EqualTo("(`<02>ABC<00>`,)")); + + Assert.That(STuple.Create("Hello", 123, "World", '!', false).ToString(), Is.EqualTo(@"(""Hello"", 123, ""World"", '!', false)")); + } + + #endregion + + #region Splicing... + + private static void VerifyTuple(string message, ITuple t, object[] expected) + { + // count + if (t.Count != expected.Length) + { +#if DEBUG + if (System.Diagnostics.Debugger.IsAttached) System.Diagnostics.Debugger.Break(); +#endif + Assert.Fail("{0}: Count mismatch between observed {1} and expected {2} for tuple of type {3}", message, t, STuple.Formatter.ToString(expected), t.GetType().Name); + } + + // direct access + for (int i = 0; i < expected.Length; i++) + { + Assert.That(ComparisonHelper.AreSimilar(t[i], expected[i]), Is.True, "{0}: t[{1}] != expected[{1}]", message, i); + } + + // iterator + int p = 0; + foreach (var obj in t) + { + if (p >= expected.Length) Assert.Fail("Spliced iterator overshoot at t[{0}] = {1}", p, obj); + Assert.That(ComparisonHelper.AreSimilar(obj, expected[p]), Is.True, "{0}: Iterator[{1}], {2} ~= {3}", message, p, obj, expected[p]); + ++p; + } + Assert.That(p, Is.EqualTo(expected.Length), "{0}: t.GetEnumerator() returned only {1} elements out of {2} exected", message, p, expected.Length); + + // CopyTo + var tmp = new object[expected.Length]; + t.CopyTo(tmp, 0); + for (int i = 0; i < tmp.Length; i++) + { + Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: CopyTo[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); + } + + // Memoize + //tmp = t.Memoize().ToArray(); + //for (int i = 0; i < tmp.Length; i++) + //{ + // Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: Memoize.Items[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); + //} + + // Append + //if (!(t is SlicedTuple)) + { + var u = t.Append("last"); + Assert.That(u.Get(-1), Is.EqualTo("last")); + tmp = u.ToArray(); + for (int i = 0; i < tmp.Length - 1; i++) + { + Assert.That(ComparisonHelper.AreSimilar(tmp[i], expected[i]), Is.True, "{0}: Appended[{1}], {2} ~= {3}", message, i, tmp[i], expected[i]); + } + } + } + + [Test] + public void Test_Can_Splice_ListTuple() + { + var items = new object[] { "hello", "world", 123, "foo", 456, "bar" }; + // 0 1 2 3 4 5 + // -6 -5 -4 -3 -2 -1 + + var tuple = new ListTuple(items); + Assert.That(tuple.Count, Is.EqualTo(6)); + + // get all + VerifyTuple("[:]", tuple[null, null], items); + VerifyTuple("[:]", tuple[null, 6], items); + VerifyTuple("[:]", tuple[0, null], items); + VerifyTuple("[:]", tuple[0, 6], items); + VerifyTuple("[:]", tuple[0, null], items); + VerifyTuple("[:]", tuple[-6, null], items); + VerifyTuple("[:]", tuple[-6, 6], items); + + // tail + VerifyTuple("[n:]", tuple[4, null], new object[] { 456, "bar" }); + VerifyTuple("[n:+]", tuple[4, 6], new object[] { 456, "bar" }); + VerifyTuple("[-n:+]", tuple[-2, 6], new object[] { 456, "bar" }); + VerifyTuple("[-n:-]", tuple[-2, null], new object[] { 456, "bar" }); + + // head + VerifyTuple("[:n]", tuple[null, 3], new object[] { "hello", "world", 123 }); + VerifyTuple("[0:n]", tuple[0, 3], new object[] { "hello", "world", 123 }); + VerifyTuple("[0:-n]", tuple[0, -3], new object[] { "hello", "world", 123 }); + VerifyTuple("[-:n]", tuple[-6, 3], new object[] { "hello", "world", 123 }); + VerifyTuple("[-:-n]", tuple[-6, -3], new object[] { "hello", "world", 123 }); + + // single + VerifyTuple("[0:1]", tuple[0, 1], new object[] { "hello" }); + VerifyTuple("[-6:-5]", tuple[-6, -5], new object[] { "hello" }); + VerifyTuple("[1:2]", tuple[1, 2], new object[] { "world" }); + VerifyTuple("[-5:-4]", tuple[-5, -4], new object[] { "world" }); + VerifyTuple("[5:6]", tuple[5, 6], new object[] { "bar" }); + VerifyTuple("[-1:]", tuple[-1, null], new object[] { "bar" }); + + // chunk + VerifyTuple("[2:4]", tuple[2, 4], new object[] { 123, "foo" }); + VerifyTuple("[2:-2]", tuple[2, -2], new object[] { 123, "foo" }); + VerifyTuple("[-4:4]", tuple[-4, 4], new object[] { 123, "foo" }); + VerifyTuple("[-4:-2]", tuple[-4, -2], new object[] { 123, "foo" }); + + // remove first + VerifyTuple("[1:]", tuple[1, null], new object[] { "world", 123, "foo", 456, "bar" }); + VerifyTuple("[1:+]", tuple[1, 6], new object[] { "world", 123, "foo", 456, "bar" }); + VerifyTuple("[-5:]", tuple[-5, null], new object[] { "world", 123, "foo", 456, "bar" }); + VerifyTuple("[-5:+]", tuple[-5, 6], new object[] { "world", 123, "foo", 456, "bar" }); + + // remove last + VerifyTuple("[:5]", tuple[null, 5], new object[] { "hello", "world", 123, "foo", 456 }); + VerifyTuple("[:-1]", tuple[null, -1], new object[] { "hello", "world", 123, "foo", 456 }); + VerifyTuple("[0:5]", tuple[0, 5], new object[] { "hello", "world", 123, "foo", 456 }); + VerifyTuple("[0:-1]", tuple[0, -1], new object[] { "hello", "world", 123, "foo", 456 }); + + // out of range + VerifyTuple("[2:7]", tuple[2, 7], new object[] { 123, "foo", 456, "bar" }); + VerifyTuple("[2:42]", tuple[2, 42], new object[] { 123, "foo", 456, "bar" }); + VerifyTuple("[2:123456]", tuple[2, 123456], new object[] { 123, "foo", 456, "bar" }); + VerifyTuple("[-7:2]", tuple[-7, 2], new object[] { "hello", "world" }); + VerifyTuple("[-42:2]", tuple[-42, 2], new object[] { "hello", "world" }); + } + + private static object[] GetRange(int fromIncluded, int toExcluded, int count) + { + if (count == 0) return new object[0]; + + if (fromIncluded < 0) fromIncluded += count; + if (toExcluded < 0) toExcluded += count; + + if (toExcluded > count) toExcluded = count; + var tmp = new object[toExcluded - fromIncluded]; + for (int i = 0; i < tmp.Length; i++) tmp[i] = new string((char) (65 + fromIncluded + i), 1); + return tmp; + } + + [Test] + public void Test_Randomized_Splices() + { + // Test a random mix of sizes, and indexes... + + const int N = 100 * 1000; + + var tuples = new ITuple[14]; + tuples[0] = STuple.Empty; + tuples[1] = STuple.Create("A"); + tuples[2] = STuple.Create("A", "B"); + tuples[3] = STuple.Create("A", "B", "C"); + tuples[4] = STuple.Create("A", "B", "C", "D"); + tuples[5] = STuple.Create("A", "B", "C", "D", "E"); + tuples[6] = STuple.Create("A", "B", "C", "D", "E", "F"); + tuples[7] = STuple.Create("A", "B", "C", "D", "E", "F", "G"); + tuples[8] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H"); + tuples[9] = STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I"); + tuples[10]= STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J"); + tuples[11] = new JoinedTuple(tuples[6], STuple.Create("G", "H", "I", "J", "K")); + tuples[12] = new LinkedTuple(tuples[11], "L"); + tuples[13] = new LinkedTuple(STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L"), "M"); + +#if false + Console.Write("Checking tuples"); + + foreach (var tuple in tuples) + { + var t = STuple.Unpack(tuple.ToSlice()); + Assert.That(t.Equals(tuple), Is.True, t.ToString() + " != unpack(" + tuple.ToString() + ")"); + } +#endif + + var rnd = new Random(123456); + + for (int i = 0; i < N; i++) + { + if (i % 500 == 0) Console.Write("."); + var len = rnd.Next(tuples.Length); + var tuple = tuples[len]; + if (tuple.Count != len) + { + Assert.That(tuple.Count, Is.EqualTo(len), "Invalid length for tuple {0}", tuple); + } + + string prefix = tuple.ToString(); + + //if (rnd.Next(5) == 0) + //{ // randomly pack/unpack + // tuple = STuple.Unpack(tuple.ToSlice()); + // prefix = "unpacked:" + prefix; + //} + //else if (rnd.Next(5) == 0) + //{ // randomly memoize + // tuple = tuple.Memoize(); + // prefix = "memoized:" + prefix; + //} + + switch (rnd.Next(6)) + { + case 0: + { // [:+rnd] + int x = rnd.Next(len); + VerifyTuple(prefix + "[:" + x.ToString() + "]", tuple[null, x], GetRange(0, x, len)); + break; + } + case 1: + { // [+rnd:] + int x = rnd.Next(len); + VerifyTuple(prefix + "[" + x.ToString() + ":]", tuple[x, null], GetRange(x, int.MaxValue, len)); + break; + } + case 2: + { // [:-rnd] + int x = -1 - rnd.Next(len); + VerifyTuple(prefix + "[:" + x.ToString() + "]", tuple[null, x], GetRange(0, len + x, len)); + break; + } + case 3: + { // [-rnd:] + int x = -1 - rnd.Next(len); + VerifyTuple(prefix + "[" + x.ToString() + ":]", tuple[x, null], GetRange(len + x, int.MaxValue, len)); + break; + } + case 4: + { // [rnd:rnd] + int x = rnd.Next(len); + int y; + do { y = rnd.Next(len); } while (y < x); + VerifyTuple(prefix + " [" + x.ToString() + ":" + y.ToString() + "]", tuple[x, y], GetRange(x, y, len)); + break; + } + case 5: + { // [-rnd:-rnd] + int x = -1 - rnd.Next(len); + int y; + do { y = -1 - rnd.Next(len); } while (y < x); + VerifyTuple(prefix + " [" + x.ToString() + ":" + y.ToString() + "]", tuple[x, y], GetRange(len + x, len + y, len)); + break; + } + } + + } + Log(" done"); + + } + + #endregion + + #region Equality / Comparison + + private static void AssertEquality(ITuple x, ITuple y) + { + Assert.That(x.Equals(y), Is.True, "x.Equals(y)"); + Assert.That(x.Equals((object)y), Is.True, "x.Equals((object)y)"); + Assert.That(y.Equals(x), Is.True, "y.Equals(x)"); + Assert.That(y.Equals((object)x), Is.True, "y.Equals((object)y"); + } + + private static void AssertInequality(ITuple x, ITuple y) + { + Assert.That(x.Equals(y), Is.False, "!x.Equals(y)"); + Assert.That(x.Equals((object)y), Is.False, "!x.Equals((object)y)"); + Assert.That(y.Equals(x), Is.False, "!y.Equals(x)"); + Assert.That(y.Equals((object)x), Is.False, "!y.Equals((object)y"); + } + + [Test] + public void Test_Tuple_Equals() + { + var t1 = STuple.Create(1, 2); + // self equality + AssertEquality(t1, t1); + + var t2 = STuple.Create(1, 2); + // same type equality + AssertEquality(t1, t2); + + var t3 = STuple.Create(new object[] { 1, 2 }); + // other tuple type equality + AssertEquality(t1, t3); + + var t4 = STuple.Create(1).Append(2); + // multi step + AssertEquality(t1, t4); + } + + [Test] + public void Test_Tuple_Similar() + { + var t1 = STuple.Create(1, 2); + var t2 = STuple.Create((long)1, (short)2); + var t3 = STuple.Create("1", "2"); + var t4 = STuple.Create(new object[] { 1, 2L }); + //var t5 = STuple.Unpack(Slice.Unescape("<02>1<00><15><02>")); + + AssertEquality(t1, t1); + AssertEquality(t1, t2); + AssertEquality(t1, t3); + AssertEquality(t1, t4); + //AssertEquality(t1, t5); + AssertEquality(t2, t2); + AssertEquality(t2, t3); + AssertEquality(t2, t4); + //AssertEquality(t2, t5); + AssertEquality(t3, t3); + AssertEquality(t3, t4); + //AssertEquality(t3, t5); + AssertEquality(t4, t4); + //AssertEquality(t4, t5); + //AssertEquality(t5, t5); + } + + [Test] + public void Test_Tuple_Not_Equal() + { + var t1 = STuple.Create(1, 2); + + var x1 = STuple.Create(2, 1); + var x2 = STuple.Create("11", "22"); + var x3 = STuple.Create(1, 2, 3); + //var x4 = STuple.Unpack(Slice.Unescape("<15><01>")); + + AssertInequality(t1, x1); + AssertInequality(t1, x2); + AssertInequality(t1, x3); + //AssertInequality(t1, x4); + + AssertInequality(x1, x2); + AssertInequality(x1, x3); + //AssertInequality(x1, x4); + AssertInequality(x2, x3); + //AssertInequality(x2, x4); + //AssertInequality(x3, x4); + } + + [Test] + public void Test_Tuple_Substring_Equality() + { + ITuple x = STuple.FromArray(new[] {"A", "C"}); + ITuple y = STuple.FromArray(new[] {"A", "B", "C"}); + + Assert.That(x.Substring(0, 1), Is.EqualTo(y.Substring(0, 1))); + Assert.That(x.Substring(1, 1), Is.EqualTo(y.Substring(2, 1))); + + ITuple a = x.Substring(0, 1); + ITuple b = y.Substring(0, 1); + Assert.That(a.Equals(b), Is.True); + Assert.That(a.Equals((object)b), Is.True); + Assert.That(object.Equals(a, b), Is.True); + Assert.That(STuple.Equals(a, b), Is.True); + Assert.That(STuple.Equivalent(a, b), Is.True); + + // this is very unfortunate, but 'a == b' does NOT work because ITuple is an interface, and there is no known way to make it work :( + // ReSharper disable PossibleUnintendedReferenceComparison + // ReSharper disable CannotApplyEqualityOperatorToType + Assert.That(a == b, Is.False, "Tuples A and B, even if they contain the same values, are pointers to two different instances on the heap, and should not ReferenceEquals !"); + // ReSharper restore CannotApplyEqualityOperatorToType + // ReSharper restore PossibleUnintendedReferenceComparison + + // It should work on STuple<..> though (but with a compiler warning) + STuple aa = STuple.Create("A"); + STuple bb = STuple.Create("A"); + // ReSharper disable CannotApplyEqualityOperatorToType + Assert.That(aa == bb, Is.True, "Operator '==' should work on struct tuples."); + // ReSharper restore CannotApplyEqualityOperatorToType + Assert.That(aa.Equals(bb), Is.True, "Equals(..) should work on struct tuples."); + STuple cc = STuple.Create(new string('A', 1)); // make sure we have an "A" string that is not the same pointers as the others + Assert.That(aa.Item1, Is.Not.SameAs(cc.Item1), "Did your compiler optimize the new string('A', 1). If so, need to find another way"); + Assert.That(aa.Equals(cc), Is.True, "Equals(..) should compare the values, not the pointers."); + + + } + + [Test] + public void Test_Tuple_String_AutoCast() + { + // 'a' ~= "A" + AssertEquality(STuple.Create("A"), STuple.Create('A')); + AssertInequality(STuple.Create("A"), STuple.Create('B')); + AssertInequality(STuple.Create("A"), STuple.Create('a')); + + // ASCII ~= Unicode + AssertEquality(STuple.Create("ABC"), STuple.Create(Slice.FromStringAscii("ABC"))); + AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromStringAscii("DEF"))); + AssertInequality(STuple.Create("ABC"), STuple.Create(Slice.FromStringAscii("abc"))); + + // 'a' ~= ASCII 'a' + AssertEquality(STuple.Create(Slice.FromStringAscii("A")), STuple.Create('A')); + AssertInequality(STuple.Create(Slice.FromStringAscii("A")), STuple.Create('B')); + AssertInequality(STuple.Create(Slice.FromStringAscii("A")), STuple.Create('a')); + } + + [Test] + public void Test_Tuple_Comparers() + { + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create(123), STuple.Create(123)), Is.True, "(123,) == (123,)"); + Assert.That(cmp.Equals(STuple.Create(123), STuple.Create(456)), Is.False, "(123,) != (456,)"); + Assert.That(cmp.GetHashCode(STuple.Create(123)), Is.EqualTo(STuple.Create(123).GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create(123)), Is.Not.EqualTo(STuple.Create(456).GetHashCode())); + } + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo"), STuple.Create("foo")), Is.True, "('foo',) == ('foo',)"); + Assert.That(cmp.Equals(STuple.Create("foo"), STuple.Create("bar")), Is.False, "('foo',) != ('bar',)"); + Assert.That(cmp.GetHashCode(STuple.Create("foo")), Is.EqualTo(STuple.Create("foo").GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo")), Is.Not.EqualTo(STuple.Create("bar").GetHashCode())); + } + + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo", 123), STuple.Create("foo", 123)), Is.True, "('foo',123) == ('foo',123)"); + Assert.That(cmp.Equals(STuple.Create("foo", 123), STuple.Create("bar", 123)), Is.False, "('foo',123) != ('bar',123)"); + Assert.That(cmp.Equals(STuple.Create("foo", 123), STuple.Create("foo", 456)), Is.False, "('foo',123) != ('foo',456)"); + Assert.That(cmp.GetHashCode(STuple.Create("foo", 123)), Is.EqualTo(STuple.Create("foo", 123).GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo", 123)), Is.Not.EqualTo(STuple.Create("foo", 456).GetHashCode())); + } + + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo", true, 123), STuple.Create("foo", true, 123)), Is.True, "('foo',true,123) == ('foo',true,123)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123), STuple.Create("bar", true, 123)), Is.False, "('foo',true,123) != ('bar',true,123)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123), STuple.Create("foo", false, 123)), Is.False, "('foo',true,123) != ('foo',false,123)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123), STuple.Create("foo", true, 456)), Is.False, "('foo',true,123) != ('foo',true,456)"); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123)), Is.EqualTo(STuple.Create("foo", true, 123).GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123)), Is.Not.EqualTo(STuple.Create("foo", true, 456).GetHashCode())); + } + + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L), STuple.Create("foo", true, 123, -1L)), Is.True, "('foo',true,123,-1) == ('foo',true,123,-1)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L), STuple.Create("bar", true, 123, -1L)), Is.False, "('foo',true,123,-1) != ('bar',true,123,-1)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L), STuple.Create("foo", false, 123, -1L)), Is.False, "('foo',true,123,-1) != ('foo',false,123,-1)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L), STuple.Create("foo", true, 456, -1L)), Is.False, "('foo',true,123,-1) != ('foo',true,456,-1)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L), STuple.Create("foo", true, 123, -2L)), Is.False, "('foo',true,123,-1) != ('foo',true,123,-2)"); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L)), Is.EqualTo(STuple.Create("foo", true, 123, -1L).GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L)), Is.Not.EqualTo(STuple.Create("foo", true, 456, 123L).GetHashCode())); + } + + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("foo", true, 123, -1L, "narf")), Is.True, "('foo',true,123,-1) == ('foo',true,123,-1,'narf')"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("bar", true, 123, -1L, "narf")), Is.False, "('foo',true,123,-1) != ('bar',true,123,-1,'narf')"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("foo", false, 123, -1L, "narf")), Is.False, "('foo',true,123,-1) != ('foo',false,123,-1,'narf')"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("foo", true, 456, -1L, "narf")), Is.False, "('foo',true,123,-1) != ('foo',true,456,-1,'narf')"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("foo", true, 123, -2L, "narf")), Is.False, "('foo',true,123,-1) != ('foo',true,123,-2,'narf')"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf"), STuple.Create("foo", true, 123, -1L, "zort")), Is.False, "('foo',true,123,-1) != ('foo',true,123,-1,'zort')"); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L, "narf")), Is.EqualTo(STuple.Create("foo", true, 123, -1L, "narf").GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L, "narf")), Is.Not.EqualTo(STuple.Create("foo", true, 456, -1L, "narf").GetHashCode())); + } + + { + var cmp = STuple.EqualityComparer.Default; + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", true, 123, -1L, "narf", Math.PI)), Is.True, "('foo',true,123,-1) == ('foo',true,123,-1,'narf',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("bar", true, 123, -1L, "narf", Math.PI)), Is.False, "('foo',true,123,-1) != ('bar',true,123,-1,'narf',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", false, 123, -1L, "narf", Math.PI)), Is.False, "('foo',true,123,-1) != ('foo',false,123,-1,'narf',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", true, 456, -1L, "narf", Math.PI)), Is.False, "('foo',true,123,-1) != ('foo',true,456,-1,'narf',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", true, 123, -2L, "narf", Math.PI)), Is.False, "('foo',true,123,-1) != ('foo',true,123,-2,'narf',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", true, 123, -1L, "zort", Math.PI)), Is.False, "('foo',true,123,-1) != ('foo',true,123,-1,'zort',PI)"); + Assert.That(cmp.Equals(STuple.Create("foo", true, 123, -1L, "narf", Math.PI), STuple.Create("foo", true, 123, -1L, "narf", Math.E)), Is.False, "('foo',true,123,-1) != ('foo',true,123,-1,'narf',E)"); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L, "narf", Math.PI)), Is.EqualTo(STuple.Create("foo", true, 123, -1L, "narf", Math.PI).GetHashCode())); + Assert.That(cmp.GetHashCode(STuple.Create("foo", true, 123, -1L, "narf", Math.PI)), Is.Not.EqualTo(STuple.Create("foo", true, 123, -1L, "narf", Math.E).GetHashCode())); + } + + } + + #endregion + + #region Formatters + + [Test] + public void Test_Default_TupleFormatter_For_Common_Types() + { + + // common simple types + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + + // corner cases + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + //Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + + // ITupleFormattable types + Assert.That(TupleFormatter.Default, Is.InstanceOf>()); + } + + [Test] + public void Test_Format_Common_Types() + { + Assert.That(TupleFormatter.Default.ToTuple(123), Is.EqualTo(STuple.Create(123))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(123)), Is.EqualTo(123)); + + Assert.That(TupleFormatter.Default.ToTuple(true), Is.EqualTo(STuple.Create(true))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create(true)), Is.True); + + Assert.That(TupleFormatter.Default.ToTuple("hello"), Is.EqualTo(STuple.Create("hello"))); + Assert.That(TupleFormatter.Default.FromTuple(STuple.Create("hello")), Is.EqualTo("hello")); + + var t = STuple.Create(new object[] { "hello", 123, false }); + Assert.That(TupleFormatter.Default.ToTuple(t), Is.SameAs(t)); + Assert.That(TupleFormatter.Default.FromTuple(t), Is.SameAs(t)); + + var thing = new Thing { Foo = 123, Bar = "hello" }; + Assert.That(TupleFormatter.Default.ToTuple(thing), Is.EqualTo(STuple.Create(123, "hello"))); + + var thing2 = TupleFormatter.Default.FromTuple(STuple.Create(456, "world")); + Assert.That(thing2, Is.Not.Null); + Assert.That(thing2.Foo, Is.EqualTo(456)); + Assert.That(thing2.Bar, Is.EqualTo("world")); + + } + + [Test] + public void Test_Create_Appender_Formatter() + { + // create an appender formatter that will always add the values after the same prefix + + var fmtr = TupleFormatter.CreateAppender(STuple.Create("hello", "world")); + Assert.That(fmtr, Is.InstanceOf>()); + + Assert.That(fmtr.ToTuple(123), Is.EqualTo(STuple.Create("hello", "world", 123))); + Assert.That(fmtr.ToTuple(456), Is.EqualTo(STuple.Create("hello", "world", 456))); + Assert.That(fmtr.ToTuple(-1), Is.EqualTo(STuple.Create("hello", "world", -1))); + + Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", 42)), Is.EqualTo(42)); + Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", -1)), Is.EqualTo(-1)); + + Assert.That(() => fmtr.FromTuple(null), Throws.InstanceOf()); + Assert.That(() => fmtr.FromTuple(STuple.Empty), Throws.InstanceOf()); + Assert.That(() => fmtr.FromTuple(STuple.Create("hello", "world", 42, 77)), Throws.InstanceOf(), "Too many values"); + Assert.That(() => fmtr.FromTuple(STuple.Create("hello_world", 42)), Throws.InstanceOf(), "not enough values"); + Assert.That(() => fmtr.FromTuple(STuple.Create("world", "hello", "42")), Throws.InstanceOf(), "incorrect type"); + Assert.That(() => fmtr.FromTuple(STuple.Create(42)), Throws.InstanceOf(), "missing prefix"); + Assert.That(() => fmtr.FromTuple(STuple.Create("extra", "hello", "world", 42)), Throws.InstanceOf(), "prefix must match exactly"); + Assert.That(() => fmtr.FromTuple(STuple.Create("Hello", "World", 42)), Throws.InstanceOf(), "case sensitive"); + } + + #endregion + + #region Deformatters + + [Test] + public void Test_Can_Deformat_Simple_Tuples() + { + + void Check(string expr, ITuple expected) + { + Log("> " + expr); + var actual = STuple.Deformatter.Parse(expr); + if (!expected.Equals(actual)) + { + Log("- EXPECTED: " + expected); + Log("- ACTUAL : " + actual); + Log("- " + TuPack.Pack(actual)); + Log("- " + TuPack.Pack(expected)); + Assert.That(actual, Is.EqualTo(expected), expr); + } + } + + Check("()", STuple.Empty); + Check("(true)", STuple.Create(true)); + Check("(false)", STuple.Create(false)); + Check("(123)", STuple.Create(123)); + Check("(-42)", STuple.Create(-42)); + Check("(123.4)", STuple.Create(123.4d)); + Check("(1E10)", STuple.Create(1E10)); + Check("('x')", STuple.Create('x')); + Check("(\"Hello World\")", STuple.Create("Hello World")); + Check("(\"Foo\\\"Bar\\tBaz\")", STuple.Create("Foo\"Bar\tBaz")); + Check("({4626466c-fdac-4230-af3a-4029fab668ab})", STuple.Create(Guid.Parse("4626466c-fdac-4230-af3a-4029fab668ab"))); + + Check("(\"Hello\",123,false)", STuple.Create("Hello", 123, false)); + Check("('M',123456789,{4626466c-fdac-4230-af3a-4029fab668ab})", STuple.Create('M', 123456789, Guid.Parse("4626466c-fdac-4230-af3a-4029fab668ab"))); + Check("(123, true , \"Hello\")", STuple.Create(123, true, "Hello")); + + Check("(\"Hello\",(123,true),\"World!\")", STuple.Create("Hello", STuple.Create(123, true), "World!")); + Check("(9223372036854775807,)", STuple.Create(long.MaxValue)); + Check("(-9223372036854775808,)", STuple.Create(long.MinValue)); + Check("(18446744073709551615,)", STuple.Create(ulong.MaxValue)); + Check("(3.1415926535897931, 2.7182818284590451)", STuple.Create(Math.PI, Math.E)); + Check("(123E45,-123E-45)", STuple.Create(123E45, -123E-45)); + } + + #endregion + + #region Bench.... + +#if false + + [Test] + public void Bench_Tuple_Unpack_Random() + { + const int N = 100 * 1000; + + Slice FUNKY_ASCII = Slice.FromAscii("bonjour\x00le\x00\xFFmonde"); + string FUNKY_STRING = "hello\x00world"; + string UNICODE_STRING = "héllø 世界"; + + Console.Write("Creating {0:N0} random tuples", N); + var tuples = new List(N); + var rnd = new Random(777); + var guids = Enumerable.Range(0, 10).Select(_ => Guid.NewGuid()).ToArray(); + var uuid128s = Enumerable.Range(0, 10).Select(_ => Uuid128.NewUuid()).ToArray(); + var uuid64s = Enumerable.Range(0, 10).Select(_ => Uuid64.NewUuid()).ToArray(); + var fuzz = new byte[1024 + 1000]; rnd.NextBytes(fuzz); + var sw = Stopwatch.StartNew(); + for (int i = 0; i < N; i++) + { + ITuple tuple = STuple.Empty; + int s = 1 + (int)Math.Sqrt(rnd.Next(128)); + if (i % (N / 100) == 0) Console.Write("."); + for (int j = 0; j < s; j++) + { + switch (rnd.Next(17)) + { + case 0: tuple = tuple.Append(rnd.Next(255)); break; + case 1: tuple = tuple.Append(-1 - rnd.Next(255)); break; + case 2: tuple = tuple.Append(256 + rnd.Next(65536 - 256)); break; + case 3: tuple = tuple.Append(rnd.Next(int.MaxValue)); break; + case 4: tuple = tuple.Append((rnd.Next(int.MaxValue) << 32) | rnd.Next(int.MaxValue)); break; + case 5: tuple = tuple.Append(new string('A', 1 + rnd.Next(16))); break; + case 6: tuple = tuple.Append(new string('B', 8 + (int)Math.Sqrt(rnd.Next(1024)))); break; + case 7: tuple = tuple.Append(UNICODE_STRING); break; + case 8: tuple = tuple.Append(FUNKY_STRING); break; + case 9: tuple = tuple.Append(FUNKY_ASCII); break; + case 10: tuple = tuple.Append(guids[rnd.Next(10)]); break; + case 11: tuple = tuple.Append(uuid128s[rnd.Next(10)]); break; + case 12: tuple = tuple.Append(uuid64s[rnd.Next(10)]); break; + case 13: tuple = tuple.Append(Slice.Create(fuzz, rnd.Next(1000), 1 + (int)Math.Sqrt(rnd.Next(1024)))); break; + case 14: tuple = tuple.Append(default(string)); break; + case 15: tuple = tuple.Append("hello"); break; + case 16: tuple = tuple.Append(rnd.Next(2) == 0); break; + } + } + tuples.Add(tuple); + } + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(" > {0:N0} items", tuples.Sum(x => x.Count)); + Log(" > {0}", tuples[42]); + Log(); + + Console.Write("Packing tuples..."); + sw.Restart(); + var slices = STuple.Pack(tuples); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); + Log(" > {0:N0} bytes", slices.Sum(x => x.Count)); + Log(" > {0}", slices[42]); + Log(); + + Console.Write("Unpacking tuples..."); + sw.Restart(); + var unpacked = slices.Select(slice => STuple.Unpack(slice)).ToList(); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(" > {0:N0} tps", N / sw.Elapsed.TotalSeconds); + Log(" > {0}", unpacked[42]); + Log(); + + Console.Write("Comparing ..."); + sw.Restart(); + tuples.Zip(unpacked, (x, y) => x.Equals(y)).All(b => b); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(); + + Console.Write("Tuples.ToString ..."); + sw.Restart(); + var strings = tuples.Select(x => x.ToString()).ToList(); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(" > {0:N0} chars", strings.Sum(x => x.Length)); + Log(" > {0}", strings[42]); + Log(); + + Console.Write("Unpacked.ToString ..."); + sw.Restart(); + strings = unpacked.Select(x => x.ToString()).ToList(); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log(" > {0:N0} chars", strings.Sum(x => x.Length)); + Log(" > {0}", strings[42]); + Log(); + + Console.Write("Memoizing ..."); + sw.Restart(); + var memoized = tuples.Select(x => x.Memoize()).ToList(); + sw.Stop(); + Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); + } + +#endif + + #endregion + + private class Thing : ITupleFormattable + { + + public int Foo { get; set; } + public string Bar { get; set; } + + ITuple ITupleFormattable.ToTuple() + { + return STuple.Create(this.Foo, this.Bar); + } + + void ITupleFormattable.FromTuple(ITuple tuple) + { + this.Foo = tuple.Get(0); + this.Bar = tuple.Get(1); + } + } + + #region System.ValueTuple integration... + +#if ENABLE_VALUETUPLE + + [Test] + public void Test_Implicit_Cast_STuple_To_ValueTuple() + { + { + ValueTuple t = STuple.Create(11); + Assert.That(t.Item1, Is.EqualTo(11)); + } + { + ValueTuple t = STuple.Create(11, 22); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + } + { + ValueTuple t = STuple.Create(11, 22, 33); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + } + { + ValueTuple t = STuple.Create(11, 22, 33, 44); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + } + { + ValueTuple t = STuple.Create(11, 22, 33, 44, 55); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + Assert.That(t.Item5, Is.EqualTo(55)); + } + { + ValueTuple t = STuple.Create(11, 22, 33, 44, 55, 66); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + Assert.That(t.Item5, Is.EqualTo(55)); + Assert.That(t.Item6, Is.EqualTo(66)); + } + } + + [Test] + public void Test_Implicit_Cast_ValueTuple_To_STuple() + { + { + STuple t = ValueTuple.Create(11); + Assert.That(t.Item1, Is.EqualTo(11)); + } + { + STuple t = ValueTuple.Create(11, 22); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + } + { + STuple t = ValueTuple.Create(11, 22, 33); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + } + { + STuple t = ValueTuple.Create(11, 22, 33, 44); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + } + { + STuple t = ValueTuple.Create(11, 22, 33, 44, 55); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + Assert.That(t.Item5, Is.EqualTo(55)); + } + { + STuple t = ValueTuple.Create(11, 22, 33, 44, 55, 66); + Assert.That(t.Item1, Is.EqualTo(11)); + Assert.That(t.Item2, Is.EqualTo(22)); + Assert.That(t.Item3, Is.EqualTo(33)); + Assert.That(t.Item4, Is.EqualTo(44)); + Assert.That(t.Item5, Is.EqualTo(55)); + Assert.That(t.Item6, Is.EqualTo(66)); + } + } + + private static ValueTuple ProduceValueTuple(int item1, int item2) => ValueTuple.Create(item1, item2); + + private static int[] ConsumeValueTuple(STuple t) => new[] { t.Item1, t.Item2 }; + + private static STuple ProduceSTuple(int item1, int item2) => STuple.Create(item1, item2); + + private static int[] ConsumeSTuple(STuple t) => new[] { t.Item1, t.Item2 }; + + [Test] + public void Test_Can_AutoCast_Transparently() + { + + { // (int, int) => STuple + var res = ConsumeSTuple(ProduceValueTuple(1234, 5)); + Assert.That(res[0], Is.EqualTo(1234)); + Assert.That(res[1], Is.EqualTo(5)); + } + { // literal => STuple + var res = ConsumeSTuple(ValueTuple.Create(1234, 5)); + Assert.That(res[0], Is.EqualTo(1234)); + Assert.That(res[1], Is.EqualTo(5)); + } + { // STuple => (int, int) + var res = ConsumeValueTuple(ProduceSTuple(1234, 5)); + Assert.That(res[0], Is.EqualTo(1234)); + Assert.That(res[1], Is.EqualTo(5)); + } + } + +#endif + + [Test] + public void Test_Deconstruct_STuple() + { + { + STuple.Create(11, 22).Deconstruct(out int a, out int b); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + } + { + STuple.Create(11, 22, 33).Deconstruct(out int a, out int b, out int c); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + } + { + STuple.Create(11, 22, 33, 44).Deconstruct(out int a, out int b, out int c, out int d); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + } + { + STuple.Create(11, 22, 33, 44, 55).Deconstruct(out int a, out int b, out int c, out int d, out int e); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + Assert.That(e, Is.EqualTo(55)); + } + { + STuple.Create(11, 22, 33, 44, 55, 66).Deconstruct(out int a, out int b, out int c, out int d, out int e, out _); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + Assert.That(e, Is.EqualTo(55)); + } + } + +#if ENABLE_VALUETUPLE + + [Test] + public void Test_Deconstruct_STuple_TupleSyntax() + { + { + (var a, var b) = STuple.Create(11, 22); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + } + { + (var a, var b, var c) = STuple.Create(11, 22, 33); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + } + { + (var a, var b, var c, var d) = STuple.Create(11, 22, 33, 44); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + } + { + (var a, var b, var c, var d, var e) = STuple.Create(11, 22, 33, 44, 55); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + Assert.That(e, Is.EqualTo(55)); + } + { + (var a, var b, var c, var d, var e, var f) = STuple.Create(11, 22, 33, 44, 55, 66); + Assert.That(a, Is.EqualTo(11)); + Assert.That(b, Is.EqualTo(22)); + Assert.That(c, Is.EqualTo(33)); + Assert.That(d, Is.EqualTo(44)); + Assert.That(e, Is.EqualTo(55)); + } + } + +#endif + + #endregion + + } +} diff --git a/FoundationDB.Tests/Utils/TypeConvertersFacts.cs b/FoundationDB.Tests/Utils/TypeConvertersFacts.cs new file mode 100644 index 000000000..21b1f6ef9 --- /dev/null +++ b/FoundationDB.Tests/Utils/TypeConvertersFacts.cs @@ -0,0 +1,176 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace FoundationDB.Client.Converters.Tests +{ + using System; + using Doxense.Runtime.Converters; + using FoundationDB.Client.Tests; + using NUnit.Framework; + + [TestFixture] + public class TypeConvertersFacts : FdbTest + { + + [Test] + public void Test_Can_Convert_Numbers_To_Bool() + { + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0), Is.False); + Assert.That(TypeConverters.Convert(0.0f), Is.False); + Assert.That(TypeConverters.Convert(float.NaN), Is.False); + Assert.That(TypeConverters.Convert(0.0d), Is.False); + Assert.That(TypeConverters.Convert(double.NaN), Is.False); + + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123), Is.True); + Assert.That(TypeConverters.Convert(123.0f), Is.True); + Assert.That(TypeConverters.Convert(123.0d), Is.True); + } + + [Test] + public void Test_Can_Convert_Numbers_To_Int32() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_UInt32() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123U)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123U)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_Int64() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123L)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123L)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_UInt64() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123UL)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123UL)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_Single() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123f)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123f)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_Double() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo(123d)); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo(123d)); + } + + [Test] + public void Test_Can_Convert_Numbers_To_String() + { + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123.0f), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123.4f), Is.EqualTo("123.4")); + Assert.That(TypeConverters.Convert(123.0d), Is.EqualTo("123")); + Assert.That(TypeConverters.Convert(123.4d), Is.EqualTo("123.4")); + } + + } +} diff --git a/FoundationDB.Tests/Uuid128Facts.cs b/FoundationDB.Tests/Utils/Uuid128Facts.cs similarity index 84% rename from FoundationDB.Tests/Uuid128Facts.cs rename to FoundationDB.Tests/Utils/Uuid128Facts.cs index 253b760c7..c40a8b153 100644 --- a/FoundationDB.Tests/Uuid128Facts.cs +++ b/FoundationDB.Tests/Utils/Uuid128Facts.cs @@ -28,14 +28,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client.Tests { - using FoundationDB.Client; - using NUnit.Framework; using System; using System.Collections.Generic; using System.Linq; + using NUnit.Framework; [TestFixture] - public class UuidFacts + public class Uuid128Facts : FdbTest { [Test] public void Test_Uuid_Empty() @@ -126,6 +125,46 @@ public void Test_Uuid_NewUuid() Assert.That(uuid.ToGuid().ToString(), Is.EqualTo(uuid.ToString())); } + [Test] + public void Test_Uuid_Increment() + { + var @base = Uuid128.Parse("6be5d394-03a6-42ab-aac2-89b7d9312402"); + Log(@base); + //DumpHexa(@base.ToByteArray()); + + { // +1 + var uuid = @base.Increment(1); + Log(uuid); + //DumpHexa(uuid.ToByteArray()); + Assert.That(uuid.ToString(), Is.EqualTo("6be5d394-03a6-42ab-aac2-89b7d9312403")); + } + { // +256 + var uuid = @base.Increment(256); + Log(uuid); + //DumpHexa(uuid.ToByteArray()); + Assert.That(uuid.ToString(), Is.EqualTo("6be5d394-03a6-42ab-aac2-89b7d9312502")); + } + { // almost overflow (low) + var uuid = @base.Increment(0x553D764826CEDBFDUL); // delta nécessaire pour avoir 0xFFFFFFFFFFFFFFFF a la fin + Log(uuid); + //DumpHexa(uuid.ToByteArray()); + Assert.That(uuid.ToString(), Is.EqualTo("6be5d394-03a6-42ab-ffff-ffffffffffff")); + } + { // overflow (low) + var uuid = @base.Increment(0x553D764826CEDBFEUL); // encore 1 de plus pour trigger l'overflow + Log(uuid); + //DumpHexa(uuid.ToByteArray()); + Assert.That(uuid.ToString(), Is.EqualTo("6be5d394-03a6-42ac-0000-000000000000")); + } + { // overflow (cascade) + var uuid = Uuid128.Parse("ffffffff-ffff-ffff-ffff-ffffffffffff").Increment(1); + Log(uuid); + //DumpHexa(uuid.ToByteArray()); + Assert.That(uuid.ToString(), Is.EqualTo("00000000-0000-0000-0000-000000000000")); + } + + } + [Test] public void Test_Uuid_ToSlice() { @@ -134,7 +173,7 @@ public void Test_Uuid_ToSlice() Assert.That(uuid.ToSlice().Offset, Is.GreaterThanOrEqualTo(0)); Assert.That(uuid.ToSlice().Array, Is.Not.Null); Assert.That(uuid.ToSlice().Array.Length, Is.GreaterThanOrEqualTo(16)); - Assert.That(uuid.ToSlice(), Is.EqualTo(Slice.Create(uuid.ToByteArray()))); + Assert.That(uuid.ToSlice(), Is.EqualTo(uuid.ToByteArray().AsSlice())); Assert.That(uuid.ToSlice().GetBytes(), Is.EqualTo(uuid.ToByteArray())); } diff --git a/FoundationDB.Tests/Utils/Uuid64Facts.cs b/FoundationDB.Tests/Utils/Uuid64Facts.cs new file mode 100644 index 000000000..fa8eaff55 --- /dev/null +++ b/FoundationDB.Tests/Utils/Uuid64Facts.cs @@ -0,0 +1,690 @@ +#region BSD Licence +/* Copyright (c) 2013, Doxense SARL +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 + +//#define ENABLE_SPAN + +// ReSharper disable AssignNullToNotNullAttribute +namespace FoundationDB.Client.Tests +{ + using System; + using System.Collections.Generic; + using System.Linq; + using Doxense.Memory; + using NUnit.Framework; + + [TestFixture] + public class Uuid64Facts : FdbTest + { + [Test] + public void Test_Uuid64_Empty() + { + Assert.That(Uuid64.Empty.ToString(), Is.EqualTo("00000000-00000000")); + Assert.That(Uuid64.Empty, Is.EqualTo(default(Uuid64))); + Assert.That(Uuid64.Empty, Is.EqualTo(new Uuid64(0L))); + Assert.That(Uuid64.Empty, Is.EqualTo(new Uuid64(0UL))); + Assert.That(Uuid64.Empty, Is.EqualTo(Uuid64.Read(new byte[8]))); + } + + [Test] + public void Test_Uuid64_Casting() + { + // implicit + Uuid64 a = (long)0; + Uuid64 b = (long)42; + Uuid64 c = (long)0xDEADBEEF; + Uuid64 d = 0xBADC0FFEE0DDF00DUL; + Uuid64 e = ulong.MaxValue; + + // ToUInt64 + Assert.That(a.ToUInt64(), Is.EqualTo(0UL)); + Assert.That(b.ToUInt64(), Is.EqualTo(42UL)); + Assert.That(c.ToUInt64(), Is.EqualTo(3735928559UL)); + Assert.That(d.ToUInt64(), Is.EqualTo(13464654573299691533UL)); + Assert.That(e.ToUInt64(), Is.EqualTo(ulong.MaxValue)); + + // ToInt64 + Assert.That(a.ToInt64(), Is.EqualTo(0L)); + Assert.That(b.ToInt64(), Is.EqualTo(42L)); + Assert.That(c.ToInt64(), Is.EqualTo(3735928559L)); + Assert.That(d.ToInt64(), Is.EqualTo(-4982089500409860083L)); + Assert.That(e.ToInt64(), Is.EqualTo(-1L)); + + // explict + Assert.That((long)a, Is.EqualTo(0)); + Assert.That((long)b, Is.EqualTo(42)); + Assert.That((long)c, Is.EqualTo(0xDEADBEEF)); + Assert.That((ulong)d, Is.EqualTo(13464654573299691533UL)); + Assert.That((ulong)e, Is.EqualTo(ulong.MaxValue)); + Assert.That((long)e, Is.EqualTo(-1L)); + } + + [Test] + public void Test_Uuid64_ToString() + { + var guid = new Uuid64(0xBADC0FFEE0DDF00DUL); + Assert.That(guid.ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(guid.ToString(), Is.EqualTo("BADC0FFE-E0DDF00D")); + Assert.That(guid.ToString("X"), Is.EqualTo("BADC0FFEE0DDF00D")); + Assert.That(guid.ToString("B"), Is.EqualTo("{BADC0FFE-E0DDF00D}")); + Assert.That(guid.ToString("C"), Is.EqualTo("G2eGAUq82Hd")); + + guid = new Uuid64(0xDEADBEEFUL); + Assert.That(guid.ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); + Assert.That(guid.ToString(), Is.EqualTo("00000000-DEADBEEF")); + Assert.That(guid.ToString("X"), Is.EqualTo("00000000DEADBEEF")); + Assert.That(guid.ToString("B"), Is.EqualTo("{00000000-DEADBEEF}")); + Assert.That(guid.ToString("C"), Is.EqualTo("44pZgF")); + } + + [Test] + public void Test_Uuid64_Parse_Hexa16() + { + // string + + Assert.That(Uuid64.Parse("badc0ffe-e0ddf00d").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("BADC0FFE-E0DDF00D").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL), "Should be case-insensitive"); + + Assert.That(Uuid64.Parse("badc0ffee0ddf00d").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("BADC0FFEE0DDF00D").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL), "Should be case-insensitive"); + + Assert.That(Uuid64.Parse("{badc0ffe-e0ddf00d}").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("{BADC0FFE-E0DDF00D}").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL), "Should be case-insensitive"); + + Assert.That(Uuid64.Parse("{badc0ffee0ddf00d}").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("{BADC0FFEE0DDF00D}").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL), "should be case-insensitive"); + + Assert.That(Uuid64.Parse("00000000-deadbeef").ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); + Assert.That(Uuid64.Parse("{00000000-deadbeef}").ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); + + // errors + Assert.That(() => Uuid64.Parse(default(string)), Throws.InstanceOf()); + Assert.That(() => Uuid64.Parse("hello"), Throws.InstanceOf()); + Assert.That(() => Uuid64.Parse("12345678-9ABCDEFG"), Throws.InstanceOf(), "Invalid hexa character 'G'"); + Assert.That(() => Uuid64.Parse("00000000-0000000 "), Throws.InstanceOf(), "Two short + extra space"); + Assert.That(() => Uuid64.Parse("zzzzzzzz-zzzzzzzz"), Throws.InstanceOf(), "Invalid char"); + Assert.That(() => Uuid64.Parse("badc0ffe-e0ddf00"), Throws.InstanceOf(), "Missing last char"); + Assert.That(() => Uuid64.Parse("baadc0ffe-e0ddf00"), Throws.InstanceOf(), "'-' at invalid position"); + Assert.That(() => Uuid64.Parse("badc0fe-ee0ddf00d"), Throws.InstanceOf(), "'-' at invalid position"); + Assert.That(() => Uuid64.Parse("badc0ffe-e0ddf00d "), Throws.InstanceOf(), "Extra space at the end"); + Assert.That(() => Uuid64.Parse(" badc0ffe-e0ddf00d"), Throws.InstanceOf(), "Extra space at the start"); + +#if ENABLE_SPAN + // span from string + + Assert.That(Uuid64.Parse("badc0ffe-e0ddf00d".AsSpan()).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("badc0ffee0ddf00d".AsSpan()).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("hello badc0ffe-e0ddf00d world!".AsSpan().Slice(6, 17)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("hello badc0ffee0ddf00d world!".AsSpan().Slice(6, 16)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + // span from char[] + + Assert.That(Uuid64.Parse("badc0ffe-e0ddf00d".ToCharArray().AsSpan()).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("badc0ffee0ddf00d".ToCharArray().AsSpan()).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("hello badc0ffe-e0ddf00d world!".ToCharArray().AsSpan().Slice(6, 17)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + Assert.That(Uuid64.Parse("hello badc0ffee0ddf00d world!".ToCharArray().AsSpan().Slice(6, 16)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + // span from stackalloc + + unsafe + { + char* buf = stackalloc char[64]; + var span = new Span(buf, 64); + + span.Clear(); + "badc0ffe-e0ddf00d".AsSpan().CopyTo(span); + Assert.That(Uuid64.Parse(span.Slice(0, 17)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + span.Clear(); + "badc0ffee0ddf00d".AsSpan().CopyTo(span); + Assert.That(Uuid64.Parse(span.Slice(0, 16)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + span.Clear(); + "{badc0ffe-e0ddf00d}".AsSpan().CopyTo(span); + Assert.That(Uuid64.Parse(span.Slice(0, 19)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + span.Clear(); + "{badc0ffee0ddf00d}".AsSpan().CopyTo(span); + Assert.That(Uuid64.Parse(span.Slice(0, 18)).ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + } +#endif + } + + [Test] + public void Test_Uuid64_ToString_Base62() + { + char[] chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz".ToCharArray(); + Assert.That(chars.Length, Is.EqualTo(62)); + + // single digit + for (int i = 0; i < 62;i++) + { + Assert.That(new Uuid64(i).ToString("C"), Is.EqualTo(chars[i].ToString())); + Assert.That(new Uuid64(i).ToString("Z"), Is.EqualTo("0000000000" + chars[i])); + } + + // two digits + for (int j = 1; j < 62; j++) + { + var prefix = chars[j].ToString(); + for (int i = 0; i < 62; i++) + { + Assert.That(new Uuid64(j * 62 + i).ToString("C"), Is.EqualTo(prefix + chars[i])); + Assert.That(new Uuid64(j * 62 + i).ToString("Z"), Is.EqualTo("000000000" + prefix + chars[i])); + } + } + + // 4 digits + var rnd = new Random(); + for (int i = 0; i < 100 * 1000; i++) + { + var a = rnd.Next(2) == 0 ? 0 : rnd.Next(62); + var b = rnd.Next(2) == 0 ? 0 : rnd.Next(62); + var c = rnd.Next(2) == 0 ? 0 : rnd.Next(62); + var d = rnd.Next(62); + + ulong x = (ulong)a; + x += 62 * (ulong)b; + x += 62 * 62 * (ulong)c; + x += 62 * 62 * 62 * (ulong)d; + var uuid = new Uuid64(x); + + // no padding + string expected = + d > 0 ? ("" + chars[d] + chars[c] + chars[b] + chars[a]) : + c > 0 ? ("" + chars[c] + chars[b] + chars[a]) : + b > 0 ? ("" + chars[b] + chars[a]) : + ("" + chars[a]); + Assert.That(uuid.ToString("C"), Is.EqualTo(expected)); + + // padding + Assert.That(uuid.ToString("Z"), Is.EqualTo("0000000" + chars[d] + chars[c] + chars[b] + chars[a])); + } + + // Numbers of the form 62^n should be encoded as '1' followed by n x '0', for n from 0 to 10 + ulong val = 1; + for (int i = 0; i <= 10; i++) + { + Assert.That(new Uuid64(val).ToString("C"), Is.EqualTo("1" + new string('0', i)), "62^{0}", i); + val *= 62; + } + + // Numbers of the form 62^n - 1 should be encoded as n x 'z', for n from 1 to 10 + val = 0; + for (int i = 1; i <= 10; i++) + { + val += 61; + Assert.That(new Uuid64(val).ToString("C"), Is.EqualTo(new string('z', i)), "62^{0} - 1", i); + val *= 62; + } + + // well known values + Assert.That(new Uuid64(0xB45B07).ToString("C"), Is.EqualTo("narf")); + Assert.That(new Uuid64(0xE0D0ED).ToString("C"), Is.EqualTo("zort")); + Assert.That(new Uuid64(0xDEADBEEF).ToString("C"), Is.EqualTo("44pZgF")); + Assert.That(new Uuid64(0xDEADBEEF).ToString("Z"), Is.EqualTo("0000044pZgF")); + Assert.That(new Uuid64(0xBADC0FFEE0DDF00DUL).ToString("C"), Is.EqualTo("G2eGAUq82Hd")); + Assert.That(new Uuid64(0xBADC0FFEE0DDF00DUL).ToString("Z"), Is.EqualTo("G2eGAUq82Hd")); + + Assert.That(new Uuid64(255).ToString("C"), Is.EqualTo("47")); + Assert.That(new Uuid64(ushort.MaxValue).ToString("C"), Is.EqualTo("H31")); + Assert.That(new Uuid64(uint.MaxValue).ToString("C"), Is.EqualTo("4gfFC3")); + Assert.That(new Uuid64(ulong.MaxValue - 1).ToString("C"), Is.EqualTo("LygHa16AHYE")); + Assert.That(new Uuid64(ulong.MaxValue).ToString("C"), Is.EqualTo("LygHa16AHYF")); + } + + [Test] + public void Test_Uuid64_Parse_Base62() + { + + Assert.That(Uuid64.FromBase62("").ToUInt64(), Is.EqualTo(0)); + Assert.That(Uuid64.FromBase62("0").ToUInt64(), Is.EqualTo(0)); + Assert.That(Uuid64.FromBase62("9").ToUInt64(), Is.EqualTo(9)); + Assert.That(Uuid64.FromBase62("A").ToUInt64(), Is.EqualTo(10)); + Assert.That(Uuid64.FromBase62("Z").ToUInt64(), Is.EqualTo(35)); + Assert.That(Uuid64.FromBase62("a").ToUInt64(), Is.EqualTo(36)); + Assert.That(Uuid64.FromBase62("z").ToUInt64(), Is.EqualTo(61)); + Assert.That(Uuid64.FromBase62("10").ToUInt64(), Is.EqualTo(62)); + Assert.That(Uuid64.FromBase62("zz").ToUInt64(), Is.EqualTo(3843)); + Assert.That(Uuid64.FromBase62("100").ToUInt64(), Is.EqualTo(3844)); + Assert.That(Uuid64.FromBase62("zzzzzzzzzz").ToUInt64(), Is.EqualTo(839299365868340223UL)); + Assert.That(Uuid64.FromBase62("10000000000").ToUInt64(), Is.EqualTo(839299365868340224UL)); + Assert.That(Uuid64.FromBase62("LygHa16AHYF").ToUInt64(), Is.EqualTo(ulong.MaxValue), "ulong.MaxValue in base 62"); + + // well known values + + Assert.That(Uuid64.FromBase62("narf").ToUInt64(), Is.EqualTo(0xB45B07)); + Assert.That(Uuid64.FromBase62("zort").ToUInt64(), Is.EqualTo(0xE0D0ED)); + Assert.That(Uuid64.FromBase62("44pZgF").ToUInt64(), Is.EqualTo(0xDEADBEEF)); + Assert.That(Uuid64.FromBase62("0000044pZgF").ToUInt64(), Is.EqualTo(0xDEADBEEF)); + + Assert.That(Uuid64.FromBase62("G2eGAUq82Hd").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); + + Assert.That(Uuid64.FromBase62("4gfFC3").ToUInt64(), Is.EqualTo(uint.MaxValue)); + Assert.That(Uuid64.FromBase62("000004gfFC3").ToUInt64(), Is.EqualTo(uint.MaxValue)); + + + // invalid chars + Assert.That(() => Uuid64.FromBase62("/"), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("@"), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("["), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("`"), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("{"), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("zaz/"), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("z/o&r=g"), Throws.InstanceOf()); + + // overflow + Assert.That(() => Uuid64.FromBase62("zzzzzzzzzzz"), Throws.InstanceOf(), "62^11 - 1 => OVERFLOW"); + Assert.That(() => Uuid64.FromBase62("LygHa16AHYG"), Throws.InstanceOf(), "ulong.MaxValue + 1 => OVERFLOW"); + + // invalid length + Assert.That(() => Uuid64.FromBase62(default(string)), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62("100000000000"), Throws.InstanceOf(), "62^11 => TOO BIG"); + + } + + [Test] + public void Test_Uuid64_NewUid() + { + var a = Uuid64.NewUuid(); + var b = Uuid64.NewUuid(); + Assert.That(a.ToUInt64(), Is.Not.EqualTo(b.ToUInt64())); + Assert.That(a, Is.Not.EqualTo(b)); + + const int N = 1 * 1000; + var uids = new HashSet(); + for (int i = 0; i < N; i++) + { + var uid = Uuid64.NewUuid(); + if (uids.Contains(uid.ToUInt64())) Assert.Fail("Duplicate Uuid64 generated: {0}", uid); + uids.Add(uid.ToUInt64()); + } + Assert.That(uids.Count, Is.EqualTo(N)); + } + + [Test] + public void Test_Uuid64RangomGenerator_NewUid() + { + var gen = Uuid64RandomGenerator.Default; + Assert.That(gen, Is.Not.Null); + + var a = gen.NewUuid(); + var b = gen.NewUuid(); + Assert.That(a.ToUInt64(), Is.Not.EqualTo(b.ToUInt64())); + Assert.That(a, Is.Not.EqualTo(b)); + + const int N = 1 * 1000; + var uids = new HashSet(); + for (int i = 0; i < N; i++) + { + var uid = gen.NewUuid(); + if (uids.Contains(uid.ToUInt64())) Assert.Fail("Duplicate Uuid64 generated: {0}", uid); + uids.Add(uid.ToUInt64()); + } + Assert.That(uids.Count, Is.EqualTo(N)); + } + + [Test] + public void Test_Uuid64_Equality_Check() + { + var a = new Uuid64(42); + var b = new Uuid64(42); + var c = new Uuid64(40) + 2; + var d = new Uuid64(0xDEADBEEF); + + // Equals(Uuid64) + Assert.That(a.Equals(a), Is.True, "a == a"); + Assert.That(a.Equals(b), Is.True, "a == b"); + Assert.That(a.Equals(c), Is.True, "a == c"); + Assert.That(a.Equals(d), Is.False, "a != d"); + + // == Uuid64 + Assert.That(a == b, Is.True, "a == b"); + Assert.That(a == c, Is.True, "a == c"); + Assert.That(a == d, Is.False, "a != d"); + + // != Uuid64 + Assert.That(a != b, Is.False, "a == b"); + Assert.That(a != c, Is.False, "a == c"); + Assert.That(a != d, Is.True, "a != d"); + + // == numbers + Assert.That(a == 42L, Is.True, "a == 42"); + Assert.That(a == 42UL, Is.True, "a == 42"); + Assert.That(d == 42L, Is.False, "d != 42"); + Assert.That(d == 42UL, Is.False, "d != 42"); + + // != numbers + Assert.That(a != 42L, Is.False, "a == 42"); + Assert.That(a != 42UL, Is.False, "a == 42"); + Assert.That(d != 42L, Is.True, "d != 42"); + Assert.That(d != 42UL, Is.True, "d != 42"); + + // Equals(objecct) + Assert.That(a.Equals((object)a), Is.True, "a == a"); + Assert.That(a.Equals((object)b), Is.True, "a == b"); + Assert.That(a.Equals((object)c), Is.True, "a == c"); + Assert.That(a.Equals((object)d), Is.False, "a != d"); + Assert.That(a.Equals((object)42L), Is.True, "a == 42"); + Assert.That(a.Equals((object)42UL), Is.True, "a == 42"); + Assert.That(d.Equals((object)42L), Is.False, "d != 42"); + Assert.That(d.Equals((object)42UL), Is.False, "d != 42"); + + } + + [Test] + public void Test_Uuid64_Ordering() + { + var a = new Uuid64(42); + var a2 = new Uuid64(42); + var b = new Uuid64(77); + + Assert.That(a.CompareTo(a), Is.EqualTo(0)); + Assert.That(a.CompareTo(b), Is.EqualTo(-1)); + Assert.That(b.CompareTo(a), Is.EqualTo(+1)); + + Assert.That(a < b, Is.True, "a < b"); + Assert.That(a <= b, Is.True, "a <= b"); + Assert.That(a < a2, Is.False, "a < a"); + Assert.That(a <= a2, Is.True, "a <= a"); + + Assert.That(a > b, Is.False, "a > b"); + Assert.That(a >= b, Is.False, "a >= b"); + Assert.That(a > a2, Is.False, "a > a"); + Assert.That(a >= a2, Is.True, "a >= a"); + + // parsed from string + Assert.That(Uuid64.Parse("137bcf31-0c8873a2") < Uuid64.Parse("604bdf8a-2512b4ad"), Is.True); + Assert.That(Uuid64.Parse("d8f17a26-82adb1a4") < Uuid64.Parse("22abbf33-1b2c1db0"), Is.False); + Assert.That(Uuid64.Parse("{137bcf31-0c8873a2}") > Uuid64.Parse("{604bdf8a-2512b4ad}"), Is.False); + Assert.That(Uuid64.Parse("{d8f17a26-82adb1a4}") > Uuid64.Parse("{22abbf33-1b2c1db0}"), Is.True); + Assert.That(Uuid64.FromBase62("2w6CTjUiXVp") < Uuid64.FromBase62("DVM0UnynZ1Q"), Is.True); + Assert.That(Uuid64.FromBase62("0658JY2ORSJ") > Uuid64.FromBase62("FMPaNaMEUWc"), Is.False); + + // verify byte ordering + var c = new Uuid64(0x0000000100000002); + var d = new Uuid64(0x0000000200000001); + Assert.That(c.CompareTo(d), Is.EqualTo(-1)); + Assert.That(d.CompareTo(c), Is.EqualTo(+1)); + + // verify that we can sort an array of Uuid64 + var uids = new Uuid64[100]; + for (int i = 0; i < uids.Length; i++) + { + uids[i] = Uuid64.NewUuid(); + } + Assume.That(uids, Is.Not.Ordered, "This can happen with a very small probability. Please try again"); + Array.Sort(uids); + Assert.That(uids, Is.Ordered); + + // ordering should be preserved in integer or textual form + + Assert.That(uids.Select(x => x.ToUInt64()), Is.Ordered, "order should be preserved when ordering by unsigned value"); + //note: ToInt64() will not work because of negative values + Assert.That(uids.Select(x => x.ToString()), Is.Ordered.Using(StringComparer.Ordinal), "order should be preserved when ordering by text (hexa)"); + Assert.That(uids.Select(x => x.ToString("Z")), Is.Ordered.Using(StringComparer.Ordinal), "order should be preserved when ordering by text (base62)"); + //note: ToString("C") will not work for ordering because it will produce "z" > "aa", instead of expected "0z" < "aa" + } + + [Test] + public void Test_Uuid64_Arithmetic() + { + var uid = Uuid64.Empty; + + Assert.That(uid + 42L, Is.EqualTo(new Uuid64(42))); + Assert.That(uid + 42UL, Is.EqualTo(new Uuid64(42))); + uid++; + Assert.That(uid.ToInt64(), Is.EqualTo(1)); + uid++; + Assert.That(uid.ToInt64(), Is.EqualTo(2)); + uid--; + Assert.That(uid.ToInt64(), Is.EqualTo(1)); + uid--; + Assert.That(uid.ToInt64(), Is.EqualTo(0)); + + uid = Uuid64.NewUuid(); + + Assert.That(uid + 123L, Is.EqualTo(new Uuid64(uid.ToInt64() + 123))); + Assert.That(uid + 123UL, Is.EqualTo(new Uuid64(uid.ToUInt64() + 123))); + + Assert.That(uid - 123L, Is.EqualTo(new Uuid64(uid.ToInt64() - 123))); + Assert.That(uid - 123UL, Is.EqualTo(new Uuid64(uid.ToUInt64() - 123))); + } + + [Test] + public void Test_Uuid64_Read_From_Bytes() + { + // test buffer with included padding + byte[] buf = { 0x55, 0x55, 0x55, 0x55, /* start */ 0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF, /* stop */ 0xAA, 0xAA, 0xAA, 0xAA, 0xAA, 0xAA, 0xAA, 0xAA }; + var original = Uuid64.Parse("01234567-89ABCDEF"); + Assume.That(original.ToUInt64(), Is.EqualTo(0x0123456789ABCDEF)); + +#if ENABLE_SPAN + // ReadOnlySpan + Assert.That(Uuid64.Read(buf.AsSpan(4, 8)), Is.EqualTo(original)); +#endif + + // Slice + Assert.That(Uuid64.Read(buf.AsSlice(4, 8)), Is.EqualTo(original)); + + // byte[] + Assert.That(Uuid64.Read(buf.AsSlice(4, 8).GetBytesOrEmpty()), Is.EqualTo(original)); + + unsafe + { + fixed (byte* ptr = &buf[4]) + { + Assert.That(Uuid64.Read(ptr, 8), Is.EqualTo(original)); +#if ENABLE_SPAN + Assert.That(Uuid64.Read(new ReadOnlySpan(ptr, 8)), Is.EqualTo(original)); +#endif + } + } + } + +#if ENABLE_SPAN + + [Test] + public void Test_UUid64_WriteTo() + { + var original = Uuid64.Parse("01234567-89ABCDEF"); + Assume.That(original.ToUInt64(), Is.EqualTo(0x0123456789ABCDEF)); + + // span with more space + var scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.AsSpan()); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with no offset and exact size + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.AsSpan(0, 8)); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with offset + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.AsSpan(4)); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // span with offset and exact size + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.AsSpan(4, 8)); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + scratch = Slice.Repeat(0xAA, 16); + original.WriteToUnsafe(scratch.Array, scratch.Offset); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + unsafe + { + byte* buf = stackalloc byte[16]; + var span = USlice.FromUnmanagedPointer(buf, 16); + span.Fill(0xAA); + + original.WriteToUnsafe(buf + 2); + Assert.That(span.ToString("X"), Is.EqualTo("AA AA 01 23 45 67 89 AB CD EF AA AA AA AA AA AA")); + } + + // errors + + Assert.That(() => original.WriteTo(Span.Empty), Throws.InstanceOf(), "Target buffer is empty"); + Assert.That(() => original.WriteTo(null, 8), Throws.InstanceOf(), "Target buffer is null"); + Assert.That(() => original.WriteTo(null, 0), Throws.InstanceOf(), "Target buffer is null"); + + scratch = Slice.Repeat(0xAA, 16); + Assert.That(() => original.WriteTo(scratch.AsSpan(0, 7)), Throws.InstanceOf(), "Target buffer is too small"); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA"), "Buffer should not have been overwritten!"); + + } + + [Test] + public void Test_Uuid64_TryWriteTo() + { + var original = Uuid64.Parse("01234567-89ABCDEF"); + Assume.That(original.ToUInt64(), Is.EqualTo(0x0123456789ABCDEF)); + + // span with more space + var scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.AsSpan()), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with no offset and exact size + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.AsSpan(0, 8)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with offset + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.AsSpan(4)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // span with offset and exact size + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.AsSpan(4, 8)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // errors + + Assert.That(original.TryWriteTo(Span.Empty), Is.False, "Target buffer is empty"); + + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.AsSpan(0, 7)), Is.False, "Target buffer is too small"); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA"), "Buffer should not have been overwritten!"); + + } + +#else + + [Test] + public void Test_UUid64_WriteTo() + { + var original = Uuid64.Parse("01234567-89ABCDEF"); + Assume.That(original.ToUInt64(), Is.EqualTo(0x0123456789ABCDEF)); + + // span with more space + var scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with no offset and exact size + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.Substring(0, 8)); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with offset + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.Substring(4)); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // span with offset and exact size + scratch = Slice.Repeat(0xAA, 16); + original.WriteTo(scratch.Substring(4, 8)); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + scratch = Slice.Repeat(0xAA, 16); + original.WriteToUnsafe(scratch.Array, scratch.Offset); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + unsafe + { + byte* buf = stackalloc byte[16]; + UnsafeHelpers.FillUnsafe(buf, 16, 0xAA); + + original.WriteToUnsafe(buf + 2); + Assert.That(Slice.Copy(buf, 16).ToString("X"), Is.EqualTo("AA AA 01 23 45 67 89 AB CD EF AA AA AA AA AA AA")); + } + + // errors + + Assert.That(() => original.WriteTo(Slice.Empty), Throws.InstanceOf()); //, "Target buffer is empty"); + Assert.That(() => original.WriteTo(null, 8), Throws.InstanceOf()); //, "Target buffer is null"); + Assert.That(() => original.WriteTo(null, 0), Throws.InstanceOf()); //, "Target buffer is null"); + + scratch = Slice.Repeat(0xAA, 16); + Assert.That(() => original.WriteTo(scratch.Substring(0, 7)), Throws.InstanceOf()); //, "Target buffer is too small"); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA"), "Buffer should not have been overwritten!"); + + } + + [Test] + public void Test_Uuid64_TryWriteTo() + { + var original = Uuid64.Parse("01234567-89ABCDEF"); + Assume.That(original.ToUInt64(), Is.EqualTo(0x0123456789ABCDEF)); + + // span with more space + var scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with no offset and exact size + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.Substring(0, 8)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("01 23 45 67 89 AB CD EF AA AA AA AA AA AA AA AA")); + + // span with offset + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.Substring(4)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // span with offset and exact size + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.Substring(4, 8)), Is.True); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA 01 23 45 67 89 AB CD EF AA AA AA AA")); + + // errors + + Assert.That(original.TryWriteTo(Slice.Empty), Is.False, "Target buffer is empty"); + + scratch = Slice.Repeat(0xAA, 16); + Assert.That(original.TryWriteTo(scratch.Substring(0, 7)), Is.False, "Target buffer is too small"); + Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA"), "Buffer should not have been overwritten!"); + + } +#endif + } + +} diff --git a/FoundationDB.Tests/Uuid64Facts.cs b/FoundationDB.Tests/Uuid64Facts.cs deleted file mode 100644 index 6b9b4cf54..000000000 --- a/FoundationDB.Tests/Uuid64Facts.cs +++ /dev/null @@ -1,420 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013, Doxense SARL -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 - -namespace FoundationDB.Client.Tests -{ - using FoundationDB.Client; - using NUnit.Framework; - using System; - using System.Collections.Generic; - using System.Linq; - - [TestFixture] - public class Uuid64Facts - { - [Test] - public void Test_Uuid64_Empty() - { - Assert.That(Uuid64.Empty.ToString(), Is.EqualTo("00000000-00000000")); - Assert.That(Uuid64.Empty, Is.EqualTo(default(Uuid64))); - Assert.That(Uuid64.Empty, Is.EqualTo(new Uuid64(0L))); - Assert.That(Uuid64.Empty, Is.EqualTo(new Uuid64(0UL))); - Assert.That(Uuid64.Empty, Is.EqualTo(new Uuid64(new byte[8]))); - } - - [Test] - public void Test_Uuid64_Casting() - { - // implicit - Uuid64 a = (long)0; - Uuid64 b = (long)42; - Uuid64 c = (long)0xDEADBEEF; - Uuid64 d = 0xBADC0FFEE0DDF00DUL; - Uuid64 e = ulong.MaxValue; - - // ToUInt64 - Assert.That(a.ToUInt64(), Is.EqualTo(0UL)); - Assert.That(b.ToUInt64(), Is.EqualTo(42UL)); - Assert.That(c.ToUInt64(), Is.EqualTo(3735928559UL)); - Assert.That(d.ToUInt64(), Is.EqualTo(13464654573299691533UL)); - Assert.That(e.ToUInt64(), Is.EqualTo(ulong.MaxValue)); - - // ToInt64 - Assert.That(a.ToInt64(), Is.EqualTo(0L)); - Assert.That(b.ToInt64(), Is.EqualTo(42L)); - Assert.That(c.ToInt64(), Is.EqualTo(3735928559L)); - Assert.That(d.ToInt64(), Is.EqualTo(-4982089500409860083L)); - Assert.That(e.ToInt64(), Is.EqualTo(-1L)); - - // explict - Assert.That((long)a, Is.EqualTo(0)); - Assert.That((long)b, Is.EqualTo(42)); - Assert.That((long)c, Is.EqualTo(0xDEADBEEF)); - Assert.That((ulong)d, Is.EqualTo(13464654573299691533UL)); - Assert.That((ulong)e, Is.EqualTo(ulong.MaxValue)); - Assert.That((long)e, Is.EqualTo(-1L)); - } - - [Test] - public void Test_Uuid64_ToString() - { - var guid = new Uuid64(0xBADC0FFEE0DDF00DUL); - Assert.That(guid.ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); - Assert.That(guid.ToString(), Is.EqualTo("badc0ffe-e0ddf00d")); - Assert.That(guid.ToString("X"), Is.EqualTo("badc0ffee0ddf00d")); - Assert.That(guid.ToString("B"), Is.EqualTo("{badc0ffe-e0ddf00d}")); - Assert.That(guid.ToString("C"), Is.EqualTo("G2eGAUq82Hd")); - - guid = new Uuid64(0xDEADBEEFUL); - Assert.That(guid.ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); - Assert.That(guid.ToString(), Is.EqualTo("00000000-deadbeef")); - Assert.That(guid.ToString("X"), Is.EqualTo("00000000deadbeef")); - Assert.That(guid.ToString("B"), Is.EqualTo("{00000000-deadbeef}")); - Assert.That(guid.ToString("C"), Is.EqualTo("44pZgF")); - } - - [Test] - public void Test_Uuid64_Parse_Hexa16() - { - var uuid = Uuid64.Parse("badc0ffe-e0ddf00d"); - Assert.That(uuid.ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); - - uuid = Uuid64.Parse("{badc0ffe-e0ddf00d}"); - Assert.That(uuid.ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); - - uuid = Uuid64.Parse("00000000-deadbeef"); - Assert.That(uuid.ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); - - uuid = Uuid64.Parse("{00000000-deadbeef}"); - Assert.That(uuid.ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); - } - - [Test] - public void Test_Uuid64_ToString_Base62() - { - char[] chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz".ToCharArray(); - Assert.That(chars.Length, Is.EqualTo(62)); - - // single digit - for (int i = 0; i < 62;i++) - { - Assert.That(new Uuid64(i).ToString("C"), Is.EqualTo(chars[i].ToString())); - Assert.That(new Uuid64(i).ToString("Z"), Is.EqualTo("0000000000" + chars[i])); - } - - // two digits - for (int j = 1; j < 62; j++) - { - var prefix = chars[j].ToString(); - for (int i = 0; i < 62; i++) - { - Assert.That(new Uuid64(j * 62 + i).ToString("C"), Is.EqualTo(prefix + chars[i])); - Assert.That(new Uuid64(j * 62 + i).ToString("Z"), Is.EqualTo("000000000" + prefix + chars[i])); - } - } - - // 4 digits - var rnd = new Random(); - for (int i = 0; i < 100 * 1000; i++) - { - var a = rnd.Next(2) == 0 ? 0 : rnd.Next(62); - var b = rnd.Next(2) == 0 ? 0 : rnd.Next(62); - var c = rnd.Next(2) == 0 ? 0 : rnd.Next(62); - var d = rnd.Next(62); - - ulong x = (ulong)a; - x += 62 * (ulong)b; - x += 62 * 62 * (ulong)c; - x += 62 * 62 * 62 * (ulong)d; - var uuid = new Uuid64(x); - - // no padding - string expected = - d > 0 ? ("" + chars[d] + chars[c] + chars[b] + chars[a]) : - c > 0 ? ("" + chars[c] + chars[b] + chars[a]) : - b > 0 ? ("" + chars[b] + chars[a]) : - ("" + chars[a]); - Assert.That(uuid.ToString("C"), Is.EqualTo(expected)); - - // padding - Assert.That(uuid.ToString("Z"), Is.EqualTo("0000000" + chars[d] + chars[c] + chars[b] + chars[a])); - } - - // Numbers of the form 62^n should be encoded as '1' followed by n x '0', for n from 0 to 10 - ulong val = 1; - for (int i = 0; i <= 10; i++) - { - Assert.That(new Uuid64(val).ToString("C"), Is.EqualTo("1" + new string('0', i)), "62^{0}", i); - val *= 62; - } - - // Numbers of the form 62^n - 1 should be encoded as n x 'z', for n from 1 to 10 - val = 0; - for (int i = 1; i <= 10; i++) - { - val += 61; - Assert.That(new Uuid64(val).ToString("C"), Is.EqualTo(new string('z', i)), "62^{0} - 1", i); - val *= 62; - } - - // well known values - Assert.That(new Uuid64(0xB45B07).ToString("C"), Is.EqualTo("narf")); - Assert.That(new Uuid64(0xE0D0ED).ToString("C"), Is.EqualTo("zort")); - Assert.That(new Uuid64(0xDEADBEEF).ToString("C"), Is.EqualTo("44pZgF")); - Assert.That(new Uuid64(0xDEADBEEF).ToString("Z"), Is.EqualTo("0000044pZgF")); - Assert.That(new Uuid64(0xBADC0FFEE0DDF00DUL).ToString("C"), Is.EqualTo("G2eGAUq82Hd")); - Assert.That(new Uuid64(0xBADC0FFEE0DDF00DUL).ToString("Z"), Is.EqualTo("G2eGAUq82Hd")); - - Assert.That(new Uuid64(255).ToString("C"), Is.EqualTo("47")); - Assert.That(new Uuid64(ushort.MaxValue).ToString("C"), Is.EqualTo("H31")); - Assert.That(new Uuid64(uint.MaxValue).ToString("C"), Is.EqualTo("4gfFC3")); - Assert.That(new Uuid64(ulong.MaxValue - 1).ToString("C"), Is.EqualTo("LygHa16AHYE")); - Assert.That(new Uuid64(ulong.MaxValue).ToString("C"), Is.EqualTo("LygHa16AHYF")); - } - - [Test] - public void Test_Uuid64_Parse_Base62() - { - - Assert.That(Uuid64.Parse("0").ToUInt64(), Is.EqualTo(0)); - Assert.That(Uuid64.Parse("9").ToUInt64(), Is.EqualTo(9)); - Assert.That(Uuid64.Parse("A").ToUInt64(), Is.EqualTo(10)); - Assert.That(Uuid64.Parse("Z").ToUInt64(), Is.EqualTo(35)); - Assert.That(Uuid64.Parse("a").ToUInt64(), Is.EqualTo(36)); - Assert.That(Uuid64.Parse("z").ToUInt64(), Is.EqualTo(61)); - Assert.That(Uuid64.Parse("10").ToUInt64(), Is.EqualTo(62)); - Assert.That(Uuid64.Parse("zz").ToUInt64(), Is.EqualTo(3843)); - Assert.That(Uuid64.Parse("100").ToUInt64(), Is.EqualTo(3844)); - Assert.That(Uuid64.Parse("zzzzzzzzzz").ToUInt64(), Is.EqualTo(839299365868340223UL)); - Assert.That(Uuid64.Parse("10000000000").ToUInt64(), Is.EqualTo(839299365868340224UL)); - Assert.That(Uuid64.Parse("LygHa16AHYF").ToUInt64(), Is.EqualTo(ulong.MaxValue), "ulong.MaxValue in base 62"); - - // well known values - - Assert.That(Uuid64.Parse("narf").ToUInt64(), Is.EqualTo(0xB45B07)); - Assert.That(Uuid64.Parse("zort").ToUInt64(), Is.EqualTo(0xE0D0ED)); - Assert.That(Uuid64.Parse("44pZgF").ToUInt64(), Is.EqualTo(0xDEADBEEF)); - Assert.That(Uuid64.Parse("0000044pZgF").ToUInt64(), Is.EqualTo(0xDEADBEEF)); - - Assert.That(Uuid64.Parse("G2eGAUq82Hd").ToUInt64(), Is.EqualTo(0xBADC0FFEE0DDF00DUL)); - - Assert.That(Uuid64.Parse("4gfFC3").ToUInt64(), Is.EqualTo(uint.MaxValue)); - Assert.That(Uuid64.Parse("000004gfFC3").ToUInt64(), Is.EqualTo(uint.MaxValue)); - - - // invalid chars - Assert.That(() => Uuid64.Parse("/"), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("@"), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("["), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("`"), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("{"), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("zaz/"), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("z/o&r=g"), Throws.InstanceOf()); - - // overflow - Assert.That(() => Uuid64.Parse("zzzzzzzzzzz"), Throws.InstanceOf(), "62^11 - 1 => OVERFLOW"); - Assert.That(() => Uuid64.Parse("LygHa16AHYG"), Throws.InstanceOf(), "ulong.MaxValue + 1 => OVERFLOW"); - - // invalid length - Assert.That(() => Uuid64.Parse(null), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse(""), Throws.InstanceOf()); - Assert.That(() => Uuid64.Parse("100000000000"), Throws.InstanceOf(), "62^11 => TOO BIG"); - - } - - [Test] - public void Test_Uuid64_NewUid() - { - var a = Uuid64.NewUuid(); - var b = Uuid64.NewUuid(); - Assert.That(a.ToUInt64(), Is.Not.EqualTo(b.ToUInt64())); - Assert.That(a, Is.Not.EqualTo(b)); - - const int N = 1 * 1000; - var uids = new HashSet(); - for (int i = 0; i < N; i++) - { - var uid = Uuid64.NewUuid(); - if (uids.Contains(uid.ToUInt64())) Assert.Fail("Duplicate Uuid64 generated: {0}", uid); - uids.Add(uid.ToUInt64()); - } - Assert.That(uids.Count, Is.EqualTo(N)); - } - - [Test] - public void Test_Uuid64RangomGenerator_NewUid() - { - var gen = Uuid64RandomGenerator.Default; - Assert.That(gen, Is.Not.Null); - - var a = gen.NewUuid(); - var b = gen.NewUuid(); - Assert.That(a.ToUInt64(), Is.Not.EqualTo(b.ToUInt64())); - Assert.That(a, Is.Not.EqualTo(b)); - - const int N = 1 * 1000; - var uids = new HashSet(); - for (int i = 0; i < N; i++) - { - var uid = gen.NewUuid(); - if (uids.Contains(uid.ToUInt64())) Assert.Fail("Duplicate Uuid64 generated: {0}", uid); - uids.Add(uid.ToUInt64()); - } - Assert.That(uids.Count, Is.EqualTo(N)); - } - - [Test] - public void Test_Uuid64_Equality_Check() - { - var a = new Uuid64(42); - var b = new Uuid64(42); - var c = new Uuid64(40) + 2; - var d = new Uuid64(0xDEADBEEF); - - // Equals(Uuid64) - Assert.That(a.Equals(a), Is.True, "a == a"); - Assert.That(a.Equals(b), Is.True, "a == b"); - Assert.That(a.Equals(c), Is.True, "a == c"); - Assert.That(a.Equals(d), Is.False, "a != d"); - - // == Uuid64 - Assert.That(a == b, Is.True, "a == b"); - Assert.That(a == c, Is.True, "a == c"); - Assert.That(a == d, Is.False, "a != d"); - - // != Uuid64 - Assert.That(a != b, Is.False, "a == b"); - Assert.That(a != c, Is.False, "a == c"); - Assert.That(a != d, Is.True, "a != d"); - - // == numbers - Assert.That(a == 42L, Is.True, "a == 42"); - Assert.That(a == 42UL, Is.True, "a == 42"); - Assert.That(d == 42L, Is.False, "d != 42"); - Assert.That(d == 42UL, Is.False, "d != 42"); - - // != numbers - Assert.That(a != 42L, Is.False, "a == 42"); - Assert.That(a != 42UL, Is.False, "a == 42"); - Assert.That(d != 42L, Is.True, "d != 42"); - Assert.That(d != 42UL, Is.True, "d != 42"); - - // Equals(objecct) - Assert.That(a.Equals((object)a), Is.True, "a == a"); - Assert.That(a.Equals((object)b), Is.True, "a == b"); - Assert.That(a.Equals((object)c), Is.True, "a == c"); - Assert.That(a.Equals((object)d), Is.False, "a != d"); - Assert.That(a.Equals((object)42L), Is.True, "a == 42"); - Assert.That(a.Equals((object)42UL), Is.True, "a == 42"); - Assert.That(d.Equals((object)42L), Is.False, "d != 42"); - Assert.That(d.Equals((object)42UL), Is.False, "d != 42"); - - } - - [Test] - public void Test_Uuid64_Ordering() - { - var a = new Uuid64(42); - var a2 = new Uuid64(42); - var b = new Uuid64(77); - - Assert.That(a.CompareTo(a), Is.EqualTo(0)); - Assert.That(a.CompareTo(b), Is.EqualTo(-1)); - Assert.That(b.CompareTo(a), Is.EqualTo(+1)); - - Assert.That(a < b, Is.True, "a < b"); - Assert.That(a <= b, Is.True, "a <= b"); - Assert.That(a < a2, Is.False, "a < a"); - Assert.That(a <= a2, Is.True, "a <= a"); - - Assert.That(a > b, Is.False, "a > b"); - Assert.That(a >= b, Is.False, "a >= b"); - Assert.That(a > a2, Is.False, "a > a"); - Assert.That(a >= a2, Is.True, "a >= a"); - - // parsed from string - Assert.That(new Uuid64("137bcf31-0c8873a2") < new Uuid64("604bdf8a-2512b4ad"), Is.True); - Assert.That(new Uuid64("d8f17a26-82adb1a4") < new Uuid64("22abbf33-1b2c1db0"), Is.False); - Assert.That(new Uuid64("{137bcf31-0c8873a2}") > new Uuid64("{604bdf8a-2512b4ad}"), Is.False); - Assert.That(new Uuid64("{d8f17a26-82adb1a4}") > new Uuid64("{22abbf33-1b2c1db0}"), Is.True); - Assert.That(new Uuid64("2w6CTjUiXVp") < new Uuid64("DVM0UnynZ1Q"), Is.True); - Assert.That(new Uuid64("0658JY2ORSJ") > new Uuid64("FMPaNaMEUWc"), Is.False); - - // verify byte ordering - var c = new Uuid64(0x0000000100000002); - var d = new Uuid64(0x0000000200000001); - Assert.That(c.CompareTo(d), Is.EqualTo(-1)); - Assert.That(d.CompareTo(c), Is.EqualTo(+1)); - - // verify that we can sort an array of Uuid64 - var uids = new Uuid64[100]; - for (int i = 0; i < uids.Length; i++) - { - uids[i] = Uuid64.NewUuid(); - } - Assume.That(uids, Is.Not.Ordered, "This can happen with a very small probability. Please try again"); - Array.Sort(uids); - Assert.That(uids, Is.Ordered); - - // ordering should be preserved in integer or textual form - - Assert.That(uids.Select(x => x.ToUInt64()), Is.Ordered, "order should be preserved when ordering by unsigned value"); - //note: ToInt64() will not work because of negative values - Assert.That(uids.Select(x => x.ToString()), Is.Ordered.Using(StringComparer.Ordinal), "order should be preserved when ordering by text (hexa)"); - Assert.That(uids.Select(x => x.ToString("Z")), Is.Ordered.Using(StringComparer.Ordinal), "order should be preserved when ordering by text (base62)"); - //note: ToString("C") will not work for ordering because it will produce "z" > "aa", instead of expected "0z" < "aa" - } - - [Test] - public void Test_Uuid64_Arithmetic() - { - var uid = Uuid64.Empty; - - Assert.That(uid + 42L, Is.EqualTo(new Uuid64(42))); - Assert.That(uid + 42UL, Is.EqualTo(new Uuid64(42))); - uid++; - Assert.That(uid.ToInt64(), Is.EqualTo(1)); - uid++; - Assert.That(uid.ToInt64(), Is.EqualTo(2)); - uid--; - Assert.That(uid.ToInt64(), Is.EqualTo(1)); - uid--; - Assert.That(uid.ToInt64(), Is.EqualTo(0)); - - uid = Uuid64.NewUuid(); - - Assert.That(uid + 123L, Is.EqualTo(new Uuid64(uid.ToInt64() + 123))); - Assert.That(uid + 123UL, Is.EqualTo(new Uuid64(uid.ToUInt64() + 123))); - - Assert.That(uid - 123L, Is.EqualTo(new Uuid64(uid.ToInt64() - 123))); - Assert.That(uid - 123UL, Is.EqualTo(new Uuid64(uid.ToUInt64() - 123))); - } - - } - -} diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index 784ed0abe..8727ec4ce 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -51,8 +51,11 @@ False CHOP_ALWAYS CHOP_IF_LONG + True System System + False + True UseVarWhenEvident UseVarWhenEvident BE diff --git a/README.md b/README.md index 493423dcd..da950ef00 100644 --- a/README.md +++ b/README.md @@ -109,14 +109,14 @@ using (var db = await Fdb.OpenAsync()) // and 5 bytes). The values are raw slices, which means that your // application MUST KNOW that they are strings in order to decode // them. If you wan't any tool to be able to find out the type of - // your values, you can also use STuple.Pack("AAA") to create + // your values, you can also use TuPack.EncodeKey("AAA") to create // the values, at the cost of 2 extra bytes per entry. // This is always a good idea to maintain a counter of keys in our array. // The cheapest way to do that, is to reuse the subspace key itself, which // is 'in' the subspace, but not 'inside': trans.Set(list.Key, Slice.FromFixed32(3)); - // We could use STuple.Pack(3) here, but have a fixed size counter + // We could use TuPack.EncodeKey(3) here, but have a fixed size counter // makes it easy to use AtomicAdd(...) to increment (or decrement) the value // when adding or removing entries in the array. From f8344ee5301706c72746d85099d096af8b3fd583 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 11:44:30 +0200 Subject: [PATCH 071/153] Update NUnit to 3.10.1 --- FoundationDB.Tests/FoundationDB.Tests.csproj | 14 +++++++++++--- FoundationDB.Tests/packages.config | 2 +- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index a88afe825..4c9a893ea 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -1,5 +1,6 @@  + Debug @@ -14,6 +15,8 @@ ..\ true + + true @@ -55,9 +58,8 @@ False ..\packages\Newtonsoft.Json.6.0.6\lib\net45\Newtonsoft.Json.dll - - False - ..\packages\NUnit.2.6.4\lib\nunit.framework.dll + + ..\packages\NUnit.3.10.1\lib\net45\nunit.framework.dll ..\packages\protobuf-net.2.0.0.668\lib\net40\protobuf-net.dll @@ -145,6 +147,12 @@ + + + This project references NuGet package(s) that are missing on this computer. Use NuGet Package Restore to download them. For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}. + + + 99"); + Assert.That(res.Count, Is.Zero, "100 --> 99"); // |xxxxxxxxxxxxxxxxxxxxxxxxxxxxx|_____________(150->) res = await query.Skip(150).ToListAsync(); - Assert.That(res.Count, Is.EqualTo(0), "150 --> 100"); + Assert.That(res.Count, Is.Zero, "150 --> 100"); } // from the end @@ -410,11 +410,11 @@ public async Task Test_Can_Skip() // (<- -1)|<<<<<<<<<<<<<<<<<<<<<<<<<<<<<| res = await query.Reverse().Skip(100).ToListAsync(); - Assert.That(res.Count, Is.EqualTo(0), "0 <-- -1"); + Assert.That(res.Count, Is.Zero, "0 <-- -1"); // (<- -51)<<<<<<<<<<<<<|<<<<<<<<<<<<<<<<<<<<<<<<<<<<<| res = await query.Reverse().Skip(100).ToListAsync(); - Assert.That(res.Count, Is.EqualTo(0), "0 <-- -51"); + Assert.That(res.Count, Is.Zero, "0 <-- -51"); } // from both sides diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index f4e7b6dbd..458f339ce 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -45,8 +45,7 @@ public void Test_Empty_Subspace_Is_Empty() Assert.That(subspace, Is.Not.Null, "FdbSubspace.Empty should not return null"); Assert.That(KeySubspace.Empty, Is.SameAs(subspace), "FdbSubspace.Empty is a singleton"); - Assert.That(subspace.GetPrefix().Count, Is.EqualTo(0), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); - Assert.That(subspace.GetPrefix().HasValue, Is.True, "FdbSubspace.Empty.Key should be equal to Slice.Empty"); + Assert.That(subspace.GetPrefix(), Is.EqualTo(Slice.Empty), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); } diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 6e8e545b7..054db81f0 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -56,7 +56,7 @@ public async Task Test_Can_Create_And_Dispose_Transactions() Assert.That(tr.StillAlive, Is.True, "Transaction should be alive"); Assert.That(tr.Handler.IsClosed, Is.False, "Transaction handle should not be closed"); Assert.That(tr.Database, Is.SameAs(db), "Transaction should reference the parent Database"); - Assert.That(tr.Size, Is.EqualTo(0), "Estimated size should be zero"); + Assert.That(tr.Size, Is.Zero, "Estimated size should be zero"); Assert.That(tr.IsReadOnly, Is.False, "Transaction is not read-only"); Assert.That(tr.IsSnapshot, Is.False, "Transaction is not in snapshot mode by default"); @@ -1297,7 +1297,7 @@ await db.WriteAsync((tr) => tr.Set(A, Slice.FromString("aa")); tr.Set(C, Slice.FromString("cc")); await db.WriteAsync((tr2) => - { + { // have another transaction change B and D under our nose tr2.Set(B, Slice.FromString("bb")); tr2.Set(D, Slice.FromString("dd")); }, this.Cancellation); @@ -1466,9 +1466,9 @@ public async Task Test_Can_Set_Timeout_And_RetryLimit() { using (var tr = db.BeginTransaction(this.Cancellation)) { - Assert.That(tr.Timeout, Is.EqualTo(0), "Timeout (default)"); - Assert.That(tr.RetryLimit, Is.EqualTo(0), "RetryLimit (default)"); - Assert.That(tr.MaxRetryDelay, Is.EqualTo(0), "MaxRetryDelay (default)"); + Assert.That(tr.Timeout, Is.Zero, "Timeout (default)"); + Assert.That(tr.RetryLimit, Is.Zero, "RetryLimit (default)"); + Assert.That(tr.MaxRetryDelay, Is.Zero, "MaxRetryDelay (default)"); tr.Timeout = 1000; // 1 sec max tr.RetryLimit = 5; // 5 retries max @@ -1486,9 +1486,9 @@ public async Task Test_Timeout_And_RetryLimit_Inherits_Default_From_Database() { using (var db = await OpenTestDatabaseAsync()) { - Assert.That(db.DefaultTimeout, Is.EqualTo(0), "db.DefaultTimeout (default)"); - Assert.That(db.DefaultRetryLimit, Is.EqualTo(0), "db.DefaultRetryLimit (default)"); - Assert.That(db.DefaultMaxRetryDelay, Is.EqualTo(0), "db.DefaultMaxRetryDelay (default)"); + Assert.That(db.DefaultTimeout, Is.Zero, "db.DefaultTimeout (default)"); + Assert.That(db.DefaultRetryLimit, Is.Zero, "db.DefaultRetryLimit (default)"); + Assert.That(db.DefaultMaxRetryDelay, Is.Zero, "db.DefaultMaxRetryDelay (default)"); db.DefaultTimeout = 500; db.DefaultRetryLimit = 3; @@ -1535,8 +1535,8 @@ public async Task Test_Transaction_RetryLoop_Respects_DefaultRetryLimit_Value() using (var db = await OpenTestDatabaseAsync()) using (var go = new CancellationTokenSource()) { - Assert.That(db.DefaultTimeout, Is.EqualTo(0), "db.DefaultTimeout (default)"); - Assert.That(db.DefaultRetryLimit, Is.EqualTo(0), "db.DefaultRetryLimit (default)"); + Assert.That(db.DefaultTimeout, Is.Zero, "db.DefaultTimeout (default)"); + Assert.That(db.DefaultRetryLimit, Is.Zero, "db.DefaultRetryLimit (default)"); // By default, a transaction that gets reset or retried, clears the RetryLimit and Timeout settings, which needs to be reset everytime. // But if the DefaultRetryLimit and DefaultTimeout are set on the database instance, they should automatically be re-applied inside transaction loops! @@ -1585,19 +1585,19 @@ public async Task Test_Transaction_RetryLoop_Resets_RetryLimit_And_Timeout() // simulate a first error tr.RetryLimit = 10; await tr.OnErrorAsync(FdbError.PastVersion); - Assert.That(tr.RetryLimit, Is.EqualTo(0), "Retry limit should be reset"); + Assert.That(tr.RetryLimit, Is.Zero, "Retry limit should be reset"); // simulate some more errors await tr.OnErrorAsync(FdbError.PastVersion); await tr.OnErrorAsync(FdbError.PastVersion); await tr.OnErrorAsync(FdbError.PastVersion); await tr.OnErrorAsync(FdbError.PastVersion); - Assert.That(tr.RetryLimit, Is.EqualTo(0), "Retry limit should be reset"); + Assert.That(tr.RetryLimit, Is.Zero, "Retry limit should be reset"); // we still haven't failed 10 times.. tr.RetryLimit = 10; await tr.OnErrorAsync(FdbError.PastVersion); - Assert.That(tr.RetryLimit, Is.EqualTo(0), "Retry limit should be reset"); + Assert.That(tr.RetryLimit, Is.Zero, "Retry limit should be reset"); // we already have failed 6 times, so this one should abort tr.RetryLimit = 2; // value is too low diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index 8a0497fe1..40c18dae7 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -82,7 +82,7 @@ public async Task Test_Transactionals_Rethrow_Regular_Exceptions() // ReadAsync should return a failed Task, and not bubble up the exception. var task = db.ReadAsync((tr) => { - Assert.That(called, Is.EqualTo(0), "ReadAsync should not retry on regular exceptions"); + Assert.That(called, Is.Zero, "ReadAsync should not retry on regular exceptions"); ++called; throw new InvalidOperationException("Boom"); }, this.Cancellation); diff --git a/FoundationDB.Tests/Utils/SliceComparerFacts.cs b/FoundationDB.Tests/Utils/SliceComparerFacts.cs index 9ee9084e8..9e4daec6e 100644 --- a/FoundationDB.Tests/Utils/SliceComparerFacts.cs +++ b/FoundationDB.Tests/Utils/SliceComparerFacts.cs @@ -91,13 +91,13 @@ public void Test_SliceComparer_Compare() var cmp = Slice.Comparer.Default; Assert.That(cmp, Is.Not.Null); - Assert.That(cmp.Compare(Slice.Nil, Slice.Nil), Is.EqualTo(0)); - Assert.That(cmp.Compare(Slice.Empty, Slice.Empty), Is.EqualTo(0)); - Assert.That(cmp.Compare(Slice.FromByte(42), Slice.FromByte(42)), Is.EqualTo(0)); + Assert.That(cmp.Compare(Slice.Nil, Slice.Nil), Is.Zero); + Assert.That(cmp.Compare(Slice.Empty, Slice.Empty), Is.Zero); + Assert.That(cmp.Compare(Slice.FromByte(42), Slice.FromByte(42)), Is.Zero); //REVIEW: Inconsistency: compare(nil, empty) == 0, but Equals(nil, empty) == false - Assert.That(cmp.Compare(Slice.Nil, Slice.Empty), Is.EqualTo(0), "Nil and Empty are considered similar regarding ordering"); - Assert.That(cmp.Compare(Slice.Empty, Slice.Nil), Is.EqualTo(0), "Nil and Empty are considered similar regarding ordering"); + Assert.That(cmp.Compare(Slice.Nil, Slice.Empty), Is.Zero, "Nil and Empty are considered similar regarding ordering"); + Assert.That(cmp.Compare(Slice.Empty, Slice.Nil), Is.Zero, "Nil and Empty are considered similar regarding ordering"); Assert.That(cmp.Compare(Slice.FromByte(42), Slice.FromByte(77)), Is.LessThan(0)); Assert.That(cmp.Compare(Slice.FromByte(42), Slice.FromByte(21)), Is.GreaterThan(0)); diff --git a/FoundationDB.Tests/Utils/SliceFacts.cs b/FoundationDB.Tests/Utils/SliceFacts.cs index e15eaf731..ee1cb2078 100644 --- a/FoundationDB.Tests/Utils/SliceFacts.cs +++ b/FoundationDB.Tests/Utils/SliceFacts.cs @@ -217,8 +217,8 @@ public void Test_Slice_Pseudo_Random() Assert.That(Slice.Random(rng, 0), Is.EqualTo(Slice.Empty)); // ReSharper disable once AssignNullToNotNullAttribute - Assert.That(() => Slice.Random(default(Random), 16), Throws.InstanceOf()); - Assert.That(() => Slice.Random(rng, -1), Throws.InstanceOf()); + Assert.That(() => Slice.Random(default(Random), 16), Throws.ArgumentNullException); + Assert.That(() => Slice.Random(rng, -1), Throws.InstanceOf()); } [Test] @@ -247,7 +247,7 @@ public void Test_Slice_Cryptographic_Random() Assert.That(Slice.Random(rng, 0), Is.EqualTo(Slice.Empty)); // ReSharper disable once AssignNullToNotNullAttribute - Assert.That(() => Slice.Random(default(System.Security.Cryptography.RandomNumberGenerator), 16), Throws.InstanceOf()); + Assert.That(() => Slice.Random(default(System.Security.Cryptography.RandomNumberGenerator), 16), Throws.ArgumentNullException); Assert.That(() => Slice.Random(rng, -1), Throws.InstanceOf()); } @@ -2039,7 +2039,7 @@ public void Test_Slice_FromStream() Assert.That(slice.ToUnicode(), Is.EqualTo(UNICODE_TEXT)); // ReSharper disable once AssignNullToNotNullAttribute - Assert.That(() => Slice.FromStream(null), Throws.InstanceOf(), "Should throw if null"); + Assert.That(() => Slice.FromStream(null), Throws.ArgumentNullException, "Should throw if null"); Assert.That(Slice.FromStream(Stream.Null), Is.EqualTo(Slice.Nil), "Stream.Null should return Slice.Nil"); using(var ms = new MemoryStream()) @@ -2266,8 +2266,8 @@ public void Test_Slice_JoinBytes() Assert.That(joined.Length, Is.EqualTo(0)); // ReSharper disable AssignNullToNotNullAttribute - Assert.That(() => Slice.JoinBytes(sep, default(Slice[]), 0, 0), Throws.InstanceOf()); - Assert.That(() => Slice.JoinBytes(sep, default(IEnumerable)), Throws.InstanceOf()); + Assert.That(() => Slice.JoinBytes(sep, default(Slice[]), 0, 0), Throws.ArgumentNullException); + Assert.That(() => Slice.JoinBytes(sep, default(IEnumerable)), Throws.ArgumentNullException); // ReSharper restore AssignNullToNotNullAttribute Assert.That(() => Slice.JoinBytes(sep, tokens, 0, 4), Throws.InstanceOf()); diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index a639f6c28..162c6ab58 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -48,7 +48,7 @@ public class TupleFacts : FdbTest public void Test_Tuple_0() { var t0 = STuple.Create(); - Assert.That(t0.Count, Is.EqualTo(0)); + Assert.That(t0.Count, Is.Zero); Assert.That(t0.ToArray(), Is.EqualTo(new object[0])); Assert.That(t0.ToString(), Is.EqualTo("()")); Assert.That(t0, Is.InstanceOf()); @@ -1046,9 +1046,9 @@ void Verify(ITuple t) // ReSharper disable ExpressionIsAlwaysNull ITuple none = null; - Assert.That(() => none.OfSize(0), Throws.InstanceOf()); - Assert.That(() => none.OfSizeAtLeast(0), Throws.InstanceOf()); - Assert.That(() => none.OfSizeAtMost(0), Throws.InstanceOf()); + Assert.That(() => none.OfSize(0), Throws.ArgumentNullException); + Assert.That(() => none.OfSizeAtLeast(0), Throws.ArgumentNullException); + Assert.That(() => none.OfSizeAtMost(0), Throws.ArgumentNullException); // ReSharper restore ExpressionIsAlwaysNull } @@ -1745,7 +1745,7 @@ public void Test_Create_Appender_Formatter() Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", 42)), Is.EqualTo(42)); Assert.That(fmtr.FromTuple(STuple.Create("hello", "world", -1)), Is.EqualTo(-1)); - Assert.That(() => fmtr.FromTuple(null), Throws.InstanceOf()); + Assert.That(() => fmtr.FromTuple(null), Throws.ArgumentNullException); Assert.That(() => fmtr.FromTuple(STuple.Empty), Throws.InstanceOf()); Assert.That(() => fmtr.FromTuple(STuple.Create("hello", "world", 42, 77)), Throws.InstanceOf(), "Too many values"); Assert.That(() => fmtr.FromTuple(STuple.Create("hello_world", 42)), Throws.InstanceOf(), "not enough values"); diff --git a/FoundationDB.Tests/Utils/Uuid64Facts.cs b/FoundationDB.Tests/Utils/Uuid64Facts.cs index fa8eaff55..d7ae0c242 100644 --- a/FoundationDB.Tests/Utils/Uuid64Facts.cs +++ b/FoundationDB.Tests/Utils/Uuid64Facts.cs @@ -122,7 +122,7 @@ public void Test_Uuid64_Parse_Hexa16() Assert.That(Uuid64.Parse("{00000000-deadbeef}").ToUInt64(), Is.EqualTo(0xDEADBEEFUL)); // errors - Assert.That(() => Uuid64.Parse(default(string)), Throws.InstanceOf()); + Assert.That(() => Uuid64.Parse(default(string)), Throws.ArgumentNullException); Assert.That(() => Uuid64.Parse("hello"), Throws.InstanceOf()); Assert.That(() => Uuid64.Parse("12345678-9ABCDEFG"), Throws.InstanceOf(), "Invalid hexa character 'G'"); Assert.That(() => Uuid64.Parse("00000000-0000000 "), Throws.InstanceOf(), "Two short + extra space"); @@ -302,7 +302,7 @@ public void Test_Uuid64_Parse_Base62() Assert.That(() => Uuid64.FromBase62("LygHa16AHYG"), Throws.InstanceOf(), "ulong.MaxValue + 1 => OVERFLOW"); // invalid length - Assert.That(() => Uuid64.FromBase62(default(string)), Throws.InstanceOf()); + Assert.That(() => Uuid64.FromBase62(default(string)), Throws.ArgumentNullException); Assert.That(() => Uuid64.FromBase62("100000000000"), Throws.InstanceOf(), "62^11 => TOO BIG"); } @@ -639,12 +639,12 @@ public void Test_UUid64_WriteTo() // errors - Assert.That(() => original.WriteTo(Slice.Empty), Throws.InstanceOf()); //, "Target buffer is empty"); - Assert.That(() => original.WriteTo(null, 8), Throws.InstanceOf()); //, "Target buffer is null"); - Assert.That(() => original.WriteTo(null, 0), Throws.InstanceOf()); //, "Target buffer is null"); + Assert.That(() => original.WriteTo(Slice.Empty), Throws.InstanceOf(), "Target buffer is empty"); + Assert.That(() => original.WriteTo(null, 8), Throws.InstanceOf(), "Target buffer is null"); + Assert.That(() => original.WriteTo(null, 0), Throws.InstanceOf(), "Target buffer is null"); scratch = Slice.Repeat(0xAA, 16); - Assert.That(() => original.WriteTo(scratch.Substring(0, 7)), Throws.InstanceOf()); //, "Target buffer is too small"); + Assert.That(() => original.WriteTo(scratch.Substring(0, 7)), Throws.InstanceOf(), "Target buffer is too small"); Assert.That(scratch.ToString("X"), Is.EqualTo("AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA AA"), "Buffer should not have been overwritten!"); } From 338d171ff1b20d09f6e1fe3911b857cff82f73d0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 11:47:51 +0200 Subject: [PATCH 073/153] Update some test dependencies to the latest versions - Json.NET => 11.0.2 - protobuf-net => 2.3.7 - MathNet.Numerics => 3.20.2 --- FoundationDB.Tests/FoundationDB.Tests.csproj | 17 +++++++++-------- FoundationDB.Tests/packages.config | 6 +++--- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index 4c9a893ea..0c78ba023 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -50,23 +50,24 @@ ..\Common\foundationdb-net-client.snk - - False - ..\packages\MathNet.Numerics.Signed.3.6.0\lib\net40\MathNet.Numerics.dll + + ..\packages\MathNet.Numerics.Signed.3.20.2\lib\net40\MathNet.Numerics.dll - - False - ..\packages\Newtonsoft.Json.6.0.6\lib\net45\Newtonsoft.Json.dll + + ..\packages\Newtonsoft.Json.11.0.2\lib\net45\Newtonsoft.Json.dll ..\packages\NUnit.3.10.1\lib\net45\nunit.framework.dll - - ..\packages\protobuf-net.2.0.0.668\lib\net40\protobuf-net.dll + + ..\packages\protobuf-net.2.3.7\lib\net40\protobuf-net.dll + + + diff --git a/FoundationDB.Tests/packages.config b/FoundationDB.Tests/packages.config index 7f8cdf999..147974643 100644 --- a/FoundationDB.Tests/packages.config +++ b/FoundationDB.Tests/packages.config @@ -1,7 +1,7 @@  - - + + - + \ No newline at end of file From 5597c11fbf886e6d2683344d04984a44f50f2a08 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 12:24:11 +0200 Subject: [PATCH 074/153] Bumped version number to v5.1.0-alpha1 - Q: Wait, what, 5.1 already? What happened to 1.0, 2.0 ??? - A: v1.0 was almost released but not quite, and the API has changed a lot since then, so I'm synching the version number with the database version to simplify things. --- Common/VersionInfo.cs | 8 ++++---- build/FoundationDB.Client.nuspec | 6 +++--- build/FoundationDB.Layers.Common.nuspec | 6 +++--- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/Common/VersionInfo.cs b/Common/VersionInfo.cs index 9a2fceed4..8c8f4ef16 100644 --- a/Common/VersionInfo.cs +++ b/Common/VersionInfo.cs @@ -30,10 +30,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY [assembly: AssemblyCompany("Doxense")] [assembly: AssemblyProduct("FoundationDB.Client")] -[assembly: AssemblyCopyright("Copyright Doxense SAS 2013-2015")] +[assembly: AssemblyCopyright("Copyright Doxense SAS 2013-2018")] [assembly: AssemblyTrademark("")] [assembly: AssemblyCulture("")] -[assembly: AssemblyVersion("0.9.9.0")] //note: Only change this when doing breaking API changes! -[assembly: AssemblyFileVersion("0.9.8.0")] //note: change this to "x.y.revision.build_step" in a build step of your favorite C.I. build server -[assembly: AssemblyInformationalVersion("0.9.9-pre")] //note: this is the version seen by NuGet, add "-alpha"/"-beta"/"-rc1" at the end to create pre-release packages +[assembly: AssemblyVersion("5.1.0.0")] //note: Only change this when doing breaking API changes! +[assembly: AssemblyFileVersion("5.1.0.0")] //note: change this to "x.y.revision.build_step" in a build step of your favorite C.I. build server +[assembly: AssemblyInformationalVersion("0.5.1-alpha1")] //note: this is the version seen by NuGet, add "-alpha"/"-beta"/"-rc1" at the end to create pre-release packages diff --git a/build/FoundationDB.Client.nuspec b/build/FoundationDB.Client.nuspec index 48a87cfff..63a6421f5 100644 --- a/build/FoundationDB.Client.nuspec +++ b/build/FoundationDB.Client.nuspec @@ -2,7 +2,7 @@ FoundationDB.Client - 0.9.9-pre + 5.1.0-alpha1 FoundationDB Client Doxense Doxense @@ -12,9 +12,9 @@ false .NET Binding for FoundationDB This is a pre-release of the .NET Binding, the public API is still subject to changes. - Copyright 2013-2015 Doxense SAS + Copyright 2013-2018 Doxense SAS en-US - foundationdb nosql + foundationdb fdb nosql diff --git a/build/FoundationDB.Layers.Common.nuspec b/build/FoundationDB.Layers.Common.nuspec index fd3f73888..699fcd55e 100644 --- a/build/FoundationDB.Layers.Common.nuspec +++ b/build/FoundationDB.Layers.Common.nuspec @@ -2,7 +2,7 @@ FoundationDB.Layers.Common - 0.9.9-pre + 5.1.0-alpha1 FoundationDB Common Layers Doxense Doxense @@ -12,11 +12,11 @@ false Common Layers for the FoundationDB .NET Binding This is a pre-release of the .NET Binding, the public API is still subject to changes. - Copyright 2013-2015 Doxense SAS + Copyright 2013-2018 Doxense SAS en-US foundationdb nosql layers - + From dc4420a8ed80fcebcb59167d7671259b661e19fa Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 13:08:04 +0200 Subject: [PATCH 075/153] cancellationToken => ct --- FoundationDB.Client/Async/AsyncTransformQueue.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Async/AsyncTransformQueue.cs index 38c57a174..de67cadbd 100644 --- a/FoundationDB.Client/Async/AsyncTransformQueue.cs +++ b/FoundationDB.Client/Async/AsyncTransformQueue.cs @@ -213,18 +213,18 @@ public void OnError(ExceptionDispatchInfo error) #region IAsyncBatchTarget... - public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken cancellationToken) + public async Task OnNextBatchAsync([NotNull] TInput[] batch, CancellationToken ct) { Contract.NotNull(batch, nameof(batch)); if (batch.Length == 0) return; - if (cancellationToken.IsCancellationRequested) cancellationToken.ThrowIfCancellationRequested(); + if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); //TODO: optimized version ! foreach (var item in batch) { - await OnNextAsync(item, cancellationToken).ConfigureAwait(false); + await OnNextAsync(item, ct).ConfigureAwait(false); } } From bbde5616098d1bbf755f2e1bfa2243d42c95483c Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 13:09:20 +0200 Subject: [PATCH 076/153] Replace all occurrences of Assert.Throws<..>(async lambda) into Assert.That(..., Throws.InstanceOf<..>()) following update to NUnit 3.x --- FoundationDB.Tests/ClusterFacts.cs | 2 +- FoundationDB.Tests/DatabaseFacts.cs | 3 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 55 ++++++++++--------- .../Linq/AsyncEnumerableFacts.cs | 6 +- FoundationDB.Tests/RangeQueryFacts.cs | 10 ++-- FoundationDB.Tests/TransactionFacts.cs | 18 +++--- FoundationDB.Tests/TransactionalFacts.cs | 4 +- 7 files changed, 52 insertions(+), 46 deletions(-) diff --git a/FoundationDB.Tests/ClusterFacts.cs b/FoundationDB.Tests/ClusterFacts.cs index 4e5d47715..2ab26ed07 100644 --- a/FoundationDB.Tests/ClusterFacts.cs +++ b/FoundationDB.Tests/ClusterFacts.cs @@ -55,7 +55,7 @@ public void Test_Connecting_To_Cluster_With_Cancelled_Token_Should_Fail() { cts.Cancel(); - Assert.Throws(() => Fdb.CreateClusterAsync(cts.Token).GetAwaiter().GetResult()); + Assert.That(() => Fdb.CreateClusterAsync(cts.Token).GetAwaiter().GetResult(), Throws.InstanceOf()); } } diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index 752edb950..b8c9ac5b8 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +// ReSharper disable AccessToDisposedClosure namespace FoundationDB.Client.Tests { using System; @@ -77,7 +78,7 @@ public async Task Test_Open_Database_With_Cancelled_Token_Should_Fail() using (var cluster = await Fdb.CreateClusterAsync(cts.Token)) { cts.Cancel(); - Assert.Throws(() => cluster.OpenDatabaseAsync("DB", KeySubspace.Empty, false, cts.Token).GetAwaiter().GetResult()); + Assert.That(async () => await cluster.OpenDatabaseAsync("DB", KeySubspace.Empty, false, cts.Token), Throws.InstanceOf()); } } } diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 0fad3b902..22878c887 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +// ReSharper disable AssignNullToNotNullAttribute #undef ENABLE_LOGGING namespace FoundationDB.Layers.Directories @@ -217,7 +218,7 @@ public async Task Test_CreateOrOpen_With_Layer() Assert.That(foo2.GetPrefix(), Is.EqualTo(foo.GetPrefix()), "Second call to CreateOrOpen should return the same subspace"); // opening it with wrong layer id should fail - Assert.Throws(async () => await directory.OpenAsync(logged, new[] { "Foo" }, Slice.FromString("OtherLayer"), this.Cancellation), "Opening with invalid layer id should fail"); + Assert.That(async () => await directory.OpenAsync(logged, new[] { "Foo" }, Slice.FromString("OtherLayer"), this.Cancellation), Throws.InstanceOf(), "Opening with invalid layer id should fail"); // opening without specifying a layer should disable the layer check var foo3 = await directory.OpenAsync(logged, "Foo", layer: Slice.Nil, ct: this.Cancellation); @@ -228,7 +229,7 @@ public async Task Test_CreateOrOpen_With_Layer() Assert.DoesNotThrow(() => foo3.CheckLayer(Slice.FromString("AcmeLayer")), "CheckLayer should not throw if the layer id is correct"); // CheckLayer with the incorrect value should fail - Assert.Throws(() => foo3.CheckLayer(Slice.FromString("OtherLayer")), "CheckLayer should throw if the layer id is not correct"); + Assert.That(() => foo3.CheckLayer(Slice.FromString("OtherLayer")), Throws.InstanceOf(), "CheckLayer should throw if the layer id is not correct"); // CheckLayer with empty string should do nothing foo3.CheckLayer(Slice.Empty); @@ -426,7 +427,7 @@ public async Task Test_Move_Folder() Assert.That(renamed.GetPrefix(), Is.EqualTo(original.GetPrefix())); // opening the old path should fail - Assert.Throws(async () => await directory.OpenAsync(logged, "Foo", this.Cancellation)); + Assert.That(async () => await directory.OpenAsync(logged, "Foo", this.Cancellation), Throws.InstanceOf()); // opening the new path should succeed var folder = await directory.OpenAsync(logged, "Bar", this.Cancellation); @@ -436,7 +437,7 @@ public async Task Test_Move_Folder() Assert.That(folder.GetPrefix(), Is.EqualTo(renamed.GetPrefix())); // moving the folder under itself should fail - Assert.Throws(async () => await folder.MoveToAsync(logged, new[] { "Bar", "Baz" }, this.Cancellation)); + Assert.That(async () => await folder.MoveToAsync(logged, new[] { "Bar", "Baz" }, this.Cancellation), Throws.InstanceOf()); #if ENABLE_LOGGING foreach (var log in list) { @@ -478,7 +479,7 @@ public async Task Test_Remove_Folder() //TODO: call ExistsAsync(...) once it is implemented! // Removing it a second time should fail - Assert.Throws(async () => await directory.RemoveAsync(logged, path, this.Cancellation), "Removing a non-existent directory should fail"); + Assert.That(async () => await directory.RemoveAsync(logged, path, this.Cancellation), Throws.InstanceOf(), "Removing a non-existent directory should fail"); // TryRemoveAsync @@ -495,7 +496,7 @@ public async Task Test_Remove_Folder() // Corner Cases // removing the root folder is not allowed (too dangerous) - Assert.Throws(async () => await directory.RemoveAsync(logged, new string[0], this.Cancellation), "Attempting to remove the root directory should fail"); + Assert.That(async () => await directory.RemoveAsync(logged, new string[0], this.Cancellation), Throws.InstanceOf(), "Attempting to remove the root directory should fail"); #if ENABLE_LOGGING foreach (var log in list) @@ -545,7 +546,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() Assert.That(folder3, Is.Not.Null); // opening the directory with the old layer should fail - Assert.Throws(async () => await directory.OpenAsync(logged, "Test", layer: Slice.FromString("foo"), ct: this.Cancellation)); + Assert.That(async () => await directory.OpenAsync(logged, "Test", layer: Slice.FromString("foo"), ct: this.Cancellation), Throws.InstanceOf()); #if ENABLE_LOGGING foreach (var log in list) @@ -798,36 +799,36 @@ public async Task Test_Directory_Methods_Should_Fail_With_Empty_Paths() var directory = FdbDirectoryLayer.Create(location); // CreateOrOpen - Assert.Throws(async () => await directory.CreateOrOpenAsync(db, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.CreateOrOpenAsync(db, new string[0], this.Cancellation)); - Assert.Throws(async () => await directory.CreateOrOpenAsync(db, default(string), this.Cancellation)); + Assert.That(async () => await directory.CreateOrOpenAsync(db, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.CreateOrOpenAsync(db, new string[0], this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.CreateOrOpenAsync(db, default(string), this.Cancellation), Throws.InstanceOf()); // Create - Assert.Throws(async () => await directory.CreateAsync(db, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.CreateAsync(db, new string[0], this.Cancellation)); - Assert.Throws(async () => await directory.CreateAsync(db, default(string), this.Cancellation)); + Assert.That(async () => await directory.CreateAsync(db, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.CreateAsync(db, new string[0], this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.CreateAsync(db, default(string), this.Cancellation), Throws.InstanceOf()); // Open - Assert.Throws(async () => await directory.OpenAsync(db, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.OpenAsync(db, new string[0], this.Cancellation)); - Assert.Throws(async () => await directory.OpenAsync(db, default(string), this.Cancellation)); + Assert.That(async () => await directory.OpenAsync(db, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.OpenAsync(db, new string[0], this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.OpenAsync(db, default(string), this.Cancellation), Throws.InstanceOf()); // Move - Assert.Throws(async () => await directory.MoveAsync(db, default(string[]), new[] { "foo" }, this.Cancellation)); - Assert.Throws(async () => await directory.MoveAsync(db, new[] { "foo" }, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.MoveAsync(db, new string[0], new[] { "foo" }, this.Cancellation)); - Assert.Throws(async () => await directory.MoveAsync(db, new[] { "foo" }, new string[0], this.Cancellation)); + Assert.That(async () => await directory.MoveAsync(db, default(string[]), new[] { "foo" }, this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.MoveAsync(db, new[] { "foo" }, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.MoveAsync(db, new string[0], new[] { "foo" }, this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.MoveAsync(db, new[] { "foo" }, new string[0], this.Cancellation), Throws.InstanceOf()); // Remove - Assert.Throws(async () => await directory.RemoveAsync(db, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.RemoveAsync(db, new string[0], this.Cancellation)); - Assert.Throws(async () => await directory.RemoveAsync(db, new string[] { "Foo", " ", "Bar" }, this.Cancellation)); - Assert.Throws(async () => await directory.RemoveAsync(db, default(string), this.Cancellation)); + Assert.That(async () => await directory.RemoveAsync(db, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.RemoveAsync(db, new string[0], this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.RemoveAsync(db, new string[] { "Foo", " ", "Bar" }, this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.RemoveAsync(db, default(string), this.Cancellation), Throws.InstanceOf()); // List - Assert.Throws(async () => await directory.ListAsync(db, default(string[]), this.Cancellation)); - Assert.Throws(async () => await directory.ListAsync(db, new string[] { "Foo", "", "Bar" }, this.Cancellation)); - Assert.Throws(async () => await directory.ListAsync(db, default(string), this.Cancellation)); + Assert.That(async () => await directory.ListAsync(db, default(string[]), this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.ListAsync(db, new string[] { "Foo", "", "Bar" }, this.Cancellation), Throws.InstanceOf()); + Assert.That(async () => await directory.ListAsync(db, default(string), this.Cancellation), Throws.InstanceOf()); } } diff --git a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs index 5730b46b0..d52f4322a 100644 --- a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +// ReSharper disable AccessToDisposedClosure namespace FoundationDB.Linq.Tests { using System; @@ -1126,14 +1127,13 @@ public async Task Test_Exceptions_Are_Propagated_To_Caller() Assert.That(res, Is.True); // second move next should fail - var x = Assert.Throws(async () => await iterator.MoveNextAsync(), "Should have failed"); - Assert.That(x.Message, Is.EqualTo("KABOOM")); + Assert.That(async () => await iterator.MoveNextAsync(), Throws.InstanceOf().With.Message.EqualTo("KABOOM"), "Should have failed"); // accessing current should rethrow the exception Assert.That(() => iterator.Current, Throws.InstanceOf()); // another attempt at MoveNext should fail immediately but with a different error - Assert.Throws(async () => await iterator.MoveNextAsync()); + Assert.That(async () => await iterator.MoveNextAsync(), Throws.InstanceOf()); } } diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 5ef38cc16..3dd348d3a 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -169,10 +169,10 @@ await db.WriteAsync((tr) => Assert.That(res.Value, Is.EqualTo(Slice.FromInt32(9))); // should fail because there is more than one - Assert.Throws(async () => await query.SingleOrDefaultAsync(), "SingleOrDefaultAsync should throw if the range returns more than 1 result"); + Assert.That(async () => await query.SingleOrDefaultAsync(), Throws.InstanceOf(), "SingleOrDefaultAsync should throw if the range returns more than 1 result"); // should fail because there is more than one - Assert.Throws(async () => await query.SingleAsync(), "SingleAsync should throw if the range returns more than 1 result"); + Assert.That(async () => await query.SingleAsync(), Throws.InstanceOf(), "SingleAsync should throw if the range returns more than 1 result"); } // B: exactly one item @@ -222,7 +222,7 @@ await db.WriteAsync((tr) => Assert.That(res.Value, Is.EqualTo(Slice.Nil)); // should return the first one - Assert.Throws(async () => await query.FirstAsync(), "FirstAsync should throw if the range returns nothing"); + Assert.That(async () => await query.FirstAsync(), Throws.InstanceOf(), "FirstAsync should throw if the range returns nothing"); // should return the last one res = await query.LastOrDefaultAsync(); @@ -230,7 +230,7 @@ await db.WriteAsync((tr) => Assert.That(res.Value, Is.EqualTo(Slice.Nil)); // should return the last one - Assert.Throws(async () => await query.LastAsync(), "LastAsync should throw if the range returns nothing"); + Assert.That(async () => await query.LastAsync(), Throws.InstanceOf(), "LastAsync should throw if the range returns nothing"); // should fail because there is more than one res = await query.SingleOrDefaultAsync(); @@ -238,7 +238,7 @@ await db.WriteAsync((tr) => Assert.That(res.Value, Is.EqualTo(Slice.Nil)); // should fail because there is none - Assert.Throws(async () => await query.SingleAsync(), "SingleAsync should throw if the range returns nothing"); + Assert.That(async () => await query.SingleAsync(), Throws.InstanceOf(), "SingleAsync should throw if the range returns nothing"); } // A: with a size limit diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 054db81f0..90d3ef9d0 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -312,7 +312,7 @@ public async Task Test_Cancelling_Token_During_Commit_Should_Abort_Task() Assume.That(t.IsCompleted, Is.False, "Commit task already completed before having a chance to cancel"); cts.Cancel(); - Assert.Throws(async () => await t, "Cancelling a token passed to CommitAsync that is still pending should cancel the task"); + Assert.That(async () => await t, Throws.InstanceOf(), "Cancelling a token passed to CommitAsync that is still pending should cancel the task"); } } } @@ -1864,7 +1864,7 @@ public async Task Test_Can_Get_Boundary_Keys() // the node id seems to be at offset 12 - //Console.WriteLine("- " + key.Value.Substring(0, 12).ToAsciiOrHexaString() + " : " + String.Join(", ", ids) + " = " + key.Key); + //Log("- " + key.Value.Substring(0, 12).ToAsciiOrHexaString() + " : " + String.Join(", ", ids) + " = " + key.Key); } Log(); Log("Distinct nodes: {0}", distinctNodes.Count); @@ -1908,7 +1908,7 @@ public async Task Test_Simple_Read_Transaction() //tr.Set(location.Concat(Slice.FromString("C")), Slice.Empty); //var slice = await tr.GetRange(location.Concat(Slice.FromString("A")), location.Concat(Slice.FromString("Z"))).FirstOrDefaultAsync(); - //Console.WriteLine(slice); + //Log(slice); //tr.AddReadConflictKey(location.Concat(Slice.FromString("READ_CONFLICT"))); //tr.AddWriteConflictKey(location.Concat(Slice.FromString("WRITE_CONFLICT"))); @@ -1928,7 +1928,11 @@ public async Task Test_Simple_Read_Transaction() [Test, Category("LongRunning")] public async Task Test_BadPractice_Future_Fuzzer() { - const int DURATION_SEC = 30; +#if DEBUG + const int DURATION_SEC = 5; +#else + const int DURATION_SEC = 20; +#endif const int R = 100; using (var db = await OpenTestDatabaseAsync()) @@ -1991,7 +1995,7 @@ await db.WriteAsync((tr) => Console.Write('C'); var tr = db.BeginTransaction(FdbTransactionMode.ReadOnly, this.Cancellation); m_alive.Add(tr); - await tr.GetReadVersionAsync(); + _ = await tr.GetReadVersionAsync(); break; } @@ -2007,7 +2011,7 @@ await db.WriteAsync((tr) => int x = rnd.Next(R); try { - var res = await tr.GetAsync(location.Keys.Encode(x)); + _ = await tr.GetAsync(location.Keys.Encode(x)); } catch (FdbException) { @@ -2025,7 +2029,7 @@ await db.WriteAsync((tr) => var tr = m_alive[p]; int x = rnd.Next(R); - var t = tr.GetAsync(location.Keys.Encode(x)).ContinueWith((_) => Console.Write('!'), TaskContinuationOptions.NotOnRanToCompletion); + _ = tr.GetAsync(location.Keys.Encode(x)).ContinueWith((_) => Console.Write('!'), TaskContinuationOptions.NotOnRanToCompletion); // => t is not stored break; } diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index 40c18dae7..72e8ccfdc 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -88,7 +88,7 @@ public async Task Test_Transactionals_Rethrow_Regular_Exceptions() }, this.Cancellation); Assert.That(task, Is.Not.Null); // the exception should be unwrapped (ie: we should not see an AggregateException, but the actual exception) - Assert.Throws(async () => await task, "ReadAsync should rethrow any regular exceptions"); + Assert.That(async () => await task, Throws.InstanceOf(), "ReadAsync should rethrow any regular exceptions"); } } @@ -253,7 +253,7 @@ public async Task Test_Transactionals_ReadOnly_Should_Deny_Write_Attempts() return Task.FromResult(123); }, this.Cancellation); - Assert.Throws(async () => await t, "Forcing writes on a read-only transaction should fail"); + Assert.That(async () => await t, Throws.InstanceOf(), "Forcing writes on a read-only transaction should fail"); } } From b57a86c51d38dc49c36de91881d75d2f32aa07b1 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 13:34:12 +0200 Subject: [PATCH 077/153] UnitTests: Use TextContext.Progress by logging by default instead of Console.WriteLine --- FoundationDB.Tests/ExoticTestCases.cs | 24 +- .../Indexing/CompressedBitmapsFacts.cs | 155 ++++---- FoundationDB.Tests/FdbTest.cs | 14 +- .../Filters/LoggingFilterFacts.cs | 8 +- FoundationDB.Tests/Layers/CounterFacts.cs | 8 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 64 ++-- FoundationDB.Tests/Layers/IndexingFacts.cs | 6 +- FoundationDB.Tests/Layers/QueuesFacts.cs | 38 +- FoundationDB.Tests/Layers/RankedSetFacts.cs | 6 +- FoundationDB.Tests/Layers/VectorFacts.cs | 86 ++--- .../Linq/AsyncEnumerableFacts.cs | 78 ++--- .../Linq/AsyncQueryableFacts.cs | 22 +- .../Linq/FdbQueryExpressionFacts.cs | 331 +++++++++--------- FoundationDB.Tests/RangeQueryFacts.cs | 12 +- FoundationDB.Tests/TestHelpers.cs | 8 +- FoundationDB.Tests/TransactionFacts.cs | 22 +- FoundationDB.Tests/TransactionalFacts.cs | 18 +- FoundationDB.Tests/Utils/TupleFacts.cs | 23 +- 18 files changed, 465 insertions(+), 458 deletions(-) diff --git a/FoundationDB.Tests/ExoticTestCases.cs b/FoundationDB.Tests/ExoticTestCases.cs index 9ea5fcc41..2a147c3b5 100644 --- a/FoundationDB.Tests/ExoticTestCases.cs +++ b/FoundationDB.Tests/ExoticTestCases.cs @@ -223,7 +223,7 @@ public async void Test_Case_7() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -303,7 +303,7 @@ public async void Test_Case_8() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -321,7 +321,7 @@ await db.WriteAsync((tr) => using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) { var res = await tr.GetAsync(subspace.Keys.Encode("K" + i.ToString("D4"))); - Console.WriteLine(res); + Log(res); } } } @@ -333,7 +333,7 @@ public async void Test_Case_9() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -374,7 +374,7 @@ public async void Test_Case_10() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -411,7 +411,7 @@ public async void Test_Case_11() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -465,7 +465,7 @@ public async void Test_Case_12() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -500,7 +500,7 @@ public async void Test_Case_13() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -543,7 +543,7 @@ public async void Test_Case_14() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -582,7 +582,7 @@ public async void Test_Case_15() { using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -619,7 +619,7 @@ public async void Test_Case_16() using (var zedb = await OpenTestDatabaseAsync()) { - var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var db = FoundationDB.Filters.Logging.FdbLoggingExtensions.Logged(zedb, (tr) => Log(tr.Log.GetTimingsReport(true))); { var subspace = db.GlobalSpace; @@ -751,4 +751,4 @@ public async void Test_Case_17() } } -} \ No newline at end of file +} diff --git a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs index 779b8ab3e..556783181 100644 --- a/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs +++ b/FoundationDB.Tests/Experimental/Indexing/CompressedBitmapsFacts.cs @@ -36,12 +36,13 @@ namespace FoundationDB.Layers.Experimental.Indexing.Tests using System.Linq; using System.Text; using Doxense.Collections.Tuples; + using FoundationDB.Client.Tests; using MathNet.Numerics.Distributions; using NUnit.Framework; [TestFixture] [Category("LongRunning")] - public class CompressedBitmapsFacts + public class CompressedBitmapsFacts : FdbTest { [Test] @@ -71,25 +72,25 @@ private static void Verify(CompressedBitmapBuilder builder, SuperSlowUncompresse { var bmpBuilder = builder.ToBitmap(); var bmpWitness = witness.ToBitmap(); - Console.WriteLine("> B: {0,12} ({1,3}) {2}", bmpBuilder.Bounds, bmpBuilder.CountBits(), bmpBuilder.ToSlice().ToHexaString()); - Console.WriteLine("> W: {0,12} ({1,3}) {2}", bmpWitness.Bounds, bmpWitness.CountBits(), bmpWitness.ToSlice().ToHexaString()); + Log("> B: {0,12} ({1,3}) {2}", bmpBuilder.Bounds, bmpBuilder.CountBits(), bmpBuilder.ToSlice().ToHexaString()); + Log("> W: {0,12} ({1,3}) {2}", bmpWitness.Bounds, bmpWitness.CountBits(), bmpWitness.ToSlice().ToHexaString()); var rawBuilder = builder.ToBooleanArray(); var rawWitness = witness.ToBooleanArray(); - Console.WriteLine("> B: " + bmpBuilder.Dump()); - Console.WriteLine("> W: " + bmpWitness.Dump()); + Log("> B: " + bmpBuilder.Dump()); + Log("> W: " + bmpWitness.Dump()); var a = SuperSlowUncompressedBitmap.Dump(rawBuilder).ToString().Split('\n'); var b = SuperSlowUncompressedBitmap.Dump(rawWitness).ToString().Split('\n'); - Console.WriteLine(String.Join("\n", a.Zip(b, (x, y) => (x == y ? "= " : "##") + x + "\n " + y))); + Log(String.Join("\n", a.Zip(b, (x, y) => (x == y ? "= " : "##") + x + "\n " + y))); Assert.That(rawBuilder, Is.EqualTo(rawWitness), "Uncompressed bitmap does not match"); } private static bool SetBitAndVerify(CompressedBitmapBuilder builder, SuperSlowUncompressedBitmap witness, int offset) { - Console.WriteLine(); - Console.WriteLine("Set({0}):", offset); + Log(); + Log("Set({0}):", offset); bool actual = builder.Set(offset); bool expected = witness.Set(offset); Assert.That(actual, Is.EqualTo(expected), "Set({0})", offset); @@ -100,8 +101,8 @@ private static bool SetBitAndVerify(CompressedBitmapBuilder builder, SuperSlowUn private static bool ClearBitAndVerify(CompressedBitmapBuilder builder, SuperSlowUncompressedBitmap witness, int offset) { - Console.WriteLine(); - Console.WriteLine("Clear({0}):", offset); + Log(); + Log("Clear({0}):", offset); bool actual = builder.Clear(offset); bool expected = witness.Clear(offset); Assert.That(actual, Is.EqualTo(expected), "Clear({0})", offset); @@ -206,7 +207,7 @@ public void Test_CompressedBitmapBuilder_Random_Sets_And_Clears() var rnd = new Random(12345678); for (int k = 0; k < K; k++) { - Console.WriteLine("### Generation " + k); + Log("### Generation " + k); // convert to builder var builder = bmp.ToBuilder(); @@ -232,10 +233,10 @@ public void Test_CompressedBitmapBuilder_Random_Sets_And_Clears() // pack back to bitmap bmp = builder.ToBitmap(); - Console.WriteLine(); - Console.WriteLine("> Result of gen #{0}: {1}", k, bmp.Dump()); - Console.WriteLine("> " + bmp.ToSlice().ToHexaString()); - Console.WriteLine(); + Log(); + Log("> Result of gen #{0}: {1}", k, bmp.Dump()); + Log("> " + bmp.ToSlice().ToHexaString()); + Log(); } } @@ -246,16 +247,16 @@ public void TestFoo() Func compress = (input) => { - Console.WriteLine("IN [{0}] => {1}", input.Count, input); + Log("IN [{0}] => {1}", input.Count, input); var writer = new CompressedBitmapWriter(); int r = WordAlignHybridEncoder.CompressTo(input, writer); Slice compressed = writer.GetBuffer(); - Console.WriteLine("OUT [{0}] => {1} [r={2}]", compressed.Count, compressed, r); + Log("OUT [{0}] => {1} [r={2}]", compressed.Count, compressed, r); var sb = new StringBuilder(); - Console.WriteLine(WordAlignHybridEncoder.DumpCompressed(compressed).ToString()); - Console.WriteLine(); + Log(WordAlignHybridEncoder.DumpCompressed(compressed).ToString()); + Log(); return compressed; }; @@ -285,7 +286,7 @@ public void TestFoo() { setBit(buf, rnd.Next(buf.Length * 8)); } - Console.WriteLine("Mostly zeroes: " + count); + Log("Mostly zeroes: " + count); return buf; }; @@ -305,7 +306,7 @@ public void TestFoo() { clearBit(buf, rnd.Next(buf.Length * 8)); } - Console.WriteLine("Mostly ones: " + count); + Log("Mostly ones: " + count); return buf; }; @@ -321,7 +322,7 @@ public void TestFoo() var buffer = new byte[VALUES / 8]; var output = new CompressedBitmapWriter(); WordAlignHybridEncoder.CompressTo(buffer.AsSlice(), output); - Console.WriteLine("{0}\t{1}\t1024", 0, output.Length); + Log("{0}\t{1}\t1024", 0, output.Length); for (int i = 0; i < VALUES / 8; i++) { int p; @@ -335,7 +336,7 @@ public void TestFoo() output.Reset(); WordAlignHybridEncoder.CompressTo(buffer.AsSlice(), output); - Console.WriteLine("{0}\t{1}\t1024", 1.0d * (i + 1) / VALUES, output.Length); + Log("{0}\t{1}\t1024", 1.0d * (i + 1) / VALUES, output.Length); } } @@ -427,9 +428,9 @@ private static void DumpIndex(string label, MemoryIndex index, long totalLegacy = 0; int[] map = new int[100]; double r = (double)(map.Length - 1) / total; - Console.WriteLine("__{0}__", label); - Console.WriteLine("| Indexed Value | Count | Total % | Words | Lit% | 1-Bits | Word% | Bitmap | ratio % | Legacy | ratio % |" + (heatMaps ? " HeatMap |" : "")); - Console.WriteLine("|:------------------------|-------:|--------:|------:|-------:|-------:|-------:|---------:|--------:|----------:|--------:|" + (heatMaps ? ":-----------------------------------------------------------------------|" : "")); + Log("__{0}__", label); + Log("| Indexed Value | Count | Total % | Words | Lit% | 1-Bits | Word% | Bitmap | ratio % | Legacy | ratio % |" + (heatMaps ? " HeatMap |" : "")); + Log("|:------------------------|-------:|--------:|------:|-------:|-------:|-------:|---------:|--------:|----------:|--------:|" + (heatMaps ? ":-----------------------------------------------------------------------|" : "")); foreach (var kv in index.Values.OrderBy((kv) => orderBy(kv.Key, index.Count(kv.Key)), comparer)) { var t = STuple.Create(kv.Key); @@ -448,7 +449,7 @@ private static void DumpIndex(string label, MemoryIndex index, int bytes = kv.Value.ToSlice().Count; - Console.WriteLine(string.Format( + Log(string.Format( CultureInfo.InvariantCulture, "| {0,-24}| {1,6:N0} | {2,6:N2}% | {3,5:N0} | {4,5:N1}% | {5,6:N0} | {6,6:N2} | {7,8:N0} | {8,6:N2}% | {9,9:N0} | {10,6:N2}% |" + (heatMaps ? " `{11}` |" : ""), /*0*/ t, @@ -466,7 +467,7 @@ private static void DumpIndex(string label, MemoryIndex index, )); } - Console.WriteLine(string.Format( + Log(string.Format( CultureInfo.InvariantCulture, "> {0:N0} distinct value(s), {1:N0} document(s), {2:N0} bitmap bytes, {3:N0} legacy bytes", index.Values.Count, @@ -484,7 +485,7 @@ private static List DumpIndexQueryResult(Dictionary c Assert.That(characters.TryGetValue(docId, out Character charac), Is.True); results.Add(charac); - Console.WriteLine("- {0}: {1} {2}{3}", docId, charac.Name, charac.Gender == "Male" ? "\u2642" : charac.Gender == "Female" ? "\u2640" : charac.Gender, charac.Dead ? " (\u271D)" : ""); + Log("- {0}: {1} {2}{3}", docId, charac.Name, charac.Gender == "Male" ? "\u2642" : charac.Gender == "Female" ? "\u2640" : charac.Gender, charac.Dead ? " (\u271D)" : ""); } return results; } @@ -520,7 +521,7 @@ public void Test_Merging_Multiple_Bitmaps() MakeInserter(indexOfTheDead, (doc) => doc.Id, (doc) => doc.Dead), }; - Console.WriteLine("Inserting into database..."); + Log("Inserting into database..."); foreach (var character in dataSet) { database[character.Id] = character; @@ -531,41 +532,41 @@ public void Test_Merging_Multiple_Bitmaps() } // dump the indexes - Console.WriteLine(); + Log(); DumpIndex("Genders", indexByGender, (s, _) => s); - Console.WriteLine(); + Log(); DumpIndex("Jobs", indexByJob, (s, _) => s); - Console.WriteLine(); + Log(); DumpIndex("DeadOrAlive", indexOfTheDead, (s, _) => s); // Où sont les femmes ? - Console.WriteLine(); - Console.WriteLine("indexByGender.Lookup('Female')"); + Log(); + Log("indexByGender.Lookup('Female')"); CompressedBitmap females = indexByGender.Lookup("Female"); Assert.That(females, Is.Not.Null); - Console.WriteLine("=> {0}", females.Dump()); + Log("=> {0}", females.Dump()); DumpIndexQueryResult(database, females); // R.I.P - Console.WriteLine(); - Console.WriteLine("indexOfTheDead.Lookup(dead: true)"); + Log(); + Log("indexOfTheDead.Lookup(dead: true)"); CompressedBitmap deadPeople = indexOfTheDead.Lookup(true); Assert.That(deadPeople, Is.Not.Null); - Console.WriteLine("=> {0}", deadPeople.Dump()); + Log("=> {0}", deadPeople.Dump()); DumpIndexQueryResult(database, deadPeople); // combination of both - Console.WriteLine(); - Console.WriteLine("indexByGender.Lookup('Female') AND indexOfTheDead.Lookup(dead: true)"); + Log(); + Log("indexByGender.Lookup('Female') AND indexOfTheDead.Lookup(dead: true)"); var julia = WordAlignHybridEncoder.And(females, deadPeople); - Console.WriteLine("=> {0}", julia.Dump()); + Log("=> {0}", julia.Dump()); DumpIndexQueryResult(database, julia); // the crew - Console.WriteLine(); - Console.WriteLine("indexByJob.Lookup('Bounty_Hunter' OR 'Hacker' OR 'Dog')"); + Log(); + Log("indexByJob.Lookup('Bounty_Hunter' OR 'Hacker' OR 'Dog')"); var bmps = new[] { "Bounty_Hunter", "Hacker", "Dog" }.Select(job => indexByJob.Lookup(job)).ToList(); CompressedBitmap crew = null; foreach (var bmp in bmps) @@ -576,7 +577,7 @@ public void Test_Merging_Multiple_Bitmaps() crew = WordAlignHybridEncoder.Or(crew, bmp); } crew = crew ?? CompressedBitmap.Empty; - Console.WriteLine("=> {0}", crew.Dump()); + Log("=> {0}", crew.Dump()); DumpIndexQueryResult(database, crew); } @@ -691,9 +692,9 @@ public void Test_Randomized_Data() //foreach (var N in new[] { 1000, 2000, 5000, 10 * 1000, 20 * 1000, 50 * 1000, 100 * 1000 }) const int N = 10 * 1000; { - Console.WriteLine("================================================================================================================================================================================================================================="); - Console.WriteLine("N = {0:N0}", N); - Console.WriteLine("================================================================================================================================================================================================================================="); + Log("================================================================================================================================================================================================================================="); + Log("N = {0:N0}", N); + Log("================================================================================================================================================================================================================================="); rnd = new Random(123456); @@ -729,37 +730,37 @@ public void Test_Randomized_Data() }; var database = new Dictionary(); - //Console.Write("Inserting data: ..."); + //Log("Inserting data: ..."); foreach (var data in dataSet) { - //if (database.Count % 1000 == 0) Console.Write("\rInserting data: {0} / {1}", database.Count, N); + //if (database.Count % 1000 == 0) Log("\rInserting data: {0} / {1}", database.Count, N); database[data.Key] = data.Value; foreach (var inserter in inserters) inserter(data); } - //Console.WriteLine("\rInserting data: {0} / {1}", database.Count, N); + //Log("\rInserting data: {0} / {1}", database.Count, N); - Console.WriteLine(); + Log(); DumpIndex("Result", indexResult, (s, _) => s, heatMaps: true); - Console.WriteLine(); + Log(); DumpIndex("Valid", indexValid, (s, _) => s, heatMaps: true); - Console.WriteLine(); + Log(); DumpIndex("FlipFlops", indexFlipFlop, (s, _) => s, heatMaps: true); - Console.WriteLine(); + Log(); DumpIndex("Flips", indexFlips, (s, _) => s, heatMaps: true); - Console.WriteLine(); + Log(); DumpIndex("Location", indexLoc, (_, n) => -n, heatMaps: true); - Console.WriteLine(); + Log(); DumpIndex("Elevation", indexElevation, (s, _) => s, heatMaps: true); - //Console.WriteLine(indexValid.Values[true].Dump()); - //Console.WriteLine(indexValid.Values[true].ToSlice().ToHexaString()); - Console.WriteLine(); - Console.WriteLine(); + //Log(indexValid.Values[true].Dump()); + //Log(indexValid.Values[true].ToSlice().ToHexaString()); + Log(); + Log(); } } @@ -778,7 +779,7 @@ public void Test_BigBadIndexOfTheDead() #region create a non uniform random distribution for the users // step1: create a semi random distribution for the values - Console.WriteLine("Creating Probability Distribution Function for {0:N0} users...", K); + Log("Creating Probability Distribution Function for {0:N0} users...", K); var pk = new double[K]; // step1: each gets a random score for (int i = 0; i < pk.Length; i++) @@ -816,14 +817,14 @@ public void Test_BigBadIndexOfTheDead() //for (int i = 0; i < pk.Length; i += 500) //{ - // Console.WriteLine(pk[i].ToString("R", CultureInfo.InvariantCulture)); + // Log(pk[i].ToString("R", CultureInfo.InvariantCulture)); //} int p25 = Array.BinarySearch(pk, 0.25 * sum); p25 = p25 < 0 ? ~p25 : p25; int p50 = Array.BinarySearch(pk, 0.50 * sum); p50 = p50 < 0 ? ~p50 : p50; int p75 = Array.BinarySearch(pk, 0.75 * sum); p75 = p75 < 0 ? ~p75 : p75; int p95 = Array.BinarySearch(pk, 0.95 * sum); p95 = p95 < 0 ? ~p95 : p95; - Console.WriteLine("> PDF: P25={0:G2} %, P50={1:G2} %, P75={2:G2} %, P95={3:G2} %", 100.0 * p25 / K, 100.0 * p50 / K, 100.0 * p75 / K, 100.0 * p95 / K); + Log("> PDF: P25={0:G2} %, P50={1:G2} %, P75={2:G2} %, P95={3:G2} %", 100.0 * p25 / K, 100.0 * p50 / K, 100.0 * p75 / K, 100.0 * p95 / K); #endregion @@ -831,7 +832,7 @@ public void Test_BigBadIndexOfTheDead() // a user will be selected randomnly, and will be able to produce a random number of consecutive events, until we reach the desired amount of events - Console.WriteLine("Creating dataset for {0:N0} documents...", N); + Log("Creating dataset for {0:N0} documents...", N); var dataSet = new int[N]; //int j = 0; //for (int i = 0; i < N; i++) @@ -864,18 +865,18 @@ public void Test_BigBadIndexOfTheDead() if (user == K) user = 0; } - Console.WriteLine("Computing control statistics..."); + Log("Computing control statistics..."); // compute the control value for the counts per value var controlStats = dataSet .GroupBy(x => x).Select(g => new { Value = g.Key, Count = g.Count() }) .OrderByDescending(x => x.Count) .ToList(); - Console.WriteLine("> Found {0:N0} unique values", controlStats.Count); + Log("> Found {0:N0} unique values", controlStats.Count); #endregion // create pseudo-index - Console.WriteLine("Indexing {0:N0} documents...", N); + Log("Indexing {0:N0} documents...", N); var sw = Stopwatch.StartNew(); var index = new Dictionary(K); for (int id = 0; id < dataSet.Length; id++) @@ -890,10 +891,10 @@ public void Test_BigBadIndexOfTheDead() builder.Set(id); } sw.Stop(); - Console.WriteLine("> Found {0:N0} unique values in {1:N1} sec", index.Count, sw.Elapsed.TotalSeconds); + Log("> Found {0:N0} unique values in {1:N1} sec", index.Count, sw.Elapsed.TotalSeconds); // verify the counts - Console.WriteLine("Verifying index results..."); + Log("Verifying index results..."); var log = new StringWriter(CultureInfo.InvariantCulture); long totalBitmapSize = 0; j = 0; @@ -906,17 +907,17 @@ public void Test_BigBadIndexOfTheDead() int sz = bmp.ToSlice().Count; log.WriteLine("{0,8} : {1,5} bits, {2} words ({3} lit. / {4} fil.), {5:N0} bytes, {6:N3} bytes/doc, {7:N2}% compression", kv.Value, bits, words, a, b, sz, 1.0 * sz / bits, 100.0 * (4 + 17 + sz) / (17 + (4 + 17) * bits)); totalBitmapSize += sz; - //if (j % 500 == 0) Console.WriteLine((100.0 * b / words)); - //if (j % 500 == 0) Console.WriteLine(bmp.Dump()); + //if (j % 500 == 0) Log((100.0 * b / words)); + //if (j % 500 == 0) Log(bmp.Dump()); j++; } Assert.That(index.Count, Is.EqualTo(controlStats.Count), "Some values have not been indexed properly"); - Console.WriteLine("> success!"); - Console.WriteLine("Total index size for {0:N0} documents and {1:N0} values is {2:N0} bytes", N, K, totalBitmapSize); + Log("> success!"); + Log("Total index size for {0:N0} documents and {1:N0} values is {2:N0} bytes", N, K, totalBitmapSize); - Console.WriteLine(); - Console.WriteLine("Dumping results:"); - Trace.WriteLine(log.ToString()); + Log(); + Log("Dumping results:"); + Log(log.ToString()); } diff --git a/FoundationDB.Tests/FdbTest.cs b/FoundationDB.Tests/FdbTest.cs index 5323dae3e..e61decae0 100644 --- a/FoundationDB.Tests/FdbTest.cs +++ b/FoundationDB.Tests/FdbTest.cs @@ -150,19 +150,19 @@ protected async Task DeleteSubspace(IFdbDatabase db, IKeySubspace subspace) // These methods are just there to help with the problem of culture-aware string formatting [DebuggerStepThrough] - protected static void Log(string text) + public static void Log(string text) { - Console.WriteLine(text); + TestContext.Progress.WriteLine(text); } [DebuggerStepThrough] - protected static void Log() + public static void Log() { Log(String.Empty); } [DebuggerStepThrough] - protected static void Log(object item) + public static void Log(object item) { if (item == null) { @@ -175,19 +175,19 @@ protected static void Log(object item) } [DebuggerStepThrough] - protected static void Log(string format, object arg0) + public static void Log(string format, object arg0) { Log(String.Format(CultureInfo.InvariantCulture, format, arg0)); } [DebuggerStepThrough] - protected static void Log(string format, object arg0, object arg1) + public static void Log(string format, object arg0, object arg1) { Log(String.Format(CultureInfo.InvariantCulture, format, arg0, arg1)); } [DebuggerStepThrough] - protected static void Log(string format, params object[] args) + public static void Log(string format, params object[] args) { Log(String.Format(CultureInfo.InvariantCulture, format, args)); } diff --git a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs index 405687e15..600d253b2 100644 --- a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs +++ b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs @@ -86,11 +86,11 @@ await db.WriteAsync((tr) => { if (first) { - Console.WriteLine(tr.Log.GetCommandsReport()); + Log(tr.Log.GetCommandsReport()); first = false; } - Console.WriteLine(tr.Log.GetTimingsReport(true)); + Log(tr.Log.GetTimingsReport(true)); }; // create a logged version of the database @@ -98,8 +98,8 @@ await db.WriteAsync((tr) => for (int k = 0; k < N; k++) { - Console.WriteLine("==== " + k + " ==== "); - Console.WriteLine(); + Log("==== " + k + " ==== "); + Log(); await logged.ReadWriteAsync(async (tr) => { diff --git a/FoundationDB.Tests/Layers/CounterFacts.cs b/FoundationDB.Tests/Layers/CounterFacts.cs index 26d7c0453..1830fb3af 100644 --- a/FoundationDB.Tests/Layers/CounterFacts.cs +++ b/FoundationDB.Tests/Layers/CounterFacts.cs @@ -72,7 +72,7 @@ public async Task Bench_FdbCounter_Increment_Sequentially() var c = new FdbHighContentionCounter(db, location); - Console.WriteLine("Doing " + N + " inserts in one thread..."); + Log("Doing " + N + " inserts in one thread..."); var sw = Stopwatch.StartNew(); for (int i = 0; i < N; i++) @@ -81,7 +81,7 @@ public async Task Bench_FdbCounter_Increment_Sequentially() } sw.Stop(); - Console.WriteLine("> " + N + " completed in " + sw.Elapsed.TotalMilliseconds.ToString("N1") + " ms (" + (sw.Elapsed.TotalMilliseconds * 1000 / N).ToString("N0") + " µs/add)"); + Log("> " + N + " completed in " + sw.Elapsed.TotalMilliseconds.ToString("N1") + " ms (" + (sw.Elapsed.TotalMilliseconds * 1000 / N).ToString("N0") + " µs/add)"); #if DEBUG await DumpSubspace(db, location); @@ -108,7 +108,7 @@ public async Task Bench_FdbCounter_Increment_Concurrently() var c = new FdbHighContentionCounter(db, location); - Console.WriteLine("Doing " + W + " x " + B + " inserts in " + W + " threads..."); + Log("Doing " + W + " x " + B + " inserts in " + W + " threads..."); var signal = new TaskCompletionSource(); var done = new TaskCompletionSource(); @@ -128,7 +128,7 @@ public async Task Bench_FdbCounter_Increment_Concurrently() // wait await Task.WhenAll(workers); sw.Stop(); - Console.WriteLine("> " + N + " completed in " + sw.Elapsed.TotalMilliseconds.ToString("N1") + " ms (" + (sw.Elapsed.TotalMilliseconds * 1000 / B).ToString("N0") + " µs/add)"); + Log("> " + N + " completed in " + sw.Elapsed.TotalMilliseconds.ToString("N1") + " ms (" + (sw.Elapsed.TotalMilliseconds * 1000 / B).ToString("N0") + " µs/add)"); long n = await c.GetSnapshotAsync(this.Cancellation); if (n != N) diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 22878c887..81368c819 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -99,7 +99,7 @@ public async Task Test_Allocator() #if ENABLE_LOGGING foreach(var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -166,7 +166,7 @@ public async Task Test_CreateOrOpen_Simple() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -238,7 +238,7 @@ public async Task Test_CreateOrOpen_With_Layer() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -290,7 +290,7 @@ public async Task Test_CreateOrOpen_SubFolder() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -343,7 +343,7 @@ public async Task Test_List_SubFolders() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -441,7 +441,7 @@ public async Task Test_Move_Folder() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -501,7 +501,7 @@ public async Task Test_Remove_Folder() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -551,7 +551,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } #endif } @@ -566,12 +566,12 @@ public async Task Test_Directory_Partitions() await db.ClearRangeAsync(location, this.Cancellation); var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); var partition = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); // we can't get the partition key directory (because it's a root directory) so we need to cheat a little bit var partitionKey = KeySubspace.Copy(partition).GetPrefix(); - Console.WriteLine(partition); + Log(partition); Assert.That(partition, Is.InstanceOf()); Assert.That(partition.Layer, Is.EqualTo(Slice.FromStringAscii("partition"))); Assert.That(partition.FullName, Is.EqualTo("Foo")); @@ -581,14 +581,14 @@ public async Task Test_Directory_Partitions() Assert.That(partition.DirectoryLayer.NodeSubspace.GetPrefix(), Is.EqualTo(partitionKey + FdbKey.Directory), "Partition's nodes should be under the partition's prefix"); var bar = await partition.CreateAsync(db, "Bar", this.Cancellation); - Console.WriteLine(bar); + Log(bar); Assert.That(bar, Is.InstanceOf()); Assert.That(bar.Path, Is.EqualTo(new [] { "Foo", "Bar" }), "Path of directories under a partition should be absolute"); Assert.That(bar.GetPrefix(), Is.Not.EqualTo(partitionKey), "{0} should be located under {1}", bar, partition); Assert.That(bar.GetPrefix().StartsWith(partitionKey), Is.True, "{0} should be located under {1}", bar, partition); var baz = await partition.CreateAsync(db, "Baz", this.Cancellation); - Console.WriteLine(baz); + Log(baz); Assert.That(baz, Is.InstanceOf()); Assert.That(baz.FullName, Is.EqualTo("Foo/Baz")); Assert.That(baz.Path, Is.EqualTo(new[] { "Foo", "Baz" }), "Path of directories under a partition should be absolute"); @@ -597,7 +597,7 @@ public async Task Test_Directory_Partitions() // Rename 'Bar' to 'BarBar' var bar2 = await bar.MoveToAsync(db, new[] { "Foo", "BarBar" }, this.Cancellation); - Console.WriteLine(bar2); + Log(bar2); Assert.That(bar2, Is.InstanceOf()); Assert.That(bar2, Is.Not.SameAs(bar)); Assert.That(bar2.GetPrefix(), Is.EqualTo(bar.GetPrefix())); @@ -616,14 +616,14 @@ public async Task Test_Directory_Cannot_Move_To_Another_Partition() await db.ClearRangeAsync(location, this.Cancellation); var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); var foo = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); - Console.WriteLine(foo); + Log(foo); // create a 'Bar' under the 'Foo' partition var bar = await foo.CreateAsync(db, "Bar", this.Cancellation); - Console.WriteLine(bar); + Log(bar); Assert.That(bar.FullName, Is.EqualTo("Foo/Bar")); Assert.That(bar.Path, Is.EqualTo(new string[] { "Foo", "Bar" })); Assert.That(bar.DirectoryLayer, Is.Not.SameAs(directory)); @@ -645,14 +645,14 @@ public async Task Test_Directory_Cannot_Move_To_A_Sub_Partition() await db.ClearRangeAsync(location, this.Cancellation); var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); var outer = await directory.CreateAsync(db, "Outer", Slice.FromStringAscii("partition"), this.Cancellation); - Console.WriteLine(outer); + Log(outer); // create a 'Inner' subpartition under the 'Outer' partition var inner = await outer.CreateAsync(db, "Inner", Slice.FromString("partition"), this.Cancellation); - Console.WriteLine(inner); + Log(inner); Assert.That(inner.FullName, Is.EqualTo("Outer/Inner")); Assert.That(inner.Path, Is.EqualTo(new string[] { "Outer", "Inner" })); Assert.That(inner.DirectoryLayer, Is.Not.SameAs(directory)); @@ -845,7 +845,7 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K await db.ClearRangeAsync(location, this.Cancellation); var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); var partition = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); //note: if we want a testable key INSIDE the partition, we have to get it from a sub-directory @@ -952,12 +952,12 @@ public async Task Test_Concurrent_Directory_Creation() await db.ClearRangeAsync(location, this.Cancellation); var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); //to prevent any side effect from first time initialization of the directory layer, already create one dummy folder await directory.CreateAsync(db, "Zero", this.Cancellation); - var logdb = db.Logged((tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var logdb = db.Logged((tr) => Log(tr.Log.GetTimingsReport(true))); var f = FdbDirectoryLayer.AnnotateTransactions; try @@ -979,14 +979,14 @@ await Task.WhenAll( var second = await directory.CreateAsync(tr2, new[] { "Second" }, Slice.Nil); tr2.Set(second.GetPrefix(), Slice.FromString("This belongs to the second directory")); - Console.WriteLine("Committing T1..."); + Log("Committing T1..."); await tr1.CommitAsync(); - Console.WriteLine("T1 committed"); + Log("T1 committed"); tr1.Dispose(); // force T1 to be dumped immediately - Console.WriteLine("Committing T2..."); + Log("Committing T2..."); await tr2.CommitAsync(); - Console.WriteLine("T2 committed"); + Log("T2 committed"); } } finally @@ -1011,12 +1011,12 @@ public async Task Test_Concurrent_Directory_Creation_With_Custom_Prefix() // ie: regular prefix would be ("DL", 123) and our custom prefixes will be ("DL", "abc") var directory = FdbDirectoryLayer.Create(location); - Console.WriteLine(directory); + Log(directory); //to prevent any side effect from first time initialization of the directory layer, already create one dummy folder await directory.CreateAsync(db, "Zero", this.Cancellation); - var logdb = db.Logged((tr) => Console.WriteLine(tr.Log.GetTimingsReport(true))); + var logdb = db.Logged((tr) => Log(tr.Log.GetTimingsReport(true))); var f = FdbDirectoryLayer.AnnotateTransactions; try @@ -1038,12 +1038,12 @@ await Task.WhenAll( var second = await directory.RegisterAsync(tr2, new[] { "Second" }, Slice.Nil, location.Keys.Encode("def")); tr2.Set(second.GetPrefix(), Slice.FromString("This belongs to the second directory")); - Console.WriteLine("Committing T1..."); + Log("Committing T1..."); await tr1.CommitAsync(); - Console.WriteLine("T1 committed"); + Log("T1 committed"); tr1.Dispose(); // force T1 to be dumped immediately - Console.WriteLine("Committing T2..."); + Log("Committing T2..."); try { await tr2.CommitAsync(); @@ -1058,7 +1058,7 @@ await Task.WhenAll( } throw; } - Console.WriteLine("T2 committed"); + Log("T2 committed"); } } finally diff --git a/FoundationDB.Tests/Layers/IndexingFacts.cs b/FoundationDB.Tests/Layers/IndexingFacts.cs index 6045af1db..564b6cd9e 100644 --- a/FoundationDB.Tests/Layers/IndexingFacts.cs +++ b/FoundationDB.Tests/Layers/IndexingFacts.cs @@ -170,7 +170,7 @@ await db.WriteAsync((tr) => using (var tr = db.BeginTransaction(this.Cancellation)) { var superHeroes = await indexSuperHero.LookupAsync(tr, value: true); - Console.WriteLine("SuperHeroes: " + string.Join(", ", superHeroes)); + Log("SuperHeroes: " + string.Join(", ", superHeroes)); Assert.That(superHeroes, Is.EqualTo(characters.Where(c => c.HasSuperPowers).Select(c => c.Id).ToList())); } @@ -178,11 +178,11 @@ await db.WriteAsync((tr) => using (var tr = db.BeginTransaction(this.Cancellation)) { var dc = await indexBrand.LookupAsync(tr, value: "DC"); - Console.WriteLine("DC: " + string.Join(", ", dc)); + Log("DC: " + string.Join(", ", dc)); Assert.That(dc, Is.EqualTo(characters.Where(c => c.Brand == "DC").Select(c => c.Id).ToList())); var marvel = await indexBrand.LookupAsync(tr, value: "Marvel"); - Console.WriteLine("Marvel: " + string.Join(", ", dc)); + Log("Marvel: " + string.Join(", ", dc)); Assert.That(marvel, Is.EqualTo(characters.Where(c => c.Brand == "Marvel").Select(c => c.Id).ToList())); } diff --git a/FoundationDB.Tests/Layers/QueuesFacts.cs b/FoundationDB.Tests/Layers/QueuesFacts.cs index b85708990..f7a6dd152 100644 --- a/FoundationDB.Tests/Layers/QueuesFacts.cs +++ b/FoundationDB.Tests/Layers/QueuesFacts.cs @@ -55,12 +55,12 @@ public async Task Test_Queue_Fast() var queue = new FdbQueue(location, highContention: false); - Console.WriteLine("Clear Queue"); + Log("Clear Queue"); await db.WriteAsync((tr) => queue.Clear(tr), this.Cancellation); - Console.WriteLine("Empty? " + await db.ReadAsync((tr) => queue.EmptyAsync(tr), this.Cancellation)); + Log("Empty? " + await db.ReadAsync((tr) => queue.EmptyAsync(tr), this.Cancellation)); - Console.WriteLine("Push 10, 8, 6"); + Log("Push 10, 8, 6"); await db.ReadWriteAsync((tr) => queue.PushAsync(tr, 10), this.Cancellation); await db.ReadWriteAsync((tr) => queue.PushAsync(tr, 8), this.Cancellation); await db.ReadWriteAsync((tr) => queue.PushAsync(tr, 6), this.Cancellation); @@ -71,51 +71,51 @@ public async Task Test_Queue_Fast() // Empty? bool empty = await db.ReadAsync((tr) => queue.EmptyAsync(tr), this.Cancellation); - Console.WriteLine("Empty? " + empty); + Log("Empty? " + empty); Assert.That(empty, Is.False); Optional item = await queue.PopAsync(db, this.Cancellation); - Console.WriteLine("Pop item: " + item); + Log("Pop item: " + item); Assert.That((int)item, Is.EqualTo(10)); item = await db.ReadWriteAsync((tr) => queue.PeekAsync(tr), this.Cancellation); - Console.WriteLine("Next item: " + item); + Log("Next item: " + item); Assert.That((int)item, Is.EqualTo(8)); #if DEBUG await DumpSubspace(db, location); #endif item = await queue.PopAsync(db, this.Cancellation); - Console.WriteLine("Pop item: " + item); + Log("Pop item: " + item); Assert.That((int)item, Is.EqualTo(8)); #if DEBUG await DumpSubspace(db, location); #endif item = await queue.PopAsync(db, this.Cancellation); - Console.WriteLine("Pop item: " + item); + Log("Pop item: " + item); Assert.That((int)item, Is.EqualTo(6)); #if DEBUG await DumpSubspace(db, location); #endif empty = await db.ReadAsync((tr) => queue.EmptyAsync(tr), this.Cancellation); - Console.WriteLine("Empty? " + empty); + Log("Empty? " + empty); Assert.That(empty, Is.True); - Console.WriteLine("Push 5"); + Log("Push 5"); await db.ReadWriteAsync((tr) => queue.PushAsync(tr, 5), this.Cancellation); #if DEBUG await DumpSubspace(db, location); #endif - Console.WriteLine("Clear Queue"); + Log("Clear Queue"); await db.WriteAsync((tr) => queue.Clear(tr), this.Cancellation); #if DEBUG await DumpSubspace(db, location); #endif empty = await db.ReadAsync((tr) => queue.EmptyAsync(tr), this.Cancellation); - Console.WriteLine("Empty? " + empty); + Log("Empty? " + empty); Assert.That(empty, Is.True); } } @@ -151,7 +151,7 @@ public async Task Test_Single_Client() private static async Task RunMultiClientTest(IFdbDatabase db, KeySubspace location, bool highContention, string desc, int K, int NUM, CancellationToken ct) { - Console.WriteLine("Starting {0} test with {1} threads and {2} iterations", desc, K, NUM); + Log("Starting {0} test with {1} threads and {2} iterations", desc, K, NUM); var queue = new FdbQueue(location, highContention); await db.WriteAsync((tr) => queue.Clear(tr), ct); @@ -233,8 +233,8 @@ private static async Task RunMultiClientTest(IFdbDatabase db, KeySubspace locati } sw.Stop(); - Console.WriteLine("> Finished {0} test in {1} seconds", desc, sw.Elapsed.TotalSeconds); - Console.WriteLine("> Pushed {0}, Popped {1} and Stalled {2}", pushCount, popCount, stalls); + Log("> Finished {0} test in {1} seconds", desc, sw.Elapsed.TotalSeconds); + Log("> Pushed {0}, Popped {1} and Stalled {2}", pushCount, popCount, stalls); var pushedItems = pushTreads.SelectMany(t => t.Result).ToList(); var poppedItems = popThreads.SelectMany(t => t.Result).ToList(); @@ -305,23 +305,23 @@ public async Task Test_Log_Queue() #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } list.Clear(); #endif - Console.WriteLine("------------------------------------------------"); + Log("------------------------------------------------"); await RunMultiClientTest(logged, location, true, "high contention queue", 4, NUM, this.Cancellation); #if ENABLE_LOGGING foreach (var log in list) { - Console.WriteLine(log.GetTimingsReport(true)); + Log(log.GetTimingsReport(true)); } list.Clear(); #endif - Console.WriteLine("------------------------------------------------"); + Log("------------------------------------------------"); } diff --git a/FoundationDB.Tests/Layers/RankedSetFacts.cs b/FoundationDB.Tests/Layers/RankedSetFacts.cs index 89608f6e5..aad3c0da7 100644 --- a/FoundationDB.Tests/Layers/RankedSetFacts.cs +++ b/FoundationDB.Tests/Layers/RankedSetFacts.cs @@ -56,7 +56,7 @@ await db.ReadWriteAsync(async (tr) => await PrintRankedSet(vector, tr); }, this.Cancellation); - Console.WriteLine(); + Log(); var rnd = new Random(); var sw = Stopwatch.StartNew(); for (int i = 0; i < 100; i++) @@ -65,7 +65,7 @@ await db.ReadWriteAsync(async (tr) => await db.ReadWriteAsync((tr) => vector.InsertAsync(tr, TuPack.EncodeKey(rnd.Next())), this.Cancellation); } sw.Stop(); - Console.WriteLine("\rDone in {0:N3} sec", sw.Elapsed.TotalSeconds); + Log("\rDone in {0:N3} sec", sw.Elapsed.TotalSeconds); await db.ReadAsync((tr) => PrintRankedSet(vector, tr), this.Cancellation); } @@ -82,7 +82,7 @@ await tr.GetRange(rs.Subspace.Partition.ByKey(l).Keys.ToRange()).ForEachAsync((k sb.AppendFormat("\t{0} = {1}\r\n", rs.Subspace.Keys.Unpack(kvp.Key), kvp.Value.ToInt64()); }); } - Console.WriteLine(sb.ToString()); + Log(sb.ToString()); } } diff --git a/FoundationDB.Tests/Layers/VectorFacts.cs b/FoundationDB.Tests/Layers/VectorFacts.cs index 8fd0d3392..5bdf5e6d0 100644 --- a/FoundationDB.Tests/Layers/VectorFacts.cs +++ b/FoundationDB.Tests/Layers/VectorFacts.cs @@ -51,11 +51,11 @@ public async Task Test_Vector_Fast() using (var tr = db.BeginTransaction(this.Cancellation)) { - Console.WriteLine("Clearing any previous values in the vector"); + Log("Clearing any previous values in the vector"); vector.Clear(tr); - Console.WriteLine(); - Console.WriteLine("MODIFIERS"); + Log(); + Log("MODIFIERS"); // Set + Push vector.Set(tr, 0, Slice.FromInt32(1)); @@ -68,31 +68,31 @@ public async Task Test_Vector_Fast() await PrintVector(vector, tr); // Pop - Console.WriteLine("> Popped: " + await vector.PopAsync(tr)); + Log("> Popped: " + await vector.PopAsync(tr)); await PrintVector(vector, tr); // Clear vector.Clear(tr); - Console.WriteLine("> Pop empty: " + await vector.PopAsync(tr)); + Log("> Pop empty: " + await vector.PopAsync(tr)); await PrintVector(vector, tr); await vector.PushAsync(tr, Slice.FromString("foo")); - Console.WriteLine("> Pop size 1: " + await vector.PopAsync(tr)); + Log("> Pop size 1: " + await vector.PopAsync(tr)); await PrintVector(vector, tr); - Console.WriteLine(); - Console.WriteLine("CAPACITY OPERATIONS"); + Log(); + Log("CAPACITY OPERATIONS"); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Empty: " + await vector.EmptyAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); + Log("> Empty: " + await vector.EmptyAsync(tr)); - Console.WriteLine("> Resizing to length 5"); + Log("> Resizing to length 5"); await vector.ResizeAsync(tr, 5); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("Settings values"); + Log("Settings values"); vector.Set(tr, 0, Slice.FromString("Portez")); vector.Set(tr, 1, Slice.FromString("ce vieux")); vector.Set(tr, 2, Slice.FromString("whisky")); @@ -101,63 +101,63 @@ public async Task Test_Vector_Fast() vector.Set(tr, 5, Slice.FromString("fume")); await PrintVector(vector, tr); - Console.WriteLine("FRONT"); - Console.WriteLine("> " + await vector.FrontAsync(tr)); + Log("FRONT"); + Log("> " + await vector.FrontAsync(tr)); - Console.WriteLine("BACK"); - Console.WriteLine("> " + await vector.BackAsync(tr)); + Log("BACK"); + Log("> " + await vector.BackAsync(tr)); - Console.WriteLine(); - Console.WriteLine("ELEMENT ACCESS"); - Console.WriteLine("> Index 0: " + await vector.GetAsync(tr, 0)); - Console.WriteLine("> Index 5: " + await vector.GetAsync(tr, 5)); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log(); + Log("ELEMENT ACCESS"); + Log("> Index 0: " + await vector.GetAsync(tr, 0)); + Log("> Index 5: " + await vector.GetAsync(tr, 5)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine(); - Console.WriteLine("RESIZING"); - Console.WriteLine("> Resizing to 3"); + Log(); + Log("RESIZING"); + Log("> Resizing to 3"); await vector.ResizeAsync(tr, 3); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Resizing to 3 again"); + Log("> Resizing to 3 again"); await vector.ResizeAsync(tr, 3); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Resizing to 6"); + Log("> Resizing to 6"); await vector.ResizeAsync(tr, 6); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine(); - Console.WriteLine("SPARSE TEST"); + Log(); + Log("SPARSE TEST"); - Console.WriteLine("> Popping sparse vector"); + Log("> Popping sparse vector"); await vector.PopAsync(tr); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Resizing to 4"); + Log("> Resizing to 4"); await vector.ResizeAsync(tr, 4); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Adding 'word' to index 10, resize to 25"); + Log("> Adding 'word' to index 10, resize to 25"); vector.Set(tr, 10, Slice.FromString("word")); await vector.ResizeAsync(tr, 25); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Swapping with sparse element"); + Log("> Swapping with sparse element"); await vector.SwapAsync(tr, 10, 15); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); - Console.WriteLine("> Swapping sparse elements"); + Log("> Swapping sparse elements"); await vector.SwapAsync(tr, 12, 13); await PrintVector(vector, tr); - Console.WriteLine("> Size: " + await vector.SizeAsync(tr)); + Log("> Size: " + await vector.SizeAsync(tr)); } } } @@ -173,7 +173,7 @@ await tr.GetRange(vector.Subspace.Keys.ToRange()).ForEachAsync((kvp) => sb.Append($"{vector.Subspace.Keys.DecodeLast(kvp.Key)}:{kvp.Value:P}"); }); - Console.WriteLine("> Vector: (" + sb.ToString() + ")"); + Log("> Vector: (" + sb.ToString() + ")"); } } diff --git a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs index d52f4322a..273808dff 100644 --- a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs @@ -955,7 +955,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() int called = 0; var sw = new Stopwatch(); - Console.WriteLine("CONSTANT LATENCY GENERATOR:"); + Log("CONSTANT LATENCY GENERATOR:"); // this iterator waits on each item produced var source = new AnonymousAsyncGenerator((index, ct) => @@ -977,7 +977,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() called = 0; sw.Restart(); var withoutPrefetching = await source.Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P0: {0}", String.Join(", ", withoutPrefetching)); + Log("P0: {0}", String.Join(", ", withoutPrefetching)); Assert.That(withoutPrefetching.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); Assert.That(withoutPrefetching.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(1, 10))); @@ -985,7 +985,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() called = 0; sw.Restart(); var withPrefetching1 = await source.Prefetch().Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P1: {0}", String.Join(", ", withPrefetching1)); + Log("P1: {0}", String.Join(", ", withPrefetching1)); Assert.That(withPrefetching1.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); Assert.That(withPrefetching1.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(2, 10))); @@ -994,7 +994,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() called = 0; sw.Restart(); var withPrefetching2 = await source.Prefetch(2).Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P2: {0}", String.Join(", ", withPrefetching2)); + Log("P2: {0}", String.Join(", ", withPrefetching2)); Assert.That(withPrefetching2.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); Assert.That(withPrefetching2.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(2, 10))); } @@ -1005,7 +1005,7 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() int called = 0; var sw = new Stopwatch(); - Console.WriteLine("BURSTY GENERATOR:"); + Log("BURSTY GENERATOR:"); // this iterator produce burst of items var source = new AnonymousAsyncGenerator((index, ct) => @@ -1027,7 +1027,7 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() called = 0; sw.Restart(); var withoutPrefetching = await source.Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P0: {0}", String.Join(", ", withoutPrefetching)); + Log("P0: {0}", String.Join(", ", withoutPrefetching)); Assert.That(withoutPrefetching.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); // with prefetching K, the consumer should always have K items in advance @@ -1037,7 +1037,7 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() called = 0; sw.Restart(); var withPrefetchingK = await source.Prefetch(K).Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P{0}: {1}", K, String.Join(", ", withPrefetchingK)); + Log("P{0}: {1}", K, String.Join(", ", withPrefetchingK)); Assert.That(withPrefetchingK.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); Assert.That(withPrefetchingK[0].Item2, Is.EqualTo(K + 1), "Generator must have {0} call(s) in advance!", K); Assert.That(withPrefetchingK.Select(x => x.Item2), Is.All.LessThanOrEqualTo(11)); @@ -1047,7 +1047,7 @@ public async Task Test_Can_Prefetch_On_Bursty_Source() called = 0; sw.Restart(); var withPrefetching5 = await source.Prefetch(5).Select(record).ToListAsync(this.Cancellation); - Console.WriteLine("P5: {0}", String.Join(", ", withPrefetching5)); + Log("P5: {0}", String.Join(", ", withPrefetching5)); Assert.That(withPrefetching5.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); Assert.That(withPrefetching5[0].Item2, Is.EqualTo(5), "Generator must have only 4 calls in advance because it only produces 4 items at a time!"); Assert.That(withPrefetching5.Select(x => x.Item2), Is.All.LessThanOrEqualTo(11)); @@ -1170,14 +1170,14 @@ public async Task Test_Parallel_Select_Async() try { Assert.That(n, Is.LessThanOrEqualTo(MAX_CONCURRENCY)); - Console.WriteLine("** " + sw.Elapsed + " start " + x + " (" + n + ")"); + Log("** " + sw.Elapsed + " start " + x + " (" + n + ")"); #if DEBUG_STACK_TRACES - Console.WriteLine("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); + Log("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); #endif int ms; lock (rnd) { ms = rnd.Next(25) + 50; } await Task.Delay(ms); - Console.WriteLine("** " + sw.Elapsed + " stop " + x + " (" + Volatile.Read(ref concurrent) + ")"); + Log("** " + sw.Elapsed + " stop " + x + " (" + Volatile.Read(ref concurrent) + ")"); return x * x; } @@ -1199,7 +1199,7 @@ public async Task Test_Parallel_Select_Async() var results = await query.ToListAsync(token); Assert.That(Volatile.Read(ref concurrent), Is.EqualTo(0)); - Console.WriteLine("Results: " + string.Join(", ", results)); + Log("Results: " + string.Join(", ", results)); Assert.That(results, Is.EqualTo(Enumerable.Range(1, N).Select(x => x * x).ToArray())); } @@ -1224,24 +1224,24 @@ public async Task Test_FdbAsyncBuffer() { while (!token.IsCancellationRequested) { - Console.WriteLine("[consumer] start receiving next..."); + Log("[consumer] start receiving next..."); var msg = await buffer.ReceiveAsync(token); #if DEBUG_STACK_TRACES - Console.WriteLine("[consumer] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[consumer] > ")); + Log("[consumer] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[consumer] > ")); #endif if (msg.HasValue) { - Console.WriteLine("[consumer] Got value " + msg.Value); + Log("[consumer] Got value " + msg.Value); } else if (msg.HasValue) { - Console.WriteLine("[consumer] Got error: " + msg.Error); + Log("[consumer] Got error: " + msg.Error); msg.ThrowForNonSuccess(); break; } else { - Console.WriteLine("[consumer] Done!"); + Log("[consumer] Done!"); break; } @@ -1254,25 +1254,25 @@ public async Task Test_FdbAsyncBuffer() // first 5 calls to enqueue should already be completed while (!token.IsCancellationRequested && i < MAX_CAPACITY * 10) { - Console.WriteLine("[PRODUCER] Publishing " + i); + Log("[PRODUCER] Publishing " + i); #if DEBUG_STACK_TRACES - Console.WriteLine("[PRODUCER] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[PRODUCER] > ")); + Log("[PRODUCER] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[PRODUCER] > ")); #endif await buffer.OnNextAsync(i, token); ++i; - Console.WriteLine("[PRODUCER] Published"); + Log("[PRODUCER] Published"); #if DEBUG_STACK_TRACES - Console.WriteLine("[PRODUCER] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[PRODUCER] > ")); + Log("[PRODUCER] > " + new StackTrace().ToString().Replace("\r\n", "\r\n[PRODUCER] > ")); #endif if (rnd.Next(10) < 2) { - Console.WriteLine("[PRODUCER] Thinking " + i); + Log("[PRODUCER] Thinking " + i); await Task.Delay(10); } } - Console.WriteLine("[PRODUCER] COMPLETED!"); + Log("[PRODUCER] COMPLETED!"); buffer.OnCompleted(); var t = await Task.WhenAny(pump, Task.Delay(TimeSpan.FromSeconds(10), token)); @@ -1298,7 +1298,7 @@ public async Task Test_FdbASyncIteratorPump() { await Task.Delay(15); } - Console.WriteLine("[PRODUCER] publishing " + x + " at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); + Log("[PRODUCER] publishing " + x + " at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); return x; }); @@ -1314,9 +1314,9 @@ public async Task Test_FdbASyncIteratorPump() var queue = AsyncHelpers.CreateTarget( onNextAsync: (x, ct) => { - Console.WriteLine("[consumer] onNextAsync(" + x + ") at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); + Log("[consumer] onNextAsync(" + x + ") at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); #if DEBUG_STACK_TRACES - Console.WriteLine("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); + Log("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); #endif ct.ThrowIfCancellationRequested(); items.Add(x); @@ -1324,16 +1324,16 @@ public async Task Test_FdbASyncIteratorPump() }, onCompleted: () => { - Console.WriteLine("[consumer] onCompleted() at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); + Log("[consumer] onCompleted() at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); #if DEBUG_STACK_TRACES - Console.WriteLine("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); + Log("> " + new StackTrace().ToString().Replace("\r\n", "\r\n> ")); #endif done = true; }, onError: (x) => { - Console.WriteLine("[consumer] onError() at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); - Console.WriteLine("[consumer] > " + x); + Log("[consumer] onError() at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); + Log("[consumer] > " + x); error = x; go.Cancel(); } @@ -1343,11 +1343,11 @@ public async Task Test_FdbASyncIteratorPump() { var pump = new AsyncIteratorPump(inner, queue); - Console.WriteLine("[PUMP] Start pumping on #" + Thread.CurrentThread.ManagedThreadId); + Log("[PUMP] Start pumping on #" + Thread.CurrentThread.ManagedThreadId); sw.Start(); await pump.PumpAsync(token); sw.Stop(); - Console.WriteLine("[PUMP] Pumping completed! at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); + Log("[PUMP] Pumping completed! at " + sw.Elapsed.TotalMilliseconds + " on #" + Thread.CurrentThread.ManagedThreadId); // We should have N items, plus 1 message for the completion Assert.That(items.Count, Is.EqualTo(N)); @@ -1395,7 +1395,7 @@ private static async Task VerifyResult(Func> asyncQuery, Func refe } catch(AssertionException x) { - Console.WriteLine("FAIL: " + witness.Expression + "\r\n > " + x.Message); + Log("FAIL: " + witness.Expression + "\r\n > " + x.Message); } } @@ -1433,7 +1433,7 @@ private static async Task VerifySequence(Func> asyncQuery, F } catch (AssertionException x) { - Console.WriteLine("FAIL: " + witness.Expression + "\r\n > " + x.Message); + Log("FAIL: " + witness.Expression + "\r\n > " + x.Message); } } @@ -1655,14 +1655,14 @@ public async Task Test_Record_Items() .Observe((x) => after.Add(x)) .Select((x) => x + 1); - Console.WriteLine("query: " + query); + Log("query: " + query); var results = await query.ToListAsync(); - Console.WriteLine("input : " + String.Join(", ", items)); - Console.WriteLine("before: " + String.Join(", ", before)); - Console.WriteLine("after : " + String.Join(", ", after)); - Console.WriteLine("output: " + String.Join(", ", results)); + Log("input : " + String.Join(", ", items)); + Log("before: " + String.Join(", ", before)); + Log("after : " + String.Join(", ", after)); + Log("output: " + String.Join(", ", results)); Assert.That(before, Is.EqualTo(Enumerable.Range(0, 10).ToList())); Assert.That(after, Is.EqualTo(Enumerable.Range(0, 10).Where(x => x % 2 == 1).ToList())); diff --git a/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs b/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs index 742d19573..ec87f4034 100644 --- a/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncQueryableFacts.cs @@ -62,22 +62,22 @@ await db.WriteAsync((tr) => var range = db.Query().RangeStartsWith(location.GetPrefix()); Assert.That(range, Is.InstanceOf>>()); Assert.That(range.Expression, Is.InstanceOf()); - Console.WriteLine(range.Expression.DebugView); + Log(range.Expression.DebugView); var projection = range.Select(kvp => kvp.Value.ToString()); Assert.That(projection, Is.InstanceOf>()); Assert.That(projection.Expression, Is.InstanceOf, string>>()); - Console.WriteLine(projection.Expression.DebugView); + Log(projection.Expression.DebugView); var results = await projection.ToListAsync(); - Console.WriteLine("ToListAsync() => [ " + String.Join(", ", results) + " ]"); + Log("ToListAsync() => [ " + String.Join(", ", results) + " ]"); var count = await projection.CountAsync(); - Console.WriteLine("CountAsync() => " + count); + Log("CountAsync() => " + count); Assert.That(count, Is.EqualTo(2)); var first = await projection.FirstAsync(); - Console.WriteLine("FirstAsync() => " + first); + Log("FirstAsync() => " + first); Assert.That(first, Is.EqualTo("World!")); } } @@ -107,10 +107,10 @@ await db.WriteAsync((tr) => Assert.That(lookup, Is.InstanceOf>()); Assert.That(lookup.Expression, Is.InstanceOf>()); - Console.WriteLine(lookup.Expression.DebugView); + Log(lookup.Expression.DebugView); var ids = await lookup.ToListAsync(); - Console.WriteLine("=> [ " + String.Join(", ", ids) + " ]"); + Log("=> [ " + String.Join(", ", ids) + " ]"); } @@ -146,19 +146,19 @@ await db.WriteAsync((tr) => var lookup = index.Query(db).Lookup(x => x <= 100); Assert.That(lookup, Is.InstanceOf>()); Assert.That(lookup.Expression, Is.InstanceOf>()); - Console.WriteLine(lookup.Expression.DebugView); + Log(lookup.Expression.DebugView); var ids = await lookup.ToListAsync(); - Console.WriteLine("=> [ " + String.Join(", ", ids) + " ]"); + Log("=> [ " + String.Join(", ", ids) + " ]"); // find all that are over nine thousand lookup = index.Query(db).Lookup(x => x >= 9000); Assert.That(lookup, Is.InstanceOf>()); Assert.That(lookup.Expression, Is.InstanceOf>()); - Console.WriteLine(lookup.Expression.DebugView); + Log(lookup.Expression.DebugView); ids = await lookup.ToListAsync(); - Console.WriteLine("=> [ " + String.Join(", ", ids) + " ]"); + Log("=> [ " + String.Join(", ", ids) + " ]"); } diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index b483e6118..5e22282cc 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -34,176 +34,179 @@ namespace FoundationDB.Linq.Expressions.Tests using Doxense.Collections.Tuples; using Doxense.Linq; using FoundationDB.Client; + using FoundationDB.Client.Tests; using FoundationDB.Layers.Indexing; using NUnit.Framework; [TestFixture] - public class FdbQueryExpressionFacts + public class FdbQueryExpressionFacts : FdbTest + + { + + private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(TuPack.EncodeKey("Foos", 1))); + + private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(TuPack.EncodeKey("Foos", 2))); + + [Test] + public void Test_FdbQueryIndexLookupExpression() + { + var expr = FdbQueryIndexLookupExpression.Lookup( + FooBarIndex, + ExpressionType.Equal, + Expression.Constant("world") + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Index, Is.SameAs(FooBarIndex)); //TODO: .Index.Index does not look very nice + Assert.That(expr.Operator, Is.EqualTo(ExpressionType.Equal)); + Assert.That(expr.Value, Is.Not.Null); + Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + + } + + [Test] + public void Test_FdbQueryIndexLookupExpression_From_Lambda() + { + var expr = FdbQueryIndexLookupExpression.Lookup( + FooBarIndex, + (bar) => bar == "world" + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Index, Is.SameAs(FooBarIndex)); //TODO: .Index.Index does not look very nice + Assert.That(expr.Operator, Is.EqualTo(ExpressionType.Equal)); + Assert.That(expr.Value, Is.Not.Null); + Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + + } + + [Test] + public void Test_FdbQueryRangeExpression() + { + var expr = FdbQueryExpressions.Range( + KeySelectorPair.Create(TuPack.ToRange(STuple.Create("Foo"))) + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Range.Begin.Key.ToString(), Is.EqualTo("<02>Foo<00>")); + Assert.That(expr.Range.End.Key.ToString(), Is.EqualTo("<02>Foo<01>")); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + } + + [Test] + public void Test_FdbQueryIntersectExpression() + { + var expr1 = FdbQueryIndexLookupExpression.Lookup( + FooBarIndex, + (x) => x == "world" + ); + var expr2 = FdbQueryIndexLookupExpression.Lookup( + FooBazIndex, + (x) => x == 1234L + ); + + var expr = FdbQueryExpressions.Intersect( + expr1, + expr2 + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Terms, Is.Not.Null); + Assert.That(expr.Terms.Count, Is.EqualTo(2)); + Assert.That(expr.Terms[0], Is.SameAs(expr1)); + Assert.That(expr.Terms[1], Is.SameAs(expr2)); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + } + + [Test] + public void Test_FdbQueryUnionExpression() + { + var expr1 = FdbQueryIndexLookupExpression.Lookup( + FooBarIndex, + (x) => x == "world" + ); + var expr2 = FdbQueryIndexLookupExpression.Lookup( + FooBazIndex, + (x) => x == 1234L + ); + + var expr = FdbQueryExpressions.Union( + expr1, + expr2 + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Terms, Is.Not.Null); + Assert.That(expr.Terms.Count, Is.EqualTo(2)); + Assert.That(expr.Terms[0], Is.SameAs(expr1)); + Assert.That(expr.Terms[1], Is.SameAs(expr2)); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + } + + [Test] + public void Test_FdbQueryTransformExpression() + { + var expr = FdbQueryExpressions.Transform( + FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), + (kvp) => kvp.Value.ToUnicode() + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); + Assert.That(expr.Transform, Is.Not.Null); + + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(string))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + } + + [Test] + public void Test_FdbQueryFilterExpression() { + var expr = FdbQueryExpressions.Filter( + FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), + (kvp) => kvp.Value.ToInt32() % 2 == 0 + ); + Log(expr); + + Assert.That(expr, Is.Not.Null); + Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); + Assert.That(expr.Filter, Is.Not.Null); - private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(TuPack.EncodeKey("Foos", 1))); - private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(TuPack.EncodeKey("Foos", 2))); - - [Test] - public void Test_FdbQueryIndexLookupExpression() - { - var expr = FdbQueryIndexLookupExpression.Lookup( - FooBarIndex, - ExpressionType.Equal, - Expression.Constant("world") - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Index, Is.SameAs(FooBarIndex)); //TODO: .Index.Index does not look very nice - Assert.That(expr.Operator, Is.EqualTo(ExpressionType.Equal)); - Assert.That(expr.Value, Is.Not.Null); - Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - - } - - [Test] - public void Test_FdbQueryIndexLookupExpression_From_Lambda() - { - var expr = FdbQueryIndexLookupExpression.Lookup( - FooBarIndex, - (bar) => bar == "world" - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Index, Is.SameAs(FooBarIndex)); //TODO: .Index.Index does not look very nice - Assert.That(expr.Operator, Is.EqualTo(ExpressionType.Equal)); - Assert.That(expr.Value, Is.Not.Null); - Assert.That(expr.Value, Is.InstanceOf().With.Property("Value").EqualTo("world")); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - - } - - [Test] - public void Test_FdbQueryRangeExpression() - { - var expr = FdbQueryExpressions.Range( - KeySelectorPair.Create(TuPack.ToRange(STuple.Create("Foo"))) - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Range.Begin.Key.ToString(), Is.EqualTo("<02>Foo<00>")); - Assert.That(expr.Range.End.Key.ToString(), Is.EqualTo("<02>Foo<01>")); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - } - - [Test] - public void Test_FdbQueryIntersectExpression() - { - var expr1 = FdbQueryIndexLookupExpression.Lookup( - FooBarIndex, - (x) => x == "world" - ); - var expr2 = FdbQueryIndexLookupExpression.Lookup( - FooBazIndex, - (x) => x == 1234L - ); - - var expr = FdbQueryExpressions.Intersect( - expr1, - expr2 - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Terms, Is.Not.Null); - Assert.That(expr.Terms.Count, Is.EqualTo(2)); - Assert.That(expr.Terms[0], Is.SameAs(expr1)); - Assert.That(expr.Terms[1], Is.SameAs(expr2)); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - } - - [Test] - public void Test_FdbQueryUnionExpression() - { - var expr1 = FdbQueryIndexLookupExpression.Lookup( - FooBarIndex, - (x) => x == "world" - ); - var expr2 = FdbQueryIndexLookupExpression.Lookup( - FooBazIndex, - (x) => x == 1234L - ); - - var expr = FdbQueryExpressions.Union( - expr1, - expr2 - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Terms, Is.Not.Null); - Assert.That(expr.Terms.Count, Is.EqualTo(2)); - Assert.That(expr.Terms[0], Is.SameAs(expr1)); - Assert.That(expr.Terms[1], Is.SameAs(expr2)); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(int))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - } - - [Test] - public void Test_FdbQueryTransformExpression() - { - var expr = FdbQueryExpressions.Transform( - FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), - (kvp) => kvp.Value.ToUnicode() - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); - Assert.That(expr.Transform, Is.Not.Null); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(string))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - } - - [Test] - public void Test_FdbQueryFilterExpression() - { - var expr = FdbQueryExpressions.Filter( - FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), - (kvp) => kvp.Value.ToInt32() % 2 == 0 - ); - Console.WriteLine(expr); - - Assert.That(expr, Is.Not.Null); - Assert.That(expr.Source, Is.Not.Null.And.InstanceOf()); - Assert.That(expr.Filter, Is.Not.Null); - - Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); - Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); - - Console.WriteLine(FdbQueryExpressions.ExplainSequence(expr)); - } + Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); + Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); + + Log(FdbQueryExpressions.ExplainSequence(expr)); + } } diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 3dd348d3a..26a74bc0f 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -555,19 +555,19 @@ public async Task Test_Range_Intersect() var series = Enumerable.Range(1, K).Select(k => Enumerable.Range(1, N).Select(x => k * x).ToArray()).ToArray(); //foreach(var serie in series) //{ - // Console.WriteLine(String.Join(", ", serie)); + // Log(String.Join(", ", serie)); //} for (int k = 0; k < K; k++) { - //Console.WriteLine("> k = " + k); + //Log("> k = " + k); using (var tr = db.BeginTransaction(this.Cancellation)) { for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); var value = TuPack.EncodeKey(k, i); - //Console.WriteLine("> " + key + " = " + value); + //Log("> " + key + " = " + value); tr.Set(key, value); } await tr.CommitAsync(); @@ -627,19 +627,19 @@ public async Task Test_Range_Except() var series = Enumerable.Range(1, K).Select(k => Enumerable.Range(1, N).Select(x => k * x).ToArray()).ToArray(); //foreach(var serie in series) //{ - // Console.WriteLine(String.Join(", ", serie)); + // Log(String.Join(", ", serie)); //} for (int k = 0; k < K; k++) { - //Console.WriteLine("> k = " + k); + //Log("> k = " + k); using (var tr = db.BeginTransaction(this.Cancellation)) { for (int i = 0; i < N; i++) { var key = lists[k].Keys.Encode(series[k][i]); var value = TuPack.EncodeKey(k, i); - //Console.WriteLine("> " + key + " = " + value); + //Log("> " + key + " = " + value); tr.Set(key, value); } await tr.CommitAsync(); diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index 009ea39a7..b6b7a8abb 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -101,7 +101,7 @@ public static async Task DumpSubspace([NotNull] IFdbReadOnlyTransaction tr, [Not { Assert.That(tr, Is.Not.Null); - Console.WriteLine("Dumping content of subspace " + subspace.ToString() + " :"); + FdbTest.Log("Dumping content of subspace " + subspace.ToString() + " :"); int count = 0; await tr .GetRange(KeyRange.StartsWith(subspace.GetPrefix())) @@ -121,13 +121,13 @@ await tr keyDump = "'" + key.ToString() + "'"; } - Console.WriteLine("- " + keyDump + " = " + kvp.Value.ToString()); + FdbTest.Log("- " + keyDump + " = " + kvp.Value.ToString()); }); if (count == 0) - Console.WriteLine("> empty !"); + FdbTest.Log("> empty !"); else - Console.WriteLine("> Found " + count + " values"); + FdbTest.Log("> Found " + count + " values"); } public static async Task AssertThrowsFdbErrorAsync([NotNull] Func asyncTest, FdbError expectedCode, string message = null, object[] args = null) diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 90d3ef9d0..ec6329201 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -1958,13 +1958,14 @@ await db.WriteAsync((tr) => int time = 0; List m_alive = new List(); + var sb = new StringBuilder(); while (DateTime.UtcNow - start < TimeSpan.FromSeconds(DURATION_SEC)) { switch (rnd.Next(10)) { case 0: { // start a new transaction - Console.Write('T'); + sb.Append('T'); var tr = db.BeginTransaction(FdbTransactionMode.Default, this.Cancellation); m_alive.Add(tr); break; @@ -1972,7 +1973,7 @@ await db.WriteAsync((tr) => case 1: { // drop a random transaction if (m_alive.Count == 0) continue; - Console.Write('L'); + sb.Append('L'); int p = rnd.Next(m_alive.Count); m_alive.RemoveAt(p); @@ -1982,7 +1983,7 @@ await db.WriteAsync((tr) => case 2: { // dispose a random transaction if (m_alive.Count == 0) continue; - Console.Write('D'); + sb.Append('D'); int p = rnd.Next(m_alive.Count); var tr = m_alive[p]; @@ -1992,7 +1993,7 @@ await db.WriteAsync((tr) => } case 3: { // GC! - Console.Write('C'); + sb.Append('C'); var tr = db.BeginTransaction(FdbTransactionMode.ReadOnly, this.Cancellation); m_alive.Add(tr); _ = await tr.GetReadVersionAsync(); @@ -2003,7 +2004,7 @@ await db.WriteAsync((tr) => case 5: case 6: { // read a random value from a random transaction - Console.Write('G'); + sb.Append('G'); if (m_alive.Count == 0) break; int p = rnd.Next(m_alive.Count); var tr = m_alive[p]; @@ -2015,7 +2016,7 @@ await db.WriteAsync((tr) => } catch (FdbException) { - Console.Write('!'); + sb.Append('!'); } break; } @@ -2023,13 +2024,13 @@ await db.WriteAsync((tr) => case 8: case 9: { // read a random value, but drop the task - Console.Write('g'); + sb.Append('g'); if (m_alive.Count == 0) break; int p = rnd.Next(m_alive.Count); var tr = m_alive[p]; int x = rnd.Next(R); - _ = tr.GetAsync(location.Keys.Encode(x)).ContinueWith((_) => Console.Write('!'), TaskContinuationOptions.NotOnRanToCompletion); + _ = tr.GetAsync(location.Keys.Encode(x)).ContinueWith((_) => sb.Append('!') /*BUGBUG: locking ?*/, TaskContinuationOptions.NotOnRanToCompletion); // => t is not stored break; } @@ -2037,9 +2038,10 @@ await db.WriteAsync((tr) => } if ((time++) % 80 == 0) { - Console.WriteLine(); + Log(sb.ToString()); Log("State: {0}", m_alive.Count); - Console.Write('C'); + sb.Clear(); + sb.Append('C'); GC.Collect(); GC.WaitForPendingFinalizers(); GC.Collect(); diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index 72e8ccfdc..daa76aeaa 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -160,19 +160,19 @@ public async Task Test_Transactionals_Retries_Do_Not_Leak_When_Reading_Too_Much( // insert a good amount of test data var sw = Stopwatch.StartNew(); - Console.WriteLine("Inserting test data (this may take a few minutes)..."); + Log("Inserting test data (this may take a few minutes)..."); var rnd = new Random(); await Fdb.Bulk.WriteAsync(db, Enumerable.Range(0, 100 * 1000).Select(i => new KeyValuePair(location.Keys.Encode(i), Slice.Random(rnd, 4096))), this.Cancellation); sw.Stop(); - Console.WriteLine("> done in " + sw.Elapsed); + Log("> done in " + sw.Elapsed); - using (var timer = new System.Threading.Timer((_) => { Console.WriteLine("WorkingSet: {0:N0}, Managed: {1:N0}", Environment.WorkingSet, GC.GetTotalMemory(false)); }, null, 1000, 1000)) + using (var timer = new System.Threading.Timer((_) => { Log("WorkingSet: {0:N0}, Managed: {1:N0}", Environment.WorkingSet, GC.GetTotalMemory(false)); }, null, 1000, 1000)) { try { var result = await db.ReadAsync((tr) => { - Console.WriteLine("Retry #" + tr.Context.Retries + " @ " + tr.Context.ElapsedTotal); + Log("Retry #" + tr.Context.Retries + " @ " + tr.Context.ElapsedTotal); return tr.GetRange(location.Keys.ToRange()).ToListAsync(); }, this.Cancellation); @@ -188,18 +188,18 @@ public async Task Test_Transactionals_Retries_Do_Not_Leak_When_Reading_Too_Much( } } // to help see the effect in a profiler, dispose the transaction first, wait 5 sec then do a full GC, and then wait a bit before exiting the process - Console.WriteLine("Transaction destroyed!"); + Log("Transaction destroyed!"); Thread.Sleep(5000); - Console.WriteLine("Cleaning managed memory"); + Log("Cleaning managed memory"); GC.Collect(); GC.WaitForPendingFinalizers(); GC.Collect(); - Console.WriteLine("Waiting..."); + Log("Waiting..."); Thread.Sleep(5000); - Console.WriteLine("byte"); + Log("byte"); } } @@ -218,7 +218,7 @@ public async Task Test_Transactionals_Should_Not_Execute_If_Already_Canceled() var t = db.ReadAsync((tr) => { called = true; - Console.WriteLine("FAILED"); + Log("FAILED"); throw new InvalidOperationException("Failed"); }, go.Token); diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index 162c6ab58..e112c66d2 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -1379,7 +1379,7 @@ public void Test_Randomized_Splices() tuples[13] = new LinkedTuple(STuple.Create("A", "B", "C", "D", "E", "F", "G", "H", "I", "J", "K", "L"), "M"); #if false - Console.Write("Checking tuples"); + LogPartial("Checking tuples"); foreach (var tuple in tuples) { @@ -1390,9 +1390,10 @@ public void Test_Randomized_Splices() var rnd = new Random(123456); + Log($"Generating {N:N0} random tuples:"); for (int i = 0; i < N; i++) { - if (i % 500 == 0) Console.Write("."); + if (i % 1000 == 0) Log($"- {100.0 * i / N:N1} %"); var len = rnd.Next(tuples.Length); var tuple = tuples[len]; if (tuple.Count != len) @@ -1458,7 +1459,7 @@ public void Test_Randomized_Splices() } } - Log(" done"); + Log("> success"); } @@ -1816,7 +1817,7 @@ public void Bench_Tuple_Unpack_Random() string FUNKY_STRING = "hello\x00world"; string UNICODE_STRING = "héllø 世界"; - Console.Write("Creating {0:N0} random tuples", N); + LogPartial("Creating {0:N0} random tuples", N); var tuples = new List(N); var rnd = new Random(777); var guids = Enumerable.Range(0, 10).Select(_ => Guid.NewGuid()).ToArray(); @@ -1828,7 +1829,7 @@ public void Bench_Tuple_Unpack_Random() { ITuple tuple = STuple.Empty; int s = 1 + (int)Math.Sqrt(rnd.Next(128)); - if (i % (N / 100) == 0) Console.Write("."); + if (i % (N / 100) == 0) LogPartial('.'); for (int j = 0; j < s; j++) { switch (rnd.Next(17)) @@ -1860,7 +1861,7 @@ public void Bench_Tuple_Unpack_Random() Log(" > {0}", tuples[42]); Log(); - Console.Write("Packing tuples..."); + LogPartial("Packing tuples..."); sw.Restart(); var slices = STuple.Pack(tuples); sw.Stop(); @@ -1870,7 +1871,7 @@ public void Bench_Tuple_Unpack_Random() Log(" > {0}", slices[42]); Log(); - Console.Write("Unpacking tuples..."); + LogPartial("Unpacking tuples..."); sw.Restart(); var unpacked = slices.Select(slice => STuple.Unpack(slice)).ToList(); sw.Stop(); @@ -1879,14 +1880,14 @@ public void Bench_Tuple_Unpack_Random() Log(" > {0}", unpacked[42]); Log(); - Console.Write("Comparing ..."); + LogPartial("Comparing ..."); sw.Restart(); tuples.Zip(unpacked, (x, y) => x.Equals(y)).All(b => b); sw.Stop(); Log(" done in {0:N3} sec", sw.Elapsed.TotalSeconds); Log(); - Console.Write("Tuples.ToString ..."); + LogPartial("Tuples.ToString ..."); sw.Restart(); var strings = tuples.Select(x => x.ToString()).ToList(); sw.Stop(); @@ -1895,7 +1896,7 @@ public void Bench_Tuple_Unpack_Random() Log(" > {0}", strings[42]); Log(); - Console.Write("Unpacked.ToString ..."); + LogPartial("Unpacked.ToString ..."); sw.Restart(); strings = unpacked.Select(x => x.ToString()).ToList(); sw.Stop(); @@ -1904,7 +1905,7 @@ public void Bench_Tuple_Unpack_Random() Log(" > {0}", strings[42]); Log(); - Console.Write("Memoizing ..."); + LogPartial("Memoizing ..."); sw.Restart(); var memoized = tuples.Select(x => x.Memoize()).ToList(); sw.Stop(); From e08ea4dde5819395f6614b2fc0587103c657a642 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 13:43:48 +0200 Subject: [PATCH 078/153] Convert projects to .NET Standard 2.0 - FoundationDB.Client, FoundationDB.Layers.Common and FoundationDB.Layers.Experimental are now .NET Standard 2.0 class libraries - the rest of the projects is still .NET Framework (having issues with tests when converted to .NET Standard at the moment) - .exe tools are still .NET Framework console apps for now also. - breaks the build scripts and nuget packages generation, need to asses that --- .../FoundationDB.Client.NET461.csproj | 285 +++++++++++++++++ .../FoundationDB.Client.csproj | 302 ++---------------- .../Properties/AssemblyInfo.cs | 6 +- .../FoundationDB.Layers.Common.NET461.csproj | 82 +++++ .../FoundationDB.Layers.Common.csproj | 103 ++---- .../Properties/AssemblyInfo.cs | 4 - ...dationDB.Layers.Experimental.NET461.csproj | 88 +++++ .../FoundationDB.Layers.Experimental.csproj | 106 ++---- .../Properties/AssemblyInfo.cs | 4 - .../FoundationDB.Linq.Providers.NET461.csproj | 95 ++++++ .../FoundationDB.Linq.Providers.csproj | 119 ++----- .../Properties/AssemblyInfo.cs | 6 +- 12 files changed, 665 insertions(+), 535 deletions(-) create mode 100644 FoundationDB.Client/FoundationDB.Client.NET461.csproj create mode 100644 FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj create mode 100644 FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj create mode 100644 FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj diff --git a/FoundationDB.Client/FoundationDB.Client.NET461.csproj b/FoundationDB.Client/FoundationDB.Client.NET461.csproj new file mode 100644 index 000000000..2abec7a00 --- /dev/null +++ b/FoundationDB.Client/FoundationDB.Client.NET461.csproj @@ -0,0 +1,285 @@ + + + + + Debug + AnyCPU + {773166B7-DE74-4FCC-845C-84080CC89533} + Library + Properties + FoundationDB.Client + FoundationDB.Client + v4.6.1 + 512 + + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + true + false + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + false + true + bin\Release\FoundationDB.Client.XML + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + true + + + ..\Common\foundationdb-net-client.snk + + + + + + + + + Properties\VersionInfo.cs + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 2abec7a00..4959b130b 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -1,285 +1,33 @@ - - - + + - Debug - AnyCPU - {773166B7-DE74-4FCC-845C-84080CC89533} - Library - Properties + netstandard2.0 FoundationDB.Client FoundationDB.Client - v4.6.1 - 512 - + true + ..\Common\foundationdb-net-client.snk + 5.1.0-alpha1 + Doxense + http://opensource.org/licenses/BSD-3-Clause + http://github.com/Doxense/foundationdb-dotnet-client + http://doxense.github.io/foundationdb-dotnet-client/nuget/foundationdb.png + http://github.com/Doxense/foundationdb-dotnet-client + foundationdb fdb nosql + This is a pre-release of the .NET Binding, the public API is still subject to changes. + .NET Binding for FoundationDB + Copyright 2013-2018 Doxense SAS - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 + + true - false - 105,108,109,114,472,660,661,628,1066 - AnyCPU + latest - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - false + + true - bin\Release\FoundationDB.Client.XML - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - true - - - ..\Common\foundationdb-net-client.snk + 105,108,109,114,472,660,661,628,1066,NU1605 + bin\Release\netstandard2.0\FoundationDB.Client.xml + latest - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file + + diff --git a/FoundationDB.Client/Properties/AssemblyInfo.cs b/FoundationDB.Client/Properties/AssemblyInfo.cs index faab1f0f1..c32816960 100644 --- a/FoundationDB.Client/Properties/AssemblyInfo.cs +++ b/FoundationDB.Client/Properties/AssemblyInfo.cs @@ -30,10 +30,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY using System.Runtime.CompilerServices; using System.Runtime.InteropServices; -[assembly: AssemblyTitle("FoundationDB.Client")] -[assembly: AssemblyDescription(".NET Binding for FoundationDB")] -[assembly: AssemblyConfiguration("")] - [assembly: ComVisible(false)] [assembly: Guid("0fce138d-cb61-49fd-bb0a-a0ecb37abe78")] @@ -41,4 +37,4 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY [assembly: InternalsVisibleTo("FoundationDB.Layers.Common, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] [assembly: InternalsVisibleTo("FoundationDB.Layers.Experimental, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] [assembly: InternalsVisibleTo("FoundationDB.Linq.Providers, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] -[assembly: InternalsVisibleTo("FoundationDB.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] \ No newline at end of file +[assembly: InternalsVisibleTo("FoundationDB.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] diff --git a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj new file mode 100644 index 000000000..a7175438c --- /dev/null +++ b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj @@ -0,0 +1,82 @@ + + + + + Debug + AnyCPU + {7C7717D6-A1E7-4541-AF8B-1AC762B5ED0F} + Library + Properties + FoundationDB.Layers.Common + FoundationDB.Layers.Common + v4.6.1 + 512 + + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + false + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + false + bin\Release\FoundationDB.Layers.Common.XML + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + true + + + ..\Common\foundationdb-net-client.snk + + + + + + + + + Properties\VersionInfo.cs + + + + + + + + + + + + + + + + + {773166b7-de74-4fcc-845c-84080cc89533} + FoundationDB.Client + + + + + \ No newline at end of file diff --git a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj index a7175438c..fd8e950aa 100644 --- a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj +++ b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj @@ -1,82 +1,37 @@ - - - + + - Debug - AnyCPU - {7C7717D6-A1E7-4541-AF8B-1AC762B5ED0F} - Library - Properties + netstandard2.0 FoundationDB.Layers.Common FoundationDB.Layers.Common - v4.6.1 - 512 - - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - false - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - false - bin\Release\FoundationDB.Layers.Common.XML - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - true - - ..\Common\foundationdb-net-client.snk + 5.1.0-alpha1 + Doxense + http://opensource.org/licenses/BSD-3-Clause + http://github.com/Doxense/foundationdb-dotnet-client + http://doxense.github.io/foundationdb-dotnet-client/nuget/foundationdb.png + http://github.com/Doxense/foundationdb-dotnet-client + foundationdb fdb nosql + This is a pre-release of the .NET Binding, the public API is still subject to changes. + Common Layers for the FoundationDB .NET Binding + Copyright 2013-2018 Doxense SAS + + + true + latest + + + + true + 105,108,109,114,472,660,661,628,1066,NU1605 + bin\Release\netstandard2.0\FoundationDB.Layers.Common.xml + latest + + - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - + - - - \ No newline at end of file + + diff --git a/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs b/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs index 4646390c0..e8cd9b9ac 100644 --- a/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs +++ b/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs @@ -29,10 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY using System.Reflection; using System.Runtime.InteropServices; -[assembly: AssemblyTitle("FoundationDB.Layers.Common")] -[assembly: AssemblyDescription("Common Layers for the FoundationDB .NET Binding")] -[assembly: AssemblyConfiguration("")] - [assembly: ComVisible(false)] [assembly: Guid("0fce138d-cb61-49fd-bb0a-a0ecb37abe79")] diff --git a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj new file mode 100644 index 000000000..8800e80dc --- /dev/null +++ b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj @@ -0,0 +1,88 @@ + + + + + Debug + AnyCPU + {E631BCD4-386C-4EB1-AD4D-CABCE77BB4C8} + Library + Properties + FoundationDB.Layers + FoundationDB.Layers.Experimental + v4.6.1 + 512 + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + 105,108,109,114,472,660,661,628,1066 + AnyCPU + true + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + bin\Release\FoundationDB.Layers.Experimental.xml + 105,108,109,114,472,660,661,628,1066 + AnyCPU + true + + + true + + + ..\Common\foundationdb-net-client.snk + + + + + + + + + Properties\VersionInfo.cs + + + + + + + + + + + + + + + + + + + + {773166b7-de74-4fcc-845c-84080cc89533} + FoundationDB.Client + + + {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} + FoundationDB.Layers.Common + + + + + \ No newline at end of file diff --git a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj index 8800e80dc..e387400df 100644 --- a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj +++ b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj @@ -1,88 +1,38 @@ - - - + + - Debug - AnyCPU - {E631BCD4-386C-4EB1-AD4D-CABCE77BB4C8} - Library - Properties + netstandard2.0 FoundationDB.Layers FoundationDB.Layers.Experimental - v4.6.1 - 512 + true + ..\Common\foundationdb-net-client.snk + 5.1.0-alpha1 + Doxense + http://opensource.org/licenses/BSD-3-Clause + http://github.com/Doxense/foundationdb-dotnet-client + http://doxense.github.io/foundationdb-dotnet-client/nuget/foundationdb.png + http://github.com/Doxense/foundationdb-dotnet-client + foundationdb fdb nosql + This is a pre-release of the .NET Binding, the public API is still subject to changes. + Experimental Layers for the FoundationDB .NET Binding + Copyright 2013-2018 Doxense SAS - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - 105,108,109,114,472,660,661,628,1066 - AnyCPU + + true + latest - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - bin\Release\FoundationDB.Layers.Experimental.xml - 105,108,109,114,472,660,661,628,1066 - AnyCPU + + true + 105,108,109,114,472,660,661,628,1066,NU1605 + bin\Release\netstandard2.0\FoundationDB.Layers.Common.xml + latest - - true - - - ..\Common\foundationdb-net-client.snk - - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - + - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} - FoundationDB.Layers.Common - + + - - - \ No newline at end of file + + diff --git a/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs b/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs index e86b441dd..8bbe8a266 100644 --- a/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs +++ b/FoundationDB.Layers.Experimental/Properties/AssemblyInfo.cs @@ -1,10 +1,6 @@ using System.Reflection; using System.Runtime.InteropServices; -[assembly: AssemblyTitle("FoundationDB.Layers.Messaging")] -[assembly: AssemblyDescription("")] -[assembly: AssemblyConfiguration("")] - [assembly: ComVisible(false)] [assembly: Guid("6f0b5dad-33c3-43c3-9b4f-6617b0ca965a")] diff --git a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj new file mode 100644 index 000000000..eb329ccd5 --- /dev/null +++ b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj @@ -0,0 +1,95 @@ + + + + + Debug + AnyCPU + {FAF14E3F-6662-4084-8B92-E6697F6B9D5A} + Library + Properties + FoundationDB.Linq + FoundationDB.Linq.Providers + v4.6.1 + 512 + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + bin\Release\FoundationDB.Linq.Providers.xml + 105,108,109,114,472,660,661,628,1066 + AnyCPU + + + true + + + ..\Common\foundationdb-net-client.snk + + + + + + + + + Properties\VersionInfo.cs + + + + + + + + + + + + + + + + + + + + + + + + + + + + {773166b7-de74-4fcc-845c-84080cc89533} + FoundationDB.Client + + + {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} + FoundationDB.Layers.Common + + + + + + \ No newline at end of file diff --git a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj index eb329ccd5..9ce55f3dd 100644 --- a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj +++ b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj @@ -1,95 +1,38 @@ - - - + + - Debug - AnyCPU - {FAF14E3F-6662-4084-8B92-E6697F6B9D5A} - Library - Properties - FoundationDB.Linq + netstandard2.0 + FoundationDB.Linq.Providers FoundationDB.Linq.Providers - v4.6.1 - 512 - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - bin\Release\FoundationDB.Linq.Providers.xml - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - true - - ..\Common\foundationdb-net-client.snk + 5.1.0-alpha1 + Doxense + http://opensource.org/licenses/BSD-3-Clause + http://github.com/Doxense/foundationdb-dotnet-client + http://doxense.github.io/foundationdb-dotnet-client/nuget/foundationdb.png + http://github.com/Doxense/foundationdb-dotnet-client + foundationdb fdb nosql + This is a pre-release of the .NET Binding, the public API is still subject to changes. + Experimental LINQ providers for the FoundationDB .NET Binding + Copyright 2013-2018 Doxense SAS + + + true + latest + + + + true + 105,108,109,114,472,660,661,628,1066,NU1605 + bin\Release\netstandard2.0\FoundationDB.Layers.Common.xml + latest + + - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} - FoundationDB.Layers.Common - + + - - - - \ No newline at end of file + + diff --git a/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs b/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs index 9a9a66e87..ca7155e57 100644 --- a/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs +++ b/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs @@ -30,12 +30,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY using System.Runtime.CompilerServices; using System.Runtime.InteropServices; -[assembly: AssemblyTitle("FoundationDB.Linq")] -[assembly: AssemblyDescription("")] -[assembly: AssemblyConfiguration("")] - [assembly: ComVisible(false)] [assembly: Guid("dfd43b61-0d9b-42d6-bbec-a74eafed2631")] -[assembly: InternalsVisibleTo("FoundationDB.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] \ No newline at end of file +[assembly: InternalsVisibleTo("FoundationDB.Tests, PublicKey=0024000004800000940000000602000000240000525341310004000001000100a9e653303024d91e3e98cdb33228897aebc9aeb0dd5e0890a2362ff08231643525d86e955d52a9be450a9602eedbc1c0eb463d227320a6b6ad1c7129f21353b2b28242d712a0e7b3aaff55c0ab1019c92bea6806b9cf64e93d976143dc57e0a8e73a65c03422ab2624c1220d84f7e88c5a5c3c9edefcf4a76969d458348403ce")] From 2bd6e54a65216538a18eecf464ec435e9bb5cf35 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 17:17:32 +0200 Subject: [PATCH 079/153] Fixed Test_Read_Isolation_From_Writes() by enabling the SnapshotReadYourWriteDisable option - behavior changed between 200 and 300 --- FoundationDB.Tests/TransactionFacts.cs | 70 ++++++++++++++++++++++++-- 1 file changed, 67 insertions(+), 3 deletions(-) diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index ec6329201..6cd69f041 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -1252,10 +1252,72 @@ public async Task Test_Read_Isolation_From_Writes() { // By default: // - Regular reads see the writes made by the transaction itself, but not the writes made by other transactions that committed in between - // - Snapshot reads never see the writes made since the transaction read version, including the writes made by the transaction itself + // - Snapshot reads never see the writes made since the transaction read version, but will see the writes made by the transaction itself - //Fdb.Start(200); // <-- the test passes - //Fdb.Start(300); // <-- the test fails + using (var db = await OpenTestPartitionAsync()) + { + var location = db.Partition.ByKey("test"); + await db.ClearRangeAsync(location, this.Cancellation); + + var A = location.Keys.Encode("A"); + var B = location.Keys.Encode("B"); + var C = location.Keys.Encode("C"); + var D = location.Keys.Encode("D"); + + // Reads (before and after): + // - A and B will use regular reads + // - C and D will use snapshot reads + // Writes: + // - A and C will be modified by the transaction itself + // - B and D will be modified by a different transaction + + await db.WriteAsync((tr) => + { + tr.Set(A, Slice.FromString("a")); + tr.Set(B, Slice.FromString("b")); + tr.Set(C, Slice.FromString("c")); + tr.Set(D, Slice.FromString("d")); + }, this.Cancellation); + + Log("Initial db state:"); + await DumpSubspace(db, location); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + + // check initial state + Assert.That((await tr.GetAsync(A)).ToStringUtf8(), Is.EqualTo("a")); + Assert.That((await tr.GetAsync(B)).ToStringUtf8(), Is.EqualTo("b")); + Assert.That((await tr.Snapshot.GetAsync(C)).ToStringUtf8(), Is.EqualTo("c")); + Assert.That((await tr.Snapshot.GetAsync(D)).ToStringUtf8(), Is.EqualTo("d")); + + // mutate (not yet comitted) + tr.Set(A, Slice.FromString("aa")); + tr.Set(C, Slice.FromString("cc")); + await db.WriteAsync((tr2) => + { // have another transaction change B and D under our nose + tr2.Set(B, Slice.FromString("bb")); + tr2.Set(D, Slice.FromString("dd")); + }, this.Cancellation); + + // check what the transaction sees + Assert.That((await tr.GetAsync(A)).ToStringUtf8(), Is.EqualTo("aa"), "The transaction own writes should change the value of regular reads"); + Assert.That((await tr.GetAsync(B)).ToStringUtf8(), Is.EqualTo("b"), "Other transaction writes should not change the value of regular reads"); + Assert.That((await tr.Snapshot.GetAsync(C)).ToStringUtf8(), Is.EqualTo("cc"), "The transaction own writes should be visible in snapshot reads"); + Assert.That((await tr.Snapshot.GetAsync(D)).ToStringUtf8(), Is.EqualTo("d"), "Other transaction writes should not change the value of snapshot reads"); + + //note: committing here would conflict + } + } + } + + [Test] + public async Task Test_Read_Isolation_From_Writes_Pre_300() + { + // By in API v200 and below: + // - Regular reads see the writes made by the transaction itself, but not the writes made by other transactions that committed in between + // - Snapshot reads never see the writes made since the transaction read version, but will see the writes made by the transaction itself + // In API 300, this can be emulated by setting the SnapshotReadYourWriteDisable options using (var db = await OpenTestPartitionAsync()) { @@ -1287,6 +1349,8 @@ await db.WriteAsync((tr) => using (var tr = db.BeginTransaction(this.Cancellation)) { + tr.SetOption(FdbTransactionOption.SnapshotReadYourWriteDisable); + // check initial state Assert.That((await tr.GetAsync(A)).ToStringUtf8(), Is.EqualTo("a")); Assert.That((await tr.GetAsync(B)).ToStringUtf8(), Is.EqualTo("b")); From db1d417297ea8fdc104f83948e34f1960c76e752 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 17:22:09 +0200 Subject: [PATCH 080/153] Misc cleanup --- FoundationDB.Client/Async/AsyncBuffer.cs | 1 - FoundationDB.Client/Async/AsyncHelpers.cs | 3 +- FoundationDB.Client/Async/AsyncPump.cs | 200 ------------------ FoundationDB.Client/Async/IAsyncPump.cs | 55 ----- FoundationDB.Client/Async/IAsyncSource.cs | 1 - FoundationDB.Client/Fdb.System.cs | 1 - FoundationDB.Client/FdbKey.cs | 1 - .../FdbMergeQueryExtensions.cs | 23 +- FoundationDB.Client/FdbOperationContext.cs | 8 +- .../FdbRangeQuery.PagingIterator.cs | 3 +- .../FdbRangeQuery.ResultIterator.cs | 1 - FoundationDB.Client/FdbRangeQuery.cs | 15 +- FoundationDB.Client/FdbTransaction.cs | 72 +++---- .../FdbTransactionExtensions.cs | 13 +- .../Filters/FdbDatabaseFilter.cs | 38 +--- .../Logging/FdbTransactionLog.Commands.cs | 13 +- .../Layers/Directories/FdbDirectoryLayer.cs | 112 +++++----- FoundationDb.Client.sln.DotSettings | 2 +- 18 files changed, 135 insertions(+), 427 deletions(-) delete mode 100644 FoundationDB.Client/Async/AsyncPump.cs delete mode 100644 FoundationDB.Client/Async/IAsyncPump.cs diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Async/AsyncBuffer.cs index 97b5a6cfc..b034b0421 100644 --- a/FoundationDB.Client/Async/AsyncBuffer.cs +++ b/FoundationDB.Client/Async/AsyncBuffer.cs @@ -36,7 +36,6 @@ namespace Doxense.Async using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Buffer that holds a fixed number of items and can rate-limit the producer diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Async/AsyncHelpers.cs index 5216bb8c5..c18a556e6 100644 --- a/FoundationDB.Client/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Async/AsyncHelpers.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense.Linq; - namespace Doxense.Async { using System; @@ -36,6 +34,7 @@ namespace Doxense.Async using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; /// Helper methods for creating and manipulating async sequences. public static class AsyncHelpers diff --git a/FoundationDB.Client/Async/AsyncPump.cs b/FoundationDB.Client/Async/AsyncPump.cs deleted file mode 100644 index 05f2c0041..000000000 --- a/FoundationDB.Client/Async/AsyncPump.cs +++ /dev/null @@ -1,200 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -//#define FULL_DEBUG - -namespace Doxense.Async -{ - using JetBrains.Annotations; - using System; - using System.Diagnostics; - using System.Runtime.ExceptionServices; - using System.Threading; - using System.Threading.Tasks; - - /// Pumps item from a source, and into a target - public sealed class AsyncPump : IAsyncPump - { - private const int STATE_IDLE = 0; - private const int STATE_WAITING_FOR_NEXT = 1; - private const int STATE_PUBLISHING_TO_TARGET = 2; - private const int STATE_FAILED = 3; - private const int STATE_DONE = 4; - private const int STATE_DISPOSED = 5; - - private volatile int m_state; - private readonly IAsyncSource m_source; - private readonly IAsyncTarget m_target; - - public AsyncPump( - [NotNull] IAsyncSource source, - [NotNull] IAsyncTarget target - ) - { - if (source == null) throw new ArgumentNullException("source"); - if (target == null) throw new ArgumentNullException("target"); - - m_source = source; - m_target = target; - } - - /// Returns true if the pump has completed (with success or failure) - public bool IsCompleted - { - get { return m_state >= STATE_FAILED; } - } - - internal int State - { - get { return m_state; } - } - - public IAsyncSource Source { [NotNull] get { return m_source; } } - - public IAsyncTarget Target { [NotNull] get { return m_target; } } - - /// Run the pump until the inner iterator is done, an error occurs, or the cancellation token is fired - public async Task PumpAsync(bool stopOnFirstError, CancellationToken ct) - { - if (m_state != STATE_IDLE) - { - // either way, we need to stop ! - Exception error; - - if (m_state == STATE_DISPOSED) - { - error = new ObjectDisposedException(null, "Pump has already been disposed"); - } - else if (m_state >= STATE_FAILED) - { - error = new InvalidOperationException("Pump has already completed once"); - } - else - { - error = new InvalidOperationException("Pump is already running"); - } - - try - { - m_target.OnError(ExceptionDispatchInfo.Capture(error)); - } - catch - { - m_target.OnCompleted(); - } - - throw error; - } - - try - { - LogPump("Starting pump"); - - while (!ct.IsCancellationRequested && m_state != STATE_DISPOSED) - { - LogPump("Waiting for next"); - m_state = STATE_WAITING_FOR_NEXT; - var current = await m_source.ReceiveAsync(ct).ConfigureAwait(false); - - LogPump("Received " + (current.HasValue ? "value" : current.Failed ? "error" : "completion") + ", publishing..."); - m_state = STATE_PUBLISHING_TO_TARGET; - - await m_target.Publish(current, ct).ConfigureAwait(false); - - if (current.Failed && stopOnFirstError) - { - m_state = STATE_FAILED; - LogPump("Stopping after this error"); - current.ThrowForNonSuccess(); - } - else if (current.IsEmpty) - { - m_state = STATE_DONE; - LogPump("Completed"); - return; - } - } - - // push the cancellation on the queue, and throw - throw new OperationCanceledException(ct); - } - catch (Exception e) - { - LogPump("Failed " + e.Message); - - switch (m_state) - { - case STATE_WAITING_FOR_NEXT: - { // push the info to the called - try - { - m_target.OnError(ExceptionDispatchInfo.Capture(e)); - } - catch(Exception x) - { - LogPump("Failed to notify target of error: " + x.Message); - throw; - } - break; - } - case STATE_PUBLISHING_TO_TARGET: // the error comes from the target itself, push back to caller! - case STATE_FAILED: // we want to notify the caller of some problem - { - throw; - } - } - } - finally - { - if (m_state != STATE_DISPOSED) - { - m_target.OnCompleted(); - } - LogPump("Stopped pump"); - } - } - - public void Dispose() - { - m_state = STATE_DISPOSED; - } - - #region Debugging... - - [Conditional("FULL_DEBUG")] - private static void LogPump(string msg) - { -#if FULL_DEBUG - Console.WriteLine("[pump#{0}] {1}", Thread.CurrentThread.ManagedThreadId, msg); -#endif - } - - #endregion - } - -} diff --git a/FoundationDB.Client/Async/IAsyncPump.cs b/FoundationDB.Client/Async/IAsyncPump.cs deleted file mode 100644 index 748c1506a..000000000 --- a/FoundationDB.Client/Async/IAsyncPump.cs +++ /dev/null @@ -1,55 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace Doxense.Async -{ - using JetBrains.Annotations; - using System; - using System.Threading; - using System.Threading.Tasks; - - /// Defines a pump that can move items between a source and a target - public interface IAsyncPump : IDisposable - { - /// Source of the pump (that produces new items) - IAsyncSource Source { [NotNull] get; } - - /// Target of the tump (that will consume the items) - IAsyncTarget Target { [NotNull] get; } - - /// True if all the items of the source have been consumed by the target - bool IsCompleted { get; } - - /// Consume all the items of the source by passing them to the Target - /// If true, aborts on the first error. If false, continue processing items until the source has finished. - /// Cancellation token that can be used to abort the pump at any time. Any unprocessed items will be lost. - /// Task that will complete successfully if all the items from the source have been processed by the target, or fails if an error occurred or the pump was cancelled. - Task PumpAsync(bool stopOnFirstError, CancellationToken ct); - } - -} diff --git a/FoundationDB.Client/Async/IAsyncSource.cs b/FoundationDB.Client/Async/IAsyncSource.cs index b3c5e71e7..c4dc1487a 100644 --- a/FoundationDB.Client/Async/IAsyncSource.cs +++ b/FoundationDB.Client/Async/IAsyncSource.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Async { using System; diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 00d0db065..9e38daee9 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -28,7 +28,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //#define TRACE_COUNTING - namespace FoundationDB.Client { using System; diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index e731cb764..9536c0f08 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB { using System; diff --git a/FoundationDB.Client/FdbMergeQueryExtensions.cs b/FoundationDB.Client/FdbMergeQueryExtensions.cs index 4c3a765ba..545d0a85a 100644 --- a/FoundationDB.Client/FdbMergeQueryExtensions.cs +++ b/FoundationDB.Client/FdbMergeQueryExtensions.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; @@ -43,7 +42,7 @@ public static class FdbMergeQueryExtensions #region MergeSort (x OR y) [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable> MergeSort([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -61,7 +60,7 @@ public static IAsyncEnumerable> MergeSort([NotN } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable MergeSort([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -80,7 +79,7 @@ public static IAsyncEnumerable MergeSort([NotNull] this } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Union([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Union([NotNull] IEnumerable> sources, [NotNull] Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); @@ -111,7 +110,7 @@ public static IAsyncEnumerable Union([NotNull] IEnumerable> Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Intersect([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -129,7 +128,7 @@ public static IAsyncEnumerable> Intersect([NotN } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Intersect([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? @@ -144,7 +143,7 @@ public static IAsyncEnumerable Intersect([NotNull] this } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, IAsyncEnumerable second, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, [NotNull] IAsyncEnumerable second, [NotNull] Func keySelector, IComparer keyComparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); @@ -158,7 +157,7 @@ public static IAsyncEnumerable Intersect([NotNull] this } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, IAsyncEnumerable second, IComparer comparer = null) + public static IAsyncEnumerable Intersect([NotNull] this IAsyncEnumerable first, [NotNull] IAsyncEnumerable second, IComparer comparer = null) { Contract.NotNull(first, nameof(first)); Contract.NotNull(second, nameof(second)); @@ -172,7 +171,7 @@ public static IAsyncEnumerable Intersect([NotNull] this IAsync } [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Intersect([NotNull] IEnumerable> sources, Func keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable Intersect([NotNull] IEnumerable> sources, [NotNull] Func keySelector, IComparer keyComparer = null) { Contract.NotNull(sources, nameof(sources)); Contract.NotNull(keySelector, nameof(keySelector)); @@ -210,7 +209,7 @@ public static IAsyncEnumerable Intersect([NotNull] IEnumerable /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, IComparer keyComparer = null) { //TODO: Range options ? Contract.NotNull(trans, nameof(trans)); @@ -235,7 +234,7 @@ public static IAsyncEnumerable> Except([NotNull /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, IComparer keyComparer = null) + public static IAsyncEnumerable> Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, IComparer keyComparer = null) { Contract.NotNull(ranges, nameof(ranges)); return Except(trans, ranges.Select(r => KeySelectorPair.Create(r)), keySelector, keyComparer); @@ -251,7 +250,7 @@ public static IAsyncEnumerable> Except([NotNull /// Instance used to compare the keys returned by /// Async query that returns only the results that are in the first range, and not in any other range. [Pure, NotNull, LinqTunnel] - public static IAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, IEnumerable ranges, Func, TKey> keySelector, Func, TResult> resultSelector, IComparer keyComparer = null) + public static IAsyncEnumerable Except([NotNull] this IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ranges, [NotNull] Func, TKey> keySelector, [NotNull] Func, TResult> resultSelector, IComparer keyComparer = null) { //TODO: Range options ? diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index 088a12ed9..aee9f0e20 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense.Async; - namespace FoundationDB.Client { using System; @@ -35,6 +33,7 @@ namespace FoundationDB.Client using System.Globalization; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; @@ -239,10 +238,7 @@ internal static async Task ExecuteInternal([NotNull] IFdbDatabase db, [NotNull] public void Dispose() { this.Abort = true; - if (this.TokenSource != null) - { - this.TokenSource.SafeCancelAndDispose(); - } + this.TokenSource?.SafeCancelAndDispose(); } #region Read-Only operations... diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 7b9d3afe9..18ea35048 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -29,8 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //enable this to enable verbose traces when doing paging //#define DEBUG_RANGE_PAGING -using Doxense.Linq; - namespace FoundationDB.Client { using System; @@ -40,6 +38,7 @@ namespace FoundationDB.Client using System.Threading.Tasks; using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Linq; using Doxense.Linq.Async.Iterators; using JetBrains.Annotations; diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 83db267ab..1f4e5c6a0 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -29,7 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY //enable this to enable verbose traces when doing paging //#define DEBUG_RANGE_ITERATOR - namespace FoundationDB.Client { using System; diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index 8703e8e28..39cbaff8c 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -26,14 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System.Threading; - namespace FoundationDB.Client { using System; using System.Collections.Generic; using System.Diagnostics; using System.Globalization; + using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; @@ -120,9 +119,9 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption /// Maximum number of results to return /// A new query object that will only return up to results when executed [NotNull] - public FdbRangeQuery Take(int count) + public FdbRangeQuery Take([Positive] int count) { - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Value cannot be less than zero"); + Contract.Positive(count, nameof(count)); if (this.Options.Limit == count) { @@ -139,9 +138,9 @@ public FdbRangeQuery Take(int count) /// /// A new query object that will skip the first results when executed [NotNull] - public FdbRangeQuery Skip(int count) + public FdbRangeQuery Skip([Positive] int count) { - if (count < 0) throw new ArgumentOutOfRangeException("count", count, "Value cannot be less than zero"); + Contract.Positive(count, nameof(count)); var limit = this.Options.Limit; var begin = this.Begin; @@ -272,9 +271,9 @@ public IAsyncEnumerator GetEnumerator() return this.GetEnumerator(this.Transaction.Cancellation, AsyncIterationHint.Default); } - public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint mode) + public IAsyncEnumerator GetEnumerator(CancellationToken ct, AsyncIterationHint hint) { - return new ResultIterator(this, this.Transaction, this.Transform).GetEnumerator(ct, mode); + return new ResultIterator(this, this.Transaction, this.Transform).GetEnumerator(ct, hint); } /// Return a list of all the elements of the range results diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index b39e7792f..583c29f19 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -29,8 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY // enable this to help debug Transactions //#define DEBUG_TRANSACTIONS -using Doxense.Async; - namespace FoundationDB.Client { using System; @@ -38,6 +36,7 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; using FoundationDB.Client.Core; using FoundationDB.Client.Native; @@ -46,7 +45,7 @@ namespace FoundationDB.Client /// FounrationDB transaction handle. /// An instance of this class can be used to read from and/or write to a snapshot of a FoundationDB database. [DebuggerDisplay("Id={Id}, StillAlive={StillAlive}, Size={Size}")] - public sealed partial class FdbTransaction : IFdbTransaction, IFdbReadOnlyTransaction + public sealed partial class FdbTransaction : IFdbTransaction { #region Private Members... @@ -117,10 +116,10 @@ internal FdbTransaction(FdbDatabase db, FdbOperationContext context, int id, IFd /// Internal local identifier of the transaction /// Should only used for logging/debugging purpose. - public int Id { get { return m_id; } } + public int Id => m_id; /// Always returns false. Use the property to get a different view of this transaction that will perform snapshot reads. - public bool IsSnapshot { get { return false; } } + public bool IsSnapshot => false; /// Returns the context of this transaction public FdbOperationContext Context @@ -144,22 +143,19 @@ internal IFdbTransactionHandler Handler } /// If true, the transaction is still pending (not committed or rolledback). - internal bool StillAlive { get { return this.State == STATE_READY; } } + internal bool StillAlive => this.State == STATE_READY; /// Estimated size of the transaction payload (in bytes) - public int Size { get { return m_handler.Size; } } + public int Size => m_handler.Size; /// Cancellation Token that is cancelled when the transaction is disposed - public CancellationToken Cancellation { get { return m_cancellation; } } + public CancellationToken Cancellation => m_cancellation; /// Returns true if this transaction only supports read operations, or false if it supports both read and write operations - public bool IsReadOnly { get { return m_readOnly; } } + public bool IsReadOnly => m_readOnly; /// Returns the isolation level of this transaction. - public FdbIsolationLevel IsolationLevel - { - get { return m_handler.IsolationLevel; } - } + public FdbIsolationLevel IsolationLevel => m_handler.IsolationLevel; #endregion @@ -174,10 +170,10 @@ public FdbIsolationLevel IsolationLevel /// The transaction can be used again after it is reset. public int Timeout { - get { return m_timeout; } + get => m_timeout; set { - if (value < 0) throw new ArgumentOutOfRangeException("value", value, "Timeout value cannot be negative"); + if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "Timeout value cannot be negative"); SetOption(FdbTransactionOption.Timeout, value); m_timeout = value; } @@ -189,10 +185,10 @@ public int Timeout /// public int RetryLimit { - get { return m_retryLimit; } + get => m_retryLimit; set { - if (value < -1) throw new ArgumentOutOfRangeException("value", value, "Retry count cannot be negative"); + if (value < -1) throw new ArgumentOutOfRangeException(nameof(value), value, "Retry count cannot be negative"); SetOption(FdbTransactionOption.RetryLimit, value); m_retryLimit = value; } @@ -204,10 +200,10 @@ public int RetryLimit /// public int MaxRetryDelay { - get { return m_maxRetryDelay; } + get => m_maxRetryDelay; set { - if (value < 0) throw new ArgumentOutOfRangeException("value", value, "Max retry delay cannot be negative"); + if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "Max retry delay cannot be negative"); SetOption(FdbTransactionOption.MaxRetryDelay, value); m_maxRetryDelay = value; } @@ -221,7 +217,7 @@ public void SetOption(FdbTransactionOption option) { EnsureNotFailedOrDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting transaction option {0}", option.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting transaction option {option.ToString()}"); m_handler.SetOption(option, Slice.Nil); } @@ -233,7 +229,7 @@ public void SetOption(FdbTransactionOption option, string value) { EnsureNotFailedOrDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting transaction option {0} to '{1}'", option.ToString(), value ?? "")); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting transaction option {option.ToString()} to '{value ?? ""}'"); var data = FdbNative.ToNativeString(value, nullTerminated: true); m_handler.SetOption(option, data); @@ -246,7 +242,7 @@ public void SetOption(FdbTransactionOption option, long value) { EnsureNotFailedOrDisposed(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting transaction option {0} to {1}", option.ToString(), value)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting transaction option {option.ToString()} to {value}"); // Spec says: "If the option is documented as taking an Int parameter, value must point to a signed 64-bit integer (little-endian), and value_length must be 8." var data = Slice.FromFixed64(value); @@ -308,7 +304,7 @@ public Task GetAsync(Slice key) m_database.EnsureKeyIsValid(ref key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", String.Format("Getting value for '{0}'", key.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", $"Getting value for '{key.ToString()}'"); #endif return m_handler.GetAsync(key, snapshot: false, ct: m_cancellation); @@ -323,7 +319,7 @@ public Task GetAsync(Slice key) /// public Task GetValuesAsync(Slice[] keys) { - if (keys == null) throw new ArgumentNullException("keys"); + if (keys == null) throw new ArgumentNullException(nameof(keys)); //TODO: should we make a copy of the key array ? EnsureCanRead(); @@ -331,7 +327,7 @@ public Task GetValuesAsync(Slice[] keys) m_database.EnsureKeysAreValid(keys); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", String.Format("Getting batch of {0} values ...", keys.Length)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", $"Getting batch of {keys.Length} values ..."); #endif return m_handler.GetValuesAsync(keys, snapshot: false, ct: m_cancellation); @@ -380,7 +376,7 @@ internal FdbRangeQuery> GetRangeCore(KeySelector begi options.EnsureLegalValues(); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetRangeCore", String.Format("Getting range '{0} <= x < {1}'", begin.ToString(), end.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetRangeCore", $"Getting range '{begin.ToString()} <= x < {end.ToString()}'"); #endif return new FdbRangeQuery>(this, begin, end, (kv) => kv, snapshot, options); @@ -414,7 +410,7 @@ public async Task GetKeyAsync(KeySelector selector) m_database.EnsureKeyIsValid(selector.Key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", String.Format("Getting key '{0}'", selector.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", $"Getting key '{selector.ToString()}'"); #endif var key = await m_handler.GetKeyAsync(selector, snapshot: false, ct: m_cancellation).ConfigureAwait(false); @@ -442,7 +438,7 @@ public Task GetKeysAsync(KeySelector[] selectors) } #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysAsync", String.Format("Getting batch of {0} keys ...", selectors.Length)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysAsync", $"Getting batch of {selectors.Length} keys ..."); #endif return m_handler.GetKeysAsync(selectors, snapshot: false, ct: m_cancellation); @@ -466,7 +462,7 @@ public void Set(Slice key, Slice value) m_database.EnsureValueIsValid(ref value); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Set", String.Format("Setting '{0}' = {1}", FdbKey.Dump(key), Slice.Dump(value))); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Set", $"Setting '{FdbKey.Dump(key)}' = {Slice.Dump(value)}"); #endif m_handler.Set(key, value); @@ -541,7 +537,7 @@ public void Atomic(Slice key, Slice param, FdbMutationType mutation) EnsureMutationTypeIsSupported(mutation, Fdb.ApiVersion); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "AtomicCore", String.Format("Atomic {0} on '{1}' = {2}", mutation.ToString(), FdbKey.Dump(key), Slice.Dump(param))); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "AtomicCore", $"Atomic {mutation.ToString()} on '{FdbKey.Dump(key)}' = {Slice.Dump(param)}"); #endif m_handler.Atomic(key, param, mutation); @@ -562,7 +558,7 @@ public void Clear(Slice key) m_database.EnsureKeyIsValid(ref key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Clear", String.Format("Clearing '{0}'", FdbKey.Dump(key))); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "Clear", $"Clearing '{FdbKey.Dump(key)}'"); #endif m_handler.Clear(key); @@ -586,7 +582,7 @@ public void ClearRange(Slice beginKeyInclusive, Slice endKeyExclusive) m_database.EnsureKeyIsValid(ref endKeyExclusive, endExclusive: true); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "ClearRange", String.Format("Clearing Range '{0}' <= k < '{1}'", beginKeyInclusive.ToString(), endKeyExclusive.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "ClearRange", $"Clearing Range '{beginKeyInclusive.ToString()}' <= k < '{endKeyExclusive.ToString()}'"); #endif m_handler.ClearRange(beginKeyInclusive, endKeyExclusive); @@ -632,7 +628,7 @@ public Task GetAddressesForKeyAsync(Slice key) m_database.EnsureKeyIsValid(ref key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAddressesForKeyAsync", String.Format("Getting addresses for key '{0}'", FdbKey.Dump(key))); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAddressesForKeyAsync", $"Getting addresses for key '{FdbKey.Dump(key)}'"); #endif return m_handler.GetAddressesForKeyAsync(key, ct: m_cancellation); @@ -789,7 +785,7 @@ public void Cancel() 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)); + default: throw new InvalidOperationException($"Cannot cancel transaction because it is in unknown state {state}"); } } @@ -807,7 +803,7 @@ public void Cancel() /// Get/Sets the internal state of the exception internal int State { - get { return Volatile.Read(ref m_state); } + get => Volatile.Read(ref m_state); set { Contract.Requires(value >= STATE_DISPOSED && value <= STATE_FAILED, "Invalid state value"); @@ -896,7 +892,7 @@ internal static void ThrowOnInvalidState(FdbTransaction trans) case STATE_FAILED: throw new InvalidOperationException("The transaction is in a failed state and cannot be used anymore"); case STATE_COMMITTED: throw new InvalidOperationException("The transaction has already been committed"); case STATE_CANCELED: throw new FdbException(FdbError.TransactionCancelled, "The transaction has already been cancelled"); - default: throw new InvalidOperationException(String.Format("The transaction is unknown state {0}", trans.State)); + default: throw new InvalidOperationException($"The transaction is unknown state {trans.State}"); } } @@ -920,7 +916,7 @@ public void Dispose() this.Database.UnregisterTransaction(this); m_cts.SafeCancelAndDispose(); - if (Logging.On) Logging.Verbose(this, "Dispose", String.Format("Transaction #{0} has been disposed", m_id)); + if (Logging.On) Logging.Verbose(this, "Dispose", $"Transaction #{m_id} has been disposed"); } finally { @@ -930,7 +926,7 @@ public void Dispose() try { m_handler.Dispose(); } catch(Exception e) { - if (Logging.On) Logging.Error(this, "Dispose", String.Format("Transaction #{0} failed to dispose the transaction handler: {1}", m_id, e.Message)); + if (Logging.On) Logging.Error(this, "Dispose", $"Transaction #{m_id} failed to dispose the transaction handler: [{e.GetType().Name}] {e.Message}"); } } if (!m_context.Shared) m_context.Dispose(); diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 07cf4394b..5191778fa 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; @@ -415,7 +414,7 @@ public static void AtomicMax([NotNull] this IFdbTransaction trans, Slice key, Sl } /// Modify the database snapshot represented by this transaction to update a value if it is smaller than the value in the database. - /// Transaction instance + /// Transaction to use for the operation /// Name of the key whose value is to be mutated. /// Bit mask. public static void AtomicMin([NotNull] this IFdbTransaction trans, Slice key, Slice value) @@ -438,12 +437,12 @@ public static FdbRangeQuery> GetRange([NotNull] this public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeyRange range, FdbRangeOptions options = null) { - return FdbTransactionExtensions.GetRange(trans, KeySelectorPair.Create(range), options); + return GetRange(trans, KeySelectorPair.Create(range), options); } public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, KeyRange range, int limit, bool reverse = false) { - return FdbTransactionExtensions.GetRange(trans, range, new FdbRangeOptions(limit: limit, reverse: reverse)); + return GetRange(trans, range, new FdbRangeOptions(limit: limit, reverse: reverse)); } public static FdbRangeQuery> GetRange([NotNull] this IFdbReadOnlyTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive, FdbRangeOptions options = null) @@ -547,7 +546,7 @@ public static void ClearRange([NotNull] this IFdbTransaction trans, KeyRange ran { Contract.NotNull(trans, nameof(trans)); - trans.ClearRange(range.Begin, range.End); + trans.ClearRange(range.Begin, range.End.HasValue ? range.End : FdbKey.MaxValue); } #endregion @@ -564,7 +563,7 @@ public static void AddConflictRange([NotNull] this IFdbTransaction trans, KeyRan { Contract.NotNull(trans, nameof(trans)); - trans.AddConflictRange(range.Begin, range.End, type); + trans.AddConflictRange(range.Begin, range.End.HasValue ? range.End : FdbKey.MaxValue, type); } @@ -607,7 +606,7 @@ public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, K /// public static void AddWriteConflictRange([NotNull] this IFdbTransaction trans, Slice beginKeyInclusive, Slice endKeyExclusive) { - if (trans == null) throw new ArgumentNullException("trans"); + Contract.NotNull(trans, nameof(trans)); trans.AddConflictRange(beginKeyInclusive, endKeyExclusive, FdbConflictRangeType.Write); } diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index 694086804..6c19ba85d 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -77,11 +77,8 @@ protected FdbDatabaseFilter([NotNull] IFdbDatabase database, bool forceReadOnly, #region Public Properties... /// Database instance configured to read and write data from this partition - protected IFdbDatabase Database - { - [NotNull] - get { return m_database; } - } + [NotNull] + protected IFdbDatabase Database => m_database; [NotNull] internal IFdbDatabase GetInnerDatabase() @@ -90,31 +87,19 @@ internal IFdbDatabase GetInnerDatabase() } /// Name of the database - public string Name - { - get { return m_database.Name; } - } + public string Name => m_database.Name; /// Cluster of the database - public virtual IFdbCluster Cluster - { - //REVIEW: do we need a Cluster Filter ? - [NotNull] - get { return m_database.Cluster; } - } + [NotNull] + public virtual IFdbCluster Cluster => m_database.Cluster; + //REVIEW: do we need a Cluster Filter ? /// Returns a cancellation token that is linked with the lifetime of this database instance - public CancellationToken Cancellation - { - get { return m_database.Cancellation; } - } + public CancellationToken Cancellation => m_database.Cancellation; /// Returns the global namespace used by this database instance - public virtual IDynamicKeySubspace GlobalSpace - { - [NotNull] - get { return m_database.GlobalSpace; } - } + [NotNull] + public virtual IDynamicKeySubspace GlobalSpace => m_database.GlobalSpace; /// Directory partition of this database instance public virtual FdbDatabasePartition Directory @@ -130,10 +115,7 @@ public virtual FdbDatabasePartition Directory } /// If true, this database instance will only allow starting read-only transactions. - public virtual bool IsReadOnly - { - get { return m_readOnly; } - } + public virtual bool IsReadOnly => m_readOnly; Slice IKeySubspace.GetPrefix() { diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index fc8e1df1b..6439cb75e 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense; - namespace FoundationDB.Filters.Logging { using System; @@ -36,6 +34,7 @@ namespace FoundationDB.Filters.Logging using System.Globalization; using System.Text; using System.Threading.Tasks; + using Doxense; using FoundationDB.Client; using FoundationDB.Layers.Directories; using JetBrains.Annotations; @@ -465,7 +464,7 @@ public override int? ArgumentBytes public override string GetArguments(KeyResolver resolver) { - return String.Concat(resolver.Resolve(this.Key), " = ", this.Value.PrettyPrint()); + return String.Concat(resolver.Resolve(this.Key), " = ", this.Value.ToString("K")); } } @@ -547,7 +546,7 @@ public override int? ArgumentBytes public override string GetArguments(KeyResolver resolver) { - return String.Concat(resolver.Resolve(this.Key), " ", this.Mutation.ToString(), " ", this.Param.PrettyPrint()); + return String.Concat(resolver.Resolve(this.Key), " ", this.Mutation.ToString(), " ", this.Param.ToString("K")); } public override string ToString(KeyResolver resolver) @@ -630,7 +629,7 @@ public override string GetResult(KeyResolver resolver) protected override string Dump(Slice value) { - return value.PrettyPrint(); + return value.ToString("P"); } } @@ -712,8 +711,8 @@ public override string GetResult(KeyResolver resolver) if (!this.Result.HasValue) return base.GetResult(resolver); var res = this.Result.Value; string s = String.Concat("[", res.Length.ToString(), "] {"); - if (res.Length > 0) s += res[0].PrettyPrint(); - if (res.Length > 1) s += " ... " + res[res.Length - 1].PrettyPrint(); + if (res.Length > 0) s += res[0].ToString("P"); + if (res.Length > 1) s += " ... " + res[res.Length - 1].ToString("P"); return s + " }"; } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index f08d7b9c4..04394e701 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -112,7 +112,7 @@ void IFdbDirectory.CheckLayer(Slice layer) { if (layer.IsPresent) { - throw new InvalidOperationException($"The directory layer {this.FullName} is not compatible with layer {layer:P}."); + throw new InvalidOperationException($"The directory layer {this.FullName} is not compatible with layer {layer:K}."); } } @@ -176,7 +176,7 @@ public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = [NotNull] public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); var location = path != null ? ParsePath(path) : STuple.Empty; var space = subspace.Using(TypeSystem.Tuples); @@ -190,8 +190,8 @@ public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null) { - if (nodeSubspace == null) throw new ArgumentNullException("nodeSubspace"); - if (contentSubspace == null) throw new ArgumentNullException("contentSubspace"); + if (nodeSubspace == null) throw new ArgumentNullException(nameof(nodeSubspace)); + if (contentSubspace == null) throw new ArgumentNullException(nameof(contentSubspace)); var location = path != null ? ParsePath(path) : STuple.Empty; //TODO: check that nodeSubspace != contentSubspace? @@ -211,8 +211,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemNotNull] public Task CreateOrOpenAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(null, trans, ParsePath(path), layer, Slice.Nil, allowCreate: true, allowOpen: true, throwOnError: true); } @@ -226,8 +226,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemNotNull] public Task OpenAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(trans, null, ParsePath(path), layer, prefix: Slice.Nil, allowCreate: false, allowOpen: true, throwOnError: true); } @@ -241,8 +241,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemNotNull] public Task CreateAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(null, trans, ParsePath(path), layer, prefix: Slice.Nil, allowCreate: true, allowOpen: false, throwOnError: true); } @@ -254,8 +254,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemCanBeNull] public Task TryOpenAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(trans, null, ParsePath(path), layer, prefix: Slice.Nil, allowCreate: false, allowOpen: true, throwOnError: false); } @@ -267,8 +267,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemCanBeNull] public Task TryCreateAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(null, trans, ParsePath(path), layer, prefix: Slice.Nil, allowCreate: true, allowOpen: false, throwOnError: false); } @@ -281,8 +281,8 @@ public static FdbDirectoryLayer Create(IDynamicKeySubspace nodeSubspace, IDynami [ItemNotNull] public Task RegisterAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer, Slice prefix) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(null, trans, ParsePath(path), layer, prefix: prefix, allowCreate: true, allowOpen: false, throwOnError: true); } @@ -295,8 +295,8 @@ public Task RegisterAsync([NotNull] IFdbTransaction trans, [ItemCanBeNull] public Task TryRegisterAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer, Slice prefix) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return CreateOrOpenInternalAsync(null, trans, ParsePath(path), layer, prefix: prefix, allowCreate: true, allowOpen: false, throwOnError: false); } @@ -315,9 +315,9 @@ public Task TryRegisterAsync([NotNull] IFdbTransaction tra [ItemNotNull] public Task MoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) { - if (trans == null) throw new ArgumentNullException("trans"); - if (oldPath == null) throw new ArgumentNullException("oldPath"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (oldPath == null) throw new ArgumentNullException(nameof(oldPath)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); var oldLocation = STuple.FromEnumerable(oldPath); VerifyPath(oldLocation, "oldPath"); @@ -337,9 +337,9 @@ public Task MoveAsync(IFdbTransaction trans, IEnumerable TryMoveAsync(IFdbTransaction trans, IEnumerable oldPath, IEnumerable newPath) { - if (trans == null) throw new ArgumentNullException("trans"); - if (oldPath == null) throw new ArgumentNullException("oldPath"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (oldPath == null) throw new ArgumentNullException(nameof(oldPath)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); var oldLocation = STuple.FromEnumerable(oldPath); VerifyPath(oldLocation, "oldPath"); @@ -374,8 +374,8 @@ Task IFdbDirectory.TryMoveToAsync(IFdbTransaction trans, I /// Path of the directory to remove (including any subdirectories) public Task RemoveAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return RemoveInternalAsync(trans, ParsePath(path), throwIfMissing: true); } @@ -387,8 +387,8 @@ public Task RemoveAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerablePath of the directory to remove (including any subdirectories) public Task TryRemoveAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); var location = ParsePath(path); if (location.Count == 0) throw new NotSupportedException("Cannot remove a directory layer"); @@ -405,8 +405,8 @@ public Task TryRemoveAsync([NotNull] IFdbTransaction trans, [NotNull] IEnu /// Returns true if the directory exists, otherwise false. public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); // no reason to disallow checking for the root directory (could be used to check if a directory layer is initialized?) var location = ParsePath(path); @@ -425,7 +425,7 @@ public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] [ItemNotNull] public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return ListInternalAsync(trans, ParsePath(path), throwIfMissing: true); } @@ -435,7 +435,7 @@ public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans, [No [ItemNotNull] public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return ListInternalAsync(trans, STuple.Empty, throwIfMissing: true); } @@ -446,7 +446,7 @@ public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans) [ItemCanBeNull] public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return ListInternalAsync(trans, ParsePath(path), throwIfMissing: false); } @@ -454,7 +454,7 @@ public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans, [ItemCanBeNull] public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return ListInternalAsync(trans, STuple.Empty, throwIfMissing: false); } @@ -467,8 +467,8 @@ public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans) [ItemCanBeNull] public async Task ChangeLayerAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice newLayer) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); var location = ParsePath(path); @@ -574,7 +574,7 @@ internal IReadOnlyList ToAbsolutePath([NotNull] ITuple path) [NotNull] internal ITuple ToRelativePath([NotNull] ITuple path) { - if (path == null) throw new ArgumentNullException("path"); + if (path == null) throw new ArgumentNullException(nameof(path)); if (!path.StartsWith(this.Location)) throw new InvalidOperationException("The path cannot be outside of this partition"); return path.Substring(this.Location.Count); @@ -614,20 +614,20 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly if (!allowOpen) { - if (throwOnError) throw new InvalidOperationException(string.Format("The directory {0} already exists.", path)); + if (throwOnError) throw new InvalidOperationException($"The directory {path} already exists."); return null; } if (layer.IsPresent && layer != existingNode.Layer) { - throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", path, layer.PrettyPrint(), existingNode.Layer.PrettyPrint())); + throw new InvalidOperationException($"The directory {path} was created with incompatible layer {layer:P} instead of expected {existingNode.Layer:P}."); } return ContentsOfNode(existingNode.Subspace, path, existingNode.Layer); } if (!allowCreate) { - if (throwOnError) throw new InvalidOperationException(string.Format("The directory {0} does not exist.", path)); + if (throwOnError) throw new InvalidOperationException($"The directory {path} does not exist."); return null; } @@ -645,7 +645,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly if (FdbDirectoryLayer.AnnotateTransactions) trans.Annotate("Ensure that there is no data already present under prefix {0}", prefix); if (await trans.GetRange(KeyRange.StartsWith(prefix)).AnyAsync().ConfigureAwait(false)) { - throw new InvalidOperationException(String.Format("The database has keys stored at the prefix chosen by the automatic prefix allocator: {0}", prefix.PrettyPrint())); + throw new InvalidOperationException($"The database has keys stored at the prefix chosen by the automatic prefix allocator: {prefix:K}"); } // ensure that the prefix has not already been allocated @@ -676,7 +676,7 @@ internal async Task CreateOrOpenInternalAsync(IFdbReadOnly { parentNode = this.RootNode; } - if (parentNode == null) throw new InvalidOperationException(string.Format("The parent directory of {0} doesn't exist.", path)); + if (parentNode == null) throw new InvalidOperationException($"The parent directory of {path} doesn't exist."); // initialize the metadata for this new directory var node = NodeWithPrefix(prefix); @@ -702,7 +702,7 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa } if (newPath.StartsWith(oldPath)) { - throw new InvalidOperationException(string.Format("The destination directory({0}) cannot be a subdirectory of the source directory({1}).", newPath, oldPath)); + throw new InvalidOperationException($"The destination directory({newPath}) cannot be a subdirectory of the source directory({oldPath})."); } await CheckWriteVersionAsync(trans).ConfigureAwait(false); @@ -710,7 +710,7 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa var oldNode = await FindAsync(trans, oldPath).ConfigureAwait(false); if (!oldNode.Exists) { - if (throwOnError) throw new InvalidOperationException(string.Format("The source directory '{0}' does not exist.", oldPath)); + if (throwOnError) throw new InvalidOperationException($"The source directory '{oldPath}' does not exist."); return null; } @@ -729,14 +729,14 @@ internal async Task MoveInternalAsync([NotNull] IFdbTransa if (newNode.Exists) { - if (throwOnError) throw new InvalidOperationException(string.Format("The destination directory '{0}' already exists. Remove it first.", newPath)); + if (throwOnError) throw new InvalidOperationException($"The destination directory '{newPath}' already exists. Remove it first."); return null; } var parentNode = await FindAsync(trans, newPath.Substring(0, newPath.Count - 1)).ConfigureAwait(false); if (!parentNode.Exists) { - if (throwOnError) throw new InvalidOperationException(string.Format("The parent of the destination directory '{0}' does not exist. Create it first.", newPath)); + if (throwOnError) throw new InvalidOperationException($"The parent of the destination directory '{newPath}' does not exist. Create it first."); return null; } @@ -759,7 +759,7 @@ internal async Task RemoveInternalAsync([NotNull] IFdbTransaction trans, [ var n = await FindAsync(trans, path).ConfigureAwait(false); if (!n.Exists) { - if (throwIfMissing) throw new InvalidOperationException(string.Format("The directory '{0}' does not exist.", path)); + if (throwIfMissing) throw new InvalidOperationException($"The directory '{path}' does not exist."); return false; } @@ -788,7 +788,7 @@ internal async Task> ListInternalAsync([NotNull] IFdbReadOnlyTransa if (!node.Exists) { - if (throwIfMissing) throw new InvalidOperationException(string.Format("The directory '{0}' does not exist.", path)); + if (throwIfMissing) throw new InvalidOperationException($"The directory '{path}' does not exist."); return null; } @@ -831,7 +831,7 @@ internal async Task ChangeLayerInternalAsync([NotNull] IFdbTransaction trans, [N if (!node.Exists) { - throw new InvalidOperationException(string.Format("The directory '{0}' does not exist, or as already been removed.", path)); + throw new InvalidOperationException($"The directory '{path}' does not exist, or as already been removed."); } if (node.IsInPartition(includeEmptySubPath: false)) @@ -1076,7 +1076,7 @@ private static Slice GetSubDirKey([NotNull] IDynamicKeySubspace parent, [NotNull [NotNull] public static string[] ParsePath([NotNull] ITuple path) { - if (path == null) throw new ArgumentNullException("path"); + if (path == null) throw new ArgumentNullException(nameof(path)); var tmp = new string[path.Count]; for (int i = 0; i < tmp.Length; i++) { @@ -1088,23 +1088,23 @@ public static string[] ParsePath([NotNull] ITuple path) [NotNull] public static string[] Combine([NotNull] IEnumerable parent, string path) { - if (parent == null) throw new ArgumentNullException("parent"); + if (parent == null) throw new ArgumentNullException(nameof(parent)); return parent.Concat(new[] { path }).ToArray(); } [NotNull] public static string[] Combine(IEnumerable parent, params string[] paths) { - if (parent == null) throw new ArgumentNullException("parent"); - if (paths == null) throw new ArgumentNullException("paths"); + if (parent == null) throw new ArgumentNullException(nameof(parent)); + if (paths == null) throw new ArgumentNullException(nameof(paths)); return parent.Concat(paths).ToArray(); } [NotNull] public static string[] Combine([NotNull] IEnumerable parent, [NotNull] IEnumerable paths) { - if (parent == null) throw new ArgumentNullException("parent"); - if (paths == null) throw new ArgumentNullException("paths"); + if (parent == null) throw new ArgumentNullException(nameof(parent)); + if (paths == null) throw new ArgumentNullException(nameof(paths)); return parent.Concat(paths).ToArray(); } @@ -1159,7 +1159,7 @@ public static string[] Parse(string path) [NotNull] public static string FormatPath([NotNull] IEnumerable paths) { - if (paths == null) throw new ArgumentNullException("paths"); + if (paths == null) throw new ArgumentNullException(nameof(paths)); return String.Join("/", paths.Select(path => { diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index 8727ec4ce..b8937379c 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -64,7 +64,7 @@ <Policy Inspect="True" Prefix="" Suffix="" Style="aaBb"><ExtraRule Prefix="_" Suffix="" Style="aaBb" /></Policy> <Policy Inspect="True" Prefix="" Suffix="" Style="aaBb"><ExtraRule Prefix="_" Suffix="" Style="aaBb" /></Policy> <Policy Inspect="True" Prefix="" Suffix="" Style="AaBb"><ExtraRule Prefix="" Suffix="" Style="AA_BB" /></Policy> - <Policy Inspect="True" Prefix="m_" Suffix="" Style="aaBb" /> + <Policy Inspect="True" Prefix="m_" Suffix="" Style="aaBb"><ExtraRule Prefix="" Suffix="" Style="AaBb" /></Policy> <Policy Inspect="True" Prefix="s_" Suffix="" Style="aaBb" /> <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> From 012146dd7191e6aebdcdbf6724c55bf274f124a2 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 17:47:47 +0200 Subject: [PATCH 081/153] [BREAKING CHANGE] Landed the new API for KeySubspaces and partitions - Made change to dynamic and typed subspaces to simplify the API - IKeySubspace.ConcatKey(..) is now an indexer on the interface - Keys and Partitions are now singleton (subspace are supposed to be constructed once and reused multiple time) - Removed a lot of overloads (may need to add them back via extension methods later) - Better support for creating ranges with composite keys - Better support for serializing partial keys --- FoundationDB.Client/Fdb.cs | 2 +- FoundationDB.Client/FdbDatabase.cs | 56 +- FoundationDB.Client/FdbRangeChunk.cs | 1 + .../FdbTransactionExtensions.cs | 1 + .../Filters/FdbDatabaseFilter.cs | 46 +- .../Filters/Logging/FdbLoggedTransaction.cs | 7 +- .../Filters/PrefixRewriterTransaction.cs | 21 +- .../Layers/Directories/FdbDirectoryLayer.cs | 7 +- .../Directories/FdbDirectoryPartition.cs | 18 +- .../Directories/FdbDirectorySubspace.cs | 100 +- .../Layers/Tuples/Encoding/TupleCodec`1.cs | 1 + .../Layers/Tuples/Encoding/TupleEncoder.cs | 4 +- .../Layers/Tuples/Encoding/TupleKeyEncoder.cs | 9 +- .../Tuples/Encoding/TupleKeyEncoding.cs | 1 + .../Subspaces/DynamicKeySubspace.cs | 627 ++++--------- .../Subspaces/Fdb.Directory.cs | 9 +- .../Subspaces/IDynamicKeySubspace.cs | 13 +- FoundationDB.Client/Subspaces/IKeySubspace.cs | 64 +- .../Subspaces/ITypedKeySubspace.cs | 96 -- FoundationDB.Client/Subspaces/KeySubspace.cs | 398 ++------ .../Subspaces/KeySubspaceExtensions.cs | 21 +- .../Subspaces/TypedKeySubspace`1.cs | 153 ++-- .../Subspaces/TypedKeySubspace`2.cs | 232 +++-- .../Subspaces/TypedKeySubspace`3.cs | 219 +++-- .../Subspaces/TypedKeySubspace`4.cs | 240 +++-- .../Encoders/DynamicKeyEncoderBase.cs | 8 +- .../Encoders/ICompositeKeyEncoder.cs | 270 ++++++ .../{ => Encoders}/IDynamicKeyEncoder.cs | 30 +- .../IKeyEncoder.cs} | 34 +- .../{ => Encoders}/IValueEncoder.cs | 4 +- .../Encoders/KeyValueEncoders.Ordered.cs | 175 ++++ .../Encoders/KeyValueEncoders.Tuples.cs | 223 +++++ .../Encoders/KeyValueEncoders.Unordered.cs | 35 + .../Encoders/KeyValueEncoders.Values.cs | 121 +++ .../TypeSystem/Encoders/KeyValueEncoders.cs | 854 ++---------------- .../TypeSystem/ICompositeKeyEncoder.cs | 56 -- .../TypeSystem/IKeyEncoding.cs | 4 +- .../TypeSystem/IOrderedTypeCodec.cs | 2 +- .../TypeSystem/IUnorderedTypeCodec.cs | 2 +- FoundationDB.Client/TypeSystem/TypeCodec`1.cs | 2 +- FoundationDB.Client/TypeSystem/TypeSystem.cs | 18 +- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 1 + .../Collections/FdbMap`2.cs | 6 +- .../Collections/FdbMultimap`2.cs | 7 +- .../Collections/FdbQueue`1.cs | 1 + .../Collections/FdbRankedSet.cs | 1 + .../Collections/FdbVector`1.cs | 1 + .../Counters/FdbCounterMap.cs | 1 + .../Counters/FdbHighContentionCounter.cs | 1 + .../Indexes/FdbIndex`2.cs | 7 +- .../Interning/FdbStringIntern.cs | 1 + FoundationDB.Layers.Common/Optional`1.cs | 1 + .../Documents/FdbDocumentCollection.cs | 10 +- .../Documents/FdbHashSetCollection.cs | 1 + .../Indexes/FdbCompressedBitmapIndex.cs | 1 + .../Messaging/FdbWorkerPool.cs | 1 + FoundationDB.Tests.Sandbox/Program.cs | 4 +- FoundationDB.Tests/DatabaseFacts.cs | 4 +- FoundationDB.Tests/Encoders/EncoderFacts.cs | 13 +- .../Experimental/JsonNetCodec.cs | 1 + .../Experimental/ProtobufCodec.cs | 1 + FoundationDB.Tests/Layers/DirectoryFacts.cs | 104 +-- FoundationDB.Tests/Layers/MapFacts.cs | 1 + FoundationDB.Tests/Layers/MultiMapFacts.cs | 1 + FoundationDB.Tests/Layers/VectorFacts.cs | 1 + .../Linq/FdbQueryExpressionFacts.cs | 4 +- FoundationDB.Tests/SubspaceFacts.cs | 82 +- FoundationDB.Tests/TransactionFacts.cs | 8 +- 68 files changed, 1990 insertions(+), 2459 deletions(-) delete mode 100644 FoundationDB.Client/Subspaces/ITypedKeySubspace.cs create mode 100644 FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs rename FoundationDB.Client/TypeSystem/{ => Encoders}/IDynamicKeyEncoder.cs (94%) rename FoundationDB.Client/TypeSystem/{IKeyEncoder`1.cs => Encoders/IKeyEncoder.cs} (63%) rename FoundationDB.Client/TypeSystem/{ => Encoders}/IValueEncoder.cs (97%) create mode 100644 FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs create mode 100644 FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs create mode 100644 FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs create mode 100644 FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs delete mode 100644 FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index 8b82040f5..fd1b13a5f 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -545,7 +545,7 @@ internal static async Task OpenInternalAsync(string clusterFile, st dbName = dbName ?? "DB"; globalSpace = globalSpace ?? KeySubspace.Empty; - if (Logging.On) Logging.Info(typeof(Fdb), "OpenAsync", String.Format("Connecting to database '{0}' using cluster file '{1}' and subspace '{2}' ...", dbName, clusterFile, globalSpace)); + if (Logging.On) Logging.Info(typeof(Fdb), "OpenAsync", $"Connecting to database '{dbName}' using cluster file '{clusterFile}' and subspace '{globalSpace}' ..."); FdbCluster cluster = null; FdbDatabase db = null; diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 1c91886b0..ac2da2107 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -38,6 +38,7 @@ namespace FoundationDB.Client using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client.Core; using FoundationDB.Client.Native; using FoundationDB.Layers.Directories; @@ -455,8 +456,8 @@ internal void ChangeRoot(IKeySubspace subspace, IFdbDirectory directory, bool re lock (this)//TODO: don't use this for locking { m_readOnly = readOnly; - m_globalSpace = KeySubspace.CopyDynamic(subspace, TypeSystem.Tuples); - m_globalSpaceCopy = KeySubspace.CopyDynamic(subspace, TypeSystem.Tuples); // keep another copy + m_globalSpace = KeySubspace.Copy(subspace).Using(TypeSystem.Tuples); + m_globalSpaceCopy = KeySubspace.Copy(subspace).Using(TypeSystem.Tuples); // keep another copy m_directory = directory == null ? null : new FdbDatabasePartition(this, directory); } } @@ -534,15 +535,7 @@ public Slice BoundCheck(Slice key, bool allowSystemKeys) return m_globalSpace.BoundCheck(key, allowSystemKeys); } - Slice IKeySubspace.ConcatKey(Slice key) - { - return m_globalSpace.ConcatKey(key); - } - - Slice[] IKeySubspace.ConcatKeys(IEnumerable keys) - { - return m_globalSpace.ConcatKeys(keys); - } + Slice IKeySubspace.this[Slice relativeKey] => m_globalSpace[relativeKey]; /// Remove the database global subspace prefix from a binary key, or throw if the key is outside of the global subspace. Slice IKeySubspace.ExtractKey(Slice key, bool boundCheck) @@ -550,55 +543,22 @@ Slice IKeySubspace.ExtractKey(Slice key, bool boundCheck) return m_globalSpace.ExtractKey(key, boundCheck); } - /// Remove the database global subspace prefix from a binary key, or throw if the key is outside of the global subspace. - Slice[] IKeySubspace.ExtractKeys(IEnumerable keys, bool boundCheck) - { - return m_globalSpace.ExtractKeys(keys, boundCheck); - } - - SliceWriter IKeySubspace.GetWriter(int capacity) - { - return m_globalSpace.GetWriter(capacity); - } - Slice IKeySubspace.GetPrefix() { return m_globalSpace.GetPrefix(); } - IKeySubspace IKeySubspace.this[Slice suffix] - { - get - { - return m_globalSpace[suffix]; - } - } - KeyRange IKeySubspace.ToRange() { return m_globalSpace.ToRange(); } - KeyRange IKeySubspace.ToRange(Slice suffix) - { - return m_globalSpace.ToRange(suffix); - } + public DynamicPartition Partition => m_globalSpace.Partition; + //REVIEW: should we hide this on the main db? - public DynamicPartition Partition - { - //REVIEW: should we hide this on the main db? - get { return m_globalSpace.Partition; } - } - - IDynamicKeyEncoder IDynamicKeySubspace.Encoder - { - get { return m_globalSpace.Encoder; } - } + IKeyEncoding IDynamicKeySubspace.Encoding => m_globalSpace.Encoding; - public DynamicKeys Keys - { - get { return m_globalSpace.Keys; } - } + public DynamicKeys Keys => m_globalSpace.Keys; /// Returns true if the key is inside the system key space (starts with '\xFF') internal static bool IsSystemKey(ref Slice key) diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index 286c53f37..8e410d6bb 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; + using Doxense.Serialization.Encoders; [DebuggerDisplay("Count={Chunk!=null?Chunk.Length:0}, HasMore={HasMore}, Reversed={Reversed}, Iteration={Iteration}")] public struct FdbRangeChunk diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 5191778fa..16d4d362c 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Provides a set of extensions methods shared by all FoundationDB transaction implementations. diff --git a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs index 6c19ba85d..96cefcb7e 100644 --- a/FoundationDB.Client/Filters/FdbDatabaseFilter.cs +++ b/FoundationDB.Client/Filters/FdbDatabaseFilter.cs @@ -35,6 +35,7 @@ namespace FoundationDB.Filters using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -127,25 +128,9 @@ KeyRange IKeySubspace.ToRange() return this.GlobalSpace.ToRange(); } - KeyRange IKeySubspace.ToRange(Slice suffix) - { - return this.GlobalSpace.ToRange(suffix); - } - - IKeySubspace IKeySubspace.this[Slice suffix] - { - get { return this.GlobalSpace[suffix]; } - } - - public virtual DynamicPartition Partition - { - get { return m_database.Partition; } - } + public virtual DynamicPartition Partition => m_database.Partition; - public virtual DynamicKeys Keys - { - get { return m_database.Keys; } - } + public virtual DynamicKeys Keys => m_database.Keys; public virtual bool Contains(Slice key) { @@ -157,35 +142,14 @@ public virtual Slice BoundCheck(Slice key, bool allowSystemKeys) return m_database.BoundCheck(key, allowSystemKeys); } - public virtual Slice ConcatKey(Slice key) - { - return m_database.ConcatKey(key); - } - - public virtual Slice[] ConcatKeys(IEnumerable keys) - { - return m_database.ConcatKeys(keys); - } + public virtual Slice this[Slice key] => m_database[key]; public virtual Slice ExtractKey(Slice key, bool boundCheck = false) { return m_database.ExtractKey(key, boundCheck); } - public virtual Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) - { - return m_database.ExtractKeys(keys, boundCheck); - } - - public virtual SliceWriter GetWriter(int capacity = 0) - { - return m_database.GetWriter(capacity); - } - - public virtual IDynamicKeyEncoder Encoder - { - get { return m_database.Encoder; } - } + public virtual IKeyEncoding Encoding => m_database.Encoding; #endregion diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index d9a29aac2..58de2beab 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Filters.Logging { using System; @@ -147,7 +146,11 @@ private Slice[] Grab(Slice[] slices) private KeySelector Grab(KeySelector selector) { - return new KeySelector(Grab(selector.Key), selector.OrEqual, selector.Offset); + return new KeySelector( + Grab(selector.Key), + selector.OrEqual, + selector.Offset + ); } private KeySelector[] Grab(KeySelector[] selectors) diff --git a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs index a658b67d9..d351bfcaf 100644 --- a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs +++ b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs @@ -30,6 +30,7 @@ namespace FoundationDB.Filters { using FoundationDB.Client; using System; + using System.Linq; using System.Threading; using System.Threading.Tasks; @@ -38,31 +39,28 @@ public sealed class PrefixRewriterTransaction : FdbTransactionFilter { // We will add a prefix to all keys sent to the db, and remove it on the way back - private readonly IKeySubspace m_prefix; - public PrefixRewriterTransaction(IKeySubspace prefix, IFdbTransaction trans, bool ownsTransaction) : base(trans, false, ownsTransaction) { - if (prefix == null) throw new ArgumentNullException("prefix"); - m_prefix = prefix; + this.Prefix = prefix ?? throw new ArgumentNullException(nameof(prefix)); } - public IKeySubspace Prefix { get { return m_prefix; } } + public IKeySubspace Prefix { get; } private Slice Encode(Slice key) { - return m_prefix.ConcatKey(key); + return this.Prefix[key]; } private Slice[] Encode(Slice[] keys) { - return m_prefix.ConcatKeys(keys); + return keys.Select(k => this.Prefix[k]).ToArray(); } private KeySelector Encode(KeySelector selector) { return new KeySelector( - m_prefix.ConcatKey(selector.Key), + this.Prefix[selector.Key], selector.OrEqual, selector.Offset ); @@ -73,9 +71,8 @@ private KeySelector[] Encode(KeySelector[] selectors) var keys = new Slice[selectors.Length]; for (int i = 0; i < selectors.Length;i++) { - keys[i] = selectors[i].Key; + keys[i] = this.Prefix[selectors[i].Key]; } - keys = m_prefix.ConcatKeys(keys); var res = new KeySelector[selectors.Length]; for (int i = 0; i < selectors.Length; i++) @@ -91,7 +88,7 @@ private KeySelector[] Encode(KeySelector[] selectors) private Slice Decode(Slice key) { - return m_prefix.ExtractKey(key); + return this.Prefix.ExtractKey(key); } private Slice[] Decode(Slice[] keys) @@ -99,7 +96,7 @@ private Slice[] Decode(Slice[] keys) var res = new Slice[keys.Length]; for (int i = 0; i < keys.Length;i++) { - res[i] = m_prefix.ExtractKey(keys[i]); + res[i] = this.Prefix.ExtractKey(keys[i]); } return res; } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 04394e701..d2ec2f485 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -38,6 +38,7 @@ namespace FoundationDB.Layers.Directories using Doxense.Diagnostics.Contracts; using Doxense.Linq; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Filters.Logging; @@ -165,7 +166,7 @@ public static FdbDirectoryLayer Create() [NotNull] public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null) { - var subspace = KeySubspace.CreateDynamic(prefix, TypeSystem.Tuples); + var subspace = KeySubspace.FromKey(prefix).Using(TypeSystem.Tuples); var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } @@ -481,7 +482,7 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti public override string ToString() { - return String.Format("DirectoryLayer(path={0}, contents={1}, nodes={2})", this.FullName, this.ContentSubspace.GetPrefix().PrettyPrint(), this.NodeSubspace.GetPrefix().PrettyPrint()); + return $"DirectoryLayer(path={this.FullName}, contents={this.ContentSubspace.GetPrefix():K}, nodes={this.NodeSubspace.GetPrefix():K})"; } #endregion @@ -947,7 +948,7 @@ private FdbDirectorySubspace ContentsOfNode([NotNull] IKeySubspace node, [NotNul } else { - return new FdbDirectorySubspace(path, relativePath, prefix, this, layer, TypeSystem.Default.GetDynamicEncoder()); + return new FdbDirectorySubspace(path, relativePath, prefix, this, layer, TypeSystem.Default); } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 767d37270..22d199562 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -30,29 +30,33 @@ namespace FoundationDB.Layers.Directories { using System; using Doxense.Collections.Tuples; + using Doxense.Serialization.Encoders; using FoundationDB.Client; public class FdbDirectoryPartition : FdbDirectorySubspace { /// Returns a slice with the ASCII string "partition" - public static Slice LayerId { get { return Slice.FromString("partition"); } } - - private readonly FdbDirectoryLayer m_parentDirectoryLayer; + public static Slice LayerId => Slice.FromString("partition"); internal FdbDirectoryPartition(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer) - : base(location, relativeLocation, prefix, new FdbDirectoryLayer(KeySubspace.CreateDynamic(prefix + FdbKey.Directory, TypeSystem.Tuples), KeySubspace.CreateDynamic(prefix, TypeSystem.Tuples), location), LayerId, TypeSystem.Tuples.GetDynamicEncoder()) + : base(location, relativeLocation, prefix, new FdbDirectoryLayer(FromKey(prefix + FdbKey.Directory).Using(TypeSystem.Default), FromKey(prefix).Using(TypeSystem.Default), location), LayerId, TypeSystem.Default) { - m_parentDirectoryLayer = directoryLayer; + this.ParentDirectoryLayer = directoryLayer; } - internal FdbDirectoryLayer ParentDirectoryLayer { get { return m_parentDirectoryLayer; } } + internal FdbDirectoryLayer ParentDirectoryLayer { get; } protected override Slice GetKeyPrefix() { throw new InvalidOperationException("Cannot create keys in the root of a directory partition."); } + protected override KeyRange GetKeyRange() + { + throw new InvalidOperationException("Cannot create a key range in the root of a directory partition."); + } + public override bool Contains(Slice key) { throw new InvalidOperationException("Cannot check whether a key belongs to the root of a directory partition."); @@ -77,7 +81,7 @@ protected override FdbDirectoryLayer GetLayerForPath(ITuple relativeLocation) public override string ToString() { - return String.Format("DirectoryPartition(path={0}, prefix={1})", this.FullName, this.InternalKey.PrettyPrint()); + return $"DirectoryPartition(path={this.FullName}, prefix={GetPrefixUnsafe():K})"; } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index b67b194ca..a9dcd1bc2 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -34,17 +34,18 @@ namespace FoundationDB.Layers.Directories using System.Threading.Tasks; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; /// A Directory Subspace represents the contents of a directory, but it also remembers the path with which it was opened and offers convenience methods to operate on the directory at that path. /// An instance of DirectorySubspace can be used for all the usual subspace operations. It can also be used to operate on the directory with which it was opened. - [DebuggerDisplay("Path={this.FullName}, Prefix={InternalKey}, Layer={Layer}")] + [DebuggerDisplay("Path={this.FullName}, Prefix={Key}, Layer={Layer}")] public class FdbDirectorySubspace : DynamicKeySubspace, IFdbDirectory { - internal FdbDirectorySubspace(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IDynamicKeyEncoder encoder) - : base(prefix, encoder) + internal FdbDirectorySubspace(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IKeyEncoding encoding) + : base(prefix, encoding) { Contract.Requires(location != null && relativeLocation != null && prefix != null && directoryLayer != null); if (layer.IsNull) layer = Slice.Empty; @@ -121,7 +122,7 @@ public void CheckLayer(Slice layer) { if (layer.IsPresent && layer != this.Layer) { - throw new InvalidOperationException(String.Format("The directory {0} was created with incompatible layer {1} instead of expected {2}.", this.FullName, this.Layer.PrettyPrint(), layer.PrettyPrint())); + throw new InvalidOperationException($"The directory {this.FullName} was created with incompatible layer {this.Layer:P} instead of expected {layer:P}."); } } @@ -129,9 +130,9 @@ public void CheckLayer(Slice layer) /// Transaction to use for the operation /// New layer id of this directory [ItemNotNull] - public async Task ChangeLayerAsync([NotNull] IFdbTransaction trans, Slice newLayer) + public async Task ChangeLayerAsync(IFdbTransaction trans, Slice newLayer) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (newLayer.IsNull) newLayer = Slice.Empty; if (this.RelativeLocation.Count == 0) @@ -152,7 +153,7 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti // set the layer to the new value await this.DirectoryLayer.ChangeLayerInternalAsync(trans, this.RelativeLocation, newLayer).ConfigureAwait(false); // and return the new version of the subspace - return new FdbDirectorySubspace(this.Location, this.RelativeLocation, this.InternalKey, this.DirectoryLayer, newLayer, TypeSystem.Default.GetDynamicEncoder()); + return new FdbDirectorySubspace(this.Location, this.RelativeLocation, GetKeyPrefix(), this.DirectoryLayer, newLayer, TypeSystem.Default); } /// Opens a subdirectory with the given . @@ -161,10 +162,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Transaction to use for the operation /// Relative path of the subdirectory to create or open /// If is specified, it is checked against the layer of an existing subdirectory or set as the layer of a new subdirectory. - public Task CreateOrOpenAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) + public Task CreateOrOpenAsync(IFdbTransaction trans, IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(null, trans, ToRelativePath(path), layer, Slice.Nil, allowCreate: true, allowOpen: true, throwOnError: true); } @@ -175,10 +176,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Transaction to use for the operation /// Relative path of the subdirectory to open /// If specified, the opened directory must have the same layer id. - public Task OpenAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) + public Task OpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(trans, null, ToRelativePath(path), layer, prefix: Slice.Nil, allowCreate: false, allowOpen: true, throwOnError: true); } @@ -189,10 +190,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Relative path of the subdirectory to open /// If specified, the opened directory must have the same layer id. /// Returns the directory if it exists, or null if it was not found - public Task TryOpenAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) + public Task TryOpenAsync(IFdbReadOnlyTransaction trans, IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(trans, null, ToRelativePath(path), layer, prefix: Slice.Nil, allowCreate: false, allowOpen: true, throwOnError: false); } @@ -202,10 +203,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Transaction to use for the operation /// Relative path of the subdirectory to create /// If is specified, it is recorded with the subdirectory and will be checked by future calls to open. - public Task CreateAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) + public Task CreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(null, trans, ToRelativePath(path), layer, prefix: Slice.Nil, allowCreate: true, allowOpen: false, throwOnError: true); } @@ -215,10 +216,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Transaction to use for the operation /// Relative path of the subdirectory to create /// If is specified, it is recorded with the subdirectory and will be checked by future calls to open. - public Task TryCreateAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer = default(Slice)) + public Task TryCreateAsync(IFdbTransaction trans, IEnumerable path, Slice layer = default(Slice)) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(null, trans, ToRelativePath(path), layer, prefix: Slice.Nil, allowCreate: true, allowOpen: false, throwOnError: false); } @@ -227,10 +228,10 @@ public async Task ChangeLayerAsync([NotNull] IFdbTransacti /// Path of the directory to create /// If is specified, it is recorded with the directory and will be checked by future calls to open. /// The directory will be created with the given physical prefix; otherwise a prefix is allocated automatically. - public Task RegisterAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable path, Slice layer, Slice prefix) + public Task RegisterAsync(IFdbTransaction trans, IEnumerable path, Slice layer, Slice prefix) { - if (trans == null) throw new ArgumentNullException("trans"); - if (path == null) throw new ArgumentNullException("path"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (path == null) throw new ArgumentNullException(nameof(path)); return this.DirectoryLayer.CreateOrOpenInternalAsync(null, trans, ToRelativePath(path), layer, prefix: prefix, allowCreate: true, allowOpen: false, throwOnError: true); } @@ -240,10 +241,10 @@ public Task RegisterAsync([NotNull] IFdbTransaction trans, /// /// Transaction to use for the operation /// Full path (from the root) where this directory will be moved - public Task MoveToAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable newAbsolutePath) + public Task MoveToAsync(IFdbTransaction trans, IEnumerable newAbsolutePath) { - if (trans == null) throw new ArgumentNullException("trans"); - if (newAbsolutePath == null) throw new ArgumentNullException("newAbsolutePath"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (newAbsolutePath == null) throw new ArgumentNullException(nameof(newAbsolutePath)); // if 'this' is a Directory Partition, we need to move it via the parent DL ! var directoryLayer = GetLayerForPath(STuple.Empty); @@ -274,10 +275,10 @@ Task IFdbDirectory.MoveAsync(IFdbTransaction trans, IEnume /// /// Transaction to use for the operation /// Full path (from the root) where this directory will be moved - public Task TryMoveToAsync([NotNull] IFdbTransaction trans, [NotNull] IEnumerable newPath) + public Task TryMoveToAsync(IFdbTransaction trans, IEnumerable newPath) { - if (trans == null) throw new ArgumentNullException("trans"); - if (newPath == null) throw new ArgumentNullException("newPath"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (newPath == null) throw new ArgumentNullException(nameof(newPath)); // if 'this' is a Directory Partition, we need to move it via the parent DL ! var directoryLayer = GetLayerForPath(STuple.Empty); @@ -308,7 +309,7 @@ Task IFdbDirectory.TryMoveAsync(IFdbTransaction trans, IEn /// Transaction to use for the operation public Task RemoveAsync([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! var directoryLayer = GetLayerForPath(STuple.Empty); @@ -321,9 +322,9 @@ public Task RemoveAsync([NotNull] IFdbTransaction trans) /// /// Transaction to use for the operation /// Path of the sub-directory to remove (relative to this directory) - public Task RemoveAsync([NotNull] IFdbTransaction trans, IEnumerable path) + public Task RemoveAsync(IFdbTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // If path is empty, we are removing ourselves! var location = FdbDirectoryLayer.ParsePath(path, "path"); @@ -341,7 +342,7 @@ public Task RemoveAsync([NotNull] IFdbTransaction trans, IEnumerable pat /// Transaction to use for the operation public Task TryRemoveAsync([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! var directoryLayer = GetLayerForPath(STuple.Empty); @@ -354,9 +355,9 @@ public Task TryRemoveAsync([NotNull] IFdbTransaction trans) /// /// Transaction to use for the operation /// Path of the sub-directory to remove (relative to this directory) - public Task TryRemoveAsync([NotNull] IFdbTransaction trans, IEnumerable path) + public Task TryRemoveAsync(IFdbTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // If path is empty, we are removing ourselves! var location = FdbDirectoryLayer.ParsePath(path, "path"); @@ -372,7 +373,7 @@ public Task TryRemoveAsync([NotNull] IFdbTransaction trans, IEnumerableReturns true if the directory exists, otherwise false. public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // if 'this' is a Directory Partition, we need to remove it from the parent DL ! var directoryLayer = GetLayerForPath(STuple.Empty); @@ -382,9 +383,9 @@ public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans) /// Checks if a sub-directory exists /// Returns true if the directory exists, otherwise false. - public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans, IEnumerable path) + public Task ExistsAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); // If path is empty, we are checking ourselves! var location = FdbDirectoryLayer.ParsePath(path, "path"); @@ -399,35 +400,34 @@ public Task ExistsAsync([NotNull] IFdbReadOnlyTransaction trans, IEnumerab /// Returns the list of all the subdirectories of the current directory. public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return this.DirectoryLayer.ListInternalAsync(trans, this.RelativeLocation, throwIfMissing: true); } /// Returns the list of all the subdirectories of a sub-directory. - public Task> ListAsync([NotNull] IFdbReadOnlyTransaction trans, IEnumerable path) + public Task> ListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return this.DirectoryLayer.ListInternalAsync(trans, ToRelativePath(path), throwIfMissing: true); } /// Returns the list of all the subdirectories of a sub-directory, it it exists. public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return this.DirectoryLayer.ListInternalAsync(trans, this.RelativeLocation, throwIfMissing: false); } /// Returns the list of all the subdirectories of the current directory, it it exists. - public Task> TryListAsync([NotNull] IFdbReadOnlyTransaction trans, IEnumerable path) + public Task> TryListAsync(IFdbReadOnlyTransaction trans, IEnumerable path) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); return this.DirectoryLayer.ListInternalAsync(trans, ToRelativePath(path), throwIfMissing: false); } public override string DumpKey(Slice key) { - string str = base.DumpKey(key); - return String.Format("[/{0}]:{1}", this.FullName, str); + return $"[/{this.FullName}]:{base.DumpKey(key)}"; } /// Returns a user-friendly description of this directory @@ -435,11 +435,11 @@ public override string ToString() { if (this.Layer.IsNullOrEmpty) { - return String.Format("DirectorySubspace(path={0}, prefix={1})", this.FullName, this.InternalKey.PrettyPrint()); + return $"DirectorySubspace(path={this.FullName}, prefix={GetPrefixUnsafe():K})"; } else { - return String.Format("DirectorySubspace(path={0}, prefix={1}, layer={2})", this.FullName, this.InternalKey.PrettyPrint(), this.Layer.PrettyPrint()); + return $"DirectorySubspace(path={this.FullName}, prefix={GetPrefixUnsafe():K}, layer={this.Layer:P})"; } } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index fd0aea49b..36eb6dd8b 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -32,6 +32,7 @@ namespace Doxense.Collections.Tuples.Encoding using FoundationDB.Client; using JetBrains.Annotations; using Doxense.Memory; + using Doxense.Serialization.Encoders; /// Type codec that uses the Tuple Encoding format /// Type of the values encoded by this codec diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs index 15fc5148f..649d1448a 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs @@ -19,9 +19,9 @@ public static class TupleEncoder /// Internal helper that serializes the content of a Tuple into a TupleWriter, meant to be called by implementers of types. /// Warning: This method will call into if inmplements - internal static void WriteTo(ref TupleWriter writer, [NotNull] ITuple tuple) + internal static void WriteTo(ref TupleWriter writer, [NotNull] TTuple tuple) + where TTuple : ITuple { - Contract.Requires(tuple != null); // ReSharper disable once SuspiciousTypeConversion.Global if (tuple is ITupleSerializable ts) { // optimized version diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs index 0d4ec147b..f6d98a7e9 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs @@ -32,6 +32,7 @@ namespace Doxense.Collections.Tuples.Encoding using System; using Doxense.Collections.Tuples; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB; using FoundationDB.Client; @@ -49,7 +50,8 @@ public IKeyEncoding Encoding get { return TypeSystem.Tuples; } } - public void PackKey(ref SliceWriter writer, ITuple items) + public void PackKey(ref SliceWriter writer, TTuple items) + where TTuple : ITuple { var tw = new TupleWriter(writer); TupleEncoder.WriteTo(ref tw, items); @@ -180,6 +182,11 @@ public STuple DecodeKey(Slice packed) return TuPack.DecodeKey(packed); } + public STuple DecodeKey(Slice packed) + { + return TuPack.DecodeKey(packed); + } + public KeyRange ToRange(Slice prefix) { return TuPack.ToRange(prefix); diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs index ff460024e..6893ea6ee 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs @@ -30,6 +30,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Collections.Tuples.Encoding { using System; + using Doxense.Serialization.Encoders; using FoundationDB.Client; /// Encoding that uses the Tuple Binary Encoding format diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 7aa8a103d..03163f552 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -30,158 +30,187 @@ namespace FoundationDB.Client { using System; using System.Collections.Generic; - using System.Linq; + using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; - using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; public class DynamicKeySubspace : KeySubspace, IDynamicKeySubspace { - /// Create a new subspace from a binary prefix - /// Prefix of the new subspace - /// If true, take a copy of the prefix - /// Type System used to encode keys in this subspace (optional, will use Tuple Encoding by default) - internal DynamicKeySubspace(Slice rawPrefix, bool copy, IDynamicKeyEncoder encoder) - : base (rawPrefix, copy) - { - this.Encoder = encoder ?? TypeSystem.Default.GetDynamicEncoder(); - this.Keys = new DynamicKeys(this, this.Encoder); - this.Partition = new DynamicPartition(this, this.Encoder); - } - public DynamicKeySubspace(Slice rawPrefix, IDynamicKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } + /// Encoder for the keys of this subspace + public IKeyEncoding Encoding { get; } + + internal IDynamicKeyEncoder KeyEncoder { get; } - protected override IKeySubspace CreateChildren(Slice suffix) + /// Create a new subspace from a binary prefix + /// Prefix of the new subspace + /// Type System used to encode keys in this subspace (optional, will use Tuple Encoding by default) + internal DynamicKeySubspace(Slice prefix, IKeyEncoding encoding) + : base(prefix) { - return new DynamicKeySubspace(ConcatKey(suffix), this.Encoder); + this.Encoding = encoding; + this.KeyEncoder = encoding.GetDynamicEncoder(); + this.Keys = new DynamicKeys(this, this.KeyEncoder); + this.Partition = new DynamicPartition(this); } - /// Encoder for the keys of this subspace - public IDynamicKeyEncoder Encoder { get; } - /// Return a view of all the possible binary keys of this subspace public DynamicKeys Keys { get; } - /// Returns an helper object that knows how to create sub-partitions of this subspace + /// Return a view of all the possible binary keys of this subspace public DynamicPartition Partition { get; } + public Slice this[ITuple item] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => this.Keys.Pack(item); + } + } /// Key helper for a dynamic TypeSystem - public /*readonly*/ struct DynamicKeys + public sealed class DynamicKeys { - //NOTE: everytime an ITuple is used here, it is as a container (vector of objects), and NOT as the Tuple Encoding scheme ! (separate concept) /// Parent subspace [NotNull] - public readonly IKeySubspace Subspace; + private readonly DynamicKeySubspace Parent; /// Encoder used to format keys in this subspace [NotNull] - public readonly IDynamicKeyEncoder Encoder; + public IDynamicKeyEncoder Encoder { get; } - public DynamicKeys([NotNull] IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + internal DynamicKeys(DynamicKeySubspace parent, IDynamicKeyEncoder encoder) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; + Contract.Requires(parent != null && encoder != null); + this.Parent = parent; this.Encoder = encoder; } + /// Convert a tuple into a key of this subspace + /// Tuple that will be packed and appended to the subspace prefix + public Slice Pack([NotNull] TTuple tuple) + where TTuple : ITuple + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + var sw = this.Parent.OpenWriter(); + this.Encoder.PackKey(ref sw, tuple); + return sw.ToSlice(); + } + + /// Unpack a key of this subspace, back into a tuple + /// Key that was produced by a previous call to + /// Original tuple + public ITuple Unpack(Slice packedKey) + { + return this.Encoder.UnpackKey(this.Parent.ExtractKey(packedKey)); + } + + #region ToRange()... + /// Return a key range that encompass all the keys inside this subspace, according to the current key encoder public KeyRange ToRange() { - return this.Encoder.ToRange(this.Subspace.GetPrefix()); + return this.Encoder.ToRange(this.Parent.GetPrefix()); } /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder /// Tuple used as a prefix for the range public KeyRange ToRange([NotNull] ITuple tuple) { - return this.Encoder.ToRange(Pack(tuple)); + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); } - /// Return a key range that encompass all the keys inside a partition of this subspace, according to the current key encoder - /// Convertible item used as a prefix for the range - public KeyRange ToRange([NotNull] ITupleFormattable item) + public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(Pack(item)); + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); } - /// Convert a tuple into a key of this subspace - /// Tuple that will be packed and appended to the subspace prefix - /// This is a shortcut for - public Slice this[[NotNull] ITuple tuple] => Pack(tuple); - - /// Convert an item into a key of this subspace - /// Convertible item that will be packed and appended to the subspace prefix - /// This is a shortcut for - public Slice this[[NotNull] ITupleFormattable item] => Pack(item); - - /// Convert a tuple into a key of this subspace - /// Tuple that will be packed and appended to the subspace prefix - public Slice Pack([NotNull] ITuple tuple) + public KeyRange ToRange(STuple tuple) { - Contract.NotNull(tuple, nameof(tuple)); + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + } - var writer = this.Subspace.GetWriter(); - this.Encoder.PackKey(ref writer, tuple); - return writer.ToSlice(); + public KeyRange ToRange(STuple tuple) + { + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); } - /// Convert a batch of tuples into keys of this subspace, in an optimized way. - /// Sequence of tuple that will be packed and appended to the subspace prefix - public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable tuples) + public KeyRange ToRange(STuple tuple) { - if (tuples == null) throw new ArgumentNullException(nameof(tuples)); + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + } + public KeyRange ToRange(STuple tuple) + { + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + } + public KeyRange ToRange(STuple tuple) + { + return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + } - return Batched.Convert( - this.Subspace.GetWriter(), - tuples, - (ref SliceWriter writer, ITuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), - this.Encoder - ); + #endregion + + #region ToKeyRange()... + + public KeyRange ToKeyRange(T1 item1) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1); } - /// Convert an item into a key of this subspace - /// Convertible item that will be packed and appended to the subspace prefix - public Slice Pack([NotNull] ITupleFormattable item) + public KeyRange ToKeyRange(T1 item1, T2 item2) { - if (item == null) throw new ArgumentNullException(nameof(item)); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2); + } - return Pack(item.ToTuple()); + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3); } - /// Convert a batch of items into keys of this subspace, in an optimized way. - /// Sequence of convertible items that will be packed and appended to the subspace prefix - public Slice[] PackMany([NotNull, ItemNotNull] IEnumerable items) + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4) { - if (items == null) throw new ArgumentNullException(nameof(items)); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3, item4); + } - return Batched.Convert( - this.Subspace.GetWriter(), - items.Select(item => item.ToTuple()), - (ref SliceWriter writer, ITuple tuple, IDynamicKeyEncoder encoder) => encoder.PackKey(ref writer, tuple), - this.Encoder - ); + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3, item4, item5); + } + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3, item4, item5, item6); + } + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3, item4, item5, item6, item7); } + public KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), item1, item2, item3, item4, item5, item6, item7, item8); + } + + #endregion + + #region Encode... /// Encode a key which is composed of a single element - public Slice Encode(T item1) + public Slice Encode(T1 item1) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1); - return writer.ToSlice(); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1); + return sw.ToSlice(); } /// Encode a batch of keys, each one composed of a single element public Slice[] EncodeMany(IEnumerable items) { return Batched.Convert( - this.Subspace.GetWriter(), + this.Parent.OpenWriter(), items, (ref SliceWriter writer, T item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, item), this.Encoder @@ -192,7 +221,7 @@ public Slice[] EncodeMany(IEnumerable items) public Slice[] EncodeMany(IEnumerable items, Func selector) { return Batched.Convert( - this.Subspace.GetWriter(), + this.Parent.OpenWriter(), items, (ref SliceWriter writer, TSource item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector(item)), this.Encoder @@ -202,373 +231,115 @@ public Slice[] EncodeMany(IEnumerable items, FuncEncode a key which is composed of a two elements public Slice Encode(T1 item1, T2 item2) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of two values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item)), - this.Encoder - ); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1, item2); + return sw.ToSlice(); } /// Encode a key which is composed of three elements public Slice Encode(T1 item1, T2 item2, T3 item3) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of three values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item)), - this.Encoder - ); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1, item2, item3); + return sw.ToSlice(); } /// Encode a key which is composed of four elements public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3, item4); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of four values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3, Func selector4) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item), selector4(item)), - this.Encoder - ); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1, item2, item3, item4); + return sw.ToSlice(); } /// Encode a key which is composed of five elements public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3, item4, item5); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of five values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3, Func selector4, Func selector5) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item), selector4(item), selector5(item)), - this.Encoder - ); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1, item2, item3, item4, item5); + return sw.ToSlice(); } /// Encode a key which is composed of six elements public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3, item4, item5, item6); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of six values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3, Func selector4, Func selector5, Func selector6) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item), selector4(item), selector5(item), selector6(item)), - this.Encoder - ); - } - - /// Encode a key which is composed of seven elements - public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) - { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3, item4, item5, item6, item7); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of seven values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3, Func selector4, Func selector5, Func selector6, Func selector7) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item), selector4(item), selector5(item), selector6(item), selector7(item)), - this.Encoder - ); - } - - /// Encode a key which is composed of eight elements - public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) - { - var writer = this.Subspace.GetWriter(); - this.Encoder.EncodeKey(ref writer, item1, item2, item3, item4, item5, item6, item7, item8); - return writer.ToSlice(); - } - - /// Encode a batch of keys, each one composed of eight values extracted from each elements - public Slice[] EncodeMany(IEnumerable items, Func selector1, Func selector2, Func selector3, Func selector4, Func selector5, Func selector6, Func selector7, Func selector8) - { - return Batched.Convert( - this.Subspace.GetWriter(), - items, - (ref SliceWriter writer, TItem item, IDynamicKeyEncoder encoder) => encoder.EncodeKey(ref writer, selector1(item), selector2(item), selector3(item), selector4(item), selector5(item), selector6(item), selector7(item), selector8(item)), - this.Encoder - ); - } - - /// Unpack a key of this subspace, back into a tuple - /// Key that was produced by a previous call to - /// Original tuple - public ITuple Unpack(Slice packed) - { - return this.Encoder.UnpackKey(this.Subspace.ExtractKey(packed)); + var sw = this.Parent.OpenWriter(); + this.Encoder.EncodeKey(ref sw, item1, item2, item3, item4, item5, item6); + return sw.ToSlice(); } - private static T[] BatchDecode(IEnumerable packed, IKeySubspace subspace, IDynamicKeyEncoder encoder, Func decode) - { - if (packed is ICollection coll) - { - var res = new T[coll.Count]; - int p = 0; - foreach (var data in packed) - { - res[p++] = decode(subspace.ExtractKey(data), encoder); - } - Contract.Assert(p == res.Length); - return res; - } - else - { - var res = new List(); - foreach (var data in packed) - { - res.Add(decode(subspace.ExtractKey(data), encoder)); - } - return res.ToArray(); - } - } + #endregion - /// Unpack a batch of keys of this subspace, back into an array of tuples - /// Sequence of keys that were produced by a previous call to or - /// Array containing the original tuples - public ITuple[] UnpackMany(IEnumerable packed) - { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.UnpackKey(data)); - } + #region Decode... /// Decode a key of this subspace, composed of a single element - public T1 Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, each one composed of a single element - public IEnumerable DecodeMany(IEnumerable packed) + public T1 Decode(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, composed of exactly two elements - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, each one composed of exactly two elements - public IEnumerable> DecodeMany(IEnumerable packed) + public STuple Decode(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, composed of exactly three elements - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, each one composed of exactly three elements - public IEnumerable> DecodeMany(IEnumerable packed) + public STuple Decode(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, composed of exactly four elements - public STuple Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, each one composed of exactly four elements - public IEnumerable> DecodeMany(IEnumerable packed) + public STuple Decode(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, composed of exactly five elements - public STuple Decode(Slice packed) + public STuple Decode(Slice packedKey) { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } - /// Decode a batch of keys of this subspace, each one composed of exactly five elements - public IEnumerable> DecodeMany(IEnumerable packed) + public STuple Decode(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKey(data)); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, and return only the first element without decoding the rest the key. /// This method is faster than unpacking the complete key and reading only the first element. - public T DecodeFirst(Slice packed) - { - return this.Encoder.DecodeKeyFirst(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, and for each one, return only the first element without decoding the rest of the key. - /// This method is faster than unpacking the complete key and reading only the first element. - public IEnumerable DecodeFirstMany(IEnumerable packed) + public TFirst DecodeFirst(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKeyFirst(data)); + return this.Encoder.DecodeKeyFirst(this.Parent.ExtractKey(packedKey)); } /// Decode a key of this subspace, and return only the last element without decoding the rest. /// This method is faster than unpacking the complete key and reading only the last element. - public T DecodeLast(Slice packed) - { - return this.Encoder.DecodeKeyLast(this.Subspace.ExtractKey(packed)); - } - - /// Decode a batch of keys of this subspace, and for each one, return only the last element without decoding the rest of the key. - /// This method is faster than unpacking the complete key and reading only the last element. - public IEnumerable DecodeLastMany(Slice[] packed) + public TLast DecodeLast(Slice packedKey) { - return BatchDecode(packed, this.Subspace, this.Encoder, (data, encoder) => encoder.DecodeKeyLast(data)); + return this.Encoder.DecodeKeyLast(this.Parent.ExtractKey(packedKey)); } - #region Append: Subspace => Tuple - - /// Return an empty tuple that is attached to this subspace - /// Empty tuple that can be extended, and whose packed representation will always be prefixed by the subspace key - [NotNull] - public ITuple ToTuple() - { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Empty); - } - - /// Attach a tuple to an existing subspace. - /// Tuple whose items will be appended at the end of the current subspace - /// Tuple that wraps the items of and whose packed representation will always be prefixed by the subspace key. - [NotNull] - public ITuple Concat([NotNull] ITuple tuple) - { - return new PrefixedTuple(this.Subspace.GetPrefix(), tuple); - } - - /// Convert a formattable item into a tuple that is attached to this subspace. - /// Item that can be converted into a tuple - /// Tuple that is the logical representation of the item, and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(formattable.ToTuple())' - [NotNull] - public ITuple Concat([NotNull] ITupleFormattable formattable) - { - if (formattable == null) throw new ArgumentNullException(nameof(formattable)); - var tuple = formattable.ToTuple(); - if (tuple == null) throw new InvalidOperationException("Formattable item cannot return an empty tuple"); - return new PrefixedTuple(this.Subspace.GetPrefix(), tuple); - } - - /// Create a new 1-tuple that is attached to this subspace - /// Type of the value to append - /// Value that will be appended - /// Tuple of size 1 that contains , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(STuple.Create<T>(value))' - [NotNull] - public ITuple Append(T value) - { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(value)); - } - - /// Create a new 2-tuple that is attached to this subspace - /// Type of the first value to append - /// Type of the second value to append - /// First value that will be appended - /// Second value that will be appended - /// Tuple of size 2 that contains and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2>(item1, item2))' - [NotNull] - public ITuple Append(T1 item1, T2 item2) - { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2)); - } + #endregion - /// Create a new 3-tuple that is attached to this subspace - /// Type of the first value to append - /// Type of the second value to append - /// Type of the third value to append - /// First value that will be appended - /// Second value that will be appended - /// Third value that will be appended - /// Tuple of size 3 that contains , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3>(item1, item2, item3))' - [NotNull] - public ITuple Append(T1 item1, T2 item2, T3 item3) - { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3)); - } - - /// Create a new 4-tuple that is attached to this subspace - /// Type of the first value to append - /// Type of the second value to append - /// Type of the third value to append - /// Type of the fourth value to append - /// First value that will be appended - /// Second value that will be appended - /// Third value that will be appended - /// Fourth value that will be appended - /// Tuple of size 4 that contains , , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3, T4>(item1, item2, item3, item4))' - [NotNull] - public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4) - { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3, item4)); - } - - /// Create a new 5-tuple that is attached to this subspace - /// Type of the first value to append - /// Type of the second value to append - /// Type of the third value to append - /// Type of the fourth value to append - /// Type of the fifth value to append - /// First value that will be appended - /// Second value that will be appended - /// Third value that will be appended - /// Fourth value that will be appended - /// Fifth value that will be appended - /// Tuple of size 5 that contains , , , and , and whose packed representation will always be prefixed by the subspace key. - /// This is the equivalent of calling 'subspace.Create(STuple.Create<T1, T2, T3, T4, T5>(item1, item2, item3, item4, item5))' - [NotNull] - public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + /// Return a user-friendly string representation of a key of this subspace + public string Dump(Slice packedKey) { - return new PrefixedTuple(this.Subspace.GetPrefix(), STuple.Create(item1, item2, item3, item4, item5)); + //TODO: defer to the encoding itself? + var key = this.Parent.ExtractKey(packedKey); + try + { + var tuple = TuPack.Unpack(key); + return tuple.ToString(); + } + catch (FormatException) + { + // this is not a tuple??? + } + return key.PrettyPrint(); } - #endregion - } public /*readonly*/ struct DynamicPartition @@ -577,79 +348,23 @@ public ITuple Append(T1 item1, T2 item2, T3 item3, T4 item4, [NotNull] public readonly IDynamicKeySubspace Subspace; - [NotNull] - public readonly IDynamicKeyEncoder Encoder; - public DynamicPartition([NotNull] IDynamicKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + internal DynamicPartition([NotNull] DynamicKeySubspace subspace) { - Contract.Requires(subspace != null && encoder != null); + Contract.Requires(subspace != null); this.Subspace = subspace; - this.Encoder = encoder; - } - - /// Returns the same view but using a different Type System - /// Type System that will code keys in this new view - /// Review that will partition this subspace using a different Type System - /// - /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. - /// If you are calling this in a loop, consider creating a new subspace using that encoding. - /// - [Pure] - public DynamicPartition Using([NotNull] IKeyEncoding encoding) - { - Contract.NotNull(encoding, nameof(encoding)); - var encoder = encoding.GetDynamicEncoder(); - return UsingEncoder(encoder); - } - - /// Returns the same view but using a different Type System - /// Type System that will code keys in this new view - /// Review that will partition this subspace using a different Type System - /// - /// This should only be used for one-off usages where creating a new subspace just to encode one key would be overkill. - /// If you are calling this in a loop, consider creating a new subspace using that encoder. - /// - [Pure] - public DynamicPartition UsingEncoder([NotNull] IDynamicKeyEncoder encoder) - { - return new DynamicPartition(this.Subspace, encoder); } - /// Create a new subspace by appdending a suffix to the current subspace - /// Suffix of the new subspace - /// New subspace with prefix equal to the current subspace's prefix, followed by - public IDynamicKeySubspace this[Slice suffix] + public IDynamicKeySubspace this[Slice binarySuffix] { [Pure, NotNull] - get - { - if (suffix.IsNull) throw new ArgumentException("Partition suffix cannot be null", nameof(suffix)); - //TODO: find a way to limit the number of copies of the key? - return new DynamicKeySubspace(this.Subspace.ConcatKey(suffix), false, this.Encoder); - } - } - - public IDynamicKeySubspace this[ITuple tuple] - { - [Pure, ContractAnnotation("null => halt; notnull => notnull")] - get - { - if (tuple == null) throw new ArgumentNullException(nameof(tuple)); - //TODO: find a way to limit the number of copies of the packed tuple? - return new DynamicKeySubspace(this.Subspace.Keys.Pack(tuple), false, this.Encoder); - } + get => new DynamicKeySubspace(this.Subspace[binarySuffix], this.Subspace.Encoding); } - public IDynamicKeySubspace this[ITupleFormattable item] + public IDynamicKeySubspace this[ITuple suffix] { - [Pure, ContractAnnotation("null => halt; notnull => notnull")] - get - { - Contract.NotNull(item, nameof(item)); - var tuple = item.ToTuple(); - if (tuple == null) throw new InvalidOperationException("Formattable item returned an empty tuple"); - return this[tuple]; - } + [Pure, NotNull] + get => new DynamicKeySubspace(this.Subspace.Keys.Pack(suffix), this.Subspace.Encoding); } /// Partition this subspace into a child subspace @@ -663,7 +378,7 @@ public IDynamicKeySubspace this[ITupleFormattable item] [Pure, NotNull] public IDynamicKeySubspace ByKey(T value) { - return new DynamicKeySubspace(this.Subspace.Keys.Encode(value), false, this.Encoder); + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value), this.Subspace.Encoding); } /// Partition this subspace into a child subspace @@ -679,7 +394,7 @@ public IDynamicKeySubspace ByKey(T value) [Pure, NotNull] public IDynamicKeySubspace ByKey(T1 value1, T2 value2) { - return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2), false, this.Encoder); + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2), this.Subspace.Encoding); } /// Partition this subspace into a child subspace @@ -696,7 +411,7 @@ public IDynamicKeySubspace ByKey(T1 value1, T2 value2) [Pure, NotNull] public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3) { - return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3), false, this.Encoder); + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3), this.Subspace.Encoding); } /// Partition this subspace into a child subspace @@ -715,7 +430,7 @@ public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3) [Pure, NotNull] public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) { - return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3, value4), false, this.Encoder); + return new DynamicKeySubspace(this.Subspace.Keys.Encode(value1, value2, value3, value4), this.Subspace.Encoding); } } diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index c83167039..e396287ba 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; @@ -79,24 +78,24 @@ public static async Task OpenNamedPartitionAsync(string clusterFil { db = await Fdb.OpenInternalAsync(clusterFile, dbName, rootSpace, readOnly: false, ct: ct).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)); + if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", $"Opened root layer of database {db.Name} using cluster file '{db.Cluster.Path}'"); // look up in the root layer for the named partition var descriptor = await rootLayer.CreateOrOpenAsync(db, partitionPath, layer: FdbDirectoryPartition.LayerId, ct: ct).ConfigureAwait(false); - if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", String.Format("Found named partition '{0}' at prefix {1}", descriptor.FullName, descriptor)); + if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", $"Found named partition '{descriptor.FullName}' at prefix {descriptor}"); // we have to chroot the database to the new prefix, and create a new DirectoryLayer with a new '/' rootSpace = KeySubspace.Copy(descriptor); //note: create a copy of the key //TODO: find a nicer way to do that! db.ChangeRoot(rootSpace, FdbDirectoryLayer.Create(rootSpace, partitionPath), readOnly); - if (Logging.On) Logging.Info(typeof(Fdb.Directory), "OpenNamedPartitionAsync", String.Format("Opened partition {0} at {1}, using directory layer at {2}", descriptor.FullName, db.GlobalSpace, db.Directory.DirectoryLayer.NodeSubspace)); + if (Logging.On) Logging.Info(typeof(Fdb.Directory), "OpenNamedPartitionAsync", $"Opened partition {descriptor.FullName} at {db.GlobalSpace}, using directory layer at {db.Directory.DirectoryLayer.NodeSubspace}"); return db; } catch(Exception e) { - if (db != null) db.Dispose(); + db?.Dispose(); if (Logging.On) Logging.Exception(typeof(Fdb.Directory), "OpenNamedPartitionAsync", e); throw; } diff --git a/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs index 82e0e56f4..d0ab4cd71 100644 --- a/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/IDynamicKeySubspace.cs @@ -29,11 +29,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Represents a Key Subspace which can encode and decode keys of arbitrary size. /// This is usefull when dealing with subspaces that store keys of different types and shapes. - /// In pseudo code, we obtain a dynamic subspace that wraps a prefix, and uses the Tuple Encoder Format to encode variable-size tuples into binary: + /// In pseudo code, we obtain a dynamic subspace that wraps a prefix, and uses the Tuple Encoder Format to encode variable-size tuples into binary: /// /// subspace = {...}.OpenOrCreate(..., "/some/path/to/data", TypeSystem.Tuples) /// subspace.GetPrefix() => {prefix} @@ -46,16 +47,16 @@ namespace FoundationDB.Client public interface IDynamicKeySubspace : IKeySubspace { - /// Codec used by this subspace to convert keys into/from binary - [NotNull] - IDynamicKeyEncoder Encoder {get; } - /// View of the keys of this subspace + [NotNull] DynamicKeys Keys { get; } /// Returns an helper object that knows how to create sub-partitions of this subspace + [NotNull] DynamicPartition Partition { get; } - } + /// Encoding used to generate and parse the keys of this subspace + [NotNull] IKeyEncoding Encoding { get; } + } } diff --git a/FoundationDB.Client/Subspaces/IKeySubspace.cs b/FoundationDB.Client/Subspaces/IKeySubspace.cs index 086a01068..00c3d78c2 100644 --- a/FoundationDB.Client/Subspaces/IKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/IKeySubspace.cs @@ -26,12 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; - using System.Collections.Generic; using Doxense.Memory; + using FoundationDB.Layers.Directories; using JetBrains.Annotations; /// Represents a sub-partition of the global key space. @@ -61,27 +60,17 @@ public interface IKeySubspace [Pure] KeyRange ToRange(); - /// Return a key range that contains all the keys under a suffix in this subspace - /// Binary suffix that will be appended to the current prefix, before computing the range - /// Return the range: (this.Key + suffix) <= x <= Increment(this.Key + suffix) - [Pure] - KeyRange ToRange(Slice suffix); - - /// Create a new subspace by adding a suffix to the key of the current subspace. - /// Binary suffix that will be appended to the current prefix - /// New subspace whose prefix is the concatenation of the parent prefix, and - IKeySubspace this[Slice suffix] - { - [Pure, NotNull] get; - } - //REVIEW this should probably be renamed into GetSubspace(suffix) or Partition(suffix) in order to make it explicit that it is for creating subspaces instances that can be reused multiple times, and not single-use to generate a single key! + /// Return the key that is composed of the subspace's prefix and a binary suffix + /// Binary suffix that will be appended to the current prefix + /// Full binary key + Slice this[Slice relativeKey] { [Pure] get; } /// Test if a key is inside the range of keys logically contained by this subspace - /// Key to test + /// Key to test /// True if the key can exist inside the current subspace. /// Please note that this method does not test if the key *actually* exists in the database, only if the key is not ouside the range of keys defined by the subspace. [Pure] - bool Contains(Slice key); + bool Contains(Slice absoluteKey); //REVIEW: should this be renamed to "ContainsKey" ? /// Check that a key fits inside this subspace, and return '' or '\xFF' if it is outside the bounds /// Key that needs to be checked @@ -89,43 +78,14 @@ IKeySubspace this[Slice suffix] /// The unchanged if it is contained in the namespace, Slice.Empty if it was before the subspace, or FdbKey.MaxValue if it was after. Slice BoundCheck(Slice key, bool allowSystemKeys); - /// Return the key that is composed of the subspace's prefix and a binary suffix - /// Binary suffix that will be appended to the current prefix - /// Full binary key - [Pure] - Slice ConcatKey(Slice suffix); - - /// Concatenate a batch of keys under this subspace - /// List of suffixes to process - /// Array of which is equivalent to calling on each entry in - [Pure, NotNull] - Slice[] ConcatKeys([NotNull] IEnumerable suffixes); - //REVIEW: could this be done via an extension method? - /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace - /// Complete key that contains the current subspace prefix, and a binary suffix - /// If true, verify that is inside the bounds of the subspace + /// Complete key that contains the current subspace prefix, and a binary suffix + /// If true, verify that is inside the bounds of the subspace /// Binary suffix of the key (or Slice.Empty if the key is exactly equal to the subspace prefix). If the key is outside of the subspace, returns Slice.Nil - /// This is the inverse operation of - /// If is true and is outside the current subspace. - [Pure] - Slice ExtractKey(Slice key, bool boundCheck = false); - - /// Remove the subspace prefix from a batch of binary keys, and only return the tail, or Slice.Nil if a key does not fit inside the namespace - /// Sequence of complete keys that contains the current subspace prefix, and a binary suffix - /// If true, verify that each key in is inside the bounds of the subspace - /// Array of only the binary suffix of the keys, Slice.Empty for a key that is exactly equal to the subspace prefix, or Slice.Nil for a key that is outside of the subspace - /// If is true and at least one key in is outside the current subspace. - [Pure, NotNull] - Slice[] ExtractKeys([NotNull] IEnumerable keys, bool boundCheck = false); - //REVIEW: could this be done via an extension method? - - /// Return a new slice buffer, initialized with the subspace prefix, that can be used for custom key serialization - /// If non-zero, the expected buffer capacity. The size of the subspace prefix will be added to this value. - /// Instance of a SliceWriter with the prefix of this subspace already copied. + /// This is the inverse operation of + /// If is true and is outside the current subspace. [Pure] - SliceWriter GetWriter(int capacity = 0); - //REVIEW: this is an internal implementation detail that may be moved to a different interface? + Slice ExtractKey(Slice absoluteKey, bool boundCheck = false); } diff --git a/FoundationDB.Client/Subspaces/ITypedKeySubspace.cs b/FoundationDB.Client/Subspaces/ITypedKeySubspace.cs deleted file mode 100644 index 558f5c057..000000000 --- a/FoundationDB.Client/Subspaces/ITypedKeySubspace.cs +++ /dev/null @@ -1,96 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using JetBrains.Annotations; - - [PublicAPI] - public interface ITypedKeySubspace : IKeySubspace - { - - IKeyEncoder Encoder { [NotNull] get; } - - /// Return a view of all the possible keys of this subspace - TypedKeys Keys { get; } - - /// Returns an helper object that knows how to create sub-partitions of this subspace - TypedSubspacePartition Partition { get; } - - } - - public interface ITypedKeySubspace : IKeySubspace - { - - ICompositeKeyEncoder Encoder {[NotNull] get; } - - /// Return a view of all the possible keys of this subspace - TypedKeys Keys { get; } - - /// Returns an helper object that knows how to create sub-partitions of this subspace - TypedSubspacePartition Partition { get; } - - ITypedKeySubspace Partial {[NotNull] get; } - - } - - public interface ITypedKeySubspace : IKeySubspace - { - - ICompositeKeyEncoder Encoder {[NotNull] get; } - - /// Return a view of all the possible keys of this subspace - TypedKeys Keys { get; } - - /// Returns an helper object that knows how to create sub-partitions of this subspace - TypedSubspacePartition Partition { get; } - - ITypedKeySubspace Head { [NotNull] get; } - - ITypedKeySubspace Partial {[NotNull] get; } - } - - public interface ITypedKeySubspace : IKeySubspace - { - - ICompositeKeyEncoder Encoder {[NotNull] get; } - - /// Return a view of all the possible keys of this subspace - TypedKeys Keys { get; } - - /// Returns an helper object that knows how to create sub-partitions of this subspace - TypedSubspacePartition Partition { get; } - - ITypedKeySubspace Head {[NotNull] get; } - - ITypedKeySubspace Partial {[NotNull] get; } - - } - -} diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 185950c99..639394a5e 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -32,269 +32,61 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; + using FoundationDB.Layers.Directories; using JetBrains.Annotations; /// Adds a prefix on every keys, to group them inside a common subspace [PublicAPI] public class KeySubspace : IKeySubspace, IEquatable, IComparable { - /// Empty subspace, that does not add any prefix to the keys - public static readonly IKeySubspace Empty = new KeySubspace(Slice.Empty); - /// Binary prefix of this subspace - private Slice m_rawPrefix; //PERF: readonly struct + /// Prefix common to all keys in this subspace + private readonly Slice Key; - /// Returns the key of this directory subspace - /// This should only be used by methods that can use the key internally, even if it is not supposed to be exposed (as is the case for directory partitions) - protected Slice InternalKey => m_rawPrefix; + /// Precomputed range that encompass all the keys in this subspace + private readonly KeyRange Range; #region Constructors... - /// Wraps an existing subspace, without copying the prefix (if possible) - protected KeySubspace([NotNull] IKeySubspace copy) - { - Contract.NotNull(copy, nameof(copy)); - Slice key = copy is KeySubspace sub ? sub.m_rawPrefix : copy.GetPrefix(); - if (key.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(copy)); - m_rawPrefix = key; - } - - /// Create a new subspace from a binary prefix - /// Prefix of the new subspace - /// If true, take a copy of the prefix - internal KeySubspace(Slice rawPrefix, bool copy) - { - if (rawPrefix.IsNull) throw new ArgumentException("The subspace key cannot be null. Use Slice.Empty if you want a subspace with no prefix.", nameof(rawPrefix)); - if (copy) rawPrefix = rawPrefix.Memoize(); - m_rawPrefix = rawPrefix.Memoize(); - } - - /// Create a new subspace from a binary prefix - /// Prefix of the new subspace - public KeySubspace(Slice rawPrefix) - : this(rawPrefix, true) - { } - - #endregion - - #region Static Prefix Helpers... - - /// Create a new Subspace using a binary key as the prefix - /// Prefix of the new subspace - /// New subspace that will use a copy of as its prefix - [Pure, NotNull] - public static IKeySubspace Create(Slice slice) - { - return new DynamicKeySubspace(slice, TypeSystem.Default.GetDynamicEncoder()); - } - - /// Create a new Subspace using a binary key as the prefix - /// Prefix of the new subspace - /// Type System used to encode the keys of this subspace - /// New subspace that will use a copy of as its prefix - [Pure, NotNull] - public static IDynamicKeySubspace CreateDynamic(Slice slice, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new DynamicKeySubspace(slice, encoder); - } - - /// Create a new Subspace using a binary key as the prefix - /// Prefix of the new subspace - /// Type System used to encode the keys of this subspace - /// New subspace that will use a copy of as its prefix - [Pure, NotNull] - public static IDynamicKeySubspace CreateDynamic(Slice slice, [NotNull] IDynamicKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new DynamicKeySubspace(slice, encoder); - } - - /// Create a new Subspace using a tuples as the prefix - /// Tuple that represents the prefix of the new subspace - /// Optional type encoding used by this subspace. - /// New subspace instance that will use the packed representation of as its prefix - [Pure, NotNull] - public static IDynamicKeySubspace CreateDynamic([NotNull] ITuple tuple, IKeyEncoding encoding = null) - { - Contract.NotNull(tuple, nameof(tuple)); - var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new DynamicKeySubspace(TuPack.Pack(tuple), copy: true, encoder: encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(slice, encoder); - } - - [Pure, NotNull] - public static ITypedKeySubspace CreateEncoder(Slice slice, ICompositeKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(slice, encoder); - } - - /// Clone this subspace - /// New Subspace that uses the same prefix key - /// Hint: Cloning a special Subspace like a or will not keep all the "special abilities" of the parent. - [Pure, NotNull] - public static IKeySubspace Copy([NotNull] IKeySubspace subspace) - { - if (subspace is DynamicKeySubspace dyn) - { - return new DynamicKeySubspace(dyn.InternalKey, copy: true, encoder: dyn.Encoder); - } - - if (subspace is KeySubspace sub) - { - //SPOILER WARNING: You didn't hear it from me, but some say that you can use this to bypass the fact that FdbDirectoryPartition.get_Key and ToRange() throws in v2.x ... If you bypass this protection and bork your database, don't come crying! - return new KeySubspace(sub.InternalKey, copy: true); - } - - return new KeySubspace(subspace.GetPrefix(), copy: true); - } - - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static IDynamicKeySubspace CopyDynamic([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetDynamicEncoder(); - return new DynamicKeySubspace(subspace.GetPrefix(), true, encoder); - } - - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static IDynamicKeySubspace CopyDynamic([NotNull] IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new DynamicKeySubspace(subspace.GetPrefix(), true, encoder); - } - - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(subspace.GetPrefix(), true, encoder); - } - - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); - } - - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) - { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); - } + [NotNull] + public static KeySubspace Empty => new KeySubspace(Slice.Empty); - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System + /// Initializes a new subspace with the given prefix [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static KeySubspace FromKey(Slice prefix) { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); + return new KeySubspace(prefix.Memoize()); } - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System + /// Initializes a new subspace with the given prefix [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) + public static KeySubspace FromKey(ITuple prefix) { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); + return new KeySubspace(TuPack.Pack(prefix).Memoize()); } - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static KeySubspace Copy(IKeySubspace subspace) { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); + return subspace is KeySubspace ks ? new KeySubspace(ks.Key.Memoize()) : new KeySubspace(subspace.GetPrefix().Memoize()); } - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, IKeyEncoding encoding = null) + internal KeySubspace(Slice prefix) { - var encoder = (encoding ?? TypeSystem.Default).GetEncoder(); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); + this.Key = prefix; + this.Range = KeyRange.StartsWith(prefix); } - /// Create a copy of a subspace, using a specific Type System - /// New Subspace that uses the same prefix key, and the provided Type System - [Pure, NotNull] - public static ITypedKeySubspace CopyEncoder([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + internal KeySubspace(Slice prefix, KeyRange range) { - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), copy: true, encoder: encoder); + this.Key = prefix; + this.Range = range; } - #endregion - - #region IFdbSubspace... - /// Returns the raw prefix of this subspace /// Will throw if the prefix is not publicly visible, as is the case for Directory Partitions public Slice GetPrefix() @@ -308,129 +100,77 @@ public Slice GetPrefix() [DebuggerStepThrough] protected virtual Slice GetKeyPrefix() { - return m_rawPrefix; + return this.Key; } - protected virtual IKeySubspace CreateChildren(Slice suffix) + /// Returns the master instance of the prefix, without any safety checks + /// This instance should NEVER be exposed to anyone else, and should ONLY be used for logging/troubleshooting + protected Slice GetPrefixUnsafe() { - return new KeySubspace(ConcatKey(suffix)); + return this.Key; } public KeyRange ToRange() { - return ToRange(Slice.Empty); + return GetKeyRange(); } - public virtual KeyRange ToRange(Slice suffix) + protected virtual KeyRange GetKeyRange() { - return KeyRange.StartsWith(ConcatKey(suffix)); + return this.Range; } - public IKeySubspace this[Slice suffix] => CreateChildren(suffix); + public virtual KeyRange ToRange(Slice suffix) + { + return KeyRange.StartsWith(this[suffix]); + } - /// Tests whether the specified starts with this Subspace's prefix, indicating that the Subspace logically contains . - /// The key to be tested + /// Tests whether the specified key starts with this Subspace's prefix, indicating that the Subspace logically contains key. + /// The key to be tested /// The key Slice.Nil is not contained by any Subspace, so subspace.Contains(Slice.Nil) will always return false - public virtual bool Contains(Slice key) + public virtual bool Contains(Slice absoluteKey) { - return key.HasValue && key.StartsWith(this.InternalKey); + return absoluteKey.StartsWith(this.Key); } /// Append a key to the subspace key /// This is the equivalent of calling 'subspace.Key + suffix' - public Slice ConcatKey(Slice suffix) + public Slice this[Slice relativeKey] { - //REVIEW: what to do with Slice.Nil? - return GetKeyPrefix().Concat(suffix); - } - - /// Merge an array of keys with the subspace's prefix, all sharing the same buffer - /// Array of keys to pack - /// Array of slices (for all keys) that share the same underlying buffer - public Slice[] ConcatKeys(IEnumerable keys) - { - Contract.NotNull(keys, nameof(keys)); - //REVIEW: what to do with keys that are Slice.Nil ? - return Slice.ConcatRange(GetKeyPrefix(), keys); + get + { + //note: we don't want to leak our key! + var key = GetKeyPrefix(); + if (relativeKey.IsNullOrEmpty) return key.Memoize(); //TODO: better solution! + return key.Concat(relativeKey); + } } /// Remove the subspace prefix from a binary key, and only return the tail, or Slice.Nil if the key does not fit inside the namespace - /// Complete key that contains the current subspace prefix, and a binary suffix - /// If true, verify that is inside the bounds of the subspace + /// Complete key that contains the current subspace prefix, and a binary suffix + /// If true, verify that is inside the bounds of the subspace /// Binary suffix of the key (or Slice.Empty is the key is exactly equal to the subspace prefix). If the key is outside of the subspace, returns Slice.Nil /// This is the inverse operation of - /// If is true and is outside the current subspace. - public Slice ExtractKey(Slice key, bool boundCheck = false) + /// If is true and is outside the current subspace. + public virtual Slice ExtractKey(Slice absoluteKey, bool boundCheck = false) { - if (key.IsNull) return Slice.Nil; + if (absoluteKey.IsNull) return Slice.Nil; - var prefix = GetKeyPrefix(); - if (!key.StartsWith(prefix)) + var key = GetKeyPrefix(); + if (!absoluteKey.StartsWith(key)) { - if (boundCheck) FailKeyOutOfBound(key); + if (boundCheck) FailKeyOutOfBound(absoluteKey); return Slice.Nil; } - - return key.Substring(prefix.Count); - } - - /// Remove the subspace prefix from a batch of binary keys, and only return the tail, or Slice.Nil if a key does not fit inside the namespace - /// Sequence of complete keys that contains the current subspace prefix, and a binary suffix - /// If true, verify that each key in is inside the bounds of the subspace - /// Array of only the binary suffix of the keys, Slice.Empty for a key that is exactly equal to the subspace prefix, or Slice.Nil for a key that is outside of the subspace - /// If is true and at least one key in is outside the current subspace. - public Slice[] ExtractKeys(IEnumerable keys, bool boundCheck = false) - { - Contract.NotNull(keys, nameof(keys)); - - var prefix = GetKeyPrefix(); - - if (keys is Slice[] arr) - { // fast-path for Sice[] (frequent for range reads) - - var res = new Slice[arr.Length]; - for (int i = 0; i < arr.Length; i++) - { - if (arr[i].StartsWith(prefix)) - { - res[i] = arr[i].Substring(prefix.Count); - } - else if (boundCheck) - { - FailKeyOutOfBound(arr[i]); - } - } - return res; - } - else - { // slow path for the rest - var res = keys is ICollection coll ? new List(coll.Count) : new List(); - foreach(var key in keys) - { - if (key.StartsWith(prefix)) - { - res.Add(key.Substring(prefix.Count)); - } - else if (boundCheck) - { - FailKeyOutOfBound(key); - } - } - return res.ToArray(); - } + return absoluteKey.Substring(key.Count); } - public SliceWriter GetWriter(int capacity = 0) + public SliceWriter OpenWriter(int extra = 32) { - if (capacity < 0) throw new ArgumentOutOfRangeException(nameof(capacity)); - - var prefix = GetKeyPrefix(); - if (capacity > 0) - { - capacity += prefix.Count; - //TODO: round up to multiple of 8? - } - return new SliceWriter(prefix, capacity); + var key = GetKeyPrefix(); + var sw = new SliceWriter(key.Count + extra); //TODO: BufferPool ? + sw.WriteBytes(key); + return sw; } #endregion @@ -443,9 +183,9 @@ public int CompareTo(IKeySubspace other) if (other == null) return +1; if (object.ReferenceEquals(this, other)) return 0; if (other is KeySubspace sub) - return this.InternalKey.CompareTo(sub.InternalKey); + return this.Key.CompareTo(sub.Key); else - return this.InternalKey.CompareTo(other.GetPrefix()); + return this.Key.CompareTo(other.GetPrefix()); } /// Test if both subspaces have the same prefix @@ -454,9 +194,9 @@ public bool Equals(IKeySubspace other) if (other == null) return false; if (object.ReferenceEquals(this, other)) return true; if (other is KeySubspace sub) - return this.InternalKey.Equals(sub.InternalKey); + return this.Key.Equals(sub.Key); else - return this.InternalKey.Equals(other.GetPrefix()); + return this.Key.Equals(other.GetPrefix()); } /// Test if an object is a subspace with the same prefix @@ -469,7 +209,7 @@ public override bool Equals(object obj) /// public override int GetHashCode() { - return this.InternalKey.GetHashCode(); + return this.Key.GetHashCode(); } #endregion @@ -483,7 +223,7 @@ public override int GetHashCode() public Slice BoundCheck(Slice key, bool allowSystemKeys) { //note: Since this is needed to make GetRange/GetKey work properly, this should work for all subspace, include directory partitions - var prefix = this.InternalKey; + var prefix = this.Key; // don't touch to nil and keys inside the globalspace if (key.IsNull || key.StartsWith(prefix)) return key; @@ -521,7 +261,7 @@ protected void FailKeyOutOfBound(Slice key) public virtual string DumpKey(Slice key) { // note: we can't use ExtractAndCheck(...) because it may throw in derived classes - var prefix = this.InternalKey; + var prefix = this.Key; if (!key.StartsWith(prefix)) FailKeyOutOfBound(key); return FdbKey.Dump(key.Substring(prefix.Count)); @@ -530,7 +270,7 @@ public virtual string DumpKey(Slice key) /// Printable representation of this subspace public override string ToString() { - return "Subspace(" + this.InternalKey.ToString() + ")"; + return "Subspace(" + this.Key.ToString() + ")"; } #endregion diff --git a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index bd2bba752..3a118cf52 100644 --- a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Extensions methods and helpers to work with Key Subspaces @@ -48,7 +49,7 @@ public static IDynamicKeySubspace Using([NotNull] this IKeySubspace subspace, [N { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return KeySubspace.CopyDynamic(subspace, encoding); + return new DynamicKeySubspace(subspace.GetPrefix(), encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -60,7 +61,7 @@ public static IDynamicKeySubspace UsingEncoder([NotNull] this IKeySubspace subsp { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return KeySubspace.CopyDynamic(subspace, encoder); + return new DynamicKeySubspace(subspace.GetPrefix(), encoder.Encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -72,7 +73,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace s { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return KeySubspace.CopyEncoder(subspace, encoding); + return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -84,7 +85,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace s { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return KeySubspace.CopyEncoder(subspace, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -96,7 +97,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKey { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return KeySubspace.CopyEncoder(subspace, encoding); + return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -108,7 +109,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKey { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return KeySubspace.CopyEncoder(subspace, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -120,7 +121,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] t { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return KeySubspace.CopyEncoder(subspace, encoding); + return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -132,7 +133,7 @@ public static ITypedKeySubspace UsingEncoder([NotNull] t { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return KeySubspace.CopyEncoder(subspace, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -144,7 +145,7 @@ public static ITypedKeySubspace UsingEncoder([No { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); - return KeySubspace.CopyEncoder(subspace, encoding); + return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -156,7 +157,7 @@ public static ITypedKeySubspace UsingEncoder([No { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return KeySubspace.CopyEncoder(subspace, encoder); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Clear the entire content of a subspace diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs index fe6f003a5..5a1ae7279 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs @@ -29,133 +29,122 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; - using System.Collections.Generic; - using System.Diagnostics.Contracts; using Doxense.Collections.Tuples; + using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; + using FoundationDB.Layers.Directories; using JetBrains.Annotations; + [PublicAPI] + public interface ITypedKeySubspace : IKeySubspace + { + /// Return a view of all the possible keys of this subspace + [NotNull] + TypedKeys Keys { get; } + + /// Encoding used to generate and parse the keys of this subspace + [NotNull] + IKeyEncoder KeyEncoder { get; } + + } + /// Subspace that knows how to encode and decode its key - /// Type of the key handled by this subspace - public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + /// Type of the key handled by this subspace + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace { - public TypedKeySubspace(Slice rawPrefix, [NotNull] IKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } + public IKeyEncoder KeyEncoder { get; } - internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] IKeyEncoder encoder) - : base(rawPrefix, copy) + internal TypedKeySubspace(Slice prefix, [NotNull] IKeyEncoder encoder) + : base(prefix) { - this.Encoder = encoder ?? throw new ArgumentNullException(nameof(encoder)); - this.Keys = new TypedKeys(this, encoder); + Contract.Requires(encoder != null); + this.KeyEncoder = encoder; + this.Keys = new TypedKeys(this, this.KeyEncoder); } - [NotNull] - public IKeyEncoder Encoder { get; } - - public TypedKeys Keys { get; } + public TypedKeys Keys { get; } - public TypedSubspacePartition Partition => new TypedSubspacePartition(this, Encoder); } /// Encodes and Decodes keys composed of a single element - /// Type of the key handled by this subspace - public /*readonly*/ struct TypedKeys + /// Type of the key handled by this subspace + public sealed class TypedKeys { [NotNull] - public readonly IKeySubspace Subspace; + private readonly TypedKeySubspace Parent; [NotNull] - public readonly IKeyEncoder Encoder; + public IKeyEncoder Encoder { get; } - public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) + internal TypedKeys( + [NotNull] TypedKeySubspace parent, + [NotNull] IKeyEncoder encoder) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; + Contract.Requires(parent != null && encoder != null); + this.Parent = parent; this.Encoder = encoder; } - public Slice this[T value] => Encode(value); - - public Slice Encode(T value) - { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)); - } - - public Slice[] Encode([NotNull] IEnumerable values) - { - if (values == null) throw new ArgumentNullException(nameof(values)); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, T value, IKeyEncoder encoder) => { writer.WriteBytes(encoder.EncodeKey(value)); }, - this.Encoder - ); - } - - public T Decode(Slice packed) - { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); - } - - public KeyRange ToRange(T value) + public KeyRange ToRange(T1 item1) { - //REVIEW: which semantic for ToRange() should we use? - return TuPack.ToRange(Encode(value)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + var prefix = Encode(item1); + return KeyRange.PrefixedBy(prefix); } - } - - public /*readonly*/ struct TypedSubspacePartition - { - - [NotNull] - public readonly IKeySubspace Subspace; - - [NotNull] - public readonly IKeyEncoder Encoder; - - public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder encoder) + [Pure] + public Slice Pack(STuple tuple) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; + return Encode(tuple.Item1); } - [NotNull] - public IKeySubspace this[T value] => ByKey(value); - - [NotNull] - public IKeySubspace ByKey(T value) + [Pure] + public Slice Pack([NotNull] ITuple tuple) { - return this.Subspace[this.Encoder.EncodeKey(value)]; + return Encode(tuple.OfSize(1).Get(0)); } - [NotNull] - public IDynamicKeySubspace ByKey(T value, [NotNull] IKeyEncoding encoding) + [Pure] + public Slice Encode(T1 item1) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); + var bytes = this.Encoder.EncodeKey(item1); + var sw = this.Parent.OpenWriter(bytes.Count); + sw.WriteBytes(bytes); + return sw.ToSlice(); } - [NotNull] - public IDynamicKeySubspace ByKey(T value, [NotNull] IDynamicKeyEncoder encoder) + [Pure] + public T1 Decode(Slice packedKey) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } - [NotNull] - public ITypedKeySubspace ByKey(T value, [NotNull] IKeyEncoding encoding) + public void Decode(Slice packedKey, out T1 item1) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoding); + item1 = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } - [NotNull] - public ITypedKeySubspace ByKey(T value, [NotNull] IKeyEncoder encoder) + /// Return a user-friendly string representation of a key of this subspace + [Pure] + public string Dump(Slice packedKey) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value)), encoder); + if (packedKey.IsNull) return String.Empty; + //TODO: defer to the encoding itself? + var key = this.Parent.ExtractKey(packedKey); + try + { + //REVIEW: we need a TryUnpack! + return this.Encoder.DecodeKey(key).ToString(); + } + catch (Exception) + { // decoding failed, or some other non-trival + return key.PrettyPrint(); + } } } + } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs index 00e4c0e5f..f4e064c04 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -26,138 +26,224 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +//#define ENABLE_VALUETUPLES + namespace FoundationDB.Client { using System; - using System.Collections.Generic; + using System.Runtime.CompilerServices; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; - using Doxense.Collections.Tuples; - using Doxense.Memory; - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + public interface ITypedKeySubspace : IKeySubspace { - // ReSharper disable once FieldCanBeMadeReadOnly.Local + /// Helper to encode/decode keys using this subspace's default encoding + [NotNull] + TypedKeys Keys { get; } - public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } + /// Encoding used to generate and parse the keys of this subspace + [NotNull] + ICompositeKeyEncoder KeyEncoder { get; } - internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) - { - this.Encoder = encoder ?? throw new ArgumentNullException(nameof(encoder)); - this.Keys = new TypedKeys(this, encoder); - this.Partition = new TypedSubspacePartition(this, Encoder); - } + } - public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); - private TypedKeySubspace m_partial; + public class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + public ICompositeKeyEncoder KeyEncoder { get; } - public ICompositeKeyEncoder Encoder { get; } - public TypedKeys Keys { get; } + internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + : base(prefix) + { + Contract.Requires(encoder != null); + this.KeyEncoder = encoder; + this.Keys = new TypedKeys(this, this.KeyEncoder); + } - public TypedSubspacePartition Partition { get; } + public TypedKeys Keys { get; } } - public /*readonly*/ struct TypedKeys + public sealed class TypedKeys { [NotNull] - public readonly IKeySubspace Subspace; + private readonly TypedKeySubspace Parent; [NotNull] - public readonly ICompositeKeyEncoder Encoder; + public ICompositeKeyEncoder Encoder { get; } - public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + internal TypedKeys( + [NotNull] TypedKeySubspace parent, + [NotNull] ICompositeKeyEncoder encoder) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; + Contract.Requires(parent != null && encoder != null); + this.Parent = parent; this.Encoder = encoder; } - public Slice this[T1 value1, T2 value2] => Encode(value1, value2); + /// Return the range of all legal keys in this subpsace + /// A "legal" key is one that can be decoded into the original pair of values + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange() + { + return this.Parent.ToRange(); + } - public Slice Encode(T1 value1, T2 value2) + /// Return the range of all legal keys in this subpsace, that start with the specified pair of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(STuple tuple) { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)); + return ToRange(tuple.Item1, tuple.Item2); } - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2) + /// Return the range of all legal keys in this subpsace, that start with the specified pair of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange((T1, T2) tuple) { - Contract.NotNull(values, nameof(values)); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value))), - this.Encoder - ); + return ToRange(tuple.Item1, tuple.Item2); } - public STuple Decode(Slice packed) + /// Return the range of all legal keys in this subpsace, that start with the specified pair of values + /// Range that encompass all keys that start with (item1, item2, ..) + public KeyRange ToRange(T1 item1, T2 item2) { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + var prefix = Encode(item1, item2); + return KeyRange.PrefixedBy(prefix); } - public KeyRange ToRange(T1 value1, T2 value2) + /// Return the range of all legal keys in this subpsace, that start with the specified first item + /// Range that encompass all keys that start with (tuple.Item1, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRangePartial(STuple tuple) { - //REVIEW: which semantic for ToRange() should we use? - return TuPack.ToRange(Encode(value1, value2)); + return ToRangePartial(tuple.Item1); } - } + /// Return the range of all legal keys in this subpsace, that start with the specified first item + /// Range that encompass all keys that start with (item1, ..) + [Pure] + public KeyRange ToRangePartial(T1 item1) + { + return KeyRange.PrefixedBy(EncodePartial(item1)); + } - public /*readonly*/ struct TypedSubspacePartition - { - [NotNull] - public readonly IKeySubspace Subspace; + /// Pack a 2-tuple into a key in this subspace + /// Pair of values + /// Encoded key in this subspace + [Pure] + public Slice Pack(STuple tuple) + { + //REVIEW: how could we better guess the capacity, depending on the values of T1/T2? + var sw = this.Parent.OpenWriter(24); + this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); + return sw.ToSlice(); + } - [NotNull] - public readonly ICompositeKeyEncoder Encoder; +#if ENABLE_VALUETUPLES + /// Pack a 2-tuple into a key in this subspace + /// Pair of values + /// Encoded key in this subspace + [Pure] + public Slice Pack(ValueTuple tuple) + { + return Encode(tuple.Item1, tuple.Item2); + } +#endif - public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + /// Pack a 2-tuple into a key in this subspace + /// Tuple that must be of size 2 + /// Encoded key in this subspace + [Pure] + public Slice Pack([NotNull] ITuple tuple) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; + tuple.OfSize(2); + return Encode(tuple.Get(0), tuple.Get(1)); } - [NotNull] - public IKeySubspace this[T1 value1, T2 value2] => ByKey(value1, value2); + /// Pack a partial key only containing the first item of a key + /// Tuple containing a single item + /// Encoded partial key, to be used for generationg key ranges or key selectors + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice PackPartial(STuple tuple) + { + return EncodePartial(tuple.Item1); + } - [NotNull] - public IKeySubspace ByKey(T1 value1, T2 value2) + /// Pack a partial key only containing the first item of a key + /// Tuple containing a single item + /// Encoded partial key, to be used for generationg key ranges or key selectors + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice PackPartial(ValueTuple tuple) { - return this.Subspace[this.Encoder.EncodeKey(value1, value2)]; + return EncodePartial(tuple.Item1); } - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoding encoding) + /// Encode a pair of values into a key in this subspace + /// First part of the key + /// Second part of the key + /// Encoded key in this subspace + /// The key can be decoded back into its original components using or + [Pure] + public Slice Encode(T1 item1, T2 item2) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); + var sw = this.Parent.OpenWriter(24); + var tuple = new STuple(item1, item2); + this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); + return sw.ToSlice(); } - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, IDynamicKeyEncoder encoder) + [Pure] + public Slice EncodePartial(T1 item1) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); + var sw = this.Parent.OpenWriter(16); + var tuple = new STuple(item1, default(T2)); + this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); + return sw.ToSlice(); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoding encoding) + [Pure] + //REVIEW: return ValueTuple<..> instead? (C#7) + public STuple Decode(Slice packedKey) //REVIEW: => Unpack() { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoding); + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, IKeyEncoder encoder) + [Pure] + public T1 DecodePartial(Slice packedKey) + { + var parts = this.Encoder.DecodeKeyParts(1, packedKey); + return parts.Item1; + } + + public void Decode(Slice packedKey, out T1 item1, out T2 item2) + { + var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); + item1 = tuple.Item1; + item2 = tuple.Item2; + } + + /// Return a user-friendly string representation of a key of this subspace + [Pure] + public string Dump(Slice packedKey) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2)), encoder); + if (packedKey.IsNull) return String.Empty; + //TODO: defer to the encoding itself? + var key = this.Parent.ExtractKey(packedKey); + try + { + //REVIEW: we need a TryUnpack! + return this.Encoder.DecodeKey(key).ToString(); + } + catch (Exception) + { // decoding failed, or some other non-trival + return key.PrettyPrint(); + } } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index c3c3a9f57..6cd64b2c1 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -26,141 +26,206 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +//#define ENABLE_VALUETUPLES + namespace FoundationDB.Client { using System; - using System.Collections.Generic; + using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using Doxense.Memory; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace + public interface ITypedKeySubspace : IKeySubspace + { + /// Helper to encode/decode keys using this subspace's default encoding + [NotNull] + TypedKeys Keys { get; } + + /// Encoding used to generate and parse the keys of this subspace + [NotNull] + ICompositeKeyEncoder KeyEncoder { get; } + + } + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace { - public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } + public ICompositeKeyEncoder KeyEncoder { get; } - internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) + internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + : base(prefix) { Contract.Requires(encoder != null); - this.Encoder = encoder; - this.Keys = new TypedKeys(this, encoder); - this.Partition = new TypedSubspacePartition(this, Encoder); + this.KeyEncoder = encoder; + this.Keys = new TypedKeys(this, this.KeyEncoder); } - public ITypedKeySubspace Head => m_head ?? (m_head = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); - private TypedKeySubspace m_head; - - public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(Encoder))); - private TypedKeySubspace m_partial; - - public ICompositeKeyEncoder Encoder { get; } - public TypedKeys Keys { get; } - public TypedSubspacePartition Partition { get; } - } - /// Encodes and Decodes keys composed of three elements - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace - public /*readonly*/ struct TypedKeys + public sealed class TypedKeys { - public readonly IKeySubspace Subspace; - public readonly ICompositeKeyEncoder Encoder; + [NotNull] + private readonly TypedKeySubspace Parent; + + [NotNull] + public ICompositeKeyEncoder Encoder { get; } - public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + internal TypedKeys( + [NotNull] TypedKeySubspace parent, + [NotNull] ICompositeKeyEncoder encoder) { - this.Subspace = subspace; + Contract.Requires(parent != null && encoder != null); + this.Parent = parent; this.Encoder = encoder; } - public Slice this[T1 value1, T2 value2, T3 value3] => Encode(value1, value2, value3); + /// Return the range of all legal keys in this subpsace + /// A "legal" key is one that can be decoded into the original triple of values + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange() + { + return this.Parent.ToRange(); + } - public Slice Encode(T1 value1, T2 value2, T3 value3) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(STuple tuple) { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)); + return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(ValueTuple tuple) { - if (values == null) throw new ArgumentNullException(nameof(values)); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value))), - this.Encoder - ); + return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } - public STuple Decode(Slice packed) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRange(T1 item1, T2 item2, T3 item3) { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(Encode(item1, item2, item3)); } - public KeyRange ToRange(T1 value1, T2 value2, T3 value3) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(STuple tuple) { - //REVIEW: which semantic for ToRange() should we use? - return TuPack.ToRange(Encode(value1, value2, value3)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); } - } + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(ValueTuple tuple) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); + } - public /*readonly*/ struct TypedSubspacePartition - { - [NotNull] - public readonly IKeySubspace Subspace; + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(T1 item1, T2 item2) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(item1, item2)); + } - [NotNull] - public readonly ICompositeKeyEncoder Encoder; + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(T1 item1) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(item1)); + } - public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(STuple tuple) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; + return Encode(tuple.Item1, tuple.Item2, tuple.Item3); } - [NotNull] - public IKeySubspace this[T1 value1, T2 value2, T3 value3] => ByKey(value1, value2, value3); +#if ENABLE_VALUETUPLES + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(ValueTuple tuple) + { + return Encode(tuple.Item1, tuple.Item2, tuple.Item3); + } +#endif - [NotNull] - public IKeySubspace ByKey(T1 value1, T2 value2, T3 value3) + [Pure] + public Slice Pack(TTuple tuple) + where TTuple : ITuple { - return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3)]; + tuple.OfSize(3); + return Encode(tuple.Get(0), tuple.Get(1), tuple.Get(2)); } - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoding encoding) + [Pure] + public Slice Encode(T1 item1, T2 item2, T3 item3) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); + var bytes = this.Encoder.EncodeKey(item1, item2, item3); + var sw = this.Parent.OpenWriter(bytes.Count); + sw.WriteBytes(bytes); + return sw.ToSlice(); } - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IDynamicKeyEncoder encoder) + [Pure] + public Slice EncodePartial(T1 item1, T2 item2) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); + var sw = this.Parent.OpenWriter(16); + var tuple = new STuple(item1, item2, default(T3)); + this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); + return sw.ToSlice(); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoding encoding) + [Pure] + public Slice EncodePartial(T1 item1) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoding); + var sw = this.Parent.OpenWriter(16); + var tuple = new STuple(item1, default(T2), default(T3)); + this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); + return sw.ToSlice(); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, IKeyEncoder encoder) + [Pure] + public STuple Decode(Slice packedKey) + { + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); + } + + public void Decode(Slice packedKey, out T1 item1, out T2 item2, out T3 item3) + { + var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); + item1 = tuple.Item1; + item2 = tuple.Item2; + item3 = tuple.Item3; + } + + /// Return a user-friendly string representation of a key of this subspace + [Pure] + public string Dump(Slice packedKey) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3)), encoder); + if (packedKey.IsNull) return String.Empty; + //TODO: defer to the encoding itself? + var key = this.Parent.ExtractKey(packedKey); + try + { + //REVIEW: we need a TryUnpack! + return this.Encoder.DecodeKey(key).ToString(); + } + catch (Exception) + { // decoding failed, or some other non-trival + return key.PrettyPrint(); + } } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index 394ad0857..7cf2ee4b6 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -26,150 +26,224 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +//#define ENABLE_VALUETUPLES + namespace FoundationDB.Client { using System; - using System.Collections.Generic; + using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; - /// Subspace that knows how to encode and decode its key - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace - /// Type of the fourth item of the keys handled by this subspace - public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + public interface ITypedKeySubspace : IKeySubspace { - // ReSharper disable once FieldCanBeMadeReadOnly.Local - private TypedKeySubspace m_head; - private TypedKeySubspace m_partial; - - public TypedKeySubspace(Slice rawPrefix, [NotNull] ICompositeKeyEncoder encoder) - : this(rawPrefix, true, encoder) - { } + /// Helper to encode/decode keys using this subspace's default encoding + [NotNull] + TypedKeys Keys { get; } - internal TypedKeySubspace(Slice rawPrefix, bool copy, [NotNull] ICompositeKeyEncoder encoder) - : base(rawPrefix, copy) - { - Contract.Requires(encoder != null); - this.Encoder = encoder; - this.Keys = new TypedKeys(this, encoder); - this.Partition = new TypedSubspacePartition(this, Encoder); - } + /// Encoding used to generate and parse the keys of this subspace + [NotNull] + ICompositeKeyEncoder KeyEncoder { get; } - public ITypedKeySubspace Head => m_head ?? (m_head = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Head(Encoder))); + } - public ITypedKeySubspace Partial => m_partial ?? (m_partial = new TypedKeySubspace(GetKeyPrefix(), false, KeyValueEncoders.Pair(Encoder))); + public sealed class TypedKeySubspace : KeySubspace, ITypedKeySubspace + { + public ICompositeKeyEncoder KeyEncoder { get; } - public ICompositeKeyEncoder Encoder { get; } + internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + : base(prefix) + { + this.KeyEncoder = encoder; + this.Keys = new TypedKeys(this, this.KeyEncoder); + } public TypedKeys Keys { get; } - public TypedSubspacePartition Partition { get; } - } - /// Encodes and Decodes keys composed of four elements - /// Type of the first item of the keys handled by this subspace - /// Type of the second item of the keys handled by this subspace - /// Type of the third item of the keys handled by this subspace - /// Type of the fourth item of the keys handled by this subspace - public /*readonly*/ struct TypedKeys + public sealed class TypedKeys { [NotNull] - public readonly IKeySubspace Subspace; + private readonly TypedKeySubspace Parent; [NotNull] - public readonly ICompositeKeyEncoder Encoder; + public ICompositeKeyEncoder Encoder { get; } - public TypedKeys([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + internal TypedKeys( + [NotNull] TypedKeySubspace parent, + [NotNull] ICompositeKeyEncoder encoder) { - System.Diagnostics.Contracts.Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; + Contract.Requires(parent != null && encoder != null); + this.Parent = parent; this.Encoder = encoder; } - public Slice this[T1 value1, T2 value2, T3 value3, T4 value4] => Encode(value1, value2, value3, value4); + /// Return the range of all legal keys in this subpsace + /// A "legal" key is one that can be decoded into the original triple of values + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange() + { + return this.Parent.ToRange(); + } - public Slice Encode(T1 value1, T2 value2, T3 value3, T4 value4) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(STuple tuple) { - return this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)); + return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } - [NotNull] - public Slice[] Encode([NotNull] IEnumerable values, [NotNull] Func selector1, [NotNull] Func selector2, [NotNull] Func selector3, [NotNull] Func selector4) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(ValueTuple tuple) { - if (values == null) throw new ArgumentNullException(nameof(values)); - return Batched>.Convert( - this.Subspace.GetWriter(), - values, - (ref SliceWriter writer, TSource value, ICompositeKeyEncoder encoder) => writer.WriteBytes(encoder.EncodeKey(selector1(value), selector2(value), selector3(value), selector4(value))), - this.Encoder - ); + return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } - public STuple Decode(Slice packed) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRange(T1 item1, T2 item2, T3 item3, T4 item4) { - return this.Encoder.DecodeKey(this.Subspace.ExtractKey(packed)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(Encode(item1, item2, item3, item4)); } - public KeyRange ToRange(T1 value1, T2 value2, T3 value3, T4 value4) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(STuple tuple) { - //REVIEW: which semantic for ToRange() should we use? - return TuPack.ToRange(Encode(value1, value2, value3, value4)); + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); } - } + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(ValueTuple tuple) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); + } - public /*readonly*/ struct TypedSubspacePartition - { - [NotNull] - public readonly IKeySubspace Subspace; + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(T1 item1, T2 item2, T3 item3) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(item1, item2, item3)); + } - [NotNull] - public readonly ICompositeKeyEncoder Encoder; + /// Return the range of all legal keys in this subpsace, that start with the specified pair of values + /// Range that encompass all keys that start with (item1, item2) + public KeyRange ToRangePartial(T1 item1, T2 item2) + { + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(item1, item2)); + } - public TypedSubspacePartition([NotNull] IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values + /// Range that encompass all keys that start with (item1, item2, item3) + public KeyRange ToRangePartial(T1 item1) { - Contract.Requires(subspace != null && encoder != null); - this.Subspace = subspace; - this.Encoder = encoder; + //HACKHACK: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(EncodePartial(item1)); } - [NotNull] - public IKeySubspace this[T1 value1, T2 value2, T3 value3, T4 value4] => ByKey(value1, value2, value3, value4); + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(STuple tuple) + { + return Encode(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); + } - [NotNull] - public IKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4) +#if ENABLE_VALUETUPLES + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(ValueTuple tuple) { - return this.Subspace[this.Encoder.EncodeKey(value1, value2, value3, value4)]; + return Encode(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } +#endif - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoding encoding) + [Pure] + public Slice Pack(TTuple tuple) + where TTuple : ITuple { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); + tuple.OfSize(4); + return Encode(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3)); } - [NotNull] - public IDynamicKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IDynamicKeyEncoder encoder) + [Pure] + public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4) { - return KeySubspace.CreateDynamic(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); + var bytes = this.Encoder.EncodeKey(item1, item2, item3, item4); + var sw = this.Parent.OpenWriter(bytes.Count); + sw.WriteBytes(bytes); + return sw.ToSlice(); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoding encoding) + [Pure] + public Slice EncodePartial(T1 item1, T2 item2, T3 item3) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoding); + var sw = this.Parent.OpenWriter(24); + var tuple = new STuple(item1, item2, item3, default(T4)); + this.Encoder.WriteKeyPartsTo(ref sw, 3, ref tuple); + return sw.ToSlice(); } - [NotNull] - public ITypedKeySubspace ByKey(T1 value1, T2 value2, T3 value3, T4 value4, [NotNull] IKeyEncoder encoder) + [Pure] + public Slice EncodePartial(T1 item1, T2 item2) + { + var sw = this.Parent.OpenWriter(16); + var tuple = new STuple(item1, item2, default(T3), default(T4)); + this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); + return sw.ToSlice(); + } + + [Pure] + public Slice EncodePartial(T1 item1) + { + var sw = this.Parent.OpenWriter(16); + var tuple = new STuple(item1, default(T2), default(T3), default(T4)); + this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); + return sw.ToSlice(); + } + + [Pure] + public STuple Decode(Slice packedKey) + { + return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); + } + + public void Decode(Slice packedKey, out T1 item1, out T2 item2, out T3 item3, out T4 item4) + { + var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); + item1 = tuple.Item1; + item2 = tuple.Item2; + item3 = tuple.Item3; + item4 = tuple.Item4; + } + + /// Return a user-friendly string representation of a key of this subspace + [Pure] + public string Dump(Slice packedKey) { - return KeySubspace.CreateEncoder(this.Subspace.ConcatKey(this.Encoder.EncodeKey(value1, value2, value3, value4)), encoder); + if (packedKey.IsNull) return String.Empty; + //TODO: defer to the encoding itself? + var key = this.Parent.ExtractKey(packedKey); + try + { + //REVIEW: we need a TryUnpack! + return this.Encoder.DecodeKey(key).ToString(); + } + catch (Exception) + { // decoding failed, or some other non-trival + return key.PrettyPrint(); + } } } diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index 861c972be..1471d4554 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Client using System; using Doxense.Collections.Tuples; using Doxense.Memory; + using Doxense.Serialization.Encoders; public abstract class DynamicKeyEncoderBase : IDynamicKeyEncoder { @@ -43,7 +44,7 @@ public virtual KeyRange ToRange(Slice prefix) return KeyRange.StartsWith(prefix); } - public abstract void PackKey(ref SliceWriter writer, ITuple items); + public abstract void PackKey(ref SliceWriter writer, TTuple items) where TTuple : ITuple; public virtual void EncodeKey(ref SliceWriter writer, T1 item1) { @@ -122,6 +123,11 @@ public virtual STuple DecodeKey(Slice pa return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e) => STuple.Create(a, b, c, d, e)); } + public virtual STuple DecodeKey(Slice packed) + { + return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e, T6 f) => STuple.Create(a, b, c, d, e, f)); + } + public virtual KeyRange ToRange(Slice prefix, ITuple items) { var writer = new SliceWriter(prefix, 16); diff --git a/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs new file mode 100644 index 000000000..92d259548 --- /dev/null +++ b/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs @@ -0,0 +1,270 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Serialization.Encoders +{ + using System; + using Doxense.Collections.Tuples; + using Doxense.Memory; + + /// Encoder for keys that are tuples + /// Type of tuple + public interface ICompositeKeyEncoder : IKeyEncoder + where TTuple : ITuple + { + /// Write some or all parts of a composite key + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref TTuple key); + + /// Read some or all parts of a composite key + void ReadKeyPartsFrom(ref SliceReader reader, int count, out TTuple items); + } + + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + + public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + + public static partial class KeyEncoderExtensions + { + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2) + { + var tuple = new STuple(value1, value2); + encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2) + { + var writer = default(SliceWriter); + var tuple = new STuple(item1, item2); + encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2) + { + var writer = new SliceWriter(prefix.Count + 24); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2); + return writer.ToSlice(); + } + + public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, T1 item1) + { + var writer = default(SliceWriter); + var tuple = new STuple(item1, default(T2)); + encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1) + { + var writer = new SliceWriter(prefix.Count + 16); + writer.WriteBytes(prefix); + var tuple = new STuple(item1, default(T2)); + encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out STuple items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3) + { + var tuple = new STuple(value1, value2, value3); + encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3) + { + var writer = default(SliceWriter); + var tuple = new STuple(item1, item2, item3); + encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3) + { + var writer = new SliceWriter(prefix.Count + 32); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out STuple items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4) + { + var tuple = new STuple(value1, value2, value3, value4); + encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4) + { + var writer = default(SliceWriter); + var tuple = new STuple(item1, item2, item3, item4); + encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + { + var writer = new SliceWriter(prefix.Count + 48); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3, item4); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out STuple items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) + { + var tuple = new STuple(value1, value2, value3, value4, value5); + encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var writer = default(SliceWriter); + var tuple = new STuple(item1, item2, item3, item4, item5); + encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var writer = new SliceWriter(prefix.Count + 56); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3, item4, item5); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out STuple items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + return items; + } + + + #endregion + + } + +} diff --git a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs similarity index 94% rename from FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs rename to FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs index 5cb9b5736..d59acf4d1 100644 --- a/FoundationDB.Client/TypeSystem/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013-2018, Doxense SAS +/* Copyright (c) 2013-2015, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without @@ -26,11 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using Doxense.Collections.Tuples; using Doxense.Memory; + using FoundationDB; using JetBrains.Annotations; /// Encoder that can process keys of variable size and types @@ -40,19 +41,13 @@ public interface IDynamicKeyEncoder /// Return the parent key encoding IKeyEncoding Encoding {[NotNull] get; } - /// Return a range that contains all the keys under a subspace of the encoder subspace, using the semantic of the encoding - /// Optional binary prefix - /// Key range which derives from the semantic of the current encoding - /// For example, the Tuple encoding will produce ranges of the form "(Key + \x00) <= x < (Key + \xFF)", while a binary-based encoding would produce ranges of the form "Key <= x < Increment(Key)" - KeyRange ToRange(Slice prefix = default(Slice)); - #region Encoding... /// Pack a tuple of arbitrary length into a binary slice /// Buffer where to append the binary representation /// Tuple of any size (0 to N) /// If some elements in are not supported by this type system - void PackKey(ref SliceWriter writer, ITuple items); + void PackKey(ref SliceWriter writer, TTuple items) where TTuple : ITuple; /// Encode a key composed of a single element into a binary slice /// Type of the element @@ -211,10 +206,27 @@ public interface IDynamicKeyEncoder /// Tuple containing five elements, or an exception if the data is invalid, or the tuples has less or more than five elements STuple DecodeKey(Slice packed); + /// Decode a binary slice containing exactly six elements + /// Expected type of the first element + /// Expected type of the second element + /// Expected type of the third element + /// Expected type of the fourth element + /// Expected type of the fifth element + /// Expected type of the sixth element + /// Binary slice produced by a previous call to or + /// Tuple containing five elements, or an exception if the data is invalid, or the tuples has less or more than five elements + STuple DecodeKey(Slice packed); + #endregion #region Ranges... + /// Return a range that contains all the keys under a subspace of the encoder subspace, using the semantic of the encoding + /// Optional binary prefix + /// Key range which derives from the semantic of the current encoding + /// For example, the Tuple encoding will produce ranges of the form "(Key + \x00) <= x < (Key + \xFF)", while a binary-based encoding would produce ranges of the form "Key <= x < Increment(Key)" + KeyRange ToRange(Slice prefix = default(Slice)); + /// Return a key range using a tuple as a prefix /// Optional binary prefix that should be added before encoding the key /// Tuple of any size (0 to N) diff --git a/FoundationDB.Client/TypeSystem/IKeyEncoder`1.cs b/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs similarity index 63% rename from FoundationDB.Client/TypeSystem/IKeyEncoder`1.cs rename to FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs index 33f40c62b..b355227f9 100644 --- a/FoundationDB.Client/TypeSystem/IKeyEncoder`1.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs @@ -26,16 +26,44 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; + using Doxense.Memory; public interface IKeyEncoder { /// Encode a single value - Slice EncodeKey(T1 value); + void WriteKeyTo(ref SliceWriter writer, T1 value); /// Decode a single value - T1 DecodeKey(Slice encoded); + void ReadKeyFrom(ref SliceReader reader, out T1 value); + } + + public static partial class KeyEncoderExtensions + { + + public static Slice EncodeKey(this IKeyEncoder encoder, T1 value) + { + var writer = default(SliceWriter); + encoder.WriteKeyTo(ref writer, value); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this IKeyEncoder encoder, Slice prefix, T1 value) + { + var writer = new SliceWriter(prefix.Count + 16); // ~16 bytes si T1 = Guid + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, value); + return writer.ToSlice(); + } + + public static T1 DecodeKey(this IKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out T1 item); + //TODO: should we fail if extra bytes? + return item; + } } } diff --git a/FoundationDB.Client/TypeSystem/IValueEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/IValueEncoder.cs similarity index 97% rename from FoundationDB.Client/TypeSystem/IValueEncoder.cs rename to FoundationDB.Client/TypeSystem/Encoders/IValueEncoder.cs index a677f671c..7d8b5ba0e 100644 --- a/FoundationDB.Client/TypeSystem/IValueEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IValueEncoder.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using JetBrains.Annotations; @@ -44,4 +44,4 @@ public interface IValueEncoder T DecodeValue(Slice encoded); } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs new file mode 100644 index 000000000..2cdf95c40 --- /dev/null +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs @@ -0,0 +1,175 @@ + +namespace Doxense.Serialization.Encoders +{ + using JetBrains.Annotations; + using System; + using Doxense.Collections.Tuples; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + + /// Helper class for all key/value encoders + public static partial class KeyValueEncoders + { + + /// Encoders that produce lexicographically ordered slices, suitable for keys where lexicographical ordering is required + public static class Ordered + { + [NotNull] + public static IKeyEncoder BinaryEncoder => Tuples.Key(); + + [NotNull] + public static IKeyEncoder StringEncoder => Tuples.Key(); + + [NotNull] + public static IKeyEncoder Int32Encoder => Tuples.Key(); + + [NotNull] + public static IKeyEncoder Int64Encoder => Tuples.Key(); + + [NotNull] + public static IKeyEncoder UInt64Encoder => Tuples.Key(); + + [NotNull] + public static IKeyEncoder GuidEncoder => Tuples.Key(); + + public sealed class OrderedKeyEncoder : IKeyEncoder + { + private readonly IOrderedTypeCodec m_codec; + + public OrderedKeyEncoder(IOrderedTypeCodec codec) + { + Contract.Requires(codec != null); + m_codec = codec; + } + + public void WriteKeyTo(ref SliceWriter writer, T key) + { + //TODO: PERF: optimize this! + writer.WriteBytes(m_codec.EncodeOrdered(key)); + } + + public void ReadKeyFrom(ref SliceReader reader, out T key) + { + key = m_codec.DecodeOrdered(reader.ReadToEnd()); + } + } + + public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder + { + private readonly IOrderedTypeCodec m_codec1; + private readonly IOrderedTypeCodec m_codec2; + + public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec codec2) + { + m_codec1 = codec1; + m_codec2 = codec2; + } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items) + { + Contract.Requires(count > 0); + if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); + if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items) + { + Contract.Requires(count > 0); + + T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default(T1); + T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default(T2); + if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); + items = new STuple(key1, key2); + } + } + + public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder + { + private readonly IOrderedTypeCodec m_codec1; + private readonly IOrderedTypeCodec m_codec2; + private readonly IOrderedTypeCodec m_codec3; + + public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec codec2, IOrderedTypeCodec codec3) + { + m_codec1 = codec1; + m_codec2 = codec2; + m_codec3 = codec3; + } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items) + { + Contract.Requires(count > 0 && count <= 3); + if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); + if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); + if (count >= 3) m_codec3.EncodeOrderedSelfTerm(ref writer, items.Item3); + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items) + { + Contract.Requires(count > 0); + + T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default(T1); + T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default(T2); + T3 key3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default(T3); + if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); + items = new STuple(key1, key2, key3); + } + + } + + /// Create a simple encoder from a codec + [NotNull] + public static IKeyEncoder Bind([NotNull] IOrderedTypeCodec codec) + { + Contract.NotNull(codec, nameof(codec)); + + return new OrderedKeyEncoder(codec); + } + + /// Create a composite encoder from a pair of codecs + [NotNull] + public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2) + { + Contract.NotNull(codec1, nameof(codec1)); + Contract.NotNull(codec2, nameof(codec2)); + + return new CodecCompositeKeyEncoder(codec1, codec2); + } + + /// Create a composite encoder from a triplet of codecs + [NotNull] + public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2, [NotNull] IOrderedTypeCodec codec3) + { + Contract.NotNull(codec1, nameof(codec1)); + Contract.NotNull(codec2, nameof(codec2)); + Contract.NotNull(codec3, nameof(codec3)); + + return new CodecCompositeKeyEncoder(codec1, codec2, codec3); + } + + public static void Partial(ref SliceWriter writer, IOrderedTypeCodec codec1, T1 value1) + { + Contract.Assert(codec1 != null); + codec1.EncodeOrderedSelfTerm(ref writer, value1); + } + + public static void Encode(ref SliceWriter writer, [NotNull] IOrderedTypeCodec codec1, T1 value1, [NotNull] IOrderedTypeCodec codec2, T2 value2) + { + Contract.Assert(codec1 != null && codec2 != null); + codec1.EncodeOrderedSelfTerm(ref writer, value1); + codec2.EncodeOrderedSelfTerm(ref writer, value2); + } + + public static void Encode(ref SliceWriter writer, [NotNull] IOrderedTypeCodec codec1, T1 value1, [NotNull] IOrderedTypeCodec codec2, T2 value2, [NotNull] IOrderedTypeCodec codec3, T3 value3) + { + Contract.Assert(codec1 != null && codec2 != null && codec3 != null); + codec1.EncodeOrderedSelfTerm(ref writer, value1); + codec2.EncodeOrderedSelfTerm(ref writer, value2); + codec3.EncodeOrderedSelfTerm(ref writer, value3); + } + + } + + } + +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs new file mode 100644 index 000000000..9950c3069 --- /dev/null +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -0,0 +1,223 @@ + +namespace Doxense.Serialization.Encoders +{ + using JetBrains.Annotations; + using System; + using Doxense.Collections.Tuples; + using Doxense.Collections.Tuples.Encoding; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + + /// Helper class for all key/value encoders + public static partial class KeyValueEncoders + { + + /// Encoders that use the Tuple Encoding, suitable for keys + public static class Tuples + { + + internal class TupleEncoder : IKeyEncoder, IValueEncoder + { + public static readonly TupleEncoder Default = new TupleEncoder(); + + private TupleEncoder() { } + + public void WriteKeyTo(ref SliceWriter writer, T key) + { + TupleEncoder.WriteKeysTo(ref writer, key); + } + + public void ReadKeyFrom(ref SliceReader reader, out T key) + { + key = !reader.HasMore + ? default(T) //BUGBUG + : TuPack.DecodeKey(reader.ReadToEnd()); + } + + public Slice EncodeValue(T key) + { + return TupleEncoder.EncodeKey(key); + } + + public T DecodeValue(Slice encoded) + { + if (encoded.IsNullOrEmpty) return default(T); //BUGBUG + return TuPack.DecodeKey(encoded); + } + + } + + internal class TupleCompositeEncoder : CompositeKeyEncoder + { + + public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); + + private TupleCompositeEncoder() { } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + { + switch (count) + { + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + { + if (count != 1 & count != 2) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key = new STuple( + t.Get(0), + count == 2 ? t.Get(1) : default(T2) + ); + } + } + + internal class TupleCompositeEncoder : CompositeKeyEncoder + { + + public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); + + private TupleCompositeEncoder() { } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + { + switch (count) + { + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + { + if (count < 1 | count > 3) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key = new STuple( + t.Get(0), + count >= 2 ? t.Get(1) : default(T2), + count >= 3 ? t.Get(2) : default(T3) + ); + } + } + + internal class TupleCompositeEncoder : CompositeKeyEncoder + { + + public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); + + private TupleCompositeEncoder() { } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + { + switch (count) + { + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + { + if (count < 1 || count > 4) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key = new STuple( + t.Get(0), + count >= 2 ? t.Get(1) : default(T2), + count >= 3 ? t.Get(2) : default(T3), + count >= 4 ? t.Get(3) : default(T4) + ); + } + } + + internal class TupleCompositeEncoder : CompositeKeyEncoder + { + + public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); + + private TupleCompositeEncoder() { } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + { + switch (count) + { + case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + { + if (count < 1 || count > 5) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key = new STuple( + t.Get(0), + count >= 2 ? t.Get(1) : default(T2), + count >= 3 ? t.Get(2) : default(T3), + count >= 4 ? t.Get(3) : default(T4), + count >= 5 ? t.Get(4) : default(T5) + ); + } + } + #region Keys + + [NotNull] + public static IKeyEncoder Key() + { + return TupleEncoder.Default; + } + + [NotNull] + public static ICompositeKeyEncoder CompositeKey() + { + return TupleCompositeEncoder.Default; + } + + [NotNull] + public static ICompositeKeyEncoder CompositeKey() + { + return TupleCompositeEncoder.Default; + } + + [NotNull] + public static ICompositeKeyEncoder CompositeKey() + { + return TupleCompositeEncoder.Default; + } + + #endregion + + #region Values... + + [NotNull] + public static IValueEncoder Value() + { + return TupleEncoder.Default; + } + + #endregion + + } + + } + +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs new file mode 100644 index 000000000..796fe6a4c --- /dev/null +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs @@ -0,0 +1,35 @@ + +namespace Doxense.Serialization.Encoders +{ + using JetBrains.Annotations; + using System; + using Doxense.Diagnostics.Contracts; + + /// Helper class for all key/value encoders + public static partial class KeyValueEncoders + { + + /// Encoders that produce compact but unordered slices, suitable for keys that don't benefit from having lexicographical ordering + public static class Unordered + { + + /// Create a simple encoder from a codec + [NotNull] + public static IKeyEncoder Bind([NotNull] IUnorderedTypeCodec codec) + { + Contract.NotNull(codec, nameof(codec)); + + // ReSharper disable once SuspiciousTypeConversion.Global + if (codec is IKeyEncoder encoder) return encoder; + + return new Singleton( + (value) => codec.EncodeUnordered(value), + (encoded) => codec.DecodeUnordered(encoded) + ); + } + + } + + } + +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs new file mode 100644 index 000000000..cf8326c20 --- /dev/null +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs @@ -0,0 +1,121 @@ + +namespace Doxense.Serialization.Encoders +{ + using JetBrains.Annotations; + using System; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + + /// Helper class for all key/value encoders + public static partial class KeyValueEncoders + { + + /// Encoders that produce compact but unordered slices, suitable for values + public static class Values + { + private static readonly GenericEncoder s_default = new GenericEncoder(); + + public static IValueEncoder BinaryEncoder + { + [NotNull] + get { return s_default; } + } + + public static IValueEncoder StringEncoder + { + [NotNull] + get { return s_default; } + } + + public static IValueEncoder Int32Encoder + { + [NotNull] + get { return s_default; } + } + + public static IValueEncoder Int64Encoder + { + [NotNull] + get { return s_default; } + } + + public static IValueEncoder GuidEncoder + { + [NotNull] + get { return s_default; } + } + + /// Create a simple encoder from a codec + [NotNull] + public static IValueEncoder Bind([NotNull] IUnorderedTypeCodec codec) + { + Contract.NotNull(codec, nameof(codec)); + + if (codec is IValueEncoder encoder) return encoder; + + return new Singleton( + (value) => codec.EncodeUnordered(value), + (encoded) => codec.DecodeUnordered(encoded) + ); + } + + internal sealed class GenericEncoder : IValueEncoder, IValueEncoder, IValueEncoder, IValueEncoder, IValueEncoder + { + + public Slice EncodeValue(Slice value) + { + return value; + } + + Slice IValueEncoder.DecodeValue(Slice encoded) + { + return encoded; + } + + public Slice EncodeValue(string value) + { + return Slice.FromString(value); + } + + string IValueEncoder.DecodeValue(Slice encoded) + { + return encoded.ToUnicode(); + } + + public Slice EncodeValue(int value) + { + return Slice.FromInt32(value); + } + + int IValueEncoder.DecodeValue(Slice encoded) + { + return encoded.ToInt32(); + } + + public Slice EncodeValue(long value) + { + return Slice.FromInt64(value); + } + + long IValueEncoder.DecodeValue(Slice encoded) + { + return encoded.ToInt64(); + } + + public Slice EncodeValue(Guid value) + { + return Slice.FromGuid(value); + } + + Guid IValueEncoder.DecodeValue(Slice encoded) + { + return encoded.ToGuid(); + } + + } + + } + + } + +} diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index 0c32aaea9..552b2548d 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -26,20 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using System.Collections.Generic; using System.Linq; + using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Helper class for all key/value encoders [PublicAPI] - public static class KeyValueEncoders + public static partial class KeyValueEncoders { /// Identity function for binary slices public static readonly IdentityEncoder Binary = new IdentityEncoder(); @@ -52,14 +53,14 @@ public sealed class IdentityEncoder : IKeyEncoder, IValueEncoder internal IdentityEncoder() { } - public Slice EncodeKey(Slice key) + public void WriteKeyTo(ref SliceWriter writer, Slice key) { - return key; + writer.WriteBytes(key); } - public Slice DecodeKey(Slice encoded) + public void ReadKeyFrom(ref SliceReader reader, out Slice value) { - return encoded; + value = reader.ReadToEnd(); } public Slice EncodeValue(Slice value) @@ -92,14 +93,14 @@ public Type[] GetTypes() return new[] { typeof(T) }; } - public Slice EncodeKey(T value) + public void WriteKeyTo(ref SliceWriter writer, T value) { - return m_encoder(value); + writer.WriteBytes(m_encoder(value)); } - public T DecodeKey(Slice encoded) + public void ReadKeyFrom(ref SliceReader reader, out T value) { - return m_decoder(encoded); + value = m_decoder(reader.ReadToEnd()); } public Slice EncodeValue(T value) @@ -118,33 +119,19 @@ public T DecodeValue(Slice encoded) public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(STuple key, int items); - - public abstract STuple DecodeComposite(Slice encoded, int items); - - public Slice EncodeKey(STuple key) - { - return EncodeComposite(key, 2); - } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); - public virtual Slice EncodeKey(T1 item1, T2 item2) - { - return EncodeComposite(STuple.Create(item1, item2), 2); - } + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); - public virtual STuple DecodeKey(Slice encoded) + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteKeyTo(ref SliceWriter writer, STuple items) { - return DecodeComposite(encoded, 2); + WriteKeyPartsTo(ref writer, 2, ref items); } - public T1 DecodePartialKey(Slice encoded) + public void ReadKeyFrom(ref SliceReader reader, out STuple items) { - return DecodeComposite(encoded, 1).Item1; - } - - public HeadEncoder Head() - { - return new HeadEncoder(this); + ReadKeyPartsFrom(ref reader, 2, out items); } } @@ -153,38 +140,18 @@ public HeadEncoder Head() public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(STuple items, int count); - - public abstract STuple DecodeComposite(Slice encoded, int count); + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); - public Slice EncodeKey(STuple key) - { - return EncodeComposite(key, 3); - } + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); - public virtual Slice EncodeKey(T1 item1, T2 item2, T3 item3) + public void WriteKeyTo(ref SliceWriter writer, STuple items) { - return EncodeComposite(STuple.Create(item1, item2, item3), 3); + WriteKeyPartsTo(ref writer, 3, ref items); } - public virtual STuple DecodeKey(Slice encoded) + public void ReadKeyFrom(ref SliceReader reader, out STuple items) { - return DecodeComposite(encoded, 3); - } - - public STuple DecodeKey(Slice encoded, int items) - { - return DecodeComposite(encoded, items); - } - - public HeadEncoder Head() - { - return new HeadEncoder(this); - } - - public PairEncoder Pair() - { - return new PairEncoder(this); + ReadKeyPartsFrom(ref reader, 3, out items); } } @@ -193,675 +160,44 @@ public PairEncoder Pair() public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - public abstract Slice EncodeComposite(STuple items, int count); - - public abstract STuple DecodeComposite(Slice encoded, int count); - - public Slice EncodeKey(STuple key) - { - return EncodeComposite(key, 4); - } - - public virtual Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) - { - return EncodeComposite(STuple.Create(item1, item2, item3, item4), 4); - } - - public virtual STuple DecodeKey(Slice encoded) - { - return DecodeComposite(encoded, 4); - } - - public STuple DecodeKey(Slice encoded, int items) - { - return DecodeComposite(encoded, items); - } - } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); - /// Wrapper for a composite encoder that will only output the first key - public struct HeadEncoder : IKeyEncoder - { - [NotNull] - public readonly ICompositeKeyEncoder Encoder; + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); - public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) + public void WriteKeyTo(ref SliceWriter writer, STuple items) { - Contract.Requires(encoder != null); - this.Encoder = encoder; + WriteKeyPartsTo(ref writer, 4, ref items); } - public Slice EncodeKey(T1 value) + public void ReadKeyFrom(ref SliceReader reader, out STuple items) { - return this.Encoder.EncodeComposite(new STuple(value, default(T2)), 1); + ReadKeyPartsFrom(ref reader, 4, out items); } - public T1 DecodeKey(Slice encoded) - { - return this.Encoder.DecodeComposite(encoded, 1).Item1; - } } - /// Wrapper for a composite encoder that will only output the first key - public struct HeadEncoder : IKeyEncoder - { - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(encoder != null); - this.Encoder = encoder; - } - - public Slice EncodeKey(T1 value) - { - return this.Encoder.EncodeComposite(new STuple(value, default(T2), default(T3)), 1); - } - - public T1 DecodeKey(Slice encoded) - { - return this.Encoder.DecodeComposite(encoded, 1).Item1; - } - } - - /// Wrapper for a composite encoder that will only output the first key - public struct HeadEncoder : IKeyEncoder - { - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public HeadEncoder([NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(encoder != null); - this.Encoder = encoder; - } - - public Slice EncodeKey(T1 value) - { - return this.Encoder.EncodeComposite(new STuple(value, default(T2), default(T3), default(T4)), 1); - } - - public T1 DecodeKey(Slice encoded) - { - return this.Encoder.DecodeComposite(encoded, 1).Item1; - } - } - - /// Wrapper for a composite encoder that will only output the first and second keys - public struct PairEncoder : ICompositeKeyEncoder - { - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - - public PairEncoder([NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(encoder != null); - this.Encoder = encoder; - } - - public Slice EncodeKey(T1 value1, T2 value2) - { - return this.Encoder.EncodeComposite(new STuple(value1, value2, default(T3)), 2); - } - - public Slice EncodeComposite(STuple key, int items) - { - return this.Encoder.EncodeComposite(new STuple(key.Item1, key.Item2, default(T3)), items); - } - - public STuple DecodeComposite(Slice encoded, int items) - { - var t = this.Encoder.DecodeComposite(encoded, items); - return new STuple(t.Item1, t.Item2); - } - - public Slice EncodeKey(STuple value) - { - return EncodeComposite(value, 2); - } - - public STuple DecodeKey(Slice encoded) - { - return DecodeComposite(encoded, 2); - } - - public HeadEncoder Head() - { - return new HeadEncoder(this.Encoder); - } - } - - /// Wrapper for a composite encoder that will only output the first and second keys - public struct PairEncoder : ICompositeKeyEncoder + /// Wrapper for encoding and decoding a quad with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { - [NotNull] - public readonly ICompositeKeyEncoder Encoder; - public PairEncoder([NotNull] ICompositeKeyEncoder encoder) - { - Contract.Requires(encoder != null); - this.Encoder = encoder; - } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); - public Slice EncodeKey(T1 value1, T2 value2) - { - return this.Encoder.EncodeComposite(new STuple(value1, value2, default(T3), default(T4)), 2); - } - - public Slice EncodeComposite(STuple key, int items) - { - return this.Encoder.EncodeComposite(new STuple(key.Item1, key.Item2, default(T3), default(T4)), items); - } - - public STuple DecodeComposite(Slice encoded, int items) - { - var t = this.Encoder.DecodeComposite(encoded, items); - return new STuple(t.Item1, t.Item2); - } + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); - public Slice EncodeKey(STuple value) + public void WriteKeyTo(ref SliceWriter writer, STuple items) { - return EncodeComposite(value, 2); + WriteKeyPartsTo(ref writer, 5, ref items); } - public STuple DecodeKey(Slice encoded) + public void ReadKeyFrom(ref SliceReader reader, out STuple items) { - return DecodeComposite(encoded, 2); + ReadKeyPartsFrom(ref reader, 5, out items); } - public HeadEncoder Head() - { - return new HeadEncoder(this.Encoder); - } } #endregion - /// Encoders that produce lexicographically ordered slices, suitable for keys where lexicographical ordering is required - public static class Ordered - { - public static IKeyEncoder BinaryEncoder - { - [NotNull] - get { return Tuples.Key(); } - } - - public static IKeyEncoder StringEncoder - { - [NotNull] - get { return Tuples.Key(); } - } - - public static IKeyEncoder Int32Encoder - { - [NotNull] - get { return Tuples.Key(); } - } - - public static IKeyEncoder Int64Encoder - { - [NotNull] - get { return Tuples.Key(); } - } - - public static IKeyEncoder UInt64Encoder - { - [NotNull] - get { return Tuples.Key(); } - } - - public sealed class OrderedKeyEncoder : IKeyEncoder - { - private readonly IOrderedTypeCodec m_codec; - - public OrderedKeyEncoder(IOrderedTypeCodec codec) - { - Contract.Requires(codec != null); - m_codec = codec; - } - - public Slice EncodeKey(T key) - { - return m_codec.EncodeOrdered(key); - } - - public T DecodeKey(Slice encoded) - { - return m_codec.DecodeOrdered(encoded); - } - } - - public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder - { - private readonly IOrderedTypeCodec m_codec1; - private readonly IOrderedTypeCodec m_codec2; - - public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec codec2) - { - m_codec1 = codec1; - m_codec2 = codec2; - } - - public override Slice EncodeComposite(STuple items, int count) - { - Contract.Requires(count > 0); - - var writer = default(SliceWriter); - if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); - if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); - return writer.ToSlice(); - } - - public override STuple DecodeComposite(Slice encoded, int count) - { - Contract.Requires(count > 0); - - var reader = new SliceReader(encoded); - T1 key1 = default(T1); - T2 key2 = default(T2); - if (count >= 1) key1 = m_codec1.DecodeOrderedSelfTerm(ref reader); - if (count >= 2) key2 = m_codec2.DecodeOrderedSelfTerm(ref reader); - if (reader.HasMore) throw new InvalidOperationException(String.Format("Unexpected data at the end of composite key after {0} items", count)); - return STuple.Create(key1, key2); - } - - } - - public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder - { - private readonly IOrderedTypeCodec m_codec1; - private readonly IOrderedTypeCodec m_codec2; - private readonly IOrderedTypeCodec m_codec3; - - public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec codec2, IOrderedTypeCodec codec3) - { - m_codec1 = codec1; - m_codec2 = codec2; - m_codec3 = codec3; - } - - public override Slice EncodeComposite(STuple items, int count) - { - Contract.Requires(count > 0 && count <= 3); - - var writer = default(SliceWriter); - if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); - if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); - if (count >= 3) m_codec3.EncodeOrderedSelfTerm(ref writer, items.Item3); - return writer.ToSlice(); - } - - public override STuple DecodeComposite(Slice encoded, int count) - { - Contract.Requires(count > 0); - - var reader = new SliceReader(encoded); - T1 key1 = default(T1); - T2 key2 = default(T2); - T3 key3 = default(T3); - if (count >= 1) key1 = m_codec1.DecodeOrderedSelfTerm(ref reader); - if (count >= 2) key2 = m_codec2.DecodeOrderedSelfTerm(ref reader); - if (count >= 3) key3 = m_codec3.DecodeOrderedSelfTerm(ref reader); - if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); - return STuple.Create(key1, key2, key3); - } - - } - - /// Create a simple encoder from a codec - [NotNull] - public static IKeyEncoder Bind([NotNull] IOrderedTypeCodec codec) - { - if (codec == null) throw new ArgumentNullException(nameof(codec)); - - return new OrderedKeyEncoder(codec); - } - - /// Create a composite encoder from a pair of codecs - [NotNull] - public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2) - { - if (codec1 == null) throw new ArgumentNullException(nameof(codec1)); - if (codec2 == null) throw new ArgumentNullException(nameof(codec2)); - - return new CodecCompositeKeyEncoder(codec1, codec2); - } - - /// Create a composite encoder from a triplet of codecs - [NotNull] - public static ICompositeKeyEncoder Bind([NotNull] IOrderedTypeCodec codec1, [NotNull] IOrderedTypeCodec codec2, [NotNull] IOrderedTypeCodec codec3) - { - if (codec1 == null) throw new ArgumentNullException(nameof(codec1)); - if (codec2 == null) throw new ArgumentNullException(nameof(codec2)); - if (codec3 == null) throw new ArgumentNullException(nameof(codec2)); - - return new CodecCompositeKeyEncoder(codec1, codec2, codec3); - } - - public static void Partial(ref SliceWriter writer, IOrderedTypeCodec codec1, T1 value1) - { - Contract.Assert(codec1 != null); - codec1.EncodeOrderedSelfTerm(ref writer, value1); - } - - public static void Encode(ref SliceWriter writer, [NotNull] IOrderedTypeCodec codec1, T1 value1, [NotNull] IOrderedTypeCodec codec2, T2 value2) - { - Contract.Assert(codec1 != null && codec2 != null); - codec1.EncodeOrderedSelfTerm(ref writer, value1); - codec2.EncodeOrderedSelfTerm(ref writer, value2); - } - - public static void Encode(ref SliceWriter writer, [NotNull] IOrderedTypeCodec codec1, T1 value1, [NotNull] IOrderedTypeCodec codec2, T2 value2, [NotNull] IOrderedTypeCodec codec3, T3 value3) - { - Contract.Assert(codec1 != null && codec2 != null && codec3 != null); - codec1.EncodeOrderedSelfTerm(ref writer, value1); - codec2.EncodeOrderedSelfTerm(ref writer, value2); - codec3.EncodeOrderedSelfTerm(ref writer, value3); - } - - } - - /// Encoders that produce compact but unordered slices, suitable for keys that don't benefit from having lexicographical ordering - public static class Unordered - { - - /// Create a simple encoder from a codec - [NotNull] - public static IKeyEncoder Bind([NotNull] IUnorderedTypeCodec codec) - { - if (codec == null) throw new ArgumentNullException(nameof(codec)); - - var encoder = codec as IKeyEncoder; - if (encoder != null) return encoder; - - return new Singleton( - (value) => codec.EncodeUnordered(value), - (encoded) => codec.DecodeUnordered(encoded) - ); - } - - } - - /// Encoders that produce compact but unordered slices, suitable for values - public static class Values - { - private static readonly GenericEncoder s_default = new GenericEncoder(); - - public static IValueEncoder BinaryEncoder - { - [NotNull] - get { return s_default; } - } - - public static IValueEncoder StringEncoder - { - [NotNull] - get { return s_default; } - } - - public static IValueEncoder Int32Encoder - { - [NotNull] - get { return s_default; } - } - - public static IValueEncoder Int64Encoder - { - [NotNull] - get { return s_default; } - } - - public static IValueEncoder GuidEncoder - { - [NotNull] - get { return s_default; } - } - - /// Create a simple encoder from a codec - [NotNull] - public static IValueEncoder Bind([NotNull] IUnorderedTypeCodec codec) - { - if (codec == null) throw new ArgumentNullException(nameof(codec)); - - var encoder = codec as IValueEncoder; - if (encoder != null) return encoder; - - return new Singleton( - (value) => codec.EncodeUnordered(value), - (encoded) => codec.DecodeUnordered(encoded) - ); - } - - internal sealed class GenericEncoder : IValueEncoder, IValueEncoder, IValueEncoder, IValueEncoder, IValueEncoder - { - - public Slice EncodeValue(Slice value) - { - return value; - } - - Slice IValueEncoder.DecodeValue(Slice encoded) - { - return encoded; - } - - public Slice EncodeValue(string value) - { - return Slice.FromString(value); - } - - string IValueEncoder.DecodeValue(Slice encoded) - { - return encoded.ToUnicode(); - } - - public Slice EncodeValue(int value) - { - return Slice.FromInt32(value); - } - - int IValueEncoder.DecodeValue(Slice encoded) - { - return encoded.ToInt32(); - } - - public Slice EncodeValue(long value) - { - return Slice.FromInt64(value); - } - - long IValueEncoder.DecodeValue(Slice encoded) - { - return encoded.ToInt64(); - } - - public Slice EncodeValue(Guid value) - { - return Slice.FromGuid(value); - } - - Guid IValueEncoder.DecodeValue(Slice encoded) - { - return encoded.ToGuid(); - } - - } - - } - - /// Encoders that use the Tuple Encoding, suitable for keys - public static class Tuples - { - - //TODO: rename to TupleEncoder! - internal class TupleKeyEncoder : IKeyEncoder, IValueEncoder - { - public static readonly TupleKeyEncoder Default = new TupleKeyEncoder(); - - private TupleKeyEncoder() { } - - public Slice EncodeKey(T key) - { - return TuPack.EncodeKey(key); - } - - public T DecodeKey(Slice encoded) - { - if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return TuPack.DecodeKey(encoded); - } - - public Slice EncodeValue(T key) - { - return TuPack.EncodeKey(key); - } - - public T DecodeValue(Slice encoded) - { - if (encoded.IsNullOrEmpty) return default(T); //BUGBUG - return TuPack.DecodeKey(encoded); - } - - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override Slice EncodeComposite(STuple key, int items) - { - switch (items) - { - case 2: return TuPack.EncodeKey(key.Item1, key.Item2); - case 1: return TuPack.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be either 1 or 2"); - } - } - - public override STuple DecodeComposite(Slice encoded, int items) - { - if (items < 1 || items > 2) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be either 1 or 2"); - - var t = TuPack.Unpack(encoded).OfSize(items); - Contract.Assert(t != null); - - return STuple.Create( - t.Get(0), - items >= 2 ? t.Get(1) : default(T2) - ); - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override Slice EncodeComposite(STuple key, int items) - { - switch (items) - { - case 3: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3); - case 2: return TuPack.EncodeKey(key.Item1, key.Item2); - case 1: return TuPack.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 3"); - } - } - - public override STuple DecodeComposite(Slice encoded, int items) - { - if (items < 1 || items > 3) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 3"); - - var t = TuPack.Unpack(encoded).OfSize(items); - Contract.Assert(t != null); - - return STuple.Create( - t.Get(0), - items >= 2 ? t.Get(1) : default(T2), - items >= 3 ? t.Get(2) : default(T3) - ); - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override Slice EncodeComposite(STuple key, int items) - { - switch (items) - { - case 4: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3, key.Item4); - case 3: return TuPack.EncodeKey(key.Item1, key.Item2, key.Item3); - case 2: return TuPack.EncodeKey(key.Item1, key.Item2); - case 1: return TuPack.EncodeKey(key.Item1); - default: throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 4"); - } - } - - public override STuple DecodeComposite(Slice encoded, int items) - { - if (items < 1 || items > 4) throw new ArgumentOutOfRangeException(nameof(items), items, "Item count must be between 1 and 4"); - - var t = TuPack.Unpack(encoded).OfSize(items); - - return STuple.Create( - t.Get(0), - items >= 2 ? t.Get(1) : default(T2), - items >= 3 ? t.Get(2) : default(T3), - items >= 4 ? t.Get(3) : default(T4) - ); - } - } - - #region Keys - - [NotNull] - public static IKeyEncoder Key() - { - return TupleKeyEncoder.Default; - } - - [NotNull] - public static ICompositeKeyEncoder CompositeKey() - { - return TupleCompositeEncoder.Default; - } - - [NotNull] - public static ICompositeKeyEncoder CompositeKey() - { - return TupleCompositeEncoder.Default; - } - - [NotNull] - public static ICompositeKeyEncoder CompositeKey() - { - return TupleCompositeEncoder.Default; - } - - #endregion - - #region Values... - - [NotNull] - public static IValueEncoder Value() - { - return TupleKeyEncoder.Default; - } - - #endregion - - } - #region Keys... /// Binds a pair of lambda functions to a key encoder @@ -872,8 +208,8 @@ public static IValueEncoder Value() [NotNull] public static IKeyEncoder Bind([NotNull] Func encoder, [NotNull] Func decoder) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (decoder == null) throw new ArgumentNullException(nameof(decoder)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(decoder, nameof(decoder)); return new Singleton(encoder, decoder); } @@ -881,8 +217,8 @@ public static IKeyEncoder Bind([NotNull] Func encoder, [NotNull] [NotNull] public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params T[] values) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (values == null) throw new ArgumentNullException(nameof(values)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); var slices = new Slice[values.Length]; for (int i = 0; i < values.Length; i++) @@ -896,9 +232,9 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotN [NotNull] public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable elements, Func selector) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (elements == null) throw new ArgumentNullException(nameof(elements)); - if (selector == null) throw new ArgumentNullException(nameof(selector)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(selector, nameof(selector)); TElement[] arr; ICollection coll; @@ -907,7 +243,7 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder(encoder, arr, selector); } - else if ((coll = elements as ICollection) != null) + if ((coll = elements as ICollection) != null) { // we can pre-allocate the result array var slices = new Slice[coll.Count]; int p = 0; @@ -917,22 +253,17 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder.EncodeKey(selector(item))) - .ToArray(); - } - + // slow path + return elements.Select((item) => encoder.EncodeKey(selector(item))).ToArray(); } /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) [NotNull] public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] TElement[] elements, Func selector) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (elements == null) throw new ArgumentNullException(nameof(elements)); - if (selector == null) throw new ArgumentNullException(nameof(selector)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(selector, nameof(selector)); var slices = new Slice[elements.Length]; for (int i = 0; i < elements.Length; i++) @@ -946,19 +277,17 @@ public static Slice[] EncodeKeys([NotNull] this IKeyEncoder EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable values) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (values == null) throw new ArgumentNullException(nameof(values)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed - var array = values as T[]; - if (array != null) + if (values is T[] array) { // optimized path for arrays return EncodeKeys(encoder, array); } - var coll = values as ICollection; - if (coll != null) + if (values is ICollection coll) { // optimized path when we know the count var slices = new List(coll.Count); foreach (var value in coll) @@ -976,8 +305,8 @@ public static IEnumerable EncodeKeys([NotNull] this IKeyEncoder enc [NotNull] public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params Slice[] slices) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); var values = new T[slices.Length]; for (int i = 0; i < slices.Length; i++) @@ -991,8 +320,8 @@ public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] [NotNull] public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] KeyValuePair[] items) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (items == null) throw new ArgumentNullException(nameof(items)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(items, nameof(items)); var values = new T[items.Length]; for (int i = 0; i < items.Length; i++) @@ -1006,53 +335,14 @@ public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] [NotNull] public static IEnumerable DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable slices) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) return slices.Select(slice => encoder.DecodeKey(slice)); } - /// Returns a partial encoder that will only encode the first element - public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) - { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - return new HeadEncoder(encoder); - } - - /// Returns a partial encoder that will only encode the first element - public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) - { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - - return new HeadEncoder(encoder); - } - - /// Returns a partial encoder that will only encode the first element - public static HeadEncoder Head([NotNull] this ICompositeKeyEncoder encoder) - { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - - return new HeadEncoder(encoder); - } - - /// Returns a partial encoder that will only encode the first and second elements - public static PairEncoder Pair([NotNull] this ICompositeKeyEncoder encoder) - { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - - return new PairEncoder(encoder); - } - - /// Returns a partial encoder that will only encode the first and second elements - public static PairEncoder Pair([NotNull] this ICompositeKeyEncoder encoder) - { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - - return new PairEncoder(encoder); - } - #endregion #region Values... @@ -1061,8 +351,8 @@ public static PairEncoder Pair([NotNull] this IC [NotNull] public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] params T[] values) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (values == null) throw new ArgumentNullException(nameof(values)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); var slices = new Slice[values.Length]; for (int i = 0; i < values.Length; i++) @@ -1077,8 +367,8 @@ public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [ [NotNull] public static IEnumerable EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable values) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (values == null) throw new ArgumentNullException(nameof(values)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed @@ -1106,8 +396,8 @@ public static IEnumerable EncodeValues([NotNull] this IValueEncoder [NotNull] public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] params Slice[] slices) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); var values = new T[slices.Length]; for (int i = 0; i < slices.Length; i++) @@ -1122,8 +412,8 @@ public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotN [NotNull] public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] KeyValuePair[] items) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (items == null) throw new ArgumentNullException(nameof(items)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(items, nameof(items)); var values = new T[items.Length]; for (int i = 0; i < items.Length; i++) @@ -1138,8 +428,8 @@ public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotN [NotNull] public static IEnumerable DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable slices) { - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - if (slices == null) throw new ArgumentNullException(nameof(slices)); + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) diff --git a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs deleted file mode 100644 index c1f64ca2d..000000000 --- a/FoundationDB.Client/TypeSystem/ICompositeKeyEncoder.cs +++ /dev/null @@ -1,56 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - using Doxense.Collections.Tuples; - - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> - { - Slice EncodeKey(T1 value1, T2 value2, T3 value3, T4 value4); - } - - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> - { - Slice EncodeKey(T1 value1, T2 value2, T3 value3); - } - - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> - { - Slice EncodeKey(T1 value1, T2 value2); - } - - public interface ICompositeKeyEncoder : IKeyEncoder - where TTuple : ITuple - { - Slice EncodeComposite(TTuple key, int items); - - TTuple DecodeComposite(Slice encoded, int items); - } -} diff --git a/FoundationDB.Client/TypeSystem/IKeyEncoding.cs b/FoundationDB.Client/TypeSystem/IKeyEncoding.cs index 8d6a2807d..682788d93 100644 --- a/FoundationDB.Client/TypeSystem/IKeyEncoding.cs +++ b/FoundationDB.Client/TypeSystem/IKeyEncoding.cs @@ -29,7 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY using System; using JetBrains.Annotations; -namespace FoundationDB.Client //REVIEW: what namespace? +namespace Doxense.Serialization.Encoders { /// Type system that handles encoding and decoding of differnt types of keys /// @@ -81,4 +81,4 @@ public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTyp } -} \ No newline at end of file +} diff --git a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs index c99b461f7..c792e6b75 100644 --- a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs @@ -27,7 +27,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using Doxense.Memory; diff --git a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs index 60cd9a5f0..d6c15f524 100644 --- a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs +++ b/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using Doxense.Memory; diff --git a/FoundationDB.Client/TypeSystem/TypeCodec`1.cs b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs index 06300d7cc..76527f00f 100644 --- a/FoundationDB.Client/TypeSystem/TypeCodec`1.cs +++ b/FoundationDB.Client/TypeSystem/TypeCodec`1.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB.Client +namespace Doxense.Serialization.Encoders { using System; using Doxense.Memory; diff --git a/FoundationDB.Client/TypeSystem/TypeSystem.cs b/FoundationDB.Client/TypeSystem/TypeSystem.cs index 1eb8129d5..c9008d342 100644 --- a/FoundationDB.Client/TypeSystem/TypeSystem.cs +++ b/FoundationDB.Client/TypeSystem/TypeSystem.cs @@ -26,29 +26,21 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB.Client //REVIEW: what namespace? +namespace Doxense.Serialization.Encoders { using System; - using JetBrains.Annotations; using Doxense.Collections.Tuples.Encoding; + using JetBrains.Annotations; public static class TypeSystem { [NotNull] - public static readonly IKeyEncoding Default; + public static readonly IKeyEncoding Default = new TupleKeyEncoding(); [NotNull] - public static readonly IKeyEncoding Tuples; - - static TypeSystem() - { - var tuples = new TupleKeyEncoding(); - Tuples = tuples; + public static readonly IKeyEncoding Tuples = Default; - // default is the same a Tuples (for now?) - Default = tuples; - } + //TODO: more? or remove this class? } diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 8ed5fc450..f1fc88b21 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -35,6 +35,7 @@ namespace FoundationDB.Layers.Blobs using FoundationDB.Client; using JetBrains.Annotations; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization.Encoders; /// Represents a potentially large binary value in FoundationDB. [DebuggerDisplay("Subspace={Subspace}")] diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index eacd476b2..9cc4bdbcb 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -26,17 +26,17 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Layers.Collections { using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -160,7 +160,7 @@ public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTrans if (trans == null) throw new ArgumentNullException(nameof(trans)); if (ids == null) throw new ArgumentNullException(nameof(ids)); - var results = await trans.GetValuesAsync(this.Location.Keys.Encode(ids)).ConfigureAwait(false); + var results = await trans.GetValuesAsync(ids.Select(id => this.Location.Keys.Encode(id))).ConfigureAwait(false); return Optional.DecodeRange(this.ValueEncoder, results); } diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index 5c34910fc..a8e37639f 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -34,6 +34,7 @@ namespace FoundationDB.Layers.Collections using System.Diagnostics; using System.Threading.Tasks; using Doxense.Linq; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -169,7 +170,7 @@ public IAsyncEnumerable Get([NotNull] IFdbReadOnlyTransaction trans, TKe { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); + var range = KeyRange.StartsWith(this.Location.Keys.EncodePartial(key)); if (this.AllowNegativeValues) { return trans @@ -201,7 +202,7 @@ public Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey [NotNull] public IAsyncEnumerable> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) { - var range = KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key)); + var range = KeyRange.StartsWith(this.Location.Keys.EncodePartial(key)); var query = trans .GetRange(range) @@ -235,7 +236,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.ClearRange(KeyRange.StartsWith(this.Location.Partial.Keys.Encode(key))); + trans.ClearRange(KeyRange.StartsWith(this.Location.Keys.EncodePartial(key))); } /// Remove a value for a specific key diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 01192c4ca..8a8cd9473 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Collections using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; #if DEBUG using FoundationDB.Filters.Logging; diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index 27ac9f9e3..2e5e136a9 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Collections using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index 37ceb453c..dc951af2d 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Collections using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; diff --git a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs index 254502bbd..72ffe3de0 100644 --- a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs +++ b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Counters using JetBrains.Annotations; using System; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; /// Providers a dictionary of 64-bit counters that can be updated atomically /// Type of the key in the counter map diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index 494833c02..c7efbfc80 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Counters using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; /// Represents an integer value which can be incremented without conflict. /// Uses a sharded representation (which scales with contention) along with background coalescing... diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index fd69098f4..da530aa8f 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Indexing using System.Collections.Generic; using System.Diagnostics; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -146,7 +147,7 @@ public Task> LookupAsync([NotNull] IFdbReadOnlyTransaction trans, TVal [NotNull] public FdbRangeQuery Lookup(IFdbReadOnlyTransaction trans, TValue value, bool reverse = false) { - var prefix = this.Location.Partial.Keys.Encode(value); + var prefix = this.Location.Keys.EncodePartial(value); return trans .GetRange(KeyRange.StartsWith(prefix), new FdbRangeOptions { Reverse = reverse }) @@ -156,7 +157,7 @@ public FdbRangeQuery Lookup(IFdbReadOnlyTransaction trans, TValue value, bo [NotNull] public FdbRangeQuery LookupGreaterThan([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool orEqual, bool reverse = false) { - var prefix = this.Location.Partial.Keys.Encode(value); + var prefix = this.Location.Keys.EncodePartial(value); if (!orEqual) prefix = FdbKey.Increment(prefix); var space = new KeySelectorPair( @@ -172,7 +173,7 @@ public FdbRangeQuery LookupGreaterThan([NotNull] IFdbReadOnlyTransaction tr [NotNull] public FdbRangeQuery LookupLessThan([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool orEqual, bool reverse = false) { - var prefix = this.Location.Partial.Keys.Encode(value); + var prefix = this.Location.Keys.EncodePartial(value); if (orEqual) prefix = FdbKey.Increment(prefix); var space = new KeySelectorPair( diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index b646a694a..f13d80c54 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Layers.Interning using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; /// Provides a class for interning (aka normalizing, aliasing) commonly-used long strings into shorter representations. diff --git a/FoundationDB.Layers.Common/Optional`1.cs b/FoundationDB.Layers.Common/Optional`1.cs index 62d301494..82247328c 100644 --- a/FoundationDB.Layers.Common/Optional`1.cs +++ b/FoundationDB.Layers.Common/Optional`1.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers using System.Collections.Generic; using System.Linq; using System.Runtime.InteropServices; + using Doxense.Serialization.Encoders; /// Helper class for the value type public static class Optional diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index 6125688d5..d68320884 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Documents using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; /// Represents a collection of dictionaries of fields. @@ -60,7 +61,7 @@ public FdbDocumentCollection(KeySubspace subspace, Func selector protected virtual Task> LoadPartsAsync(IFdbReadOnlyTransaction trans, TId id) { - var key = this.Location.Partial.Keys.Encode(id); + var key = this.Location.Keys.EncodePartial(id); return trans .GetRange(KeyRange.StartsWith(key)) //TODO: options ? @@ -101,7 +102,7 @@ public void Insert(IFdbTransaction trans, TDocument document) var packed = this.ValueEncoder.EncodeValue(document); // Key Prefix = ...(id,) - var key = this.Location.Partial.Keys.Encode(id); + var key = this.Location.Keys.EncodePartial(id); // clear previous value trans.ClearRange(KeyRange.StartsWith(key)); @@ -166,7 +167,7 @@ public void Delete(IFdbTransaction trans, TId id) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var key = this.Location.Partial.Keys.Encode(id); + var key = this.Location.Keys.EncodePartial(id); trans.ClearRange(KeyRange.StartsWith(key)); } @@ -179,8 +180,9 @@ public void DeleteMultiple(IFdbTransaction trans, IEnumerable ids) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (ids == null) throw new ArgumentNullException(nameof(ids)); - foreach (var key in this.Location.Partial.Keys.Encode(ids)) + foreach (var id in ids) { + var key = this.Location.Keys.EncodePartial(id); trans.ClearRange(KeyRange.StartsWith(key)); } } diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 4113c952c..83c281821 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Blobs using System.Threading.Tasks; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index 955c5e32d..118e3ba13 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -38,6 +38,7 @@ namespace FoundationDB.Layers.Experimental.Indexing using System.Globalization; using System.Linq; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; /// Simple index that maps values of type into lists of ids of type /// Type of the unique id of each document or entity diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 3ac58d757..2e4feaeb5 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -34,6 +34,7 @@ namespace FoundationDB.Layers.Messaging using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Filters.Logging; using FoundationDB.Layers.Counters; diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index c9292d81c..cb1b38c8a 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -179,7 +179,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.Create(TuPack.EncodeKey(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.FromKey(STuple.Create(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); @@ -701,7 +701,7 @@ private static async Task BenchBulkInsertThenBulkReadAsync(IFdbDatabase db, int private static async Task BenchMergeSortAsync(IFdbDatabase db, int N, int K, int B, CancellationToken ct) { // create multiple lists - var location = db.Partition.ByKey("MergeSort"); + var location = db.GlobalSpace.Partition.ByKey("MergeSort"); await db.ClearRangeAsync(location, ct); var sources = Enumerable.Range(0, K).Select(i => 'A' + i).ToArray(); diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index b8c9ac5b8..ba9a5c317 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -278,7 +278,7 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.Create(TuPack.EncodeKey("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.FromKey(STuple.Create("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); @@ -353,7 +353,7 @@ public async Task Test_Database_Instance_Should_Have_Default_Root_Directory() Assert.That(dl.ContentSubspace, Is.Not.Null); Assert.That(dl.ContentSubspace.GetPrefix(), Is.EqualTo(db.GlobalSpace.GetPrefix())); Assert.That(dl.NodeSubspace, Is.Not.Null); - Assert.That(dl.NodeSubspace.GetPrefix(), Is.EqualTo(db.GlobalSpace.ConcatKey(Slice.FromByte(254)))); + Assert.That(dl.NodeSubspace.GetPrefix(), Is.EqualTo(db.GlobalSpace[Slice.FromByte(254)])); Assert.That(db.GlobalSpace.Contains(dl.ContentSubspace.GetPrefix()), Is.True); Assert.That(db.GlobalSpace.Contains(dl.NodeSubspace.GetPrefix()), Is.True); diff --git a/FoundationDB.Tests/Encoders/EncoderFacts.cs b/FoundationDB.Tests/Encoders/EncoderFacts.cs index 73ba2c1c8..4604ac9da 100644 --- a/FoundationDB.Tests/Encoders/EncoderFacts.cs +++ b/FoundationDB.Tests/Encoders/EncoderFacts.cs @@ -30,6 +30,7 @@ namespace FoundationDB.Client.Converters.Tests { using System; using Doxense.Collections.Tuples; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; @@ -120,23 +121,23 @@ public void Test_Tuple_Composite_Encoder() // partial key encoding - data = encoder.EncodeComposite(items, 2); + data = encoder.EncodeKeyParts(2, items); Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x, y))); - items = encoder.DecodeComposite(TuPack.EncodeKey(x, y), 2); + items = encoder.DecodeKeyParts(2, TuPack.EncodeKey(x, y)); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(y)); Assert.That(items.Item3, Is.EqualTo(default(Guid))); - data = encoder.EncodeComposite(items, 1); + data = encoder.EncodeKeyParts(1, items); Assert.That(data, Is.EqualTo(TuPack.EncodeKey(x))); - items = encoder.DecodeComposite(TuPack.EncodeKey(x), 1); + items = encoder.DecodeKeyParts(1, TuPack.EncodeKey(x)); Assert.That(items.Item1, Is.EqualTo(x)); Assert.That(items.Item2, Is.EqualTo(default(long))); Assert.That(items.Item3, Is.EqualTo(default(Guid))); // should fail if number of items to encode is out of range - Assert.That(() => { encoder.EncodeComposite(items, 4); }, Throws.InstanceOf()); - Assert.That(() => { encoder.EncodeComposite(items, 0); }, Throws.InstanceOf()); + Assert.That(() => { encoder.EncodeKeyParts(4, items); }, Throws.InstanceOf()); + Assert.That(() => { encoder.EncodeKeyParts(0, items); }, Throws.InstanceOf()); } } diff --git a/FoundationDB.Tests/Experimental/JsonNetCodec.cs b/FoundationDB.Tests/Experimental/JsonNetCodec.cs index 09e127d0b..32c1822c2 100644 --- a/FoundationDB.Tests/Experimental/JsonNetCodec.cs +++ b/FoundationDB.Tests/Experimental/JsonNetCodec.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Types.Json using System.Text; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using Newtonsoft.Json; diff --git a/FoundationDB.Tests/Experimental/ProtobufCodec.cs b/FoundationDB.Tests/Experimental/ProtobufCodec.cs index b9d05ae55..0ca86749a 100644 --- a/FoundationDB.Tests/Experimental/ProtobufCodec.cs +++ b/FoundationDB.Tests/Experimental/ProtobufCodec.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Types.ProtocolBuffers using System.IO; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client; public class ProtobufCodec : IValueEncoder, IUnorderedTypeCodec diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 81368c819..913e7e3ca 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -40,6 +40,7 @@ namespace FoundationDB.Layers.Directories using FoundationDB.Client.Tests; using FoundationDB.Filters.Logging; using NUnit.Framework; + using NUnit.Framework.Constraints; [TestFixture] public class DirectoryFacts : FdbTest @@ -853,92 +854,73 @@ public async Task Test_Directory_Partitions_Should_Disallow_Creation_Of_Direct_K var barKey = subdir.GetPrefix(); // the constraint will always be the same for all the checks - Action shouldFail = (del) => + void ShouldFail(ActualValueDelegate del) { Assert.That(del, Throws.InstanceOf().With.Message.Contains("root of a directory partition")); - }; - Action shouldPass = (del) => + } + + void ShouldPass(ActualValueDelegate del) { Assert.That(del, Throws.Nothing); - }; + } // === PASS === // these methods are allowed to succeed on directory partitions, because we need them for the rest to work - shouldPass(() => { var _ = KeySubspace.Copy(partition).GetPrefix(); }); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! - shouldPass(() => partition.ToString()); // EXCEPTION: this should never fail! - shouldPass(() => partition.DumpKey(barKey)); // EXCEPTION: this should always work, because this can be used for debugging and logging... - shouldPass(() => partition.BoundCheck(barKey, true)); // EXCEPTION: needs to work because it is used by GetRange() and GetKey() + ShouldPass(() => KeySubspace.Copy(partition).GetPrefix()); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! + ShouldPass(() => partition.ToString()); // EXCEPTION: this should never fail! + ShouldPass(() => partition.DumpKey(barKey)); // EXCEPTION: this should always work, because this can be used for debugging and logging... + ShouldPass(() => partition.BoundCheck(barKey, true)); // EXCEPTION: needs to work because it is used by GetRange() and GetKey() // === FAIL ==== // Key - shouldFail(() => { var _ = partition.GetPrefix(); }); + ShouldFail(() => partition.GetPrefix()); // Contains - shouldFail(() => partition.Contains(barKey)); + ShouldFail(() => partition.Contains(barKey)); // Extract / ExtractAndCheck / BoundCheck - shouldFail(() => partition.ExtractKey(barKey, boundCheck: false)); - shouldFail(() => partition.ExtractKey(barKey, boundCheck: true)); - shouldFail(() => partition.ExtractKeys(new[] { barKey, barKey + FdbKey.MinValue })); + ShouldFail(() => partition.ExtractKey(barKey, boundCheck: false)); + ShouldFail(() => partition.ExtractKey(barKey, boundCheck: true)); // Partition - shouldFail(() => partition.Partition.ByKey(123)); - shouldFail(() => partition.Partition.ByKey(123, "hello")); - shouldFail(() => partition.Partition.ByKey(123, "hello", false)); - shouldFail(() => partition.Partition.ByKey(123, "hello", false, "world")); + ShouldFail(() => partition.Partition.ByKey(123)); + ShouldFail(() => partition.Partition.ByKey(123, "hello")); + ShouldFail(() => partition.Partition.ByKey(123, "hello", false)); + ShouldFail(() => partition.Partition.ByKey(123, "hello", false, "world")); // Keys - shouldFail(() => partition.ConcatKey(Slice.FromString("hello"))); - shouldFail(() => partition.ConcatKey(location.GetPrefix())); - shouldFail(() => partition.ConcatKeys(new[] { Slice.FromString("hello"), Slice.FromString("world"), Slice.FromString("!") })); + ShouldFail(() => partition[Slice.FromString("hello")]); + ShouldFail(() => partition[location.GetPrefix()]); + ShouldFail(() => partition[STuple.Create("hello", 123)]); - shouldFail(() => { var _ = partition[Slice.FromString("hello")]; }); - shouldFail(() => { var _ = partition[location.GetPrefix()]; }); - - shouldFail(() => partition.ToRange()); - shouldFail(() => partition.ToRange(Slice.FromString("hello"))); - shouldFail(() => partition.ToRange(TuPack.EncodeKey("hello"))); + ShouldFail(() => partition.ToRange()); + ShouldFail(() => partition.ToRange(Slice.FromString("hello"))); + ShouldFail(() => partition.ToRange(TuPack.EncodeKey("hello"))); // Tuples - shouldFail(() => partition.Keys.Encode(123)); - shouldFail(() => partition.Keys.Encode(123, "hello")); - shouldFail(() => partition.Keys.Encode(123, "hello", false)); - shouldFail(() => partition.Keys.Encode(123, "hello", false, "world")); - shouldFail(() => partition.Keys.Encode(123)); - - shouldFail(() => partition.Keys.EncodeMany(new[] { 123, 456, 789 })); - shouldFail(() => partition.Keys.EncodeMany((IEnumerable)new[] { 123, 456, 789 })); - shouldFail(() => partition.Keys.EncodeMany(new object[] { 123, "hello", true })); - shouldFail(() => partition.Keys.EncodeMany((IEnumerable)new object[] { 123, "hello", true })); - - shouldFail(() => partition.Keys.Unpack(barKey)); - shouldFail(() => partition.Keys.UnpackMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); - shouldFail(() => partition.Keys.Decode(barKey)); - shouldFail(() => partition.Keys.DecodeMany(new[] { barKey, barKey })); - shouldFail(() => partition.Keys.DecodeLast(barKey)); - shouldFail(() => partition.Keys.DecodeLastMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); - shouldFail(() => partition.Keys.DecodeFirst(barKey)); - shouldFail(() => partition.Keys.DecodeFirstMany(new[] { barKey, barKey + TuPack.EncodeKey(123) })); - - //FIXME: need to re-enable this code! -#if REFACTORING_IN_PROGRESS - shouldFail(() => partition.Keys.ToTuple()); - - shouldFail(() => partition.Keys.Append(123)); - shouldFail(() => partition.Keys.Append(123, "hello")); - shouldFail(() => partition.Keys.Append(123, "hello", false)); - shouldFail(() => partition.Keys.Append(123, "hello", false, "world")); - shouldFail(() => partition.Keys.Concat(STuple.Create(123, "hello", false, "world"))); - shouldFail(() => partition.Keys.Append(new object[] { 123, "hello", false, "world" })); -#endif - - shouldFail(() => partition.Keys.ToRange()); - shouldFail(() => partition.ToRange(Slice.FromString("hello"))); - shouldFail(() => partition.Keys.ToRange(STuple.Create("hello"))); + ShouldFail(() => partition.Keys.Encode(123)); + ShouldFail(() => partition.Keys.Encode(123, "hello")); + ShouldFail(() => partition.Keys.Encode(123, "hello", false)); + ShouldFail(() => partition.Keys.Encode(123, "hello", false, "world")); + ShouldFail(() => partition.Keys.Encode(123)); + + ShouldFail(() => partition.Keys.EncodeMany(new[] { 123, 456, 789 })); + ShouldFail(() => partition.Keys.EncodeMany((IEnumerable)new[] { 123, 456, 789 })); + ShouldFail(() => partition.Keys.EncodeMany(new object[] { 123, "hello", true })); + ShouldFail(() => partition.Keys.EncodeMany((IEnumerable)new object[] { 123, "hello", true })); + + ShouldFail(() => partition.Keys.Unpack(barKey)); + ShouldFail(() => partition.Keys.Decode(barKey)); + ShouldFail(() => partition.Keys.DecodeLast(barKey)); + ShouldFail(() => partition.Keys.DecodeFirst(barKey)); + + ShouldFail(() => partition.Keys.ToRange()); + ShouldFail(() => partition.ToRange(Slice.FromString("hello"))); + ShouldFail(() => partition.Keys.ToRange(STuple.Create("hello"))); } } diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index 08530dc3f..d95437903 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Collections.Tests using System.Net; using System.Threading.Tasks; using Doxense.Collections.Tuples; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; diff --git a/FoundationDB.Tests/Layers/MultiMapFacts.cs b/FoundationDB.Tests/Layers/MultiMapFacts.cs index ec1009945..048df6424 100644 --- a/FoundationDB.Tests/Layers/MultiMapFacts.cs +++ b/FoundationDB.Tests/Layers/MultiMapFacts.cs @@ -30,6 +30,7 @@ namespace FoundationDB.Layers.Collections.Tests { using System; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; diff --git a/FoundationDB.Tests/Layers/VectorFacts.cs b/FoundationDB.Tests/Layers/VectorFacts.cs index 5bdf5e6d0..87f288f24 100644 --- a/FoundationDB.Tests/Layers/VectorFacts.cs +++ b/FoundationDB.Tests/Layers/VectorFacts.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Layers.Collections.Tests using System; using System.Text; using System.Threading.Tasks; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index 5e22282cc..4f52222f8 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -43,9 +43,9 @@ public class FdbQueryExpressionFacts : FdbTest { - private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.Create(TuPack.EncodeKey("Foos", 1))); + private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.FromKey(STuple.Create("Foos", 1))); - private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.Create(TuPack.EncodeKey("Foos", 2))); + private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.FromKey(STuple.Create("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 458f339ce..7a3db250b 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -30,6 +30,7 @@ namespace FoundationDB.Layers.Tuples.Tests { using System; using Doxense.Collections.Tuples; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Client.Tests; using NUnit.Framework; @@ -41,12 +42,9 @@ public class SubspaceFacts : FdbTest [Test] public void Test_Empty_Subspace_Is_Empty() { - var subspace = KeySubspace.Empty; + var subspace = KeySubspace.FromKey(Slice.Empty); Assert.That(subspace, Is.Not.Null, "FdbSubspace.Empty should not return null"); - Assert.That(KeySubspace.Empty, Is.SameAs(subspace), "FdbSubspace.Empty is a singleton"); - Assert.That(subspace.GetPrefix(), Is.EqualTo(Slice.Empty), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); - Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); } @@ -54,15 +52,17 @@ public void Test_Empty_Subspace_Is_Empty() [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = KeySubspace.CreateDynamic(new byte[] { 42, 255, 0, 127 }.AsSlice()); + var subspace = KeySubspace + .FromKey(new byte[] { 42, 255, 0, 127 }.AsSlice()) + .Using(TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the binary prefix directly - Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("*<00><7F><04><03><02><01>")); - Assert.That(subspace.ConcatKey(Slice.FromStringAscii("hello")).ToString(), Is.EqualTo("*<00><7F>hello")); + Assert.That(subspace[Slice.FromInt32(0x01020304)].ToString(), Is.EqualTo("*<00><7F><04><03><02><01>")); + Assert.That(subspace[Slice.FromStringAscii("hello")].ToString(), Is.EqualTo("*<00><7F>hello")); // pack(...) should use tuple serialization Assert.That(subspace.Keys.Encode(123).ToString(), Is.EqualTo("*<00><7F><15>{")); @@ -70,14 +70,8 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(subspace.Keys.Encode(Slice.FromStringAscii("world")).ToString(), Is.EqualTo("*<00><7F><01>world<00>")); Assert.That(subspace.Keys.Pack(STuple.Create("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); - // if we derive a tuple from this subspace, it should keep the binary prefix when converted to a key - var t = subspace.Keys.Append("world", 123, false); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(3)); - Assert.That(t.Get(0), Is.EqualTo("world")); - Assert.That(t.Get(1), Is.EqualTo(123)); - Assert.That(t.Get(2), Is.False); - var k = TuPack.Pack(t); + // if we encode a tuple from this subspace, it should keep the binary prefix when converted to a key + var k = subspace.Keys.Pack(STuple.Create("world", 123, false)); Assert.That(k.ToString(), Is.EqualTo("*<00><7F><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -92,7 +86,7 @@ public void Test_Subspace_With_Binary_Prefix() [Test] public void Test_Subspace_Copy_Does_Not_Share_Key_Buffer() { - var original = KeySubspace.Create(Slice.FromString("Hello")); + var original = KeySubspace.FromKey(Slice.FromString("Hello")); var copy = KeySubspace.Copy(original); Assert.That(copy, Is.Not.Null); Assert.That(copy, Is.Not.SameAs(original), "Copy should be a new instance"); @@ -108,7 +102,7 @@ public void Test_Subspace_Copy_Does_Not_Share_Key_Buffer() public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() { Assert.That(() => new KeySubspace(Slice.Nil), Throws.ArgumentException); - Assert.That(() => KeySubspace.Create(Slice.Nil), Throws.ArgumentException); + Assert.That(() => KeySubspace.FromKey(Slice.Nil), Throws.ArgumentException); //FIXME: typed subspaces refactoring ! //Assert.That(() => FdbSubspace.Empty.Partition[Slice.Nil], Throws.ArgumentException); //Assert.That(() => FdbSubspace.Create(FdbKey.Directory).Partition[Slice.Nil], Throws.ArgumentException); @@ -118,29 +112,24 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = KeySubspace.CreateDynamic(STuple.Create("hello")); + var subspace = KeySubspace + .FromKey(STuple.Create("hello")) + .Using(TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the tuple prefix directly - Assert.That(subspace.ConcatKey(Slice.FromInt32(0x01020304)).ToString(), Is.EqualTo("<02>hello<00><04><03><02><01>")); - Assert.That(subspace.ConcatKey(Slice.FromStringAscii("world")).ToString(), Is.EqualTo("<02>hello<00>world")); + Assert.That(subspace[Slice.FromInt32(0x01020304)].ToString(), Is.EqualTo("<02>hello<00><04><03><02><01>")); + Assert.That(subspace[Slice.FromStringAscii("world")].ToString(), Is.EqualTo("<02>hello<00>world")); // pack(...) should use tuple serialization Assert.That(subspace.Keys.Encode(123).ToString(), Is.EqualTo("<02>hello<00><15>{")); Assert.That(subspace.Keys.Encode("world").ToString(), Is.EqualTo("<02>hello<00><02>world<00>")); // even though the subspace prefix is a tuple, appending to it will only return the new items - var t = subspace.Keys.Append("world", 123, false); - Assert.That(t, Is.Not.Null); - Assert.That(t.Count, Is.EqualTo(3)); - Assert.That(t.Get(0), Is.EqualTo("world")); - Assert.That(t.Get(1), Is.EqualTo(123)); - Assert.That(t.Get(2), Is.False); - // but ToSlice() should include the prefix - var k = TuPack.Pack(t); + var k = subspace.Keys.Pack(STuple.Create("world", 123, false)); Assert.That(k.ToString(), Is.EqualTo("<02>hello<00><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -157,7 +146,7 @@ public void Test_Subspace_With_Tuple_Prefix() public void Test_Subspace_Partitioning_With_Binary_Suffix() { // start from a parent subspace - var parent = KeySubspace.Empty.Using(TypeSystem.Tuples); + var parent = KeySubspace.FromKey(Slice.Empty).Using(TypeSystem.Tuples); Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple @@ -166,7 +155,7 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() Assert.That(child.GetPrefix().ToString(), Is.EqualTo("")); // create a key from this child subspace - var key = child.ConcatKey(Slice.FromFixed32(0x01020304)); + var key = child[Slice.FromFixed32(0x01020304)]; Assert.That(key.ToString(), Is.EqualTo("<04><03><02><01>")); // create another child @@ -174,44 +163,13 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() Assert.That(grandChild, Is.Not.Null); Assert.That(grandChild.GetPrefix().ToString(), Is.EqualTo("hello")); - key = grandChild.ConcatKey(Slice.FromFixed32(0x01020304)); + key = grandChild[Slice.FromFixed32(0x01020304)]; Assert.That(key.ToString(), Is.EqualTo("hello<04><03><02><01>")); // cornercase Assert.That(child.Partition[Slice.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); } - [Test] - [Category("LocalCluster")] - public void Test_Subspace_Partitioning_With_Tuple_Suffix() - { - // start from a parent subspace - var parent = KeySubspace.CreateDynamic(Slice.FromByte(254), TypeSystem.Tuples); - Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); - - // create a child subspace using a tuple - var child = parent.Partition.ByKey("hca"); - Assert.That(child, Is.Not.Null); - Assert.That(child.GetPrefix().ToString(), Is.EqualTo("<02>hca<00>")); - - // create a tuple from this child subspace - var tuple = child.Keys.Append(123); - Assert.That(tuple, Is.Not.Null); - Assert.That(TuPack.Pack(tuple).ToString(), Is.EqualTo("<02>hca<00><15>{")); - - // derive another tuple from this one - var t1 = tuple.Append(false); - Assert.That(TuPack.Pack(t1).ToString(), Is.EqualTo("<02>hca<00><15>{<14>")); - - // check that we could also create the same tuple starting from the parent subspace - var t2 = parent.Keys.Append("hca", 123, false); - Assert.That(TuPack.Pack(t2), Is.EqualTo(TuPack.Pack(t1))); - - // cornercase - Assert.That(child.Partition[STuple.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); - - } - } } diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 6cd69f041..efa889908 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -1953,10 +1953,10 @@ public async Task Test_Simple_Read_Transaction() { await tr.GetReadVersionAsync(); - var a = location.ConcatKey(Slice.FromString("A")); - var b = location.ConcatKey(Slice.FromString("B")); - var c = location.ConcatKey(Slice.FromString("C")); - var z = location.ConcatKey(Slice.FromString("Z")); + var a = location[Slice.FromString("A")]; + var b = location[Slice.FromString("B")]; + var c = location[Slice.FromString("C")]; + var z = location[Slice.FromString("Z")]; //await tr.GetAsync(location.Concat(Slice.FromString("KEY"))); From 4d816ce82c5235798a5ebff0c2bb2dfcff2d9727 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 17:48:14 +0200 Subject: [PATCH 082/153] Use the proper cancellation token for paging and result iterators --- FoundationDB.Client/FdbRangeQuery.PagingIterator.cs | 1 + FoundationDB.Client/FdbRangeQuery.ResultIterator.cs | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 18ea35048..4f807672e 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -165,6 +165,7 @@ private Task FetchNextPageAsync() Contract.Requires(!this.AtEnd); Contract.Requires(this.Iteration >= 0); + m_ct.ThrowIfCancellationRequested(); this.Transaction.EnsureCanRead(); this.Iteration++; diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index 1f4e5c6a0..fd779a7a8 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -93,7 +93,7 @@ protected override Task OnFirstAsync() // on first call, setup the page iterator if (m_chunkIterator == null) { - m_chunkIterator = new PagingIterator(m_query, m_transaction).GetEnumerator(m_transaction.Cancellation, m_mode); + m_chunkIterator = new PagingIterator(m_query, m_transaction).GetEnumerator(m_ct, m_mode); } return TaskHelpers.True; } From e03215af85e07b99bcbd4fa54cf3f97f396de621 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sun, 22 Apr 2018 17:49:12 +0200 Subject: [PATCH 083/153] Removed IFdbKey interface --- FoundationDB.Client/IFdbKey.cs | 42 ---------------------------------- 1 file changed, 42 deletions(-) delete mode 100644 FoundationDB.Client/IFdbKey.cs diff --git a/FoundationDB.Client/IFdbKey.cs b/FoundationDB.Client/IFdbKey.cs deleted file mode 100644 index da2bec25d..000000000 --- a/FoundationDB.Client/IFdbKey.cs +++ /dev/null @@ -1,42 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Client -{ - using System; - - /// Allows a layer or custom class to be used as keys in a FoundationDB database - [Obsolete("Remove Me")] - public interface IFdbKey - { - /// Returns a binary representation of this instance, to be used as a complete key in the database - [Obsolete("Remove Me", error: true)] - Slice ToFoundationDbKey(); - } - -} From b06c24f06fe2a606a4709d449d92d545550b2702 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 18:38:34 +0200 Subject: [PATCH 084/153] Fixed equality comparison of tuples containing tuples --- FoundationDB.Client/Converters/ComparisonHelper.cs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs index 5547f07e6..17d304697 100644 --- a/FoundationDB.Client/Converters/ComparisonHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -234,6 +234,11 @@ private static Func CreateTypeComparator([NotNull] Type t1 } } + if (typeof(ITuple).IsAssignableFrom(t1) && typeof(ITuple).IsAssignableFrom(t2)) + { + return (x, y) => x == null ? y == null : y != null && ((ITuple) x).Equals((ITuple) y); + } + //TODO: some other way to compare ? return (x, y) => false; } From 7a49e01dc16ce5c2451088e5ee809971a994570c Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 18:41:29 +0200 Subject: [PATCH 085/153] Fixed usings and copyrights --- .../Tuples/Encoding/ITupleSerializable.cs | 28 +++++++++++++++++-- .../Layers/Tuples/Encoding/PrefixedTuple.cs | 1 - .../Layers/Tuples/Encoding/TupleCodec`1.cs | 3 +- .../Layers/Tuples/Encoding/TupleEncoder.cs | 28 +++++++++++++++++-- .../Layers/Tuples/Encoding/TupleKeyEncoder.cs | 2 -- .../Tuples/Encoding/TupleKeyEncoding.cs | 2 -- .../Layers/Tuples/Encoding/TuplePacker.cs | 3 +- .../Layers/Tuples/Encoding/TupleParser.cs | 2 +- .../Layers/Tuples/Encoding/TupleReader.cs | 2 +- .../Layers/Tuples/Encoding/TupleTypes.cs | 2 -- FoundationDB.Client/Layers/Tuples/STuple.cs | 1 - FoundationDB.Client/Layers/Tuples/STuple`5.cs | 28 +++++++++++++++++-- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 28 +++++++++++++++++-- .../Layers/Tuples/TupleHelpers.cs | 1 - 14 files changed, 108 insertions(+), 23 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs index c20072756..919772fbd 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs @@ -1,5 +1,29 @@ -#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. -// See License.MD for license information +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace Doxense.Collections.Tuples.Encoding diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs index 2dee08675..d4dc5e088 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding { using System; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index 36eb6dd8b..c92f0373a 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -29,10 +29,9 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Collections.Tuples.Encoding { using System; - using FoundationDB.Client; - using JetBrains.Annotations; using Doxense.Memory; using Doxense.Serialization.Encoders; + using JetBrains.Annotations; /// Type codec that uses the Tuple Encoding format /// Type of the values encoded by this codec diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs index 649d1448a..21aebfa00 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs @@ -1,5 +1,29 @@ -#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. -// See License.MD for license information +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace Doxense.Collections.Tuples.Encoding diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs index f6d98a7e9..1b01cbe91 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs @@ -26,14 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding { using System; using Doxense.Collections.Tuples; using Doxense.Memory; using Doxense.Serialization.Encoders; - using FoundationDB; using FoundationDB.Client; /// Encoder for variable-length elements, that uses the Tuple Binary Encoding format diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs index 6893ea6ee..5f1aa645c 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs @@ -26,12 +26,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding { using System; using Doxense.Serialization.Encoders; - using FoundationDB.Client; /// Encoding that uses the Tuple Binary Encoding format public sealed class TupleKeyEncoding : IKeyEncoding diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs index 1fc6a469d..ffec57bfc 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs @@ -26,12 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Runtime.CompilerServices; using Doxense.Memory; - using FoundationDB; using JetBrains.Annotations; /// Helper class that can serialize values of type to the tuple binary format diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs index ca7a07fdf..503dd12da 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Runtime.CompilerServices; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs index d53c3f667..5b0d4d8de 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Collections.Tuples.Encoding +namespace Doxense.Collections.Tuples.Encoding { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs index 85cf0664f..06cb064c1 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using FoundationDB; - namespace Doxense.Collections.Tuples.Encoding { using System; diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index 47aae9396..a301fd7f9 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -39,7 +39,6 @@ namespace Doxense.Collections.Tuples using System.Text; using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Factory class for Tuples diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index 1d310ade8..1db3347cf 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -1,5 +1,29 @@ -#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. -// See License.MD for license information +#region BSD Licence +/* Copyright (c) 2013-2018, 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 //#define ENABLE_VALUETUPLES diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs index 751723789..a8e17064e 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -1,5 +1,29 @@ -#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. -// See License.MD for license information +#region BSD Licence +/* Copyright (c) 2013-2018, 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 //#define ENABLE_VALUETUPLES diff --git a/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs b/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs index 118e1d61f..7e0f65b98 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs @@ -184,7 +184,6 @@ public static bool DeepEquals([NotNull] ITuple x, [NotNull] ITuple y, [NotNull] while (xs.MoveNext()) { if (!ys.MoveNext()) return false; - if (!comparer.Equals(xs.Current, ys.Current)) return false; } From 313235d97d440efeed481c5d38f6e15d9d9ae596 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 18:43:41 +0200 Subject: [PATCH 086/153] Moved back KeyRange / KeySelector / FdbKey into the FoundationDB.Client namespace --- FdbBurner/Program.cs | 1 - FdbShell/Commands/BasicCommands.cs | 1 - FoundationDB.Client/Converters/ComparisonHelper.cs | 2 -- FoundationDB.Client/FdbKey.cs | 2 +- FoundationDB.Client/KeyRange.cs | 3 +-- FoundationDB.Client/KeyRangeComparer.cs | 2 +- FoundationDB.Client/KeySelector.cs | 3 +-- FoundationDB.Client/KeySelectorPair.cs | 3 +-- FoundationDB.Client/Layers/Tuples/TuPack.cs | 2 +- FoundationDB.Client/Linq/AsyncEnumerable.cs | 4 +--- FoundationDB.Client/Linq/EnumerableExtensions.cs | 2 -- .../TypeSystem/Encoders/DynamicKeyEncoderBase.cs | 1 - FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs | 2 +- FoundationDB.Client/Utils/Memory/SliceBuffer.cs | 1 - FoundationDB.Client/Utils/Memory/SliceListStream.cs | 1 - FoundationDB.Client/Utils/Memory/SlicePairComparer.cs | 1 - FoundationDB.Client/Utils/Memory/SliceReader.cs | 1 - FoundationDB.Client/Utils/Memory/SliceStream.cs | 1 - FoundationDB.Client/Utils/Memory/SliceWriter.cs | 1 - 19 files changed, 8 insertions(+), 26 deletions(-) diff --git a/FdbBurner/Program.cs b/FdbBurner/Program.cs index dc962817d..cc2a77af2 100644 --- a/FdbBurner/Program.cs +++ b/FdbBurner/Program.cs @@ -9,7 +9,6 @@ namespace FdbBurner using System.Text; using System.Threading; using System.Threading.Tasks; - using FoundationDB; using FoundationDB.Client; public class Program diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 3c6d54ffa..d09be5f24 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -12,7 +12,6 @@ namespace FdbShell using System.Threading; using System.Threading.Tasks; using Doxense.Collections.Tuples; - using FoundationDB; using FoundationDB.Client; using FoundationDB.Layers.Directories; diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs index 17d304697..279d304e3 100644 --- a/FoundationDB.Client/Converters/ComparisonHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Runtime.Converters { using System; @@ -34,7 +33,6 @@ namespace Doxense.Runtime.Converters using System.Collections.Generic; using System.Globalization; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Helper classe used to compare object of "compatible" types diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 9536c0f08..33807e5d6 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace FoundationDB.Client { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index 1efb20cd3..5918d057e 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -27,12 +27,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY #endregion -namespace FoundationDB +namespace FoundationDB.Client { using System; using System.Diagnostics; using Doxense.Diagnostics.Contracts; - using FoundationDB.Client; /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] diff --git a/FoundationDB.Client/KeyRangeComparer.cs b/FoundationDB.Client/KeyRangeComparer.cs index 27d091727..94aaabdb9 100644 --- a/FoundationDB.Client/KeyRangeComparer.cs +++ b/FoundationDB.Client/KeyRangeComparer.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace FoundationDB +namespace FoundationDB.Client { using System; using System.Collections.Generic; diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index b0248d46f..1f9dcea0d 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -26,8 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB +namespace FoundationDB.Client { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index 3891ee1f5..d72e32b96 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -26,8 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - -namespace FoundationDB +namespace FoundationDB.Client { using System; using System.Diagnostics; diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index 26fe00069..d0f6b687a 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -36,7 +36,7 @@ namespace Doxense.Collections.Tuples using Doxense.Diagnostics.Contracts; using Doxense.Collections.Tuples.Encoding; using Doxense.Memory; - using FoundationDB; + using FoundationDB.Client; using JetBrains.Annotations; /// Tuple Binary Encoding diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.cs b/FoundationDB.Client/Linq/AsyncEnumerable.cs index 738724e6c..937da6b1d 100644 --- a/FoundationDB.Client/Linq/AsyncEnumerable.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense.Async; -using FoundationDB; namespace Doxense.Linq { @@ -35,10 +33,10 @@ namespace Doxense.Linq using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq.Async.Expressions; using Doxense.Linq.Async.Iterators; - using FoundationDB.Client; using JetBrains.Annotations; /// Provides a set of static methods for querying objects that implement . diff --git a/FoundationDB.Client/Linq/EnumerableExtensions.cs b/FoundationDB.Client/Linq/EnumerableExtensions.cs index 63faab43b..14f12933c 100644 --- a/FoundationDB.Client/Linq/EnumerableExtensions.cs +++ b/FoundationDB.Client/Linq/EnumerableExtensions.cs @@ -32,8 +32,6 @@ namespace Doxense.Linq using System.Collections.Generic; using System.Linq; using Doxense.Diagnostics.Contracts; - using Doxense.Linq.Async.Iterators; - using FoundationDB; using JetBrains.Annotations; /// Provides a set of static methods for querying objects that implement . diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index 1471d4554..5b735f838 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; diff --git a/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs index d59acf4d1..4643a12c2 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs @@ -31,7 +31,7 @@ namespace Doxense.Serialization.Encoders using System; using Doxense.Collections.Tuples; using Doxense.Memory; - using FoundationDB; + using FoundationDB.Client; using JetBrains.Annotations; /// Encoder that can process keys of variable size and types diff --git a/FoundationDB.Client/Utils/Memory/SliceBuffer.cs b/FoundationDB.Client/Utils/Memory/SliceBuffer.cs index 1ff5961f0..c513f4ba5 100644 --- a/FoundationDB.Client/Utils/Memory/SliceBuffer.cs +++ b/FoundationDB.Client/Utils/Memory/SliceBuffer.cs @@ -32,7 +32,6 @@ namespace Doxense.Memory using System.Collections.Generic; using System.Diagnostics; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Buffer that can be used to efficiently store multiple slices into as few chunks as possible diff --git a/FoundationDB.Client/Utils/Memory/SliceListStream.cs b/FoundationDB.Client/Utils/Memory/SliceListStream.cs index d422ba0d3..3b84be488 100644 --- a/FoundationDB.Client/Utils/Memory/SliceListStream.cs +++ b/FoundationDB.Client/Utils/Memory/SliceListStream.cs @@ -35,7 +35,6 @@ namespace Doxense.Memory using System.Runtime.CompilerServices; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; //REVIEW: this is somewhat similar to the proposed new IBufferList (from System.IO.Pipeline) ? diff --git a/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs b/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs index 65c3d9f08..12e447bd3 100644 --- a/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs +++ b/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs @@ -31,7 +31,6 @@ namespace Doxense.Memory using System; using System.Collections.Generic; using Doxense.Diagnostics.Contracts; - using FoundationDB; /// Performs optimized equality and comparison checks on key/value pairs of public sealed class SlicePairComparer : IComparer>, IEqualityComparer> diff --git a/FoundationDB.Client/Utils/Memory/SliceReader.cs b/FoundationDB.Client/Utils/Memory/SliceReader.cs index c54c16fae..5dfc3ef9e 100644 --- a/FoundationDB.Client/Utils/Memory/SliceReader.cs +++ b/FoundationDB.Client/Utils/Memory/SliceReader.cs @@ -33,7 +33,6 @@ namespace Doxense.Memory using System.Runtime.CompilerServices; using System.Text; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Helper class that holds the internal state used to parse tuples from slices diff --git a/FoundationDB.Client/Utils/Memory/SliceStream.cs b/FoundationDB.Client/Utils/Memory/SliceStream.cs index 7fb2bfd4e..cdb0370ca 100644 --- a/FoundationDB.Client/Utils/Memory/SliceStream.cs +++ b/FoundationDB.Client/Utils/Memory/SliceStream.cs @@ -32,7 +32,6 @@ namespace Doxense.Memory using System.IO; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; /// Stream that wraps a Slice for reading diff --git a/FoundationDB.Client/Utils/Memory/SliceWriter.cs b/FoundationDB.Client/Utils/Memory/SliceWriter.cs index 0deaad781..70661724e 100644 --- a/FoundationDB.Client/Utils/Memory/SliceWriter.cs +++ b/FoundationDB.Client/Utils/Memory/SliceWriter.cs @@ -37,7 +37,6 @@ namespace Doxense.Memory using System.Runtime.CompilerServices; using System.Text; using Doxense.Diagnostics.Contracts; - using FoundationDB; using JetBrains.Annotations; #if ENABLE_SPAN using System.Runtime.InteropServices; From 777fec9c5d9cbeda0345967240f7f48c9c96c441 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 18:45:15 +0200 Subject: [PATCH 087/153] code cleanup --- .../Converters/ComparisonHelper.cs | 57 +++++++++---------- .../Converters/SimilarValueComparer.cs | 6 +- FoundationDB.Client/FdbRangeQuery.cs | 49 +++++++++------- 3 files changed, 57 insertions(+), 55 deletions(-) diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs index 279d304e3..ede62675e 100644 --- a/FoundationDB.Client/Converters/ComparisonHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -32,6 +32,7 @@ namespace Doxense.Runtime.Converters using System.Collections.Concurrent; using System.Collections.Generic; using System.Globalization; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; @@ -53,8 +54,7 @@ public TypePair(Type left, Type right) public override bool Equals(object obj) { - if (obj == null) return false; - return Equals((TypePair)obj); + return obj is TypePair tp && Equals(tp); } public bool Equals(TypePair other) @@ -65,11 +65,7 @@ public bool Equals(TypePair other) public override int GetHashCode() { - // note: we cannot just xor both hash codes, because if left and right are the same, we will return 0 - int h = this.Left.GetHashCode(); - h = (h >> 13) | (h << 19); - h ^= this.Right.GetHashCode(); - return h; + return HashCodes.Combine(this.Left.GetHashCode(), this.Right.GetHashCode()); } } @@ -99,20 +95,23 @@ public int GetHashCode(TypePair obj) /// Tries to convert an object into an equivalent string representation (for equality comparison) /// Object to adapt /// String equivalent of the object - internal static string TryAdaptToString(object value) + public static string TryAdaptToString(object value) { - if (value == null) return null; - - if (value is string s) return s; - - if (value is char c) return new string(c, 1); - - if (value is Slice slice) return slice.ToStringAscii(); //REVIEW: or ToUnicode() ? - - if (value is byte[] bstr) return bstr.AsSlice().ToStringAscii(); //REVIEW: or ToUnicode() ? - - if (value is IFormattable fmt) return fmt.ToString(null, CultureInfo.InvariantCulture); - + switch (value) + { + case null: + return null; + case string s: + return s; + case char c: + return new string(c, 1); + case Slice sl: + return sl.ToStringUtf8(); //BUGBUG: ASCII? Ansi? UTF8? + case byte[] bstr: + return bstr.AsSlice().ToStringUtf8(); //BUGBUG: ASCII? Ansi? UTF8? + case IFormattable fmt: + return fmt.ToString(null, CultureInfo.InvariantCulture); + } return null; } @@ -121,7 +120,7 @@ internal static string TryAdaptToString(object value) /// Type of the object to adapt /// Double equivalent of the object /// True if is compatible with a decimal. False if the type is not compatible - internal static bool TryAdaptToDecimal(object value, [NotNull] Type type, out double result) + public static bool TryAdaptToDecimal(object value, [NotNull] Type type, out double result) { if (value != null) { @@ -147,7 +146,7 @@ internal static bool TryAdaptToDecimal(object value, [NotNull] Type type, out do /// Type of the object to adapt /// Int64 equivalent of the object /// True if is compatible with a decimal. False if the type is not compatible - internal static bool TryAdaptToInteger(object value, [NotNull] Type type, out long result) + public static bool TryAdaptToInteger(object value, [NotNull] Type type, out long result) { if (value != null) { @@ -203,12 +202,7 @@ private static Func CreateTypeComparator([NotNull] Type t1 if (IsStringType(t1) || IsStringType(t2)) { - return (x, y) => - { - if (x == null) return y == null; - if (y == null) return false; - return object.ReferenceEquals(x, y) || (TryAdaptToString(x) == TryAdaptToString(y)); - }; + return (x, y) => x == null ? y == null : y != null && (object.ReferenceEquals(x, y) || (TryAdaptToString(x) == TryAdaptToString(y))); } if (IsNumericType(t1) || IsNumericType(t2)) @@ -218,6 +212,7 @@ private static Func CreateTypeComparator([NotNull] Type t1 return (x, y) => { double d1, d2; + // ReSharper disable once CompareOfFloatsByEqualityOperator return x == null ? y == null : y != null && TryAdaptToDecimal(x, t1, out d1) && TryAdaptToDecimal(y, t2, out d2) && d1 == d2; }; } @@ -241,7 +236,7 @@ private static Func CreateTypeComparator([NotNull] Type t1 return (x, y) => false; } - internal static Func GetTypeComparator(Type t1, Type t2) + public static Func GetTypeComparator(Type t1, Type t2) { var pair = new TypePair(t1, t2); Func comparator; @@ -264,7 +259,7 @@ internal static Func GetTypeComparator(Type t1, Type t2) /// AreSimilar(false, 0) => true /// AreSimilar(true, 1) => true /// - internal static bool AreSimilar(object x, object y) + public static bool AreSimilar(object x, object y) { if (object.ReferenceEquals(x, y)) return true; if (x == null || y == null) return false; @@ -274,7 +269,7 @@ internal static bool AreSimilar(object x, object y) return comparator(x, y); } - internal static bool AreSimilar(T1 x, T2 y) + public static bool AreSimilar(T1 x, T2 y) { var comparator = GetTypeComparator(typeof(T1), typeof(T2)); Contract.Requires(comparator != null); diff --git a/FoundationDB.Client/Converters/SimilarValueComparer.cs b/FoundationDB.Client/Converters/SimilarValueComparer.cs index 3657eb5d1..32ce5171e 100644 --- a/FoundationDB.Client/Converters/SimilarValueComparer.cs +++ b/FoundationDB.Client/Converters/SimilarValueComparer.cs @@ -31,11 +31,13 @@ namespace Doxense.Runtime.Converters using System; using System.Collections; using System.Collections.Generic; + using JetBrains.Annotations; /// Object comparer that returns true if both values are "similar" /// This comparer SHOULD NOT be used in a Dictioanry, because it violates on of the conditions: Two objects could be considered equal, but have different hashcode! - internal class SimilarValueComparer : IEqualityComparer, IEqualityComparer + internal sealed class SimilarValueComparer : IEqualityComparer, IEqualityComparer { + [NotNull] public static readonly IEqualityComparer Default = new SimilarValueComparer(); private SimilarValueComparer() @@ -48,7 +50,7 @@ bool IEqualityComparer.Equals(object x, object y) int IEqualityComparer.GetHashCode(object obj) { - return obj == null ? -1 : obj.GetHashCode(); + return obj?.GetHashCode() ?? -1; } bool IEqualityComparer.Equals(object x, object y) diff --git a/FoundationDB.Client/FdbRangeQuery.cs b/FoundationDB.Client/FdbRangeQuery.cs index 39cbaff8c..5ed847fa6 100644 --- a/FoundationDB.Client/FdbRangeQuery.cs +++ b/FoundationDB.Client/FdbRangeQuery.cs @@ -44,7 +44,7 @@ public sealed partial class FdbRangeQuery : IAsyncEnumerable { /// Construct a query with a set of initial settings - internal FdbRangeQuery([NotNull] IFdbReadOnlyTransaction transaction, KeySelector begin, KeySelector end, [NotNull] Func, T> transform, bool snapshot, FdbRangeOptions options) + internal FdbRangeQuery([NotNull] IFdbReadOnlyTransaction transaction, KeySelector begin, KeySelector end, [NotNull] Func, T> transform, bool snapshot, [CanBeNull] FdbRangeOptions options) { Contract.Requires(transaction != null && transform != null); @@ -80,7 +80,7 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption public KeySelector End { get; private set; } /// Key selector pair describing the beginning and end of the range that will be queried - public KeySelectorPair Range { get { return new KeySelectorPair(this.Begin, this.End); } } + public KeySelectorPair Range => new KeySelectorPair(this.Begin, this.End); /// Stores all the settings for this range query internal FdbRangeOptions Options { get; } @@ -90,26 +90,28 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption internal KeySelectorPair OriginalRange { get; } /// Limit in number of rows to return - public int? Limit { get { return this.Options.Limit; } } + public int? Limit => this.Options.Limit; /// Limit in number of bytes to return - public int? TargetBytes { get { return this.Options.TargetBytes; } } + public int? TargetBytes => this.Options.TargetBytes; /// Streaming mode //REVIEW: nullable? - public FdbStreamingMode Mode { get { return this.Options.Mode ?? FdbStreamingMode.Iterator; } } + public FdbStreamingMode Mode => this.Options.Mode ?? FdbStreamingMode.Iterator; /// Should we perform the range using snapshot mode ? public bool Snapshot { get; } /// Should the results be returned in reverse order (from last key to first key) - public bool Reversed { get { return this.Options.Reverse ?? false; } } + public bool Reversed => this.Options.Reverse ?? false; /// Parent transaction used to perform the GetRange operation - internal IFdbReadOnlyTransaction Transaction { [NotNull] get; } + [NotNull] + internal IFdbReadOnlyTransaction Transaction { get; } /// Transformation applied to the result - internal Func, T> Transform { [NotNull] get; } + [NotNull] + internal Func, T> Transform { get; } #endregion @@ -118,7 +120,7 @@ private FdbRangeQuery([NotNull] FdbRangeQuery query, [NotNull] FdbRangeOption /// Only return up to a specific number of results /// Maximum number of results to return /// A new query object that will only return up to results when executed - [NotNull] + [Pure, NotNull] public FdbRangeQuery Take([Positive] int count) { Contract.Positive(count, nameof(count)); @@ -137,7 +139,7 @@ public FdbRangeQuery Take([Positive] int count) /// Bypasses a specified number of elements in a sequence and then returns the remaining elements. /// /// A new query object that will skip the first results when executed - [NotNull] + [Pure, NotNull] public FdbRangeQuery Skip([Positive] int count) { Contract.Positive(count, nameof(count)); @@ -184,7 +186,7 @@ public FdbRangeQuery Skip([Positive] int count) /// A new query object that will return the results in reverse order when executed /// Calling Reverse() on an already reversed query will cancel the effect, and the results will be returned in their natural order. /// Note: Combining the effects of Take()/Skip() and Reverse() may have an impact on performance, especially if the ReadYourWriteDisabled transaction is options set. - [NotNull] + [Pure, NotNull] public FdbRangeQuery Reverse() { var begin = this.Begin; @@ -216,7 +218,7 @@ public FdbRangeQuery Reverse() /// Use a specific target bytes size /// /// A new query object that will use the specified target bytes size when executed - [NotNull] + [Pure, NotNull] public FdbRangeQuery WithTargetBytes([Positive] int bytes) { Contract.Positive(bytes, nameof(bytes)); @@ -230,7 +232,7 @@ public FdbRangeQuery WithTargetBytes([Positive] int bytes) /// Use a different Streaming Mode /// Streaming mode to use when reading the results from the database /// A new query object that will use the specified streaming mode when executed - [NotNull] + [Pure, NotNull] public FdbRangeQuery WithMode(FdbStreamingMode mode) { if (!Enum.IsDefined(typeof(FdbStreamingMode), mode)) @@ -247,7 +249,7 @@ public FdbRangeQuery WithMode(FdbStreamingMode mode) /// Force the query to use a specific transaction /// Transaction to use when executing this query /// A new query object that will use the specified transaction when executed - [NotNull] + [Pure, NotNull] public FdbRangeQuery UseTransaction([NotNull] IFdbReadOnlyTransaction transaction) { Contract.NotNull(transaction, nameof(transaction)); @@ -300,7 +302,7 @@ public Task CountAsync() return AsyncEnumerable.CountAsync(this, this.Transaction.Cancellation); } - [NotNull] + [Pure, NotNull] internal FdbRangeQuery Map([NotNull] Func, TResult> transform) { Contract.Requires(transform != null); @@ -315,7 +317,7 @@ internal FdbRangeQuery Map([NotNull] FuncProjects each element of the range results into a new form. - [NotNull] + [Pure, NotNull] public FdbRangeQuery Select([NotNull] Func lambda) { Contract.Requires(lambda != null); @@ -327,12 +329,13 @@ public FdbRangeQuery Select([NotNull] Func lambda) /// Filters the range results based on a predicate. /// Caution: filtering occurs on the client side ! - [NotNull] + [Pure, NotNull] public IAsyncEnumerable Where([NotNull] Func predicate) { return AsyncEnumerable.Where(this, predicate); } + [ItemCanBeNull] public Task FirstOrDefaultAsync() { // we can optimize this by passing Limit=1 @@ -345,6 +348,7 @@ public Task FirstAsync() return HeadAsync(single: false, orDefault: false); } + [ItemCanBeNull] public Task LastOrDefaultAsync() { //BUGBUG: if there is a Take(N) on the query, Last() will mean "The Nth key" and not the "last key in the original range". @@ -361,6 +365,7 @@ public Task LastAsync() return this.Reverse().HeadAsync(single: false, orDefault:false); } + [ItemCanBeNull] public Task SingleOrDefaultAsync() { // we can optimize this by passing Limit=2 @@ -469,7 +474,7 @@ internal async Task AnyOrNoneAsync(bool any) /// Returns a printable version of the range query public override string ToString() { - return String.Format(CultureInfo.InvariantCulture, "Range({0}, {1}, {2})", this.Range, this.Limit, this.Reversed ? "reverse" : "forward"); + return $"Range({this.Range}, {this.Limit}, {(this.Reversed ? "reverse" : "forward")})"; } } @@ -478,7 +483,7 @@ public override string ToString() public static class FdbRangeQueryExtensions { - [NotNull] + [Pure, NotNull] public static FdbRangeQuery Keys([NotNull] this FdbRangeQuery> query) { Contract.NotNull(query, nameof(query)); @@ -490,7 +495,7 @@ public static FdbRangeQuery Keys([NotNull] this FdbRangeQuer return query.Map((x) => f(x).Key); } - [NotNull] + [Pure, NotNull] public static FdbRangeQuery Keys([NotNull] this FdbRangeQuery> query, [NotNull] Func transform) { Contract.NotNull(query, nameof(query)); @@ -503,7 +508,7 @@ public static FdbRangeQuery Keys([NotNull] this return query.Map((x) => transform(f(x).Key)); } - [NotNull] + [Pure, NotNull] public static FdbRangeQuery Values([NotNull] this FdbRangeQuery> query) { Contract.NotNull(query, nameof(query)); @@ -515,7 +520,7 @@ public static FdbRangeQuery Values([NotNull] this FdbRange return query.Map((x) => f(x).Value); } - [NotNull] + [Pure, NotNull] public static FdbRangeQuery Values([NotNull] this FdbRangeQuery> query, [NotNull] Func transform) { Contract.NotNull(query, nameof(query)); From cd817d271e78453982bb0302c68256c045d73d93 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 18:48:38 +0200 Subject: [PATCH 088/153] Don't display the column guids in DEBUG + code cleanup --- FdbTop/Program.cs | 78 +++++++++++++++++++++++++++++++------------- FdbTop/RingBuffer.cs | 53 +++++++++++++++++++----------- 2 files changed, 89 insertions(+), 42 deletions(-) diff --git a/FdbTop/Program.cs b/FdbTop/Program.cs index e4d92b743..e25d437df 100644 --- a/FdbTop/Program.cs +++ b/FdbTop/Program.cs @@ -1,20 +1,47 @@ -using FoundationDB.Client; -using FoundationDB.Client.Status; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +// ReSharper disable CompareOfFloatsByEqualityOperator namespace FdbTop { + using System; + using System.Diagnostics; + using System.Globalization; + using System.Linq; + using System.Text; + using System.Threading; + using System.Threading.Tasks; + using FoundationDB.Client; + using FoundationDB.Client.Status; + public static class Program { - - static string ClusterPath = null; + private static string ClusterPath; public static void Main(string[] args) { @@ -260,7 +287,7 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) updated = false; } - await Task.Delay(100); + await Task.Delay(100, cancel); } } @@ -276,7 +303,12 @@ private static async Task MainAsync(string[] args, CancellationToken cancel) private static readonly DateTime Epoch = new DateTime(1970, 1, 1, 0, 0, 0, DateTimeKind.Utc); private const int HistoryCapacity = 50; - private static RingBuffer History = new RingBuffer(HistoryCapacity); + private static readonly RingBuffer History = new RingBuffer(HistoryCapacity); + + static Program() + { + Program.ClusterPath = null; + } private const int MAX_RW_WIDTH = 40; private const int MAX_WS_WIDTH = 20; @@ -690,13 +722,13 @@ private static void ShowTransactionScreen(FdbSystemStatus status, HistoryMetric private struct RoleMap { - public bool Master; - public bool ClusterController; - public bool Proxy; - public bool Log; - public bool Storage; - public bool Resolver; - public bool Other; + private bool Master; + private bool ClusterController; + private bool Proxy; + private bool Log; + private bool Storage; + private bool Resolver; + private bool Other; public void Add(string role) { @@ -751,7 +783,7 @@ private static void ShowProcessesScreen(FdbSystemStatus status, HistoryMetric cu WriteAt(COL6, 5, "HDD (%busy)"); WriteAt(COL7, 5, "Roles"); -#if DEBUG +#if DEBUG_LAYOUT Console.ForegroundColor = ConsoleColor.DarkGray; WriteAt(COL0, 6, "0 - - - - - -"); WriteAt(COL1, 6, "1 - - - - - -"); diff --git a/FdbTop/RingBuffer.cs b/FdbTop/RingBuffer.cs index ee9770708..1133c0ba3 100644 --- a/FdbTop/RingBuffer.cs +++ b/FdbTop/RingBuffer.cs @@ -1,15 +1,36 @@ -using FoundationDB.Client; -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace FdbTop { + using System; + using System.Collections.Generic; + public class RingBuffer : IReadOnlyCollection { private readonly Queue m_store; @@ -17,21 +38,15 @@ public class RingBuffer : IReadOnlyCollection public RingBuffer(int capacity) { - if (capacity < 0) throw new ArgumentOutOfRangeException("capacity"); + if (capacity < 0) throw new ArgumentOutOfRangeException(nameof(capacity)); m_store = new Queue(capacity); m_size = capacity; } - public int Count - { - get { return m_store.Count; } - } + public int Count => m_store.Count; - public int Capacity - { - get { return m_size; } - } + public int Capacity => m_size; public void Clear() { @@ -40,7 +55,7 @@ public void Clear() public void Resize(int newCapacity) { - if (newCapacity < 0) throw new ArgumentOutOfRangeException("newCapacity"); + if (newCapacity < 0) throw new ArgumentOutOfRangeException(nameof(newCapacity)); var store = m_store; if (newCapacity < store.Count) From ff272165a169db811f68c813f604cec92adf540d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 19:11:19 +0200 Subject: [PATCH 089/153] Cleanup, and nest KeyRangeComparer instead of KeyRange --- FoundationDB.Client/KeyRange.cs | 165 ++++++++++++++++-------- FoundationDB.Client/KeyRangeComparer.cs | 86 ------------ FoundationDB.Client/KeySelector.cs | 84 ++++++------ FoundationDB.Client/KeySelectorPair.cs | 14 +- 4 files changed, 157 insertions(+), 192 deletions(-) delete mode 100644 FoundationDB.Client/KeyRangeComparer.cs diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index 5918d057e..3efa32e27 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -26,57 +26,59 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; + using System.Collections.Generic; using System.Diagnostics; + using System.Runtime.CompilerServices; + using Doxense; using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] - public struct KeyRange : IEquatable, IComparable + public /*readonly*/ struct KeyRange : IEquatable, IComparable { - /// Returns an empty pair of keys - public static KeyRange Empty => default(KeyRange); - - /// Returns a range that contains all the keys in the database - public static KeyRange All => new KeyRange(FdbKey.MinValue, FdbKey.MaxValue); /// Start of the range - public Slice Begin { get { return m_begin; } } - private Slice m_begin; //PERF: readonly struct + public /*readonly*/ Slice Begin; /// End of the range - public Slice End { get { return m_end; } } - private Slice m_end; //PERF: readonly struct + public /*readonly*/ Slice End; - /// - /// Create a new range of keys - /// + /// Create a new range of keys /// Start of range (usually included) /// End of range (usually excluded) public KeyRange(Slice begin, Slice end) { - m_begin = begin; - m_end = end; + this.Begin = begin; + this.End = end; - Contract.Ensures(m_begin <= m_end, "The range is inverted"); + Contract.Ensures(this.Begin <= this.End, "The range is inverted"); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static KeyRange Create(Slice a, Slice b) { return new KeyRange(a, b); } + /// Returns an empty pair of keys + public static readonly KeyRange Empty = default(KeyRange); + + /// Returns a range that contains all the keys in the database + public static KeyRange All => new KeyRange(FdbKey.MinValue, FdbKey.MaxValue); + /// Create a range that will return all keys starting with : ('prefix' <= k < strinc('prefix')) - /// - /// + [Pure] public static KeyRange StartsWith(Slice prefix) { - if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull("prefix"); - if (prefix.Count == 0) return new KeyRange(Slice.Empty, FdbKey.MaxValue); - + if (prefix.Count == 0) + { + if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull(nameof(prefix)); + return new KeyRange(Slice.Empty, FdbKey.MaxValue); + } // prefix => [ prefix, prefix + 1 ) return new KeyRange( @@ -88,9 +90,10 @@ public static KeyRange StartsWith(Slice prefix) /// Create a range that selects all keys starting with , but not the prefix itself: ('prefix\x00' <= k < string('prefix') /// Key prefix (that will be excluded from the range) /// Range including all keys with the specified prefix. + [Pure] public static KeyRange PrefixedBy(Slice prefix) { - if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull("prefix"); + if (prefix.IsNull) throw Fdb.Errors.KeyCannotBeNull(nameof(prefix)); // prefix => [ prefix."\0", prefix + 1) return new KeyRange( @@ -102,12 +105,12 @@ public static KeyRange PrefixedBy(Slice prefix) /// Create a range that will only return itself ('key' <= k < 'key\x00') /// Key that will be returned by the range /// Range that only return the specified key. + [Pure] public static KeyRange FromKey(Slice key) { - if (key.IsNull) throw Fdb.Errors.KeyCannotBeNull(); - if (key.Count == 0) { // "" => [ "", "\x00" ) + if (key.IsNull) throw Fdb.Errors.KeyCannotBeNull(); return new KeyRange(Slice.Empty, FdbKey.MinValue); } // key => [ key, key + '\0' ) @@ -119,37 +122,35 @@ public static KeyRange FromKey(Slice key) public override bool Equals(object obj) { - return (obj is KeyRange) && Equals((KeyRange)obj); + return obj is KeyRange range && Equals(range); } public override int GetHashCode() { - // ReSharper disable once NonReadonlyMemberInGetHashCode - int h1 = m_begin.GetHashCode(); - // ReSharper disable once NonReadonlyMemberInGetHashCode - int h2 = m_end.GetHashCode(); - return ((h1 << 5) + h1) ^ h2; + // ReSharper disable NonReadonlyMemberInGetHashCode + return HashCodes.Combine(this.Begin.GetHashCode(), this.End.GetHashCode()); + // ReSharper restore NonReadonlyMemberInGetHashCode } public bool Equals(KeyRange other) { - return m_begin.Equals(other.m_begin) && m_end.Equals(other.m_end); + return this.Begin.Equals(other.Begin) && this.End.Equals(other.End); } public static bool operator ==(KeyRange left, KeyRange right) { - return left.m_begin.Equals(right.m_begin) && left.m_end.Equals(right.m_end); + return left.Begin.Equals(right.Begin) && left.End.Equals(right.End); } public static bool operator !=(KeyRange left, KeyRange right) { - return !left.m_begin.Equals(right.m_begin) || !left.m_end.Equals(right.m_end); + return !left.Begin.Equals(right.Begin) || !left.End.Equals(right.End); } public int CompareTo(KeyRange other) { - int c = m_begin.CompareTo(other.m_begin); - if (c == 0) c = m_end.CompareTo(other.m_end); + int c = this.Begin.CompareTo(other.Begin); + if (c == 0) c = this.End.CompareTo(other.End); return c; } @@ -157,10 +158,11 @@ public int CompareTo(KeyRange other) /// Range to merge with the current range /// New range where the Begin key is the smallest bound and the End key is the largest bound of both ranges. /// If both range are disjoint, then the resulting range will also contain the keys in between. + [Pure] public KeyRange Merge(KeyRange other) { - Slice begin = m_begin.CompareTo(other.m_begin) <= 0 ? m_begin : other.m_begin; - Slice end = m_end.CompareTo(other.m_end) >= 0 ? m_end : other.m_end; + Slice begin = this.Begin.CompareTo(other.Begin) <= 0 ? this.Begin : other.Begin; + Slice end = this.End.CompareTo(other.End) >= 0 ? this.End : other.End; return new KeyRange(begin, end); } @@ -168,71 +170,122 @@ public KeyRange Merge(KeyRange other) /// Range that is being checked for interection /// True if the other range intersects the current range. /// Note that ranges [0, 1) and [1, 2) do not intersect, since the end is exclusive by default + [Pure] public bool Intersects(KeyRange other) { - int c = m_begin.CompareTo(other.m_begin); + int c = this.Begin.CompareTo(other.Begin); if (c == 0) { // share the same begin key return true; } - else if (c < 0) + if (c < 0) { // after us - return m_end.CompareTo(other.m_begin) > 0; - } - else - { // before us - return m_begin.CompareTo(other.m_end) < 0; + return this.End.CompareTo(other.Begin) > 0; } + // before us + return this.Begin.CompareTo(other.End) < 0; } /// Checks whether the current and the specified range are disjoint (i.e: there exists at least one key between both ranges) /// /// /// Note that ranges [0, 1) and [1, 2) are not disjoint because, even though they do not intersect, they are both contiguous. + [Pure] public bool Disjoint(KeyRange other) { - int c = m_begin.CompareTo(other.m_begin); + int c = this.Begin.CompareTo(other.Begin); if (c == 0) { // share the same begin key return false; } - else if (c < 0) + if (c < 0) { // after us - return m_end.CompareTo(other.m_begin) < 0; - } - else - { // before us - return m_begin.CompareTo(other.m_end) > 0; + return this.End.CompareTo(other.Begin) < 0; } + // before us + return this.Begin.CompareTo(other.End) > 0; } /// Returns true, if the key is contained in the range /// /// + [Pure] public bool Contains(Slice key) { - return key.CompareTo(m_begin) >= 0 && key.CompareTo(m_end) < 0; + return key.CompareTo(this.Begin) >= 0 && key.CompareTo(this.End) < 0; } /// Test if is contained inside the range /// Key that will be compared with the the range's bounds /// If true, the End bound is inclusive, otherwise it is exclusive /// -1 if key is less than the lower bound of the range ( < Begin), +1 if the key is greater or equal to the higher bound of the range ( >= End) or 0 if it is inside the range (Begin <= < End) + [Pure] public int Test(Slice key, bool endIncluded = false) { // note: if the range is empty (Begin = End = Slice.Empty) then it should return 0 - if (m_begin.IsPresent && key.CompareTo(m_begin) < 0) return -1; - if (m_end.IsPresent && key.CompareTo(m_end) >= (endIncluded ? 1 : 0)) return +1; + if (this.Begin.IsPresent && key.CompareTo(this.Begin) < 0) return -1; + if (this.End.IsPresent && key.CompareTo(this.End) >= (endIncluded ? 1 : 0)) return +1; return 0; } /// Returns a printable version of the range public override string ToString() { - return "{" + FdbKey.PrettyPrint(m_begin, FdbKey.PrettyPrintMode.Begin) + ", " + FdbKey.PrettyPrint(m_end, FdbKey.PrettyPrintMode.End) + "}"; + return "{" + FdbKey.PrettyPrint(this.Begin, FdbKey.PrettyPrintMode.Begin) + ", " + FdbKey.PrettyPrint(this.End, FdbKey.PrettyPrintMode.End) + "}"; } + [DebuggerDisplay("Mode={m_mode}")] + public sealed class Comparer : IComparer, IEqualityComparer + { + private const int BOTH = 0; + private const int BEGIN = 1; + private const int END = 2; + + public static readonly Comparer Default = new Comparer(BOTH); + public static readonly Comparer Begin = new Comparer(BEGIN); + public static readonly Comparer End = new Comparer(END); + + private readonly int m_mode; + + private Comparer(int mode) + { + Contract.Requires(mode >= BOTH && mode <= END); + m_mode = mode; + } + + public int Compare(KeyRange x, KeyRange y) + { + switch (m_mode) + { + case BEGIN: return x.Begin.CompareTo(y.Begin); + case END: return x.End.CompareTo(y.End); + default: return x.CompareTo(y); + } + } + + public bool Equals(KeyRange x, KeyRange y) + { + switch(m_mode) + { + case BEGIN: return x.Begin.Equals(y.Begin); + case END: return x.End.Equals(y.End); + default: return x.Equals(y); + } + } + + public int GetHashCode(KeyRange obj) + { + switch(m_mode) + { + case BEGIN: return obj.Begin.GetHashCode(); + case END: return obj.End.GetHashCode(); + default: return obj.GetHashCode(); + } + } + } + + } } diff --git a/FoundationDB.Client/KeyRangeComparer.cs b/FoundationDB.Client/KeyRangeComparer.cs deleted file mode 100644 index 94aaabdb9..000000000 --- a/FoundationDB.Client/KeyRangeComparer.cs +++ /dev/null @@ -1,86 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013, Doxense SARL -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 - -namespace FoundationDB.Client -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using Doxense.Diagnostics.Contracts; - - [DebuggerDisplay("Mode={m_mode}")] - public sealed class KeyRangeComparer : IComparer, IEqualityComparer - { - private const int BOTH = 0; - private const int BEGIN = 1; - private const int END = 2; - - public static readonly KeyRangeComparer Default = new KeyRangeComparer(BOTH); - public static readonly KeyRangeComparer Begin = new KeyRangeComparer(BEGIN); - public static readonly KeyRangeComparer End = new KeyRangeComparer(END); - - private readonly int m_mode; - - private KeyRangeComparer(int mode) - { - Contract.Requires(mode >= BOTH && mode <= END); - m_mode = mode; - } - - public int Compare(KeyRange x, KeyRange y) - { - switch (m_mode) - { - case BEGIN: return x.Begin.CompareTo(y.Begin); - case END: return x.End.CompareTo(y.End); - default: return x.CompareTo(y); - } - } - - public bool Equals(KeyRange x, KeyRange y) - { - switch(m_mode) - { - case BEGIN: return x.Begin.Equals(y.Begin); - case END: return x.End.Equals(y.End); - default: return x.Equals(y); - } - } - - public int GetHashCode(KeyRange obj) - { - switch(m_mode) - { - case BEGIN: return obj.Begin.GetHashCode(); - case END: return obj.End.GetHashCode(); - default: return obj.GetHashCode(); - } - } - } - -} diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index 1f9dcea0d..32bb1fb75 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -35,14 +35,11 @@ namespace FoundationDB.Client /// Defines a selector for a key in the database [DebuggerDisplay("{ToString(),nq}")] - public struct KeySelector : IEquatable + public /*readonly*/ struct KeySelector : IEquatable { - /// Empty key selector - public static readonly KeySelector None = default(KeySelector); /// Key of the selector - public Slice Key { get { return m_key; } } - private Slice m_key; //PERF: readonly struct + public /*readonly*/ Slice Key; /// If true, the selected key can be equal to . public readonly bool OrEqual; @@ -53,57 +50,28 @@ public struct KeySelector : IEquatable /// Creates a new selector public KeySelector(Slice key, bool orEqual, int offset) { - m_key = key; + Key = key; this.OrEqual = orEqual; this.Offset = offset; } - /// Returns a displayable representation of the key selector - [Pure] - public string PrettyPrint(FdbKey.PrettyPrintMode mode) - { - var sb = new StringBuilder(); - int offset = this.Offset; - if (offset < 1) - { - sb.Append(this.OrEqual ? "lLE{" : "lLT{"); - } - else - { - --offset; - sb.Append(this.OrEqual ? "fGT{" : "fGE{"); - } - sb.Append(FdbKey.PrettyPrint(m_key, mode)); - sb.Append("}"); - - if (offset > 0) - sb.Append(" + ").Append(offset); - else if (offset < 0) - sb.Append(" - ").Append(-offset); - - return sb.ToString(); - } - - /// Converts the value of the current object into its equivalent string representation - public override string ToString() - { - return PrettyPrint(FdbKey.PrettyPrintMode.Single); - } + /// Empty key selector + public static readonly KeySelector None = default(KeySelector); public bool Equals(KeySelector other) { - return this.Offset == other.Offset && this.OrEqual == other.OrEqual && m_key.Equals(other.m_key); + return this.Offset == other.Offset && this.OrEqual == other.OrEqual && Key.Equals(other.Key); } public override bool Equals(object obj) { - return obj is KeySelector && Equals((KeySelector)obj); + return obj is KeySelector selector && Equals(selector); } public override int GetHashCode() { // ReSharper disable once NonReadonlyMemberInGetHashCode - return m_key.GetHashCode() ^ this.Offset ^ (this.OrEqual ? 0 : -1); + return Key.GetHashCode() ^ this.Offset ^ (this.OrEqual ? 0 : -1); } /// Creates a key selector that will select the last key that is less than @@ -140,7 +108,7 @@ public static KeySelector FirstGreaterOrEqual(Slice key) /// fGE('abc')+7 public static KeySelector operator +(KeySelector selector, int offset) { - return new KeySelector(selector.m_key, selector.OrEqual, selector.Offset + offset); + return new KeySelector(selector.Key, selector.OrEqual, selector.Offset + offset); } /// Substract a value to the selector's offset @@ -149,7 +117,7 @@ public static KeySelector FirstGreaterOrEqual(Slice key) /// fGE('abc')-7 public static KeySelector operator -(KeySelector selector, int offset) { - return new KeySelector(selector.m_key, selector.OrEqual, selector.Offset - offset); + return new KeySelector(selector.Key, selector.OrEqual, selector.Offset - offset); } public static bool operator ==(KeySelector left, KeySelector right) @@ -162,6 +130,38 @@ public static KeySelector FirstGreaterOrEqual(Slice key) return !left.Equals(right); } + /// Converts the value of the current object into its equivalent string representation + public override string ToString() + { + return PrettyPrint(FdbKey.PrettyPrintMode.Single); + } + + /// Returns a displayable representation of the key selector + [Pure] + public string PrettyPrint(FdbKey.PrettyPrintMode mode) + { + var sb = new StringBuilder(); + int offset = this.Offset; + if (offset < 1) + { + sb.Append(this.OrEqual ? "lLE{" : "lLT{"); + } + else + { + --offset; + sb.Append(this.OrEqual ? "fGT{" : "fGE{"); + } + sb.Append(FdbKey.PrettyPrint(Key, mode)); + sb.Append("}"); + + if (offset > 0) + sb.Append(" + ").Append(offset); + else if (offset < 0) + sb.Append(" - ").Append(-offset); + + return sb.ToString(); + } + } } diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index d72e32b96..af9b72983 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -33,23 +33,21 @@ namespace FoundationDB.Client /// Represents of pair of key selectors that range 'GetKey(Begin) <= key < GetKey(End)' [DebuggerDisplay("[ToString()]")] - public struct KeySelectorPair + public /*readonly*/ struct KeySelectorPair { /// Start of the range - public KeySelector Begin { get { return m_begin; } } - private KeySelector m_begin; //PERF: readonly struct + public /*readonly*/ KeySelector Begin; /// End of the range - public KeySelector End { get { return m_end; } } - private KeySelector m_end; //PERF: readonly struct + public /*readonly*/ KeySelector End; /// Create a new pair of key selectors /// Selector for key from which to start iterating /// Selector for key where to stop iterating public KeySelectorPair(KeySelector beginInclusive, KeySelector endExclusive) { - m_begin = beginInclusive; - m_end = endExclusive; + this.Begin = beginInclusive; + this.End = endExclusive; } /// Factory method for a pair of key selectors @@ -93,7 +91,7 @@ public static KeySelectorPair StartsWith(Slice prefix) /// Returns a printable version of the pair of key selectors public override string ToString() { - return "[ " + m_begin.PrettyPrint(FdbKey.PrettyPrintMode.Begin) + ", " + m_end.PrettyPrint(FdbKey.PrettyPrintMode.End) + " )"; + return "[ " + this.Begin.PrettyPrint(FdbKey.PrettyPrintMode.Begin) + ", " + this.End.PrettyPrint(FdbKey.PrettyPrintMode.End) + " )"; } } From 883a2d99cfe5b6752b1e4f15838a62c881247873 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 19:12:03 +0200 Subject: [PATCH 090/153] Mark all candidates for readonly struct with a comment --- FoundationDB.Client/Converters/ComparisonHelper.cs | 2 +- FoundationDB.Client/FdbRangeChunk.cs | 2 +- .../Layers/Directories/FdbDirectoryLayer.cs | 9 +++++---- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 2 +- .../Indexes/Bitmaps/BitRange.cs | 2 +- .../Indexes/Bitmaps/CompressedWord.cs | 2 +- 6 files changed, 10 insertions(+), 9 deletions(-) diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs index ede62675e..8418e6c48 100644 --- a/FoundationDB.Client/Converters/ComparisonHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -41,7 +41,7 @@ internal static class ComparisonHelper { /// Pair of types that can be used as a key in a dictionary - internal struct TypePair : IEquatable + internal /*readonly*/ struct TypePair : IEquatable { public readonly Type Left; public readonly Type Right; diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index 8e410d6bb..a527d36fb 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -35,7 +35,7 @@ namespace FoundationDB.Client using Doxense.Serialization.Encoders; [DebuggerDisplay("Count={Chunk!=null?Chunk.Length:0}, HasMore={HasMore}, Reversed={Reversed}, Iteration={Iteration}")] - public struct FdbRangeChunk + public /*readonly*/ struct FdbRangeChunk { /// Set to true if there are more results in the database than could fit in a single chunk public readonly bool HasMore; diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index d2ec2f485..8a3d15317 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -492,7 +492,7 @@ public override string ToString() private struct Node { - public Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice layer) + public /*readonly*/ Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice layer) { this.Subspace = subspace; this.Path = path; @@ -503,11 +503,12 @@ public Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice public readonly IDynamicKeySubspace Subspace; public readonly ITuple Path; public readonly ITuple TargetPath; - public Slice Layer; //PERF: readonly struct + public /*readonly*/ Slice Layer; //PERF: readonly struct - public bool Exists { get { return this.Subspace != null; } } + public bool Exists => this.Subspace != null; - public ITuple PartitionSubPath { [NotNull] get { return this.TargetPath.Substring(this.Path.Count); } } + [NotNull] + public ITuple PartitionSubPath => this.TargetPath.Substring(this.Path.Count); public bool IsInPartition(bool includeEmptySubPath) { diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index f1fc88b21..1ec803e29 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -91,7 +91,7 @@ protected virtual Slice AttributeKey(string name) #region Internal Helpers... - private struct Chunk + private /*readonly*/ struct Chunk { public readonly Slice Key; public readonly Slice Data; diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs index f96190ae4..89597d804 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs @@ -33,7 +33,7 @@ namespace FoundationDB.Layers.Experimental.Indexing /// Bounds of a Compressed Bitmaps, from the Lowest Set Bit to the Highest Set Bit [DebuggerDisplay("[{Lowest}, {Highest}]")] - public struct BitRange : IEquatable + public /*readonly*/ struct BitRange : IEquatable { private const int LOWEST_UNDEFINED = 0; private const int HIGHEST_UNDEFINED = -1; diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs index 241a4d397..f2dee079c 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs @@ -35,7 +35,7 @@ namespace FoundationDB.Layers.Experimental.Indexing /// Represent a 32-bit word in a Compressed Bitmap [DebuggerDisplay("Literal={IsLiteral}, {WordCount} x {WordValue}")] - public struct CompressedWord + public /*readonly*/ struct CompressedWord { internal const uint ALL_ZEROES = 0x0; internal const uint ALL_ONES = 0x7FFFFFFF; From 23862260801af243639e3ef60b66f3c867a56697 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 19:12:23 +0200 Subject: [PATCH 091/153] code cleanup --- FoundationDB.Client/Status/FdbSystemStatus.cs | 366 +++++------------- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 22 +- FoundationDb.Client.sln.DotSettings | 2 +- 3 files changed, 113 insertions(+), 277 deletions(-) diff --git a/FoundationDB.Client/Status/FdbSystemStatus.cs b/FoundationDB.Client/Status/FdbSystemStatus.cs index 07d9de3c5..e272c205e 100644 --- a/FoundationDB.Client/Status/FdbSystemStatus.cs +++ b/FoundationDB.Client/Status/FdbSystemStatus.cs @@ -26,6 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +// ReSharper disable UnusedMember.Global namespace FoundationDB.Client.Status { using FoundationDB.Client.Utils; @@ -39,40 +40,34 @@ namespace FoundationDB.Client.Status [PublicAPI] public sealed class FdbSystemStatus : MetricsBase { - private readonly ClientStatus m_client; - private readonly ClusterStatus m_cluster; - private readonly long m_readVersion; - private readonly string m_raw; - internal FdbSystemStatus(Dictionary doc, long readVersion, string raw) : base(doc) { - m_client = new ClientStatus(TinyJsonParser.GetMapField(doc, "client")); - m_cluster = new ClusterStatus(TinyJsonParser.GetMapField(doc, "cluster")); - m_readVersion = readVersion; - m_raw = raw; + this.Client = new ClientStatus(TinyJsonParser.GetMapField(doc, "client")); + this.Cluster = new ClusterStatus(TinyJsonParser.GetMapField(doc, "cluster")); + this.ReadVersion = readVersion; + this.RawText = raw; } /// Details about the local Client - public ClientStatus Client { get { return m_client; } } + public ClientStatus Client { get; } /// Details about the remote Cluster - public ClusterStatus Cluster { get { return m_cluster; } } + public ClusterStatus Cluster { get; } /// Read Version of the snapshot - public long ReadVersion { get { return m_readVersion; } } + public long ReadVersion { get; } /// Raw JSON text of this snapshot. /// This is the same value that is returned by running 'status json' in fdbcli - public string RawText { get { return m_raw; } } - + public string RawText { get; } } #region Common... /// Details about a notification, alert or error, as reported by a component of a FoundationDB cluster [DebuggerDisplay("{Name}")] - public struct Message + public /*readonly*/ struct Message { /// Code for this message public readonly string Name; @@ -107,7 +102,7 @@ internal static Message[] FromArray(Dictionary data, string fiel public override string ToString() { - return String.Format("[{0}] {1}", this.Name, this.Description); + return $"[{this.Name}] {this.Description}"; } public override int GetHashCode() @@ -117,7 +112,7 @@ public override int GetHashCode() public override bool Equals(object obj) { - return (obj is Message) && Equals((Message)obj); + return obj is Message message && Equals(message); } public bool Equals(Message other) @@ -129,7 +124,7 @@ public bool Equals(Message other) /// Measured quantity that changes over time [DebuggerDisplay("Counter={Counter}, Hz={Hz}, Roughness={Roughness}")] - public struct LoadCounter + public /*readonly*/ struct LoadCounter { /// Absolute value, since the start (ex: "UNIT") public readonly long Counter; @@ -237,51 +232,29 @@ internal ClientStatus(Dictionary data) : base(data) { } private Message[] m_messages; /// Path to the '.cluster' file used by the client to connect to the cluster - public string ClusterFilePath - { - get { return GetString("cluster_file", "path"); } - } + public string ClusterFilePath => GetString("cluster_file", "path"); /// Indicates if the content of the '.cluster' file is up to date with the current topology of the cluster - public bool ClusterFileUpToDate - { - get { return GetBoolean("cluster_file", "up_to_date") ?? false; } - } + public bool ClusterFileUpToDate => GetBoolean("cluster_file", "up_to_date") ?? false; /// Liste of active messages for the client /// The most common client messages are listed in . - public Message[] Messages - { - [NotNull] - get { return m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); } - } + [NotNull] + public Message[] Messages => m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); /// Timestamp of the local client (unix time) /// Number of seconds since 1970-01-01Z, using the local system clock - public long Timestamp - { - get { return GetInt64("timestamp") ?? 0; } - } + public long Timestamp => GetInt64("timestamp") ?? 0; /// Local system time on the client - public DateTime SystemTime - { - get { return new DateTime(checked(621355968000000000L + this.Timestamp * TimeSpan.TicksPerSecond), DateTimeKind.Utc); } - } + public DateTime SystemTime => new DateTime(checked(621355968000000000L + this.Timestamp * TimeSpan.TicksPerSecond), DateTimeKind.Utc); /// Specifies if the local client was able to connect to the cluster - public bool DatabaseAvailable - { - get { return GetBoolean("database_status", "available") ?? false; } - } + public bool DatabaseAvailable => GetBoolean("database_status", "available") ?? false; /// Specifies if the database is currently healthy //REVIEW: what does it mean if available=true, but healthy=false ? - public bool DatabaseHealthy - { - get { return GetBoolean("database_status", "healthy") ?? false; } - } - + public bool DatabaseHealthy => GetBoolean("database_status", "healthy") ?? false; } /// List of well known client messages @@ -321,58 +294,31 @@ internal ClusterStatus(Dictionary data) /// Unix time of the cluster controller /// Number of seconds since the Unix epoch (1970-01-01Z) - public long ClusterControllerTimestamp - { - get { return GetInt64("cluster_controller_timestamp") ?? 0; } - } + public long ClusterControllerTimestamp => GetInt64("cluster_controller_timestamp") ?? 0; /// License string of the cluster - public string License - { - [NotNull] - get { return GetString("license") ?? String.Empty; } - } + [NotNull] + public string License => GetString("license") ?? String.Empty; /// List of currently active messages /// Includes notifications, warnings, errors, ... - public Message[] Messages - { - [NotNull] - get { return m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); } - } + [NotNull] + public Message[] Messages => m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); /// Recovery state of the cluster - public Message RecoveryState - { - get { return Message.From(m_data, "recovery_state"); } - } + public Message RecoveryState => Message.From(m_data, "recovery_state"); - public ClusterConfiguration Configuration - { - get { return m_configuration ?? (m_configuration = new ClusterConfiguration(GetMap("configuration"))); } - } + public ClusterConfiguration Configuration => m_configuration ?? (m_configuration = new ClusterConfiguration(GetMap("configuration"))); - public DataMetrics Data - { - get { return m_dataMetrics ?? (m_dataMetrics = new DataMetrics(GetMap("data"))); } - } + public DataMetrics Data => m_dataMetrics ?? (m_dataMetrics = new DataMetrics(GetMap("data"))); - public LatencyMetrics Latency - { - get { return m_latency ?? (m_latency = new LatencyMetrics(GetMap("latency_probe"))); } - } + public LatencyMetrics Latency => m_latency ?? (m_latency = new LatencyMetrics(GetMap("latency_probe"))); /// QoS metrics - public QosMetrics Qos - { - get { return m_qos ?? (m_qos = new QosMetrics(GetMap("qos"))); } - } + public QosMetrics Qos => m_qos ?? (m_qos = new QosMetrics(GetMap("qos"))); /// Workload metrics - public WorkloadMetrics Workload - { - get { return m_workload ?? (m_workload = new WorkloadMetrics(GetMap("workload"))); } - } + public WorkloadMetrics Workload => m_workload ?? (m_workload = new WorkloadMetrics(GetMap("workload"))); /// List of the processes that are currently active in the cluster public IReadOnlyDictionary Processes @@ -445,11 +391,11 @@ internal ClusterConfiguration(Dictionary data) private string[] m_excludedServers; - public int CoordinatorsCount { get; private set; } + public int CoordinatorsCount { get; } - public string StorageEngine { get; private set; } + public string StorageEngine { get; } - public string RedundancyFactor { get; private set; } + public string RedundancyFactor { get; } public IReadOnlyList ExcludedServers { @@ -482,11 +428,11 @@ internal LatencyMetrics(Dictionary data) this.TransactionStartSeconds = GetDouble("transaction_start_seconds") ?? 0; } - public double CommitSeconds { get; private set; } + public double CommitSeconds { get; } - public double ReadSeconds { get; private set; } + public double ReadSeconds { get; set; } - public double TransactionStartSeconds { get; private set; } + public double TransactionStartSeconds { get; } } /// Details about the volume of data stored in the cluster @@ -494,56 +440,25 @@ public sealed class DataMetrics : MetricsBase { internal DataMetrics(Dictionary data) : base(data) { } - public long AveragePartitionSizeBytes - { - get { return GetInt64("average_partition_size_bytes") ?? 0; } - } + public long AveragePartitionSizeBytes => GetInt64("average_partition_size_bytes") ?? 0; - public long LeastOperatingSpaceBytesLogServer - { - get { return GetInt64("least_operating_space_bytes_log_server") ?? 0; } - } - - public long LeastOperatingSpaceBytesStorageServer - { - get { return GetInt64("least_operating_space_bytes_storage_server") ?? 0; } - } + public long LeastOperatingSpaceBytesLogServer => GetInt64("least_operating_space_bytes_log_server") ?? 0; - public long MovingDataInFlightBytes - { - get { return GetInt64("moving_data", "in_flight_bytes") ?? 0; } - } + public long LeastOperatingSpaceBytesStorageServer => GetInt64("least_operating_space_bytes_storage_server") ?? 0; - public long MovingDataInQueueBytes - { - get { return GetInt64("moving_data", "in_queue_bytes") ?? 0; } - } + public long MovingDataInFlightBytes => GetInt64("moving_data", "in_flight_bytes") ?? 0; - public long PartitionsCount - { - get { return GetInt64("partitions_count") ?? 0; } - } + public long MovingDataInQueueBytes => GetInt64("moving_data", "in_queue_bytes") ?? 0; - public long TotalDiskUsedBytes - { - get { return GetInt64("total_disk_used_bytes") ?? 0; } - } + public long PartitionsCount => GetInt64("partitions_count") ?? 0; - public long TotalKVUsedBytes - { - get { return GetInt64("total_kv_size_bytes") ?? 0; } - } + public long TotalDiskUsedBytes => GetInt64("total_disk_used_bytes") ?? 0; - public bool StateHealthy - { - get { return GetBoolean("state", "healthy") ?? false; } - } + public long TotalKVUsedBytes => GetInt64("total_kv_size_bytes") ?? 0; - public string StateName - { - get { return GetString("state", "name"); } - } + public bool StateHealthy => GetBoolean("state", "healthy") ?? false; + public string StateName => GetString("state", "name"); } /// Details about the quality of service offered by the cluster @@ -552,23 +467,13 @@ public sealed class QosMetrics : MetricsBase internal QosMetrics(Dictionary data) : base(data) { } /// Current limiting factor for the performance of the cluster - public Message PerformanceLimitedBy - { - get { return Message.From(m_data, "performance_limited_by"); } - } + public Message PerformanceLimitedBy => Message.From(m_data, "performance_limited_by"); //REVIEW: what is this? - public long WorstQueueBytesLogServer - { - get { return GetInt64("worst_queue_bytes_log_server") ?? 0; } - } + public long WorstQueueBytesLogServer => GetInt64("worst_queue_bytes_log_server") ?? 0; //REVIEW: what is this? - public long WorstQueueBytesStorageServer - { - get { return GetInt64("worst_queue_bytes_storage_server") ?? 0; } - } - + public long WorstQueueBytesStorageServer => GetInt64("worst_queue_bytes_storage_server") ?? 0; } /// Details about the current wokrload of the cluster @@ -581,22 +486,13 @@ internal WorkloadMetrics(Dictionary data) : base(data) { } private WorkloadTransactionsMetrics m_transactions; /// Performance counters for the volume of data processed by the database - public WorkloadBytesMetrics Bytes - { - get { return m_bytes ?? (m_bytes = new WorkloadBytesMetrics(GetMap("bytes"))); } - } + public WorkloadBytesMetrics Bytes => m_bytes ?? (m_bytes = new WorkloadBytesMetrics(GetMap("bytes"))); /// Performance counters for the operations on the keys in the database - public WorkloadOperationsMetrics Operations - { - get { return m_operations ?? (m_operations = new WorkloadOperationsMetrics(GetMap("operations"))); } - } + public WorkloadOperationsMetrics Operations => m_operations ?? (m_operations = new WorkloadOperationsMetrics(GetMap("operations"))); /// Performance counters for the transactions. - public WorkloadTransactionsMetrics Transactions - { - get { return m_transactions ?? (m_transactions = new WorkloadTransactionsMetrics(GetMap("transactions"))); } - } + public WorkloadTransactionsMetrics Transactions => m_transactions ?? (m_transactions = new WorkloadTransactionsMetrics(GetMap("transactions"))); } /// Throughput of a FoundationDB cluster @@ -610,7 +506,7 @@ internal WorkloadBytesMetrics(Dictionary data) /// Bytes written //REVIEW: this looks like the size of writes in transactions, NOT the number of bytes written to the disk! - public LoadCounter Written { get; private set; } + public LoadCounter Written { get; } } @@ -625,10 +521,10 @@ internal WorkloadOperationsMetrics(Dictionary data) } /// Details about read operations - public LoadCounter Reads { get; private set; } + public LoadCounter Reads { get; } /// Details about write operations - public LoadCounter Writes { get; private set; } + public LoadCounter Writes { get; } } /// Transaction workload of a FoundationDB cluster @@ -642,11 +538,11 @@ internal WorkloadTransactionsMetrics(Dictionary data) this.Started = LoadCounter.From(data, "started"); } - public LoadCounter Committed { get; private set; } + public LoadCounter Committed { get; } - public LoadCounter Conflicted { get; private set; } + public LoadCounter Conflicted { get; } - public LoadCounter Started { get; private set; } + public LoadCounter Started { get; } } #endregion @@ -674,75 +570,46 @@ internal ProcessStatus(Dictionary data, string id) /// Unique identifier for this process. //TODO: is it stable accross reboots? what are the conditions for a process to change its ID ? - public string Id { [NotNull] get; private set; } + [NotNull] + public string Id { get; } /// Identifier of the machine that is hosting this process /// All processes that have the same MachineId are running on the same (physical) machine. - public string MachineId - { - [NotNull] - get { return m_machineId ?? (m_machineId = GetString("machine_id") ?? String.Empty); } - } + [NotNull] + public string MachineId => m_machineId ?? (m_machineId = GetString("machine_id") ?? String.Empty); /// Version of this process /// "3.0.4" - public string Version - { - [NotNull] - get { return GetString("version") ?? String.Empty; } - } + [NotNull] + public string Version => GetString("version") ?? String.Empty; /// Address and port of this process, with syntax "IP_ADDRESS:port" /// "10.1.2.34:4500" - public string Address - { - [NotNull] - get { return m_address ?? (m_address = GetString("address") ?? String.Empty); } - } + [NotNull] + public string Address => m_address ?? (m_address = GetString("address") ?? String.Empty); /// Command line that was used to start this process - public string CommandLine - { - [NotNull] - get { return GetString("command_line") ?? String.Empty; } - } + [NotNull] + public string CommandLine => GetString("command_line") ?? String.Empty; /// If true, this process is currently excluded from the cluster - public bool Excluded - { - get { return GetBoolean("excluded") ?? false; } - } + public bool Excluded => GetBoolean("excluded") ?? false; /// List of messages that are currently published by this process - public Message[] Messages - { - [NotNull] - get { return m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); } - } + [NotNull] + public Message[] Messages => m_messages ?? (m_messages = Message.FromArray(m_data, "messages")); /// Network performance counters - public ProcessNetworkMetrics Network - { - get { return m_network ?? (m_network = new ProcessNetworkMetrics(GetMap("network"))); } - } + public ProcessNetworkMetrics Network => m_network ?? (m_network = new ProcessNetworkMetrics(GetMap("network"))); /// CPU performance counters - public ProcessCpuMetrics Cpu - { - get { return m_cpu ?? (m_cpu = new ProcessCpuMetrics(GetMap("cpu"))); } - } + public ProcessCpuMetrics Cpu => m_cpu ?? (m_cpu = new ProcessCpuMetrics(GetMap("cpu"))); /// Disk performance counters - public ProcessDiskMetrics Disk - { - get { return m_disk ?? (m_disk = new ProcessDiskMetrics(GetMap("disk"))); } - } + public ProcessDiskMetrics Disk => m_disk ?? (m_disk = new ProcessDiskMetrics(GetMap("disk"))); /// Memory performance counters - public ProcessMemoryMetrics Memory - { - get { return m_memory ?? (m_memory = new ProcessMemoryMetrics(GetMap("memory"))); } - } + public ProcessMemoryMetrics Memory => m_memory ?? (m_memory = new ProcessMemoryMetrics(GetMap("memory"))); /// List of the roles assumed by this process /// The key is the unique role ID in the cluster, and the value is the type of the role itself @@ -786,16 +653,9 @@ internal ProcessMemoryMetrics(Dictionary data) : base(data) { } - public long AvailableBytes - { - get { return GetInt64("available_bytes") ?? 0; } - } - - public long UsedBytes - { - get { return GetInt64("used_bytes") ?? 0; } - } + public long AvailableBytes => GetInt64("available_bytes") ?? 0; + public long UsedBytes => GetInt64("used_bytes") ?? 0; } /// CPU performane counters for a FoundationDB process @@ -805,11 +665,7 @@ internal ProcessCpuMetrics(Dictionary data) : base(data) { } - public double UsageCores - { - get { return GetDouble("usage_cores") ?? 0; } - } - + public double UsageCores => GetDouble("usage_cores") ?? 0; } /// Disk performane counters for a FoundationDB process @@ -819,10 +675,7 @@ internal ProcessDiskMetrics(Dictionary data) : base(data) { } - public double Busy - { - get { return GetDouble("busy") ?? 0; } - } + public double Busy => GetDouble("busy") ?? 0; } /// Network performane counters for a FoundationDB process or machine @@ -835,9 +688,9 @@ internal ProcessNetworkMetrics(Dictionary data) this.MegabitsSent = LoadCounter.From(data, "megabits_sent"); } - public LoadCounter MegabitsReceived { get; private set; } + public LoadCounter MegabitsReceived { get; } - public LoadCounter MegabitsSent { get; private set; } + public LoadCounter MegabitsSent { get; } } @@ -861,47 +714,30 @@ internal MachineStatus(Dictionary data, string id) /// Unique identifier for this machine. //TODO: is it stable accross reboots? what are the conditions for a process to change its ID ? - public string Id { [NotNull] get; private set; } + [NotNull] + public string Id { get; } /// Identifier of the data center that is hosting this machine /// All machines that have the same DataCenterId are probably running on the same (physical) network. - public string DataCenterId - { - [NotNull] - get { return GetString("datacenter_id") ?? String.Empty; } - } + [NotNull] + public string DataCenterId => GetString("datacenter_id") ?? String.Empty; /// Address of this machine /// "10.1.2.34" - public string Address - { - [NotNull] - get { return m_address ?? (m_address = GetString("address") ?? String.Empty); } - } + [NotNull] + public string Address => m_address ?? (m_address = GetString("address") ?? String.Empty); /// If true, this process is currently excluded from the cluster - public bool Excluded - { - get { return GetBoolean("excluded") ?? false; } - } + public bool Excluded => GetBoolean("excluded") ?? false; /// Network performance counters - public MachineNetworkMetrics Network - { - get { return m_network ?? (m_network = new MachineNetworkMetrics(GetMap("network"))); } - } + public MachineNetworkMetrics Network => m_network ?? (m_network = new MachineNetworkMetrics(GetMap("network"))); /// CPU performance counters - public MachineCpuMetrics Cpu - { - get { return m_cpu ?? (m_cpu = new MachineCpuMetrics(GetMap("cpu"))); } - } + public MachineCpuMetrics Cpu => m_cpu ?? (m_cpu = new MachineCpuMetrics(GetMap("cpu"))); /// Memory performance counters - public MachineMemoryMetrics Memory - { - get { return m_memory ?? (m_memory = new MachineMemoryMetrics(GetMap("memory"))); } - } + public MachineMemoryMetrics Memory => m_memory ?? (m_memory = new MachineMemoryMetrics(GetMap("memory"))); } /// Memory performane counters for machine hosting one or more FoundationDB processes @@ -915,11 +751,11 @@ internal MachineMemoryMetrics(Dictionary data) this.TotalBytes = GetInt64("total_bytes") ?? 0; } - public long CommittedBytes { get; private set; } + public long CommittedBytes { get; } - public long FreeBytes { get; private set; } + public long FreeBytes { get; } - public long TotalBytes { get; private set; } + public long TotalBytes { get; } } @@ -932,7 +768,7 @@ internal MachineCpuMetrics(Dictionary data) this.LogicalCoreUtilization = GetDouble("logical_core_utilization") ?? 0; } - public double LogicalCoreUtilization { get; private set; } + public double LogicalCoreUtilization { get; } } @@ -947,15 +783,15 @@ internal MachineNetworkMetrics(Dictionary data) this.TcpSegmentsRetransmitted = LoadCounter.From(data, "tcp_segments_retransmitted"); } - public LoadCounter MegabitsReceived { get; private set; } + public LoadCounter MegabitsReceived { get; } - public LoadCounter MegabitsSent { get; private set; } + public LoadCounter MegabitsSent { get; } - public LoadCounter TcpSegmentsRetransmitted { get; private set; } + public LoadCounter TcpSegmentsRetransmitted { get; } } #endregion -} \ No newline at end of file +} diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 1ec803e29..f1cbb42ad 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -38,7 +38,7 @@ namespace FoundationDB.Layers.Blobs using Doxense.Serialization.Encoders; /// Represents a potentially large binary value in FoundationDB. - [DebuggerDisplay("Subspace={Subspace}")] + [DebuggerDisplay("Subspace={" + nameof(FdbBlob.Subspace) + "}")] public class FdbBlob { private const long CHUNK_LARGE = 10000; // all chunks will be not greater than this size @@ -55,7 +55,7 @@ public class FdbBlob /// Subspace to be used for storing the blob data and metadata public FdbBlob([NotNull] IKeySubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace.Using(TypeSystem.Tuples); } @@ -201,7 +201,7 @@ private void SetSize([NotNull] IFdbTransaction trans, long size) /// public void Delete([NotNull] IFdbTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); trans.ClearRange(this.Subspace); } @@ -212,7 +212,7 @@ public void Delete([NotNull] IFdbTransaction trans) /// Return null if the blob does not exists, 0 if is empty, or the size in bytes public async Task GetSizeAsync([NotNull] IFdbReadOnlyTransaction trans) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); Slice value = await trans.GetAsync(SizeKey()).ConfigureAwait(false); @@ -229,8 +229,8 @@ public void Delete([NotNull] IFdbTransaction trans) /// public async Task ReadAsync([NotNull] IFdbReadOnlyTransaction trans, long offset, int n) { - if (trans == null) throw new ArgumentNullException("trans"); - if (offset < 0) throw new ArgumentNullException("offset", "Offset cannot be less than zero"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (offset < 0) throw new ArgumentNullException(nameof(offset), "Offset cannot be less than zero"); long? size = await GetSizeAsync(trans).ConfigureAwait(false); if (size == null) return Slice.Nil; // not found @@ -282,8 +282,8 @@ await trans /// public async Task WriteAsync([NotNull] IFdbTransaction trans, long offset, Slice data) { - if (trans == null) throw new ArgumentNullException("trans"); - if (offset < 0) throw new ArgumentOutOfRangeException("offset", "Offset cannot be less than zero"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (offset < 0) throw new ArgumentOutOfRangeException(nameof(offset), "Offset cannot be less than zero"); if (data.IsNullOrEmpty) return; @@ -308,7 +308,7 @@ public async Task WriteAsync([NotNull] IFdbTransaction trans, long offset, Slice /// public async Task AppendAsync([NotNull] IFdbTransaction trans, Slice data) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); if (data.IsNullOrEmpty) return; @@ -323,8 +323,8 @@ public async Task AppendAsync([NotNull] IFdbTransaction trans, Slice data) /// public async Task TruncateAsync([NotNull] IFdbTransaction trans, long newLength) { - if (trans == null) throw new ArgumentNullException("trans"); - if (newLength < 0) throw new ArgumentOutOfRangeException("newLength", "Length cannot be less than zero"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (newLength < 0) throw new ArgumentOutOfRangeException(nameof(newLength), "Length cannot be less than zero"); long? length = await GetSizeAsync(trans).ConfigureAwait(false); if (length != null) diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index b8937379c..e627bc3ce 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -65,7 +65,7 @@ <Policy Inspect="True" Prefix="" Suffix="" Style="aaBb"><ExtraRule Prefix="_" Suffix="" Style="aaBb" /></Policy> <Policy Inspect="True" Prefix="" Suffix="" Style="AaBb"><ExtraRule Prefix="" Suffix="" Style="AA_BB" /></Policy> <Policy Inspect="True" Prefix="m_" Suffix="" Style="aaBb"><ExtraRule Prefix="" Suffix="" Style="AaBb" /></Policy> - <Policy Inspect="True" Prefix="s_" Suffix="" Style="aaBb" /> + <Policy Inspect="True" Prefix="s_" Suffix="" Style="aaBb"><ExtraRule Prefix="" Suffix="" Style="AaBb" /></Policy> <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 From dcee767b3742e0d212579aef94a930ff3dcacc62 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 19:12:39 +0200 Subject: [PATCH 092/153] Mark Optional as obsolete (redundant with Maybe) --- FoundationDB.Layers.Common/Optional`1.cs | 1 + 1 file changed, 1 insertion(+) diff --git a/FoundationDB.Layers.Common/Optional`1.cs b/FoundationDB.Layers.Common/Optional`1.cs index 82247328c..259b65891 100644 --- a/FoundationDB.Layers.Common/Optional`1.cs +++ b/FoundationDB.Layers.Common/Optional`1.cs @@ -246,6 +246,7 @@ public static IEnumerable> Decode(this IEnumerable source, /// Container that is either empty (no value) or null (for reference types), or contains a value of type . /// Type of the value [Serializable, StructLayout(LayoutKind.Sequential)] + [Obsolete("Use Maybe instead")] public struct Optional : IEquatable>, IEquatable { // This is the equivalent of Nullable that would accept reference types. From 34a699ca3fb5f0ed5dc4eb3e519178ceaadbc6d8 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 20:16:11 +0200 Subject: [PATCH 093/153] Use "readonly struct" modifier for all immutable structs (Slice, STuple<..>, KeyRange, KeySelector, FdbRangeChunk, ...) - This should remove a lot of struct copies when passing them arround (with a modern JIT). - This needs C# 7.x support! --- FoundationDB.Client/Async/Maybe.cs | 2 +- FoundationDB.Client/Converters/ComparisonHelper.cs | 2 +- FoundationDB.Client/FdbRangeChunk.cs | 2 +- FoundationDB.Client/FdbWatch.cs | 2 +- FoundationDB.Client/KeyRange.cs | 6 +++--- FoundationDB.Client/KeySelector.cs | 4 ++-- FoundationDB.Client/KeySelectorPair.cs | 6 +++--- .../Layers/Directories/FdbDirectoryLayer.cs | 6 +++--- .../Layers/Tuples/Encoding/ITupleSerializable.cs | 2 +- .../Layers/Tuples/Encoding/PrefixedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`1.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`2.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`3.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`4.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`5.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 4 ++-- FoundationDB.Client/Layers/Tuples/TupleSerializer.cs | 12 ++++++------ FoundationDB.Client/Status/FdbSystemStatus.cs | 4 ++-- FoundationDB.Client/Subspaces/DynamicKeySubspace.cs | 4 ++-- FoundationDB.Client/Utils/Memory/Slice.cs | 2 +- FoundationDB.Client/Utils/Uuid128.cs | 2 +- FoundationDB.Client/Utils/Uuid64.cs | 2 +- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 2 +- .../Indexes/Bitmaps/BitRange.cs | 2 +- .../Indexes/Bitmaps/CompressedBitmap.cs | 4 ++-- .../Indexes/Bitmaps/CompressedWord.cs | 2 +- 28 files changed, 50 insertions(+), 50 deletions(-) diff --git a/FoundationDB.Client/Async/Maybe.cs b/FoundationDB.Client/Async/Maybe.cs index 2151c49be..7702cd3ec 100644 --- a/FoundationDB.Client/Async/Maybe.cs +++ b/FoundationDB.Client/Async/Maybe.cs @@ -37,7 +37,7 @@ namespace Doxense using Doxense.Diagnostics.Contracts; using JetBrains.Annotations; - public /*readonly*/ struct Maybe : IEquatable>, IEquatable, IComparable>, IComparable, IFormattable + public readonly struct Maybe : IEquatable>, IEquatable, IComparable>, IComparable, IFormattable { /// Réprésente un résultat vide (no computation) public static readonly Maybe Nothing = new Maybe(); diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Converters/ComparisonHelper.cs index 8418e6c48..62da240e1 100644 --- a/FoundationDB.Client/Converters/ComparisonHelper.cs +++ b/FoundationDB.Client/Converters/ComparisonHelper.cs @@ -41,7 +41,7 @@ internal static class ComparisonHelper { /// Pair of types that can be used as a key in a dictionary - internal /*readonly*/ struct TypePair : IEquatable + internal readonly struct TypePair : IEquatable { public readonly Type Left; public readonly Type Right; diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index a527d36fb..e9c7cffd0 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -35,7 +35,7 @@ namespace FoundationDB.Client using Doxense.Serialization.Encoders; [DebuggerDisplay("Count={Chunk!=null?Chunk.Length:0}, HasMore={HasMore}, Reversed={Reversed}, Iteration={Iteration}")] - public /*readonly*/ struct FdbRangeChunk + public readonly struct FdbRangeChunk { /// Set to true if there are more results in the database than could fit in a single chunk public readonly bool HasMore; diff --git a/FoundationDB.Client/FdbWatch.cs b/FoundationDB.Client/FdbWatch.cs index d23aab4f1..cbf17dafc 100644 --- a/FoundationDB.Client/FdbWatch.cs +++ b/FoundationDB.Client/FdbWatch.cs @@ -42,7 +42,7 @@ public struct FdbWatch : IDisposable //REVIEW: sould we change this to a class? private readonly FdbFuture m_future; //REVIEW: this is specific to the native handler, we should find a better abstraction for the generic case! (we need a Task-like object that can be cancelled/disposed) - private Slice m_key; //PERF: readonly slice + private readonly Slice m_key; private Slice m_value; internal FdbWatch(FdbFuture future, Slice key, Slice value) diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index 3efa32e27..f87cebf39 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -38,14 +38,14 @@ namespace FoundationDB.Client /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] - public /*readonly*/ struct KeyRange : IEquatable, IComparable + public readonly struct KeyRange : IEquatable, IComparable { /// Start of the range - public /*readonly*/ Slice Begin; + public readonly Slice Begin; /// End of the range - public /*readonly*/ Slice End; + public readonly Slice End; /// Create a new range of keys /// Start of range (usually included) diff --git a/FoundationDB.Client/KeySelector.cs b/FoundationDB.Client/KeySelector.cs index 32bb1fb75..c66564373 100644 --- a/FoundationDB.Client/KeySelector.cs +++ b/FoundationDB.Client/KeySelector.cs @@ -35,11 +35,11 @@ namespace FoundationDB.Client /// Defines a selector for a key in the database [DebuggerDisplay("{ToString(),nq}")] - public /*readonly*/ struct KeySelector : IEquatable + public readonly struct KeySelector : IEquatable { /// Key of the selector - public /*readonly*/ Slice Key; + public readonly Slice Key; /// If true, the selected key can be equal to . public readonly bool OrEqual; diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index af9b72983..9192abf74 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -33,13 +33,13 @@ namespace FoundationDB.Client /// Represents of pair of key selectors that range 'GetKey(Begin) <= key < GetKey(End)' [DebuggerDisplay("[ToString()]")] - public /*readonly*/ struct KeySelectorPair + public readonly struct KeySelectorPair { /// Start of the range - public /*readonly*/ KeySelector Begin; + public readonly KeySelector Begin; /// End of the range - public /*readonly*/ KeySelector End; + public readonly KeySelector End; /// Create a new pair of key selectors /// Selector for key from which to start iterating diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 8a3d15317..3f8ec5628 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -489,10 +489,10 @@ public override string ToString() #region Internal Helpers... - private struct Node + private readonly struct Node { - public /*readonly*/ Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice layer) + public Node(IDynamicKeySubspace subspace, ITuple path, ITuple targetPath, Slice layer) { this.Subspace = subspace; this.Path = path; @@ -503,7 +503,7 @@ private struct Node public readonly IDynamicKeySubspace Subspace; public readonly ITuple Path; public readonly ITuple TargetPath; - public /*readonly*/ Slice Layer; //PERF: readonly struct + public readonly Slice Layer; public bool Exists => this.Subspace != null; diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs index 919772fbd..6e6d28d1f 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs @@ -50,7 +50,7 @@ public interface ITupleSerializer //REVIEW: ITuplePacker ? /// Appends the packed bytes of an item to the end of a buffer /// Buffer that will received the packed bytes of this instance /// Tuple that will be packed - void PackTo(ref TupleWriter writer, ref TTuple tuple); + void PackTo(ref TupleWriter writer, in TTuple tuple); /// Decode the packed bytes from a buffer, and return the corresponding item /// Buffer that contains the bytes the decode diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs index d4dc5e088..cc424df50 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs @@ -43,7 +43,7 @@ public sealed class PrefixedTuple : ITuple, ITupleSerializable // Used in scenario where we will append keys to a common base tuple // note: linked list are not very efficient, but we do not expect a very long chain, and the head will usually be a subspace or memoized tuple - private Slice m_prefix; //PERF: readonly struct + private readonly Slice m_prefix; private readonly ITuple m_items; public PrefixedTuple(Slice prefix, ITuple items) diff --git a/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs index 9840289a4..f00eff56c 100644 --- a/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs +++ b/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs @@ -44,7 +44,7 @@ public sealed class MemoizedTuple : ITuple private readonly object[] m_items; /// Packed version of the tuple - private Slice m_packed; //PERF: readonly struct + private readonly Slice m_packed; internal MemoizedTuple(object[] items, Slice packed) { diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index a301fd7f9..10967f47d 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -43,14 +43,14 @@ namespace Doxense.Collections.Tuples /// Factory class for Tuples [PublicAPI] - public struct STuple : ITuple, ITupleSerializable + public readonly struct STuple : ITuple, ITupleSerializable { //note: We cannot use 'Tuple' because it's already used by the BCL in the System namespace, and we cannot use 'Tuples' either because it is part of the namespace... /// Empty tuple /// Not to be mistaken with a 1-tuple containing 'null' ! [NotNull] - public static ITuple Empty => new STuple(); + public static readonly ITuple Empty = new STuple(); #region Empty Tuple diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs index 302f4b230..bddd4f35f 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -44,7 +44,7 @@ namespace Doxense.Collections.Tuples /// Tuple that holds only one item /// Type of the item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -95,7 +95,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T2 value) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs index 08b7461d4..308974fc5 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -45,7 +45,7 @@ namespace Doxense.Collections.Tuples /// Type of the first item /// Type of the second item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -124,7 +124,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T3 value) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs index 968c7d6a4..22def275c 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -46,7 +46,7 @@ namespace Doxense.Collections.Tuples /// Type of the second item /// Type of the third item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -130,7 +130,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T4 value) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs index f406534e8..02df7669c 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -47,7 +47,7 @@ namespace Doxense.Collections.Tuples /// Type of the third item /// Type of the fourth item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -139,7 +139,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T5 value) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index 1db3347cf..1056f8604 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -48,7 +48,7 @@ namespace Doxense.Collections.Tuples /// Type of the 4th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -145,7 +145,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T6 value) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs index a8e17064e..b7f0831a6 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -49,7 +49,7 @@ namespace Doxense.Collections.Tuples /// Type of the 5th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public struct STuple : ITuple, ITupleSerializable, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> #if ENABLE_VALUETUPLES , IEquatable> #endif @@ -151,7 +151,7 @@ void ITupleSerializable.PackTo(ref TupleWriter writer) [MethodImpl(MethodImplOptions.AggressiveInlining)] internal void PackTo(ref TupleWriter writer) { - TupleSerializer.Default.PackTo(ref writer, ref this); + TupleSerializer.Default.PackTo(ref writer, in this); } ITuple ITuple.Append(T7 value) diff --git a/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs b/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs index 038486b45..4f38a920a 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs @@ -35,7 +35,7 @@ public sealed class TupleSerializer : ITupleSerializer> { public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); } @@ -50,7 +50,7 @@ public sealed class TupleSerializer : ITupleSerializer> { public static TupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); TuplePackers.SerializeTo(ref writer, tuple.Item2); @@ -66,7 +66,7 @@ public sealed class TupleSerializer : ITupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); TuplePackers.SerializeTo(ref writer, tuple.Item2); @@ -83,7 +83,7 @@ public sealed class TupleSerializer : ITupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); TuplePackers.SerializeTo(ref writer, tuple.Item2); @@ -101,7 +101,7 @@ public sealed class TupleSerializer : ITupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); TuplePackers.SerializeTo(ref writer, tuple.Item2); @@ -120,7 +120,7 @@ public sealed class TupleSerializer : ITupleSerializer Default { [NotNull] get; } = new TupleSerializer(); - public void PackTo(ref TupleWriter writer, ref STuple tuple) + public void PackTo(ref TupleWriter writer, in STuple tuple) { TuplePackers.SerializeTo(ref writer, tuple.Item1); TuplePackers.SerializeTo(ref writer, tuple.Item2); diff --git a/FoundationDB.Client/Status/FdbSystemStatus.cs b/FoundationDB.Client/Status/FdbSystemStatus.cs index e272c205e..8e7d55967 100644 --- a/FoundationDB.Client/Status/FdbSystemStatus.cs +++ b/FoundationDB.Client/Status/FdbSystemStatus.cs @@ -67,7 +67,7 @@ internal FdbSystemStatus(Dictionary doc, long readVersion, strin /// Details about a notification, alert or error, as reported by a component of a FoundationDB cluster [DebuggerDisplay("{Name}")] - public /*readonly*/ struct Message + public readonly struct Message { /// Code for this message public readonly string Name; @@ -124,7 +124,7 @@ public bool Equals(Message other) /// Measured quantity that changes over time [DebuggerDisplay("Counter={Counter}, Hz={Hz}, Roughness={Roughness}")] - public /*readonly*/ struct LoadCounter + public readonly struct LoadCounter { /// Absolute value, since the start (ex: "UNIT") public readonly long Counter; diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 03163f552..1ae0d821b 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -342,11 +342,11 @@ public string Dump(Slice packedKey) } - public /*readonly*/ struct DynamicPartition + public sealed class DynamicPartition { [NotNull] - public readonly IDynamicKeySubspace Subspace; + public IDynamicKeySubspace Subspace { get; } internal DynamicPartition([NotNull] DynamicKeySubspace subspace) diff --git a/FoundationDB.Client/Utils/Memory/Slice.cs b/FoundationDB.Client/Utils/Memory/Slice.cs index c46e5f395..861323b16 100644 --- a/FoundationDB.Client/Utils/Memory/Slice.cs +++ b/FoundationDB.Client/Utils/Memory/Slice.cs @@ -49,7 +49,7 @@ namespace System /// A Slice if the logical equivalent to a ReadOnlySpan<byte> [PublicAPI, ImmutableObject(true), DebuggerDisplay("Count={Count}, Offset={Offset}"), DebuggerTypeProxy(typeof(Slice.DebugView))] [DebuggerNonUserCode] //remove this when you need to troubleshoot this class! - public /*readonly*/ partial struct Slice : IEquatable, IEquatable>, IEquatable, IComparable, IFormattable + public readonly partial struct Slice : IEquatable, IEquatable>, IEquatable, IComparable, IFormattable { #region Static Members... diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Utils/Uuid128.cs index 2211b7a1c..e71429f27 100644 --- a/FoundationDB.Client/Utils/Uuid128.cs +++ b/FoundationDB.Client/Utils/Uuid128.cs @@ -42,7 +42,7 @@ namespace System /// You should use this type if you are primarily exchanging UUIDs with non-.NET platforms, that use the RFC 4122 byte ordering (big endian). The type System.Guid uses the Microsoft encoding (little endian) and is not compatible. [DebuggerDisplay("[{ToString(),nq}]")] [ImmutableObject(true), StructLayout(LayoutKind.Explicit), Serializable] - public struct Uuid128 : IFormattable, IComparable, IEquatable, IComparable, IEquatable + public readonly struct Uuid128 : IFormattable, IComparable, IEquatable, IComparable, IEquatable { // This is just a wrapper struct on System.Guid that makes sure that ToByteArray() and Parse(byte[]) and new(byte[]) will parse according to RFC 4122 (http://www.ietf.org/rfc/rfc4122.txt) // For performance reasons, we will store the UUID as a System.GUID (Microsoft in-memory format), and swap the bytes when needed. diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Utils/Uuid64.cs index 55a485da4..526b28a72 100644 --- a/FoundationDB.Client/Utils/Uuid64.cs +++ b/FoundationDB.Client/Utils/Uuid64.cs @@ -44,7 +44,7 @@ namespace System /// Represents a 64-bit UUID that is stored in high-endian format on the wire [DebuggerDisplay("[{ToString(),nq}]")] [ImmutableObject(true), Serializable] - public struct Uuid64 : IFormattable, IEquatable, IComparable + public readonly struct Uuid64 : IFormattable, IEquatable, IComparable { public static readonly Uuid64 Empty = default(Uuid64); diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index f1cbb42ad..9b9c1cd30 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -91,7 +91,7 @@ protected virtual Slice AttributeKey(string name) #region Internal Helpers... - private /*readonly*/ struct Chunk + private readonly struct Chunk { public readonly Slice Key; public readonly Slice Data; diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs index 89597d804..5d3df5b4d 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/BitRange.cs @@ -33,7 +33,7 @@ namespace FoundationDB.Layers.Experimental.Indexing /// Bounds of a Compressed Bitmaps, from the Lowest Set Bit to the Highest Set Bit [DebuggerDisplay("[{Lowest}, {Highest}]")] - public /*readonly*/ struct BitRange : IEquatable + public readonly struct BitRange : IEquatable { private const int LOWEST_UNDEFINED = 0; private const int HIGHEST_UNDEFINED = -1; diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs index 33505928c..71b0346da 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedBitmap.cs @@ -43,8 +43,8 @@ public sealed class CompressedBitmap : IEnumerable /// Returns a new instance of an empty bitmap public static readonly CompressedBitmap Empty = new CompressedBitmap(Slice.Empty, BitRange.Empty); - private /*readonly*/ Slice m_data; - private /*readonly*/ BitRange m_bounds; + private readonly Slice m_data; + private readonly BitRange m_bounds; public CompressedBitmap(Slice data) { diff --git a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs index f2dee079c..93deadbf8 100644 --- a/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs +++ b/FoundationDB.Layers.Experimental/Indexes/Bitmaps/CompressedWord.cs @@ -35,7 +35,7 @@ namespace FoundationDB.Layers.Experimental.Indexing /// Represent a 32-bit word in a Compressed Bitmap [DebuggerDisplay("Literal={IsLiteral}, {WordCount} x {WordValue}")] - public /*readonly*/ struct CompressedWord + public readonly struct CompressedWord { internal const uint ALL_ZEROES = 0x0; internal const uint ALL_ONES = 0x7FFFFFFF; From 94267eb6af92883f5118cb89f7694c2ecfaa8a35 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 20:16:23 +0200 Subject: [PATCH 094/153] code cleanup --- FoundationDB.Client/FdbKey.cs | 26 +++++++++++----------- FoundationDB.Client/FdbOperationContext.cs | 19 +++++++++------- FoundationDB.Client/FdbTransaction.cs | 2 +- 3 files changed, 25 insertions(+), 22 deletions(-) diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 33807e5d6..3d16fbfbb 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -302,20 +302,20 @@ public static string PrettyPrint(Slice key, PrettyPrintMode mode) switch (key[-1]) { case 0xFF: - { - //***README*** if you break under here, see README in the last catch() block - tuple = TuPack.Unpack(key[0, -1]); - suffix = "."; - break; - } + { + //***README*** if you break under here, see README in the last catch() block + tuple = TuPack.Unpack(key[0, -1]); + suffix = "."; + break; + } case 0x01: - { - var tmp = key[0, -1] + (byte)0; - //***README*** if you break under here, see README in the last catch() block - tuple = TuPack.Unpack(tmp); - suffix = " + 1"; - break; - } + { + var tmp = key[0, -1] + (byte)0; + //***README*** if you break under here, see README in the last catch() block + tuple = TuPack.Unpack(tmp); + suffix = " + 1"; + break; + } } break; } diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index aee9f0e20..879bb3c0b 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -46,14 +46,15 @@ public sealed class FdbOperationContext : IDisposable //REVIEW: maybe we should find a way to reduce the size of this class? (it's already almost at 100 bytes !) /// The database used by the operation - public IFdbDatabase Database { [NotNull] get; private set; /*readonly*/ } + [NotNull] + public IFdbDatabase Database { get; } /// Result of the operation (or null) public object Result { get; set; } //REVIEW: should we force using a "SetResult()/TrySetResult()" method for this ? /// Cancellation token associated with the operation - public CancellationToken Cancellation { get; private set; /*readonly*/ } + public CancellationToken Cancellation { get; } /// If set to true, will abort and not commit the transaction. If false, will try to commit the transaction (and retry on failure) public bool Abort { get; set; } @@ -65,30 +66,32 @@ public sealed class FdbOperationContext : IDisposable public DateTime StartedUtc { get; private set; } /// Stopwatch that is started at the creation of the transaction, and stopped when it commits or gets disposed - internal Stopwatch Clock { [NotNull] get; private set; /*readonly*/ } + [NotNull] + internal Stopwatch Clock { get; } /// Duration of all the previous attemps before the current one (starts at 0, and gets updated at each reset/retry) internal TimeSpan BaseDuration { get; private set; } /// Time elapsed since the start of the first attempt - public TimeSpan ElapsedTotal { get { return this.Clock.Elapsed; } } + public TimeSpan ElapsedTotal => this.Clock.Elapsed; /// Time elapsed since the start of the current attempt /// This value is reset to zero every time the transation fails and is retried. /// Note that this may not represent the actual lifetime of the transaction with the database itself, which starts at the first read operation. - public TimeSpan Elapsed { get { return this.Clock.Elapsed.Subtract(this.BaseDuration); } } + public TimeSpan Elapsed => this.Clock.Elapsed.Subtract(this.BaseDuration); /// If true, the transaction has been committed successfully public bool Committed { get; private set; } /// If true, the lifetime of the context is handled by an external retry loop. If false, the context is linked to the lifetime of the transaction instance. - internal bool Shared { get { return (this.Mode & FdbTransactionMode.InsideRetryLoop) != 0; } } + internal bool Shared => (this.Mode & FdbTransactionMode.InsideRetryLoop) != 0; /// Mode of the transaction - public FdbTransactionMode Mode { get; private set; /*readonly*/ } + public FdbTransactionMode Mode { get; } /// Internal source of cancellation, able to abort any pending IO operations attached to this transaction - internal CancellationTokenSource TokenSource { [CanBeNull] get; private set; /*readonly*/ } + [CanBeNull] + internal CancellationTokenSource TokenSource { get; } /// Create a new retry loop operation context /// Database that will be used by the retry loop diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 583c29f19..25fc38b93 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -88,7 +88,7 @@ public sealed partial class FdbTransaction : IFdbTransaction private readonly CancellationTokenSource m_cts; /// CancellationToken that should be used for all async operations executing inside this transaction - private CancellationToken m_cancellation; //PERF: readonly struct + private CancellationToken m_cancellation; #endregion From b74e2f73a799ca5f082999a2ef379d42b87f0246 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Mon, 23 Apr 2018 20:17:07 +0200 Subject: [PATCH 095/153] Fix key serilization tests for byte string in tuples - New code uses backquotes for strings decoded from Slice --- FoundationDB.Tests/KeyFacts.cs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index cddbd2c07..6423991c2 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -67,19 +67,19 @@ public void Test_FdbKey_Increment() { var key = FdbKey.Increment(Slice.FromByteString("Hello")); - Assert.That(key.ToStringAscii(), Is.EqualTo("Hellp")); + Assert.That(key.ToString(), Is.EqualTo("Hellp")); key = FdbKey.Increment(Slice.FromByteString("Hello\x00")); - Assert.That(key.ToStringAscii(), Is.EqualTo("Hello\x01")); + Assert.That(key.ToString(), Is.EqualTo("Hello<01>")); key = FdbKey.Increment(Slice.FromByteString("Hello\xFE")); - Assert.That(key.ToStringAscii(), Is.EqualTo("Hello\xFF")); + Assert.That(key.ToString(), Is.EqualTo("Hello")); key = FdbKey.Increment(Slice.FromByteString("Hello\xFF")); - Assert.That(key.ToStringAscii(), Is.EqualTo("Hellp"), "Should remove training \\xFF"); + Assert.That(key.ToString(), Is.EqualTo("Hellp"), "Should remove training \\xFF"); key = FdbKey.Increment(Slice.FromByteString("A\xFF\xFF\xFF")); - Assert.That(key.ToStringAscii(), Is.EqualTo("B"), "Should truncate all trailing \\xFFs"); + Assert.That(key.ToString(), Is.EqualTo("B"), "Should truncate all trailing \\xFFs"); // corner cases Assert.That(() => FdbKey.Increment(Slice.Nil), Throws.InstanceOf().With.Property("ParamName").EqualTo("slice")); @@ -346,19 +346,19 @@ public void Test_FdbKey_PrettyPrint() // tuples should be decoded properly Assert.That(FdbKey.Dump(TuPack.EncodeKey(123)), Is.EqualTo("(123,)"), "Singleton tuples should end with a ','"); - Assert.That(FdbKey.Dump(TuPack.EncodeKey(Slice.FromByteString("hello"))), Is.EqualTo("('hello',)"), "ASCII strings should use single quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(Slice.FromByteString("hello"))), Is.EqualTo("(`hello`,)"), "ASCII strings should use single back quotes"); Assert.That(FdbKey.Dump(TuPack.EncodeKey("héllø")), Is.EqualTo("(\"héllø\",)"), "Unicode strings should use double quotes"); - Assert.That(FdbKey.Dump(TuPack.EncodeKey(new byte[] { 1, 2, 3 }.AsSlice())), Is.EqualTo("(<01 02 03>,)")); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(new byte[] { 1, 2, 3 }.AsSlice())), Is.EqualTo("(`<01><02><03>`,)")); Assert.That(FdbKey.Dump(TuPack.EncodeKey(123, 456)), Is.EqualTo("(123, 456)"), "Elements should be separated with a space, and not end up with ','"); Assert.That(FdbKey.Dump(TuPack.EncodeKey(true, false, default(object))), Is.EqualTo("(1, 0, null)"), "Booleans should be displayed as numbers, and null should be in lowercase"); //note: even though it's tempting to using Python's "Nil", it's not very ".NETty" Assert.That(FdbKey.Dump(TuPack.EncodeKey(1.0d, Math.PI, Math.E)), Is.EqualTo("(1, 3.1415926535897931, 2.7182818284590451)"), "Doubles should used dot and have full precision (17 digits)"); Assert.That(FdbKey.Dump(TuPack.EncodeKey(1.0f, (float)Math.PI, (float)Math.E)), Is.EqualTo("(1, 3.14159274, 2.71828175)"), "Singles should used dot and have full precision (10 digits)"); var guid = Guid.NewGuid(); - Assert.That(FdbKey.Dump(TuPack.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(guid)), Is.EqualTo(String.Format("({0},)", guid.ToString("B"))), "GUIDs should be displayed as a string literal, surrounded by {{...}}, and without quotes"); var uuid128 = Uuid128.NewUuid(); - Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid128)), Is.EqualTo(String.Format("({0},)", uuid128.ToString("B"))), "Uuid128s should be displayed as a string literal, surrounded by {{...}}, and without quotes"); var uuid64 = Uuid64.NewUuid(); - Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {...}, and without quotes"); + Assert.That(FdbKey.Dump(TuPack.EncodeKey(uuid64)), Is.EqualTo(String.Format("({0},)", uuid64.ToString("B"))), "Uuid64s should be displayed as a string literal, surrounded by {{...}}, and without quotes"); // ranges should be decoded when possible var key = TuPack.ToRange(STuple.Create("hello")); From 807b9aa63685dea1ba8b5bade53c5378b2bab176 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Tue, 24 Apr 2018 12:26:57 +0200 Subject: [PATCH 096/153] Updated minimum required version of Visual Studio in README (15.5+) --- README.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/README.md b/README.md index da950ef00..e011b8047 100644 --- a/README.md +++ b/README.md @@ -195,7 +195,7 @@ How to build ### Visual Studio Solution -You will need Visual Studio .NET 2017 and .NET 4.6.1 minimum to compile the solution. +You will need Visual Studio 2017 version 15.5 or above to build the solution (C# 7.2 and .NET Standard 2.0 support is required). You will also need to obtain the 'fdb_c.dll' C API binding from the foundationdb.org wesite, by installing the client SDK: @@ -205,10 +205,6 @@ You will also need to obtain the 'fdb_c.dll' C API binding from the foundationdb * Open the FoundationDb.Client.sln file in Visual Studio 2012. * Choose the Release or Debug configuration, and rebuild the solution. -If you see errors on 'await' or 'async' keywords, please make sure that you are using Visual Studio 2012 or 2013 RC, and not an earlier version. - -If you see the error `Unable to locate '...\foundationdb-dotnet-client\.nuget\nuget.exe'` then you need to run the `Enable Nuget Package Restore` entry in the `Project` menu (or right click on the solution) that will reinstall nuget.exe in the .nuget folder. Also, Nuget should redownload the missing packages during the first build. - ### From the Command Line You can also build, test and compile the NuGet packages from the command line, using FAKE. From 7f10cc869be837bed7d2a4a7e6657e853b271d0e Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Tue, 24 Apr 2018 16:01:05 +0200 Subject: [PATCH 097/153] Missed some copyright headers --- FoundationDB.Client/Async/AsyncOrderingMode.cs | 2 +- FoundationDB.Client/Async/IAsyncBuffer.cs | 2 +- FoundationDB.Client/Async/IAsyncSource.cs | 2 +- FoundationDB.Client/Async/IAsyncTarget.cs | 2 +- FoundationDB.Client/Converters/ITypeConverter.cs | 2 +- FoundationDB.Client/Converters/SimilarValueComparer.cs | 2 +- FoundationDB.Client/FdbClusterFile.cs | 2 +- FoundationDB.Client/FdbClusterOption.cs | 2 +- FoundationDB.Client/FdbConflictRangeType.cs | 2 +- FoundationDB.Client/FdbEndPoint.cs | 2 +- FoundationDB.Client/FdbError.cs | 2 +- FoundationDB.Client/FdbException.cs | 2 +- FoundationDB.Client/FdbMergeQueryExtensions.cs | 2 +- FoundationDB.Client/FdbNetworkOption.cs | 2 +- FoundationDB.Client/FdbRangeChunk.cs | 2 +- FoundationDB.Client/FdbRangeOptions.cs | 2 +- FoundationDB.Client/FdbStreamingMode.cs | 2 +- FoundationDB.Client/FdbTransactionOption.cs | 2 +- FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs | 2 +- FoundationDB.Client/Filters/Logging/FdbLoggingExtensions.cs | 2 +- FoundationDB.Client/Filters/PrefixRewriterTransaction.cs | 2 +- FoundationDB.Client/Filters/ReadOnlyTransactionFilter.cs | 2 +- FoundationDB.Client/KeyRange.cs | 2 +- FoundationDB.Client/KeySelectorPair.cs | 2 +- .../Layers/Tuples/Formatters/AnonymousTupleFormatter.cs | 2 +- .../Layers/Tuples/Formatters/FormattableTupleFormatter.cs | 2 +- .../Layers/Tuples/Formatters/GenericTupleFormatter.cs | 2 +- .../Layers/Tuples/Formatters/ITupleFormattable.cs | 2 +- FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs | 2 +- FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs | 2 +- FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs | 2 +- .../Linq/Async/Iterators/BatchingAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs | 2 +- .../Linq/Async/Iterators/IntersectAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs | 2 +- .../Linq/Async/Iterators/MergeSortAsyncIterator.cs | 2 +- .../Linq/Async/Iterators/PrefetchingAsyncIterator.cs | 2 +- .../Linq/Async/Iterators/WindowingAsyncIterator.cs | 2 +- FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs | 2 +- FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs | 2 +- FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs | 2 +- FoundationDB.Client/Linq/AsyncIterationHint.cs | 2 +- FoundationDB.Client/Linq/EnumerableExtensions.cs | 2 +- FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs | 2 +- FoundationDB.Client/Linq/QueryStatistics.cs | 2 +- FoundationDB.Client/Native/FdbKeyValue.cs | 2 +- FoundationDB.Client/Properties/AssemblyInfo.cs | 2 +- FoundationDB.Client/Status/FdbSystemStatus.cs | 2 +- FoundationDB.Client/Utils/DebugCounters.cs | 2 +- FoundationDB.Client/Utils/Memory/SliceListStream.cs | 2 +- FoundationDB.Layers.Common/Properties/AssemblyInfo.cs | 2 +- .../Documents/FdbDocumentCollection.cs | 2 +- .../Documents/FdbDocumentHandlers.cs | 2 +- .../Documents/FdbHashSetCollection.cs | 2 +- FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs | 2 +- FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs | 2 +- .../Visitors/FdbQueryExpressionStringBuilder.cs | 2 +- .../Visitors/FdbQueryExpressionVisitor.cs | 2 +- FoundationDB.Samples/Properties/AssemblyInfo.cs | 2 +- FoundationDB.Tests.Sandbox/Properties/AssemblyInfo.cs | 2 +- FoundationDB.Tests/ClusterFacts.cs | 2 +- FoundationDB.Tests/DatabaseBulkFacts.cs | 2 +- FoundationDB.Tests/DatabaseFacts.cs | 2 +- FoundationDB.Tests/Encoders/EncoderFacts.cs | 2 +- FoundationDB.Tests/Encoders/TypeCodecFacts.cs | 2 +- FoundationDB.Tests/ErrorFacts.cs | 2 +- FoundationDB.Tests/ExoticTestCases.cs | 2 +- FoundationDB.Tests/Filters/LoggingFilterFacts.cs | 2 +- FoundationDB.Tests/KeyFacts.cs | 2 +- FoundationDB.Tests/Layers/BlobFacts.cs | 2 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 2 +- FoundationDB.Tests/Layers/DocumentCollectionFacts.cs | 2 +- FoundationDB.Tests/Layers/IndexingFacts.cs | 2 +- FoundationDB.Tests/Layers/QueuesFacts.cs | 2 +- FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs | 2 +- FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs | 2 +- FoundationDB.Tests/Properties/AssemblyInfo.cs | 2 +- FoundationDB.Tests/TestHelpers.cs | 2 +- FoundationDB.Tests/TransactionFacts.cs | 2 +- FoundationDB.Tests/TransactionalFacts.cs | 2 +- FoundationDB.Tests/Utils/ConversionFacts.cs | 2 +- FoundationDB.Tests/Utils/TupleFacts.cs | 2 +- FoundationDB.Tests/Utils/Uuid128Facts.cs | 2 +- FoundationDB.Tests/Utils/Uuid64Facts.cs | 2 +- 84 files changed, 84 insertions(+), 84 deletions(-) diff --git a/FoundationDB.Client/Async/AsyncOrderingMode.cs b/FoundationDB.Client/Async/AsyncOrderingMode.cs index b7106d96c..402ae2b23 100644 --- a/FoundationDB.Client/Async/AsyncOrderingMode.cs +++ b/FoundationDB.Client/Async/AsyncOrderingMode.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncBuffer.cs b/FoundationDB.Client/Async/IAsyncBuffer.cs index 3f304f89e..ea9b89361 100644 --- a/FoundationDB.Client/Async/IAsyncBuffer.cs +++ b/FoundationDB.Client/Async/IAsyncBuffer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncSource.cs b/FoundationDB.Client/Async/IAsyncSource.cs index c4dc1487a..d9ffc7211 100644 --- a/FoundationDB.Client/Async/IAsyncSource.cs +++ b/FoundationDB.Client/Async/IAsyncSource.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Async/IAsyncTarget.cs b/FoundationDB.Client/Async/IAsyncTarget.cs index cfdbb65e6..b185ff360 100644 --- a/FoundationDB.Client/Async/IAsyncTarget.cs +++ b/FoundationDB.Client/Async/IAsyncTarget.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Converters/ITypeConverter.cs b/FoundationDB.Client/Converters/ITypeConverter.cs index c92550c30..fceb06dff 100644 --- a/FoundationDB.Client/Converters/ITypeConverter.cs +++ b/FoundationDB.Client/Converters/ITypeConverter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Converters/SimilarValueComparer.cs b/FoundationDB.Client/Converters/SimilarValueComparer.cs index 32ce5171e..af2ccd86d 100644 --- a/FoundationDB.Client/Converters/SimilarValueComparer.cs +++ b/FoundationDB.Client/Converters/SimilarValueComparer.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbClusterFile.cs b/FoundationDB.Client/FdbClusterFile.cs index e5c69f4d7..7cc33f616 100644 --- a/FoundationDB.Client/FdbClusterFile.cs +++ b/FoundationDB.Client/FdbClusterFile.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbClusterOption.cs b/FoundationDB.Client/FdbClusterOption.cs index 510d1de85..a0f74579b 100644 --- a/FoundationDB.Client/FdbClusterOption.cs +++ b/FoundationDB.Client/FdbClusterOption.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbConflictRangeType.cs b/FoundationDB.Client/FdbConflictRangeType.cs index 7bbd5e94e..4d0a88752 100644 --- a/FoundationDB.Client/FdbConflictRangeType.cs +++ b/FoundationDB.Client/FdbConflictRangeType.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbEndPoint.cs b/FoundationDB.Client/FdbEndPoint.cs index 2b251830e..d5851605a 100644 --- a/FoundationDB.Client/FdbEndPoint.cs +++ b/FoundationDB.Client/FdbEndPoint.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbError.cs b/FoundationDB.Client/FdbError.cs index c702fcca3..0068bbe72 100644 --- a/FoundationDB.Client/FdbError.cs +++ b/FoundationDB.Client/FdbError.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbException.cs b/FoundationDB.Client/FdbException.cs index 3ee39aad5..068ad6f04 100644 --- a/FoundationDB.Client/FdbException.cs +++ b/FoundationDB.Client/FdbException.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbMergeQueryExtensions.cs b/FoundationDB.Client/FdbMergeQueryExtensions.cs index 545d0a85a..88f868f66 100644 --- a/FoundationDB.Client/FdbMergeQueryExtensions.cs +++ b/FoundationDB.Client/FdbMergeQueryExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbNetworkOption.cs b/FoundationDB.Client/FdbNetworkOption.cs index f5c583b12..ec9d094ca 100644 --- a/FoundationDB.Client/FdbNetworkOption.cs +++ b/FoundationDB.Client/FdbNetworkOption.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index e9c7cffd0..b7b2d09bd 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbRangeOptions.cs b/FoundationDB.Client/FdbRangeOptions.cs index c55d65541..94b760318 100644 --- a/FoundationDB.Client/FdbRangeOptions.cs +++ b/FoundationDB.Client/FdbRangeOptions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbStreamingMode.cs b/FoundationDB.Client/FdbStreamingMode.cs index c39716bae..89779b297 100644 --- a/FoundationDB.Client/FdbStreamingMode.cs +++ b/FoundationDB.Client/FdbStreamingMode.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/FdbTransactionOption.cs b/FoundationDB.Client/FdbTransactionOption.cs index 306351beb..98c5726dd 100644 --- a/FoundationDB.Client/FdbTransactionOption.cs +++ b/FoundationDB.Client/FdbTransactionOption.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs index bf413d693..c69c6b9da 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedDatabase.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggingExtensions.cs b/FoundationDB.Client/Filters/Logging/FdbLoggingExtensions.cs index 5dd21942c..984e982c6 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggingExtensions.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggingExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs index d351bfcaf..efa51be8b 100644 --- a/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs +++ b/FoundationDB.Client/Filters/PrefixRewriterTransaction.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Filters/ReadOnlyTransactionFilter.cs b/FoundationDB.Client/Filters/ReadOnlyTransactionFilter.cs index e74321045..e5d374ce0 100644 --- a/FoundationDB.Client/Filters/ReadOnlyTransactionFilter.cs +++ b/FoundationDB.Client/Filters/ReadOnlyTransactionFilter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index f87cebf39..20fd239bb 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/KeySelectorPair.cs b/FoundationDB.Client/KeySelectorPair.cs index 9192abf74..68f881de5 100644 --- a/FoundationDB.Client/KeySelectorPair.cs +++ b/FoundationDB.Client/KeySelectorPair.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs index 618358502..c2f157c31 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs index eeaab9b63..bce76efc2 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs index 5ee1c9396..2b87cd73c 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs index 05eceaaf4..49df60f9b 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs index d02de5f75..6ff3a2257 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs index 15a9094e9..78122f220 100644 --- a/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs +++ b/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs b/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs index 6d040ba0e..4620d6bf0 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs index fb4d54475..37fe20308 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs index 274df25c2..d14c5269b 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs index c558eef4d..117d8da61 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs index d35c9d27c..9dda44927 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs index 264cd6154..dc8b2c031 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs index f8fc010c6..67b6bab2c 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs b/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs index 14844e84c..79024ca42 100644 --- a/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs +++ b/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs index 3d0b6282c..93c543797 100644 --- a/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs index a5d7f162e..85eacc712 100644 --- a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs index ab6a9b6d3..224873260 100644 --- a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs +++ b/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/AsyncIterationHint.cs b/FoundationDB.Client/Linq/AsyncIterationHint.cs index 555269b8f..681cf95f4 100644 --- a/FoundationDB.Client/Linq/AsyncIterationHint.cs +++ b/FoundationDB.Client/Linq/AsyncIterationHint.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/EnumerableExtensions.cs b/FoundationDB.Client/Linq/EnumerableExtensions.cs index 14f12933c..8828d1fd9 100644 --- a/FoundationDB.Client/Linq/EnumerableExtensions.cs +++ b/FoundationDB.Client/Linq/EnumerableExtensions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs b/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs index cd1e10b41..72b093088 100644 --- a/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs +++ b/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Linq/QueryStatistics.cs b/FoundationDB.Client/Linq/QueryStatistics.cs index 1eceba08e..fcf73728f 100644 --- a/FoundationDB.Client/Linq/QueryStatistics.cs +++ b/FoundationDB.Client/Linq/QueryStatistics.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Native/FdbKeyValue.cs b/FoundationDB.Client/Native/FdbKeyValue.cs index 38c997f62..93d343eee 100644 --- a/FoundationDB.Client/Native/FdbKeyValue.cs +++ b/FoundationDB.Client/Native/FdbKeyValue.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Properties/AssemblyInfo.cs b/FoundationDB.Client/Properties/AssemblyInfo.cs index c32816960..c17316435 100644 --- a/FoundationDB.Client/Properties/AssemblyInfo.cs +++ b/FoundationDB.Client/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Status/FdbSystemStatus.cs b/FoundationDB.Client/Status/FdbSystemStatus.cs index 8e7d55967..1e8d89f3e 100644 --- a/FoundationDB.Client/Status/FdbSystemStatus.cs +++ b/FoundationDB.Client/Status/FdbSystemStatus.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/DebugCounters.cs b/FoundationDB.Client/Utils/DebugCounters.cs index 6c0c84091..02c21c7e0 100644 --- a/FoundationDB.Client/Utils/DebugCounters.cs +++ b/FoundationDB.Client/Utils/DebugCounters.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Client/Utils/Memory/SliceListStream.cs b/FoundationDB.Client/Utils/Memory/SliceListStream.cs index 3b84be488..d987163d5 100644 --- a/FoundationDB.Client/Utils/Memory/SliceListStream.cs +++ b/FoundationDB.Client/Utils/Memory/SliceListStream.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs b/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs index e8cd9b9ac..c4485a375 100644 --- a/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs +++ b/FoundationDB.Layers.Common/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index d68320884..14c8d23d9 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs index 979f37d30..b34986d50 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentHandlers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 83c281821..3bf2bf38e 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 2e4feaeb5..5baf44136 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs b/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs index ca7155e57..b01bdea0e 100644 --- a/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs +++ b/FoundationDB.Linq.Providers/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionStringBuilder.cs b/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionStringBuilder.cs index fc73b17a8..25e828c5f 100644 --- a/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionStringBuilder.cs +++ b/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionStringBuilder.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionVisitor.cs b/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionVisitor.cs index dd60884c0..edb168485 100644 --- a/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionVisitor.cs +++ b/FoundationDB.Linq.Providers/Visitors/FdbQueryExpressionVisitor.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Samples/Properties/AssemblyInfo.cs b/FoundationDB.Samples/Properties/AssemblyInfo.cs index e319b87d0..40b3f6b55 100644 --- a/FoundationDB.Samples/Properties/AssemblyInfo.cs +++ b/FoundationDB.Samples/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests.Sandbox/Properties/AssemblyInfo.cs b/FoundationDB.Tests.Sandbox/Properties/AssemblyInfo.cs index 7068ebfd0..44640dc4e 100644 --- a/FoundationDB.Tests.Sandbox/Properties/AssemblyInfo.cs +++ b/FoundationDB.Tests.Sandbox/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/ClusterFacts.cs b/FoundationDB.Tests/ClusterFacts.cs index 2ab26ed07..9d6deb25d 100644 --- a/FoundationDB.Tests/ClusterFacts.cs +++ b/FoundationDB.Tests/ClusterFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index b1534e192..035355505 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index ba9a5c317..aabdf968d 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Encoders/EncoderFacts.cs b/FoundationDB.Tests/Encoders/EncoderFacts.cs index 4604ac9da..5c38a51b3 100644 --- a/FoundationDB.Tests/Encoders/EncoderFacts.cs +++ b/FoundationDB.Tests/Encoders/EncoderFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs index 3374858d0..51f1caf19 100644 --- a/FoundationDB.Tests/Encoders/TypeCodecFacts.cs +++ b/FoundationDB.Tests/Encoders/TypeCodecFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/ErrorFacts.cs b/FoundationDB.Tests/ErrorFacts.cs index 5e00a5472..aa00c1cb2 100644 --- a/FoundationDB.Tests/ErrorFacts.cs +++ b/FoundationDB.Tests/ErrorFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/ExoticTestCases.cs b/FoundationDB.Tests/ExoticTestCases.cs index 2a147c3b5..c44d1fcb1 100644 --- a/FoundationDB.Tests/ExoticTestCases.cs +++ b/FoundationDB.Tests/ExoticTestCases.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs index 600d253b2..719811a51 100644 --- a/FoundationDB.Tests/Filters/LoggingFilterFacts.cs +++ b/FoundationDB.Tests/Filters/LoggingFilterFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/KeyFacts.cs b/FoundationDB.Tests/KeyFacts.cs index 6423991c2..caaf0ccc5 100644 --- a/FoundationDB.Tests/KeyFacts.cs +++ b/FoundationDB.Tests/KeyFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/BlobFacts.cs b/FoundationDB.Tests/Layers/BlobFacts.cs index 4f02a5707..69627f81f 100644 --- a/FoundationDB.Tests/Layers/BlobFacts.cs +++ b/FoundationDB.Tests/Layers/BlobFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 913e7e3ca..06bd8d476 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs b/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs index 692ea391a..ef727febe 100644 --- a/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs +++ b/FoundationDB.Tests/Layers/DocumentCollectionFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/IndexingFacts.cs b/FoundationDB.Tests/Layers/IndexingFacts.cs index 564b6cd9e..9ab392e84 100644 --- a/FoundationDB.Tests/Layers/IndexingFacts.cs +++ b/FoundationDB.Tests/Layers/IndexingFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Layers/QueuesFacts.cs b/FoundationDB.Tests/Layers/QueuesFacts.cs index f7a6dd152..45dd0e579 100644 --- a/FoundationDB.Tests/Layers/QueuesFacts.cs +++ b/FoundationDB.Tests/Layers/QueuesFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs index 273808dff..41aa83fc0 100644 --- a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index 4f52222f8..7c487950a 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Properties/AssemblyInfo.cs b/FoundationDB.Tests/Properties/AssemblyInfo.cs index 7c27100a0..28fd55b2b 100644 --- a/FoundationDB.Tests/Properties/AssemblyInfo.cs +++ b/FoundationDB.Tests/Properties/AssemblyInfo.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/TestHelpers.cs b/FoundationDB.Tests/TestHelpers.cs index b6b7a8abb..72bec8f9d 100644 --- a/FoundationDB.Tests/TestHelpers.cs +++ b/FoundationDB.Tests/TestHelpers.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index efa889908..50dbf1839 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index daa76aeaa..75c2d0063 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/ConversionFacts.cs b/FoundationDB.Tests/Utils/ConversionFacts.cs index 3a660a38e..9d116ebfa 100644 --- a/FoundationDB.Tests/Utils/ConversionFacts.cs +++ b/FoundationDB.Tests/Utils/ConversionFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index e112c66d2..3c63fe6c8 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/Uuid128Facts.cs b/FoundationDB.Tests/Utils/Uuid128Facts.cs index c40a8b153..252b594d1 100644 --- a/FoundationDB.Tests/Utils/Uuid128Facts.cs +++ b/FoundationDB.Tests/Utils/Uuid128Facts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without diff --git a/FoundationDB.Tests/Utils/Uuid64Facts.cs b/FoundationDB.Tests/Utils/Uuid64Facts.cs index d7ae0c242..74244cd96 100644 --- a/FoundationDB.Tests/Utils/Uuid64Facts.cs +++ b/FoundationDB.Tests/Utils/Uuid64Facts.cs @@ -1,5 +1,5 @@ #region BSD Licence -/* Copyright (c) 2013, Doxense SARL +/* Copyright (c) 2013-2018, Doxense SAS All rights reserved. Redistribution and use in source and binary forms, with or without From f5fb25f0cb0089d007065171487e1e12265e3b0a Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Tue, 24 Apr 2018 16:06:32 +0200 Subject: [PATCH 098/153] Optimized Slice.ToHexaString --- .../Utils/Memory/Slice.Encoding.cs | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs b/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs index 14ba1ee15..36dadcb97 100644 --- a/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs +++ b/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs @@ -1553,15 +1553,25 @@ internal static string FormatHexaString(byte[] buffer, int offset, int count, ch var sb = new StringBuilder(count * (sep == '\0' ? 2 : 3)); int letters = lower ? 87 : 55; - while (count-- > 0) + unsafe { - if (sep != '\0' && sb.Length > 0) sb.Append(sep); - byte b = buffer[offset++]; - int x = b >> 4; - sb.Append((char)(x + (x < 10 ? 48 : letters))); - x = b & 0xF; - sb.Append((char)(x + (x < 10 ? 48 : letters))); + fixed (byte* ptr = &buffer[offset]) + { + byte* inp = ptr; + byte* stop = ptr + count; + while (inp < stop) + { + if ((sep != '\0') & (sb.Length > 0)) sb.Append(sep); + byte b = *inp++; + int h = b >> 4; + int l = b & 0xF; + h += h < 10 ? 48 : letters; + l += l < 10 ? 48 : letters; + sb.Append((char) h).Append((char) l); + } + } } + return sb.ToString(); } From f763c7c1bf5283b73627cd946b40d3b802d55e2b Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:05:09 +0200 Subject: [PATCH 099/153] Mark older Tuple byte prefix as obsolete in Tuple Encoding --- FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs index 06cb064c1..9389f8b5f 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs @@ -44,8 +44,12 @@ internal static class TupleTypes /// UTF-8 String internal const byte Utf8 = 2; - /// Nested tuple [DRAFT] + /// Nested tuple start [OBSOLETE] internal const byte TupleStart = 3; + /// Nested tuple end [OBSOLETE] + internal const byte TupleEnd = 4; + + internal const byte TupleStartNew = 5; internal const byte IntNeg8 = 12; internal const byte IntNeg7 = 13; From 06c75aa6a96e4b70955774e0e3abe3215c86232e Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:05:33 +0200 Subject: [PATCH 100/153] Bump max supported API version to 510 so that we can actually select something higher than 300! --- FoundationDB.Client/Native/FdbNative.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index c8d087f36..5560a6e74 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -41,7 +41,7 @@ namespace FoundationDB.Client.Native internal static unsafe class FdbNative { public const int FDB_API_MIN_VERSION = 200; - public const int FDB_API_MAX_VERSION = 300; + public const int FDB_API_MAX_VERSION = 510; #if __MonoCS__ /// Name of the C API dll used for P/Invoking From 0cff2f2841ab200ad70ba51d07269c09094ac865 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:07:18 +0200 Subject: [PATCH 101/153] Build with full pdb symbols (in Debug) so that we can debug and breakpoint from Visual Studio - portable symbols seem to not work well with the current VS debugger? --- FoundationDB.Client/FoundationDB.Client.csproj | 2 ++ FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj | 2 ++ .../FoundationDB.Layers.Experimental.csproj | 2 ++ FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj | 2 ++ 4 files changed, 8 insertions(+) diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 4959b130b..01a1a8731 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -21,6 +21,8 @@ true latest + full + true diff --git a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj index fd8e950aa..58c62feb4 100644 --- a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj +++ b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.csproj @@ -21,6 +21,8 @@ true latest + full + true diff --git a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj index e387400df..d48d35b80 100644 --- a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj +++ b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.csproj @@ -21,6 +21,8 @@ true latest + full + true diff --git a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj index 9ce55f3dd..044e1dce9 100644 --- a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj +++ b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.csproj @@ -21,6 +21,8 @@ true latest + full + true From 2b6737c8a0b8306ea3ca6b684f5613943e9ed796 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:00:41 +0200 Subject: [PATCH 102/153] Add first prototype of VersionStamp struct - Handle both 80-bit and 96-bit sizes - Use internal flag to distinguish between both sizes, and incomplete/complete --- FoundationDB.Client/VersionStamp.cs | 413 ++++++++++++++++++ FoundationDB.Tests/FoundationDB.Tests.csproj | 3 + .../FoundationDB.Tests.csproj.DotSettings | 2 + FoundationDB.Tests/VersionStampFacts.cs | 201 +++++++++ 4 files changed, 619 insertions(+) create mode 100644 FoundationDB.Client/VersionStamp.cs create mode 100644 FoundationDB.Tests/FoundationDB.Tests.csproj.DotSettings create mode 100644 FoundationDB.Tests/VersionStampFacts.cs diff --git a/FoundationDB.Client/VersionStamp.cs b/FoundationDB.Client/VersionStamp.cs new file mode 100644 index 000000000..e9faf5bb1 --- /dev/null +++ b/FoundationDB.Client/VersionStamp.cs @@ -0,0 +1,413 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace FoundationDB.Client +{ + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Runtime.CompilerServices; + using Doxense; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; + + /// VersionStamp + /// A versionstamp is unique, monotonically (but not sequentially) increasing value for each committed transaction. + /// Its size can either be 10 bytes (80-bits) or 12-bytes (96-bits). + /// The first 8 bytes are the committed version of the database. The next 2 bytes are monotonic in the serialization order for transactions. + /// The optional last 2 bytes can contain a user-provider version number used to allow multiple stamps inside the same transaction. + /// + [DebuggerDisplay("{ToString(),nq}")] + public readonly struct VersionStamp : IEquatable, IComparable + { + //REVIEW: they are called "Versionstamp" in the doc, but "VersionStamp" seems more .NETy (like 'TimeSpan'). + // => Should we keep the uppercase 'S' or not ? + + private const ulong PLACEHOLDER_VERSION = ulong.MaxValue; + private const ushort PLACEHOLDER_ORDER = ushort.MaxValue; + private const ushort NO_USER_VERSION = 0; + + private const ushort FLAGS_NONE = 0x0; + private const ushort FLAGS_HAS_VERSION = 0x1; // unset: 80-bits, set: 96-bits + private const ushort FLAGS_IS_INCOMPLETE = 0x2; // unset: complete, set: incomplete + + /// Commit version of the transaction + /// This value is determined by the database at commit time. + + public readonly ulong TransactionVersion; // Bytes 0..7 + + /// Transaction Batch Order + /// This value is determined by the database at commit time. + public readonly ushort TransactionOrder; // Bytes 8..9 + + /// User-provided version (between 0 and 65535) + /// For 80-bits VersionStamps, this value will be 0 and will not be part of the serialized key. You can use to distinguish between both types of stamps. + public readonly ushort UserVersion; // Bytes 10..11 (if 'FLAGS_HAS_VERSION' is set) + + /// Internal flags (FLAGS_xxx constants) + private readonly ushort Flags; + //note: this flag is only present in memory, and is not serialized + + private VersionStamp(ulong version, ushort order, ushort user, ushort flags) + { + this.TransactionVersion = version; + this.TransactionOrder = order; + this.UserVersion = user; + this.Flags = flags; + } + + /// Creates an incomplete 80-bit with no user version. + /// Placeholder that will be serialized as FF FF FF FF FF FF FF FF FF FF (10 bytes). + /// + /// This stamp contains a temporary marker that will be later filled by the database with the actual VersioStamp by the database at transaction commit time. + /// If you need to create multiple distinct stamps within the same transaction, please use instead. + /// + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Incomplete() + { + return new VersionStamp(PLACEHOLDER_VERSION, PLACEHOLDER_ORDER, NO_USER_VERSION, FLAGS_IS_INCOMPLETE); + } + + /// Creates an incomplete 96-bit with the given user version. + /// Value between 0 and 65535 that will be appended at the end of the Versionstamp, making it unique within the transaction. + /// Placeholder that will be serialized as FF FF FF FF FF FF FF FF FF FF vv vv (12 bytes) where 'vv vv' is the user version encoded in little-endian. + /// If is less than 0, or greater than 65534 (0xFFFE). + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Incomplete(int userVersion) + { + Contract.Between(userVersion, 0, 0xFFFF, nameof(userVersion), "Local version must fit in 16-bits."); + return new VersionStamp(PLACEHOLDER_VERSION, PLACEHOLDER_ORDER, (ushort) userVersion, FLAGS_IS_INCOMPLETE | FLAGS_HAS_VERSION); + } + + /// Creates an incomplete 96-bit with the given user version. + /// Value between 0 and 65535 that will be appended at the end of the Versionstamp, making it unique within the transaction. + /// Placeholder that will be serialized as FF FF FF FF FF FF FF FF FF FF vv vv (12 bytes) where 'vv vv' is the user version encoded in little-endian. + /// If is less than 0, or greater than 65534 (0xFFFE). + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Incomplete(ushort userVersion) + { + return new VersionStamp(PLACEHOLDER_VERSION, PLACEHOLDER_ORDER, userVersion, FLAGS_IS_INCOMPLETE | FLAGS_HAS_VERSION); + } + + /// Creates a 80-bit , obtained from the database. + /// Complete stamp, without user version. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Complete(ulong version, ushort order) + { + return new VersionStamp(version, order, NO_USER_VERSION, FLAGS_NONE); + } + + /// Creates a 96-bit , obtained from the database. + /// Complete stamp, with a user version. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Complete(ulong version, ushort order, int userVersion) + { + Contract.Between(userVersion, 0, 0xFFFF, nameof(userVersion), "Local version must fit in 16-bits, and cannot be 0xFFFF."); + return new VersionStamp(version, order, (ushort) userVersion, FLAGS_HAS_VERSION); + } + + /// Creates a 96-bit , obtained from the database. + /// Complete stamp, with a user version. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static VersionStamp Complete(ulong version, ushort order, ushort userVersion) + { + return new VersionStamp(version, order, userVersion, FLAGS_HAS_VERSION); + } + + /// Test if the stamp has a user version (96-bits) or not (80-bits) + public bool HasUserVersion + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => (this.Flags & FLAGS_HAS_VERSION) != 0; + } + + /// Test if the stamp is marked as incomplete (true), or has already been resolved by the database (false) + public bool IsIncomplete + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => (this.Flags & FLAGS_IS_INCOMPLETE) != 0; + } + + /// Return the length (in bytes) of the versionstamp when serialized in binary format + /// Returns 12 bytes for stamps with a user version, and 10 bytes without. + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public int GetLength() => 10 + 2 * (this.Flags & FLAGS_HAS_VERSION); + + public override string ToString() + { + if (this.HasUserVersion) + { + return this.IsIncomplete + ? $"@?#{this.UserVersion}" + : $"@{this.TransactionVersion}-{this.TransactionOrder}#{this.UserVersion}"; + } + else + { + return this.IsIncomplete + ? "@?" + : $"@{this.TransactionVersion}-{this.TransactionOrder}"; + } + } + + public Slice ToSlice() + { + int len = GetLength(); // 10 or 12 + var tmp = Slice.Create(len); + unsafe + { + fixed (byte* ptr = &tmp.DangerousGetPinnableReference()) + { + WriteUnsafe(ptr, len, in this); + } + } + return tmp; + } + + public void WriteTo(in Slice buffer) + { + int len = GetLength(); // 10 or 12 + if (buffer.Count < len) throw new ArgumentException($"The target buffer must be at least {len} bytes long."); + unsafe + { + fixed (byte* ptr = &buffer.DangerousGetPinnableReference()) + { + WriteUnsafe(ptr, len, in this); + } + } + } + + public void WriteTo(ref SliceWriter writer) + { + var tmp = writer.Allocate(GetLength()); + unsafe + { + fixed (byte* ptr = &tmp.DangerousGetPinnableReference()) + { + WriteUnsafe(ptr, tmp.Count, in this); + } + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static unsafe void WriteUnsafe(byte* ptr, int len, in VersionStamp vs) + { + Contract.Debug.Assert(len == 10 || len == 12); + UnsafeHelpers.StoreUInt64BE(ptr, vs.TransactionVersion); + UnsafeHelpers.StoreUInt16BE(ptr + 8, vs.TransactionOrder); + if (len == 12) + { + UnsafeHelpers.StoreUInt16BE(ptr + 10, vs.UserVersion); + } + } + + /// Parse a VersionStamp from a sequence of 10 bytes + /// If the buffer length is not exactly 12 bytes + [Pure] + public static VersionStamp Parse(Slice data) + { + return TryParse(data, out var vs) ? vs : throw new FormatException("A VersionStamp is either 10 or 12 bytes."); + } + + /// Try parsing a VersionStamp from a sequence of bytes + public static bool TryParse(Slice data, out VersionStamp vs) + { + if (data.Count != 10 && data.Count != 12) + { + vs = default; + return false; + } + unsafe + { + fixed (byte* ptr = &data.DangerousGetPinnableReference()) + { + ReadUnsafe(ptr, data.Count, FLAGS_NONE, out vs); + return true; + } + } + } + + /// Parse a VersionStamp from a sequence of 10 bytes + /// If the buffer length is not exactly 12 bytes + [Pure] + public static VersionStamp ParseIncomplete(Slice data) + { + return TryParseIncomplete(data, out var vs) ? vs : throw new FormatException("A VersionStamp is either 10 or 12 bytes."); + } + + /// Try parsing a VersionStamp from a sequence of bytes + public static bool TryParseIncomplete(Slice data, out VersionStamp vs) + { + if (data.Count != 10 && data.Count != 12) + { + vs = default; + return false; + } + unsafe + { + fixed (byte* ptr = &data.DangerousGetPinnableReference()) + { + ReadUnsafe(ptr, data.Count, FLAGS_IS_INCOMPLETE, out vs); + return true; + } + } + } + + internal static unsafe void ReadUnsafe(byte* ptr, int len, ushort flags, out VersionStamp vs) + { + Contract.Debug.Assert(len == 10 || len == 12); + // reads a complete 12 bytes Versionstamp + ulong ver = UnsafeHelpers.LoadUInt64BE(ptr); + ushort order = UnsafeHelpers.LoadUInt16BE(ptr + 8); + ushort idx = len == 10 ? NO_USER_VERSION : UnsafeHelpers.LoadUInt16BE(ptr + 10); + flags |= len == 12 ? FLAGS_HAS_VERSION : FLAGS_NONE; + vs = new VersionStamp(ver, order, idx, flags); + } + + #region Equality, Comparision, ... + + public override bool Equals(object obj) + { + return obj is VersionStamp vs && Equals(vs); + } + + public override int GetHashCode() + { + return HashCodes.Combine(this.TransactionVersion.GetHashCode(), this.TransactionOrder, this.UserVersion, this.Flags); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(VersionStamp other) + { + //PERF: could we use Unsafe and compare the next sizeof(VersionStamp) bytes at once? + return (this.TransactionVersion == other.TransactionVersion) + & (this.TransactionOrder == other.TransactionOrder) + & (this.UserVersion == other.UserVersion) + & (this.Flags == other.Flags); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator ==(VersionStamp left, VersionStamp right) + { + return left.Equals(right); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator !=(VersionStamp left, VersionStamp right) + { + return !left.Equals(right); + } + + [Pure] + public int CompareTo(VersionStamp other) + { + //ordering rules: + // - incomplete stamps are stored AFTER resolved stamps (since if they commit they would have a value higher than any other stamp already in the database) + // - ordered by transaction number then transaction batch order + // - stamps with no user version are sorted before stamps with user version if they have the same first 10 bytes, so (XXXX) is before (XXXX, 0) + + if (this.IsIncomplete) + { // we ignore the transaction version/order! + if (!other.IsIncomplete) return +1; // we are after + } + else + { + if (other.IsIncomplete) return -1; // we are before + int cmp = this.TransactionVersion.CompareTo(other.TransactionVersion); + if (cmp != 0) return cmp; + } + + // both have same version+order, or both are incomplete + // => we need to decide on the (optional) user version + return this.HasUserVersion + ? (other.HasUserVersion ? this.UserVersion.CompareTo(other.UserVersion) : +1) + : (other.HasUserVersion ? -1 : 0); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator <(VersionStamp left, VersionStamp right) + { + return left.CompareTo(right) < 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator <=(VersionStamp left, VersionStamp right) + { + return left.CompareTo(right) <= 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator >(VersionStamp left, VersionStamp right) + { + return left.CompareTo(right) > 0; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static bool operator >=(VersionStamp left, VersionStamp right) + { + return left.CompareTo(right) >= 0; + } + + //REVIEW: does these make sense or not? + // VersionStamp - VersionStamp == ??? + // VersionStamp + 123 == ??? + // VersionStamp * 2 == ??? + + public sealed class Comparer : IEqualityComparer, IComparer + { + /// Default comparer for s + public static Comparer Default { get; } = new Comparer(); + + private Comparer() + { } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public bool Equals(VersionStamp x, VersionStamp y) + { + return x.Equals(y); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public int GetHashCode(VersionStamp obj) + { + return obj.GetHashCode(); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public int Compare(VersionStamp x, VersionStamp y) + { + return x.CompareTo(y); + } + + } + + #endregion + + } + +} diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index 0c78ba023..a67e669da 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -30,6 +30,7 @@ 105,108,109,114,472,660,661,628,1066 AnyCPU true + latest pdbonly @@ -42,6 +43,7 @@ 105,108,109,114,472,660,661,628,1066 AnyCPU true + latest true @@ -122,6 +124,7 @@ + diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj.DotSettings b/FoundationDB.Tests/FoundationDB.Tests.csproj.DotSettings new file mode 100644 index 000000000..96331d1ce --- /dev/null +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj.DotSettings @@ -0,0 +1,2 @@ + + CSharp72 \ No newline at end of file diff --git a/FoundationDB.Tests/VersionStampFacts.cs b/FoundationDB.Tests/VersionStampFacts.cs new file mode 100644 index 000000000..6075d688d --- /dev/null +++ b/FoundationDB.Tests/VersionStampFacts.cs @@ -0,0 +1,201 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace FoundationDB.Client.Tests +{ + using System; + using Doxense.Memory; + using NUnit.Framework; + + [TestFixture] + public class VersionStampFacts : FdbTest + { + + [Test] + public void Test_Incomplete_VersionStamp() + { + { // 80-bits (no user version) + var vs = VersionStamp.Incomplete(); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); + Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); + Assert.That(vs.IsIncomplete, Is.True); + Assert.That(vs.HasUserVersion, Is.False, "80-bits VersionStamps don't have a user version"); + Assert.That(vs.UserVersion, Is.Zero, "80-bits VersionStamps don't have a user version"); + + Assert.That(vs.GetLength(), Is.EqualTo(10)); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("FF FF FF FF FF FF FF FF FF FF")); + Assert.That(vs.ToString(), Is.EqualTo("@?")); + } + + { // 96-bits, default user version + var vs = VersionStamp.Incomplete(0); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); + Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); + Assert.That(vs.IsIncomplete, Is.True); + Assert.That(vs.HasUserVersion, Is.True, "96-bits VersionStamps have a user version"); + Assert.That(vs.UserVersion, Is.EqualTo(0)); + + Assert.That(vs.GetLength(), Is.EqualTo(12)); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("FF FF FF FF FF FF FF FF FF FF 00 00")); + Assert.That(vs.ToString(), Is.EqualTo("@?#0")); + } + + { // 96 bits, custom user version + var vs = VersionStamp.Incomplete(123); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); + Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); + Assert.That(vs.HasUserVersion, Is.True); + Assert.That(vs.UserVersion, Is.EqualTo(123)); + Assert.That(vs.IsIncomplete, Is.True); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("FF FF FF FF FF FF FF FF FF FF 00 7B")); + Assert.That(vs.ToString(), Is.EqualTo("@?#123")); + } + + { // 96 bits, large user version + var vs = VersionStamp.Incomplete(12345); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); + Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); + Assert.That(vs.HasUserVersion, Is.True); + Assert.That(vs.UserVersion, Is.EqualTo(12345)); + Assert.That(vs.IsIncomplete, Is.True); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("FF FF FF FF FF FF FF FF FF FF 30 39")); + Assert.That(vs.ToString(), Is.EqualTo("@?#12345")); + } + + Assert.That(() => VersionStamp.Incomplete(-1), Throws.ArgumentException, "User version cannot be negative"); + Assert.That(() => VersionStamp.Incomplete(65536), Throws.ArgumentException, "User version cannot be larger than 0xFFFF"); + + { + var writer = default(SliceWriter); + writer.WriteFixed24BE(0xAAAAAA); + VersionStamp.Incomplete(123).WriteTo(ref writer); + writer.WriteFixed24BE(0xAAAAAA); + Assert.That(writer.ToSlice().ToHexaString(' '), Is.EqualTo("AA AA AA FF FF FF FF FF FF FF FF FF FF 00 7B AA AA AA")); + + var reader = new SliceReader(writer.ToSlice()); + reader.Skip(3); + var vs = VersionStamp.Parse(reader.ReadBytes(12)); + Assert.That(reader.Remaining, Is.EqualTo(3)); + + Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); + Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); + Assert.That(vs.UserVersion, Is.EqualTo(123)); + Assert.That(vs.IsIncomplete, Is.False, "NOTE: reading stamps is only supposed to happen for stamps already in the database!"); + } + + { + var buf = Slice.Repeat(0xAA, 18); + VersionStamp.Incomplete(123).WriteTo(buf.Substring(3, 12)); + Assert.That(buf.ToHexaString(' '), Is.EqualTo("AA AA AA FF FF FF FF FF FF FF FF FF FF 00 7B AA AA AA")); + } + } + + [Test] + public void Test_Complete_VersionStamp() + { + { // 80-bits, no user version + var vs = VersionStamp.Complete(0x0123456789ABCDEFUL, 123); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(0x0123456789ABCDEFUL)); + Assert.That(vs.TransactionOrder, Is.EqualTo(123)); + Assert.That(vs.HasUserVersion, Is.False); + Assert.That(vs.UserVersion, Is.Zero); + Assert.That(vs.IsIncomplete, Is.False); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF 00 7B")); + Assert.That(vs.ToString(), Is.EqualTo("@81985529216486895-123")); + } + + { // 96 bits, default user version + var vs = VersionStamp.Complete(0x0123456789ABCDEFUL, 123, 0); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(0x0123456789ABCDEFUL)); + Assert.That(vs.TransactionOrder, Is.EqualTo(123)); + Assert.That(vs.HasUserVersion, Is.True); + Assert.That(vs.UserVersion, Is.Zero); + Assert.That(vs.IsIncomplete, Is.False); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF 00 7B 00 00")); + Assert.That(vs.ToString(), Is.EqualTo("@81985529216486895-123#0")); + } + + { // custom user version + var vs = VersionStamp.Complete(0x0123456789ABCDEFUL, 123, 456); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(0x0123456789ABCDEFUL)); + Assert.That(vs.TransactionOrder, Is.EqualTo(123)); + Assert.That(vs.HasUserVersion, Is.True); + Assert.That(vs.UserVersion, Is.EqualTo(456)); + Assert.That(vs.IsIncomplete, Is.False); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF 00 7B 01 C8")); + Assert.That(vs.ToString(), Is.EqualTo("@81985529216486895-123#456")); + } + + { // two bytes user version + var vs = VersionStamp.Complete(0x0123456789ABCDEFUL, 12345, 6789); + Log(vs); + Assert.That(vs.TransactionVersion, Is.EqualTo(0x0123456789ABCDEFUL)); + Assert.That(vs.TransactionOrder, Is.EqualTo(12345)); + Assert.That(vs.UserVersion, Is.EqualTo(6789)); + Assert.That(vs.IsIncomplete, Is.False); + Assert.That(vs.ToSlice().ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF 30 39 1A 85")); + Assert.That(vs.ToString(), Is.EqualTo("@81985529216486895-12345#6789")); + } + + Assert.That(() => VersionStamp.Complete(0x0123456789ABCDEFUL, 0, -1), Throws.ArgumentException, "User version cannot be negative"); + Assert.That(() => VersionStamp.Complete(0x0123456789ABCDEFUL, 0, 65536), Throws.ArgumentException, "User version cannot be larger than 0xFFFF"); + + { + var writer = default(SliceWriter); + writer.WriteFixed24BE(0xAAAAAA); + VersionStamp.Complete(0x0123456789ABCDEFUL, 123, 456).WriteTo(ref writer); + writer.WriteFixed24BE(0xAAAAAA); + Assert.That(writer.ToSlice().ToHexaString(' '), Is.EqualTo("AA AA AA 01 23 45 67 89 AB CD EF 00 7B 01 C8 AA AA AA")); + + var reader = new SliceReader(writer.ToSlice()); + reader.Skip(3); + var vs = VersionStamp.Parse(reader.ReadBytes(12)); + Assert.That(reader.Remaining, Is.EqualTo(3)); + + Assert.That(vs.TransactionVersion, Is.EqualTo(0x0123456789ABCDEFUL)); + Assert.That(vs.TransactionOrder, Is.EqualTo(123)); + Assert.That(vs.UserVersion, Is.EqualTo(456)); + Assert.That(vs.IsIncomplete, Is.False); + } + + { + var buf = Slice.Repeat(0xAA, 18); + VersionStamp.Complete(0x0123456789ABCDEFUL, 123, 456).WriteTo(buf.Substring(3, 12)); + Assert.That(buf.ToHexaString(' '), Is.EqualTo("AA AA AA 01 23 45 67 89 AB CD EF 00 7B 01 C8 AA AA AA")); + } + } + + } +} From 8192bb78362462ad730a66e29168962f3f9fef32 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:02:11 +0200 Subject: [PATCH 103/153] Add bascic support for VersionStamps to Tuple Encoder - Support both 80-bits and 96-bits variants - BUGBUG: cannot recognized complete/incomplete stamps yet when parsing. --- .../Layers/Tuples/Encoding/TuplePackers.cs | 25 ++++++++++++++ .../Layers/Tuples/Encoding/TupleParser.cs | 33 +++++++++++++++++++ .../Layers/Tuples/Encoding/TupleTypes.cs | 9 +++++ 3 files changed, 67 insertions(+) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index a8e8cc9b6..2e3e17b42 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -40,6 +40,7 @@ namespace Doxense.Collections.Tuples.Encoding using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Runtime.Converters; + using FoundationDB.Client; using JetBrains.Annotations; /// Helper methods used during serialization of values to the tuple binary format @@ -553,6 +554,11 @@ public static void SerializeTo(ref TupleWriter writer, Uuid64 value) TupleParser.WriteUuid64(ref writer, value); } + public static void SerializeTo(ref TupleWriter writer, VersionStamp value) + { + TupleParser.WriteVersionStamp(ref writer, value); + } + /// Writes an IPaddress as a 32-bit (IPv4) or 128-bit (IPv6) byte array public static void SerializeTo(ref TupleWriter writer, System.Net.IPAddress value) { @@ -680,6 +686,7 @@ private static Dictionary InitializeDefaultUnpackers() [typeof(TimeSpan)] = new Func(TuplePackers.DeserializeTimeSpan), [typeof(DateTime)] = new Func(TuplePackers.DeserializeDateTime), [typeof(System.Net.IPAddress)] = new Func(TuplePackers.DeserializeIPAddress), + [typeof(VersionStamp)] = new Func(TuplePackers.DeserializeVersionStamp), [typeof(ITuple)] = new Func(TuplePackers.DeserializeTuple), }; @@ -1694,6 +1701,24 @@ public static Uuid64 DeserializeUuid64(Slice slice) throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into an Uuid64"); } + public static VersionStamp DeserializeVersionStamp(Slice slice) + { + if (slice.IsNullOrEmpty) return default(VersionStamp); + + int type = slice[0]; + + if (type == TupleTypes.VersionStamp80 || type == TupleTypes.VersionStamp96) + { + if (VersionStamp.TryParse(slice.Substring(1), out var stamp)) + { + return stamp; + } + throw new FormatException("Cannot convert malformed tuple segment into a VersionStamp"); + } + + throw new FormatException($"Cannot convert tuple segment of type 0x{type:X} into a VersionStamp"); + } + /// Deserialize a tuple segment into Guid /// Slice that contains a single packed element [CanBeNull] diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs index 503dd12da..66957f676 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs @@ -34,6 +34,7 @@ namespace Doxense.Collections.Tuples.Encoding using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using FoundationDB.Client; using JetBrains.Annotations; /// Helper class that contains low-level encoders for the tuple binary format @@ -758,6 +759,28 @@ public static void WriteUuid64(ref TupleWriter writer, Uuid64? value) if (!value.HasValue) WriteNil(ref writer); else WriteUuid64(ref writer, value.Value); } + public static void WriteVersionStamp(ref TupleWriter writer, VersionStamp value) + { + if (value.HasUserVersion) + { // 96-bits Versionstamp + writer.Output.EnsureBytes(13); + writer.Output.UnsafeWriteByte(TupleTypes.VersionStamp96); + value.WriteTo(writer.Output.Allocate(12)); + } + else + { // 80-bits Versionstamp + writer.Output.EnsureBytes(11); + writer.Output.UnsafeWriteByte(TupleTypes.VersionStamp80); + value.WriteTo(writer.Output.Allocate(10)); + } + } + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public static void WriteVersionStamp(ref TupleWriter writer, VersionStamp? value) + { + if (!value.HasValue) WriteNil(ref writer); else WriteVersionStamp(ref writer, value.Value); + } + /// Mark the start of a new embedded tuple public static void BeginTuple(ref TupleWriter writer) { @@ -1096,6 +1119,16 @@ public static Slice ParseNext(ref TupleReader reader) return reader.Input.ReadBytes(9); } + case TupleTypes.VersionStamp80: + { // <32>(10 bytes) + return reader.Input.ReadBytes(11); + } + + case TupleTypes.VersionStamp96: + { // <33>(12 bytes) + return reader.Input.ReadBytes(13); + } + case TupleTypes.AliasDirectory: case TupleTypes.AliasSystem: { // or diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs index 9389f8b5f..99c736761 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs @@ -86,6 +86,11 @@ internal static class TupleTypes /// UUID (64 bits) [DRAFT] internal const byte Uuid64 = 49; //TODO: this is not official yet! may change! + //TODO: xmldoc + internal const byte VersionStamp80 = 0x32; + //TODO: xmldoc + internal const byte VersionStamp96 = 0x33; + /// Standard prefix of the Directory Layer /// This is not a part of the tuple encoding itself, but helps the tuple decoder pretty-print tuples that would otherwise be unparsable. internal const byte AliasDirectory = 254; @@ -112,6 +117,8 @@ public static TupleSegmentType DecodeSegmentType(Slice segment) case Decimal: return TupleSegmentType.Decimal; case Uuid128: return TupleSegmentType.Uuid128; case Uuid64: return TupleSegmentType.Uuid64; + case VersionStamp80: return TupleSegmentType.VersionStamp80; + case VersionStamp96: return TupleSegmentType.VersionStamp96; } if (type <= IntPos8 && type >= IntNeg8) @@ -138,6 +145,8 @@ public enum TupleSegmentType Decimal = 35, Uuid128 = 48, Uuid64 = 49, + VersionStamp80 = 0x32, + VersionStamp96 = 0x33, } } From 2ccce084144af3a8fed9380b31b379c1f4748cd2 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:03:43 +0200 Subject: [PATCH 104/153] Add initial support for fdb_transaction_get_versionstamp and add SetStampedKey() / SetStampedValue() mutations --- .../Core/IFdbTransactionHandler.cs | 3 ++ FoundationDB.Client/FdbMutationType.cs | 8 +++- FoundationDB.Client/FdbTransaction.cs | 30 ++++++++++++++ .../FdbTransactionExtensions.cs | 16 ++++++++ .../Filters/FdbTransactionFilter.cs | 6 +++ .../Filters/Logging/FdbLoggedTransaction.cs | 8 ++++ .../Logging/FdbTransactionLog.Commands.cs | 6 +++ .../Filters/Logging/FdbTransactionLog.cs | 1 + FoundationDB.Client/IFdbTransaction.cs | 8 ++++ FoundationDB.Client/Native/FdbNative.cs | 35 +++++++++++++++- .../Native/FdbNativeTransaction.cs | 19 +++++++++ FoundationDB.Tests/TransactionFacts.cs | 41 +++++++++++++++++++ 12 files changed, 179 insertions(+), 2 deletions(-) diff --git a/FoundationDB.Client/Core/IFdbTransactionHandler.cs b/FoundationDB.Client/Core/IFdbTransactionHandler.cs index 4c59c4bc0..32255efe3 100644 --- a/FoundationDB.Client/Core/IFdbTransactionHandler.cs +++ b/FoundationDB.Client/Core/IFdbTransactionHandler.cs @@ -61,6 +61,9 @@ public interface IFdbTransactionHandler : IDisposable /// long GetCommittedVersion(); + /// Returns the which was used by versionstamps operations in this transaction. + Task GetVersionStampAsync(CancellationToken ct); + /// Sets the snapshot read version used by a transaction. This is not needed in simple cases. /// Read version to use in this transaction /// diff --git a/FoundationDB.Client/FdbMutationType.cs b/FoundationDB.Client/FdbMutationType.cs index 4899b5a61..e4a4492f2 100644 --- a/FoundationDB.Client/FdbMutationType.cs +++ b/FoundationDB.Client/FdbMutationType.cs @@ -91,7 +91,13 @@ public enum FdbMutationType /// If ``param`` is shorter than the existing value in the database, the existing value is truncated to match the length of ``param``. /// The smaller of the two values is then stored in the database. /// - Min = 13 + Min = 13, + + //TODO: XML Comments! + VersionStampedKey = 14, + + //TODO: XML Comments! + VersionStampedValue = 15, } diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 25fc38b93..ac9e45120 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -284,6 +284,19 @@ public void SetReadVersion(long version) m_handler.SetReadVersion(version); } + /// Returns the which was used by versionstamps operations in this transaction. + /// + /// The Task will be ready only after the successful completion of a call to on this transaction. + /// Read-only transactions do not modify the database when committed and will result in the Task completing with an error. + /// Keep in mind that a transaction which reads keys and then sets them to their current values may be optimized to a read-only transaction. + /// + public Task GetVersionStampAsync() + { + EnsureNotFailedOrDisposed(); + + return m_handler.GetVersionStampAsync(m_cancellation); + } + #endregion #region Get... @@ -513,6 +526,23 @@ private static void EnsureMutationTypeIsSupported(FdbMutationType mutation, int return; } + if (mutation == FdbMutationType.VersionStampedKey || mutation == FdbMutationType.VersionStampedValue) + { + if (selectedApiVersion < 400) + { + if (Fdb.GetMaxApiVersion() >= 400) + { + throw new FdbException(FdbError.InvalidMutationType, "Atomic mutations for VersionStamps are only supported starting from API level 400. You need to select API level 400 or more at the start of your process."); + } + else + { + throw new FdbException(FdbError.InvalidMutationType, "Atomic mutations Max and Min are only supported starting from client version 4.x. You need to update the version of the client, and select API level 400 or more at the start of your process.."); + } + } + // ok! + return; + } + // this could be a new mutation type, or an invalid value. throw new FdbException(FdbError.InvalidMutationType, "An invalid mutation type was issued. If you are attempting to call a new mutation type, you will need to update the version of this assembly, and select the latest API level."); } diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index 16d4d362c..d4f2e8992 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -425,6 +425,22 @@ public static void AtomicMin([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, value, FdbMutationType.Min); } + //TODO: XML Comments! + public static void SetVersionStampedKey([NotNull] this IFdbTransaction trans, Slice key, Slice value) + { + Contract.NotNull(trans, nameof(trans)); + + trans.Atomic(key, value, FdbMutationType.VersionStampedKey); + } + + //TODO: XML Comments! + public static void SetVersionStampedValue([NotNull] this IFdbTransaction trans, Slice key, Slice value) + { + Contract.NotNull(trans, nameof(trans)); + + trans.Atomic(key, value, FdbMutationType.VersionStampedValue); + } + #endregion #region GetRange... diff --git a/FoundationDB.Client/Filters/FdbTransactionFilter.cs b/FoundationDB.Client/Filters/FdbTransactionFilter.cs index 688f3e6e9..78d414e68 100644 --- a/FoundationDB.Client/Filters/FdbTransactionFilter.cs +++ b/FoundationDB.Client/Filters/FdbTransactionFilter.cs @@ -251,6 +251,12 @@ public virtual long GetCommittedVersion() return m_transaction.GetCommittedVersion(); } + public virtual Task GetVersionStampAsync() + { + ThrowIfDisposed(); + return m_transaction.GetVersionStampAsync(); + } + public virtual void SetReadVersion(long version) { ThrowIfDisposed(); diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index 58de2beab..23b12cbce 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -312,6 +312,14 @@ public override Task OnErrorAsync(FdbError code) ); } + public override Task GetVersionStampAsync() + { + return ExecuteAsync( + new FdbTransactionLog.GetVersionStampCommand(), + (tr, cmd) => tr.GetVersionStampAsync() + ); + } + public override void Set(Slice key, Slice value) { Execute( diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs index 6439cb75e..8cb6093e8 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.Commands.cs @@ -832,6 +832,12 @@ public override string GetResult(KeyResolver resolver) } + public sealed class GetVersionStampCommand : Command + { + public override Operation Op { get { return Operation.GetVersionStamp; } } + + } + public sealed class GetReadVersionCommand : Command { public override Operation Op { get { return Operation.GetReadVersion; } } diff --git a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs index c6c166072..541187e22 100644 --- a/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs +++ b/FoundationDB.Client/Filters/Logging/FdbTransactionLog.cs @@ -552,6 +552,7 @@ public enum Operation Reset, OnError, SetOption, + GetVersionStamp, Log, } diff --git a/FoundationDB.Client/IFdbTransaction.cs b/FoundationDB.Client/IFdbTransaction.cs index 3d6c4ccda..39015be4c 100644 --- a/FoundationDB.Client/IFdbTransaction.cs +++ b/FoundationDB.Client/IFdbTransaction.cs @@ -112,6 +112,14 @@ public interface IFdbTransaction : IFdbReadOnlyTransaction /// long GetCommittedVersion(); + /// Returns the which was used by versionstamps operations in this transaction. + /// + /// The Task will be ready only after the successful completion of a call to on this transaction. + /// Read-only transactions do not modify the database when committed and will result in the Task completing with an error. + /// Keep in mind that a transaction which reads keys and then sets them to their current values may be optimized to a read-only transaction. + /// + Task GetVersionStampAsync(); + /// /// Watch a key for any change in the database. /// diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 5560a6e74..8af1fa1d5 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -51,7 +51,6 @@ internal static unsafe class FdbNative private const string FDB_C_DLL = "fdb_c.dll"; #endif - /// Handle on the native FDB C API library private static readonly UnmanagedLibrary FdbCLib; @@ -169,6 +168,9 @@ public static extern void fdb_transaction_clear_range( [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_get_versionstamp(TransactionHandle transaction); + [DllImport(FDB_C_DLL, CallingConvention = CallingConvention.Cdecl)] public static extern FutureHandle fdb_transaction_watch(TransactionHandle transaction, byte* keyName, int keyNameLength); @@ -531,6 +533,16 @@ public static FutureHandle TransactionCommit(TransactionHandle transaction) return future; } + public static FutureHandle TransactionGetVersionStamp(TransactionHandle transaction) + { + var future = NativeMethods.fdb_transaction_get_versionstamp(transaction); + Contract.Assert(future != null); +#if DEBUG_NATIVE_CALLS + Debug.WriteLine("fdb_transaction_get_versionstamp(0x" + transaction.Handle.ToString("x") + ") => 0x" + future.Handle.ToString("x")); +#endif + return future; + } + public static FutureHandle TransactionWatch(TransactionHandle transaction, Slice key) { if (key.IsNullOrEmpty) throw new ArgumentException("Key cannot be null or empty", "key"); @@ -822,6 +834,27 @@ public static FdbError FutureGetStringArray(FutureHandle future, out string[] re return err; } + public static FdbError FutureGetVersionStamp(FutureHandle future, out VersionStamp stamp) + { + 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); +#endif + + if (keyLength != 10 || ptr == null) + { + stamp = default; + return err; + } + //note: we assume that this is a complete stamp read from the database. + //BUGBUG: if the code serialize an incomplete stamp into a tuple, and unpacks it (logging?) it MAY be changed into a complete one! + // => we could check for the 'all FF' signature, but this only works for default incomplete tokens, not custom incomplete tokens ! + VersionStamp.ReadUnsafe(ptr, 10, /*FLAGS_NONE*/0, out stamp); + return err; + } + public static void TransactionSet(TransactionHandle transaction, Slice key, Slice value) { fixed (byte* pKey = key.Array) diff --git a/FoundationDB.Client/Native/FdbNativeTransaction.cs b/FoundationDB.Client/Native/FdbNativeTransaction.cs index dabb06068..426348100 100644 --- a/FoundationDB.Client/Native/FdbNativeTransaction.cs +++ b/FoundationDB.Client/Native/FdbNativeTransaction.cs @@ -393,6 +393,25 @@ public long GetCommittedVersion() return version; } + public Task GetVersionStampAsync(CancellationToken ct) + { + var future = FdbNative.TransactionGetVersionStamp(m_handle); + return FdbFuture.CreateTaskFromHandle(future, GetVersionStampResult, ct); + } + + private static VersionStamp GetVersionStampResult(FutureHandle h) + { + Contract.Requires(h != null); + var err = FdbNative.FutureGetVersionStamp(h, out VersionStamp stamp); +#if DEBUG_TRANSACTIONS + Debug.WriteLine("FdbTransaction[" + m_id + "].FutureGetVersionStamp() => err=" + err + ", vs=" + stamp + ")"); +#endif + Fdb.DieOnError(err); + + return stamp; + } + + /// /// Attempts to commit the sets and clears previously applied to the database snapshot represented by this transaction to the actual database. /// 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. diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 50dbf1839..2b8536ec2 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client.Tests using System.Text; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; [TestFixture] public class TransactionFacts : FdbTest @@ -1989,6 +1990,46 @@ public async Task Test_Simple_Read_Transaction() } } + [Test] + public async Task Test_VersionStamp_Operations() + { + Fdb.Start(510); + using (var db = await OpenTestDatabaseAsync()) + { + Log("API Version: " + Fdb.ApiVersion); + + var location = db.Partition.ByKey("versionstamps"); + + await db.ClearRangeAsync(location, this.Cancellation); + + using (var tr = db.BeginTransaction(this.Cancellation)) + { + Slice HACKHACK_Packify(VersionStamp stamp) + { + var x = location.Keys.Encode(stamp); + x = x.Concat(Slice.FromFixed16((short) (location.GetPrefix().Count + 1))); + Log(x.ToHexaString(' ') + " | " + location.Keys.Dump(x)); + return x; + } + + tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete()), Slice.FromString("Hello, World!")); + tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(0)), Slice.FromString("Zero")); + tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(1)), Slice.FromString("One")); + tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(2)), Slice.FromString("Two")); + + var vsTask = tr.GetVersionStampAsync(); + + await tr.CommitAsync(); + Log(tr.GetCommittedVersion()); + + var vs = await vsTask; + Log(vs); + } + + await DumpSubspace(db, location); + } + } + [Test, Category("LongRunning")] public async Task Test_BadPractice_Future_Fuzzer() { From 1cb68193e5824023416655a1cc62a313c2088443 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 14:04:10 +0200 Subject: [PATCH 105/153] Re-add unit test on the Tuple Encoding that were dropped before + add tests for VersionStamps --- FoundationDB.Tests/FoundationDB.Tests.csproj | 1 + FoundationDB.Tests/Utils/TuPackFacts.cs | 2210 ++++++++++++++++++ 2 files changed, 2211 insertions(+) create mode 100644 FoundationDB.Tests/Utils/TuPackFacts.cs diff --git a/FoundationDB.Tests/FoundationDB.Tests.csproj b/FoundationDB.Tests/FoundationDB.Tests.csproj index a67e669da..ae776350b 100644 --- a/FoundationDB.Tests/FoundationDB.Tests.csproj +++ b/FoundationDB.Tests/FoundationDB.Tests.csproj @@ -95,6 +95,7 @@ + diff --git a/FoundationDB.Tests/Utils/TuPackFacts.cs b/FoundationDB.Tests/Utils/TuPackFacts.cs new file mode 100644 index 000000000..664b40b47 --- /dev/null +++ b/FoundationDB.Tests/Utils/TuPackFacts.cs @@ -0,0 +1,2210 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +//#define ENABLE_VALUETUPLE + +// ReSharper disable AccessToModifiedClosure +namespace Doxense.Collections.Tuples.Tests +{ + using System; + using System.Collections.Generic; + using System.Diagnostics; + using System.Linq; + using System.Net; + using Doxense.Collections.Tuples.Encoding; + using FoundationDB.Client; + using FoundationDB.Client.Tests; + using NUnit.Framework; + + [TestFixture] + public class TuPackFacts : FdbTest + { + + #region Serialization... + + [Test] + public void Test_TuplePack_Serialize_Bytes() + { + // Byte arrays are stored with prefix '01' followed by the bytes, and terminated by '00'. All occurences of '00' in the byte array are escaped with '00 FF' + // - Best case: packed_size = 2 + array_len + // - Worst case: packed_size = 2 + array_len * 2 + + Slice packed; + + packed = TuPack.EncodeKey(new byte[] {0x12, 0x34, 0x56, 0x78, 0x9A, 0xBC, 0xDE, 0xF0}); + Assert.That(packed.ToString(), Is.EqualTo("<01><12>4Vx<9A><00>")); + packed = TuPack.EncodeKey(new byte[] {0x00, 0x42}); + Assert.That(packed.ToString(), Is.EqualTo("<01><00>B<00>")); + packed = TuPack.EncodeKey(new byte[] {0x42, 0x00}); + Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>")); + packed = TuPack.EncodeKey(new byte[] {0x42, 0x00, 0x42}); + Assert.That(packed.ToString(), Is.EqualTo("<01>B<00>B<00>")); + packed = TuPack.EncodeKey(new byte[] {0x42, 0x00, 0x00, 0x42}); + Assert.That(packed.ToString(), Is.EqualTo("<01>B<00><00>B<00>")); + } + + [Test] + public void Test_TuplePack_Deserialize_Bytes() + { + ITuple t; + + t = TuPack.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); + Assert.That(t.Get(0), Is.EqualTo(new byte[] {0x01, 0x23, 0x45, 0x67, 0x89, 0xAB, 0xCD, 0xEF})); + Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("01 23 45 67 89 AB CD EF")); + + t = TuPack.Unpack(Slice.Unescape("<01><42><00><00>")); + Assert.That(t.Get(0), Is.EqualTo(new byte[] {0x42, 0x00})); + Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00")); + + t = TuPack.Unpack(Slice.Unescape("<01><00><42><00>")); + Assert.That(t.Get(0), Is.EqualTo(new byte[] {0x00, 0x42})); + Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("00 42")); + + t = TuPack.Unpack(Slice.Unescape("<01><42><00><42><00>")); + Assert.That(t.Get(0), Is.EqualTo(new byte[] {0x42, 0x00, 0x42})); + Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 42")); + + t = TuPack.Unpack(Slice.Unescape("<01><42><00><00><42><00>")); + Assert.That(t.Get(0), Is.EqualTo(new byte[] {0x42, 0x00, 0x00, 0x42})); + Assert.That(t.Get(0).ToHexaString(' '), Is.EqualTo("42 00 00 42")); + } + + [Test] + public void Test_TuplePack_Serialize_Unicode_Strings() + { + // Unicode strings are stored with prefix '02' followed by the utf8 bytes, and terminated by '00'. All occurences of '00' in the UTF8 bytes are escaped with '00 FF' + + Slice packed; + + // simple string + packed = TuPack.EncodeKey("hello world"); + Assert.That(packed.ToString(), Is.EqualTo("<02>hello world<00>")); + + // empty + packed = TuPack.EncodeKey(String.Empty); + Assert.That(packed.ToString(), Is.EqualTo("<02><00>")); + + // null + packed = TuPack.EncodeKey(default(string)); + Assert.That(packed.ToString(), Is.EqualTo("<00>")); + + // unicode + packed = TuPack.EncodeKey("こんにちは世界"); + // note: Encoding.UTF8.GetBytes("こんにちは世界") => { e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c } + Assert.That(packed.ToString(), Is.EqualTo("<02><81><93><82><93><81><81><81><96><95><8C><00>")); + } + + [Test] + public void Test_TuplePack_Deserialize_Unicode_Strings() + { + ITuple t; + + // simple string + t = TuPack.Unpack(Slice.Unescape("<02>hello world<00>")); + Assert.That(t.Get(0), Is.EqualTo("hello world")); + Assert.That(t[0], Is.EqualTo("hello world")); + + // empty + t = TuPack.Unpack(Slice.Unescape("<02><00>")); + Assert.That(t.Get(0), Is.EqualTo(String.Empty)); + Assert.That(t[0], Is.EqualTo(String.Empty)); + + // null + t = TuPack.Unpack(Slice.Unescape("<00>")); + Assert.That(t.Get(0), Is.EqualTo(default(string))); + Assert.That(t[0], Is.Null); + + // unicode + t = TuPack.Unpack(Slice.Unescape("<02><81><93><82><93><81><81><81><96><95><8C><00>")); + // note: Encoding.UTF8.GetString({ e3 81 93 e3 82 93 e3 81 ab e3 81 a1 e3 81 af e4 b8 96 e7 95 8c }) => "こんにちは世界" + Assert.That(t.Get(0), Is.EqualTo("こんにちは世界")); + Assert.That(t[0], Is.EqualTo("こんにちは世界")); + } + + [Test] + public void Test_TuplePack_Serialize_Guids() + { + // 128-bit Guids are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 + + // System.Guid are stored in Little-Endian, but RFC 4122's UUIDs are stored in Big Endian, so per convention we will swap them + + Slice packed; + + // note: new Guid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; + packed = TuPack.EncodeKey(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")); + Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + + packed = TuPack.EncodeKey(Guid.Empty); + Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + + } + + [Test] + public void Test_TuplePack_Deserialize_Guids() + { + // 128-bit Guids are stored with prefix '30' followed by 16 bytes + // we also accept byte arrays (prefix '01') if they are of length 16 + + ITuple packed; + + packed = TuPack.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); + Assert.That(packed[0], Is.EqualTo(Guid.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); + + packed = TuPack.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); + Assert.That(packed[0], Is.EqualTo(Guid.Empty)); + + // unicode string + packed = TuPack.Unpack(Slice.Unescape("<02>03020100-0504-0706-0809-0a0b0c0d0e0f<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Guid.Parse("03020100-0504-0706-0809-0a0b0c0d0e0f"))); + //note: t[0] returns a string, not a GUID + + // null maps to Guid.Empty + packed = TuPack.Unpack(Slice.Unescape("<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Guid.Empty)); + //note: t[0] returns null, not a GUID + + } + + [Test] + public void Test_TuplePack_Serialize_Uuid128s() + { + // UUID128s are stored with prefix '30' followed by 16 bytes formatted according to RFC 4122 + + Slice packed; + + // note: new Uuid(bytes from 0 to 15) => "03020100-0504-0706-0809-0a0b0c0d0e0f"; + packed = TuPack.EncodeKey(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f")); + Assert.That(packed.ToString(), Is.EqualTo("0<00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + + packed = TuPack.EncodeKey(Uuid128.Empty); + Assert.That(packed.ToString(), Is.EqualTo("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + } + + [Test] + public void Test_TuplePack_Deserialize_Uuid128s() + { + // UUID128s are stored with prefix '30' followed by 16 bytes (the result of uuid.ToByteArray()) + // we also accept byte arrays (prefix '01') if they are of length 16 + + ITuple packed; + + // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; + packed = TuPack.Unpack(Slice.Unescape("<30><00><01><02><03><04><05><06><07><08><09><0A><0B><0C><0D><0E><0F>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); + Assert.That(packed[0], Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); + + packed = TuPack.Unpack(Slice.Unescape("<30><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); + Assert.That(packed[0], Is.EqualTo(Uuid128.Empty)); + + // unicode string + packed = TuPack.Unpack(Slice.Unescape("<02>00010203-0405-0607-0809-0a0b0c0d0e0f<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Parse("00010203-0405-0607-0809-0a0b0c0d0e0f"))); + //note: t[0] returns a string, not a UUID + + // null maps to Uuid.Empty + packed = TuPack.Unpack(Slice.Unescape("<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid128.Empty)); + //note: t[0] returns null, not a UUID + + } + + [Test] + public void Test_TuplePack_Serialize_Uuid64s() + { + // UUID64s are stored with prefix '31' followed by 8 bytes formatted according to RFC 4122 + + Slice packed; + + // note: new Uuid(bytes from 0 to 7) => "00010203-04050607"; + packed = TuPack.EncodeKey(Uuid64.Parse("00010203-04050607")); + Assert.That(packed.ToString(), Is.EqualTo("1<00><01><02><03><04><05><06><07>")); + + packed = TuPack.EncodeKey(Uuid64.Parse("01234567-89ABCDEF")); + Assert.That(packed.ToString(), Is.EqualTo("1<01>#Eg<89>")); + + packed = TuPack.EncodeKey(Uuid64.Empty); + Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00><00><00><00><00>")); + + packed = TuPack.EncodeKey(new Uuid64(0xBADC0FFEE0DDF00DUL)); + Assert.That(packed.ToString(), Is.EqualTo("1<0F>
<0D>")); + + packed = TuPack.EncodeKey(new Uuid64(0xDEADBEEFL)); + Assert.That(packed.ToString(), Is.EqualTo("1<00><00><00><00>")); + } + + [Test] + public void Test_TuplePack_Deserialize_Uuid64s() + { + // UUID64s are stored with prefix '31' followed by 8 bytes (the result of uuid.ToByteArray()) + // we also accept byte arrays (prefix '01') if they are of length 8, and unicode strings (prefix '02') + + ITuple packed; + + // note: new Uuid(bytes from 0 to 15) => "00010203-0405-0607-0809-0a0b0c0d0e0f"; + packed = TuPack.Unpack(Slice.Unescape("<31><01><23><45><67><89>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); + Assert.That(packed[0], Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); + + packed = TuPack.Unpack(Slice.Unescape("<31><00><00><00><00><00><00><00><00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); + Assert.That(packed[0], Is.EqualTo(Uuid64.Empty)); + + // 8 bytes + packed = TuPack.Unpack(Slice.Unescape("<01><01><23><45><67><89><00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); + //note: t[0] returns a string, not a UUID + + // unicode string + packed = TuPack.Unpack(Slice.Unescape("<02>01234567-89abcdef<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Parse("01234567-89abcdef"))); + //note: t[0] returns a string, not a UUID + + // null maps to Uuid.Empty + packed = TuPack.Unpack(Slice.Unescape("<00>")); + Assert.That(packed.Get(0), Is.EqualTo(Uuid64.Empty)); + //note: t[0] returns null, not a UUID + + } + + [Test] + public void Test_TuplePack_Serialize_Integers() + { + // Positive integers are stored with a variable-length encoding. + // - The prefix is 0x14 + the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x14 to 0x1C + // - The bytes are stored in High-Endian (ie: the upper bits first) + // Examples: + // - 0 => <14> + // - 1..255 => <15><##> + // - 256..65535 .. => <16> + // - ulong.MaxValue => <1C> + + Assert.That( + TuPack.EncodeKey(0).ToString(), + Is.EqualTo("<14>") + ); + + Assert.That( + TuPack.EncodeKey(1).ToString(), + Is.EqualTo("<15><01>") + ); + + Assert.That( + TuPack.EncodeKey(255).ToString(), + Is.EqualTo("<15>") + ); + + Assert.That( + TuPack.EncodeKey(256).ToString(), + Is.EqualTo("<16><01><00>") + ); + + Assert.That( + TuPack.EncodeKey(65535).ToString(), + Is.EqualTo("<16>") + ); + + Assert.That( + TuPack.EncodeKey(65536).ToString(), + Is.EqualTo("<17><01><00><00>") + ); + + Assert.That( + TuPack.EncodeKey(int.MaxValue).ToString(), + Is.EqualTo("<18><7F>") + ); + + // signed max + Assert.That( + TuPack.EncodeKey(long.MaxValue).ToString(), + Is.EqualTo("<1C><7F>") + ); + + // unsigned max + Assert.That( + TuPack.EncodeKey(ulong.MaxValue).ToString(), + Is.EqualTo("<1C>") + ); + } + + [Test] + public void Test_TuplePack_Deserialize_Integers() + { + + Action verify = (encoded, value) => + { + var slice = Slice.Unescape(encoded); + Assert.That(TuplePackers.DeserializeBoxed(slice), Is.EqualTo(value), "DeserializeBoxed({0})", encoded); + + // int64 + Assert.That(TuplePackers.DeserializeInt64(slice), Is.EqualTo(value), "DeserializeInt64({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo(value), "Deserialize({0})", encoded); + + // uint64 + if (value >= 0) + { + Assert.That(TuplePackers.DeserializeUInt64(slice), Is.EqualTo((ulong) value), "DeserializeUInt64({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ulong) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeUInt64(slice), Throws.InstanceOf(), "DeserializeUInt64({0})", encoded); + } + + // int32 + if (value <= int.MaxValue && value >= int.MinValue) + { + Assert.That(TuplePackers.DeserializeInt32(slice), Is.EqualTo((int) value), "DeserializeInt32({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((int) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeInt32(slice), Throws.InstanceOf(), "DeserializeInt32({0})", encoded); + } + + // uint32 + if (value <= uint.MaxValue && value >= 0) + { + Assert.That(TuplePackers.DeserializeUInt32(slice), Is.EqualTo((uint) value), "DeserializeUInt32({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((uint) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeUInt32(slice), Throws.InstanceOf(), "DeserializeUInt32({0})", encoded); + } + + // int16 + if (value <= short.MaxValue && value >= short.MinValue) + { + Assert.That(TuplePackers.DeserializeInt16(slice), Is.EqualTo((short) value), "DeserializeInt16({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((short) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeInt16(slice), Throws.InstanceOf(), "DeserializeInt16({0})", encoded); + } + + // uint16 + if (value <= ushort.MaxValue && value >= 0) + { + Assert.That(TuplePackers.DeserializeUInt16(slice), Is.EqualTo((ushort) value), "DeserializeUInt16({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((ushort) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeUInt16(slice), Throws.InstanceOf(), "DeserializeUInt16({0})", encoded); + } + + // sbyte + if (value <= sbyte.MaxValue && value >= sbyte.MinValue) + { + Assert.That(TuplePackers.DeserializeSByte(slice), Is.EqualTo((sbyte) value), "DeserializeSByte({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((sbyte) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeSByte(slice), Throws.InstanceOf(), "DeserializeSByte({0})", encoded); + } + + // byte + if (value <= 255 && value >= 0) + { + Assert.That(TuplePackers.DeserializeByte(slice), Is.EqualTo((byte) value), "DeserializeByte({0})", encoded); + Assert.That(TuplePacker.Deserialize(slice), Is.EqualTo((byte) value), "Deserialize({0})", encoded); + } + else + { + Assert.That(() => TuplePackers.DeserializeByte(slice), Throws.InstanceOf(), "DeserializeByte({0})", encoded); + } + + }; + verify("<14>", 0); + verify("<15>{", 123); + verify("<15><80>", 128); + verify("<15>", 255); + verify("<16><01><00>", 256); + verify("<16><04>", 1234); + verify("<16><80><00>", 32768); + verify("<16>", 65535); + verify("<17><01><00><00>", 65536); + verify("<13>", -1); + verify("<13><00>", -255); + verify("<12>", -256); + verify("<12><00><00>", -65535); + verify("<11>", -65536); + verify("<18><7F>", int.MaxValue); + verify("<10><7F>", int.MinValue); + verify("<1C><7F>", long.MaxValue); + verify("<0C><7F>", long.MinValue); + } + + [Test] + public void Test_TuplePack_Serialize_Negative_Integers() + { + // Negative integers are stored with a variable-length encoding. + // - The prefix is 0x14 - the minimum number of bytes to encode the integer, from 0 to 8, so valid prefixes range from 0x0C to 0x13 + // - The value is encoded as the one's complement, and stored in High-Endian (ie: the upper bits first) + // - There is no way to encode '-0', it will be encoded as '0' (<14>) + // Examples: + // - -255..-1 => <13><00> .. <13> + // - -65535..-256 => <12><00>00> .. <12> + // - long.MinValue => <0C><7F> + + Assert.That( + TuPack.EncodeKey(-1).ToString(), + Is.EqualTo("<13>") + ); + + Assert.That( + TuPack.EncodeKey(-255).ToString(), + Is.EqualTo("<13><00>") + ); + + Assert.That( + TuPack.EncodeKey(-256).ToString(), + Is.EqualTo("<12>") + ); + Assert.That( + TuPack.EncodeKey(-257).ToString(), + Is.EqualTo("<12>") + ); + + Assert.That( + TuPack.EncodeKey(-65535).ToString(), + Is.EqualTo("<12><00><00>") + ); + Assert.That( + TuPack.EncodeKey(-65536).ToString(), + Is.EqualTo("<11>") + ); + + Assert.That( + TuPack.EncodeKey(int.MinValue).ToString(), + Is.EqualTo("<10><7F>") + ); + + Assert.That( + TuPack.EncodeKey(long.MinValue).ToString(), + Is.EqualTo("<0C><7F>") + ); + } + + [Test] + public void Test_TuplePack_Serialize_Singles() + { + // 32-bit floats are stored in 5 bytes, using the prefix 0x20 followed by the High-Endian representation of their normalized form + + Assert.That(TuPack.EncodeKey(0f).ToHexaString(' '), Is.EqualTo("20 80 00 00 00")); + Assert.That(TuPack.EncodeKey(42f).ToHexaString(' '), Is.EqualTo("20 C2 28 00 00")); + Assert.That(TuPack.EncodeKey(-42f).ToHexaString(' '), Is.EqualTo("20 3D D7 FF FF")); + + Assert.That(TuPack.EncodeKey((float) Math.Sqrt(2)).ToHexaString(' '), Is.EqualTo("20 BF B5 04 F3")); + + Assert.That(TuPack.EncodeKey(float.MinValue).ToHexaString(' '), Is.EqualTo("20 00 80 00 00"), "float.MinValue"); + Assert.That(TuPack.EncodeKey(float.MaxValue).ToHexaString(' '), Is.EqualTo("20 FF 7F FF FF"), "float.MaxValue"); + Assert.That(TuPack.EncodeKey(-0f).ToHexaString(' '), Is.EqualTo("20 7F FF FF FF"), "-0f"); + Assert.That(TuPack.EncodeKey(float.NegativeInfinity).ToHexaString(' '), Is.EqualTo("20 00 7F FF FF"), "float.NegativeInfinity"); + Assert.That(TuPack.EncodeKey(float.PositiveInfinity).ToHexaString(' '), Is.EqualTo("20 FF 80 00 00"), "float.PositiveInfinity"); + Assert.That(TuPack.EncodeKey(float.Epsilon).ToHexaString(' '), Is.EqualTo("20 80 00 00 01"), "+float.Epsilon"); + Assert.That(TuPack.EncodeKey(-float.Epsilon).ToHexaString(' '), Is.EqualTo("20 7F FF FF FE"), "-float.Epsilon"); + + // all possible variants of NaN should all be equal + Assert.That(TuPack.EncodeKey(float.NaN).ToHexaString(' '), Is.EqualTo("20 00 3F FF FF"), "float.NaN"); + + // cook up a non standard NaN (with some bits set in the fraction) + float f = float.NaN; // defined as 1f / 0f + uint nan; + unsafe { nan = *((uint*) &f); } + nan += 123; + unsafe { f = *((float*) &nan); } + Assert.That(float.IsNaN(f), Is.True); + Assert.That( + TuPack.EncodeKey(f).ToHexaString(' '), + Is.EqualTo("20 00 3F FF FF"), + "All variants of NaN must be normalized" + //note: if we have 20 00 3F FF 84, that means that the NaN was not normalized + ); + + } + + [Test] + public void Test_TuplePack_Deserialize_Singles() + { + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 80 00 00 00")), Is.EqualTo(0f), "0f"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 C2 28 00 00")), Is.EqualTo(42f), "42f"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 3D D7 FF FF")), Is.EqualTo(-42f), "-42f"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 BF B5 04 F3")), Is.EqualTo((float) Math.Sqrt(2)), "Sqrt(2)"); + + // well known values + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.MinValue"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 FF 7F FF FF")), Is.EqualTo(float.MaxValue), "float.MaxValue"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 7F FF FF FF")), Is.EqualTo(-0f), "-0f"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 00 7F FF FF")), Is.EqualTo(float.NegativeInfinity), "float.NegativeInfinity"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 FF 80 00 00")), Is.EqualTo(float.PositiveInfinity), "float.PositiveInfinity"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 00 80 00 00")), Is.EqualTo(float.MinValue), "float.Epsilon"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 80 00 00 01")), Is.EqualTo(float.Epsilon), "+float.Epsilon"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 7F FF FF FE")), Is.EqualTo(-float.Epsilon), "-float.Epsilon"); + + // all possible variants of NaN should end up equal and normalized to float.NaN + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("20 00 3F FF FF")), Is.EqualTo(float.NaN), "float.NaN"); + } + + [Test] + public void Test_TuplePack_Serialize_Doubles() + { + // 64-bit floats are stored in 9 bytes, using the prefix 0x21 followed by the High-Endian representation of their normalized form + + Assert.That(TuPack.EncodeKey(0d).ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 00")); + Assert.That(TuPack.EncodeKey(42d).ToHexaString(' '), Is.EqualTo("21 C0 45 00 00 00 00 00 00")); + Assert.That(TuPack.EncodeKey(-42d).ToHexaString(' '), Is.EqualTo("21 3F BA FF FF FF FF FF FF")); + + Assert.That(TuPack.EncodeKey(Math.PI).ToHexaString(' '), Is.EqualTo("21 C0 09 21 FB 54 44 2D 18")); + Assert.That(TuPack.EncodeKey(Math.E).ToHexaString(' '), Is.EqualTo("21 C0 05 BF 0A 8B 14 57 69")); + + Assert.That(TuPack.EncodeKey(double.MinValue).ToHexaString(' '), Is.EqualTo("21 00 10 00 00 00 00 00 00"), "double.MinValue"); + Assert.That(TuPack.EncodeKey(double.MaxValue).ToHexaString(' '), Is.EqualTo("21 FF EF FF FF FF FF FF FF"), "double.MaxValue"); + Assert.That(TuPack.EncodeKey(-0d).ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FF"), "-0d"); + Assert.That(TuPack.EncodeKey(double.NegativeInfinity).ToHexaString(' '), Is.EqualTo("21 00 0F FF FF FF FF FF FF"), "double.NegativeInfinity"); + Assert.That(TuPack.EncodeKey(double.PositiveInfinity).ToHexaString(' '), Is.EqualTo("21 FF F0 00 00 00 00 00 00"), "double.PositiveInfinity"); + Assert.That(TuPack.EncodeKey(double.Epsilon).ToHexaString(' '), Is.EqualTo("21 80 00 00 00 00 00 00 01"), "+double.Epsilon"); + Assert.That(TuPack.EncodeKey(-double.Epsilon).ToHexaString(' '), Is.EqualTo("21 7F FF FF FF FF FF FF FE"), "-double.Epsilon"); + + // all possible variants of NaN should all be equal + + Assert.That(TuPack.EncodeKey(double.NaN).ToHexaString(' '), Is.EqualTo("21 00 07 FF FF FF FF FF FF"), "double.NaN"); + + // cook up a non standard NaN (with some bits set in the fraction) + double d = double.NaN; // defined as 1d / 0d + ulong nan; + unsafe { nan = *((ulong*) &d); } + nan += 123; + unsafe { d = *((double*) &nan); } + Assert.That(double.IsNaN(d), Is.True); + Assert.That( + TuPack.EncodeKey(d).ToHexaString(' '), + Is.EqualTo("21 00 07 FF FF FF FF FF FF") + //note: if we have 21 00 07 FF FF FF FF FF 84, that means that the NaN was not normalized + ); + + // roundtripping vectors of doubles + var tuple = STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)); + Assert.That(TuPack.Unpack(TuPack.EncodeKey(Math.PI, Math.E, Math.Log(1), Math.Log(2))), Is.EqualTo(tuple)); + Assert.That(TuPack.Unpack(TuPack.Pack(STuple.Create(Math.PI, Math.E, Math.Log(1), Math.Log(2)))), Is.EqualTo(tuple)); + Assert.That(TuPack.Unpack(TuPack.Pack(STuple.Empty.Append(Math.PI).Append(Math.E).Append(Math.Log(1)).Append(Math.Log(2)))), Is.EqualTo(tuple)); + } + + [Test] + public void Test_TuplePack_Deserialize_Doubles() + { + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 00")), Is.EqualTo(0d), "0d"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 C0 45 00 00 00 00 00 00")), Is.EqualTo(42d), "42d"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 3F BA FF FF FF FF FF FF")), Is.EqualTo(-42d), "-42d"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 C0 09 21 FB 54 44 2D 18")), Is.EqualTo(Math.PI), "Math.PI"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 C0 05 BF 0A 8B 14 57 69")), Is.EqualTo(Math.E), "Math.E"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 00 10 00 00 00 00 00 00")), Is.EqualTo(double.MinValue), "double.MinValue"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 FF EF FF FF FF FF FF FF")), Is.EqualTo(double.MaxValue), "double.MaxValue"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FF")), Is.EqualTo(-0d), "-0d"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 00 0F FF FF FF FF FF FF")), Is.EqualTo(double.NegativeInfinity), "double.NegativeInfinity"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 FF F0 00 00 00 00 00 00")), Is.EqualTo(double.PositiveInfinity), "double.PositiveInfinity"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 80 00 00 00 00 00 00 01")), Is.EqualTo(double.Epsilon), "+double.Epsilon"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 7F FF FF FF FF FF FF FE")), Is.EqualTo(-double.Epsilon), "-double.Epsilon"); + + // all possible variants of NaN should end up equal and normalized to double.NaN + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF FF")), Is.EqualTo(double.NaN), "double.NaN"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("21 00 07 FF FF FF FF FF 84")), Is.EqualTo(double.NaN), "double.NaN"); + } + + [Test] + public void Test_TuplePack_Serialize_Booleans() + { + // Booleans are stored as interger 0 (<14>) for false, and integer 1 (<15><01>) for true + + Slice packed; + + // bool + packed = TuPack.EncodeKey(false); + Assert.That(packed.ToString(), Is.EqualTo("<14>")); + packed = TuPack.EncodeKey(true); + Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); + + // bool? + packed = TuPack.EncodeKey(default(bool?)); + Assert.That(packed.ToString(), Is.EqualTo("<00>")); + packed = TuPack.EncodeKey((bool?) false); + Assert.That(packed.ToString(), Is.EqualTo("<14>")); + packed = TuPack.EncodeKey((bool?) true); + Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); + + // tuple containing bools + packed = TuPack.EncodeKey(true); + Assert.That(packed.ToString(), Is.EqualTo("<15><01>")); + packed = TuPack.EncodeKey(true, default(string), false); + Assert.That(packed.ToString(), Is.EqualTo("<15><01><00><14>")); + } + + [Test] + public void Test_TuplePack_Deserialize_Booleans() + { + // Null, 0, and empty byte[]/strings are equivalent to False. All others are equivalent to True + + // Falsy... + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.False, "Null => False"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<14>")), Is.False, "0 => False"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<01><00>")), Is.False, "byte[0] => False"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<02><00>")), Is.False, "String.Empty => False"); + + // Truthy + Assert.That(TuPack.DecodeKey(Slice.Unescape("<15><01>")), Is.True, "1 => True"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<13>")), Is.True, "-1 => True"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<01>Hello<00>")), Is.True, "'Hello' => True"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<02>Hello<00>")), Is.True, "\"Hello\" => True"); + Assert.That(TuPack.DecodeKey(TuPack.EncodeKey(123456789)), Is.True, "random int => True"); + + Assert.That(TuPack.DecodeKey(Slice.Unescape("<02>True<00>")), Is.True, "\"True\" => True"); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<02>False<00>")), Is.True, "\"False\" => True ***"); + // note: even though it would be tempting to convert the string "false" to False, it is not a standard behavior accross all bindings + + // When decoded to object, though, they should return 0 and 1 + Assert.That(TuplePackers.DeserializeBoxed(TuPack.EncodeKey(false)), Is.EqualTo(0)); + Assert.That(TuplePackers.DeserializeBoxed(TuPack.EncodeKey(true)), Is.EqualTo(1)); + } + + [Test] + public void Test_TuplePack_Serialize_VersionStamps() + { + // incomplete, 80 bits + Assert.That( + TuPack.EncodeKey(VersionStamp.Incomplete()).ToHexaString(' '), + Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF") + ); + + // incomplete, 96 bits + Assert.That( + TuPack.EncodeKey(VersionStamp.Incomplete(0)).ToHexaString(' '), + Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF 00 00") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Incomplete(42)).ToHexaString(' '), + Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF 00 2A") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Incomplete(456)).ToHexaString(' '), + Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF 01 C8") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Incomplete(65535)).ToHexaString(' '), + Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF FF FF") + ); + + // complete, 80 bits + Assert.That( + TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 1234)).ToHexaString(' '), + Is.EqualTo("33 01 23 45 67 89 AB CD EF 04 D2") + ); + + // complete, 96 bits + Assert.That( + TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 0)).ToHexaString(' '), + Is.EqualTo("33 01 23 45 67 89 AB CD EF 04 D2 00 00") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 42)).ToHexaString(' '), + Is.EqualTo("33 01 23 45 67 89 AB CD EF 04 D2 00 2A") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 65535, 42)).ToHexaString(' '), + Is.EqualTo("33 01 23 45 67 89 AB CD EF FF FF 00 2A") + ); + Assert.That( + TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 65535)).ToHexaString(' '), + Is.EqualTo("33 01 23 45 67 89 AB CD EF 04 D2 FF FF") + ); + } + + [Test] + public void Test_TuplePack_Deserailize_VersionStamps() + { + Assert.That(TuPack.DecodeKey(Slice.FromHexa("32 FF FF FF FF FF FF FF FF FF FF")), Is.EqualTo(VersionStamp.Incomplete()), "Incomplete()"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 00 00")), Is.EqualTo(VersionStamp.Incomplete()), "Incomplete(0)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 00 2A")), Is.EqualTo(VersionStamp.Incomplete(42)), "Incomplete(42)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 01 C8")), Is.EqualTo(VersionStamp.Incomplete(456)), "Incomplete(456)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF FF FF")), Is.EqualTo(VersionStamp.Incomplete(65535)), "Incomplete(65535)"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("32 01 23 45 67 89 AB CD EF 04 D2")), Is.EqualTo(VersionStamp.Complete(0x0123456789ABCDEF, 1234)), "Complete(..., 1234)"); + + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 01 23 45 67 89 AB CD EF 04 D2 00 00")), Is.EqualTo(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 0)), "Complete(..., 1234, 0)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 01 23 45 67 89 AB CD EF 04 D2 00 2A")), Is.EqualTo(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 42)), "Complete(..., 1234, 42)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 01 23 45 67 89 AB CD EF FF FF 00 2A")), Is.EqualTo(VersionStamp.Complete(0x0123456789ABCDEF, 65535, 42)), "Complete(..., 65535, 42)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 01 23 45 67 89 AB CD EF 04 D2 FF FF")), Is.EqualTo(VersionStamp.Complete(0x0123456789ABCDEF, 1234, 65535)), "Complete(..., 1234, 65535)"); + } + + [Test] + public void Test_TuplePack_Serialize_IPAddress() + { + // IP Addresses are stored as a byte array (<01>..<00>), in network order (big-endian) + // They will take from 6 to 10 bytes, depending on the number of '.0' in them. + + Assert.That( + TuPack.EncodeKey(IPAddress.Loopback).ToHexaString(' '), + Is.EqualTo("01 7F 00 FF 00 FF 01 00") + ); + + Assert.That( + TuPack.EncodeKey(IPAddress.Any).ToHexaString(' '), + Is.EqualTo("01 00 FF 00 FF 00 FF 00 FF 00") + ); + + Assert.That( + TuPack.EncodeKey(IPAddress.Parse("1.2.3.4")).ToHexaString(' '), + Is.EqualTo("01 01 02 03 04 00") + ); + + } + + + [Test] + public void Test_TuplePack_Deserialize_IPAddress() + { + Assert.That(TuPack.DecodeKey(Slice.Unescape("<01><7F><00><00><01><00>")), Is.EqualTo(IPAddress.Parse("127.0.0.1"))); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<01><00><00><00><00><00>")), Is.EqualTo(IPAddress.Parse("0.0.0.0"))); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<01><01><02><03><04><00>")), Is.EqualTo(IPAddress.Parse("1.2.3.4"))); + + Assert.That(TuPack.DecodeKey(TuPack.EncodeKey("127.0.0.1")), Is.EqualTo(IPAddress.Loopback)); + + var ip = IPAddress.Parse("192.168.0.1"); + Assert.That(TuPack.DecodeKey(TuPack.EncodeKey(ip.ToString())), Is.EqualTo(ip)); + Assert.That(TuPack.DecodeKey(TuPack.EncodeKey(ip.GetAddressBytes())), Is.EqualTo(ip)); +#pragma warning disable 618 + Assert.That(TuPack.DecodeKey(TuPack.EncodeKey(ip.Address)), Is.EqualTo(ip)); +#pragma warning restore 618 + } + + [Test] + public void Test_TuplePack_NullableTypes() + { + // Nullable types will either be encoded as <14> for null, or their regular encoding if not null + + // serialize + + Assert.That(TuPack.EncodeKey(0), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(TuPack.EncodeKey(123), Is.EqualTo(Slice.Unescape("<15>{"))); + Assert.That(TuPack.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + + Assert.That(TuPack.EncodeKey(0L), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(TuPack.EncodeKey(123L), Is.EqualTo(Slice.Unescape("<15>{"))); + Assert.That(TuPack.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + + Assert.That(TuPack.EncodeKey(true), Is.EqualTo(Slice.Unescape("<15><01>"))); + Assert.That(TuPack.EncodeKey(false), Is.EqualTo(Slice.Unescape("<14>"))); + Assert.That(TuPack.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>")), "Maybe it was File Not Found?"); + + Assert.That(TuPack.EncodeKey(Guid.Empty), Is.EqualTo(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>"))); + Assert.That(TuPack.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + + Assert.That(TuPack.EncodeKey(TimeSpan.Zero), Is.EqualTo(Slice.Unescape("!<80><00><00><00><00><00><00><00>"))); + Assert.That(TuPack.EncodeKey(null), Is.EqualTo(Slice.Unescape("<00>"))); + + // deserialize + + Assert.That(TuPack.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.Null); + + Assert.That(TuPack.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(0L)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<15>{")), Is.EqualTo(123L)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.Null); + + Assert.That(TuPack.DecodeKey(Slice.Unescape("<15><01>")), Is.True); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<14>")), Is.False); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.Null); + + Assert.That(TuPack.DecodeKey(Slice.Unescape("0<00><00><00><00><00><00><00><00><00><00><00><00><00><00><00><00>")), Is.EqualTo(Guid.Empty)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.Null); + + Assert.That(TuPack.DecodeKey(Slice.Unescape("<14>")), Is.EqualTo(TimeSpan.Zero)); + Assert.That(TuPack.DecodeKey(Slice.Unescape("<00>")), Is.Null); + + } + + [Test] + public void Test_TuplePack_Serialize_Embedded_Tuples() + { + Action verify = (t, expected) => + { + var key = TuPack.Pack(t); + Assert.That(key.ToHexaString(' '), Is.EqualTo(expected)); + var t2 = TuPack.Unpack(key); + Assert.That(t2, Is.Not.Null); + Assert.That(t2.Count, Is.EqualTo(t.Count), "{0}", t2); + Assert.That(t2, Is.EqualTo(t)); + }; + + // Index composite key + ITuple value = STuple.Create(2014, 11, 6); // Indexing a date value (Y, M, D) + string docId = "Doc123"; + // key would be "(..., value, id)" + + verify( + STuple.Create(42, value, docId), + "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" + ); + verify( + STuple.Create(new object[] {42, value, docId}), + "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" + ); + verify( + STuple.Create(42).Append(value).Append(docId), + "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" + ); + verify( + STuple.Create(42).Append(value, docId), + "15 2A 03 16 07 DE 15 0B 15 06 00 02 44 6F 63 31 32 33 00" + ); + + // multiple depth + verify( + STuple.Create(1, STuple.Create(2, 3), STuple.Create(STuple.Create(4, 5, 6)), 7), + "15 01 03 15 02 15 03 00 03 03 15 04 15 05 15 06 00 00 15 07" + ); + + // corner cases + verify( + STuple.Create(STuple.Empty), + "03 00" // empty tumple should have header and footer + ); + verify( + STuple.Create(STuple.Empty, default(string)), + "03 00 00" // outer null should not be escaped + ); + verify( + STuple.Create(STuple.Create(default(string)), default(string)), + "03 00 FF 00 00" // inner null should be escaped, but not outer + ); + verify( + STuple.Create(STuple.Create(0x100, 0x10000, 0x1000000)), + "03 16 01 00 17 01 00 00 18 01 00 00 00 00" + ); + verify( + STuple.Create(default(string), STuple.Empty, default(string), STuple.Create(default(string)), default(string)), + "00 03 00 00 03 00 FF 00 00" + ); + + } + + [Test] + public void Test_TuplePack_Deserialize_Embedded_Tuples() + { + // ((42, (2014, 11, 6), "Hello", true), ) + var packed = TuPack.EncodeKey(STuple.Create(42, STuple.Create(2014, 11, 6), "Hello", true)); + Log($"t = {TuPack.Unpack(packed)}"); + Assert.That(packed[0], Is.EqualTo(TupleTypes.TupleStart), "Missing Embedded Tuple marker"); + { + var t = TuPack.DecodeKey(packed); + Assert.That(t, Is.Not.Null); + Assert.That(t.Count, Is.EqualTo(4)); + Assert.That(t.Get(0), Is.EqualTo(42)); + Assert.That(t.Get(1), Is.EqualTo(STuple.Create(2014, 11, 6))); + Assert.That(t.Get(2), Is.EqualTo("Hello")); + Assert.That(t.Get(3), Is.True); + } + { + var t = TuPack.DecodeKey>(packed); + Assert.That(t, Is.Not.Null); + Assert.That(t.Item1, Is.EqualTo(42)); + Assert.That(t.Item2, Is.EqualTo(STuple.Create(2014, 11, 6))); + Assert.That(t.Item3, Is.EqualTo("Hello")); + Assert.That(t.Item4, Is.True); + } + { + var t = TuPack.DecodeKey, string, bool>>(packed); + Assert.That(t, Is.Not.Null); + Assert.That(t.Item1, Is.EqualTo(42)); + Assert.That(t.Item2, Is.EqualTo(STuple.Create(2014, 11, 6))); + Assert.That(t.Item3, Is.EqualTo("Hello")); + Assert.That(t.Item4, Is.True); + } + + // (null,) + packed = TuPack.EncodeKey(default(string)); + Log($"t = {TuPack.Unpack(packed)}"); + { + var t = TuPack.DecodeKey(packed); + Assert.That(t, Is.Null); + } + { + var t = TuPack.DecodeKey, string, bool>>(packed); + Assert.That(t.Item1, Is.EqualTo(0)); + Assert.That(t.Item2, Is.EqualTo(default(STuple))); + Assert.That(t.Item3, Is.Null); + Assert.That(t.Item4, Is.False); + } + + //fallback if encoded as slice + packed = TuPack.EncodeKey(TuPack.EncodeKey(42, STuple.Create(2014, 11, 6), "Hello", true)); + Log($"t = {TuPack.Unpack(packed)}"); + Assert.That(packed[0], Is.EqualTo(TupleTypes.Bytes), "Missing Slice marker"); + { + var t = TuPack.DecodeKey, string, bool>>(packed); + Assert.That(t, Is.Not.Null); + Assert.That(t.Item1, Is.EqualTo(42)); + Assert.That(t.Item2, Is.EqualTo(STuple.Create(2014, 11, 6))); + Assert.That(t.Item3, Is.EqualTo("Hello")); + Assert.That(t.Item4, Is.True); + } + } + + [Test] + public void Test_TuplePack_SameBytes() + { + // two ways on packing the "same" tuple yield the same binary output + { + var expected = TuPack.EncodeKey("Hello World"); + Assert.That(TuPack.Pack(STuple.Create("Hello World")), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(((ITuple) STuple.Create("Hello World"))), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create(new object[] {"Hello World"})), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create("Hello World", 1234).Substring(0, 1)), Is.EqualTo(expected)); + } + { + var expected = TuPack.EncodeKey("Hello World", 1234); + Assert.That(TuPack.Pack(STuple.Create("Hello World", 1234)), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(((ITuple) STuple.Create("Hello World", 1234))), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create("Hello World").Append(1234)), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(((ITuple) STuple.Create("Hello World")).Append(1234)), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create(new object[] {"Hello World", 1234})), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create("Hello World", 1234, "Foo").Substring(0, 2)), Is.EqualTo(expected)); + } + { + var expected = TuPack.EncodeKey("Hello World", 1234, "Foo"); + Assert.That(TuPack.Pack(STuple.Create("Hello World", 1234, "Foo")), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(((ITuple) STuple.Create("Hello World", 1234, "Foo"))), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create("Hello World").Append(1234).Append("Foo")), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(((ITuple) STuple.Create("Hello World")).Append(1234).Append("Foo")), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create(new object[] {"Hello World", 1234, "Foo"})), Is.EqualTo(expected)); + Assert.That(TuPack.Pack(STuple.Create("Hello World", 1234, "Foo", "Bar").Substring(0, 3)), Is.EqualTo(expected)); + } + + // also, there should be no differences between int,long,uint,... if they have the same value + Assert.That(TuPack.Pack(STuple.Create("Hello", 123)), Is.EqualTo(TuPack.Pack(STuple.Create("Hello", 123L)))); + Assert.That(TuPack.Pack(STuple.Create("Hello", -123)), Is.EqualTo(TuPack.Pack(STuple.Create("Hello", -123L)))); + + // GUID / UUID128 should pack the same way + var g = Guid.NewGuid(); + Assert.That(TuPack.Pack(STuple.Create(g)), Is.EqualTo(TuPack.Pack(STuple.Create((Uuid128) g))), "GUID vs UUID128"); + } + + [Test] + public void Test_TuplePack_Numbers_Are_Sorted_Lexicographically() + { + // pick two numbers 'x' and 'y' at random, and check that the order of 'x' compared to 'y' is the same as 'pack(tuple(x))' compared to 'pack(tuple(y))' + + // ie: ensure that x.CompareTo(y) always has the same sign as Tuple(x).CompareTo(Tuple(y)) + + const int N = 1 * 1000 * 1000; + var rnd = new Random(); + var sw = Stopwatch.StartNew(); + + for (int i = 0; i < N; i++) + { + int x = rnd.Next() - 1073741824; + int y = x; + while (y == x) + { + y = rnd.Next() - 1073741824; + } + + var t1 = TuPack.EncodeKey(x); + var t2 = TuPack.EncodeKey(y); + + int dint = x.CompareTo(y); + int dtup = t1.CompareTo(t2); + + if (dtup == 0) Assert.Fail("Tuples for x={0} and y={1} should not have the same packed value", x, y); + + // compare signs + if (Math.Sign(dint) != Math.Sign(dtup)) + { + Assert.Fail("Tuples for x={0} and y={1} are not sorted properly ({2} / {3}): t(x)='{4}' and t(y)='{5}'", x, y, dint, dtup, t1.ToString(), t2.ToString()); + } + } + sw.Stop(); + Log("Checked {0:N0} tuples in {1:N1} ms", N, sw.ElapsedMilliseconds); + + } + + #endregion + + [Test] + public void Test_TuplePack_Pack() + { + Assert.That( + TuPack.Pack(STuple.Create()), + Is.EqualTo(Slice.Empty) + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world")).ToString(), + Is.EqualTo("<02>hello world<00>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello", "world")).ToString(), + Is.EqualTo("<02>hello<00><02>world<00>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123)).ToString(), + Is.EqualTo("<02>hello world<00><15>{") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 1234, -1234)).ToString(), + Is.EqualTo("<02>hello world<00><16><04><12>-") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false, new byte[] {123, 1, 66, 0, 42})).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界")).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>") + ); + Assert.That( + TuPack.Pack(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界", true)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><15><01>") + ); + Assert.That( + TuPack.Pack(STuple.Create(new object[] {"hello world", 123, false, new byte[] {123, 1, 66, 0, 42}})).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.Pack(STuple.FromArray(new object[] {"hello world", 123, false, new byte[] {123, 1, 66, 0, 42}}, 1, 2)).ToString(), + Is.EqualTo("<15>{<14>") + ); + Assert.That( + TuPack.Pack(STuple.FromEnumerable(new List {"hello world", 123, false, new byte[] {123, 1, 66, 0, 42}})).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + + } + + [Test] + public void Test_TuplePack_Pack_With_Prefix() + { + + Slice prefix = Slice.FromString("ABC"); + + Assert.That( + TuPack.Pack(prefix, STuple.Create()).ToString(), + Is.EqualTo("ABC") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world")).ToString(), + Is.EqualTo("ABC<02>hello world<00>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello", "world")).ToString(), + Is.EqualTo("ABC<02>hello<00><02>world<00>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123)).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 1234, -1234)).ToString(), + Is.EqualTo("ABC<02>hello world<00><16><04><12>-") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false)).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 })).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI)).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L)).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界")).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界", true)).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><15><01>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.Create(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } })).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.FromArray(new object[] { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } }, 1, 2)).ToString(), + Is.EqualTo("ABC<15>{<14>") + ); + Assert.That( + TuPack.Pack(prefix, STuple.FromEnumerable(new List { "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 } })).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + + // Nil or Empty slice should be equivalent to no prefix + Assert.That( + TuPack.Pack(Slice.Nil, STuple.Create("hello world", 123)).ToString(), + Is.EqualTo("<02>hello world<00><15>{") + ); + Assert.That( + TuPack.Pack(Slice.Empty, STuple.Create("hello world", 123)).ToString(), + Is.EqualTo("<02>hello world<00><15>{") + ); + } + + [Test] + public void Test_TuplePack_PackTuples() + { + { + Slice[] slices; + var tuples = new ITuple[] + { + STuple.Create("hello"), + STuple.Create(123), + STuple.Create(false), + STuple.Create("world", 456, true) + }; + + // array version + slices = TuPack.PackTuples(tuples); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(tuples.Length)); + Assert.That(slices, Is.EqualTo(tuples.Select(t => TuPack.Pack(t)))); + + // IEnumerable version that is passed an array + slices = tuples.PackTuples(); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(tuples.Length)); + Assert.That(slices, Is.EqualTo(tuples.Select(t => TuPack.Pack(t)))); + + // IEnumerable version but with a "real" enumerable + slices = tuples.Select(t => t).PackTuples(); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(tuples.Length)); + Assert.That(slices, Is.EqualTo(tuples.Select(t => TuPack.Pack(t)))); + } + + //Optimized STuple<...> versions + + { + var packed = TuPack.PackTuples( + STuple.Create("Hello"), + STuple.Create(123, true), + STuple.Create(Math.PI, -1234L) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>Hello<00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<15>{<15><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("!<09>!TD-<18><12>-")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create(123), + STuple.Create(456), + STuple.Create(789) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<15>{")); + Assert.That(packed[1].ToString(), Is.EqualTo("<16><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<16><03><15>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create(123, true), + STuple.Create(456, false), + STuple.Create(789, false) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<15>{<15><01>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<16><01><14>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<16><03><15><14>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create("foo", 123, true), + STuple.Create("bar", 456, false), + STuple.Create("baz", 789, false) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>foo<00><15>{<15><01>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<02>bar<00><16><01><14>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<02>baz<00><16><03><15><14>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create("foo", 123, true, "yes"), + STuple.Create("bar", 456, false, "yes"), + STuple.Create("baz", 789, false, "no") + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>foo<00><15>{<15><01><02>yes<00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<02>bar<00><16><01><14><02>yes<00>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<02>baz<00><16><03><15><14><02>no<00>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create("foo", 123, true, "yes", 7), + STuple.Create("bar", 456, false, "yes", 42), + STuple.Create("baz", 789, false, "no", 9) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>foo<00><15>{<15><01><02>yes<00><15><07>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<02>bar<00><16><01><14><02>yes<00><15>*")); + Assert.That(packed[2].ToString(), Is.EqualTo("<02>baz<00><16><03><15><14><02>no<00><15><09>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + STuple.Create("foo", 123, true, "yes", 7, 1.5d), + STuple.Create("bar", 456, false, "yes", 42, 0.7d), + STuple.Create("baz", 789, false, "no", 9, 0.66d) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>foo<00><15>{<15><01><02>yes<00><15><07>!<00><00><00><00><00><00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<02>bar<00><16><01><14><02>yes<00><15>*!ffffff")); + Assert.That(packed[2].ToString(), Is.EqualTo("<02>baz<00><16><03><15><14><02>no<00><15><09>!<1E>Q<85><1F>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + } + + [Test] + public void Test_TuplePack_PackTuples_With_Prefix() + { + Slice prefix = Slice.FromString("ABC"); + + { + Slice[] slices; + var tuples = new ITuple[] + { + STuple.Create("hello"), + STuple.Create(123), + STuple.Create(false), + STuple.Create("world", 456, true) + }; + + // array version + slices = TuPack.PackTuples(prefix, tuples); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(tuples.Length)); + Assert.That(slices, Is.EqualTo(tuples.Select(t => prefix + TuPack.Pack(t)))); + + // LINQ version + slices = TuPack.PackTuples(prefix, tuples.Select(x => x)); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(tuples.Length)); + Assert.That(slices, Is.EqualTo(tuples.Select(t => prefix + TuPack.Pack(t)))); + + } + + //Optimized STuple<...> versions + + { + var packed = TuPack.PackTuples( + prefix, + STuple.Create("Hello"), + STuple.Create(123, true), + STuple.Create(Math.PI, -1234L) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<02>Hello<00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<15>{<15><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC!<09>!TD-<18><12>-")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + prefix, + STuple.Create(123), + STuple.Create(456), + STuple.Create(789) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<15>{")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<16><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<16><03><15>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + prefix, + STuple.Create(123, true), + STuple.Create(456, false), + STuple.Create(789, false) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<15>{<15><01>")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<16><01><14>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<16><03><15><14>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.PackTuples( + prefix, + STuple.Create("foo", 123, true), + STuple.Create("bar", 456, false), + STuple.Create("baz", 789, false) + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<02>foo<00><15>{<15><01>")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<02>bar<00><16><01><14>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<02>baz<00><16><03><15><14>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + } + + [Test] + public void Test_TuplePack_EncodeKey() + { + Assert.That( + TuPack.EncodeKey("hello world").ToString(), + Is.EqualTo("<02>hello world<00>") + ); + Assert.That( + TuPack.EncodeKey("hello", "world").ToString(), + Is.EqualTo("<02>hello<00><02>world<00>") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123).ToString(), + Is.EqualTo("<02>hello world<00><15>{") + ); + Assert.That( + TuPack.EncodeKey("hello world", 1234, -1234).ToString(), + Is.EqualTo("<02>hello world<00><16><04><12>-") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14>") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false, new byte[] {123, 1, 66, 0, 42}).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界").ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>") + ); + Assert.That( + TuPack.EncodeKey("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界", true).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><15><01>") + ); + + } + + [Test] + public void Test_TuplePack_EncodeKey_With_Prefix() + { + Slice prefix = Slice.FromString("ABC"); + + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world").ToString(), + Is.EqualTo("ABC<02>hello world<00>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello", "world").ToString(), + Is.EqualTo("ABC<02>hello<00><02>world<00>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 1234, -1234).ToString(), + Is.EqualTo("ABC<02>hello world<00><16><04><12>-") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界").ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>") + ); + Assert.That( + TuPack.EncodePrefixedKey(prefix, "hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L, "こんにちは世界", true).ToString(), + Is.EqualTo("ABC<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><15><01>") + ); + + } + + [Test] + public void Test_TuplePack_EncodeKey_Boxed() + { + Slice slice; + + slice = TuPack.EncodeKey(default(object)); + Assert.That(slice.ToString(), Is.EqualTo("<00>")); + + slice = TuPack.EncodeKey(1); + Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); + + slice = TuPack.EncodeKey(1L); + Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); + + slice = TuPack.EncodeKey(1U); + Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); + + slice = TuPack.EncodeKey(1UL); + Assert.That(slice.ToString(), Is.EqualTo("<15><01>")); + + slice = TuPack.EncodeKey(false); + Assert.That(slice.ToString(), Is.EqualTo("<14>")); + + slice = TuPack.EncodeKey(new byte[] {4, 5, 6}); + Assert.That(slice.ToString(), Is.EqualTo("<01><04><05><06><00>")); + + slice = TuPack.EncodeKey("hello"); + Assert.That(slice.ToString(), Is.EqualTo("<02>hello<00>")); + } + + [Test] + public void Test_TuplePack_EncodeKeys() + { + //Optimized STuple<...> versions + + { + var packed = TuPack.EncodeKeys( + "foo", + "bar", + "baz" + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<02>foo<00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<02>bar<00>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<02>baz<00>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.EncodeKeys( + 123, + 456, + 789 + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<15>{")); + Assert.That(packed[1].ToString(), Is.EqualTo("<16><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<16><03><15>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + + { + var packed = TuPack.EncodeKeys(Enumerable.Range(0, 3)); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<14>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<15><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<15><02>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.EncodeKeys(new[] {"Bonjour", "le", "Monde"}, (s) => s.Length); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("<15><07>")); + Assert.That(packed[1].ToString(), Is.EqualTo("<15><02>")); + Assert.That(packed[2].ToString(), Is.EqualTo("<15><05>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + } + + [Test] + public void Test_TuplePack_EncodeKeys_With_Prefix() + { + Slice prefix = Slice.FromString("ABC"); + + { + var packed = TuPack.EncodePrefixedKeys( + prefix, + "foo", + "bar", + "baz" + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<02>foo<00>")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<02>bar<00>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<02>baz<00>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.EncodePrefixedKeys( + prefix, + 123, + 456, + 789 + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<15>{")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<16><01>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<16><03><15>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + { + var packed = TuPack.EncodePrefixedKeys( + prefix, + new[] { "Bonjour", "le", "Monde" }, + (s) => s.Length + ); + Assert.That(packed, Is.Not.Null.And.Length.EqualTo(3)); + Assert.That(packed[0].ToString(), Is.EqualTo("ABC<15><07>")); + Assert.That(packed[1].ToString(), Is.EqualTo("ABC<15><02>")); + Assert.That(packed[2].ToString(), Is.EqualTo("ABC<15><05>")); + Assert.That(packed[1].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + Assert.That(packed[2].Array, Is.SameAs(packed[0].Array), "Should share same bufer"); + } + + } + + [Test] + public void Test_TuplePack_SerializersOfT() + { + Slice prefix = Slice.FromString("ABC"); + { + var serializer = TupleSerializer.Default; + var t = STuple.Create(123); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<15>{")); + } + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo"); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00>")); + } + + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo", 123); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00><15>{")); + } + + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo", false, 123); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00><14><15>{")); + } + + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo", false, 123, -1L); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00><14><15>{<13>")); + } + + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo", false, 123, -1L, "narf"); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00><14><15>{<13><02>narf<00>")); + } + + { + var serializer = TupleSerializer.Default; + var t = STuple.Create("foo", false, 123, -1L, "narf", Math.PI); + var tw = new TupleWriter(); + tw.Output.WriteBytes(prefix); + serializer.PackTo(ref tw, in t); + Assert.That(tw.ToSlice().ToString(), Is.EqualTo("ABC<02>foo<00><14><15>{<13><02>narf<00>!<09>!TD-<18>")); + } + + } + [Test] + public void Test_TuplePack_Unpack() + { + + var packed = TuPack.EncodeKey("hello world"); + Log(packed); + + var tuple = TuPack.Unpack(packed); + Assert.That(tuple, Is.Not.Null); + Log(tuple); + Assert.That(tuple.Count, Is.EqualTo(1)); + Assert.That(tuple.Get(0), Is.EqualTo("hello world")); + + packed = TuPack.EncodeKey("hello world", 123); + Log(packed); + + tuple = TuPack.Unpack(packed); + Assert.That(tuple, Is.Not.Null); + Log(tuple); + Assert.That(tuple.Count, Is.EqualTo(2)); + Assert.That(tuple.Get(0), Is.EqualTo("hello world")); + Assert.That(tuple.Get(1), Is.EqualTo(123)); + + packed = TuPack.EncodeKey(1, 256, 257, 65536, int.MaxValue, long.MaxValue); + Log(packed); + + tuple = TuPack.Unpack(packed); + Assert.That(tuple, Is.Not.Null); + Assert.That(tuple.Count, Is.EqualTo(6)); + Assert.That(tuple.Get(0), Is.EqualTo(1)); + Assert.That(tuple.Get(1), Is.EqualTo(256)); + Assert.That(tuple.Get(2), Is.EqualTo(257), ((SlicedTuple) tuple).GetSlice(2).ToString()); + Assert.That(tuple.Get(3), Is.EqualTo(65536)); + Assert.That(tuple.Get(4), Is.EqualTo(int.MaxValue)); + Assert.That(tuple.Get(5), Is.EqualTo(long.MaxValue)); + + packed = TuPack.EncodeKey(-1, -256, -257, -65536, int.MinValue, long.MinValue); + Log(packed); + + tuple = TuPack.Unpack(packed); + Assert.That(tuple, Is.Not.Null); + Assert.That(tuple, Is.InstanceOf()); + Log(tuple); + Assert.That(tuple.Count, Is.EqualTo(6)); + Assert.That(tuple.Get(0), Is.EqualTo(-1)); + Assert.That(tuple.Get(1), Is.EqualTo(-256)); + Assert.That(tuple.Get(2), Is.EqualTo(-257), "Slice is " + ((SlicedTuple) tuple).GetSlice(2).ToString()); + Assert.That(tuple.Get(3), Is.EqualTo(-65536)); + Assert.That(tuple.Get(4), Is.EqualTo(int.MinValue)); + Assert.That(tuple.Get(5), Is.EqualTo(long.MinValue)); + } + + [Test] + public void Test_TuplePack_DecodeKey() + { + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00>")), + Is.EqualTo("hello world") + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<15>{")), + Is.EqualTo(123) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello<00><02>world<00>")), + Is.EqualTo(STuple.Create("hello", "world")) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{")), + Is.EqualTo(STuple.Create("hello world", 123)) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><16><04><12>-")), + Is.EqualTo(STuple.Create("hello world", 1234, -1234L)) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14>")), + Is.EqualTo(STuple.Create("hello world", 123, false)) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>")), + Is.EqualTo(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }.AsSlice())) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>")), + Is.EqualTo(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }.AsSlice(), Math.PI)) + ); + Assert.That( + TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-")), + Is.EqualTo(STuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }.AsSlice(), Math.PI, -1234L)) + ); + //TODO: if/when we have tuples with 7 or 8 items... + //Assert.That( + // TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>")), + // Is.EqualTo(STuple.Create("hello world", 123, false, Slice.Create(new byte[] { 123, 1, 66, 0, 42 }), Math.PI, -1234L, "こんにちは世界")) + //); + //Assert.That( + // TuPack.DecodeKey(Slice.Unescape("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><15><01>")), + // Is.EqualTo(STuple.Create("hello world", 123, false, Slice.Create(new byte[] { 123, 1, 66, 0, 42 }), Math.PI, -1234L, "こんにちは世界", true)) + //); + } + + [Test] + public void Test_TuplePack_Serialize_ITupleFormattable() + { + // types that implement ITupleFormattable should be packed by calling ToTuple() and then packing the returned tuple + + Slice packed; + + packed = TuplePacker.Serialize(new Thing {Foo = 123, Bar = "hello"}); + Assert.That(packed.ToString(), Is.EqualTo("<03><15>{<02>hello<00><00>")); + + packed = TuplePacker.Serialize(new Thing()); + Assert.That(packed.ToString(), Is.EqualTo("<03><14><00><00>")); + + packed = TuplePacker.Serialize(default(Thing)); + Assert.That(packed.ToString(), Is.EqualTo("<00>")); + + } + + [Test] + public void Test_TuplePack_Deserialize_ITupleFormattable() + { + Slice slice; + Thing thing; + + slice = Slice.Unescape("<03><16><01><02>world<00><00>"); + thing = TuplePackers.DeserializeFormattable(slice); + Assert.That(thing, Is.Not.Null); + Assert.That(thing.Foo, Is.EqualTo(456)); + Assert.That(thing.Bar, Is.EqualTo("world")); + + slice = Slice.Unescape("<03><14><00><00>"); + thing = TuplePackers.DeserializeFormattable(slice); + Assert.That(thing, Is.Not.Null); + Assert.That(thing.Foo, Is.EqualTo(0)); + Assert.That(thing.Bar, Is.EqualTo(null)); + + slice = Slice.Unescape("<00>"); + thing = TuplePackers.DeserializeFormattable(slice); + Assert.That(thing, Is.Null); + } + + [Test] + public void Test_TuplePack_EncodeKeys_Of_T() + { + Slice[] slices; + + #region PackRange(Tuple, ...) + + var tuple = STuple.Create("hello"); + int[] items = new int[] {1, 2, 3, 123, -1, int.MaxValue}; + + // array version + slices = TuPack.EncodePrefixedKeys(tuple, items); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + + // IEnumerable version that is passed an array + slices = TuPack.EncodePrefixedKeys(tuple, (IEnumerable) items); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + + // IEnumerable version but with a "real" enumerable + slices = TuPack.EncodePrefixedKeys(tuple, items.Select(t => t)); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + + #endregion + + #region PackRange(Slice, ...) + + string[] words = {"hello", "world", "très bien", "断トツ", "abc\0def", null, String.Empty}; + + var merged = TuPack.EncodePrefixedKeys(Slice.FromByte(42), words); + Assert.That(merged, Is.Not.Null); + Assert.That(merged.Length, Is.EqualTo(words.Length)); + + for (int i = 0; i < words.Length; i++) + { + var expected = Slice.FromByte(42) + TuPack.EncodeKey(words[i]); + Assert.That(merged[i], Is.EqualTo(expected)); + + Assert.That(merged[i].Array, Is.SameAs(merged[0].Array), "All slices should be stored in the same buffer"); + if (i > 0) Assert.That(merged[i].Offset, Is.EqualTo(merged[i - 1].Offset + merged[i - 1].Count), "All slices should be contiguous"); + } + + // corner cases + // ReSharper disable AssignNullToNotNullAttribute + Assert.That( + () => TuPack.EncodePrefixedKeys(Slice.Empty, default(int[])), + Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); + Assert.That( + () => TuPack.EncodePrefixedKeys(Slice.Empty, default(IEnumerable)), + Throws.InstanceOf().With.Property("ParamName").EqualTo("keys")); + // ReSharper restore AssignNullToNotNullAttribute + + #endregion + } + + [Test] + public void Test_TuplePack_EncodeKeys_Boxed() + { + Slice[] slices; + var tuple = STuple.Create("hello"); + object[] items = {"world", 123, false, Guid.NewGuid(), long.MinValue}; + + // array version + slices = TuPack.EncodePrefixedKeys(tuple, items); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + + // IEnumerable version that is passed an array + slices = TuPack.EncodePrefixedKeys(tuple, (IEnumerable) items); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + + // IEnumerable version but with a "real" enumerable + slices = TuPack.EncodePrefixedKeys(tuple, items.Select(t => t)); + Assert.That(slices, Is.Not.Null); + Assert.That(slices.Length, Is.EqualTo(items.Length)); + Assert.That(slices, Is.EqualTo(items.Select(x => TuPack.Pack(tuple.Append(x))))); + } + + [Test] + public void Test_TuplePack_Unpack_First_And_Last() + { + // should only work with tuples having at least one element + + Slice packed; + + packed = TuPack.EncodeKey(1); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("1")); + + packed = TuPack.EncodeKey(1, 2); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(2)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("2")); + + packed = TuPack.EncodeKey(1, 2, 3); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(3)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("3")); + + packed = TuPack.EncodeKey(1, 2, 3, 4); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(4)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("4")); + + packed = TuPack.EncodeKey(1, 2, 3, 4, 5); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(5)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("5")); + + packed = TuPack.EncodeKey(1, 2, 3, 4, 5, 6); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(6)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("6")); + + packed = TuPack.EncodeKey(1, 2, 3, 4, 5, 6, 7); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(7)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("7")); + + packed = TuPack.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeFirst(packed), Is.EqualTo("1")); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo(8)); + Assert.That(TuPack.DecodeLast(packed), Is.EqualTo("8")); + + Assert.That(() => TuPack.DecodeFirst(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeFirst(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeLast(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeLast(Slice.Empty), Throws.InstanceOf()); + + } + + [Test] + public void Test_TuplePack_UnpackSingle() + { + // should only work with tuples having exactly one element + + Slice packed; + + packed = TuPack.EncodeKey(1); + Assert.That(TuPack.DecodeKey(packed), Is.EqualTo(1)); + Assert.That(TuPack.DecodeKey(packed), Is.EqualTo("1")); + + packed = TuPack.EncodeKey("Hello\0World"); + Assert.That(TuPack.DecodeKey(packed), Is.EqualTo("Hello\0World")); + + Assert.That(() => TuPack.DecodeKey(Slice.Nil), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(Slice.Empty), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3, 4)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3, 4, 5)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3, 4, 5, 6)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3, 4, 5, 6, 7)), Throws.InstanceOf()); + Assert.That(() => TuPack.DecodeKey(TuPack.EncodeKey(1, 2, 3, 4, 5, 6, 7, 8)), Throws.InstanceOf()); + + } + + [Test] + public void Test_TuplePack_ToRange() + { + KeyRange range; + + // ToRange() should add 0x00 and 0xFF to the packed representations of the tuples + // note: we cannot increment the key to get the End key, because it conflicts with the Tuple Binary Encoding itself + + // Slice + range = TuPack.ToRange(Slice.FromString("ABC")); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<00>"), "Begin key should be suffixed by 0x00"); + Assert.That(range.End.ToString(), Is.EqualTo("ABC"), "End key should be suffixed by 0xFF"); + + // Tuples + + range = TuPack.ToRange(STuple.Create("Hello")); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00>")); + + range = TuPack.ToRange(STuple.Create("Hello", 123)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01>")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true, -1234L)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true, -1234L, "こんにちは世界")); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18>")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI, false)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14>")); + + range = TuPack.ToRange(STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI, false, "TheEnd")); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><02>TheEnd<00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><02>TheEnd<00>")); + } + + [Test] + public void Test_TuplePack_ToRange_With_Prefix() + { + Slice prefix = Slice.FromString("ABC"); + KeyRange range; + + range = TuPack.ToRange(prefix, STuple.Create("Hello")); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00>")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123)); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true)); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01>")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true, -1234L)); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true, -1234L, "こんにちは世界")); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI)); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18>")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI, false)); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14>")); + + range = TuPack.ToRange(prefix, STuple.Create("Hello", 123, true, -1234L, "こんにちは世界", Math.PI, false, "TheEnd")); + Assert.That(range.Begin.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><02>TheEnd<00><00>")); + Assert.That(range.End.ToString(), Is.EqualTo("ABC<02>Hello<00><15>{<15><01><12>-<02><81><93><82><93><81><81><81><96><95><8C><00>!<09>!TD-<18><14><02>TheEnd<00>")); + + // Nil or Empty prefix should not add anything + + range = TuPack.ToRange(Slice.Nil, STuple.Create("Hello", 123)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{")); + + range = TuPack.ToRange(Slice.Empty, STuple.Create("Hello", 123)); + Assert.That(range.Begin.ToString(), Is.EqualTo("<02>Hello<00><15>{<00>")); + Assert.That(range.End.ToString(), Is.EqualTo("<02>Hello<00><15>{")); + + } + + private class Thing : ITupleFormattable + { + public int Foo { get; set; } + public string Bar { get; set; } + + ITuple ITupleFormattable.ToTuple() + { + return STuple.Create(this.Foo, this.Bar); + } + + void ITupleFormattable.FromTuple(ITuple tuple) + { + this.Foo = tuple.Get(0); + this.Bar = tuple.Get(1); + } + } + +#if ENABLE_VALUETUPLES + + [Test] + public void Test_TuPack_ValueTuple_Pack() + { + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world")).ToString(), + Is.EqualTo("<02>hello world<00>") + ); + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world", 123)).ToString(), + Is.EqualTo("<02>hello world<00><15>{") + ); + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world", 123, false)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14>") + ); + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 })).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>") + ); + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18>") + ); + Assert.That( + TuPack.Pack(ValueTuple.Create("hello world", 123, false, new byte[] { 123, 1, 66, 0, 42 }, Math.PI, -1234L)).ToString(), + Is.EqualTo("<02>hello world<00><15>{<14><01>{<01>B<00>*<00>!<09>!TD-<18><12>-") + ); + + { // Embedded Tuples + var packed = TuPack.Pack(ValueTuple.Create("hello", ValueTuple.Create(123, false), "world")); + Assert.That( + packed.ToString(), + Is.EqualTo("<02>hello<00><03><15>{<14><00><02>world<00>") + ); + var t = TuPack.DecodeKey, string>(packed); + Assert.That(t.Item1, Is.EqualTo("hello")); + Assert.That(t.Item2.Item1, Is.EqualTo(123)); + Assert.That(t.Item2.Item2, Is.False); + Assert.That(t.Item3, Is.EqualTo("world")); + } + + } + +#endif + + } + + +} From 26775859320d7a4c11f27c1df9580d7633166612 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 18:10:28 +0200 Subject: [PATCH 106/153] Add reference to System.ValueTuples v4.4 and enable support for ValueTuples in the core tuple API --- FoundationDB.Client/FoundationDB.Client.csproj | 4 ++++ .../Layers/Tuples/Encoding/TuplePackers.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`1.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`2.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`3.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`4.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`5.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 2 +- FoundationDB.Client/Layers/Tuples/TuPack.cs | 2 +- .../Layers/Tuples/TupleExtensions.cs | 2 +- FoundationDB.Tests/Utils/TupleFacts.cs | 16 ++++++++-------- 12 files changed, 22 insertions(+), 18 deletions(-) diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 01a1a8731..3c9e4e746 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -32,4 +32,8 @@ latest + + + + diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index a8e8cc9b6..6e47b4996 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples.Encoding { diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index 10967f47d..bef108e9f 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs index bddd4f35f..250753afe 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs index 308974fc5..ce5772996 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs index 22def275c..7fec066f7 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs index 02df7669c..63ee35fcd 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index 1056f8604..ce6fa6ecb 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs index b7f0831a6..e26cf24f5 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index d0f6b687a..7dba29c01 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs index 2e19f7fcf..19bb2b22f 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace Doxense.Collections.Tuples { diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index 3c63fe6c8..d0610e384 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLE +#define ENABLE_VALUETUPLES // ReSharper disable AccessToModifiedClosure namespace Doxense.Collections.Tuples.Tests @@ -121,7 +121,7 @@ public void Test_Tuple_2() Assert.That(item1, Is.EqualTo("hello world")); Assert.That(item2, Is.EqualTo(123)); } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2) = t2; Assert.That(item1, Is.EqualTo("hello world")); @@ -174,7 +174,7 @@ public void Test_Tuple_3() Assert.That(item2, Is.EqualTo(123)); Assert.That(item3, Is.False); } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3) = t3; Assert.That(item1, Is.EqualTo("hello world")); @@ -235,7 +235,7 @@ public void Test_Tuple_4() Assert.That(item3, Is.False); Assert.That(item4, Is.EqualTo(1234L)); } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4) = t4; Assert.That(item1, Is.EqualTo("hello world")); @@ -287,7 +287,7 @@ public void Test_Tuple_5() Assert.That(item4, Is.EqualTo(1234L)); Assert.That(item5, Is.EqualTo(-1234L)); } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4, long item5) = t5; Assert.That(item1, Is.EqualTo("hello world")); @@ -344,7 +344,7 @@ public void Test_Tuple_6() Assert.That(item5, Is.EqualTo(-1234L)); Assert.That(item6, Is.EqualTo("six")); } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4, long item5, string item6) = t6; Assert.That(item1, Is.EqualTo("hello world")); @@ -1936,7 +1936,7 @@ void ITupleFormattable.FromTuple(ITuple tuple) #region System.ValueTuple integration... -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES [Test] public void Test_Implicit_Cast_STuple_To_ValueTuple() @@ -2096,7 +2096,7 @@ public void Test_Deconstruct_STuple() } } -#if ENABLE_VALUETUPLE +#if ENABLE_VALUETUPLES [Test] public void Test_Deconstruct_STuple_TupleSyntax() From 61f3716125b0863ce823f788f40e4c1eef0f2777 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 18:11:41 +0200 Subject: [PATCH 107/153] Add first-class support for ValueTuples with subspaces - Add overloads that take ValueTuple<...> to most IKeySubspaces implementations - Add quick indexers on typed keyspace that take either the items, or a value tuple --- .../Subspaces/DynamicKeySubspace.cs | 45 +++++ FoundationDB.Client/Subspaces/KeySubspace.cs | 1 + .../Subspaces/TypedKeySubspace`1.cs | 89 ++++++++- .../Subspaces/TypedKeySubspace`2.cs | 89 ++++++--- .../Subspaces/TypedKeySubspace`3.cs | 59 +++++- .../Subspaces/TypedKeySubspace`4.cs | 66 ++++++- FoundationDB.Tests/SubspaceFacts.cs | 177 ++++++++++++++++++ 7 files changed, 471 insertions(+), 55 deletions(-) diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 1ae0d821b..7d791f134 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -26,10 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +#define ENABLE_VALUETUPLES + namespace FoundationDB.Client { using System; using System.Collections.Generic; + using System.Diagnostics; using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; @@ -72,6 +75,7 @@ public Slice this[ITuple item] } /// Key helper for a dynamic TypeSystem + [DebuggerDisplay("{Parent.ToString(),nq)}")] public sealed class DynamicKeys { @@ -92,6 +96,7 @@ internal DynamicKeys(DynamicKeySubspace parent, IDynamicKeyEncoder encoder) /// Convert a tuple into a key of this subspace /// Tuple that will be packed and appended to the subspace prefix + [Pure] public Slice Pack([NotNull] TTuple tuple) where TTuple : ITuple { @@ -102,6 +107,46 @@ public Slice Pack([NotNull] TTuple tuple) return sw.ToSlice(); } +#if ENABLE_VALUETUPLES + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(ValueTuple items) + { + return Encode(items.Item1); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2) items) + { + return Encode(items.Item1, items.Item2); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2, T3) items) + { + return Encode(items.Item1, items.Item2, items.Item3); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2, T3, T4) items) + { + return Encode(items.Item1, items.Item2, items.Item3, items.Item4); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2, T3, T4, T5) items) + { + return Encode(items.Item1, items.Item2, items.Item3, items.Item4, items.Item5); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2, T3, T4, T5, T6) items) + { + return Encode(items.Item1, items.Item2, items.Item3, items.Item4, items.Item5, items.Item6); + } + +#endif + /// Unpack a key of this subspace, back into a tuple /// Key that was produced by a previous call to /// Original tuple diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 639394a5e..7c74c6db3 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -42,6 +42,7 @@ namespace FoundationDB.Client /// Adds a prefix on every keys, to group them inside a common subspace [PublicAPI] + [DebuggerDisplay("{ToString(),nq}")] public class KeySubspace : IKeySubspace, IEquatable, IComparable { diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs index 5a1ae7279..35acb37c3 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs @@ -26,14 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +#define ENABLE_VALUETUPLES + namespace FoundationDB.Client { using System; + using System.Diagnostics; + using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using Doxense.Memory; using Doxense.Serialization.Encoders; - using FoundationDB.Layers.Directories; using JetBrains.Annotations; [PublicAPI] @@ -69,6 +71,7 @@ internal TypedKeySubspace(Slice prefix, [NotNull] IKeyEncoder encoder) /// Encodes and Decodes keys composed of a single element /// Type of the key handled by this subspace + [DebuggerDisplay("{Parent.ToString(),nq)}")] public sealed class TypedKeys { @@ -87,25 +90,84 @@ internal TypedKeys( this.Encoder = encoder; } + #region ToRange() + + /// Return the range of all legal keys in this subpsace + /// A "legal" key is one that can be decoded into the original pair of values + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange() + { + return this.Parent.ToRange(); + } + + /// Return the range of all legal keys in this subpsace, that start with the specified value + /// Range that encompass all keys that start with (tuple.Item1, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(STuple tuple) + { + return ToRange(tuple.Item1); + } + +#if ENABLE_VALUETUPLES + /// Return the range of all legal keys in this subpsace, that start with the specified value + /// Range that encompass all keys that start with (tuple.Item1, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public KeyRange ToRange(ValueTuple tuple) + { + return ToRange(tuple.Item1); + } +#endif + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public KeyRange ToRange(T1 item1) { - //HACKHACK: add concept of "range" on IKeyEncoder ? - var prefix = Encode(item1); - return KeyRange.PrefixedBy(prefix); + //TODO: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(Encode(item1)); } - [Pure] + #endregion + + #region Pack() + +#if ENABLE_VALUETUPLES + public Slice this[ValueTuple items] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(items.Item1); + } +#endif + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(STuple tuple) { return Encode(tuple.Item1); } - [Pure] - public Slice Pack([NotNull] ITuple tuple) +#if ENABLE_VALUETUPLES + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack(ValueTuple tuple) + { + return Encode(tuple.Item1); + } +#endif + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack([NotNull] TTuple tuple) + where TTuple : ITuple { return Encode(tuple.OfSize(1).Get(0)); } + #endregion + + #region Encode() + + public Slice this[T1 item1] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(item1); + } + [Pure] public Slice Encode(T1 item1) { @@ -115,6 +177,10 @@ public Slice Encode(T1 item1) return sw.ToSlice(); } + #endregion + + #region Decode() + [Pure] public T1 Decode(Slice packedKey) { @@ -126,6 +192,10 @@ public void Decode(Slice packedKey, out T1 item1) item1 = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } + #endregion + + #region Dump() + /// Return a user-friendly string representation of a key of this subspace [Pure] public string Dump(Slice packedKey) @@ -144,7 +214,8 @@ public string Dump(Slice packedKey) } } - } + #endregion + } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs index f4e064c04..560892006 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -26,11 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace FoundationDB.Client { using System; + using System.Diagnostics; using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; @@ -66,6 +67,7 @@ internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder e } + [DebuggerDisplay("{Parent.ToString(),nq)}")] public sealed class TypedKeys { @@ -84,6 +86,8 @@ internal TypedKeys( this.Encoder = encoder; } + #region ToRange() + /// Return the range of all legal keys in this subpsace /// A "legal" key is one that can be decoded into the original pair of values [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -100,6 +104,7 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2); } +#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified pair of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, ..) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -107,16 +112,21 @@ public KeyRange ToRange((T1, T2) tuple) { return ToRange(tuple.Item1, tuple.Item2); } +#endif /// Return the range of all legal keys in this subpsace, that start with the specified pair of values /// Range that encompass all keys that start with (item1, item2, ..) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public KeyRange ToRange(T1 item1, T2 item2) { - //HACKHACK: add concept of "range" on IKeyEncoder ? - var prefix = Encode(item1, item2); - return KeyRange.PrefixedBy(prefix); + //TODO: add concept of "range" on IKeyEncoder ? + return KeyRange.PrefixedBy(Encode(item1, item2)); } + #endregion + + #region ToRangePartial() + /// Return the range of all legal keys in this subpsace, that start with the specified first item /// Range that encompass all keys that start with (tuple.Item1, ..) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -133,6 +143,10 @@ public KeyRange ToRangePartial(T1 item1) return KeyRange.PrefixedBy(EncodePartial(item1)); } + #endregion + + #region Pack() + /// Pack a 2-tuple into a key in this subspace /// Pair of values /// Encoded key in this subspace @@ -149,8 +163,8 @@ public Slice Pack(STuple tuple) /// Pack a 2-tuple into a key in this subspace /// Pair of values /// Encoded key in this subspace - [Pure] - public Slice Pack(ValueTuple tuple) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack((T1, T2) tuple) { return Encode(tuple.Item1, tuple.Item2); } @@ -159,36 +173,37 @@ public Slice Pack(ValueTuple tuple) /// Pack a 2-tuple into a key in this subspace /// Tuple that must be of size 2 /// Encoded key in this subspace - [Pure] - public Slice Pack([NotNull] ITuple tuple) + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public Slice Pack([NotNull] TTuple tuple) + where TTuple : ITuple { tuple.OfSize(2); return Encode(tuple.Get(0), tuple.Get(1)); } - /// Pack a partial key only containing the first item of a key - /// Tuple containing a single item - /// Encoded partial key, to be used for generationg key ranges or key selectors - [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public Slice PackPartial(STuple tuple) + #endregion + + #region Encode() + + public Slice this[T1 item1, T2 item2] { - return EncodePartial(tuple.Item1); + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(item1, item2); } - /// Pack a partial key only containing the first item of a key - /// Tuple containing a single item - /// Encoded partial key, to be used for generationg key ranges or key selectors - [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public Slice PackPartial(ValueTuple tuple) +#if ENABLE_VALUETUPLES + public Slice this[(T1, T2) items] { - return EncodePartial(tuple.Item1); + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(items.Item1, items.Item2); } +#endif /// Encode a pair of values into a key in this subspace /// First part of the key /// Second part of the key /// Encoded key in this subspace - /// The key can be decoded back into its original components using or + /// The key can be decoded back into its original components using or [Pure] public Slice Encode(T1 item1, T2 item2) { @@ -198,6 +213,10 @@ public Slice Encode(T1 item1, T2 item2) return sw.ToSlice(); } + #endregion + + #region EncodePartial() + [Pure] public Slice EncodePartial(T1 item1) { @@ -207,27 +226,37 @@ public Slice EncodePartial(T1 item1) return sw.ToSlice(); } + #endregion + + #region Decode() + [Pure] + //REVIEW: => Unpack()? //REVIEW: return ValueTuple<..> instead? (C#7) - public STuple Decode(Slice packedKey) //REVIEW: => Unpack() + public STuple Decode(Slice packedKey) { return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); } - [Pure] - public T1 DecodePartial(Slice packedKey) + public void Decode(Slice packedKey, out T1 item1, out T2 item2) { - var parts = this.Encoder.DecodeKeyParts(1, packedKey); - return parts.Item1; + this.Encoder + .DecodeKey(this.Parent.ExtractKey(packedKey)) + .Deconstruct(out item1, out item2); } - public void Decode(Slice packedKey, out T1 item1, out T2 item2) + #endregion + + #region DecodePartial() + + [Pure] + public T1 DecodePartial(Slice packedKey) { - var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); - item1 = tuple.Item1; - item2 = tuple.Item2; + return this.Encoder.DecodeKeyParts(1, packedKey).Item1; } + #endregion + /// Return a user-friendly string representation of a key of this subspace [Pure] public string Dump(Slice packedKey) diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index 6cd64b2c1..cc1089671 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -26,11 +26,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace FoundationDB.Client { using System; + using System.Diagnostics; using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; @@ -65,6 +66,7 @@ internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder { @@ -83,6 +85,8 @@ internal TypedKeys( this.Encoder = encoder; } + #region ToRange() + /// Return the range of all legal keys in this subpsace /// A "legal" key is one that can be decoded into the original triple of values [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -99,13 +103,15 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } +#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public KeyRange ToRange(ValueTuple tuple) + public KeyRange ToRange((T1, T2, T3) tuple) { return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } +#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -115,6 +121,10 @@ public KeyRange ToRange(T1 item1, T2 item2, T3 item3) return KeyRange.PrefixedBy(Encode(item1, item2, item3)); } + #endregion + + #region ToRangePartial() + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) public KeyRange ToRangePartial(STuple tuple) @@ -123,6 +133,7 @@ public KeyRange ToRangePartial(STuple tuple) return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); } +#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) public KeyRange ToRangePartial(ValueTuple tuple) @@ -130,6 +141,7 @@ public KeyRange ToRangePartial(ValueTuple tuple) //HACKHACK: add concept of "range" on IKeyEncoder ? return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); } +#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -147,6 +159,10 @@ public KeyRange ToRangePartial(T1 item1) return KeyRange.PrefixedBy(EncodePartial(item1)); } + #endregion + + #region Pack() + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(STuple tuple) { @@ -155,7 +171,7 @@ public Slice Pack(STuple tuple) #if ENABLE_VALUETUPLES [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public Slice Pack(ValueTuple tuple) + public Slice Pack((T1, T2, T3) tuple) { return Encode(tuple.Item1, tuple.Item2, tuple.Item3); } @@ -169,6 +185,24 @@ public Slice Pack(TTuple tuple) return Encode(tuple.Get(0), tuple.Get(1), tuple.Get(2)); } + #endregion + + #region Encode() + + public Slice this[T1 item1, T2 item2, T3 item3] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(item1, item2, item3); + } + +#if ENABLE_VALUETUPLES + public Slice this[(T1, T2, T3) items] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(items.Item1, items.Item2, items.Item3); + } +#endif + [Pure] public Slice Encode(T1 item1, T2 item2, T3 item3) { @@ -196,7 +230,13 @@ public Slice EncodePartial(T1 item1) return sw.ToSlice(); } + #endregion + + #region Decode() + [Pure] + //REVIEW: => Unpack()? + //REVIEW: return ValueTuple<..> instead? (C#7) public STuple Decode(Slice packedKey) { return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); @@ -204,12 +244,15 @@ public STuple Decode(Slice packedKey) public void Decode(Slice packedKey, out T1 item1, out T2 item2, out T3 item3) { - var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); - item1 = tuple.Item1; - item2 = tuple.Item2; - item3 = tuple.Item3; + this.Encoder + .DecodeKey(this.Parent.ExtractKey(packedKey)) + .Deconstruct(out item1, out item2, out item3); } + #endregion + + #region Dump() + /// Return a user-friendly string representation of a key of this subspace [Pure] public string Dump(Slice packedKey) @@ -228,6 +271,8 @@ public string Dump(Slice packedKey) } } + #endregion + } } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index 7cf2ee4b6..b4bf83e06 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -26,15 +26,15 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLES +#define ENABLE_VALUETUPLES namespace FoundationDB.Client { using System; + using System.Diagnostics; using System.Runtime.CompilerServices; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using Doxense.Memory; using Doxense.Serialization.Encoders; using JetBrains.Annotations; @@ -65,6 +65,7 @@ internal TypedKeySubspace(Slice prefix, [NotNull] ICompositeKeyEncoder { @@ -83,6 +84,8 @@ internal TypedKeys( this.Encoder = encoder; } + #region ToRange() + /// Return the range of all legal keys in this subpsace /// A "legal" key is one that can be decoded into the original triple of values [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -99,13 +102,15 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } +#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public KeyRange ToRange(ValueTuple tuple) + public KeyRange ToRange((T1, T2, T3, T4) tuple) { return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } +#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -115,6 +120,10 @@ public KeyRange ToRange(T1 item1, T2 item2, T3 item3, T4 item4) return KeyRange.PrefixedBy(Encode(item1, item2, item3, item4)); } + #endregion + + #region ToRangePartial() + /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) public KeyRange ToRangePartial(STuple tuple) @@ -123,13 +132,15 @@ public KeyRange ToRangePartial(STuple tuple) return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); } +#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) - public KeyRange ToRangePartial(ValueTuple tuple) + public KeyRange ToRangePartial((T1, T2, T3) tuple) { //HACKHACK: add concept of "range" on IKeyEncoder ? return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); } +#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -155,6 +166,24 @@ public KeyRange ToRangePartial(T1 item1) return KeyRange.PrefixedBy(EncodePartial(item1)); } + #endregion + + #region Pack() + + public Slice this[T1 item1, T2 item2, T3 item3, T4 item4] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(item1, item2, item3, item4); + } + +#if ENABLE_VALUETUPLES + public Slice this[(T1, T2, T3, T4) items] + { + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + get => Encode(items.Item1, items.Item2, items.Item3, items.Item4); + } +#endif + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(STuple tuple) { @@ -177,6 +206,10 @@ public Slice Pack(TTuple tuple) return Encode(tuple.Get(0), tuple.Get(1), tuple.Get(2), tuple.Get(3)); } + #endregion + + #region Encode() + [Pure] public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4) { @@ -186,6 +219,11 @@ public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4) return sw.ToSlice(); } + #endregion + + + #region EncodePartial() + [Pure] public Slice EncodePartial(T1 item1, T2 item2, T3 item3) { @@ -213,7 +251,13 @@ public Slice EncodePartial(T1 item1) return sw.ToSlice(); } + #endregion + + #region Decode() + [Pure] + //REVIEW: => Unpack()? + //REVIEW: return ValueTuple<..> instead? (C#7) public STuple Decode(Slice packedKey) { return this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); @@ -221,13 +265,15 @@ public STuple Decode(Slice packedKey) public void Decode(Slice packedKey, out T1 item1, out T2 item2, out T3 item3, out T4 item4) { - var tuple = this.Encoder.DecodeKey(this.Parent.ExtractKey(packedKey)); - item1 = tuple.Item1; - item2 = tuple.Item2; - item3 = tuple.Item3; - item4 = tuple.Item4; + this.Encoder + .DecodeKey(this.Parent.ExtractKey(packedKey)) + .Deconstruct(out item1, out item2, out item3, out item4); } + #endregion + + #region Dump() + /// Return a user-friendly string representation of a key of this subspace [Pure] public string Dump(Slice packedKey) @@ -246,6 +292,8 @@ public string Dump(Slice packedKey) } } + #endregion + } } diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 7a3db250b..e095b9053 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -81,6 +81,12 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(t2.Get(0), Is.EqualTo("world")); Assert.That(t2.Get(1), Is.EqualTo(123)); Assert.That(t2.Get(2), Is.False); + + // ValueTuple + Assert.That(subspace.Keys.Pack(new ValueTuple("hello")).ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); + Assert.That(subspace.Keys.Pack(("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); + Assert.That(subspace.Keys.Pack(("hello", 123, "world")).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{<02>world<00>")); + Assert.That(subspace.Keys.Pack(("hello", 123, "world", 456)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{<02>world<00><16><01>")); } [Test] @@ -170,6 +176,177 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() Assert.That(child.Partition[Slice.Empty].GetPrefix(), Is.EqualTo(child.GetPrefix())); } + [Test] + public void Test_DynamicKeySpace_API() + { + var location = new KeySubspace(Slice.FromString("PREFIX")).Using(TypeSystem.Tuples); + + Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); + + // Encode(...) + Assert.That(location.Keys.Encode("hello").ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + Assert.That(location.Keys.Encode("hello", 123).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + Assert.That(location.Keys.Encode("hello", 123, "world").ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + Assert.That(location.Keys.Encode("hello", 123, "world", 456).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + Assert.That(location.Keys.Encode("hello", 123, "world", 456, "!").ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")); + Assert.That(location.Keys.Encode("hello", 123, "world", 456, "!", 789).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")); + + // Pack(ITuple) + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello")).ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world", 456)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world", 456, "!")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")); + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world", 456, "!", 789)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")); + + // Pack(ValueTuple) + Assert.That(location.Keys.Pack(ValueTuple.Create("hello")).ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + Assert.That(location.Keys.Pack(("hello", 123)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + Assert.That(location.Keys.Pack(("hello", 123, "world")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + Assert.That(location.Keys.Pack(("hello", 123, "world", 456)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + Assert.That(location.Keys.Pack(("hello", 123, "world", 456, "!")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")); + Assert.That(location.Keys.Pack(("hello", 123, "world", 456, "!", 789)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")); + + // ITuple Unpack(Slice) + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00>")), Is.EqualTo(STuple.Create("hello"))); + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00><15>{")), Is.EqualTo(STuple.Create("hello", 123))); + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>")), Is.EqualTo(STuple.Create("hello", 123, "world"))); + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456))); + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!"))); + Assert.That(location.Keys.Unpack(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!", 789))); + + // STuple Decode(Slice) + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00>")), Is.EqualTo("hello")); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{")), Is.EqualTo(STuple.Create("hello", 123))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>")), Is.EqualTo(STuple.Create("hello", 123, "world"))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!"))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!", 789))); + + // DecodeFirst/DecodeLast + Assert.That(location.Keys.DecodeFirst(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo("hello")); + Assert.That(location.Keys.DecodeLast(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo(789)); + + } + + [Test] + public void Test_TypedKeySpace_T1() + { + var location = new KeySubspace(Slice.FromString("PREFIX")) + .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + + // shortcuts + Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); + Assert.That(location.Keys["hello"].ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + Assert.That(location.Keys[ValueTuple.Create("hello")].ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + + // Encode(...) + Assert.That(location.Keys.Encode("hello").ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + + // Pack(ITuple) + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello")).ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + + // Pack(ValueTuple) + Assert.That(location.Keys.Pack(ValueTuple.Create("hello")).ToString(), Is.EqualTo("PREFIX<02>hello<00>")); + + // STuple Decode(...) + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00>")), Is.EqualTo("hello")); + + // Decode(..., out T) + location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00>"), out string x); + Assert.That(x, Is.EqualTo("hello")); + } + + [Test] + public void Test_TypedKeySpace_T2() + { + var location = new KeySubspace(Slice.FromString("PREFIX")) + .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + + // shortcuts + Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); + Assert.That(location.Keys["hello", 123].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + Assert.That(location.Keys[("hello", 123)].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + + // Encode(...) + Assert.That(location.Keys.Encode("hello", 123).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + + // Pack(ITuple) + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + + // Pack(ValueTuple) + Assert.That(location.Keys.Pack(("hello", 123)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{")); + + // STuple Decode(...) + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{")), Is.EqualTo(("hello", 123))); + + // Decode(..., out T) + location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{"), out string x1, out int x2); + Assert.That(x1, Is.EqualTo("hello")); + Assert.That(x2, Is.EqualTo(123)); + } + + [Test] + public void Test_TypedKeySpace_T3() + { + var location = new KeySubspace(Slice.FromString("PREFIX")) + .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + + // shortcuts + Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); + Assert.That(location.Keys["hello", 123, "world"].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + Assert.That(location.Keys[("hello", 123, "world")].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + + // Encode(...) + Assert.That(location.Keys.Encode("hello", 123, "world").ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + + // Pack(ITuple) + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + + // Pack(ValueTuple) + Assert.That(location.Keys.Pack(("hello", 123, "world")).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00>")); + + // STuple Decode(...) + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>")), Is.EqualTo(("hello", 123, "world"))); + + // Decode(..., out T) + location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>"), out string x1, out int x2, out string x3); + Assert.That(x1, Is.EqualTo("hello")); + Assert.That(x2, Is.EqualTo(123)); + Assert.That(x3, Is.EqualTo("world")); + } + + [Test] + public void Test_TypedKeySpace_T4() + { + var location = new KeySubspace(Slice.FromString("PREFIX")) + .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + + // shortcuts + Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); + Assert.That(location.Keys["hello", 123, "world", 456].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + Assert.That(location.Keys[("hello", 123, "world", 456)].ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + + // Encode(...) + Assert.That(location.Keys.Encode("hello", 123, "world", 456).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + + // Pack(ITuple) + Assert.That(location.Keys.Pack((ITuple) STuple.Create("hello", 123, "world", 456)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + + // Pack(ValueTuple) + Assert.That(location.Keys.Pack(("hello", 123, "world", 456)).ToString(), Is.EqualTo("PREFIX<02>hello<00><15>{<02>world<00><16><01>")); + + // STuple Decode(...) + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>")), Is.EqualTo(("hello", 123, "world", 456))); + + // Decode(..., out T) + location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>"), out string x1, out int x2, out string x3, out int x4); + Assert.That(x1, Is.EqualTo("hello")); + Assert.That(x2, Is.EqualTo(123)); + Assert.That(x3, Is.EqualTo("world")); + Assert.That(x4, Is.EqualTo(456)); + } + } } From cf764bd036157279d0e97d8c605ed815c65d9a84 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 18:20:41 +0200 Subject: [PATCH 108/153] Use new indexer on TypedKeySubspace.Keys --- .../Collections/FdbMap`2.cs | 10 +++---- .../Collections/FdbMultimap`2.cs | 27 ++++++++----------- .../Counters/FdbCounterMap.cs | 6 ++--- .../Indexes/FdbIndex`2.cs | 8 +++--- .../Documents/FdbDocumentCollection.cs | 2 +- .../Indexes/FdbCompressedBitmapIndex.cs | 10 +++---- 6 files changed, 29 insertions(+), 34 deletions(-) diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index 9cc4bdbcb..f4a7eadc2 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -91,7 +91,7 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var data = await trans.GetAsync(this.Location.Keys.Encode(id)).ConfigureAwait(false); + var data = await trans.GetAsync(this.Location.Keys[id]).ConfigureAwait(false); if (data.IsNull) throw new KeyNotFoundException("The given id was not present in the map."); return this.ValueEncoder.DecodeValue(data); @@ -106,7 +106,7 @@ public async Task> TryGetAsync([NotNull] IFdbReadOnlyTransactio if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var data = await trans.GetAsync(this.Location.Keys.Encode(id)).ConfigureAwait(false); + var data = await trans.GetAsync(this.Location.Keys[id]).ConfigureAwait(false); if (data.IsNull) return default(Optional); return this.ValueEncoder.DecodeValue(data); @@ -122,7 +122,7 @@ public void Set([NotNull] IFdbTransaction trans, TKey id, TValue value) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - trans.Set(this.Location.Keys.Encode(id), this.ValueEncoder.EncodeValue(value)); + trans.Set(this.Location.Keys[id], this.ValueEncoder.EncodeValue(value)); } /// Remove a single entry from the map @@ -134,7 +134,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey id) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - trans.Clear(this.Location.Keys.Encode(id)); + trans.Clear(this.Location.Keys[id]); } /// Create a query that will attempt to read all the entries in the map within a single transaction. @@ -160,7 +160,7 @@ public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTrans if (trans == null) throw new ArgumentNullException(nameof(trans)); if (ids == null) throw new ArgumentNullException(nameof(ids)); - var results = await trans.GetValuesAsync(ids.Select(id => this.Location.Keys.Encode(id))).ConfigureAwait(false); + var results = await trans.GetValuesAsync(ids.Select(id => this.Location.Keys[id])).ConfigureAwait(false); return Optional.DecodeRange(this.ValueEncoder, results); } diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index a8e37639f..b8ab0a3ed 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -100,7 +100,7 @@ public Task AddAsync([NotNull] IFdbTransaction trans, TKey key, TValue value) //note: this method does not need to be async, but subtract is, so it's better if both methods have the same shape. if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.AtomicAdd(this.Location.Keys.Encode(key, value), PlusOne); + trans.AtomicAdd(this.Location.Keys[key, value], PlusOne); return Task.CompletedTask; } @@ -113,7 +113,7 @@ public async Task SubtractAsync([NotNull] IFdbTransaction trans, TKey key, TValu { if (trans == null) throw new ArgumentNullException(nameof(trans)); - Slice k = this.Location.Keys.Encode(key, value); + Slice k = this.Location.Keys[key, value]; if (this.AllowNegativeValues) { trans.AtomicAdd(k, MinusOne); @@ -141,7 +141,7 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var v = await trans.GetAsync(this.Location.Keys.Encode(key, value)).ConfigureAwait(false); + var v = await trans.GetAsync(this.Location.Keys[key, value]).ConfigureAwait(false); return this.AllowNegativeValues ? v.IsPresent : v.ToInt64() > 0; } @@ -155,7 +155,7 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T { if (trans == null) throw new ArgumentNullException(nameof(trans)); - Slice v = await trans.GetAsync(this.Location.Keys.Encode(key, value)).ConfigureAwait(false); + Slice v = await trans.GetAsync(this.Location.Keys[key, value]).ConfigureAwait(false); if (v.IsNullOrEmpty) return null; long c = v.ToInt64(); return this.AllowNegativeValues || c > 0 ? c : default(long?); @@ -200,22 +200,17 @@ public Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey /// /// [NotNull] - public IAsyncEnumerable> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) + public IAsyncEnumerable<(TValue Value, long Count)> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) { var range = KeyRange.StartsWith(this.Location.Keys.EncodePartial(key)); var query = trans .GetRange(range) - .Select(kvp => new KeyValuePair(this.Location.Keys.Decode(kvp.Key).Item2, kvp.Value.ToInt64())); + .Select(kvp => (Value: this.Location.Keys.Decode(kvp.Key).Item2, Count: kvp.Value.ToInt64())); - if (this.AllowNegativeValues) - { - return query; - } - else - { - return query.Where(kvp => kvp.Value > 0); - } + return this.AllowNegativeValues + ? query + : query.Where(x => x.Count > 0); } /// Returns a dictionary with of the counts of each value for a specific key @@ -225,7 +220,7 @@ public IAsyncEnumerable> GetCounts([NotNull] IFdbRead /// public Task> GetCountsAsync([NotNull] IFdbReadOnlyTransaction trans, TKey key, IEqualityComparer comparer = null) { - return GetCounts(trans, key).ToDictionaryAsync(comparer); + return GetCounts(trans, key).ToDictionaryAsync(x => x.Value, x => x.Count, comparer); } /// Remove all the values for a specific key @@ -248,7 +243,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key, TValue value) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.Clear(this.Location.Keys.Encode(key, value)); + trans.Clear(this.Location.Keys[key, value]); } #endregion diff --git a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs index 72ffe3de0..91e87daba 100644 --- a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs +++ b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs @@ -77,7 +77,7 @@ public void Add([NotNull] IFdbTransaction transaction, [NotNull] TKey counterKey //REVIEW: we could no-op if value == 0 but this may change conflict behaviour for other transactions... Slice param = value == 1 ? PlusOne : value == -1 ? MinusOne : Slice.FromFixed64(value); - transaction.AtomicAdd(this.Location.Keys.Encode(counterKey), param); + transaction.AtomicAdd(this.Location.Keys[counterKey], param); } /// Subtract a value from a counter in one atomic operation @@ -117,7 +117,7 @@ public void Decrement([NotNull] IFdbTransaction transaction, [NotNull] TKey coun if (transaction == null) throw new ArgumentNullException("transaction"); if (counterKey == null) throw new ArgumentNullException("counterKey"); - var data = await transaction.GetAsync(this.Location.Keys.Encode(counterKey)).ConfigureAwait(false); + var data = await transaction.GetAsync(this.Location.Keys[counterKey]).ConfigureAwait(false); if (data.IsNullOrEmpty) return default(long?); return data.ToInt64(); } @@ -166,7 +166,7 @@ public async Task ReadThenAddAsync([NotNull] IFdbTransaction transaction, if (transaction == null) throw new ArgumentNullException("transaction"); if (counterKey == null) throw new ArgumentNullException("counterKey"); - var key = this.Location.Keys.Encode(counterKey); + var key = this.Location.Keys[counterKey]; var res = await transaction.GetAsync(key).ConfigureAwait(false); long previous = res.IsNullOrEmpty ? 0 : res.ToInt64(); diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index da530aa8f..146fb9812 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -81,7 +81,7 @@ public bool Add([NotNull] IFdbTransaction trans, TId id, TValue value) { if (this.IndexNullValues || value != null) { - trans.Set(this.Location.Keys.Encode(value, id), Slice.Empty); + trans.Set(this.Location.Keys[value, id], Slice.Empty); return true; } return false; @@ -101,13 +101,13 @@ public bool Update([NotNull] IFdbTransaction trans, TId id, TValue newValue, TVa // remove previous value if (this.IndexNullValues || previousValue != null) { - trans.Clear(this.Location.Keys.Encode(previousValue, id)); + trans.Clear(this.Location.Keys[previousValue, id]); } // add new value if (this.IndexNullValues || newValue != null) { - trans.Set(this.Location.Keys.Encode(newValue, id), Slice.Empty); + trans.Set(this.Location.Keys[newValue, id], Slice.Empty); } // cannot be both null, so we did at least something) @@ -124,7 +124,7 @@ public void Remove([NotNull] IFdbTransaction trans, TId id, TValue value) { if (trans == null) throw new ArgumentNullException("trans"); - trans.Clear(this.Location.Keys.Encode(value, id)); + trans.Clear(this.Location.Keys[value, id]); } /// Returns a list of ids matching a specific value diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index 14c8d23d9..1a6153061 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -123,7 +123,7 @@ public void Insert(IFdbTransaction trans, TDocument document) while (remaining > 0) { int sz = Math.Max(remaining, this.ChunkSize); - trans.Set(this.Location.Keys.Encode(id, index), packed.Substring(p, sz)); + trans.Set(this.Location.Keys[id, index], packed.Substring(p, sz)); ++index; p += sz; remaining -= sz; diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index 118e3ba13..f1981697f 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -87,7 +87,7 @@ public async Task AddAsync([NotNull] IFdbTransaction trans, long id, TValu if (this.IndexNullValues || value != null) { - var key = this.Location.Keys.Encode(value); + var key = this.Location.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); var builder = data.HasValue ? new CompressedBitmapBuilder(data) : CompressedBitmapBuilder.Empty; @@ -117,7 +117,7 @@ public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TV // remove previous value if (this.IndexNullValues || previousValue != null) { - var key = this.Location.Keys.Encode(previousValue); + var key = this.Location.Keys[previousValue]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.HasValue) { @@ -130,7 +130,7 @@ public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TV // add new value if (this.IndexNullValues || newValue != null) { - var key = this.Location.Keys.Encode(newValue); + var key = this.Location.Keys[newValue]; var data = await trans.GetAsync(key).ConfigureAwait(false); var builder = data.HasValue ? new CompressedBitmapBuilder(data) : CompressedBitmapBuilder.Empty; builder.Set((int)id); //BUGBUG: 64 bit id! @@ -151,7 +151,7 @@ public async Task RemoveAsync([NotNull] IFdbTransaction trans, long id, TV { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var key = this.Location.Keys.Encode(value); + var key = this.Location.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.HasValue) { @@ -170,7 +170,7 @@ public async Task RemoveAsync([NotNull] IFdbTransaction trans, long id, TV /// List of document ids matching this value for this particular index (can be empty if no document matches) public async Task> LookupAsync([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool reverse = false) { - var key = this.Location.Keys.Encode(value); + var key = this.Location.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.IsNull) return null; if (data.IsEmpty) return Enumerable.Empty(); From 0b7a53d18697ba9d74d607aaadebca4836fb02d4 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Wed, 25 Apr 2018 18:31:43 +0200 Subject: [PATCH 109/153] Use Tuples instead of KeyValuePair in some places --- FoundationDB.Client/Fdb.Bulk.cs | 50 +++++++++++++++++-- FoundationDB.Client/FdbDatabaseExtensions.cs | 10 ++-- FoundationDB.Client/Status/FdbSystemStatus.cs | 14 +++--- FoundationDB.Client/Utils/TinyJsonParser.cs | 4 +- FoundationDB.Tests/DatabaseBulkFacts.cs | 14 +++--- FoundationDB.Tests/RangeQueryFacts.cs | 4 +- FoundationDB.Tests/TransactionalFacts.cs | 2 +- 7 files changed, 71 insertions(+), 27 deletions(-) diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index be530b06e..19f5cea3f 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; @@ -98,7 +99,7 @@ public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumer return RunWriteOperationAsync( db, - data, + data.Select(x => (x.Key, x.Value)), new WriteOptions(), ct ); @@ -118,6 +119,49 @@ public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumer ct.ThrowIfCancellationRequested(); + return RunWriteOperationAsync( + db, + data.Select(x => (x.Key, x.Value)), + options ?? new WriteOptions(), + ct + ); + } + + /// Writes a potentially large sequence of key/value pairs into the database, by using as many transactions as necessary, and automatically scaling the size of each batch. + /// Database used for the operation + /// Sequence of key/value pairs + /// Token used to cancel the operation + /// Total number of values inserted in the database + /// In case of a non-retryable error, some of the keys may remain in the database. Other transactions running at the same time may observe only a fraction of the keys until the operation completes. + public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable<(Slice Key, Slice Value)> data, CancellationToken ct) + { + if (db == null) throw new ArgumentNullException(nameof(db)); + if (data == null) throw new ArgumentNullException(nameof(data)); + + ct.ThrowIfCancellationRequested(); + + return RunWriteOperationAsync( + db, + data, + new WriteOptions(), + ct + ); + } + + /// Writes a potentially large sequence of key/value pairs into the database, by using as many transactions as necessary, and automatically scaling the size of each batch. + /// Database used for the operation + /// Sequence of key/value pairs + /// Custom options used to configure the behaviour of the operation + /// Token used to cancel the operation + /// Total number of values inserted in the database + /// In case of a non-retryable error, some of the keys may remain in the database. Other transactions running at the same time may observe only a fraction of the keys until the operation completes. + public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable<(Slice Key, Slice Value)> data, WriteOptions options, CancellationToken ct) + { + if (db == null) throw new ArgumentNullException(nameof(db)); + if (data == null) throw new ArgumentNullException(nameof(data)); + + ct.ThrowIfCancellationRequested(); + return RunWriteOperationAsync( db, data, @@ -126,7 +170,7 @@ public static Task WriteAsync([NotNull] IFdbDatabase db, [NotNull] IEnumer ); } - internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable> data, WriteOptions options, CancellationToken ct) + internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase db, [NotNull] IEnumerable<(Slice Key, Slice Value)> data, WriteOptions options, CancellationToken ct) { Contract.Requires(db != null && data != null && options != null); @@ -145,7 +189,7 @@ internal static async Task RunWriteOperationAsync([NotNull] IFdbDatabase d throw new NotImplementedException("Multiple concurrent transactions are not yet supported"); } - var chunk = new List>(); + var chunk = new List<(Slice Key, Slice Value)>(); long items = 0; using (var iterator = data.GetEnumerator()) diff --git a/FoundationDB.Client/FdbDatabaseExtensions.cs b/FoundationDB.Client/FdbDatabaseExtensions.cs index d94f642a4..9e1177a04 100644 --- a/FoundationDB.Client/FdbDatabaseExtensions.cs +++ b/FoundationDB.Client/FdbDatabaseExtensions.cs @@ -318,18 +318,18 @@ public static Task SetAsync([NotNull] this IFdbRetryable db, Slice key, Slice va return db.WriteAsync((tr) => tr.Set(key, value), ct); } - /// Set the values of a list of keys in the database, using a dedicated transaction. + /// Set the values of a sequence of keys in the database, using a dedicated transaction. /// Database instance /// /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair[] keyValuePairs, CancellationToken ct) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable> items, CancellationToken ct) { Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => { - foreach (var kv in keyValuePairs) + foreach (var kv in items) { tr.Set(kv.Key, kv.Value); } @@ -342,12 +342,12 @@ public static Task SetValuesAsync([NotNull] this IFdbRetryable db, KeyValuePair< /// Use this method only if you intend to perform a single operation inside your execution context (ex: HTTP request). /// If you need to combine multiple read or write operations, consider using on of the multiple or overrides. /// - public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable> keyValuePairs, CancellationToken ct) + public static Task SetValuesAsync([NotNull] this IFdbRetryable db, IEnumerable<(Slice Key, Slice Value)> items, CancellationToken ct) { Contract.NotNull(db, nameof(db)); return db.WriteAsync((tr) => { - foreach (var kv in keyValuePairs) + foreach (var kv in items) { tr.Set(kv.Key, kv.Value); } diff --git a/FoundationDB.Client/Status/FdbSystemStatus.cs b/FoundationDB.Client/Status/FdbSystemStatus.cs index 1e8d89f3e..d3526813e 100644 --- a/FoundationDB.Client/Status/FdbSystemStatus.cs +++ b/FoundationDB.Client/Status/FdbSystemStatus.cs @@ -83,8 +83,8 @@ internal Message(string name, string description) internal static Message From(Dictionary data, string field) { - var kvp = TinyJsonParser.GetStringPair(TinyJsonParser.GetMapField(data, field), "name", "description"); - return new Message(kvp.Key ?? String.Empty, kvp.Value ?? String.Empty); + (var key, var value) = TinyJsonParser.GetStringPair(TinyJsonParser.GetMapField(data, field), "name", "description"); + return new Message(key ?? string.Empty, value ?? string.Empty); } internal static Message[] FromArray(Dictionary data, string field) @@ -94,8 +94,8 @@ internal static Message[] FromArray(Dictionary data, string fiel for (int i = 0; i < res.Length; i++) { var obj = (Dictionary)array[i]; - var kvp = TinyJsonParser.GetStringPair(obj, "name", "description"); - res[i] = new Message(kvp.Key, kvp.Value); + (var key, var value) = TinyJsonParser.GetStringPair(obj, "name", "description"); + res[i] = new Message(key, value); } return res; } @@ -566,7 +566,7 @@ internal ProcessStatus(Dictionary data, string id) private ProcessCpuMetrics m_cpu; private ProcessDiskMetrics m_disk; private ProcessMemoryMetrics m_memory; - private KeyValuePair[] m_roles; + private (string Id, string Role)[] m_roles; /// Unique identifier for this process. //TODO: is it stable accross reboots? what are the conditions for a process to change its ID ? @@ -613,7 +613,7 @@ internal ProcessStatus(Dictionary data, string id) /// List of the roles assumed by this process /// The key is the unique role ID in the cluster, and the value is the type of the role itself - public KeyValuePair[] Roles + public (string Id, string Role)[] Roles { get { @@ -622,7 +622,7 @@ public KeyValuePair[] Roles //REVIEW: should we have (K=id, V=role) or (K=role, V=id) ? var arr = GetArray("roles"); - var res = new KeyValuePair[arr.Count]; + var res = new (string, string)[arr.Count]; for (int i = 0; i < res.Length; i++) { var obj = (Dictionary)arr[i]; diff --git a/FoundationDB.Client/Utils/TinyJsonParser.cs b/FoundationDB.Client/Utils/TinyJsonParser.cs index 399582aa8..b67014911 100644 --- a/FoundationDB.Client/Utils/TinyJsonParser.cs +++ b/FoundationDB.Client/Utils/TinyJsonParser.cs @@ -400,10 +400,10 @@ internal static string GetStringField(Dictionary map, string fie return map != null && map.TryGetValue(field, out item) ? (bool)item : default(bool?); } - internal static KeyValuePair GetStringPair(Dictionary map, string key, string value) + internal static (string Key, string Value) GetStringPair(Dictionary map, string key, string value) { object item; - return new KeyValuePair( + return ( map != null && map.TryGetValue(key, out item) ? (string)item : null, map != null && map.TryGetValue(value, out item) ? (string)item : null ); diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index 035355505..9203da71e 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -59,7 +59,7 @@ public async Task Test_Can_Bulk_Insert_Raw_Data() var rnd = new Random(2403); var data = Enumerable.Range(0, N) - .Select((x) => new KeyValuePair(location.Keys.Encode(x.ToString("x8")), Slice.Random(rnd, 16 + rnd.Next(240)))) + .Select((x) => (Key: location.Keys.Encode(x.ToString("x8")), Value: Slice.Random(rnd, 16 + rnd.Next(240)))) .ToArray(); Log("Total data size is {0:N0} bytes", data.Sum(x => x.Key.Count + x.Value.Count)); @@ -197,7 +197,7 @@ public async Task Test_Can_Batch_ForEach_AsyncWithContextAndState() await Fdb.Bulk.WriteAsync( db, - Enumerable.Range(1, N).Select((x) => new KeyValuePair(location.Keys.Encode(x), Slice.FromInt32(x))), + Enumerable.Range(1, N).Select((x) => (location.Keys.Encode(x), Slice.FromInt32(x))), this.Cancellation ); @@ -348,7 +348,7 @@ public async Task Test_Can_Batch_ForEach_WithContextAndState() await Fdb.Bulk.WriteAsync( db, - Enumerable.Range(1, N).Select((x) => new KeyValuePair(location.Keys.Encode(x), Slice.FromInt32(x))), + Enumerable.Range(1, N).Select((x) => (location.Keys.Encode(x), Slice.FromInt32(x))), this.Cancellation ); @@ -413,7 +413,7 @@ public async Task Test_Can_Batch_ForEach_AsyncWithContext() await Fdb.Bulk.WriteAsync( db, - Enumerable.Range(1, N).Select((x) => new KeyValuePair(location.Keys.Encode(x), Slice.FromInt32(x))), + Enumerable.Range(1, N).Select((x) => (location.Keys.Encode(x), Slice.FromInt32(x))), this.Cancellation ); @@ -473,7 +473,7 @@ public async Task Test_Can_Batch_Aggregate() await Fdb.Bulk.WriteAsync( db, - source.Select((x) => new KeyValuePair(location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), + source.Select((x) => (location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), this.Cancellation ); @@ -534,7 +534,7 @@ public async Task Test_Can_Batch_Aggregate_With_Transformed_Result() await Fdb.Bulk.WriteAsync( db, - source.Select((x) => new KeyValuePair(location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), + source.Select((x) => (location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), this.Cancellation ); @@ -603,7 +603,7 @@ public async Task Test_Can_Export_To_Disk() await Fdb.Bulk.WriteAsync( db.WithoutLogging(), - source.Select((x) => new KeyValuePair(location.Keys.Encode(x.Key), x.Value)), + source.Select((x) => (location.Keys.Encode(x.Key), x.Value)), this.Cancellation ); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 26a74bc0f..a605b3602 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -360,7 +360,7 @@ public async Task Test_Can_Skip() var location = await GetCleanDirectory(db, "Queries", "Range"); // import test data - var data = Enumerable.Range(0, 100).Select(x => new KeyValuePair(location.Keys.Encode(x), Slice.FromFixed32(x))); + var data = Enumerable.Range(0, 100).Select(x => (location.Keys.Encode(x), Slice.FromFixed32(x))); await Fdb.Bulk.WriteAsync(db, data, this.Cancellation); // from the start @@ -442,7 +442,7 @@ public async Task Test_Original_Range_Does_Not_Overflow() var location = await GetCleanDirectory(db, "Queries", "Range"); // import test data - var data = Enumerable.Range(0, 30).Select(x => new KeyValuePair(location.Keys.Encode(x), Slice.FromFixed32(x))); + var data = Enumerable.Range(0, 30).Select(x => (location.Keys.Encode(x), Slice.FromFixed32(x))); await Fdb.Bulk.WriteAsync(db, data, this.Cancellation); using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) diff --git a/FoundationDB.Tests/TransactionalFacts.cs b/FoundationDB.Tests/TransactionalFacts.cs index 75c2d0063..397ca89bf 100644 --- a/FoundationDB.Tests/TransactionalFacts.cs +++ b/FoundationDB.Tests/TransactionalFacts.cs @@ -162,7 +162,7 @@ public async Task Test_Transactionals_Retries_Do_Not_Leak_When_Reading_Too_Much( var sw = Stopwatch.StartNew(); Log("Inserting test data (this may take a few minutes)..."); var rnd = new Random(); - await Fdb.Bulk.WriteAsync(db, Enumerable.Range(0, 100 * 1000).Select(i => new KeyValuePair(location.Keys.Encode(i), Slice.Random(rnd, 4096))), this.Cancellation); + await Fdb.Bulk.WriteAsync(db, Enumerable.Range(0, 100 * 1000).Select(i => (location.Keys.Encode(i), Slice.Random(rnd, 4096))), this.Cancellation); sw.Stop(); Log("> done in " + sw.Elapsed); From 10855a8771efa826d8c1d7112a5257ce9bff9c08 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 11:33:55 +0200 Subject: [PATCH 110/153] Add more tuple usages --- FdbShell/Commands/BasicCommands.cs | 4 +-- FdbTop/Program.cs | 4 +-- FoundationDB.Client/Fdb.System.cs | 4 +-- FoundationDB.Tests/DatabaseBulkFacts.cs | 45 ++++++++++++++----------- FoundationDB.Tests/RangeQueryFacts.cs | 8 ++--- 5 files changed, 35 insertions(+), 30 deletions(-) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index d09be5f24..1243225f4 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -220,9 +220,9 @@ public static async Task Count(string[] path, ITuple extras, IFdbDatabase db, Te var copy = KeySubspace.Copy(folder); log.WriteLine("# Counting keys under {0} ...", FdbKey.Dump(copy.GetPrefix())); - var progress = new Progress>((state) => + var progress = new Progress<(long Count, Slice Current)>((state) => { - log.Write("\r# Found {0:N0} keys...", state.Item1); + log.Write("\r# Found {0:N0} keys...", state.Count); }); long count = await Fdb.System.EstimateCountAsync(db, copy.ToRange(), progress, ct); diff --git a/FdbTop/Program.cs b/FdbTop/Program.cs index e25d437df..c56a144bc 100644 --- a/FdbTop/Program.cs +++ b/FdbTop/Program.cs @@ -819,7 +819,7 @@ private static void ShowProcessesScreen(FdbSystemStatus status, HistoryMetric cu { foreach(var role in proc.Roles) { - map.Add(role.Value); + map.Add(role.Role); } } @@ -860,7 +860,7 @@ private static void ShowProcessesScreen(FdbSystemStatus status, HistoryMetric cu map = new RoleMap(); foreach (var role in proc.Roles) { - map.Add(role.Value); + map.Add(role.Role); } Console.ForegroundColor = ConsoleColor.DarkGray; WriteAt(1, y, diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index 9e38daee9..e5a915691 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -408,7 +408,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Token used to cancel the operation /// Number of keys k such that range.Begin <= k > range.End /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress> onProgress, CancellationToken ct) + public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange range, IProgress<(long Count, Slice Current)> onProgress, CancellationToken ct) { return EstimateCountAsync(db, range.Begin, range.End, onProgress, ct); //REVIEW: BUGBUG: REFACTORING: deal with null value for End! @@ -422,7 +422,7 @@ public static Task EstimateCountAsync([NotNull] IFdbDatabase db, KeyRange /// Token used to cancel the operation /// Number of keys k such that <= k > /// If the range contains a large of number keys, the operation may need more than one transaction to complete, meaning that the number will not be transactionally accurate. - public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress> onProgress, CancellationToken ct) + public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Slice beginInclusive, Slice endExclusive, IProgress<(long Count, Slice Current)> onProgress, CancellationToken ct) { const int INIT_WINDOW_SIZE = 1 << 8; // start at 256 //1024 const int MAX_WINDOW_SIZE = 1 << 13; // never use more than 4096 diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index 9203da71e..3fb306ff7 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -210,7 +210,7 @@ await Fdb.Bulk.WriteAsync( await Fdb.Bulk.ForEachAsync( db, Enumerable.Range(1, N).Select(x => location.Keys.Encode(x)), - () => STuple.Create(0L, 0L), + () => (Total: 0L, Count: 0L), async (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -221,16 +221,19 @@ await Fdb.Bulk.ForEachAsync( await throttle; long sum = 0; - for (int i = 0; i < results.Length; i++) + foreach (Slice x in results) { - sum += results[i].ToInt32(); + sum += x.ToInt32(); } - return STuple.Create(state.Item1 + sum, state.Item2 + results.Length); + + state.Total += sum; + state.Count += results.Length; + return state; }, (state) => { - Interlocked.Add(ref total, state.Item1); - Interlocked.Add(ref count, state.Item2); + Interlocked.Add(ref total, state.Total); + Interlocked.Add(ref count, state.Count); }, this.Cancellation ); @@ -361,7 +364,7 @@ await Fdb.Bulk.WriteAsync( await Fdb.Bulk.ForEachAsync( db, Enumerable.Range(1, N).Select(x => location.Keys.Encode(x)), - () => STuple.Create(0L, 0L), // (sum, count) + () => (Total: 0L, Count: 0L), (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -372,19 +375,19 @@ await Fdb.Bulk.ForEachAsync( var results = t.Result; // <-- this is bad practice, never do that in real life, 'mkay? long sum = 0; - for (int i = 0; i < results.Length; i++) + foreach (Slice x in results) { - sum += results[i].ToInt32(); + sum += x.ToInt32(); } - return STuple.Create( - state.Item1 + sum, // updated sum - state.Item2 + results.Length // updated count - ); + + state.Total += sum; + state.Count += results.Length; + return state; }, (state) => { - Interlocked.Add(ref total, state.Item1); - Interlocked.Add(ref count, state.Item2); + Interlocked.Add(ref total, state.Total); + Interlocked.Add(ref count, state.Count); }, this.Cancellation ); @@ -545,7 +548,7 @@ await Fdb.Bulk.WriteAsync( double average = await Fdb.Bulk.AggregateAsync( db, source.Select(x => location.Keys.Encode(x.Key)), - () => STuple.Create(0L, 0L), + () => (Total: 0L, Count: 0L), async (xs, ctx, state) => { Interlocked.Increment(ref chunks); @@ -556,13 +559,15 @@ await Fdb.Bulk.WriteAsync( await throttle; long sum = 0L; - for (int i = 0; i < results.Length; i++) + foreach (Slice x in results) { - sum += results[i].ToInt32(); + sum += x.ToInt32(); } - return STuple.Create(state.Item1 + sum, state.Item2 + results.Length); + state.Total += sum; + state.Count += results.Length; + return state; }, - (state) => (double)state.Item1 / state.Item2, + (state) => (double) state.Total / state.Count, this.Cancellation ); sw.Stop(); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index a605b3602..3ae378d82 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -696,11 +696,11 @@ public async Task Test_Range_Except_Composite_Key() await db.WriteAsync((tr) => { // Items - tr.Set(locItems.Keys.Encode("userA", 10093), Slice.Empty); - tr.Set(locItems.Keys.Encode("userA", 19238), Slice.Empty); - tr.Set(locItems.Keys.Encode("userB", 20003), Slice.Empty); + tr.Set(locItems.Keys["userA", 10093], Slice.Empty); + tr.Set(locItems.Keys["userA", 19238], Slice.Empty); + tr.Set(locItems.Keys["userB", 20003], Slice.Empty); // Processed - tr.Set(locProcessed.Keys.Encode("userA", 19238), Slice.Empty); + tr.Set(locProcessed.Keys["userA", 19238], Slice.Empty); }, this.Cancellation); // the query (Items ∩ Processed) should return (userA, 10093) and (userB, 20003) From 26b359af2a0cd01e0e60f56019958e23a9b4e256 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 11:35:52 +0200 Subject: [PATCH 111/153] Add a base IKeyEncoder interface that links encoders and Encoding - Makes it easier to go back and forth between encoders and encoding --- .../TypeSystem/Encoders/IDynamicKeyEncoder.cs | 8 +- .../TypeSystem/Encoders/IKeyEncoder.cs | 17 ++- .../Encoders/KeyValueEncoders.Ordered.cs | 109 ++++++++++++++++-- .../Encoders/KeyValueEncoders.Tuples.cs | 67 ++++++++--- .../Encoders/KeyValueEncoders.Unordered.cs | 32 ++++- .../Encoders/KeyValueEncoders.Values.cs | 68 ++++++----- .../TypeSystem/Encoders/KeyValueEncoders.cs | 49 +++++++- 7 files changed, 289 insertions(+), 61 deletions(-) diff --git a/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs index 4643a12c2..a4d9b3e9c 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs @@ -39,7 +39,8 @@ public interface IDynamicKeyEncoder { /// Return the parent key encoding - IKeyEncoding Encoding {[NotNull] get; } + [NotNull] + IKeyEncoding Encoding { get; } #region Encoding... @@ -178,6 +179,7 @@ public interface IDynamicKeyEncoder /// Binary slice produced by a previous call to or /// Tuple containing two elements, or an exception if the data is invalid, or the tuples has less or more than two elements STuple DecodeKey(Slice packed); + //REVIEW: return ValueTuple instead? /// Decode a binary slice containing exactly three elements /// Expected type of the first element @@ -186,6 +188,7 @@ public interface IDynamicKeyEncoder /// Binary slice produced by a previous call to or /// Tuple containing three elements, or an exception if the data is invalid, or the tuples has less or more than three elements STuple DecodeKey(Slice packed); + //REVIEW: return ValueTuple instead? /// Decode a binary slice containing exactly four elements /// Expected type of the first element @@ -195,6 +198,7 @@ public interface IDynamicKeyEncoder /// Binary slice produced by a previous call to or /// Tuple containing four elements, or an exception if the data is invalid, or the tuples has less or more than four elements STuple DecodeKey(Slice packed); + //REVIEW: return ValueTuple instead? /// Decode a binary slice containing exactly five elements /// Expected type of the first element @@ -205,6 +209,7 @@ public interface IDynamicKeyEncoder /// Binary slice produced by a previous call to or /// Tuple containing five elements, or an exception if the data is invalid, or the tuples has less or more than five elements STuple DecodeKey(Slice packed); + //REVIEW: return ValueTuple instead? /// Decode a binary slice containing exactly six elements /// Expected type of the first element @@ -216,6 +221,7 @@ public interface IDynamicKeyEncoder /// Binary slice produced by a previous call to or /// Tuple containing five elements, or an exception if the data is invalid, or the tuples has less or more than five elements STuple DecodeKey(Slice packed); + //REVIEW: return ValueTuple instead? #endregion diff --git a/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs index b355227f9..ec9c3fbde 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs @@ -30,8 +30,17 @@ namespace Doxense.Serialization.Encoders { using System; using Doxense.Memory; + using JetBrains.Annotations; - public interface IKeyEncoder + /// Base interface for all key encoders + public interface IKeyEncoder + { + /// Parent encoding + [NotNull] + IKeyEncoding Encoding { get; } + } + + public interface IKeyEncoder : IKeyEncoder { /// Encode a single value void WriteKeyTo(ref SliceWriter writer, T1 value); @@ -43,14 +52,14 @@ public interface IKeyEncoder public static partial class KeyEncoderExtensions { - public static Slice EncodeKey(this IKeyEncoder encoder, T1 value) + public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, T1 value) { var writer = default(SliceWriter); encoder.WriteKeyTo(ref writer, value); return writer.ToSlice(); } - public static Slice EncodeKey(this IKeyEncoder encoder, Slice prefix, T1 value) + public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, Slice prefix, T1 value) { var writer = new SliceWriter(prefix.Count + 16); // ~16 bytes si T1 = Guid writer.WriteBytes(prefix); @@ -58,7 +67,7 @@ public static Slice EncodeKey(this IKeyEncoder encoder, Slice prefix, T1 return writer.ToSlice(); } - public static T1 DecodeKey(this IKeyEncoder decoder, Slice encoded) + public static T1 DecodeKey([NotNull] this IKeyEncoder decoder, Slice encoded) { var reader = new SliceReader(encoded); decoder.ReadKeyFrom(ref reader, out T1 item); diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs index 2cdf95c40..105c159d9 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs @@ -1,4 +1,31 @@ - +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + namespace Doxense.Serialization.Encoders { using JetBrains.Annotations; @@ -12,6 +39,7 @@ public static partial class KeyValueEncoders { /// Encoders that produce lexicographically ordered slices, suitable for keys where lexicographical ordering is required + [PublicAPI] public static class Ordered { [NotNull] @@ -32,7 +60,7 @@ public static class Ordered [NotNull] public static IKeyEncoder GuidEncoder => Tuples.Key(); - public sealed class OrderedKeyEncoder : IKeyEncoder + public sealed class OrderedKeyEncoder : IKeyEncoder, IKeyEncoding { private readonly IOrderedTypeCodec m_codec; @@ -52,9 +80,30 @@ public void ReadKeyFrom(ref SliceReader reader, out T key) { key = m_codec.DecodeOrdered(reader.ReadToEnd()); } + + public IKeyEncoding Encoding => this; + + #region IKeyEncoding... + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + IKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1) != typeof(T)) throw new NotSupportedException(); + return (IKeyEncoder) (object) this; + } + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + #endregion + } - public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder + public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder, IKeyEncoding { private readonly IOrderedTypeCodec m_codec1; private readonly IOrderedTypeCodec m_codec2; @@ -76,14 +125,35 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu { Contract.Requires(count > 0); - T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default(T1); - T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default(T2); + T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; + T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); items = new STuple(key1, key2); } + + public override IKeyEncoding Encoding => this; + + #region IKeyEncoding... + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2)) throw new NotSupportedException(); + return (ICompositeKeyEncoder) (object) this; + } + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + #endregion + } - public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder + public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder, IKeyEncoding { private readonly IOrderedTypeCodec m_codec1; private readonly IOrderedTypeCodec m_codec2; @@ -108,15 +178,36 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu { Contract.Requires(count > 0); - T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default(T1); - T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default(T2); - T3 key3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default(T3); + T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; + T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; + T3 key3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default; if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); items = new STuple(key1, key2, key3); } + public override IKeyEncoding Encoding => this; + + #region IKeyEncoding... + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2) && typeof(T3B) != typeof(T3)) throw new NotSupportedException(); + return (ICompositeKeyEncoder) (object) this; + } + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + #endregion } + //TODO: CompositeKeyEncoder ! + /// Create a simple encoder from a codec [NotNull] public static IKeyEncoder Bind([NotNull] IOrderedTypeCodec codec) diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs index 9950c3069..3ec793ed4 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -1,18 +1,46 @@ - +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + namespace Doxense.Serialization.Encoders { - using JetBrains.Annotations; using System; using Doxense.Collections.Tuples; using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using JetBrains.Annotations; /// Helper class for all key/value encoders public static partial class KeyValueEncoders { /// Encoders that use the Tuple Encoding, suitable for keys + [PublicAPI] public static class Tuples { @@ -22,6 +50,8 @@ internal class TupleEncoder : IKeyEncoder, IValueEncoder private TupleEncoder() { } + public IKeyEncoding Encoding => TypeSystem.Tuples; + public void WriteKeyTo(ref SliceWriter writer, T key) { TupleEncoder.WriteKeysTo(ref writer, key); @@ -30,7 +60,7 @@ public void WriteKeyTo(ref SliceWriter writer, T key) public void ReadKeyFrom(ref SliceReader reader, out T key) { key = !reader.HasMore - ? default(T) //BUGBUG + ? default //BUGBUG : TuPack.DecodeKey(reader.ReadToEnd()); } @@ -41,7 +71,7 @@ public Slice EncodeValue(T key) public T DecodeValue(Slice encoded) { - if (encoded.IsNullOrEmpty) return default(T); //BUGBUG + if (encoded.IsNullOrEmpty) return default; //BUGBUG return TuPack.DecodeKey(encoded); } @@ -54,6 +84,8 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder private TupleCompositeEncoder() { } + public override IKeyEncoding Encoding => TypeSystem.Tuples; + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) { switch (count) @@ -72,7 +104,7 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu Contract.Assert(t != null); key = new STuple( t.Get(0), - count == 2 ? t.Get(1) : default(T2) + count == 2 ? t.Get(1) : default ); } } @@ -84,6 +116,8 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) { switch (count) @@ -103,8 +137,8 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu Contract.Assert(t != null); key = new STuple( t.Get(0), - count >= 2 ? t.Get(1) : default(T2), - count >= 3 ? t.Get(2) : default(T3) + count >= 2 ? t.Get(1) : default, + count >= 3 ? t.Get(2) : default ); } } @@ -116,6 +150,8 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) { switch (count) @@ -136,9 +172,9 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu Contract.Assert(t != null); key = new STuple( t.Get(0), - count >= 2 ? t.Get(1) : default(T2), - count >= 3 ? t.Get(2) : default(T3), - count >= 4 ? t.Get(3) : default(T4) + count >= 2 ? t.Get(1) : default, + count >= 3 ? t.Get(2) : default, + count >= 4 ? t.Get(3) : default ); } } @@ -150,6 +186,8 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) { switch (count) @@ -171,13 +209,14 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STu Contract.Assert(t != null); key = new STuple( t.Get(0), - count >= 2 ? t.Get(1) : default(T2), - count >= 3 ? t.Get(2) : default(T3), - count >= 4 ? t.Get(3) : default(T4), - count >= 5 ? t.Get(4) : default(T5) + count >= 2 ? t.Get(1) : default, + count >= 3 ? t.Get(2) : default, + count >= 4 ? t.Get(3) : default, + count >= 5 ? t.Get(4) : default ); } } + #region Keys [NotNull] diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs index 796fe6a4c..2cea05e1e 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs @@ -1,15 +1,43 @@ - +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + namespace Doxense.Serialization.Encoders { - using JetBrains.Annotations; using System; using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Helper class for all key/value encoders public static partial class KeyValueEncoders { /// Encoders that produce compact but unordered slices, suitable for keys that don't benefit from having lexicographical ordering + [PublicAPI] public static class Unordered { diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs index cf8326c20..2c3ab0e05 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs @@ -1,49 +1,61 @@ - +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + namespace Doxense.Serialization.Encoders { - using JetBrains.Annotations; using System; using Doxense.Diagnostics.Contracts; - using Doxense.Memory; + using JetBrains.Annotations; /// Helper class for all key/value encoders public static partial class KeyValueEncoders { /// Encoders that produce compact but unordered slices, suitable for values + [PublicAPI] public static class Values { private static readonly GenericEncoder s_default = new GenericEncoder(); - public static IValueEncoder BinaryEncoder - { - [NotNull] - get { return s_default; } - } + [NotNull] + public static IValueEncoder BinaryEncoder => s_default; - public static IValueEncoder StringEncoder - { - [NotNull] - get { return s_default; } - } + [NotNull] + public static IValueEncoder StringEncoder => s_default; - public static IValueEncoder Int32Encoder - { - [NotNull] - get { return s_default; } - } + [NotNull] + public static IValueEncoder Int32Encoder => s_default; - public static IValueEncoder Int64Encoder - { - [NotNull] - get { return s_default; } - } + [NotNull] + public static IValueEncoder Int64Encoder => s_default; - public static IValueEncoder GuidEncoder - { - [NotNull] - get { return s_default; } - } + [NotNull] + public static IValueEncoder GuidEncoder => s_default; /// Create a simple encoder from a codec [NotNull] diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index 552b2548d..6a88e85c8 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -35,7 +35,6 @@ namespace Doxense.Serialization.Encoders using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; - using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Helper class for all key/value encoders @@ -48,11 +47,15 @@ public static partial class KeyValueEncoders #region Nested Classes /// Identity encoder - public sealed class IdentityEncoder : IKeyEncoder, IValueEncoder + public sealed class IdentityEncoder : IKeyEncoder, IValueEncoder, IKeyEncoding { internal IdentityEncoder() { } + #region IKeyEncoder... + + public IKeyEncoding Encoding => this; + public void WriteKeyTo(ref SliceWriter writer, Slice key) { writer.WriteBytes(key); @@ -72,10 +75,27 @@ public Slice DecodeValue(Slice encoded) { return encoded; } + + #endregion + + IKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1) != typeof(Slice)) throw new NotSupportedException(); + return (IKeyEncoder) (object) this; + } + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + } /// Wrapper for encoding and decoding a singleton with lambda functions - internal sealed class Singleton : IKeyEncoder, IValueEncoder + internal sealed class Singleton : IKeyEncoder, IValueEncoder, IKeyEncoding { private readonly Func m_encoder; private readonly Func m_decoder; @@ -113,12 +133,29 @@ public T DecodeValue(Slice encoded) return m_decoder(encoded); } + public IKeyEncoding Encoding => this; + + IKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1) != typeof(T)) throw new NotSupportedException(); + return (IKeyEncoder) (object) this; + } + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); } /// Wrapper for encoding and decoding a pair with lambda functions public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { + public abstract IKeyEncoding Encoding { get; } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); @@ -140,6 +177,8 @@ public void ReadKeyFrom(ref SliceReader reader, out STuple items) public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { + public abstract IKeyEncoding Encoding { get; } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); @@ -160,6 +199,8 @@ public void ReadKeyFrom(ref SliceReader reader, out STuple items) public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { + public abstract IKeyEncoding Encoding { get; } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); @@ -180,6 +221,8 @@ public void ReadKeyFrom(ref SliceReader reader, out STuple items public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { + public abstract IKeyEncoding Encoding { get; } + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); From 2bcbd457094528c1ad6d2d17e2c687aee29e9e6b Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 11:37:26 +0200 Subject: [PATCH 112/153] Add overloads to simplify creating and copying subspaces with specific encodings - Most 'encoding' arguments are optional and will default to TypeSystem.Tuples if null - Copying existing subspaces will try to reuse the original encoding --- FoundationDB.Client/Fdb.Bulk.cs | 1 - FoundationDB.Client/FdbDatabase.cs | 4 +- .../Layers/Directories/FdbDirectoryLayer.cs | 25 +-- .../Layers/Tuples/TupleComparisons.cs | 41 +++- .../Subspaces/DynamicKeySubspace.cs | 21 +- FoundationDB.Client/Subspaces/KeySubspace.cs | 196 +++++++++++++++++- .../Subspaces/KeySubspaceExtensions.cs | 45 ++-- FoundationDB.Tests/RangeQueryFacts.cs | 11 +- FoundationDB.Tests/SubspaceFacts.cs | 17 +- 9 files changed, 287 insertions(+), 74 deletions(-) diff --git a/FoundationDB.Client/Fdb.Bulk.cs b/FoundationDB.Client/Fdb.Bulk.cs index 19f5cea3f..64878fc00 100644 --- a/FoundationDB.Client/Fdb.Bulk.cs +++ b/FoundationDB.Client/Fdb.Bulk.cs @@ -1004,7 +1004,6 @@ public static Task ForEachAsync( /// Lambda function that will be called after the last batch, and will be passed the last known state. /// Token used to cancel the operation /// Task that completes when all the elements of have been processed, a non-retryable error occurs, or is triggered - [Obsolete("EXPERIMENTAL: do not use yet!")] public static Task ForEachAsync( [NotNull] IFdbDatabase db, [NotNull] IEnumerable source, diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index ac2da2107..418376da4 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -456,8 +456,8 @@ internal void ChangeRoot(IKeySubspace subspace, IFdbDirectory directory, bool re lock (this)//TODO: don't use this for locking { m_readOnly = readOnly; - m_globalSpace = KeySubspace.Copy(subspace).Using(TypeSystem.Tuples); - m_globalSpaceCopy = KeySubspace.Copy(subspace).Using(TypeSystem.Tuples); // keep another copy + m_globalSpace = KeySubspace.Copy(subspace, TypeSystem.Tuples); + m_globalSpaceCopy = KeySubspace.Copy(subspace, TypeSystem.Tuples); // keep another copy m_directory = directory == null ? null : new FdbDatabasePartition(this, directory); } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 3f8ec5628..86f4378c0 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -153,20 +153,14 @@ internal FdbDirectoryLayer(IDynamicKeySubspace nodeSubspace, IDynamicKeySubspace } } - /// Create an instance of the default Directory Layer - [NotNull] - public static FdbDirectoryLayer Create() - { - return Create(Slice.Empty); - } - /// Create an instance of a Directory Layer located under a specific prefix and path /// Prefix for the content. The nodes will be stored under + <FE> /// Optional path, if the Directory Layer is not located at the root of the database. + /// Optional key encoding scheme. If not specified, will use the type system by default. [NotNull] - public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null) + public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null, IKeyEncoding encoding = null) { - var subspace = KeySubspace.FromKey(prefix).Using(TypeSystem.Tuples); + var subspace = KeySubspace.FromKey(prefix, encoding ?? TypeSystem.Tuples); var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } @@ -174,13 +168,14 @@ public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = /// Create an instance of a Directory Layer located under a specific subspace and path /// Subspace for the content. The nodes will be stored under .Key + <FE> /// Optional path, if the Directory Layer is not located at the root of the database. + /// Optional key encoding scheme. If not specified, will use the type system by default. [NotNull] - public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null) + public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null, IKeyEncoding encoding = null) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); var location = path != null ? ParsePath(path) : STuple.Empty; - var space = subspace.Using(TypeSystem.Tuples); + var space = subspace.Using(encoding ?? TypeSystem.Tuples); return new FdbDirectoryLayer(space.Partition[FdbKey.Directory], space, location); } @@ -539,14 +534,6 @@ internal static ITuple ParsePath(IEnumerable path, string argName = null return STuple.FromArray(pathCopy); } - [NotNull] - internal static ITuple ParsePath([NotNull] string name, string argName = null) - { - if (name == null) throw new ArgumentNullException(argName ?? "name"); - - return STuple.Create(name); - } - [NotNull] internal static ITuple VerifyPath([NotNull] ITuple path, string argName = null) { diff --git a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs index 535b390de..4c77a7f4a 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs @@ -26,6 +26,8 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion +#define ENABLE_VALUETUPLES + namespace Doxense.Collections.Tuples { using System; @@ -160,7 +162,7 @@ public static IComparer Composite(int offset = 0, IComparer comp /// Comparer for the second item's type /// New comparer instance [NotNull] - public static IComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null) + public static CompositeComparer Composite(int offset = 0, IComparer comparer1 = null, IComparer comparer2 = null) { return new CompositeComparer(offset, comparer1, comparer2); } @@ -235,7 +237,10 @@ public int Compare(ITuple x, ITuple y) /// Comparer that compares tuples with at least 2 items /// Type of the first item /// Type of the second item - public sealed class CompositeComparer : IComparer + public sealed class CompositeComparer : IComparer, IComparer> +#if ENABLE_VALUETUPLES + , IComparer<(T1, T2)> +#endif { public static readonly IComparer Default = new CompositeComparer(); @@ -286,13 +291,37 @@ public int Compare(ITuple x, ITuple y) int p = this.Offset; - int c = this.Comparer1.Compare(x.Get(p), y.Get(p)); - if (c != 0) return c; + int cmp = this.Comparer1.Compare(x.Get(p), y.Get(p)); + if (cmp != 0) return cmp; if (ny == 1 || nx == 1) return nx - ny; - c = this.Comparer2.Compare(x.Get(p + 1), y.Get(p + 1)); + cmp = this.Comparer2.Compare(x.Get(p + 1), y.Get(p + 1)); - return c; + return cmp; + } + + /// Compare two tuples + /// First tuple + /// Second tuple + /// Returns a positive value if x is greater than y, a negative value if x is less than y and 0 if x is equal to y. + public int Compare(STuple x, STuple y) + { + if (this.Offset != 0) throw new InvalidOperationException("Cannot compare fixed tuples with non-zero offset."); + int cmp = this.Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = this.Comparer2.Compare(x.Item2, y.Item2); + return cmp; + } + + /// Compare two tuples + /// First tuple + /// Second tuple + /// Returns a positive value if x is greater than y, a negative value if x is less than y and 0 if x is equal to y. + public int Compare((T1, T2) x, (T1, T2) y) + { + if (this.Offset != 0) throw new InvalidOperationException("Cannot compare fixed tuples with non-zero offset."); + int cmp = this.Comparer1.Compare(x.Item1, y.Item1); + if (cmp == 0) cmp = this.Comparer2.Compare(x.Item2, y.Item2); + return cmp; } } diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 7d791f134..2971a6a2d 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -46,27 +46,42 @@ public class DynamicKeySubspace : KeySubspace, IDynamicKeySubspace /// Encoder for the keys of this subspace public IKeyEncoding Encoding { get; } + [NotNull] internal IDynamicKeyEncoder KeyEncoder { get; } /// Create a new subspace from a binary prefix /// Prefix of the new subspace - /// Type System used to encode keys in this subspace (optional, will use Tuple Encoding by default) - internal DynamicKeySubspace(Slice prefix, IKeyEncoding encoding) + /// Type System used to encode keys in this subspace + internal DynamicKeySubspace(Slice prefix, [NotNull] IKeyEncoding encoding) : base(prefix) { + Contract.Requires(encoding != null); this.Encoding = encoding; this.KeyEncoder = encoding.GetDynamicEncoder(); this.Keys = new DynamicKeys(this, this.KeyEncoder); this.Partition = new DynamicPartition(this); } + /// Create a new subspace from a binary prefix + /// Prefix of the new subspace + /// Encoder that will be used by this subspace + internal DynamicKeySubspace(Slice prefix, [NotNull] IDynamicKeyEncoder encoder) + : base(prefix) + { + Contract.Requires(encoder != null); + this.Encoding = encoder.Encoding; + this.KeyEncoder = encoder; + this.Keys = new DynamicKeys(this, encoder); + this.Partition = new DynamicPartition(this); + } + /// Return a view of all the possible binary keys of this subspace public DynamicKeys Keys { get; } /// Return a view of all the possible binary keys of this subspace public DynamicPartition Partition { get; } - public Slice this[ITuple item] + public Slice this[[NotNull] ITuple item] { [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] get => this.Keys.Pack(item); diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 7c74c6db3..7f8dc5245 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -57,25 +57,213 @@ public class KeySubspace : IKeySubspace, IEquatable, IComparable new KeySubspace(Slice.Empty); - /// Initializes a new subspace with the given prefix + #region FromKey... + + /// Initializes a new generic subspace with the given prefix. [Pure, NotNull] public static KeySubspace FromKey(Slice prefix) { return new KeySubspace(prefix.Memoize()); } - /// Initializes a new subspace with the given prefix + /// Initializes a new dynamic subspace with the given binary and key . + /// A subspace that can handle keys of any types and size. + [Pure, NotNull] + public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IDynamicKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + return new DynamicKeySubspace(prefix, encoder); + } + + /// Initializes a new subspace with the given binary , that uses a dynamic key . + /// A subspace that can handle keys of any types and size. + [Pure, NotNull] + public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncoding encoding) + { + Contract.NotNull(encoding, nameof(encoding)); + return new DynamicKeySubspace(prefix, encoding.GetDynamicEncoder()); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle keys of type . + public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + { + return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle keys of type . + public static TypedKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(prefix, encoder); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, ). + public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + { + return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, ). + public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(prefix, encoder); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, , ). + public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + { + return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, , ). + public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(prefix, encoder); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, , ). + public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + { + return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + } + + /// Initializes a new subspace with the given binary , that uses a typed key . + /// A subspace that can handle composite keys of type (, , ). + public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(prefix, encoder); + } + + /// Initializes a new generic subspace with the given . + /// A subspace that can handle keys of any types and size. [Pure, NotNull] public static KeySubspace FromKey(ITuple prefix) { + //REVIEW: this is tied to the Tuple Layer. Maybe this should be an extension method that lives in that namespace? return new KeySubspace(TuPack.Pack(prefix).Memoize()); } - public static KeySubspace Copy(IKeySubspace subspace) + /// Initializes a new dynamic subspace with the given and key . + /// A subspace that can handle keys of any types and size. + [Pure, NotNull] + public static DynamicKeySubspace FromKey(ITuple prefix, [NotNull] IDynamicKeyEncoder encoder) + { + Contract.NotNull(encoder, nameof(encoder)); + var writer = new SliceWriter(); + encoder.PackKey(ref writer, prefix); + return new DynamicKeySubspace(writer.ToSlice(), encoder); + } + + /// Initializes a new subspace with the given , that uses a dynamic key . + /// A subspace that can handle keys of any types and size. + [Pure, NotNull] + public static DynamicKeySubspace FromKey(ITuple prefix, [NotNull] IKeyEncoding encoding) + { + Contract.NotNull(encoding, nameof(encoding)); + return FromKey(prefix, encoding.GetDynamicEncoder()); + } + + #endregion + + #region Copy... + + /// Create a new copy of a subspace's prefix + [Pure] + internal static Slice StealPrefix([NotNull] IKeySubspace subspace) { - return subspace is KeySubspace ks ? new KeySubspace(ks.Key.Memoize()) : new KeySubspace(subspace.GetPrefix().Memoize()); + //note: we can workaround the 'security' in top directory partition by accessing their key prefix without triggering an exception! + return subspace is KeySubspace ks + ? ks.Key.Memoize() + : subspace.GetPrefix().Memoize(); + } + + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static KeySubspace Copy([NotNull] IKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + + var prefix = StealPrefix(subspace); + + if (subspace is IDynamicKeySubspace dyn) + { // reuse the encoding of the original + return new DynamicKeySubspace(prefix, dyn.Encoding); + } + + // no encoding + return new KeySubspace(prefix); } + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] IKeySubspace subspace, IKeyEncoding encoding) + { + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); + return new DynamicKeySubspace(StealPrefix(subspace), encoding); + } + + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] IKeySubspace subspace, IDynamicKeyEncoder encoder) + { + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); + return new DynamicKeySubspace(StealPrefix(subspace), encoder); + } + + /// Create a copy of a dynamic subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] IDynamicKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new DynamicKeySubspace(StealPrefix(subspace), subspace.Encoding); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + #endregion + internal KeySubspace(Slice prefix) { this.Key = prefix; diff --git a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index 3a118cf52..7afb018e9 100644 --- a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -46,6 +46,7 @@ public static class KeySubspaceExtensions /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] public static IDynamicKeySubspace Using([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) + //REVIEW: rename to AsDynamic() ? ToDynamic() ? would all to make encoding arg optional (and default to Tuples) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoding, nameof(encoding)); @@ -53,7 +54,7 @@ public static IDynamicKeySubspace Using([NotNull] this IKeySubspace subspace, [N } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace + /// Instance of a generic subspace to extend /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] @@ -65,19 +66,18 @@ public static IDynamicKeySubspace UsingEncoder([NotNull] this IKeySubspace subsp } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace - /// Custom key encoder + /// Instance of a generic subspace to extend + /// Encoding by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace + /// Instance of a generic subspace to extend /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] @@ -89,19 +89,18 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace s } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace - /// Custom key encoder + /// Instance of a generic subspace to extend + /// Encoding used by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace + /// Instance of a generic subspace to extend /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] @@ -113,19 +112,18 @@ public static ITypedKeySubspace UsingEncoder([NotNull] this IKey } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace - /// Custom key encoder + /// Instance of a generic subspace to extend + /// Encoding used by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace + /// Instance of a generic subspace to extend /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] @@ -138,19 +136,18 @@ public static ITypedKeySubspace UsingEncoder([NotNull] t /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace - /// Custom key encoder + /// Encoding used by the keys of this namespace. If not specified, the Tuples Type System will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new TypedKeySubspace(subspace.GetPrefix(), encoding.GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace - /// Custom key encoder + /// Encoder used to serialize the keys of this namespace. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 3ae378d82..5d180f6d1 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client.Tests using Doxense.Collections.Tuples; using Doxense.Linq; using Doxense.Linq.Async.Iterators; + using Doxense.Serialization.Encoders; using FoundationDB.Layers.Directories; using NUnit.Framework; @@ -687,9 +688,9 @@ public async Task Test_Range_Except_Composite_Key() var location = await GetCleanDirectory(db, "Queries", "ExceptComposite"); // Items contains a list of all ("user", id) that were created - var locItems = await location.CreateOrOpenAsync(db, "Items", this.Cancellation); + var locItems = (await location.CreateOrOpenAsync(db, "Items", this.Cancellation)).UsingEncoder(); // Processed contain the list of all ("user", id) that were processed - var locProcessed = await location.CreateOrOpenAsync(db, "Processed", this.Cancellation); + var locProcessed = (await location.CreateOrOpenAsync(db, "Processed", this.Cancellation)).UsingEncoder(); // the goal is to have a query that returns the list of all unprocessed items (ie: in Items but not in Processed) @@ -717,7 +718,7 @@ await db.WriteAsync((tr) => // problem: Except() still returns the original (Slice,Slice) pairs from the first range, // meaning that we still need to unpack agin the key (this time knowing the location) - return query.Select(kv => locItems.Keys.Unpack(kv.Key)); + return query.Select(kv => locItems.Keys.Decode(kv.Key)); }, this.Cancellation); foreach(var r in results) @@ -734,11 +735,11 @@ await db.WriteAsync((tr) => { var items = tr .GetRange(locItems.Keys.ToRange()) - .Select(kv => locItems.Keys.Unpack(kv.Key)); + .Select(kv => locItems.Keys.Decode(kv.Key)); var processed = tr .GetRange(locProcessed.Keys.ToRange()) - .Select(kv => locProcessed.Keys.Unpack(kv.Key)); + .Select(kv => locProcessed.Keys.Decode(kv.Key)); // items and processed are lists of (string, int) tuples, we can compare them directly var query = items.Except(processed, TupleComparisons.Composite()); diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index e095b9053..470138584 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -52,9 +52,7 @@ public void Test_Empty_Subspace_Is_Empty() [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = KeySubspace - .FromKey(new byte[] { 42, 255, 0, 127 }.AsSlice()) - .Using(TypeSystem.Tuples); + var subspace = KeySubspace.FromKey(new byte[] { 42, 255, 0, 127 }.AsSlice(), TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -118,9 +116,7 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = KeySubspace - .FromKey(STuple.Create("hello")) - .Using(TypeSystem.Tuples); + var subspace = KeySubspace.FromKey(STuple.Create("hello"), TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -152,7 +148,7 @@ public void Test_Subspace_With_Tuple_Prefix() public void Test_Subspace_Partitioning_With_Binary_Suffix() { // start from a parent subspace - var parent = KeySubspace.FromKey(Slice.Empty).Using(TypeSystem.Tuples); + var parent = KeySubspace.FromKey(Slice.Empty, TypeSystem.Tuples); Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple @@ -179,7 +175,7 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() [Test] public void Test_DynamicKeySpace_API() { - var location = new KeySubspace(Slice.FromString("PREFIX")).Using(TypeSystem.Tuples); + var location = KeySubspace.FromKey(Slice.FromString("PREFIX"), TypeSystem.Tuples); Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -232,8 +228,9 @@ public void Test_DynamicKeySpace_API() [Test] public void Test_TypedKeySpace_T1() { - var location = new KeySubspace(Slice.FromString("PREFIX")) - .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + var location = KeySubspace.FromKey(Slice.FromString("PREFIX")); + Assert.That(location.KeyEncoder, Is.Not.Null, "Should have a Key Encoder"); + Assert.That(location.KeyEncoder.Encoding, Is.SameAs(TypeSystem.Tuples), "Encoder should use Tuple type system"); // shortcuts Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); From 505d3ab542f93cd90d7b8daa6fab935ed481346d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 13:48:15 +0200 Subject: [PATCH 113/153] Refactor the core Key Encoders and Tuple Packers to use ValueTuple - Added a lot of overloads that use ValueTuples natively - Cleaned up some paths to TupleEncoder, and it will always require a prefix (can be empty) --- .../Layers/Tuples/Encoding/TupleCodec`1.cs | 2 +- .../Layers/Tuples/Encoding/TupleEncoder.cs | 228 ++++++------ .../Layers/Tuples/Encoding/TupleKeyEncoder.cs | 24 +- .../Layers/Tuples/Encoding/TuplePackers.cs | 20 +- FoundationDB.Client/Layers/Tuples/STuple.cs | 18 +- FoundationDB.Client/Layers/Tuples/TuPack.cs | 341 ++++++++++++++++-- .../Subspaces/DynamicKeySubspace.cs | 44 ++- .../Subspaces/TypedKeySubspace`2.cs | 14 +- .../Subspaces/TypedKeySubspace`3.cs | 4 +- .../Subspaces/TypedKeySubspace`4.cs | 6 +- .../Encoders/ICompositeKeyEncoder.cs | 89 +++-- .../Encoders/KeyValueEncoders.Ordered.cs | 75 +++- .../Encoders/KeyValueEncoders.Tuples.cs | 92 +++-- .../TypeSystem/Encoders/KeyValueEncoders.cs | 56 ++- 14 files changed, 728 insertions(+), 285 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs index c92f0373a..af34a14ec 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs @@ -55,7 +55,7 @@ public TupleCodec(T missingValue) public override Slice EncodeOrdered(T value) { - return TupleEncoder.EncodeKey(value); + return TupleEncoder.EncodeKey(default(Slice), value); } public override void EncodeOrderedSelfTerm(ref SliceWriter output, T value) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs index 21aebfa00..237bae37f 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs @@ -83,7 +83,6 @@ public static Slice Pack([CanBeNull] TTuple tuple) where TTuple : ITuple { if (tuple == null) return Slice.Nil; - //TODO: maybe optimize for Count==0 => Empty ? (calling .Count may not be fast for all tuples...) var writer = new TupleWriter(); WriteTo(ref writer, tuple); return writer.ToSlice(); @@ -258,7 +257,7 @@ public static Slice[] Pack(Slice prefix, [NotNull] IEnumerable /// Efficiently concatenate a prefix with the packed representation of a 1-tuple [Pure] - public static Slice EncodePrefixedKey(Slice prefix, T value) + public static Slice EncodeKey(Slice prefix, T1 value) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -268,7 +267,7 @@ public static Slice EncodePrefixedKey(Slice prefix, T value) /// Efficiently concatenate a prefix with the packed representation of a 2-tuple [Pure] - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -277,8 +276,19 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2 return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 2-tuple + [Pure] + public static Slice Pack(Slice prefix, ref (T1, T2) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 3-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -288,8 +298,19 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 va return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 3-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 4-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -300,8 +321,20 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 4-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3, T4) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + TuplePackers.SerializeTo(ref writer, items.Item4); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 5-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -313,8 +346,21 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 value return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 5-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3, T4, T5) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + TuplePackers.SerializeTo(ref writer, items.Item4); + TuplePackers.SerializeTo(ref writer, items.Item5); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 6-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -327,8 +373,22 @@ public static Slice EncodePrefixedKey(Slice prefix, T1 v return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 6-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3, T4, T5, T6) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + TuplePackers.SerializeTo(ref writer, items.Item4); + TuplePackers.SerializeTo(ref writer, items.Item5); + TuplePackers.SerializeTo(ref writer, items.Item6); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 7-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -342,8 +402,23 @@ public static Slice EncodePrefixedKey(Slice prefix, return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 7-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3, T4, T5, T6, T7) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + TuplePackers.SerializeTo(ref writer, items.Item4); + TuplePackers.SerializeTo(ref writer, items.Item5); + TuplePackers.SerializeTo(ref writer, items.Item6); + TuplePackers.SerializeTo(ref writer, items.Item7); + return writer.ToSlice(); + } + /// Efficiently concatenate a prefix with the packed representation of a 8-tuple - public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) + public static Slice EncodeKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); @@ -358,6 +433,22 @@ public static Slice EncodePrefixedKey(Slice pref return writer.ToSlice(); } + /// Efficiently concatenate a prefix with the packed representation of a 8-tuple + public static Slice Pack(Slice prefix, ref (T1, T2, T3, T4, T5, T6, T7, T8) items) + { + var writer = new TupleWriter(); + writer.Output.WriteBytes(prefix); + TuplePackers.SerializeTo(ref writer, items.Item1); + TuplePackers.SerializeTo(ref writer, items.Item2); + TuplePackers.SerializeTo(ref writer, items.Item3); + TuplePackers.SerializeTo(ref writer, items.Item4); + TuplePackers.SerializeTo(ref writer, items.Item5); + TuplePackers.SerializeTo(ref writer, items.Item6); + TuplePackers.SerializeTo(ref writer, items.Item7); + TuplePackers.SerializeTo(ref writer, items.Item8); + return writer.ToSlice(); + } + // EncodeKey... //REVIEW: do we really ned "Key" in the name? @@ -420,105 +511,6 @@ public static Slice Pack(Slice prefix, ref STuplePack a 1-tuple directly into a slice - public static Slice EncodeKey(T1 item1) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - return writer.ToSlice(); - } - - /// Pack a 2-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - return writer.ToSlice(); - } - - /// Pack a 3-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - return writer.ToSlice(); - } - - /// Pack a 4-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - TuplePackers.SerializeTo(ref writer, item4); - return writer.ToSlice(); - } - - /// Pack a 5-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - TuplePackers.SerializeTo(ref writer, item4); - TuplePackers.SerializeTo(ref writer, item5); - return writer.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - TuplePackers.SerializeTo(ref writer, item4); - TuplePackers.SerializeTo(ref writer, item5); - TuplePackers.SerializeTo(ref writer, item6); - return writer.Output.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - TuplePackers.SerializeTo(ref writer, item4); - TuplePackers.SerializeTo(ref writer, item5); - TuplePackers.SerializeTo(ref writer, item6); - TuplePackers.SerializeTo(ref writer, item7); - return writer.ToSlice(); - } - - /// Pack a 6-tuple directly into a slice - public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) - { - var writer = new TupleWriter(); - TuplePackers.SerializeTo(ref writer, item1); - TuplePackers.SerializeTo(ref writer, item2); - TuplePackers.SerializeTo(ref writer, item3); - TuplePackers.SerializeTo(ref writer, item4); - TuplePackers.SerializeTo(ref writer, item5); - TuplePackers.SerializeTo(ref writer, item6); - TuplePackers.SerializeTo(ref writer, item7); - TuplePackers.SerializeTo(ref writer, item8); - return writer.ToSlice(); - } - - [NotNull] - public static Slice[] EncodeKeys([NotNull] IEnumerable keys) - { - var empty = default(Slice); - return EncodePrefixedKeys(empty, keys); - } - /// Pack a 1-tuple directly into a slice public static void WriteKeysTo(ref SliceWriter writer, T1 item1) { @@ -617,12 +609,12 @@ public static void WriteKeysTo(ref SliceWriter w /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) + public static Slice[] EncodeKeys(Slice prefix, [NotNull] IEnumerable keys) { Contract.NotNull(keys, nameof(keys)); // use optimized version for arrays - if (keys is T[] array) return EncodePrefixedKeys(prefix, array); + if (keys is T[] array) return EncodeKeys(prefix, array); var next = new List((keys as ICollection)?.Count ?? 0); var writer = new TupleWriter(); @@ -646,7 +638,7 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable< public static Slice[] EncodeKeys([NotNull] params T[] keys) { var empty = default(Slice); - return EncodePrefixedKeys(empty, keys); + return EncodeKeys(empty, keys); } /// Merge an array of keys with a same prefix, all sharing the same buffer @@ -655,7 +647,7 @@ public static Slice[] EncodeKeys([NotNull] params T[] keys) /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) + public static Slice[] EncodeKeys(Slice prefix, [NotNull] params T[] keys) { Contract.NotNull(keys, nameof(keys)); @@ -686,7 +678,7 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] k public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] Func selector) { var empty = default(Slice); - return EncodePrefixedKeys(empty, elements, selector); + return EncodeKeys(empty, elements, selector); } /// Merge an array of elements with a same prefix, all sharing the same buffer @@ -697,7 +689,7 @@ public static Slice[] EncodeKeys([NotNull] TElement[] elements, /// Lambda that extract the key from each element /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) + public static Slice[] EncodeKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) { Contract.NotNull(elements, nameof(elements)); Contract.NotNull(selector, nameof(selector)); @@ -726,12 +718,12 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] TTuple prefix, [NotNull] IEnumerable keys) + public static Slice[] EncodeKeys([NotNull] TTuple prefix, [NotNull] IEnumerable keys) where TTuple : ITuple { Contract.NotNullAllowStructs(prefix, nameof(prefix)); var head = Pack(prefix); - return EncodePrefixedKeys(head, keys); + return EncodeKeys(head, keys); } /// Pack a sequence of keys with a same prefix, all sharing the same buffer @@ -741,13 +733,13 @@ public static Slice[] EncodePrefixedKeys([NotNull] TTuple prefix, [N /// Sequence of keys to pack /// Array of slices (for all keys) that share the same underlying buffer [NotNull] - public static Slice[] EncodePrefixedKeys([NotNull] TTuple prefix, [NotNull] params T1[] keys) + public static Slice[] EncodeKeys([NotNull] TTuple prefix, [NotNull] params T1[] keys) where TTuple : ITuple { Contract.NotNullAllowStructs(prefix, nameof(prefix)); var head = Pack(prefix); - return EncodePrefixedKeys(head, keys); + return EncodeKeys(head, keys); } #endregion diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs index 1b01cbe91..36eb31a0b 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs @@ -43,10 +43,7 @@ public sealed class TupleKeyEncoder : IDynamicKeyEncoder private TupleKeyEncoder() { } - public IKeyEncoding Encoding - { - get { return TypeSystem.Tuples; } - } + public IKeyEncoding Encoding => TypeSystem.Tuples; public void PackKey(ref SliceWriter writer, TTuple items) where TTuple : ITuple @@ -190,50 +187,49 @@ public KeyRange ToRange(Slice prefix) return TuPack.ToRange(prefix); } - public KeyRange ToRange(Slice prefix, ITuple items) { - return TuPack.ToRange(prefix, items); + return TuPack.ToPrefixedKeyRange(prefix, items); } public KeyRange ToKeyRange(Slice prefix, T1 item1) { - return TuPack.ToRange(prefix, STuple.Create(item1)); + return TuPack.ToPrefixedKeyRange(prefix, item1); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6, item7)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6, item7); } public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - return TuPack.ToRange(prefix, STuple.Create(item1, item2, item3, item4, item5, item6, item7, item8)); + return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6, item7, item8); } } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index 6e47b4996..b1a37d5ea 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -596,7 +596,7 @@ public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple< TupleParser.EndTuple(ref writer); } - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + public static void SerializeValueTupleTo(ref TupleWriter writer, (T1, T2) tuple) { TupleParser.BeginTuple(ref writer); SerializeTo(ref writer, tuple.Item1); @@ -604,7 +604,7 @@ public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTu TupleParser.EndTuple(ref writer); } - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + public static void SerializeValueTupleTo(ref TupleWriter writer, (T1, T2, T3) tuple) { TupleParser.BeginTuple(ref writer); SerializeTo(ref writer, tuple.Item1); @@ -613,7 +613,7 @@ public static void SerializeValueTupleTo(ref TupleWriter writer, Val TupleParser.EndTuple(ref writer); } - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + public static void SerializeValueTupleTo(ref TupleWriter writer, (T1, T2, T3, T4) tuple) { TupleParser.BeginTuple(ref writer); SerializeTo(ref writer, tuple.Item1); @@ -623,7 +623,7 @@ public static void SerializeValueTupleTo(ref TupleWriter writer, TupleParser.EndTuple(ref writer); } - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + public static void SerializeValueTupleTo(ref TupleWriter writer, (T1, T2, T3, T4, T5) tuple) { TupleParser.BeginTuple(ref writer); SerializeTo(ref writer, tuple.Item1); @@ -634,7 +634,7 @@ public static void SerializeValueTupleTo(ref TupleWriter wri TupleParser.EndTuple(ref writer); } - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) + public static void SerializeValueTupleTo(ref TupleWriter writer, (T1, T2, T3, T4, T5, T6) tuple) { TupleParser.BeginTuple(ref writer); SerializeTo(ref writer, tuple.Item1); @@ -1171,31 +1171,31 @@ public static ValueTuple DeserializeValueTuple(Slice slice) } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ValueTuple DeserializeValueTuple(Slice slice) + public static (T1, T2) DeserializeValueTuple(Slice slice) { return DeserializeTuple(slice).ToValueTuple(); } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ValueTuple DeserializeValueTuple(Slice slice) + public static (T1, T2, T3) DeserializeValueTuple(Slice slice) { return DeserializeTuple(slice).ToValueTuple(); } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ValueTuple DeserializeValueTuple(Slice slice) + public static (T1, T2, T3, T4) DeserializeValueTuple(Slice slice) { return DeserializeTuple(slice).ToValueTuple(); } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ValueTuple DeserializeValueTuple(Slice slice) + public static (T1, T2, T3, T4, T5) DeserializeValueTuple(Slice slice) { return DeserializeTuple(slice).ToValueTuple(); } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static ValueTuple DeserializeValueTuple(Slice slice) + public static (T1, T2, T3, T4, T5, T6) DeserializeValueTuple(Slice slice) { return DeserializeTuple(slice).ToValueTuple(); } diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index bef108e9f..74f931821 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -355,55 +355,55 @@ public static STuple Create(ref ValueTuple tuple) } [Pure] - public static STuple Create(ValueTuple tuple) + public static STuple Create((T1, T2) tuple) { return new STuple(tuple.Item1, tuple.Item2); } [Pure] - public static STuple Create(ref ValueTuple tuple) + public static STuple Create(ref (T1, T2) tuple) { return new STuple(tuple.Item1, tuple.Item2); } [Pure] - public static STuple Create(ValueTuple tuple) + public static STuple Create((T1, T2, T3) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); } [Pure] - public static STuple Create(ref ValueTuple tuple) + public static STuple Create(ref (T1, T2, T3) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); } [Pure] - public static STuple Create(ValueTuple tuple) + public static STuple Create((T1, T2, T3, T4) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } [Pure] - public static STuple Create(ref ValueTuple tuple) + public static STuple Create(ref (T1, T2, T3, T4) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } [Pure] - public static STuple Create(ValueTuple tuple) + public static STuple Create((T1, T2, T3, T4, T5) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } [Pure] - public static STuple Create(ValueTuple tuple) + public static STuple Create((T1, T2, T3, T4, T5, T6) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } [Pure] - public static STuple Create(ref ValueTuple tuple) + public static STuple Create(ref (T1, T2, T3, T4, T5, T6) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index 7dba29c01..0a02f99af 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -379,63 +379,63 @@ public static Slice[] PackTuples(Slice prefix, [NotNull] IEnum [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1) { - return TupleEncoder.EncodeKey(item1); + return TupleEncoder.EncodeKey(default(Slice), item1); } /// Pack a 2-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2) { - return TupleEncoder.EncodeKey(item1, item2); + return TupleEncoder.EncodeKey(default(Slice), item1, item2); } /// Pack a 3-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3) { - return TupleEncoder.EncodeKey(item1, item2, item3); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3); } /// Pack a 4-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4) { - return TupleEncoder.EncodeKey(item1, item2, item3, item4); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4); } /// Pack a 5-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { - return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5); } /// Pack a 6-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { - return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6); } /// Pack a 6-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { - return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6, item7); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7); } /// Pack a 6-tuple directly into a slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodeKey(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { - return TupleEncoder.EncodeKey(item1, item2, item3, item4, item5, item6, item7, item8); + return TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7, item8); } [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice[] EncodeKeys([NotNull] IEnumerable keys) { var empty = default(Slice); - return TupleEncoder.EncodePrefixedKeys(empty, keys); + return TupleEncoder.EncodeKeys(empty, keys); } /// Merge a sequence of keys with a same prefix, all sharing the same buffer @@ -446,14 +446,14 @@ public static Slice[] EncodeKeys([NotNull] IEnumerable keys) [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] IEnumerable keys) { - return TupleEncoder.EncodePrefixedKeys(prefix, keys); + return TupleEncoder.EncodeKeys(prefix, keys); } [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice[] EncodeKeys([NotNull] params T[] keys) { var empty = default(Slice); - return TupleEncoder.EncodePrefixedKeys(empty, keys); + return TupleEncoder.EncodeKeys(empty, keys); } /// Merge an array of keys with a same prefix, all sharing the same buffer @@ -464,7 +464,7 @@ public static Slice[] EncodeKeys([NotNull] params T[] keys) [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] keys) { - return TupleEncoder.EncodePrefixedKeys(prefix, keys); + return TupleEncoder.EncodeKeys(prefix, keys); } /// Merge an array of elements, all sharing the same buffer @@ -477,7 +477,7 @@ public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] params T[] k public static Slice[] EncodeKeys([NotNull] TElement[] elements, [NotNull] Func selector) { var empty = default(Slice); - return TupleEncoder.EncodePrefixedKeys(empty, elements, selector); + return TupleEncoder.EncodeKeys(empty, elements, selector); } /// Merge an array of elements with a same prefix, all sharing the same buffer @@ -490,7 +490,7 @@ public static Slice[] EncodeKeys([NotNull] TElement[] elements, [Pure, NotNull, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice[] EncodePrefixedKeys(Slice prefix, [NotNull] TElement[] elements, [NotNull] Func selector) { - return TupleEncoder.EncodePrefixedKeys(prefix, elements, selector); + return TupleEncoder.EncodeKeys(prefix, elements, selector); } /// Pack a sequence of keys with a same prefix, all sharing the same buffer @@ -564,7 +564,48 @@ public static KeyRange ToRange(STuple tuple) Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) - var packed = TupleEncoder.Pack(tuple); + var packed = TupleEncoder.EncodeKey(default(Slice), tuple.Item1); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange(ValueTuple tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), tuple.Item1); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1); return new KeyRange( packed + 0x00, packed + 0xFF @@ -587,6 +628,48 @@ public static KeyRange ToRange(STuple tuple) ); } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange((T1, T2) tuple) + { + Contract.NotNullAllowStructs(tuple, nameof(tuple)); + + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] @@ -603,6 +686,46 @@ public static KeyRange ToRange(STuple tuple) ); } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange((T1, T2, T3) tuple) + { + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] @@ -619,6 +742,46 @@ public static KeyRange ToRange(STuple tuple) ); } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange((T1, T2, T3, T4) tuple) + { + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] @@ -635,6 +798,46 @@ public static KeyRange ToRange(STuple tu ); } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange((T1, T2, T3, T4, T5) tuple) + { + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] @@ -651,6 +854,98 @@ public static KeyRange ToRange(STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToRange((T1, T2, T3, T4, T5, T6) tuple) + { + // tuple => [ packed."\0", packed."\xFF" ) + var empty = default(Slice); + var packed = TupleEncoder.Pack(empty, ref tuple); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + { + // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6, item7); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7, item8); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + + /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' + /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + [Pure] + public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + { + // tuple => [ packed."\0", packed."\xFF" ) + var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6, item7, item8); + return new KeyRange( + packed + 0x00, + packed + 0xFF + ); + } + /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// FdbTuple.ToRange(Slice.FromInt32(42), FdbTuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. @@ -905,56 +1200,56 @@ public static bool DecodeNext(ref TupleReader input, out T value) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value) { - return TupleEncoder.EncodePrefixedKey(prefix, value); + return TupleEncoder.EncodeKey(prefix, value); } /// Efficiently concatenate a prefix with the packed representation of a 2-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2); + return TupleEncoder.EncodeKey(prefix, value1, value2); } /// Efficiently concatenate a prefix with the packed representation of a 3-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3); } /// Efficiently concatenate a prefix with the packed representation of a 4-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3, value4); } /// Efficiently concatenate a prefix with the packed representation of a 5-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3, value4, value5); } /// Efficiently concatenate a prefix with the packed representation of a 6-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3, value4, value5, value6); } /// Efficiently concatenate a prefix with the packed representation of a 7-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6, value7); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3, value4, value5, value6, value7); } /// Efficiently concatenate a prefix with the packed representation of a 8-tuple [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public static Slice EncodePrefixedKey(Slice prefix, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5, T6 value6, T7 value7, T8 value8) { - return TupleEncoder.EncodePrefixedKey(prefix, value1, value2, value3, value4, value5, value6, value7, value8); + return TupleEncoder.EncodeKey(prefix, value1, value2, value3, value4, value5, value6, value7, value8); } #endregion diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 2971a6a2d..3d0316756 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -187,30 +187,62 @@ public KeyRange ToRange([NotNull] ITuple tuple) public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1); } public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2); } public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3); } public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } + public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } + public KeyRange ToRange(STuple tuple) { - return this.Encoder.ToRange(this.Parent.GetPrefix(), tuple); + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); + } + + public KeyRange ToRange(ValueTuple tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1); + } + + public KeyRange ToRange((T1, T2) tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2); + } + + public KeyRange ToRange((T1, T2, T3) tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3); + } + + public KeyRange ToRange((T1, T2, T3, T4) tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); + } + + public KeyRange ToRange((T1, T2, T3, T4, T5) tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); + } + + public KeyRange ToRange((T1, T2, T3, T4, T5, T6) tuple) + { + return this.Encoder.ToKeyRange(this.Parent.GetPrefix(), tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } #endregion diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs index 560892006..0ce670be9 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -153,10 +153,7 @@ public KeyRange ToRangePartial(T1 item1) [Pure] public Slice Pack(STuple tuple) { - //REVIEW: how could we better guess the capacity, depending on the values of T1/T2? - var sw = this.Parent.OpenWriter(24); - this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); - return sw.ToSlice(); + return Pack(tuple.ToValueTuple()); } #if ENABLE_VALUETUPLES @@ -166,7 +163,10 @@ public Slice Pack(STuple tuple) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack((T1, T2) tuple) { - return Encode(tuple.Item1, tuple.Item2); + //REVIEW: how could we better guess the capacity, depending on the values of T1/T2? + var sw = this.Parent.OpenWriter(24); + this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); + return sw.ToSlice(); } #endif @@ -208,7 +208,7 @@ public Slice this[(T1, T2) items] public Slice Encode(T1 item1, T2 item2) { var sw = this.Parent.OpenWriter(24); - var tuple = new STuple(item1, item2); + var tuple = (item1, item2); this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); return sw.ToSlice(); } @@ -221,7 +221,7 @@ public Slice Encode(T1 item1, T2 item2) public Slice EncodePartial(T1 item1) { var sw = this.Parent.OpenWriter(16); - var tuple = new STuple(item1, default(T2)); + var tuple = (item1, default(T2)); this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); return sw.ToSlice(); } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index cc1089671..9fd7aa86c 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -216,7 +216,7 @@ public Slice Encode(T1 item1, T2 item2, T3 item3) public Slice EncodePartial(T1 item1, T2 item2) { var sw = this.Parent.OpenWriter(16); - var tuple = new STuple(item1, item2, default(T3)); + var tuple = (item1, item2, default(T3)); this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); return sw.ToSlice(); } @@ -225,7 +225,7 @@ public Slice EncodePartial(T1 item1, T2 item2) public Slice EncodePartial(T1 item1) { var sw = this.Parent.OpenWriter(16); - var tuple = new STuple(item1, default(T2), default(T3)); + var tuple = (item1, default(T2), default(T3)); this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); return sw.ToSlice(); } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index b4bf83e06..e64d79a8f 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -228,7 +228,7 @@ public Slice Encode(T1 item1, T2 item2, T3 item3, T4 item4) public Slice EncodePartial(T1 item1, T2 item2, T3 item3) { var sw = this.Parent.OpenWriter(24); - var tuple = new STuple(item1, item2, item3, default(T4)); + var tuple = (item1, item2, item3, default(T4)); this.Encoder.WriteKeyPartsTo(ref sw, 3, ref tuple); return sw.ToSlice(); } @@ -237,7 +237,7 @@ public Slice EncodePartial(T1 item1, T2 item2, T3 item3) public Slice EncodePartial(T1 item1, T2 item2) { var sw = this.Parent.OpenWriter(16); - var tuple = new STuple(item1, item2, default(T3), default(T4)); + var tuple = (item1, item2, default(T3), default(T4)); this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); return sw.ToSlice(); } @@ -246,7 +246,7 @@ public Slice EncodePartial(T1 item1, T2 item2) public Slice EncodePartial(T1 item1) { var sw = this.Parent.OpenWriter(16); - var tuple = new STuple(item1, default(T2), default(T3), default(T4)); + var tuple = (item1, default(T2), default(T3), default(T4)); this.Encoder.WriteKeyPartsTo(ref sw, 1, ref tuple); return sw.ToSlice(); } diff --git a/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs b/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs index 92d259548..8e5efa1cf 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs @@ -31,26 +31,51 @@ namespace Doxense.Serialization.Encoders using System; using Doxense.Collections.Tuples; using Doxense.Memory; + + public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2)> + { + /// Write some or all parts of a composite key + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) key); + + /// Read some or all parts of a composite key + void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) items); + } - /// Encoder for keys that are tuples - /// Type of tuple - public interface ICompositeKeyEncoder : IKeyEncoder - where TTuple : ITuple + public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2, T3)> { /// Write some or all parts of a composite key - void WriteKeyPartsTo(ref SliceWriter writer, int count, ref TTuple key); + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) key); /// Read some or all parts of a composite key - void ReadKeyPartsFrom(ref SliceReader reader, int count, out TTuple items); + void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) items); } - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2, T3, T4)> + { + /// Write some or all parts of a composite key + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) key); - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + /// Read some or all parts of a composite key + void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) items); + } - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2, T3, T4, T5)> + { + /// Write some or all parts of a composite key + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) key); - public interface ICompositeKeyEncoder : ICompositeKeyEncoder> { } + /// Read some or all parts of a composite key + void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) items); + } + + public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2, T3, T4, T5, T6)> + { + /// Write some or all parts of a composite key + void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) key); + + /// Read some or all parts of a composite key + void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); + } public static partial class KeyEncoderExtensions { @@ -59,14 +84,14 @@ public static partial class KeyEncoderExtensions public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2) { - var tuple = new STuple(value1, value2); + var tuple = (value1, value2); encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); } public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2) { var writer = default(SliceWriter); - var tuple = new STuple(item1, item2); + var tuple = (item1, item2); encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); return writer.ToSlice(); } @@ -82,7 +107,7 @@ public static Slice EncodeKey(this ICompositeKeyEncoder encoder, public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, T1 item1) { var writer = default(SliceWriter); - var tuple = new STuple(item1, default(T2)); + var tuple = (item1, default(T2)); encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); return writer.ToSlice(); } @@ -91,12 +116,12 @@ public static Slice EncodePartialKey(this ICompositeKeyEncoder e { var writer = new SliceWriter(prefix.Count + 16); writer.WriteBytes(prefix); - var tuple = new STuple(item1, default(T2)); + var tuple = (item1, default(T2)); encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); return writer.ToSlice(); } - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2) items) { var writer = default(SliceWriter); encoder.WriteKeyPartsTo(ref writer, count, ref items); @@ -106,7 +131,7 @@ public static Slice EncodeKeyParts(this ICompositeKeyEncoder enc public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) { var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out STuple items); + decoder.ReadKeyFrom(ref reader, out var items); //TODO: throw if extra bytes? return items; } @@ -114,7 +139,7 @@ public static STuple DecodeKey(this ICompositeKeyEncoder public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) { var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); return items; } @@ -124,14 +149,14 @@ public static STuple DecodeKeyParts(this ICompositeKeyEncoder(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3) { - var tuple = new STuple(value1, value2, value3); + var tuple = (value1, value2, value3); encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); } public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3) { var writer = default(SliceWriter); - var tuple = new STuple(item1, item2, item3); + var tuple = (item1, item2, item3); encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); return writer.ToSlice(); } @@ -144,7 +169,7 @@ public static Slice EncodeKey(this ICompositeKeyEncoder return writer.ToSlice(); } - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, STuple items) + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3) items) { var writer = default(SliceWriter); encoder.WriteKeyPartsTo(ref writer, count, ref items); @@ -154,7 +179,7 @@ public static Slice EncodeKeyParts(this ICompositeKeyEncoder DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) { var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out STuple items); + decoder.ReadKeyFrom(ref reader, out var items); //TODO: throw if extra bytes? return items; } @@ -162,7 +187,7 @@ public static STuple DecodeKey(this ICompositeKeyEncoder public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) { var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); return items; } @@ -172,14 +197,14 @@ public static STuple DecodeKeyParts(this ICompositeKeyEn public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4) { - var tuple = new STuple(value1, value2, value3, value4); + var tuple = (value1, value2, value3, value4); encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); } public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4) { var writer = default(SliceWriter); - var tuple = new STuple(item1, item2, item3, item4); + var tuple = (item1, item2, item3, item4); encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); return writer.ToSlice(); } @@ -192,7 +217,7 @@ public static Slice EncodeKey(this ICompositeKeyEncoder(this ICompositeKeyEncoder encoder, int count, STuple items) + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4) items) { var writer = default(SliceWriter); encoder.WriteKeyPartsTo(ref writer, count, ref items); @@ -202,7 +227,7 @@ public static Slice EncodeKeyParts(this ICompositeKeyEncoder DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) { var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out STuple items); + decoder.ReadKeyFrom(ref reader, out var items); //TODO: throw if extra bytes? return items; } @@ -210,7 +235,7 @@ public static STuple DecodeKey(this ICompositeKe public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) { var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); return items; } @@ -220,14 +245,14 @@ public static STuple DecodeKeyParts(this ICompos public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) { - var tuple = new STuple(value1, value2, value3, value4, value5); + var tuple = (value1, value2, value3, value4, value5); encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); } public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { var writer = default(SliceWriter); - var tuple = new STuple(item1, item2, item3, item4, item5); + var tuple = (item1, item2, item3, item4, item5); encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); return writer.ToSlice(); } @@ -240,7 +265,7 @@ public static Slice EncodeKey(this ICompositeKeyEncoder(this ICompositeKeyEncoder encoder, int count, STuple items) + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4, T5) items) { var writer = default(SliceWriter); encoder.WriteKeyPartsTo(ref writer, count, ref items); @@ -250,7 +275,7 @@ public static Slice EncodeKeyParts(this ICompositeKeyEncoder public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) { var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out STuple items); + decoder.ReadKeyFrom(ref reader, out var items); //TODO: throw if extra bytes? return items; } @@ -258,7 +283,7 @@ public static STuple DecodeKey(this ICom public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) { var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out STuple items); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); return items; } diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs index 105c159d9..efeab16ff 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs @@ -114,21 +114,20 @@ public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec< m_codec2 = codec2; } - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) items) { Contract.Requires(count > 0); if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) items) { Contract.Requires(count > 0); - T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; - T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; + items.Item1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; + items.Item2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); - items = new STuple(key1, key2); } public override IKeyEncoding Encoding => this; @@ -166,7 +165,7 @@ public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec< m_codec3 = codec3; } - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) items) { Contract.Requires(count > 0 && count <= 3); if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); @@ -174,15 +173,14 @@ public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STup if (count >= 3) m_codec3.EncodeOrderedSelfTerm(ref writer, items.Item3); } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) items) { Contract.Requires(count > 0); - T1 key1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; - T2 key2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; - T3 key3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default; + items.Item1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; + items.Item2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; + items.Item3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default; if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); - items = new STuple(key1, key2, key3); } public override IKeyEncoding Encoding => this; @@ -206,7 +204,60 @@ ICompositeKeyEncoder IKeyEncoding.GetEncoder() #endregion } - //TODO: CompositeKeyEncoder ! + public sealed class CodecCompositeKeyEncoder : CompositeKeyEncoder, IKeyEncoding + { + private readonly IOrderedTypeCodec m_codec1; + private readonly IOrderedTypeCodec m_codec2; + private readonly IOrderedTypeCodec m_codec3; + private readonly IOrderedTypeCodec m_codec4; + + public CodecCompositeKeyEncoder(IOrderedTypeCodec codec1, IOrderedTypeCodec codec2, IOrderedTypeCodec codec3, IOrderedTypeCodec codec4) + { + m_codec1 = codec1; + m_codec2 = codec2; + m_codec3 = codec3; + m_codec4 = codec4; + } + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) items) + { + Contract.Requires(count > 0 && count <= 4); + if (count >= 1) m_codec1.EncodeOrderedSelfTerm(ref writer, items.Item1); + if (count >= 2) m_codec2.EncodeOrderedSelfTerm(ref writer, items.Item2); + if (count >= 3) m_codec3.EncodeOrderedSelfTerm(ref writer, items.Item3); + if (count >= 4) m_codec4.EncodeOrderedSelfTerm(ref writer, items.Item4); + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) items) + { + Contract.Requires(count > 0); + items.Item1 = count >= 1 ? m_codec1.DecodeOrderedSelfTerm(ref reader) : default; + items.Item2 = count >= 2 ? m_codec2.DecodeOrderedSelfTerm(ref reader) : default; + items.Item3 = count >= 3 ? m_codec3.DecodeOrderedSelfTerm(ref reader) : default; + items.Item4 = count >= 4 ? m_codec4.DecodeOrderedSelfTerm(ref reader) : default; + if (reader.HasMore) throw new InvalidOperationException($"Unexpected data at the end of composite key after {count} items"); + } + + public override IKeyEncoding Encoding => this; + + #region IKeyEncoding... + + IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + + IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + + ICompositeKeyEncoder IKeyEncoding.GetEncoder() + { + if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2) && typeof(T3B) != typeof(T3) && typeof(T4B) != typeof(T4)) throw new NotSupportedException(); + return (ICompositeKeyEncoder) (object) this; + } + + #endregion + } /// Create a simple encoder from a codec [NotNull] diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs index 3ec793ed4..baaa3177f 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -66,7 +66,7 @@ public void ReadKeyFrom(ref SliceReader reader, out T key) public Slice EncodeValue(T key) { - return TupleEncoder.EncodeKey(key); + return TupleEncoder.EncodeKey(default(Slice), key); } public T DecodeValue(Slice encoded) @@ -86,7 +86,7 @@ private TupleCompositeEncoder() { } public override IKeyEncoding Encoding => TypeSystem.Tuples; - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) key) { switch (count) { @@ -96,16 +96,14 @@ public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STup } } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) key) { if (count != 1 & count != 2) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); Contract.Assert(t != null); - key = new STuple( - t.Get(0), - count == 2 ? t.Get(1) : default - ); + key.Item1 = t.Get(0); + key.Item2 = count == 2 ? t.Get(1) : default; } } @@ -118,7 +116,7 @@ private TupleCompositeEncoder() { } public override IKeyEncoding Encoding => TypeSystem.Tuples; - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) key) { switch (count) { @@ -129,17 +127,15 @@ public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STup } } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) key) { if (count < 1 | count > 3) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); Contract.Assert(t != null); - key = new STuple( - t.Get(0), - count >= 2 ? t.Get(1) : default, - count >= 3 ? t.Get(2) : default - ); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; } } @@ -152,7 +148,7 @@ private TupleCompositeEncoder() { } public override IKeyEncoding Encoding => TypeSystem.Tuples; - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) key) { switch (count) { @@ -164,18 +160,16 @@ public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STup } } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) key) { if (count < 1 || count > 4) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); Contract.Assert(t != null); - key = new STuple( - t.Get(0), - count >= 2 ? t.Get(1) : default, - count >= 3 ? t.Get(2) : default, - count >= 4 ? t.Get(3) : default - ); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; } } @@ -188,7 +182,7 @@ private TupleCompositeEncoder() { } public override IKeyEncoding Encoding => TypeSystem.Tuples; - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple key) + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) key) { switch (count) { @@ -201,19 +195,55 @@ public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STup } } - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple key) + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) key) { if (count < 1 || count > 5) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); Contract.Assert(t != null); - key = new STuple( - t.Get(0), - count >= 2 ? t.Get(1) : default, - count >= 3 ? t.Get(2) : default, - count >= 4 ? t.Get(3) : default, - count >= 5 ? t.Get(4) : default - ); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; + key.Item5 = count >= 5 ? t.Get(4) : default; + } + } + + internal class TupleCompositeEncoder : CompositeKeyEncoder + { + + public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); + + private TupleCompositeEncoder() { } + + public override IKeyEncoding Encoding => TypeSystem.Tuples; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) key) + { + switch (count) + { + case 6: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5, key.Item6); break; + case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) key) + { + if (count < 1 || count > 6) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; + key.Item5 = count >= 5 ? t.Get(4) : default; + key.Item6 = count >= 6 ? t.Get(5) : default; } } diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs index 6a88e85c8..213f46e80 100644 --- a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs @@ -156,17 +156,17 @@ public abstract class CompositeKeyEncoder : ICompositeKeyEncoder public abstract IKeyEncoding Encoding { get; } - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) items); - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) items); [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void WriteKeyTo(ref SliceWriter writer, STuple items) + public void WriteKeyTo(ref SliceWriter writer, (T1, T2) items) { WriteKeyPartsTo(ref writer, 2, ref items); } - public void ReadKeyFrom(ref SliceReader reader, out STuple items) + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2) items) { ReadKeyPartsFrom(ref reader, 2, out items); } @@ -179,16 +179,16 @@ public abstract class CompositeKeyEncoder : ICompositeKeyEncoder items); + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) items); - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) items); - public void WriteKeyTo(ref SliceWriter writer, STuple items) + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3) items) { WriteKeyPartsTo(ref writer, 3, ref items); } - public void ReadKeyFrom(ref SliceReader reader, out STuple items) + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3) items) { ReadKeyPartsFrom(ref reader, 3, out items); } @@ -201,44 +201,66 @@ public abstract class CompositeKeyEncoder : ICompositeKeyEncoder public abstract IKeyEncoding Encoding { get; } - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) items); - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) items); - public void WriteKeyTo(ref SliceWriter writer, STuple items) + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4) items) { WriteKeyPartsTo(ref writer, 4, ref items); } - public void ReadKeyFrom(ref SliceReader reader, out STuple items) + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4) items) { ReadKeyPartsFrom(ref reader, 4, out items); } } - /// Wrapper for encoding and decoding a quad with lambda functions + /// Wrapper for encoding and decoding five items with lambda functions public abstract class CompositeKeyEncoder : ICompositeKeyEncoder { public abstract IKeyEncoding Encoding { get; } - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref STuple items); + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) items); - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out STuple items); + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) items); - public void WriteKeyTo(ref SliceWriter writer, STuple items) + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5) items) { WriteKeyPartsTo(ref writer, 5, ref items); } - public void ReadKeyFrom(ref SliceReader reader, out STuple items) + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5) items) { ReadKeyPartsFrom(ref reader, 5, out items); } } + /// Wrapper for encoding and decoding six items with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); + + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5, T6) items) + { + WriteKeyPartsTo(ref writer, 6, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5, T6) items) + { + ReadKeyPartsFrom(ref reader, 6, out items); + } + + } + #endregion #region Keys... From da5ace065ff31ddc7e3deb44493b2ac34d5d1070 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 13:48:53 +0200 Subject: [PATCH 114/153] Get rid of #if ENABLE_VALUETUPLES, ValueTuple support is now implicit --- .../Layers/Tuples/Encoding/TuplePackers.cs | 20 ------- FoundationDB.Client/Layers/Tuples/STuple.cs | 6 -- FoundationDB.Client/Layers/Tuples/STuple`1.cs | 15 +---- FoundationDB.Client/Layers/Tuples/STuple`2.cs | 13 +---- FoundationDB.Client/Layers/Tuples/STuple`3.cs | 15 +---- FoundationDB.Client/Layers/Tuples/STuple`4.cs | 15 +---- FoundationDB.Client/Layers/Tuples/STuple`5.cs | 15 +---- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 15 +---- FoundationDB.Client/Layers/Tuples/TuPack.cs | 6 -- .../Layers/Tuples/TupleComparisons.cs | 57 +------------------ .../Layers/Tuples/TupleExtensions.cs | 10 +--- .../Subspaces/DynamicKeySubspace.cs | 6 -- FoundationDB.Client/Subspaces/IKeySubspace.cs | 2 - FoundationDB.Client/Subspaces/KeySubspace.cs | 4 -- .../Subspaces/TypedKeySubspace`1.cs | 8 --- .../Subspaces/TypedKeySubspace`2.cs | 8 --- .../Subspaces/TypedKeySubspace`3.cs | 10 ---- .../Subspaces/TypedKeySubspace`4.cs | 10 ---- FoundationDB.Tests/Utils/TupleFacts.cs | 24 +------- 19 files changed, 12 insertions(+), 247 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index b1a37d5ea..96b79cfb4 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples.Encoding { using System; @@ -117,7 +115,6 @@ private static Delegate GetSerializerFor([NotNull] Type type) } } -#if ENABLE_VALUETUPLES if ((type.Name == nameof(System.ValueTuple) || type.Name.StartsWith(nameof(System.ValueTuple) + "`", StringComparison.Ordinal)) && type.Namespace == "System") { typeArgs = type.GetGenericArguments(); @@ -127,7 +124,6 @@ private static Delegate GetSerializerFor([NotNull] Type type) return method.MakeGenericMethod(typeArgs).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); } } -#endif // TODO: look for a static SerializeTo(ref TupleWriter, T) method on the type itself ? @@ -135,7 +131,6 @@ private static Delegate GetSerializerFor([NotNull] Type type) return null; } -#if ENABLE_VALUETUPLES private static MethodInfo FindValueTupleSerializerMethod(Type[] args) { //note: we want to find the correct SerializeValueTuple<...>(ref TupleWriter, ValueTuple<...>), but this cannot be done with Type.GetMethod(...) directly @@ -144,7 +139,6 @@ private static MethodInfo FindValueTupleSerializerMethod(Type[] args) .GetMethods(BindingFlags.Static | BindingFlags.Public) .SingleOrDefault(m => m.Name == nameof(SerializeValueTupleTo) && m.GetGenericArguments().Length == args.Length); } -#endif [MethodImpl(MethodImplOptions.AggressiveInlining)] internal static void SerializeTo(ref TupleWriter writer, T value) @@ -587,8 +581,6 @@ public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormatta TupleParser.EndTuple(ref writer); } -#if ENABLE_VALUETUPLES - public static void SerializeValueTupleTo(ref TupleWriter writer, ValueTuple tuple) { TupleParser.BeginTuple(ref writer); @@ -646,8 +638,6 @@ public static void SerializeValueTupleTo(ref TupleWriter TupleParser.EndTuple(ref writer); } -#endif - #endregion #region Deserializers... @@ -714,12 +704,10 @@ internal static Func GetDeserializer(bool required) return (Func) MakeSTupleDeserializer(type); } -#if ENABLE_VALUETUPLES if ((type.Name == nameof(ValueTuple) || type.Name.StartsWith(nameof(ValueTuple) + "`", StringComparison.Ordinal)) && type.Namespace == "System") { return (Func) MakeValueTupleDeserializer(type); } -#endif if (required) { // will throw at runtime @@ -793,8 +781,6 @@ private static Delegate MakeSTupleDeserializer(Type type) return Expression.Lambda(body, prmSlice).Compile(); } -#if ENABLE_VALUETUPLES - [Pure, NotNull] private static Delegate MakeValueTupleDeserializer(Type type) { @@ -821,8 +807,6 @@ private static Delegate MakeValueTupleDeserializer(Type type) return Expression.Lambda(body, prmSlice).Compile(); } -#endif - /// Deserialize a packed element into an object by choosing the most appropriate type at runtime /// Slice that contains a single packed element /// Decoded element, in the type that is the best fit. @@ -1162,8 +1146,6 @@ public static STuple DeserializeTuple DeserializeValueTuple(Slice slice) { @@ -1200,8 +1182,6 @@ public static (T1, T2, T3, T4, T5, T6) DeserializeValueTuple(slice).ToValueTuple(); } -#endif - /// Deserialize a tuple segment into a Boolean /// Slice that contains a single packed element public static bool DeserializeBoolean(Slice slice) diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Layers/Tuples/STuple.cs index 74f931821..a4597dccf 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -340,8 +338,6 @@ public static ITuple Concat([NotNull] ITuple head, [NotNull] ITuple tail) : new JoinedTuple(head, tail); } -#if ENABLE_VALUETUPLES - [Pure] public static STuple Create(ValueTuple tuple) { @@ -408,8 +404,6 @@ public static STuple Create(ref return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } -#endif - #endregion #region Internal Helpers... diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs index 250753afe..8620ff501 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -44,10 +42,7 @@ namespace Doxense.Collections.Tuples /// Tuple that holds only one item /// Type of the item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary singleton, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -208,12 +203,10 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) { return comparer.Equals(this.Item1, stuple.Item1); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -235,10 +228,6 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1); } -#if ENABLE_VALUETUPLES - - // interop with System.ValueTuple - public void Fill(ref ValueTuple t) { t.Item1 = this.Item1; @@ -337,8 +326,6 @@ bool IEquatable>.Equals(ValueTuple other) return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1); } -#endif - public sealed class Comparer : IComparer> { public static Comparer Default { [NotNull] get; } = new Comparer(); diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs index ce5772996..42d97bc46 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -45,10 +43,7 @@ namespace Doxense.Collections.Tuples /// Type of the first item /// Type of the second item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary pair, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -249,13 +244,11 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) return comparer.Equals(this.Item1, stuple.Item1) && comparer.Equals(this.Item2, stuple.Item2); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1) && comparer.Equals(this.Item2, vtuple.Item2); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -280,8 +273,6 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2); } -#if ENABLE_VALUETUPLES - // interop with System.ValueTuple public void Fill(ref ValueTuple t) @@ -370,8 +361,6 @@ bool IEquatable>.Equals(ValueTuple other) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } -#endif - public sealed class Comparer : IComparer> { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs index 7fec066f7..7cefec39a 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -46,10 +44,7 @@ namespace Doxense.Collections.Tuples /// Type of the second item /// Type of the third item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary triplet, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -277,14 +272,12 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item2, stuple.Item2) && comparer.Equals(this.Item3, stuple.Item3); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1) && comparer.Equals(this.Item2, vtuple.Item2) && comparer.Equals(this.Item3, vtuple.Item3); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -310,10 +303,6 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2, t.Item3); } -#if ENABLE_VALUETUPLES - - // interop with System.ValueTuple - public void Fill(ref ValueTuple t) { t.Item1 = this.Item1; @@ -406,8 +395,6 @@ bool IEquatable>.Equals(ValueTuple other) || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3); } -#endif - public sealed class Comparer : IComparer> { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs index 63ee35fcd..dc8d51db1 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -47,10 +45,7 @@ namespace Doxense.Collections.Tuples /// Type of the third item /// Type of the fourth item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -283,7 +278,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item3, stuple.Item3) && comparer.Equals(this.Item4, stuple.Item4); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1) @@ -291,7 +285,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item3, vtuple.Item3) && comparer.Equals(this.Item4, vtuple.Item4); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -318,10 +311,6 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4); } -#if ENABLE_VALUETUPLES - - // interop with System.ValueTuple - public void Fill(ref ValueTuple t) { t.Item1 = this.Item1; @@ -411,8 +400,6 @@ bool IEquatable>.Equals(ValueTuple ot || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4); } -#endif - public sealed class Comparer : IComparer> { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index ce6fa6ecb..39a016d52 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -48,10 +46,7 @@ namespace Doxense.Collections.Tuples /// Type of the 4th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -296,7 +291,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item4, stuple.Item4) && comparer.Equals(this.Item5, stuple.Item5); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1) @@ -305,7 +299,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item4, vtuple.Item4) && comparer.Equals(this.Item5, vtuple.Item5); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -333,10 +326,6 @@ public static explicit operator Tuple(STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } -#if ENABLE_VALUETUPLES - - // interop with System.ValueTuple - public void Fill(ref ValueTuple t) { t.Item1 = this.Item1; @@ -423,8 +412,6 @@ bool IEquatable>.Equals(ValueTuple> { diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs index e26cf24f5..713a338e2 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -49,10 +47,7 @@ namespace Doxense.Collections.Tuples /// Type of the 5th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable> -#if ENABLE_VALUETUPLES - , IEquatable> -#endif + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -312,7 +307,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item5, stuple.Item5) && comparer.Equals(this.Item6, stuple.Item6); } -#if ENABLE_VALUETUPLES if (other is ValueTuple vtuple) { return comparer.Equals(this.Item1, vtuple.Item1) @@ -322,7 +316,6 @@ bool IStructuralEquatable.Equals(object other, IEqualityComparer comparer) && comparer.Equals(this.Item5, vtuple.Item5) && comparer.Equals(this.Item6, vtuple.Item6); } -#endif return TupleHelpers.Equals(this, other, comparer); } @@ -351,10 +344,6 @@ public static explicit operator Tuple(STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } -#if ENABLE_VALUETUPLES - - // interop with System.ValueTuple - public void Fill(ref ValueTuple t) { t.Item1 = this.Item1; @@ -438,8 +427,6 @@ bool IEquatable>.Equals(ValueTuple> { diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index 0a02f99af..c3be8d694 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -110,8 +108,6 @@ public static Slice Pack(STuple return TupleEncoder.Pack(empty, ref tuple); } -#if ENABLE_VALUETUPLES - /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -166,8 +162,6 @@ public static Slice Pack(ValueTuplePack an array of N-tuples, all sharing the same buffer /// Sequence of N-tuples to pack /// Array containing the buffer segment of each packed tuple diff --git a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs index 4c77a7f4a..7c9c0c7e2 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -46,11 +44,6 @@ public static class TupleComparisons /// Tuple comparer that uses the default BCL object comparison ("123" != 123 != 123L != 123.0d) public static readonly EqualityComparer Bcl = new EqualityComparer(EqualityComparer.Default); -#if false - /// Tuple comparer that compared the packed bytes (slow!) - public static readonly BinaryComparer Binary = new BinaryComparer(); -#endif - public sealed class EqualityComparer : IEqualityComparer, IEqualityComparer { private readonly IEqualityComparer m_comparer; @@ -98,51 +91,6 @@ public int GetHashCode(object obj) } } -#if false - - public sealed class BinaryComparer : IEqualityComparer, IEqualityComparer - { - internal BinaryComparer() - { } - - - public bool Equals(ITuple x, ITuple y) - { - if (object.ReferenceEquals(x, y)) return true; - if (object.ReferenceEquals(x, null) || object.ReferenceEquals(y, null)) return false; - - return x.ToSlice().Equals(y.ToSlice()); - } - - public int GetHashCode(ITuple obj) - { - return object.ReferenceEquals(obj, null) ? 0 : obj.ToSlice().GetHashCode(); - } - - public new bool Equals(object x, object y) - { - if (object.ReferenceEquals(x, y)) return true; - if (x == null || y == null) return false; - - var tx = x as ITuple; - var ty = y as ITuple; - if (object.ReferenceEquals(tx, null) || object.ReferenceEquals(ty, null)) return false; - return tx.ToSlice().Equals(ty.ToSlice()); - } - - public int GetHashCode(object obj) - { - if (obj == null) return 0; - - var tuple = obj as ITuple; - if (!object.ReferenceEquals(tuple, null)) return tuple.ToSlice().GetHashCode(); - - return RuntimeHelpers.GetHashCode(obj); - } - } - -#endif - /// Create a new instance that compares a single item position in two tuples /// Type of the item to compare /// Offset of the item to compare (can be negative) @@ -237,10 +185,7 @@ public int Compare(ITuple x, ITuple y) /// Comparer that compares tuples with at least 2 items /// Type of the first item /// Type of the second item - public sealed class CompositeComparer : IComparer, IComparer> -#if ENABLE_VALUETUPLES - , IComparer<(T1, T2)> -#endif + public sealed class CompositeComparer : IComparer, IComparer>, IComparer<(T1, T2)> { public static readonly IComparer Default = new CompositeComparer(); diff --git a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs index 19bb2b22f..6136fe7c0 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace Doxense.Collections.Tuples { using System; @@ -565,7 +563,7 @@ public static TResult With([NotNull] th #endregion - #region Deconstruction (C#7) + #region Deconstruction [EditorBrowsable(EditorBrowsableState.Never)] public static void Deconstruct(this ITuple value, out T1 item1) @@ -652,9 +650,7 @@ public static void Deconstruct(this ITuple value #endregion - #region ValueTuple (C#7) - -#if ENABLE_VALUETUPLES + #region ValueTuple [Pure] public static STuple ToSTuple(this ValueTuple tuple) @@ -698,8 +694,6 @@ public static STuple ToSTuple(th return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } -#endif - #endregion } diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 3d0316756..49e00ef56 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace FoundationDB.Client { using System; @@ -122,8 +120,6 @@ public Slice Pack([NotNull] TTuple tuple) return sw.ToSlice(); } -#if ENABLE_VALUETUPLES - [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(ValueTuple items) { @@ -160,8 +156,6 @@ public Slice Pack((T1, T2, T3, T4, T5, T6) items) return Encode(items.Item1, items.Item2, items.Item3, items.Item4, items.Item5, items.Item6); } -#endif - /// Unpack a key of this subspace, back into a tuple /// Key that was produced by a previous call to /// Original tuple diff --git a/FoundationDB.Client/Subspaces/IKeySubspace.cs b/FoundationDB.Client/Subspaces/IKeySubspace.cs index 00c3d78c2..23383acd7 100644 --- a/FoundationDB.Client/Subspaces/IKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/IKeySubspace.cs @@ -29,8 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; - using Doxense.Memory; - using FoundationDB.Layers.Directories; using JetBrains.Annotations; /// Represents a sub-partition of the global key space. diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 7f8dc5245..d7b8b3a44 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -26,18 +26,14 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Client { using System; - using System.Collections.Generic; using System.Diagnostics; - using System.Linq; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using Doxense.Serialization.Encoders; - using FoundationDB.Layers.Directories; using JetBrains.Annotations; /// Adds a prefix on every keys, to group them inside a common subspace diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs index 35acb37c3..075760b9e 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`1.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace FoundationDB.Client { using System; @@ -108,7 +106,6 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified value /// Range that encompass all keys that start with (tuple.Item1, ..) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -116,7 +113,6 @@ public KeyRange ToRange(ValueTuple tuple) { return ToRange(tuple.Item1); } -#endif [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public KeyRange ToRange(T1 item1) @@ -129,13 +125,11 @@ public KeyRange ToRange(T1 item1) #region Pack() -#if ENABLE_VALUETUPLES public Slice this[ValueTuple items] { [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] get => Encode(items.Item1); } -#endif [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(STuple tuple) @@ -143,13 +137,11 @@ public Slice Pack(STuple tuple) return Encode(tuple.Item1); } -#if ENABLE_VALUETUPLES [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(ValueTuple tuple) { return Encode(tuple.Item1); } -#endif [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack([NotNull] TTuple tuple) diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs index 0ce670be9..85ce04d1b 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`2.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace FoundationDB.Client { using System; @@ -104,7 +102,6 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified pair of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, ..) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -112,7 +109,6 @@ public KeyRange ToRange((T1, T2) tuple) { return ToRange(tuple.Item1, tuple.Item2); } -#endif /// Return the range of all legal keys in this subpsace, that start with the specified pair of values /// Range that encompass all keys that start with (item1, item2, ..) @@ -156,7 +152,6 @@ public Slice Pack(STuple tuple) return Pack(tuple.ToValueTuple()); } -#if ENABLE_VALUETUPLES /// Pack a 2-tuple into a key in this subspace /// Pair of values /// Encoded key in this subspace @@ -168,7 +163,6 @@ public Slice Pack((T1, T2) tuple) this.Encoder.WriteKeyPartsTo(ref sw, 2, ref tuple); return sw.ToSlice(); } -#endif /// Pack a 2-tuple into a key in this subspace /// Tuple that must be of size 2 @@ -191,13 +185,11 @@ public Slice Pack([NotNull] TTuple tuple) get => Encode(item1, item2); } -#if ENABLE_VALUETUPLES public Slice this[(T1, T2) items] { [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] get => Encode(items.Item1, items.Item2); } -#endif /// Encode a pair of values into a key in this subspace /// First part of the key diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index 9fd7aa86c..8414ac263 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace FoundationDB.Client { using System; @@ -103,7 +101,6 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -111,7 +108,6 @@ public KeyRange ToRange((T1, T2, T3) tuple) { return ToRange(tuple.Item1, tuple.Item2, tuple.Item3); } -#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -133,7 +129,6 @@ public KeyRange ToRangePartial(STuple tuple) return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) public KeyRange ToRangePartial(ValueTuple tuple) @@ -141,7 +136,6 @@ public KeyRange ToRangePartial(ValueTuple tuple) //HACKHACK: add concept of "range" on IKeyEncoder ? return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); } -#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -169,13 +163,11 @@ public Slice Pack(STuple tuple) return Encode(tuple.Item1, tuple.Item2, tuple.Item3); } -#if ENABLE_VALUETUPLES [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack((T1, T2, T3) tuple) { return Encode(tuple.Item1, tuple.Item2, tuple.Item3); } -#endif [Pure] public Slice Pack(TTuple tuple) @@ -195,13 +187,11 @@ public Slice Pack(TTuple tuple) get => Encode(item1, item2, item3); } -#if ENABLE_VALUETUPLES public Slice this[(T1, T2, T3) items] { [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] get => Encode(items.Item1, items.Item2, items.Item3); } -#endif [Pure] public Slice Encode(T1 item1, T2 item2, T3 item3) diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index e64d79a8f..aefea4b3c 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - namespace FoundationDB.Client { using System; @@ -102,7 +100,6 @@ public KeyRange ToRange(STuple tuple) return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (tuple.Item1, tuple.Item2, tuple.Item3) [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] @@ -110,7 +107,6 @@ public KeyRange ToRange((T1, T2, T3, T4) tuple) { return ToRange(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } -#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -132,7 +128,6 @@ public KeyRange ToRangePartial(STuple tuple) return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); } -#if ENABLE_VALUETUPLES /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) public KeyRange ToRangePartial((T1, T2, T3) tuple) @@ -140,7 +135,6 @@ public KeyRange ToRangePartial((T1, T2, T3) tuple) //HACKHACK: add concept of "range" on IKeyEncoder ? return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2, tuple.Item3)); } -#endif /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) @@ -176,13 +170,11 @@ public KeyRange ToRangePartial(T1 item1) get => Encode(item1, item2, item3, item4); } -#if ENABLE_VALUETUPLES public Slice this[(T1, T2, T3, T4) items] { [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] get => Encode(items.Item1, items.Item2, items.Item3, items.Item4); } -#endif [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(STuple tuple) @@ -190,13 +182,11 @@ public Slice Pack(STuple tuple) return Encode(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } -#if ENABLE_VALUETUPLES [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] public Slice Pack(ValueTuple tuple) { return Encode(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } -#endif [Pure] public Slice Pack(TTuple tuple) diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index d0610e384..1ff7a88e8 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -#define ENABLE_VALUETUPLES - // ReSharper disable AccessToModifiedClosure namespace Doxense.Collections.Tuples.Tests { @@ -121,13 +119,11 @@ public void Test_Tuple_2() Assert.That(item1, Is.EqualTo("hello world")); Assert.That(item2, Is.EqualTo(123)); } -#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2) = t2; Assert.That(item1, Is.EqualTo("hello world")); Assert.That(item2, Is.EqualTo(123)); } -#endif } [Test] @@ -174,14 +170,12 @@ public void Test_Tuple_3() Assert.That(item2, Is.EqualTo(123)); Assert.That(item3, Is.False); } -#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3) = t3; Assert.That(item1, Is.EqualTo("hello world")); Assert.That(item2, Is.EqualTo(123)); Assert.That(item3, Is.False); } -#endif } [Test] @@ -235,7 +229,6 @@ public void Test_Tuple_4() Assert.That(item3, Is.False); Assert.That(item4, Is.EqualTo(1234L)); } -#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4) = t4; Assert.That(item1, Is.EqualTo("hello world")); @@ -243,7 +236,6 @@ public void Test_Tuple_4() Assert.That(item3, Is.False); Assert.That(item4, Is.EqualTo(1234L)); } -#endif } [Test] @@ -287,7 +279,6 @@ public void Test_Tuple_5() Assert.That(item4, Is.EqualTo(1234L)); Assert.That(item5, Is.EqualTo(-1234L)); } -#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4, long item5) = t5; Assert.That(item1, Is.EqualTo("hello world")); @@ -296,7 +287,6 @@ public void Test_Tuple_5() Assert.That(item4, Is.EqualTo(1234L)); Assert.That(item5, Is.EqualTo(-1234L)); } -#endif } [Test] @@ -344,7 +334,6 @@ public void Test_Tuple_6() Assert.That(item5, Is.EqualTo(-1234L)); Assert.That(item6, Is.EqualTo("six")); } -#if ENABLE_VALUETUPLES { // Deconstruct (string item1, int item2, bool item3, long item4, long item5, string item6) = t6; Assert.That(item1, Is.EqualTo("hello world")); @@ -354,7 +343,6 @@ public void Test_Tuple_6() Assert.That(item5, Is.EqualTo(-1234L)); Assert.That(item6, Is.EqualTo("six")); } -#endif } [Test] @@ -1936,8 +1924,6 @@ void ITupleFormattable.FromTuple(ITuple tuple) #region System.ValueTuple integration... -#if ENABLE_VALUETUPLES - [Test] public void Test_Implicit_Cast_STuple_To_ValueTuple() { @@ -2055,8 +2041,6 @@ public void Test_Can_AutoCast_Transparently() } } -#endif - [Test] public void Test_Deconstruct_STuple() { @@ -2087,17 +2071,16 @@ public void Test_Deconstruct_STuple() Assert.That(e, Is.EqualTo(55)); } { - STuple.Create(11, 22, 33, 44, 55, 66).Deconstruct(out int a, out int b, out int c, out int d, out int e, out _); + STuple.Create(11, 22, 33, 44, 55, 66).Deconstruct(out int a, out int b, out int c, out int d, out int e, out int f); Assert.That(a, Is.EqualTo(11)); Assert.That(b, Is.EqualTo(22)); Assert.That(c, Is.EqualTo(33)); Assert.That(d, Is.EqualTo(44)); Assert.That(e, Is.EqualTo(55)); + Assert.That(f, Is.EqualTo(66)); } } -#if ENABLE_VALUETUPLES - [Test] public void Test_Deconstruct_STuple_TupleSyntax() { @@ -2134,11 +2117,10 @@ public void Test_Deconstruct_STuple_TupleSyntax() Assert.That(c, Is.EqualTo(33)); Assert.That(d, Is.EqualTo(44)); Assert.That(e, Is.EqualTo(55)); + Assert.That(f, Is.EqualTo(66)); } } -#endif - #endregion } From ce869dda007d8e4a5cc0af7dc5ec29188c46caaf Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 14:00:40 +0200 Subject: [PATCH 115/153] Use the literal tuple syntax everywhere possible --- .../Layers/Tuples/Encoding/TuplePackers.cs | 2 +- FoundationDB.Client/Layers/Tuples/STuple`1.cs | 8 ++--- FoundationDB.Client/Layers/Tuples/STuple`2.cs | 30 +++++++++---------- FoundationDB.Client/Layers/Tuples/STuple`3.cs | 26 ++++++++-------- FoundationDB.Client/Layers/Tuples/STuple`4.cs | 26 ++++++++-------- FoundationDB.Client/Layers/Tuples/STuple`5.cs | 24 +++++++-------- FoundationDB.Client/Layers/Tuples/STuple`6.cs | 24 +++++++-------- FoundationDB.Client/Layers/Tuples/TuPack.cs | 10 +++---- .../Layers/Tuples/TupleExtensions.cs | 10 +++---- .../Subspaces/TypedKeySubspace`3.cs | 2 +- .../Subspaces/TypedKeySubspace`4.cs | 2 +- FoundationDB.Tests/SubspaceFacts.cs | 2 +- FoundationDB.Tests/Utils/TupleFacts.cs | 24 +++++++-------- 13 files changed, 94 insertions(+), 96 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index 96b79cfb4..7fa2a4d49 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -133,7 +133,7 @@ private static Delegate GetSerializerFor([NotNull] Type type) private static MethodInfo FindValueTupleSerializerMethod(Type[] args) { - //note: we want to find the correct SerializeValueTuple<...>(ref TupleWriter, ValueTuple<...>), but this cannot be done with Type.GetMethod(...) directly + //note: we want to find the correct SerializeValueTuple<...>(ref TupleWriter, (...,), but this cannot be done with Type.GetMethod(...) directly // => we have to scan for all methods with the correct name, and the same number of Type Arguments than the ValueTuple. return typeof(TuplePackers) .GetMethods(BindingFlags.Static | BindingFlags.Public) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Layers/Tuples/STuple`1.cs index 8620ff501..53a1f3e56 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`1.cs @@ -246,7 +246,7 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T2, T3) tuple) { return new STuple(this.Item1, tuple.Item1, tuple.Item2); } @@ -255,7 +255,7 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T2, T3, T4) tuple) { return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3); } @@ -264,7 +264,7 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T2, T3, T4, T5) tuple) { return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } @@ -273,7 +273,7 @@ public STuple Concat(ValueTupleTuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T2, T3, T4, T5, T6) tuple) { return new STuple(this.Item1, tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Layers/Tuples/STuple`2.cs index 42d97bc46..0c48ed299 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`2.cs @@ -43,7 +43,7 @@ namespace Doxense.Collections.Tuples /// Type of the first item /// Type of the second item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2)> { // This is mostly used by code that create a lot of temporary pair, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -273,9 +273,7 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2); } - // interop with System.ValueTuple - - public void Fill(ref ValueTuple t) + public void Fill(ref (T1, T2) t) { t.Item1 = this.Item1; t.Item2 = this.Item2; @@ -294,7 +292,7 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T3, T4) tuple) { return new STuple(this.Item1, this.Item2, tuple.Item1, tuple.Item2); } @@ -303,59 +301,59 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T3, T4, T5) tuple) { return new STuple(this.Item1, this.Item2, tuple.Item1, tuple.Item2, tuple.Item3); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTuple ToValueTuple() + public (T1, T2) ToValueTuple() { - return ValueTuple.Create(this.Item1, this.Item2); + return (this.Item1, this.Item2); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator STuple(ValueTuple t) + public static implicit operator STuple((T1, T2) t) { return new STuple(t.Item1, t.Item2); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator ValueTuple(STuple t) + public static implicit operator (T1, T2)(STuple t) { - return ValueTuple.Create(t.Item1, t.Item2); + return (t.Item1, t.Item2); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IEquatable>.Equals(ValueTuple other) + bool IEquatable<(T1, T2)>.Equals((T1, T2) other) { return SimilarValueComparer.Default.Equals(this.Item1, other.Item1) && SimilarValueComparer.Default.Equals(this.Item2, other.Item2); } - public static bool operator ==(STuple left, ValueTuple right) + public static bool operator ==(STuple left, (T1, T2) right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } - public static bool operator ==(ValueTuple left, STuple right) + public static bool operator ==((T1, T2) left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } - public static bool operator !=(STuple left, ValueTuple right) + public static bool operator !=(STuple left, (T1, T2) right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); } - public static bool operator !=(ValueTuple left, STuple right) + public static bool operator !=((T1, T2) left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2); diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Layers/Tuples/STuple`3.cs index 7cefec39a..ee15aad06 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`3.cs @@ -44,7 +44,7 @@ namespace Doxense.Collections.Tuples /// Type of the second item /// Type of the third item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3)> { // This is mostly used by code that create a lot of temporary triplet, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -303,7 +303,7 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2, t.Item3); } - public void Fill(ref ValueTuple t) + public void Fill(ref (T1, T2, T3) t) { t.Item1 = this.Item1; t.Item2 = this.Item2; @@ -332,63 +332,63 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T4, T5, T6) tuple) { return new STuple(this.Item1, this.Item2, this.Item3, tuple.Item1, tuple.Item2, tuple.Item3); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTuple ToValueTuple() + public (T1, T2, T3) ToValueTuple() { - return ValueTuple.Create(this.Item1, this.Item2, this.Item3); + return (this.Item1, this.Item2, this.Item3); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator STuple(ValueTuple t) + public static implicit operator STuple((T1, T2, T3) t) { return new STuple(t.Item1, t.Item2, t.Item3); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator ValueTuple (STuple t) + public static implicit operator (T1, T2, T3) (STuple t) { - return ValueTuple.Create(t.Item1, t.Item2, t.Item3); + return (t.Item1, t.Item2, t.Item3); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IEquatable>.Equals(ValueTuple other) + bool IEquatable<(T1, T2, T3)>.Equals((T1, T2, T3) other) { return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) && SimilarValueComparer.Default.Equals(this.Item3, this.Item3); } - public static bool operator ==(STuple left, ValueTuple right) + public static bool operator ==(STuple left, (T1, T2, T3) right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) && SimilarValueComparer.Default.Equals(left.Item3, right.Item3); } - public static bool operator ==(ValueTuple left, STuple right) + public static bool operator ==((T1, T2, T3) left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) && SimilarValueComparer.Default.Equals(left.Item3, right.Item3); } - public static bool operator !=(STuple left, ValueTuple right) + public static bool operator !=(STuple left, (T1, T2, T3) right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) || !SimilarValueComparer.Default.Equals(left.Item3, right.Item3); } - public static bool operator !=(ValueTuple left, STuple right) + public static bool operator !=((T1, T2, T3) left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Layers/Tuples/STuple`4.cs index dc8d51db1..e6bbb3fbb 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`4.cs @@ -45,7 +45,7 @@ namespace Doxense.Collections.Tuples /// Type of the third item /// Type of the fourth item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -311,7 +311,7 @@ public static explicit operator Tuple(STuple t) return new Tuple(t.Item1, t.Item2, t.Item3, t.Item4); } - public void Fill(ref ValueTuple t) + public void Fill(ref (T1, T2, T3, T4) t) { t.Item1 = this.Item1; t.Item2 = this.Item2; @@ -332,35 +332,35 @@ public STuple Concat(ValueTuple tuple) /// Tuple whose items are to be appended at the end /// New tuple composed of the current tuple's items, followed by 's items [Pure] - public STuple Concat(ValueTuple tuple) + public STuple Concat((T5, T6) tuple) { return new STuple(this.Item1, this.Item2, this.Item3, this.Item4, tuple.Item1, tuple.Item2); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTuple ToValueTuple() + public (T1, T2, T3, T4) ToValueTuple() { - return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4); + return (this.Item1, this.Item2, this.Item3, this.Item4); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator STuple(ValueTuple t) + public static implicit operator STuple((T1, T2, T3, T4) t) { return new STuple(t.Item1, t.Item2, t.Item3, t.Item4); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator ValueTuple(STuple t) + public static implicit operator (T1, T2, T3, T4) (STuple t) { - return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4); + return (t.Item1, t.Item2, t.Item3, t.Item4); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IEquatable>.Equals(ValueTuple other) + bool IEquatable<(T1, T2, T3, T4)>.Equals((T1, T2, T3, T4) other) { return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) @@ -368,7 +368,7 @@ bool IEquatable>.Equals(ValueTuple ot && SimilarValueComparer.Default.Equals(this.Item4, this.Item4); } - public static bool operator ==(STuple left, ValueTuple right) + public static bool operator ==(STuple left, (T1, T2, T3, T4) right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -376,7 +376,7 @@ bool IEquatable>.Equals(ValueTuple ot && SimilarValueComparer.Default.Equals(left.Item4, right.Item4); } - public static bool operator ==(ValueTuple left, STuple right) + public static bool operator ==((T1, T2, T3, T4) left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -384,7 +384,7 @@ bool IEquatable>.Equals(ValueTuple ot && SimilarValueComparer.Default.Equals(left.Item4, right.Item4); } - public static bool operator !=(STuple left, ValueTuple right) + public static bool operator !=(STuple left, (T1, T2, T3, T4) right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -392,7 +392,7 @@ bool IEquatable>.Equals(ValueTuple ot || !SimilarValueComparer.Default.Equals(left.Item4, right.Item4); } - public static bool operator !=(ValueTuple left, STuple right) + public static bool operator !=((T1, T2, T3, T4) left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Layers/Tuples/STuple`5.cs index 39a016d52..773d1c862 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`5.cs @@ -46,7 +46,7 @@ namespace Doxense.Collections.Tuples /// Type of the 4th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4, T5)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -326,7 +326,7 @@ public static explicit operator Tuple(STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } - public void Fill(ref ValueTuple t) + public void Fill(ref (T1, T2, T3, T4, T5) t) { t.Item1 = this.Item1; t.Item2 = this.Item2; @@ -346,28 +346,28 @@ public STuple Concat(ValueTuple tuple) [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTuple ToValueTuple() + public (T1, T2, T3, T4, T5) ToValueTuple() { - return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); + return (this.Item1, this.Item2, this.Item3, this.Item4, this.Item5); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator STuple(ValueTuple t) + public static implicit operator STuple((T1, T2, T3, T4, T5) t) { return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator ValueTuple(STuple t) + public static implicit operator (T1, T2, T3, T4, T5) (STuple t) { - return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); + return (t.Item1, t.Item2, t.Item3, t.Item4, t.Item5); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IEquatable>.Equals(ValueTuple other) + bool IEquatable<(T1, T2, T3, T4, T5)>.Equals((T1, T2, T3, T4, T5) other) { return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) @@ -376,7 +376,7 @@ bool IEquatable>.Equals(ValueTuple left, ValueTuple right) + public static bool operator ==(STuple left, (T1, T2, T3, T4, T5) right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -385,7 +385,7 @@ bool IEquatable>.Equals(ValueTuple left, STuple right) + public static bool operator ==((T1, T2, T3, T4, T5) left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -394,7 +394,7 @@ bool IEquatable>.Equals(ValueTuple left, ValueTuple right) + public static bool operator !=(STuple left, (T1, T2, T3, T4, T5) right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -403,7 +403,7 @@ bool IEquatable>.Equals(ValueTuple left, STuple right) + public static bool operator !=((T1, T2, T3, T4, T5) left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Layers/Tuples/STuple`6.cs index 713a338e2..7e10645c8 100644 --- a/FoundationDB.Client/Layers/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Layers/Tuples/STuple`6.cs @@ -47,7 +47,7 @@ namespace Doxense.Collections.Tuples /// Type of the 5th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4, T5, T6)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -344,7 +344,7 @@ public static explicit operator Tuple(STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } - public void Fill(ref ValueTuple t) + public void Fill(ref (T1, T2, T3, T4, T5, T6) t) { t.Item1 = this.Item1; t.Item2 = this.Item2; @@ -356,28 +356,28 @@ public void Fill(ref ValueTuple t) [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public ValueTuple ToValueTuple() + public (T1, T2, T3, T4, T5, T6) ToValueTuple() { - return ValueTuple.Create(this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); + return (this.Item1, this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator STuple(ValueTuple t) + public static implicit operator STuple((T1, T2, T3, T4, T5, T6) t) { return new STuple(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5, t.Item6); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - public static implicit operator ValueTuple(STuple t) + public static implicit operator (T1, T2, T3, T4, T5, T6) (STuple t) { - return ValueTuple.Create(t.Item1, t.Item2, t.Item3, t.Item4, t.Item5, t.Item6); + return (t.Item1, t.Item2, t.Item3, t.Item4, t.Item5, t.Item6); } [Pure] [MethodImpl(MethodImplOptions.AggressiveInlining)] - bool IEquatable>.Equals(ValueTuple other) + bool IEquatable<(T1, T2, T3, T4, T5, T6)>.Equals((T1, T2, T3, T4, T5, T6) other) { return SimilarValueComparer.Default.Equals(this.Item1, this.Item1) && SimilarValueComparer.Default.Equals(this.Item2, this.Item2) @@ -387,7 +387,7 @@ bool IEquatable>.Equals(ValueTuple left, ValueTuple right) + public static bool operator ==(STuple left, (T1, T2, T3, T4, T5, T6) right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -397,7 +397,7 @@ bool IEquatable>.Equals(ValueTuple left, STuple right) + public static bool operator ==((T1, T2, T3, T4, T5, T6) left, STuple right) { return SimilarValueComparer.Default.Equals(left.Item1, right.Item1) && SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -407,7 +407,7 @@ bool IEquatable>.Equals(ValueTuple left, ValueTuple right) + public static bool operator !=(STuple left, (T1, T2, T3, T4, T5, T6) right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) @@ -417,7 +417,7 @@ bool IEquatable>.Equals(ValueTuple left, STuple right) + public static bool operator !=((T1, T2, T3, T4, T5, T6) left, STuple right) { return !SimilarValueComparer.Default.Equals(left.Item1, right.Item1) || !SimilarValueComparer.Default.Equals(left.Item2, right.Item2) diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index c3be8d694..110138d35 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -120,7 +120,7 @@ public static Slice Pack(ValueTuple tuple) /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static Slice Pack(ValueTuple tuple) + public static Slice Pack((T1, T2) tuple) { var empty = default(Slice); return TupleEncoder.Pack(empty, tuple.ToSTuple()); @@ -129,7 +129,7 @@ public static Slice Pack(ValueTuple tuple) /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static Slice Pack(ValueTuple tuple) + public static Slice Pack((T1, T2, T3) tuple) { var empty = default(Slice); return TupleEncoder.Pack(empty, tuple.ToSTuple()); @@ -138,7 +138,7 @@ public static Slice Pack(ValueTuple tuple) /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static Slice Pack(ValueTuple tuple) + public static Slice Pack((T1, T2, T3, T4) tuple) { var empty = default(Slice); return TupleEncoder.Pack(empty, tuple.ToSTuple()); @@ -147,7 +147,7 @@ public static Slice Pack(ValueTuple tuple) /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static Slice Pack(ValueTuple tuple) + public static Slice Pack((T1, T2, T3, T4, T5) tuple) { var empty = default(Slice); return TupleEncoder.Pack(empty, tuple.ToSTuple()); @@ -156,7 +156,7 @@ public static Slice Pack(ValueTuple tupl /// Pack a tuple into a slice /// Tuple that must be serialized into a binary slice [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public static Slice Pack(ValueTuple tuple) + public static Slice Pack((T1, T2, T3, T4, T5, T6) tuple) { var empty = default(Slice); return TupleEncoder.Pack(empty, tuple.ToSTuple()); diff --git a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs index 6136fe7c0..0c0a47911 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs +++ b/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs @@ -665,31 +665,31 @@ public static STuple ToSTuple(this ValueTuple tuple) } [Pure] - public static STuple ToSTuple(this ValueTuple tuple) + public static STuple ToSTuple(this (T1, T2) tuple) { return new STuple(tuple.Item1, tuple.Item2); } [Pure] - public static STuple ToSTuple(this ValueTuple tuple) + public static STuple ToSTuple(this (T1, T2, T3) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3); } [Pure] - public static STuple ToSTuple(this ValueTuple tuple) + public static STuple ToSTuple(this (T1, T2, T3, T4) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } [Pure] - public static STuple ToSTuple(this ValueTuple tuple) + public static STuple ToSTuple(this (T1, T2, T3, T4, T5) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5); } [Pure] - public static STuple ToSTuple(this ValueTuple tuple) + public static STuple ToSTuple(this (T1, T2, T3, T4, T5, T6) tuple) { return new STuple(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4, tuple.Item5, tuple.Item6); } diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs index 8414ac263..91593a3df 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`3.cs @@ -131,7 +131,7 @@ public KeyRange ToRangePartial(STuple tuple) /// Return the range of all legal keys in this subpsace, that start with the specified triple of values /// Range that encompass all keys that start with (item1, item2, item3) - public KeyRange ToRangePartial(ValueTuple tuple) + public KeyRange ToRangePartial((T1, T2) tuple) { //HACKHACK: add concept of "range" on IKeyEncoder ? return KeyRange.PrefixedBy(EncodePartial(tuple.Item1, tuple.Item2)); diff --git a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs index aefea4b3c..16b870e4a 100644 --- a/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs +++ b/FoundationDB.Client/Subspaces/TypedKeySubspace`4.cs @@ -183,7 +183,7 @@ public Slice Pack(STuple tuple) } [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] - public Slice Pack(ValueTuple tuple) + public Slice Pack((T1, T2, T3, T4) tuple) { return Encode(tuple.Item1, tuple.Item2, tuple.Item3, tuple.Item4); } diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 470138584..e61384837 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -81,7 +81,7 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(t2.Get(2), Is.False); // ValueTuple - Assert.That(subspace.Keys.Pack(new ValueTuple("hello")).ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); + Assert.That(subspace.Keys.Pack(ValueTuple.Create("hello")).ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); Assert.That(subspace.Keys.Pack(("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); Assert.That(subspace.Keys.Pack(("hello", 123, "world")).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{<02>world<00>")); Assert.That(subspace.Keys.Pack(("hello", 123, "world", 456)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{<02>world<00><16><01>")); diff --git a/FoundationDB.Tests/Utils/TupleFacts.cs b/FoundationDB.Tests/Utils/TupleFacts.cs index 1ff7a88e8..8921a5709 100644 --- a/FoundationDB.Tests/Utils/TupleFacts.cs +++ b/FoundationDB.Tests/Utils/TupleFacts.cs @@ -1932,25 +1932,25 @@ public void Test_Implicit_Cast_STuple_To_ValueTuple() Assert.That(t.Item1, Is.EqualTo(11)); } { - ValueTuple t = STuple.Create(11, 22); + (int, int) t = STuple.Create(11, 22); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); } { - ValueTuple t = STuple.Create(11, 22, 33); + (int, int, int) t = STuple.Create(11, 22, 33); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); } { - ValueTuple t = STuple.Create(11, 22, 33, 44); + (int, int, int, int) t = STuple.Create(11, 22, 33, 44); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); Assert.That(t.Item4, Is.EqualTo(44)); } { - ValueTuple t = STuple.Create(11, 22, 33, 44, 55); + (int, int, int, int, int) t = STuple.Create(11, 22, 33, 44, 55); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); @@ -1958,7 +1958,7 @@ public void Test_Implicit_Cast_STuple_To_ValueTuple() Assert.That(t.Item5, Is.EqualTo(55)); } { - ValueTuple t = STuple.Create(11, 22, 33, 44, 55, 66); + (int, int, int, int, int, int) t = STuple.Create(11, 22, 33, 44, 55, 66); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); @@ -1976,25 +1976,25 @@ public void Test_Implicit_Cast_ValueTuple_To_STuple() Assert.That(t.Item1, Is.EqualTo(11)); } { - STuple t = ValueTuple.Create(11, 22); + STuple t = (11, 22); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); } { - STuple t = ValueTuple.Create(11, 22, 33); + STuple t = (11, 22, 33); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); } { - STuple t = ValueTuple.Create(11, 22, 33, 44); + STuple t = (11, 22, 33, 44); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); Assert.That(t.Item4, Is.EqualTo(44)); } { - STuple t = ValueTuple.Create(11, 22, 33, 44, 55); + STuple t = (11, 22, 33, 44, 55); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); @@ -2002,7 +2002,7 @@ public void Test_Implicit_Cast_ValueTuple_To_STuple() Assert.That(t.Item5, Is.EqualTo(55)); } { - STuple t = ValueTuple.Create(11, 22, 33, 44, 55, 66); + STuple t = (11, 22, 33, 44, 55, 66); Assert.That(t.Item1, Is.EqualTo(11)); Assert.That(t.Item2, Is.EqualTo(22)); Assert.That(t.Item3, Is.EqualTo(33)); @@ -2012,7 +2012,7 @@ public void Test_Implicit_Cast_ValueTuple_To_STuple() } } - private static ValueTuple ProduceValueTuple(int item1, int item2) => ValueTuple.Create(item1, item2); + private static (int, int) ProduceValueTuple(int item1, int item2) => (item1, item2); private static int[] ConsumeValueTuple(STuple t) => new[] { t.Item1, t.Item2 }; @@ -2030,7 +2030,7 @@ public void Test_Can_AutoCast_Transparently() Assert.That(res[1], Is.EqualTo(5)); } { // literal => STuple - var res = ConsumeSTuple(ValueTuple.Create(1234, 5)); + var res = ConsumeSTuple((1234, 5)); Assert.That(res[0], Is.EqualTo(1234)); Assert.That(res[1], Is.EqualTo(5)); } From da34f2adeedee9ac717bf280802bf3d64a78ccbf Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 14:10:01 +0200 Subject: [PATCH 116/153] Replace some STuple uses with ValueTuple --- FoundationDB.Client/Fdb.System.cs | 4 +-- FoundationDB.Client/Subspaces/KeySubspace.cs | 29 ------------------- .../Expressions/FdbQueryExpressions.cs | 1 + FoundationDB.Tests.Sandbox/Program.cs | 2 +- FoundationDB.Tests/DatabaseFacts.cs | 2 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 2 +- .../Linq/AsyncEnumerableFacts.cs | 16 +++++----- .../Linq/FdbQueryExpressionFacts.cs | 10 +++---- FoundationDB.Tests/RangeQueryFacts.cs | 8 ++--- FoundationDB.Tests/SubspaceFacts.cs | 17 ++++++----- 10 files changed, 32 insertions(+), 59 deletions(-) diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index e5a915691..bacb2e3d4 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -538,7 +538,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli .ConfigureAwait(false); counter += n; - if (onProgress != null) onProgress.Report(STuple.Create(counter, end)); + onProgress?.Report((counter, end)); #if TRACE_COUNTING ++iter; #endif @@ -552,7 +552,7 @@ public static async Task EstimateCountAsync([NotNull] IFdbDatabase db, Sli // the range is not finished, advance the cursor counter += windowSize; cursor = next; - if (onProgress != null) onProgress.Report(STuple.Create(counter, cursor)); + onProgress?.Report((counter, cursor)); if (!last) { // double the size of the window if we are not in the last segment diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index d7b8b3a44..0298c54f2 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -140,35 +140,6 @@ public static TypedKeySubspace FromKey(Slice pre return new TypedKeySubspace(prefix, encoder); } - /// Initializes a new generic subspace with the given . - /// A subspace that can handle keys of any types and size. - [Pure, NotNull] - public static KeySubspace FromKey(ITuple prefix) - { - //REVIEW: this is tied to the Tuple Layer. Maybe this should be an extension method that lives in that namespace? - return new KeySubspace(TuPack.Pack(prefix).Memoize()); - } - - /// Initializes a new dynamic subspace with the given and key . - /// A subspace that can handle keys of any types and size. - [Pure, NotNull] - public static DynamicKeySubspace FromKey(ITuple prefix, [NotNull] IDynamicKeyEncoder encoder) - { - Contract.NotNull(encoder, nameof(encoder)); - var writer = new SliceWriter(); - encoder.PackKey(ref writer, prefix); - return new DynamicKeySubspace(writer.ToSlice(), encoder); - } - - /// Initializes a new subspace with the given , that uses a dynamic key . - /// A subspace that can handle keys of any types and size. - [Pure, NotNull] - public static DynamicKeySubspace FromKey(ITuple prefix, [NotNull] IKeyEncoding encoding) - { - Contract.NotNull(encoding, nameof(encoding)); - return FromKey(prefix, encoding.GetDynamicEncoder()); - } - #endregion #region Copy... diff --git a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs index 892e85292..9c2f77bc7 100644 --- a/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs +++ b/FoundationDB.Linq.Providers/Expressions/FdbQueryExpressions.cs @@ -86,6 +86,7 @@ public static FdbQueryRangeExpression RangeStartsWith(Slice prefix, FdbRangeOpti /// Execute a Range read from the database, and return all the keys and values [NotNull] + [Obsolete] public static FdbQueryRangeExpression RangeStartsWith(ITuple tuple, FdbRangeOptions options = null) { return RangeStartsWith(TuPack.Pack(tuple), options); diff --git a/FoundationDB.Tests.Sandbox/Program.cs b/FoundationDB.Tests.Sandbox/Program.cs index cb1b38c8a..81e276dd8 100644 --- a/FoundationDB.Tests.Sandbox/Program.cs +++ b/FoundationDB.Tests.Sandbox/Program.cs @@ -179,7 +179,7 @@ private static async Task MainAsync(CancellationToken ct) Console.WriteLine("> Connected!"); Console.WriteLine("Opening database 'DB'..."); - using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.FromKey(STuple.Create(SUBSPACE)), false, ct)) + using (var db = await cluster.OpenDatabaseAsync(DB_NAME, KeySubspace.FromKey(TuPack.EncodeKey(SUBSPACE)), false, ct)) { Console.WriteLine("> Connected to db '{0}'", db.Name); diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index aabdf968d..2dc0b7cb2 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -278,7 +278,7 @@ public async Task Test_Can_Get_System_Status() public async Task Test_Can_Open_Database_With_Non_Empty_GlobalSpace() { // using a tuple prefix - using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.FromKey(STuple.Create("test")), false, this.Cancellation)) + using (var db = await Fdb.OpenAsync(null, "DB", KeySubspace.FromKey(TuPack.EncodeKey("test")), false, this.Cancellation)) { Assert.That(db, Is.Not.Null); Assert.That(db.GlobalSpace, Is.Not.Null); diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index 06bd8d476..c3fe5133a 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -539,7 +539,7 @@ public async Task Test_Can_Change_Layer_Of_Existing_Directory() Assert.That(folder2, Is.Not.Null); Assert.That(folder2.Layer.ToUnicode(), Is.EqualTo("bar")); Assert.That(folder2.FullName, Is.EqualTo("Test")); - Assert.That(folder2.Path, Is.EqualTo(STuple.Create("Test"))); + Assert.That(folder2.Path, Is.EqualTo(new [] { "Test" })); Assert.That(folder2.GetPrefix(), Is.EqualTo(folder.GetPrefix())); // opening the directory with the new layer should succeed diff --git a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs index 41aa83fc0..0a54272f2 100644 --- a/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs +++ b/FoundationDB.Tests/Linq/AsyncEnumerableFacts.cs @@ -962,7 +962,7 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() { Interlocked.Increment(ref called); if (index >= 10) return Task.FromResult(Maybe.Nothing()); - return Task.Delay(15).ContinueWith((_) => Maybe.Return((int)index)); + return Task.Delay(15, ct).ContinueWith((_) => Maybe.Return((int)index), ct); }); var results = await source.ToListAsync(); @@ -971,23 +971,23 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() // record the timing and call history to ensure that inner is called at least twice before the first item gets out - Func> record = (x) => STuple.Create(x, Volatile.Read(ref called)); + Func record = (x) => STuple.Create(x, Volatile.Read(ref called)); // without prefetching, the number of calls should match for the producer and the consumer called = 0; sw.Restart(); var withoutPrefetching = await source.Select(record).ToListAsync(this.Cancellation); Log("P0: {0}", String.Join(", ", withoutPrefetching)); - Assert.That(withoutPrefetching.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); - Assert.That(withoutPrefetching.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(1, 10))); + Assert.That(withoutPrefetching.Select(x => x.Value), Is.EqualTo(Enumerable.Range(0, 10))); + Assert.That(withoutPrefetching.Select(x => x.Called), Is.EqualTo(Enumerable.Range(1, 10))); // with prefetching, the consumer should always have one item in advance called = 0; sw.Restart(); var withPrefetching1 = await source.Prefetch().Select(record).ToListAsync(this.Cancellation); Log("P1: {0}", String.Join(", ", withPrefetching1)); - Assert.That(withPrefetching1.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); - Assert.That(withPrefetching1.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(2, 10))); + Assert.That(withPrefetching1.Select(x => x.Value), Is.EqualTo(Enumerable.Range(0, 10))); + Assert.That(withPrefetching1.Select(x => x.Called), Is.EqualTo(Enumerable.Range(2, 10))); // prefetching more than 1 item on a consumer that is not buffered should not change the picture (since we can only read one ahead anyway) //REVIEW: maybe we should change the implementation of the operator so that it still prefetch items in the background if the rest of the query is lagging a bit? @@ -995,8 +995,8 @@ public async Task Test_Can_Prefetch_On_Constant_Latency_Source() sw.Restart(); var withPrefetching2 = await source.Prefetch(2).Select(record).ToListAsync(this.Cancellation); Log("P2: {0}", String.Join(", ", withPrefetching2)); - Assert.That(withPrefetching2.Select(x => x.Item1), Is.EqualTo(Enumerable.Range(0, 10))); - Assert.That(withPrefetching2.Select(x => x.Item2), Is.EqualTo(Enumerable.Range(2, 10))); + Assert.That(withPrefetching2.Select(x => x.Value), Is.EqualTo(Enumerable.Range(0, 10))); + Assert.That(withPrefetching2.Select(x => x.Called), Is.EqualTo(Enumerable.Range(2, 10))); } [Test] diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index 7c487950a..bfcffead6 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -43,9 +43,9 @@ public class FdbQueryExpressionFacts : FdbTest { - private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.FromKey(STuple.Create("Foos", 1))); + private readonly FdbIndex FooBarIndex = new FdbIndex("Foos.ByBar", KeySubspace.FromKey(TuPack.EncodeKey("Foos", 1))); - private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.FromKey(STuple.Create("Foos", 2))); + private readonly FdbIndex FooBazIndex = new FdbIndex("Foos.ByBaz", KeySubspace.FromKey(TuPack.EncodeKey("Foos", 2))); [Test] public void Test_FdbQueryIndexLookupExpression() @@ -96,7 +96,7 @@ public void Test_FdbQueryIndexLookupExpression_From_Lambda() public void Test_FdbQueryRangeExpression() { var expr = FdbQueryExpressions.Range( - KeySelectorPair.Create(TuPack.ToRange(STuple.Create("Foo"))) + KeySelectorPair.Create(TuPack.ToKeyRange("Foo")) ); Log(expr); @@ -174,7 +174,7 @@ public void Test_FdbQueryUnionExpression() public void Test_FdbQueryTransformExpression() { var expr = FdbQueryExpressions.Transform( - FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), + FdbQueryExpressions.RangeStartsWith(TuPack.EncodeKey("Hello", "World")), (kvp) => kvp.Value.ToUnicode() ); Log(expr); @@ -193,7 +193,7 @@ public void Test_FdbQueryTransformExpression() public void Test_FdbQueryFilterExpression() { var expr = FdbQueryExpressions.Filter( - FdbQueryExpressions.RangeStartsWith(STuple.Create("Hello", "World")), + FdbQueryExpressions.RangeStartsWith(TuPack.EncodeKey("Hello", "World")), (kvp) => kvp.Value.ToInt32() % 2 == 0 ); Log(expr); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 5d180f6d1..9a015c47c 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -726,8 +726,8 @@ await db.WriteAsync((tr) => Trace.WriteLine(r); } Assert.That(results.Count, Is.EqualTo(2)); - Assert.That(results[0], Is.EqualTo(STuple.Create("userA", 10093))); - Assert.That(results[1], Is.EqualTo(STuple.Create("userB", 20003))); + Assert.That(results[0], Is.EqualTo(("userA", 10093))); + Assert.That(results[1], Is.EqualTo(("userB", 20003))); // Second Method: pre-parse the queries, and merge on the results directly Trace.WriteLine("Method 2:"); @@ -753,8 +753,8 @@ await db.WriteAsync((tr) => Trace.WriteLine(r); } Assert.That(results.Count, Is.EqualTo(2)); - Assert.That(results[0], Is.EqualTo(STuple.Create("userA", 10093))); - Assert.That(results[1], Is.EqualTo(STuple.Create("userB", 20003))); + Assert.That(results[0], Is.EqualTo(("userA", 10093))); + Assert.That(results[1], Is.EqualTo(("userB", 20003))); } diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index e61384837..8449856d9 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -67,9 +67,10 @@ public void Test_Subspace_With_Binary_Prefix() Assert.That(subspace.Keys.Encode("hello").ToString(), Is.EqualTo("*<00><7F><02>hello<00>")); Assert.That(subspace.Keys.Encode(Slice.FromStringAscii("world")).ToString(), Is.EqualTo("*<00><7F><01>world<00>")); Assert.That(subspace.Keys.Pack(STuple.Create("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); + Assert.That(subspace.Keys.Pack(("hello", 123)).ToString(), Is.EqualTo("*<00><7F><02>hello<00><15>{")); // if we encode a tuple from this subspace, it should keep the binary prefix when converted to a key - var k = subspace.Keys.Pack(STuple.Create("world", 123, false)); + var k = subspace.Keys.Pack(("world", 123, false)); Assert.That(k.ToString(), Is.EqualTo("*<00><7F><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -116,7 +117,7 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = KeySubspace.FromKey(STuple.Create("hello"), TypeSystem.Tuples); + var subspace = KeySubspace.FromKey(TuPack.EncodeKey("hello"), TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -131,7 +132,7 @@ public void Test_Subspace_With_Tuple_Prefix() Assert.That(subspace.Keys.Encode("world").ToString(), Is.EqualTo("<02>hello<00><02>world<00>")); // even though the subspace prefix is a tuple, appending to it will only return the new items - var k = subspace.Keys.Pack(STuple.Create("world", 123, false)); + var k = subspace.Keys.Pack(("world", 123, false)); Assert.That(k.ToString(), Is.EqualTo("<02>hello<00><02>world<00><15>{<14>")); // if we unpack the key with the binary prefix, we should get a valid tuple @@ -213,11 +214,11 @@ public void Test_DynamicKeySpace_API() // STuple Decode(Slice) Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00>")), Is.EqualTo("hello")); - Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{")), Is.EqualTo(STuple.Create("hello", 123))); - Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>")), Is.EqualTo(STuple.Create("hello", 123, "world"))); - Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456))); - Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!"))); - Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo(STuple.Create("hello", 123, "world", 456, "!", 789))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{")), Is.EqualTo(("hello", 123))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00>")), Is.EqualTo(("hello", 123, "world"))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01>")), Is.EqualTo(("hello", 123, "world", 456))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00>")), Is.EqualTo(("hello", 123, "world", 456, "!"))); + Assert.That(location.Keys.Decode(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo(("hello", 123, "world", 456, "!", 789))); // DecodeFirst/DecodeLast Assert.That(location.Keys.DecodeFirst(Slice.Unescape("PREFIX<02>hello<00><15>{<02>world<00><16><01><02>!<00><16><03><15>")), Is.EqualTo("hello")); From 07fa5869e8ea2872a228bb3bafd8b811bd32efae Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 14:13:56 +0200 Subject: [PATCH 117/153] fix or remove old xml comments --- FoundationDB.Client/Layers/Tuples/TuPack.cs | 50 ++++++--------------- 1 file changed, 13 insertions(+), 37 deletions(-) diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Layers/Tuples/TuPack.cs index 110138d35..bc8f6ebcc 100644 --- a/FoundationDB.Client/Layers/Tuples/TuPack.cs +++ b/FoundationDB.Client/Layers/Tuples/TuPack.cs @@ -38,6 +38,7 @@ namespace Doxense.Collections.Tuples using JetBrains.Annotations; /// Tuple Binary Encoding + [PublicAPI] public static class TuPack { @@ -520,7 +521,7 @@ public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] p /// Create a range that selects all tuples that are stored under the specified subspace: 'prefix\x00' <= k < 'prefix\xFF' /// Subspace binary prefix (that will be excluded from the range) /// Range including all possible tuples starting with the specified prefix. - /// FdbTuple.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) + /// TuPack.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) [Pure] public static KeyRange ToRange(Slice prefix) { @@ -535,7 +536,7 @@ public static KeyRange ToRange(Slice prefix) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// TuPack.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange([NotNull] TTuple tuple) where TTuple : ITuple @@ -551,7 +552,7 @@ public static KeyRange ToRange([NotNull] TTuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -566,7 +567,7 @@ public static KeyRange ToRange(STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(ValueTuple tuple) { @@ -581,7 +582,7 @@ public static KeyRange ToRange(ValueTuple tuple) } /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1) { @@ -594,7 +595,7 @@ public static KeyRange ToKeyRange(T1 item1) } /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1) { @@ -607,7 +608,7 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -623,7 +624,6 @@ public static KeyRange ToRange(STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange((T1, T2) tuple) { @@ -639,7 +639,7 @@ public static KeyRange ToRange((T1, T2) tuple) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToKeyRange("a", "b") includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2) { @@ -652,7 +652,7 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// ToPrefixedKeyRange(..., "a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2) { @@ -665,7 +665,6 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 ite } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -681,7 +680,6 @@ public static KeyRange ToRange(STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange((T1, T2, T3) tuple) { @@ -695,7 +693,6 @@ public static KeyRange ToRange((T1, T2, T3) tuple) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) { @@ -708,7 +705,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { @@ -721,7 +717,6 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -737,7 +732,6 @@ public static KeyRange ToRange(STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange((T1, T2, T3, T4) tuple) { @@ -751,7 +745,6 @@ public static KeyRange ToRange((T1, T2, T3, T4) tuple) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4) { @@ -764,7 +757,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { @@ -777,7 +769,6 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1 } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -793,7 +784,6 @@ public static KeyRange ToRange(STuple tu } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange((T1, T2, T3, T4, T5) tuple) { @@ -807,7 +797,6 @@ public static KeyRange ToRange((T1, T2, T3, T4, T5) tuple) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { @@ -820,7 +809,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 ite } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { @@ -833,7 +821,6 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 i } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(STuple tuple) { @@ -849,7 +836,6 @@ public static KeyRange ToRange(STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange((T1, T2, T3, T4, T5, T6) tuple) { @@ -863,7 +849,6 @@ public static KeyRange ToRange((T1, T2, T3, T4, T5, T6) } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { @@ -876,7 +861,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { @@ -889,7 +873,6 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { @@ -902,7 +885,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2 } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { @@ -915,7 +897,6 @@ public static KeyRange ToPrefixedKeyRange(Slice pref } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { @@ -928,7 +909,6 @@ public static KeyRange ToKeyRange(T1 item1, T2 i } /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { @@ -941,7 +921,7 @@ public static KeyRange ToPrefixedKeyRange(Slice } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(Slice.FromInt32(42), FdbTuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. + /// TuPack.ToRange(Slice.FromInt32(42), Stuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. [Pure] public static KeyRange ToRange(Slice prefix, [NotNull] TTuple tuple) @@ -958,7 +938,7 @@ public static KeyRange ToRange(Slice prefix, [NotNull] TTuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// TuPack.ToRange(STuple.Create("a")) includes all tuples ("a", ...), but not the tuple ("a") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { @@ -973,7 +953,7 @@ public static KeyRange ToRange(Slice prefix, STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. + /// TuPack.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { @@ -988,7 +968,6 @@ public static KeyRange ToRange(Slice prefix, STuple tuple) } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { @@ -1003,7 +982,6 @@ public static KeyRange ToRange(Slice prefix, STuple tupl } /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { @@ -1018,7 +996,6 @@ public static KeyRange ToRange(Slice prefix, STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { @@ -1033,7 +1010,6 @@ public static KeyRange ToRange(Slice prefix, STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' - /// FdbTuple.ToRange(FdbTuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] public static KeyRange ToRange(Slice prefix, STuple tuple) { From a94de4107b788088e226c8e6cc9744bc8f5faef9 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 14:18:31 +0200 Subject: [PATCH 118/153] Rename some KeySubspace.FromKey(..) into CreateDynamic(..) or CreateTyped(..) --- .../Layers/Directories/FdbDirectoryLayer.cs | 2 +- FoundationDB.Client/Subspaces/KeySubspace.cs | 20 +++++++++---------- FoundationDB.Tests/SubspaceFacts.cs | 10 +++++----- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 86f4378c0..73a7c089a 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -160,7 +160,7 @@ internal FdbDirectoryLayer(IDynamicKeySubspace nodeSubspace, IDynamicKeySubspace [NotNull] public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null, IKeyEncoding encoding = null) { - var subspace = KeySubspace.FromKey(prefix, encoding ?? TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(prefix, encoding ?? TypeSystem.Tuples); var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 0298c54f2..6a86d6f58 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -65,7 +65,7 @@ public static KeySubspace FromKey(Slice prefix) /// Initializes a new dynamic subspace with the given binary and key . /// A subspace that can handle keys of any types and size. [Pure, NotNull] - public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IDynamicKeyEncoder encoder) + public static DynamicKeySubspace CreateDynamic(Slice prefix, [NotNull] IDynamicKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); return new DynamicKeySubspace(prefix, encoder); @@ -74,7 +74,7 @@ public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IDynamicKeyEnco /// Initializes a new subspace with the given binary , that uses a dynamic key . /// A subspace that can handle keys of any types and size. [Pure, NotNull] - public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncoding encoding) + public static DynamicKeySubspace CreateDynamic(Slice prefix, [NotNull] IKeyEncoding encoding) { Contract.NotNull(encoding, nameof(encoding)); return new DynamicKeySubspace(prefix, encoding.GetDynamicEncoder()); @@ -82,14 +82,14 @@ public static DynamicKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncoding en /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle keys of type . - public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle keys of type . - public static TypedKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncoder encoder) + public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] IKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); return new TypedKeySubspace(prefix, encoder); @@ -97,14 +97,14 @@ public static TypedKeySubspace FromKey(Slice prefix, [NotNull] IKeyEncod /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, ). - public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, ). - public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); return new TypedKeySubspace(prefix, encoder); @@ -112,14 +112,14 @@ public static TypedKeySubspace FromKey(Slice prefix, [NotNull] I /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, , ). - public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, , ). - public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); return new TypedKeySubspace(prefix, encoder); @@ -127,14 +127,14 @@ public static TypedKeySubspace FromKey(Slice prefix, [No /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, , ). - public static TypedKeySubspace FromKey(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) + public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle composite keys of type (, , ). - public static TypedKeySubspace FromKey(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) + public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(encoder, nameof(encoder)); return new TypedKeySubspace(prefix, encoder); diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index 8449856d9..ec6983561 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -52,7 +52,7 @@ public void Test_Empty_Subspace_Is_Empty() [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = KeySubspace.FromKey(new byte[] { 42, 255, 0, 127 }.AsSlice(), TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(new byte[] { 42, 255, 0, 127 }.AsSlice(), TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -117,7 +117,7 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = KeySubspace.FromKey(TuPack.EncodeKey("hello"), TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(TuPack.EncodeKey("hello"), TypeSystem.Tuples); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); @@ -149,7 +149,7 @@ public void Test_Subspace_With_Tuple_Prefix() public void Test_Subspace_Partitioning_With_Binary_Suffix() { // start from a parent subspace - var parent = KeySubspace.FromKey(Slice.Empty, TypeSystem.Tuples); + var parent = KeySubspace.CreateDynamic(Slice.Empty, TypeSystem.Tuples); Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple @@ -176,7 +176,7 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() [Test] public void Test_DynamicKeySpace_API() { - var location = KeySubspace.FromKey(Slice.FromString("PREFIX"), TypeSystem.Tuples); + var location = KeySubspace.CreateDynamic(Slice.FromString("PREFIX"), TypeSystem.Tuples); Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -229,7 +229,7 @@ public void Test_DynamicKeySpace_API() [Test] public void Test_TypedKeySpace_T1() { - var location = KeySubspace.FromKey(Slice.FromString("PREFIX")); + var location = KeySubspace.CreateTyped(Slice.FromString("PREFIX")); Assert.That(location.KeyEncoder, Is.Not.Null, "Should have a Key Encoder"); Assert.That(location.KeyEncoder.Encoding, Is.SameAs(TypeSystem.Tuples), "Encoder should use Tuple type system"); From 5051b06ed2374be197de292e7e125d5270b96fa2 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 20:31:01 +0200 Subject: [PATCH 119/153] Fix unpacking of VersionStamps in tuple (when using the boxed API) --- .../Layers/Tuples/Encoding/TuplePackers.cs | 2 ++ .../Layers/Tuples/Encoding/TupleParser.cs | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs index e3f5c5e8a..68f28989c 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs @@ -846,6 +846,8 @@ public static object DeserializeBoxed(Slice slice) case TupleTypes.Decimal: return TupleParser.ParseDecimal(slice); case TupleTypes.Uuid128: return TupleParser.ParseGuid(slice); case TupleTypes.Uuid64: return TupleParser.ParseUuid64(slice); + case TupleTypes.VersionStamp80: return TupleParser.ParseVersionStamp(slice); + case TupleTypes.VersionStamp96: return TupleParser.ParseVersionStamp(slice); } } diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs index 66957f676..a2e24b5f6 100644 --- a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs +++ b/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs @@ -877,6 +877,7 @@ internal static Slice UnescapeByteStringSlow([NotNull] byte[] buffer, int offset } /// Parse a tuple segment containing a byte array + [Pure] public static Slice ParseBytes(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Bytes && slice[-1] == 0); @@ -888,6 +889,7 @@ public static Slice ParseBytes(Slice slice) } /// Parse a tuple segment containing an ASCII string stored as a byte array + [Pure] public static string ParseAscii(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Bytes && slice[-1] == 0); @@ -900,6 +902,7 @@ public static string ParseAscii(Slice slice) } /// Parse a tuple segment containing a unicode string + [Pure] public static string ParseUnicode(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Utf8 && slice[-1] == 0); @@ -911,6 +914,7 @@ public static string ParseUnicode(Slice slice) } /// Parse a tuple segment containing an embedded tuple + [Pure] public static ITuple ParseTuple(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.TupleStart && slice[-1] == 0); @@ -920,6 +924,7 @@ public static ITuple ParseTuple(Slice slice) } /// Parse a tuple segment containing a single precision number (float32) + [Pure] public static float ParseSingle(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Single); @@ -950,6 +955,7 @@ public static float ParseSingle(Slice slice) } /// Parse a tuple segment containing a double precision number (float64) + [Pure] public static double ParseDouble(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Double); @@ -981,6 +987,7 @@ public static double ParseDouble(Slice slice) } /// Parse a tuple segment containing a quadruple precision number (float128) + [Pure] public static decimal ParseDecimal(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Decimal); @@ -994,6 +1001,7 @@ public static decimal ParseDecimal(Slice slice) } /// Parse a tuple segment containing a 128-bit GUID + [Pure] public static Guid ParseGuid(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid128); @@ -1008,6 +1016,7 @@ public static Guid ParseGuid(Slice slice) } /// Parse a tuple segment containing a 128-bit UUID + [Pure] public static Uuid128 ParseUuid128(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid128); @@ -1021,6 +1030,7 @@ public static Uuid128 ParseUuid128(Slice slice) } /// Parse a tuple segment containing a 64-bit UUID + [Pure] public static Uuid64 ParseUuid64(Slice slice) { Contract.Requires(slice.HasValue && slice[0] == TupleTypes.Uuid64); @@ -1033,6 +1043,20 @@ public static Uuid64 ParseUuid64(Slice slice) return Uuid64.Read(slice.Substring(1, 8)); } + /// Parse a tuple segment containing an 80-bit or 96-bit VersionStamp + [Pure] + public static VersionStamp ParseVersionStamp(Slice slice) + { + Contract.Requires(slice.HasValue && (slice[0] == TupleTypes.VersionStamp80 || slice[0] == TupleTypes.VersionStamp96)); + + if (slice.Count != 11 && slice.Count != 13) + { + throw new FormatException("Slice has invalid size for a VersionStamp"); + } + + return VersionStamp.Parse(slice.Substring(1)); + } + #endregion #region Parsing... From 6ae022ac071ee6bc7dcd2ea7b1299fe337b533ce Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Thu, 26 Apr 2018 20:32:26 +0200 Subject: [PATCH 120/153] Add the logic on transactions to generate custom placeholder stamps (using a random token) - Each transaction generate a random token (and on each retry). - tr.CreateVersionStamp() can be used to get a stamp specific to this transaction --- FoundationDB.Client/FdbTransaction.cs | 70 +++++++++ .../Filters/FdbTransactionFilter.cs | 12 ++ FoundationDB.Client/IFdbTransaction.cs | 18 +++ FoundationDB.Client/VersionStamp.cs | 13 ++ FoundationDB.Tests/TransactionFacts.cs | 134 ++++++++++++++++-- 5 files changed, 232 insertions(+), 15 deletions(-) diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index ac9e45120..2d4ea3970 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -90,6 +90,9 @@ public sealed partial class FdbTransaction : IFdbTransaction /// CancellationToken that should be used for all async operations executing inside this transaction private CancellationToken m_cancellation; + /// Random token (but constant per transaction retry) used to generate incomplete VersionStamps + private ulong m_versionStampToken; + #endregion #region Constructors... @@ -297,6 +300,68 @@ public Task GetVersionStampAsync() return m_handler.GetVersionStampAsync(m_cancellation); } + private ulong GenerateNewVersionStampToken() + { + // We need to generate a 80-bits stamp, and also need to mark it as 'incomplete' by forcing the highest bit to 1. + // Since this is supposed to be a version number with a ~1M tickrate per seconds, we will play it safe, and force the 8 highest bits to 1, + // meaning that we only reduce the database potential lifetime but 1/256th, before getting into trouble. + // + // By doing some empirical testing, it also seems that the last 16 bits are a transction batch order which is usually a low number. + // Again, we will force the 4 highest bit to 1 to reduce the change of collision with a complete version stamp. + // + // So the final token will look like: 'FF xx xx xx xx xx xx xx Fy yy', were 'x' is the random token, and 'y' will lowest 12 bits of the transaction retry count + + var rnd = new Random(); //TODO: singleton? (need locking!!) + ulong x; + unsafe + { + double r = rnd.NextDouble(); + x = *(ulong*) &r; + } + x |= 0xFF00000000000000UL; + + lock (this) + { + ulong token = m_versionStampToken; + if (token == 0) + { + token = x; + m_versionStampToken = x; + } + return token; + } + } + + /// Return a place-holder 80-bit VersionStamp, whose value is not yet known, but will be filled by the database at commit time. + /// This value can used to generate temporary keys or value, for use with the or mutations + /// + /// The generate placeholder will use a random value that is unique per transaction (and changes at reach retry). + /// If the key contains the exact 80-bit byte signature of this token, the corresponding location will be tagged and replaced with the actual VersionStamp at commit time. + /// If another part of the key contains (by random chance) the same exact byte sequence, then an error will be triggered, and hopefully the transaction will retry with another byte sequence. + /// + [Pure] + public VersionStamp CreateVersionStamp() + { + var token = m_versionStampToken; + if (token == 0) token = GenerateNewVersionStampToken(); + return VersionStamp.Custom(token, (ushort) (m_context.Retries | 0xF000), incomplete: true); + } + + /// Return a place-holder 96-bit VersionStamp with an attached user version, whose value is not yet known, but will be filled by the database at commit time. + /// This value can used to generate temporary keys or value, for use with the or mutations + /// + /// The generate placeholder will use a random value that is unique per transaction (and changes at reach retry). + /// If the key contains the exact 80-bit byte signature of this token, the corresponding location will be tagged and replaced with the actual VersionStamp at commit time. + /// If another part of the key contains (by random chance) the same exact byte sequence, then an error will be triggered, and hopefully the transaction will retry with another byte sequence. + /// + public VersionStamp CreateVersionStamp(int userVersion) + { + var token = m_versionStampToken; + if (token == 0) token = GenerateNewVersionStampToken(); + + return VersionStamp.Custom(token, (ushort) (m_context.Retries | 0xF000), userVersion, incomplete: true); + } + #endregion #region Get... @@ -785,6 +850,11 @@ private void RestoreDefaultSettings() { this.Timeout = m_database.DefaultTimeout; } + + // if we have used a random token for versionstamps, we need to clear it (and generate a new one) + // => this ensure that if the error was due to a collision between the token and another part of the key, + // a transaction retry will hopefully use a different token that does not collide. + m_versionStampToken = 0; } /// Reset the transaction to its initial state. diff --git a/FoundationDB.Client/Filters/FdbTransactionFilter.cs b/FoundationDB.Client/Filters/FdbTransactionFilter.cs index 78d414e68..bfcf54712 100644 --- a/FoundationDB.Client/Filters/FdbTransactionFilter.cs +++ b/FoundationDB.Client/Filters/FdbTransactionFilter.cs @@ -257,6 +257,18 @@ public virtual Task GetVersionStampAsync() return m_transaction.GetVersionStampAsync(); } + public virtual VersionStamp CreateVersionStamp() + { + ThrowIfDisposed(); + return m_transaction.CreateVersionStamp(); + } + + public virtual VersionStamp CreateVersionStamp(int userVersion) + { + ThrowIfDisposed(); + return m_transaction.CreateVersionStamp(userVersion); + } + public virtual void SetReadVersion(long version) { ThrowIfDisposed(); diff --git a/FoundationDB.Client/IFdbTransaction.cs b/FoundationDB.Client/IFdbTransaction.cs index 39015be4c..98f684dd9 100644 --- a/FoundationDB.Client/IFdbTransaction.cs +++ b/FoundationDB.Client/IFdbTransaction.cs @@ -120,6 +120,24 @@ public interface IFdbTransaction : IFdbReadOnlyTransaction /// Task GetVersionStampAsync(); + /// Return a place-holder 80-bit VersionStamp, whose value is not yet known, but will be filled by the database at commit time. + /// This value can used to generate temporary keys or value, for use with the or mutations + /// + /// The generate placeholder will use a random value that is unique per transaction (and changes at reach retry). + /// If the key contains the exact 80-bit byte signature of this token, the corresponding location will be tagged and replaced with the actual VersionStamp at commit time. + /// If another part of the key contains (by random chance) the same exact byte sequence, then an error will be triggered, and hopefully the transaction will retry with another byte sequence. + /// + VersionStamp CreateVersionStamp(); + + /// Return a place-holder 96-bit VersionStamp with an attached user version, whose value is not yet known, but will be filled by the database at commit time. + /// This value can used to generate temporary keys or value, for use with the or mutations + /// + /// The generate placeholder will use a random value that is unique per transaction (and changes at reach retry). + /// If the key contains the exact 80-bit byte signature of this token, the corresponding location will be tagged and replaced with the actual VersionStamp at commit time. + /// If another part of the key contains (by random chance) the same exact byte sequence, then an error will be triggered, and hopefully the transaction will retry with another byte sequence. + /// + VersionStamp CreateVersionStamp(int userVersion); + /// /// Watch a key for any change in the database. /// diff --git a/FoundationDB.Client/VersionStamp.cs b/FoundationDB.Client/VersionStamp.cs index e9faf5bb1..e16a4a554 100644 --- a/FoundationDB.Client/VersionStamp.cs +++ b/FoundationDB.Client/VersionStamp.cs @@ -115,6 +115,19 @@ public static VersionStamp Incomplete(ushort userVersion) return new VersionStamp(PLACEHOLDER_VERSION, PLACEHOLDER_ORDER, userVersion, FLAGS_IS_INCOMPLETE | FLAGS_HAS_VERSION); } + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static VersionStamp Custom(ulong version, ushort order, bool incomplete) + { + return new VersionStamp(version, order, NO_USER_VERSION, incomplete ? FLAGS_IS_INCOMPLETE : FLAGS_NONE); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + internal static VersionStamp Custom(ulong version, ushort order, int userVersion, bool incomplete) + { + Contract.Between(userVersion, 0, 0xFFFF, nameof(userVersion), "Local version must fit in 16-bits."); + return new VersionStamp(version, order, (ushort) userVersion, incomplete ? (ushort) (FLAGS_IS_INCOMPLETE | FLAGS_HAS_VERSION) : FLAGS_HAS_VERSION); + } + /// Creates a 80-bit , obtained from the database. /// Complete stamp, without user version. [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index 2b8536ec2..f536d429b 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -37,6 +37,7 @@ namespace FoundationDB.Client.Tests using System.Threading; using System.Threading.Tasks; using Doxense.Collections.Tuples; + using Doxense.Memory; [TestFixture] public class TransactionFacts : FdbTest @@ -1993,37 +1994,140 @@ public async Task Test_Simple_Read_Transaction() [Test] public async Task Test_VersionStamp_Operations() { - Fdb.Start(510); + // Veryify that we can set versionstamped keys inside a transaction + using (var db = await OpenTestDatabaseAsync()) { - Log("API Version: " + Fdb.ApiVersion); - var location = db.Partition.ByKey("versionstamps"); await db.ClearRangeAsync(location, this.Cancellation); + VersionStamp vsActual; // will contain the actual version stamp used by the database + + Log("Inserting keys with version stamps:"); using (var tr = db.BeginTransaction(this.Cancellation)) { - Slice HACKHACK_Packify(VersionStamp stamp) + //TODO: HACKACK: until we add support to he transaction itself, we have to 'patch' the versionstamps by hand! + Slice HACKHACK_Stampify(Slice key) { - var x = location.Keys.Encode(stamp); - x = x.Concat(Slice.FromFixed16((short) (location.GetPrefix().Count + 1))); - Log(x.ToHexaString(' ') + " | " + location.Keys.Dump(x)); - return x; + // find the stamp byte sequence in the key + var x = tr.CreateVersionStamp().ToSlice(); + int p = key.IndexOf(x); + Assert.That(p, Is.GreaterThan(0), "Stamp pattern was not found in the key!"); + + // append the offset at the end + var writer = new SliceWriter(key.Count + 2); + writer.WriteBytes(key); + writer.WriteFixed16((ushort) p); //note: the offset is Little Endian! + var y = writer.ToSlice(); + + //Log(y.ToHexaString(' ') + " | " + location.Keys.Dump(y)); + return y; } - tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete()), Slice.FromString("Hello, World!")); - tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(0)), Slice.FromString("Zero")); - tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(1)), Slice.FromString("One")); - tr.SetVersionStampedKey(HACKHACK_Packify(VersionStamp.Incomplete(2)), Slice.FromString("Two")); - + var vs = tr.CreateVersionStamp(); + Log($"> placeholder stamp: {vs} with token '{vs.ToSlice():X}'"); + Assert.That(vs.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(vs.HasUserVersion, Is.False); + Assert.That(vs.UserVersion, Is.Zero); + Assert.That(vs.TransactionVersion >> 56, Is.EqualTo(0xFF), "Highest 8 bit of Transaction Version should be set to 1"); + Assert.That(vs.TransactionOrder >> 12, Is.EqualTo(0xF), "Hight 4 bits of Transaction Order should be set to 1"); + + var vs0 = tr.CreateVersionStamp(0); + Assert.That(vs0.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(vs0.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs0.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs0.HasUserVersion, Is.True); + Assert.That(vs0.UserVersion, Is.EqualTo(0)); + + var vs1 = tr.CreateVersionStamp(1); + Assert.That(vs1.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(vs1.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs1.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs1.HasUserVersion, Is.True); + Assert.That(vs1.UserVersion, Is.EqualTo(1)); + + var vs42 = tr.CreateVersionStamp(42); + Assert.That(vs42.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(vs42.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs42.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(vs42.HasUserVersion, Is.True); + Assert.That(vs42.UserVersion, Is.EqualTo(42)); + + // a single key using the 80-bit stamp + tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("foo", vs, 123)), Slice.FromString("Hello, World!")); + + // simulate a batch of 3 keys, using 96-bits stamps + tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs0)), Slice.FromString("Zero")); + tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs1)), Slice.FromString("One")); + tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs42)), Slice.FromString("FortyTwo")); + + // need to be request BEFORE the commit var vsTask = tr.GetVersionStampAsync(); await tr.CommitAsync(); Log(tr.GetCommittedVersion()); - var vs = await vsTask; - Log(vs); + // need to be resolved AFTER the commit + vsActual = await vsTask; + Log($"> actual stamp: {vsActual} with token '{vsActual.ToSlice():X}'"); + } + + Log("Checking database content:"); + using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) + { + { + var foo = await tr.GetRange(location.Keys.ToKeyRange("foo")).SingleAsync(); + Log("> Found 1 result under (foo,)"); + Log($"- {location.ExtractKey(foo.Key):K} = {foo.Value:V}"); + Assert.That(foo.Value.ToString(), Is.EqualTo("Hello, World!")); + + var t = location.Keys.Unpack(foo.Key); + Assert.That(t.Get(0), Is.EqualTo("foo")); + Assert.That(t.Get(2), Is.EqualTo(123)); + + var vs = t.Get(1); + Assert.That(vs.IsIncomplete, Is.False); + Assert.That(vs.HasUserVersion, Is.False); + Assert.That(vs.UserVersion, Is.Zero); + Assert.That(vs.TransactionVersion, Is.EqualTo(vsActual.TransactionVersion)); + Assert.That(vs.TransactionOrder, Is.EqualTo(vsActual.TransactionOrder)); + } + + { + var items = await tr.GetRange(location.Keys.ToKeyRange("bar")).ToListAsync(); + Log($"> Found {items.Count} results under (bar,)"); + foreach (var item in items) + { + Log($"- {location.ExtractKey(item.Key):K} = {item.Value:V}"); + } + + Assert.That(items.Count, Is.EqualTo(3), "Should have found 3 keys under 'foo'"); + + Assert.That(items[0].Value.ToString(), Is.EqualTo("Zero")); + var vs0 = location.Keys.DecodeLast(items[0].Key); + Assert.That(vs0.IsIncomplete, Is.False); + Assert.That(vs0.HasUserVersion, Is.True); + Assert.That(vs0.UserVersion, Is.EqualTo(0)); + Assert.That(vs0.TransactionVersion, Is.EqualTo(vsActual.TransactionVersion)); + Assert.That(vs0.TransactionOrder, Is.EqualTo(vsActual.TransactionOrder)); + + Assert.That(items[1].Value.ToString(), Is.EqualTo("One")); + var vs1 = location.Keys.DecodeLast(items[1].Key); + Assert.That(vs1.IsIncomplete, Is.False); + Assert.That(vs1.HasUserVersion, Is.True); + Assert.That(vs1.UserVersion, Is.EqualTo(1)); + Assert.That(vs1.TransactionVersion, Is.EqualTo(vsActual.TransactionVersion)); + Assert.That(vs1.TransactionOrder, Is.EqualTo(vsActual.TransactionOrder)); + + Assert.That(items[2].Value.ToString(), Is.EqualTo("FortyTwo")); + var vs42 = location.Keys.DecodeLast(items[2].Key); + Assert.That(vs42.IsIncomplete, Is.False); + Assert.That(vs42.HasUserVersion, Is.True); + Assert.That(vs42.UserVersion, Is.EqualTo(42)); + Assert.That(vs42.TransactionVersion, Is.EqualTo(vsActual.TransactionVersion)); + Assert.That(vs42.TransactionOrder, Is.EqualTo(vsActual.TransactionOrder)); + } } await DumpSubspace(db, location); From 3e04cbd139875076799007d075a7b0b2014866a9 Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Fri, 27 Apr 2018 12:16:32 +0500 Subject: [PATCH 121/153] git: ignore JetBrains Ryder and OSX-specific files --- .gitignore | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.gitignore b/.gitignore index 9623924d5..aedd80221 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,8 @@ fdb_c.dll /build/tools/ /build/output/ +# JetBrains +.idea + +# OSX +.DS_Store From cecf0e4930d2c8b90f71b577c2562006388d6ce0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 14:31:44 +0200 Subject: [PATCH 122/153] SetVersionStampedKey will automatically detect the location of the versionstamp present in the key, and add the 16-bit offset suffix. --- .../FdbTransactionExtensions.cs | 50 +++++- FoundationDB.Client/VersionStamp.cs | 3 + FoundationDB.Tests/TransactionFacts.cs | 142 ++++++++++++------ 3 files changed, 143 insertions(+), 52 deletions(-) diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index d4f2e8992..b1e6cd2e8 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Memory; using Doxense.Serialization.Encoders; using JetBrains.Annotations; @@ -425,19 +426,64 @@ public static void AtomicMin([NotNull] this IFdbTransaction trans, Slice key, Sl trans.Atomic(key, value, FdbMutationType.Min); } + private static int GetVersionStampOffset(Slice buffer, Slice token, string argName) + { + // the buffer MUST contain one incomplete stamp, either the random token of the current transsaction or the default token (all-FF) + + int p = token.HasValue ? buffer.IndexOf(token) : -1; + if (p >= 0) + { // found a candidate spot, we have to make sure that it is only present once in the key! + + if (buffer.IndexOf(token, p + token.Count) >= 0) + { + if (argName == "key") + throw new ArgumentException("The key should only contain one occurrence of a VersionStamp.", argName); + else + throw new ArgumentException("The value should only contain one occurrence of a VersionStamp.", argName); + } + } + else + { // not found, maybe it is using the default incomplete stamp (all FF) ? + p = buffer.IndexOf(VersionStamp.IncompleteToken); + if (p < 0) + { + if (argName == "key") + throw new ArgumentException("The key should contain at least one VersionStamp.", argName); + else + throw new ArgumentException("The value should contain at least one VersionStamp.", argName); + } + } + Contract.Assert(p >= 0 && p + token.Count <= buffer.Count); + + return p; + } + //TODO: XML Comments! public static void SetVersionStampedKey([NotNull] this IFdbTransaction trans, Slice key, Slice value) { Contract.NotNull(trans, nameof(trans)); - trans.Atomic(key, value, FdbMutationType.VersionStampedKey); + //TODO: PERF: optimize this to not have to allocate! + var token = trans.CreateVersionStamp().ToSlice(); + var offset = GetVersionStampOffset(key, token, nameof(key)); + + var writer = new SliceWriter(key.Count + 2); + writer.WriteBytes(key); + writer.WriteFixed16(checked((ushort) offset)); //note: currently stored as 16-bits in Little Endian + + trans.Atomic(writer.ToSlice(), value, FdbMutationType.VersionStampedKey); } - //TODO: XML Comments! + /// Set the of the in the database, with the first 10 bytes overwritten with the transaction's . + /// Transaction to use for the operation + /// Name of the key whose value is to be mutated. + /// Value whose first 10 bytes will be overwritten by the database with the resolved VersionStamp at commit time. The rest of the value will be untouched. public static void SetVersionStampedValue([NotNull] this IFdbTransaction trans, Slice key, Slice value) { Contract.NotNull(trans, nameof(trans)); + if (value.Count < 10) throw new ArgumentException("The value must be at least 10 bytes long.", nameof(value)); + trans.Atomic(key, value, FdbMutationType.VersionStampedValue); } diff --git a/FoundationDB.Client/VersionStamp.cs b/FoundationDB.Client/VersionStamp.cs index e16a4a554..d1f3db3eb 100644 --- a/FoundationDB.Client/VersionStamp.cs +++ b/FoundationDB.Client/VersionStamp.cs @@ -57,6 +57,9 @@ namespace FoundationDB.Client private const ushort FLAGS_HAS_VERSION = 0x1; // unset: 80-bits, set: 96-bits private const ushort FLAGS_IS_INCOMPLETE = 0x2; // unset: complete, set: incomplete + /// Serialized bytes of the default incomplete stamp (composed of only 0xFF) + internal static readonly Slice IncompleteToken = Slice.Repeat(0xFF, 10); + /// Commit version of the transaction /// This value is determined by the database at commit time. diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index f536d429b..a4c1f5f15 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -36,8 +36,6 @@ namespace FoundationDB.Client.Tests using System.Text; using System.Threading; using System.Threading.Tasks; - using Doxense.Collections.Tuples; - using Doxense.Memory; [TestFixture] public class TransactionFacts : FdbTest @@ -1991,6 +1989,77 @@ public async Task Test_Simple_Read_Transaction() } } + [Test] + public async Task Test_VersionStamps_Share_The_Same_Token_Per_Transaction_Attempt() + { + // Veryify that we can set versionstamped keys inside a transaction + + using (var db = await OpenTestDatabaseAsync()) + { + using (var tr = db.BeginTransaction(this.Cancellation)) + { + // should return a 80-bit incomplete stamp, using a random token + var x = tr.CreateVersionStamp(); + Log($"> x : {x} with token '{x.ToSlice():X}'"); + Assert.That(x.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(x.HasUserVersion, Is.False); + Assert.That(x.UserVersion, Is.Zero); + Assert.That(x.TransactionVersion >> 56, Is.EqualTo(0xFF), "Highest 8 bit of Transaction Version should be set to 1"); + Assert.That(x.TransactionOrder >> 12, Is.EqualTo(0xF), "Hight 4 bits of Transaction Order should be set to 1"); + + // should return a 96-bit incomplete stamp, using a the same random token and user version 0 + var x0 = tr.CreateVersionStamp(0); + Log($"> x0 : {x0.ToSlice():X} => {x0}"); + Assert.That(x0.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(x0.TransactionVersion, Is.EqualTo(x.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(x0.TransactionOrder, Is.EqualTo(x.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(x0.HasUserVersion, Is.True); + Assert.That(x0.UserVersion, Is.EqualTo(0)); + + // should return a 96-bit incomplete stamp, using a the same random token and user version 1 + var x1 = tr.CreateVersionStamp(1); + Log($"> x1 : {x1.ToSlice():X} => {x1}"); + Assert.That(x1.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(x1.TransactionVersion, Is.EqualTo(x.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(x1.TransactionOrder, Is.EqualTo(x.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(x1.HasUserVersion, Is.True); + Assert.That(x1.UserVersion, Is.EqualTo(1)); + + // should return a 96-bit incomplete stamp, using a the same random token and user version 42 + var x42 = tr.CreateVersionStamp(42); + Log($"> x42: {x42.ToSlice():X} => {x42}"); + Assert.That(x42.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(x42.TransactionVersion, Is.EqualTo(x.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(x42.TransactionOrder, Is.EqualTo(x.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(x42.HasUserVersion, Is.True); + Assert.That(x42.UserVersion, Is.EqualTo(42)); + + // Reset the transaction + // => stamps should use a new value + Log("Reset!"); + tr.Reset(); + + var y = tr.CreateVersionStamp(); + Log($"> y : {y.ToSlice():X} => {y}'"); + Assert.That(y, Is.Not.EqualTo(x), "VersionStamps should change when a transaction is reset"); + + Assert.That(y.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(y.HasUserVersion, Is.False); + Assert.That(y.UserVersion, Is.Zero); + Assert.That(y.TransactionVersion >> 56, Is.EqualTo(0xFF), "Highest 8 bit of Transaction Version should be set to 1"); + Assert.That(y.TransactionOrder >> 12, Is.EqualTo(0xF), "Hight 4 bits of Transaction Order should be set to 1"); + + var y42 = tr.CreateVersionStamp(42); + Log($"> y42: {y42.ToSlice():X} => {y42}"); + Assert.That(y42.IsIncomplete, Is.True, "Placeholder token should be incomplete"); + Assert.That(y42.TransactionVersion, Is.EqualTo(y.TransactionVersion), "All generated stamps by one transaction should share the random token value "); + Assert.That(y42.TransactionOrder, Is.EqualTo(y.TransactionOrder), "All generated stamps by one transaction should share the random token value "); + Assert.That(y42.HasUserVersion, Is.True); + Assert.That(y42.UserVersion, Is.EqualTo(42)); + } + } + } + [Test] public async Task Test_VersionStamp_Operations() { @@ -2007,60 +2076,23 @@ public async Task Test_VersionStamp_Operations() Log("Inserting keys with version stamps:"); using (var tr = db.BeginTransaction(this.Cancellation)) { - //TODO: HACKACK: until we add support to he transaction itself, we have to 'patch' the versionstamps by hand! - Slice HACKHACK_Stampify(Slice key) - { - // find the stamp byte sequence in the key - var x = tr.CreateVersionStamp().ToSlice(); - int p = key.IndexOf(x); - Assert.That(p, Is.GreaterThan(0), "Stamp pattern was not found in the key!"); - - // append the offset at the end - var writer = new SliceWriter(key.Count + 2); - writer.WriteBytes(key); - writer.WriteFixed16((ushort) p); //note: the offset is Little Endian! - var y = writer.ToSlice(); - - //Log(y.ToHexaString(' ') + " | " + location.Keys.Dump(y)); - return y; - } + // should return a 80-bit incomplete stamp, using a random token var vs = tr.CreateVersionStamp(); Log($"> placeholder stamp: {vs} with token '{vs.ToSlice():X}'"); - Assert.That(vs.IsIncomplete, Is.True, "Placeholder token should be incomplete"); - Assert.That(vs.HasUserVersion, Is.False); - Assert.That(vs.UserVersion, Is.Zero); - Assert.That(vs.TransactionVersion >> 56, Is.EqualTo(0xFF), "Highest 8 bit of Transaction Version should be set to 1"); - Assert.That(vs.TransactionOrder >> 12, Is.EqualTo(0xF), "Hight 4 bits of Transaction Order should be set to 1"); - - var vs0 = tr.CreateVersionStamp(0); - Assert.That(vs0.IsIncomplete, Is.True, "Placeholder token should be incomplete"); - Assert.That(vs0.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs0.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs0.HasUserVersion, Is.True); - Assert.That(vs0.UserVersion, Is.EqualTo(0)); - - var vs1 = tr.CreateVersionStamp(1); - Assert.That(vs1.IsIncomplete, Is.True, "Placeholder token should be incomplete"); - Assert.That(vs1.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs1.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs1.HasUserVersion, Is.True); - Assert.That(vs1.UserVersion, Is.EqualTo(1)); - - var vs42 = tr.CreateVersionStamp(42); - Assert.That(vs42.IsIncomplete, Is.True, "Placeholder token should be incomplete"); - Assert.That(vs42.TransactionVersion, Is.EqualTo(vs.TransactionVersion), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs42.TransactionOrder, Is.EqualTo(vs.TransactionOrder), "All generated stamps by one transaction should share the random token value "); - Assert.That(vs42.HasUserVersion, Is.True); - Assert.That(vs42.UserVersion, Is.EqualTo(42)); // a single key using the 80-bit stamp - tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("foo", vs, 123)), Slice.FromString("Hello, World!")); + tr.SetVersionStampedKey(location.Keys.Encode("foo", vs, 123), Slice.FromString("Hello, World!")); // simulate a batch of 3 keys, using 96-bits stamps - tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs0)), Slice.FromString("Zero")); - tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs1)), Slice.FromString("One")); - tr.SetVersionStampedKey(HACKHACK_Stampify(location.Keys.Encode("bar", vs42)), Slice.FromString("FortyTwo")); + tr.SetVersionStampedKey(location.Keys.Encode("bar", tr.CreateVersionStamp(0)), Slice.FromString("Zero")); + tr.SetVersionStampedKey(location.Keys.Encode("bar", tr.CreateVersionStamp(1)), Slice.FromString("One")); + tr.SetVersionStampedKey(location.Keys.Encode("bar", tr.CreateVersionStamp(42)), Slice.FromString("FortyTwo")); + + // value that contain the stamp + var val = Slice.FromString("$$$$$$$$$$Hello World!"); // '$' will be replaced by the stamp + Log($"> {val:X}"); + tr.SetVersionStampedValue(location.Keys.Encode("baz"), val); // need to be request BEFORE the commit var vsTask = tr.GetVersionStampAsync(); @@ -2073,6 +2105,8 @@ Slice HACKHACK_Stampify(Slice key) Log($"> actual stamp: {vsActual} with token '{vsActual.ToSlice():X}'"); } + await DumpSubspace(db, location); + Log("Checking database content:"); using (var tr = db.BeginReadOnlyTransaction(this.Cancellation)) { @@ -2128,9 +2162,17 @@ Slice HACKHACK_Stampify(Slice key) Assert.That(vs42.TransactionVersion, Is.EqualTo(vsActual.TransactionVersion)); Assert.That(vs42.TransactionOrder, Is.EqualTo(vsActual.TransactionOrder)); } + + { + var baz = await tr.GetAsync(location.Keys.Encode("baz")); + Log($"> {baz:X}"); + // ensure that the first 10 bytes have been overwritten with the stamp + Assert.That(baz.Count, Is.GreaterThan(0), "Key should be present in the database"); + Assert.That(baz.StartsWith(vsActual.ToSlice()), Is.True, "The first 10 bytes should match the resolved stamp"); + Assert.That(baz.Substring(10), Is.EqualTo(Slice.FromString("Hello World!")), "The rest of the slice should be untouched"); + } } - await DumpSubspace(db, location); } } From da593a6c07852ab009b2793b0ba51bdd08dd2e33 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 14:40:53 +0200 Subject: [PATCH 123/153] Add overload SetVersionStampedKey(...) where the caller can manually specifiy the offset of the versionstamp --- .../FdbTransactionExtensions.cs | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index b1e6cd2e8..e59d39fc2 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -458,7 +458,10 @@ private static int GetVersionStampOffset(Slice buffer, Slice token, string argNa return p; } - //TODO: XML Comments! + /// Set the of the in the database, with the replaced by the resolved version at commit time. + /// Transaction to use for the operation + /// Name of the key whose value is to be mutated. This key must contain a single , whose position will be automatically detected. + /// New value for this key. public static void SetVersionStampedKey([NotNull] this IFdbTransaction trans, Slice key, Slice value) { Contract.NotNull(trans, nameof(trans)); @@ -474,6 +477,25 @@ public static void SetVersionStampedKey([NotNull] this IFdbTransaction trans, Sl trans.Atomic(writer.ToSlice(), value, FdbMutationType.VersionStampedKey); } + /// Set the of the in the database, with the replaced by the resolved version at commit time. + /// Transaction to use for the operation + /// Name of the key whose value is to be mutated. This key must contain a single , whose start is defined by . + /// Offset within of the start of the 80-bit VersionStamp. + /// New value for this key. + public static void SetVersionStampedKey([NotNull] this IFdbTransaction trans, Slice key, int stampOffset, Slice value) + { + Contract.NotNull(trans, nameof(trans)); + + if (stampOffset > key.Count - 10) throw new ArgumentException("The VersionStamp overflows past the end of the key.", nameof(stampOffset)); + if (stampOffset > 0xFFFF) throw new ArgumentException("The offset is too large to fit within 16-bits."); + + var writer = new SliceWriter(key.Count + 2); + writer.WriteBytes(key); + writer.WriteFixed16(checked((ushort) stampOffset)); //note: currently stored as 16-bits in Little Endian + + trans.Atomic(writer.ToSlice(), value, FdbMutationType.VersionStampedKey); + } + /// Set the of the in the database, with the first 10 bytes overwritten with the transaction's . /// Transaction to use for the operation /// Name of the key whose value is to be mutated. From 79194993594a5d1ac92b917c8cbdcf3b38c7bd5b Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 14:46:25 +0200 Subject: [PATCH 124/153] Fix parsing of incomplete VersionStamps, using the highest bit to distinguish between cases --- FoundationDB.Client/Native/FdbNative.cs | 6 ++--- FoundationDB.Client/VersionStamp.cs | 35 +++++-------------------- FoundationDB.Tests/VersionStampFacts.cs | 2 +- 3 files changed, 9 insertions(+), 34 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 8af1fa1d5..1f865f658 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -848,10 +848,8 @@ public static FdbError FutureGetVersionStamp(FutureHandle future, out VersionSta stamp = default; return err; } - //note: we assume that this is a complete stamp read from the database. - //BUGBUG: if the code serialize an incomplete stamp into a tuple, and unpacks it (logging?) it MAY be changed into a complete one! - // => we could check for the 'all FF' signature, but this only works for default incomplete tokens, not custom incomplete tokens ! - VersionStamp.ReadUnsafe(ptr, 10, /*FLAGS_NONE*/0, out stamp); + + VersionStamp.ReadUnsafe(ptr, 10, out stamp); return err; } diff --git a/FoundationDB.Client/VersionStamp.cs b/FoundationDB.Client/VersionStamp.cs index d1f3db3eb..beec01694 100644 --- a/FoundationDB.Client/VersionStamp.cs +++ b/FoundationDB.Client/VersionStamp.cs @@ -52,6 +52,7 @@ namespace FoundationDB.Client private const ulong PLACEHOLDER_VERSION = ulong.MaxValue; private const ushort PLACEHOLDER_ORDER = ushort.MaxValue; private const ushort NO_USER_VERSION = 0; + private const ulong HSB_VERSION = 0x8000000000000000UL; private const ushort FLAGS_NONE = 0x0; private const ushort FLAGS_HAS_VERSION = 0x1; // unset: 80-bits, set: 96-bits @@ -262,46 +263,22 @@ public static bool TryParse(Slice data, out VersionStamp vs) { fixed (byte* ptr = &data.DangerousGetPinnableReference()) { - ReadUnsafe(ptr, data.Count, FLAGS_NONE, out vs); + ReadUnsafe(ptr, data.Count, out vs); return true; } } } - /// Parse a VersionStamp from a sequence of 10 bytes - /// If the buffer length is not exactly 12 bytes - [Pure] - public static VersionStamp ParseIncomplete(Slice data) - { - return TryParseIncomplete(data, out var vs) ? vs : throw new FormatException("A VersionStamp is either 10 or 12 bytes."); - } - - /// Try parsing a VersionStamp from a sequence of bytes - public static bool TryParseIncomplete(Slice data, out VersionStamp vs) - { - if (data.Count != 10 && data.Count != 12) - { - vs = default; - return false; - } - unsafe - { - fixed (byte* ptr = &data.DangerousGetPinnableReference()) - { - ReadUnsafe(ptr, data.Count, FLAGS_IS_INCOMPLETE, out vs); - return true; - } - } - } - - internal static unsafe void ReadUnsafe(byte* ptr, int len, ushort flags, out VersionStamp vs) + internal static unsafe void ReadUnsafe(byte* ptr, int len, out VersionStamp vs) { Contract.Debug.Assert(len == 10 || len == 12); // reads a complete 12 bytes Versionstamp ulong ver = UnsafeHelpers.LoadUInt64BE(ptr); ushort order = UnsafeHelpers.LoadUInt16BE(ptr + 8); ushort idx = len == 10 ? NO_USER_VERSION : UnsafeHelpers.LoadUInt16BE(ptr + 10); - flags |= len == 12 ? FLAGS_HAS_VERSION : FLAGS_NONE; + ushort flags = FLAGS_NONE; + if (len == 12) flags |= FLAGS_HAS_VERSION; + if ((ver & HSB_VERSION) != 0) flags |= FLAGS_IS_INCOMPLETE; vs = new VersionStamp(ver, order, idx, flags); } diff --git a/FoundationDB.Tests/VersionStampFacts.cs b/FoundationDB.Tests/VersionStampFacts.cs index 6075d688d..2e190ecf4 100644 --- a/FoundationDB.Tests/VersionStampFacts.cs +++ b/FoundationDB.Tests/VersionStampFacts.cs @@ -109,7 +109,7 @@ public void Test_Incomplete_VersionStamp() Assert.That(vs.TransactionVersion, Is.EqualTo(ulong.MaxValue)); Assert.That(vs.TransactionOrder, Is.EqualTo(ushort.MaxValue)); Assert.That(vs.UserVersion, Is.EqualTo(123)); - Assert.That(vs.IsIncomplete, Is.False, "NOTE: reading stamps is only supposed to happen for stamps already in the database!"); + Assert.That(vs.IsIncomplete, Is.True); } { From 906b54b5680cbc3d944d4df53721cb6cd701abbb Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 14:56:38 +0200 Subject: [PATCH 125/153] Move files around to have all "shared framework" types under the Shared framework (Slice, Tuples, AsyncLINQ, infrastructure, ...) - All these files are part of a "shared" framework which could be split off into another package in the future. --- .../{ => Shared}/Async/AsyncBuffer.cs | 0 .../{ => Shared}/Async/AsyncCancellableMutex.cs | 0 .../{ => Shared}/Async/AsyncHelpers.cs | 0 .../{ => Shared}/Async/AsyncOrderingMode.cs | 0 .../Async/AsyncProducerConsumerQueue.cs | 0 .../{ => Shared}/Async/AsyncTaskBuffer.cs | 0 .../{ => Shared}/Async/AsyncTransform.cs | 0 .../{ => Shared}/Async/AsyncTransformQueue.cs | 0 .../{ => Shared}/Async/IAsyncBuffer.cs | 0 .../{ => Shared}/Async/IAsyncSource.cs | 0 .../{ => Shared}/Async/IAsyncTarget.cs | 0 FoundationDB.Client/{ => Shared}/Async/Maybe.cs | 0 .../{ => Shared}/Async/TaskHelpers.cs | 0 .../{Utils => Shared}/CodeAnnotations.cs | 0 FoundationDB.Client/{Utils => Shared}/Contract.cs | 0 .../{Utils => Shared}/ContractException.cs | 0 .../{ => Shared}/Converters/ComparisonHelper.cs | 0 .../{ => Shared}/Converters/ITypeConverter.cs | 0 .../Converters/SimilarValueComparer.cs | 0 .../{ => Shared}/Converters/TypeConverters.cs | 0 .../{Utils => Shared}/ExceptionExtensions.cs | 0 FoundationDB.Client/{Utils => Shared}/HashCodes.cs | 0 .../{ => Shared}/Linq/Async/AsyncSequence.cs | 0 .../Async/Expressions/AsyncFilterExpression.cs | 0 .../Async/Expressions/AsyncObserverExpression.cs | 0 .../Async/Expressions/AsyncTransformExpression.cs | 0 .../Async/Iterators/AnonymousAsyncGenerator.cs | 0 .../Linq/Async/Iterators/AsyncFilterIterator.cs | 0 .../Linq/Async/Iterators/AsyncIterator.cs | 0 .../Linq/Async/Iterators/AsyncIteratorPump.cs | 0 .../Linq/Async/Iterators/BatchingAsyncIterator.cs | 0 .../Linq/Async/Iterators/DistinctAsyncIterator.cs | 0 .../Linq/Async/Iterators/ExceptAsyncIterator.cs | 0 .../Linq/Async/Iterators/IntersectAsyncIterator.cs | 0 .../Linq/Async/Iterators/MergeAsyncIterator.cs | 0 .../Linq/Async/Iterators/MergeSortAsyncIterator.cs | 0 .../Linq/Async/Iterators/ObserverAsyncIterator.cs | 0 .../Async/Iterators/ParallelSelectAsyncIterator.cs | 0 .../Async/Iterators/PrefetchingAsyncIterator.cs | 0 .../Async/Iterators/SelectManyAsyncIterator.cs | 0 .../Linq/Async/Iterators/TakeWhileAsyncIterator.cs | 0 .../Linq/Async/Iterators/WhereAsyncIterator.cs | 0 .../Async/Iterators/WhereSelectAsyncIterator.cs | 0 .../Linq/Async/Iterators/WindowingAsyncIterator.cs | 0 .../Linq/AsyncEnumerable.EmptySequence.cs | 0 .../Linq/AsyncEnumerable.EnumerableIterator.cs | 0 .../Linq/AsyncEnumerable.EnumerableSequence.cs | 0 .../{ => Shared}/Linq/AsyncEnumerable.Iterators.cs | 0 .../Linq/AsyncEnumerable.OrderedSequence.cs | 0 .../{ => Shared}/Linq/AsyncEnumerable.Sorters.cs | 0 .../{ => Shared}/Linq/AsyncEnumerable.cs | 0 .../{ => Shared}/Linq/AsyncIterationHint.cs | 0 .../{ => Shared}/Linq/EnumerableExtensions.cs | 0 .../{ => Shared}/Linq/IAsyncEnumerable.cs | 0 .../{ => Shared}/Linq/IAsyncEnumerator.cs | 0 .../{ => Shared}/Linq/IAsyncOrderedEnumerable.cs | 0 .../{ => Shared}/Linq/ParallelAsyncQueryOptions.cs | 0 .../{ => Shared}/Linq/QueryStatistics.cs | 0 .../{Utils => Shared}/Memory/BitHelpers.cs | 0 .../{Utils => Shared}/Memory/Slice.Comparer.cs | 0 .../{Utils => Shared}/Memory/Slice.Encoding.cs | 0 .../{Utils => Shared}/Memory/Slice.cs | 0 .../{Utils => Shared}/Memory/SliceBuffer.cs | 0 .../{Utils => Shared}/Memory/SliceListStream.cs | 0 .../{Utils => Shared}/Memory/SlicePairComparer.cs | 0 .../{Utils => Shared}/Memory/SliceReader.cs | 0 .../{Utils => Shared}/Memory/SliceStream.cs | 0 .../{Utils => Shared}/Memory/SliceWriter.cs | 0 .../{Utils => Shared}/Memory/UnsafeHelpers.cs | 0 FoundationDB.Client/Shared/README.MD | 1 + .../{Utils => Shared}/StringConverters.cs | 0 .../{Utils => Shared}/ThrowHelper.cs | 0 .../Tuples/Formatters/AnonymousTupleFormatter.cs | 0 .../Tuples/Formatters/FormattableTupleFormatter.cs | 0 .../Tuples/Formatters/GenericTupleFormatter.cs | 0 .../Tuples/Formatters/ITupleFormattable.cs | 0 .../Tuples/Formatters/ITupleFormatter.cs | 0 .../Tuples/Formatters/TupleFormatter.cs | 0 .../{Layers => Shared}/Tuples/ITuple.cs | 0 .../{Layers => Shared}/Tuples/JoinedTuple.cs | 0 .../{Layers => Shared}/Tuples/LinkedTuple.cs | 0 .../{Layers => Shared}/Tuples/ListTuple.cs | 0 .../{Layers => Shared}/Tuples/STuple.cs | 0 .../{Layers => Shared}/Tuples/STuple`1.cs | 0 .../{Layers => Shared}/Tuples/STuple`2.cs | 0 .../{Layers => Shared}/Tuples/STuple`3.cs | 0 .../{Layers => Shared}/Tuples/STuple`4.cs | 0 .../{Layers => Shared}/Tuples/STuple`5.cs | 0 .../{Layers => Shared}/Tuples/STuple`6.cs | 0 .../{Layers => Shared}/Tuples/TupleComparisons.cs | 0 .../{Layers => Shared}/Tuples/TupleExtensions.cs | 14 -------------- .../{Layers => Shared}/Tuples/TupleHelpers.cs | 0 .../TypeSystem/Encoders/DynamicKeyEncoderBase.cs | 0 .../TypeSystem/Encoders/ICompositeKeyEncoder.cs | 0 .../TypeSystem/Encoders/IDynamicKeyEncoder.cs | 0 .../TypeSystem/Encoders/IKeyEncoder.cs | 0 .../TypeSystem/Encoders/IValueEncoder.cs | 0 .../Encoders/KeyValueEncoders.Ordered.cs | 0 .../TypeSystem/Encoders/KeyValueEncoders.Tuples.cs | 0 .../Encoders/KeyValueEncoders.Unordered.cs | 0 .../TypeSystem/Encoders/KeyValueEncoders.Values.cs | 0 .../TypeSystem/Encoders/KeyValueEncoders.cs | 0 .../{ => Shared}/TypeSystem/IKeyEncoding.cs | 0 .../{ => Shared}/TypeSystem/IOrderedTypeCodec.cs | 0 .../{ => Shared}/TypeSystem/IUnorderedTypeCodec.cs | 0 .../{ => Shared}/TypeSystem/TypeCodec`1.cs | 0 .../{ => Shared}/TypeSystem/TypeSystem.cs | 0 FoundationDB.Client/{Utils => Shared}/Uuid128.cs | 0 FoundationDB.Client/{Utils => Shared}/Uuid64.cs | 0 .../Tuples/Encoding/ITupleSerializable.cs | 0 .../{Layers => }/Tuples/Encoding/PrefixedTuple.cs | 0 .../{Layers => }/Tuples/Encoding/SlicedTuple.cs | 0 .../{Layers => }/Tuples/Encoding/TupleCodec`1.cs | 0 .../{Layers => }/Tuples/Encoding/TupleEncoder.cs | 0 .../Tuples/Encoding/TupleKeyEncoder.cs | 0 .../Tuples/Encoding/TupleKeyEncoding.cs | 0 .../{Layers => }/Tuples/Encoding/TuplePacker.cs | 0 .../{Layers => }/Tuples/Encoding/TuplePackers.cs | 0 .../{Layers => }/Tuples/Encoding/TupleParser.cs | 0 .../{Layers => }/Tuples/Encoding/TupleReader.cs | 0 .../{Layers => }/Tuples/Encoding/TupleTypes.cs | 0 .../{Layers => }/Tuples/Encoding/TupleWriter.cs | 0 .../{Layers => }/Tuples/FdbTupleAlias.cs | 0 .../{Layers => }/Tuples/MemoizedTuple.cs | 0 FoundationDB.Client/{Layers => }/Tuples/TuPack.cs | 0 .../{Layers => }/Tuples/TupleSerializer.cs | 0 126 files changed, 1 insertion(+), 14 deletions(-) rename FoundationDB.Client/{ => Shared}/Async/AsyncBuffer.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncCancellableMutex.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncHelpers.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncOrderingMode.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncProducerConsumerQueue.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncTaskBuffer.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncTransform.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/AsyncTransformQueue.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/IAsyncBuffer.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/IAsyncSource.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/IAsyncTarget.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/Maybe.cs (100%) rename FoundationDB.Client/{ => Shared}/Async/TaskHelpers.cs (100%) rename FoundationDB.Client/{Utils => Shared}/CodeAnnotations.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Contract.cs (100%) rename FoundationDB.Client/{Utils => Shared}/ContractException.cs (100%) rename FoundationDB.Client/{ => Shared}/Converters/ComparisonHelper.cs (100%) rename FoundationDB.Client/{ => Shared}/Converters/ITypeConverter.cs (100%) rename FoundationDB.Client/{ => Shared}/Converters/SimilarValueComparer.cs (100%) rename FoundationDB.Client/{ => Shared}/Converters/TypeConverters.cs (100%) rename FoundationDB.Client/{Utils => Shared}/ExceptionExtensions.cs (100%) rename FoundationDB.Client/{Utils => Shared}/HashCodes.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/AsyncSequence.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Expressions/AsyncFilterExpression.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Expressions/AsyncObserverExpression.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Expressions/AsyncTransformExpression.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/AnonymousAsyncGenerator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/AsyncFilterIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/AsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/AsyncIteratorPump.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/BatchingAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/DistinctAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/ExceptAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/IntersectAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/MergeAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/MergeSortAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/ObserverAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/PrefetchingAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/SelectManyAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/TakeWhileAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/WhereAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/WhereSelectAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/Async/Iterators/WindowingAsyncIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.EmptySequence.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.EnumerableIterator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.EnumerableSequence.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.Iterators.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.OrderedSequence.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.Sorters.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncEnumerable.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/AsyncIterationHint.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/EnumerableExtensions.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/IAsyncEnumerable.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/IAsyncEnumerator.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/IAsyncOrderedEnumerable.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/ParallelAsyncQueryOptions.cs (100%) rename FoundationDB.Client/{ => Shared}/Linq/QueryStatistics.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/BitHelpers.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/Slice.Comparer.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/Slice.Encoding.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/Slice.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SliceBuffer.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SliceListStream.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SlicePairComparer.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SliceReader.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SliceStream.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/SliceWriter.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Memory/UnsafeHelpers.cs (100%) create mode 100644 FoundationDB.Client/Shared/README.MD rename FoundationDB.Client/{Utils => Shared}/StringConverters.cs (100%) rename FoundationDB.Client/{Utils => Shared}/ThrowHelper.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/AnonymousTupleFormatter.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/FormattableTupleFormatter.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/GenericTupleFormatter.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/ITupleFormattable.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/ITupleFormatter.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/Formatters/TupleFormatter.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/ITuple.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/JoinedTuple.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/LinkedTuple.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/ListTuple.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`1.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`2.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`3.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`4.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`5.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/STuple`6.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/TupleComparisons.cs (100%) rename FoundationDB.Client/{Layers => Shared}/Tuples/TupleExtensions.cs (98%) rename FoundationDB.Client/{Layers => Shared}/Tuples/TupleHelpers.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/DynamicKeyEncoderBase.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/ICompositeKeyEncoder.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/IDynamicKeyEncoder.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/IKeyEncoder.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/IValueEncoder.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/KeyValueEncoders.Values.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/Encoders/KeyValueEncoders.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/IKeyEncoding.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/IOrderedTypeCodec.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/IUnorderedTypeCodec.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/TypeCodec`1.cs (100%) rename FoundationDB.Client/{ => Shared}/TypeSystem/TypeSystem.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Uuid128.cs (100%) rename FoundationDB.Client/{Utils => Shared}/Uuid64.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/ITupleSerializable.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/PrefixedTuple.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/SlicedTuple.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleCodec`1.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleEncoder.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleKeyEncoder.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleKeyEncoding.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TuplePacker.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TuplePackers.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleParser.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleReader.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleTypes.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/Encoding/TupleWriter.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/FdbTupleAlias.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/MemoizedTuple.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/TuPack.cs (100%) rename FoundationDB.Client/{Layers => }/Tuples/TupleSerializer.cs (100%) diff --git a/FoundationDB.Client/Async/AsyncBuffer.cs b/FoundationDB.Client/Shared/Async/AsyncBuffer.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncBuffer.cs rename to FoundationDB.Client/Shared/Async/AsyncBuffer.cs diff --git a/FoundationDB.Client/Async/AsyncCancellableMutex.cs b/FoundationDB.Client/Shared/Async/AsyncCancellableMutex.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncCancellableMutex.cs rename to FoundationDB.Client/Shared/Async/AsyncCancellableMutex.cs diff --git a/FoundationDB.Client/Async/AsyncHelpers.cs b/FoundationDB.Client/Shared/Async/AsyncHelpers.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncHelpers.cs rename to FoundationDB.Client/Shared/Async/AsyncHelpers.cs diff --git a/FoundationDB.Client/Async/AsyncOrderingMode.cs b/FoundationDB.Client/Shared/Async/AsyncOrderingMode.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncOrderingMode.cs rename to FoundationDB.Client/Shared/Async/AsyncOrderingMode.cs diff --git a/FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs b/FoundationDB.Client/Shared/Async/AsyncProducerConsumerQueue.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncProducerConsumerQueue.cs rename to FoundationDB.Client/Shared/Async/AsyncProducerConsumerQueue.cs diff --git a/FoundationDB.Client/Async/AsyncTaskBuffer.cs b/FoundationDB.Client/Shared/Async/AsyncTaskBuffer.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncTaskBuffer.cs rename to FoundationDB.Client/Shared/Async/AsyncTaskBuffer.cs diff --git a/FoundationDB.Client/Async/AsyncTransform.cs b/FoundationDB.Client/Shared/Async/AsyncTransform.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncTransform.cs rename to FoundationDB.Client/Shared/Async/AsyncTransform.cs diff --git a/FoundationDB.Client/Async/AsyncTransformQueue.cs b/FoundationDB.Client/Shared/Async/AsyncTransformQueue.cs similarity index 100% rename from FoundationDB.Client/Async/AsyncTransformQueue.cs rename to FoundationDB.Client/Shared/Async/AsyncTransformQueue.cs diff --git a/FoundationDB.Client/Async/IAsyncBuffer.cs b/FoundationDB.Client/Shared/Async/IAsyncBuffer.cs similarity index 100% rename from FoundationDB.Client/Async/IAsyncBuffer.cs rename to FoundationDB.Client/Shared/Async/IAsyncBuffer.cs diff --git a/FoundationDB.Client/Async/IAsyncSource.cs b/FoundationDB.Client/Shared/Async/IAsyncSource.cs similarity index 100% rename from FoundationDB.Client/Async/IAsyncSource.cs rename to FoundationDB.Client/Shared/Async/IAsyncSource.cs diff --git a/FoundationDB.Client/Async/IAsyncTarget.cs b/FoundationDB.Client/Shared/Async/IAsyncTarget.cs similarity index 100% rename from FoundationDB.Client/Async/IAsyncTarget.cs rename to FoundationDB.Client/Shared/Async/IAsyncTarget.cs diff --git a/FoundationDB.Client/Async/Maybe.cs b/FoundationDB.Client/Shared/Async/Maybe.cs similarity index 100% rename from FoundationDB.Client/Async/Maybe.cs rename to FoundationDB.Client/Shared/Async/Maybe.cs diff --git a/FoundationDB.Client/Async/TaskHelpers.cs b/FoundationDB.Client/Shared/Async/TaskHelpers.cs similarity index 100% rename from FoundationDB.Client/Async/TaskHelpers.cs rename to FoundationDB.Client/Shared/Async/TaskHelpers.cs diff --git a/FoundationDB.Client/Utils/CodeAnnotations.cs b/FoundationDB.Client/Shared/CodeAnnotations.cs similarity index 100% rename from FoundationDB.Client/Utils/CodeAnnotations.cs rename to FoundationDB.Client/Shared/CodeAnnotations.cs diff --git a/FoundationDB.Client/Utils/Contract.cs b/FoundationDB.Client/Shared/Contract.cs similarity index 100% rename from FoundationDB.Client/Utils/Contract.cs rename to FoundationDB.Client/Shared/Contract.cs diff --git a/FoundationDB.Client/Utils/ContractException.cs b/FoundationDB.Client/Shared/ContractException.cs similarity index 100% rename from FoundationDB.Client/Utils/ContractException.cs rename to FoundationDB.Client/Shared/ContractException.cs diff --git a/FoundationDB.Client/Converters/ComparisonHelper.cs b/FoundationDB.Client/Shared/Converters/ComparisonHelper.cs similarity index 100% rename from FoundationDB.Client/Converters/ComparisonHelper.cs rename to FoundationDB.Client/Shared/Converters/ComparisonHelper.cs diff --git a/FoundationDB.Client/Converters/ITypeConverter.cs b/FoundationDB.Client/Shared/Converters/ITypeConverter.cs similarity index 100% rename from FoundationDB.Client/Converters/ITypeConverter.cs rename to FoundationDB.Client/Shared/Converters/ITypeConverter.cs diff --git a/FoundationDB.Client/Converters/SimilarValueComparer.cs b/FoundationDB.Client/Shared/Converters/SimilarValueComparer.cs similarity index 100% rename from FoundationDB.Client/Converters/SimilarValueComparer.cs rename to FoundationDB.Client/Shared/Converters/SimilarValueComparer.cs diff --git a/FoundationDB.Client/Converters/TypeConverters.cs b/FoundationDB.Client/Shared/Converters/TypeConverters.cs similarity index 100% rename from FoundationDB.Client/Converters/TypeConverters.cs rename to FoundationDB.Client/Shared/Converters/TypeConverters.cs diff --git a/FoundationDB.Client/Utils/ExceptionExtensions.cs b/FoundationDB.Client/Shared/ExceptionExtensions.cs similarity index 100% rename from FoundationDB.Client/Utils/ExceptionExtensions.cs rename to FoundationDB.Client/Shared/ExceptionExtensions.cs diff --git a/FoundationDB.Client/Utils/HashCodes.cs b/FoundationDB.Client/Shared/HashCodes.cs similarity index 100% rename from FoundationDB.Client/Utils/HashCodes.cs rename to FoundationDB.Client/Shared/HashCodes.cs diff --git a/FoundationDB.Client/Linq/Async/AsyncSequence.cs b/FoundationDB.Client/Shared/Linq/Async/AsyncSequence.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/AsyncSequence.cs rename to FoundationDB.Client/Shared/Linq/Async/AsyncSequence.cs diff --git a/FoundationDB.Client/Linq/Async/Expressions/AsyncFilterExpression.cs b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Expressions/AsyncFilterExpression.cs rename to FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs diff --git a/FoundationDB.Client/Linq/Async/Expressions/AsyncObserverExpression.cs b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncObserverExpression.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Expressions/AsyncObserverExpression.cs rename to FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncObserverExpression.cs diff --git a/FoundationDB.Client/Linq/Async/Expressions/AsyncTransformExpression.cs b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Expressions/AsyncTransformExpression.cs rename to FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/AnonymousAsyncGenerator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/AnonymousAsyncGenerator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/AsyncFilterIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/AsyncFilterIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/AsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/AsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncIteratorPump.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/AsyncIteratorPump.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncIteratorPump.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/BatchingAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/BatchingAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/BatchingAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/DistinctAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/DistinctAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/DistinctAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/DistinctAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/ExceptAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/ExceptAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/ExceptAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/IntersectAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/IntersectAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/IntersectAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/MergeAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeSortAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/MergeSortAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/MergeSortAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/ObserverAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/ObserverAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/ObserverAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/ObserverAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/PrefetchingAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/SelectManyAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/SelectManyAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/SelectManyAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/SelectManyAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/TakeWhileAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/TakeWhileAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/TakeWhileAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/TakeWhileAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/WhereAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/WhereAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/WhereSelectAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereSelectAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/WhereSelectAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/WhereSelectAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/Async/Iterators/WindowingAsyncIterator.cs rename to FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.EmptySequence.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EnumerableIterator.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.EnumerableIterator.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.EnumerableIterator.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EnumerableSequence.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.EnumerableSequence.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.EnumerableSequence.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.Iterators.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.Iterators.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.OrderedSequence.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.Sorters.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Sorters.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.Sorters.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.Sorters.cs diff --git a/FoundationDB.Client/Linq/AsyncEnumerable.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncEnumerable.cs rename to FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs diff --git a/FoundationDB.Client/Linq/AsyncIterationHint.cs b/FoundationDB.Client/Shared/Linq/AsyncIterationHint.cs similarity index 100% rename from FoundationDB.Client/Linq/AsyncIterationHint.cs rename to FoundationDB.Client/Shared/Linq/AsyncIterationHint.cs diff --git a/FoundationDB.Client/Linq/EnumerableExtensions.cs b/FoundationDB.Client/Shared/Linq/EnumerableExtensions.cs similarity index 100% rename from FoundationDB.Client/Linq/EnumerableExtensions.cs rename to FoundationDB.Client/Shared/Linq/EnumerableExtensions.cs diff --git a/FoundationDB.Client/Linq/IAsyncEnumerable.cs b/FoundationDB.Client/Shared/Linq/IAsyncEnumerable.cs similarity index 100% rename from FoundationDB.Client/Linq/IAsyncEnumerable.cs rename to FoundationDB.Client/Shared/Linq/IAsyncEnumerable.cs diff --git a/FoundationDB.Client/Linq/IAsyncEnumerator.cs b/FoundationDB.Client/Shared/Linq/IAsyncEnumerator.cs similarity index 100% rename from FoundationDB.Client/Linq/IAsyncEnumerator.cs rename to FoundationDB.Client/Shared/Linq/IAsyncEnumerator.cs diff --git a/FoundationDB.Client/Linq/IAsyncOrderedEnumerable.cs b/FoundationDB.Client/Shared/Linq/IAsyncOrderedEnumerable.cs similarity index 100% rename from FoundationDB.Client/Linq/IAsyncOrderedEnumerable.cs rename to FoundationDB.Client/Shared/Linq/IAsyncOrderedEnumerable.cs diff --git a/FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs b/FoundationDB.Client/Shared/Linq/ParallelAsyncQueryOptions.cs similarity index 100% rename from FoundationDB.Client/Linq/ParallelAsyncQueryOptions.cs rename to FoundationDB.Client/Shared/Linq/ParallelAsyncQueryOptions.cs diff --git a/FoundationDB.Client/Linq/QueryStatistics.cs b/FoundationDB.Client/Shared/Linq/QueryStatistics.cs similarity index 100% rename from FoundationDB.Client/Linq/QueryStatistics.cs rename to FoundationDB.Client/Shared/Linq/QueryStatistics.cs diff --git a/FoundationDB.Client/Utils/Memory/BitHelpers.cs b/FoundationDB.Client/Shared/Memory/BitHelpers.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/BitHelpers.cs rename to FoundationDB.Client/Shared/Memory/BitHelpers.cs diff --git a/FoundationDB.Client/Utils/Memory/Slice.Comparer.cs b/FoundationDB.Client/Shared/Memory/Slice.Comparer.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/Slice.Comparer.cs rename to FoundationDB.Client/Shared/Memory/Slice.Comparer.cs diff --git a/FoundationDB.Client/Utils/Memory/Slice.Encoding.cs b/FoundationDB.Client/Shared/Memory/Slice.Encoding.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/Slice.Encoding.cs rename to FoundationDB.Client/Shared/Memory/Slice.Encoding.cs diff --git a/FoundationDB.Client/Utils/Memory/Slice.cs b/FoundationDB.Client/Shared/Memory/Slice.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/Slice.cs rename to FoundationDB.Client/Shared/Memory/Slice.cs diff --git a/FoundationDB.Client/Utils/Memory/SliceBuffer.cs b/FoundationDB.Client/Shared/Memory/SliceBuffer.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SliceBuffer.cs rename to FoundationDB.Client/Shared/Memory/SliceBuffer.cs diff --git a/FoundationDB.Client/Utils/Memory/SliceListStream.cs b/FoundationDB.Client/Shared/Memory/SliceListStream.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SliceListStream.cs rename to FoundationDB.Client/Shared/Memory/SliceListStream.cs diff --git a/FoundationDB.Client/Utils/Memory/SlicePairComparer.cs b/FoundationDB.Client/Shared/Memory/SlicePairComparer.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SlicePairComparer.cs rename to FoundationDB.Client/Shared/Memory/SlicePairComparer.cs diff --git a/FoundationDB.Client/Utils/Memory/SliceReader.cs b/FoundationDB.Client/Shared/Memory/SliceReader.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SliceReader.cs rename to FoundationDB.Client/Shared/Memory/SliceReader.cs diff --git a/FoundationDB.Client/Utils/Memory/SliceStream.cs b/FoundationDB.Client/Shared/Memory/SliceStream.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SliceStream.cs rename to FoundationDB.Client/Shared/Memory/SliceStream.cs diff --git a/FoundationDB.Client/Utils/Memory/SliceWriter.cs b/FoundationDB.Client/Shared/Memory/SliceWriter.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/SliceWriter.cs rename to FoundationDB.Client/Shared/Memory/SliceWriter.cs diff --git a/FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs b/FoundationDB.Client/Shared/Memory/UnsafeHelpers.cs similarity index 100% rename from FoundationDB.Client/Utils/Memory/UnsafeHelpers.cs rename to FoundationDB.Client/Shared/Memory/UnsafeHelpers.cs diff --git a/FoundationDB.Client/Shared/README.MD b/FoundationDB.Client/Shared/README.MD new file mode 100644 index 000000000..2d9659951 --- /dev/null +++ b/FoundationDB.Client/Shared/README.MD @@ -0,0 +1 @@ +This folder contains all the shared framework libraries and tools. \ No newline at end of file diff --git a/FoundationDB.Client/Utils/StringConverters.cs b/FoundationDB.Client/Shared/StringConverters.cs similarity index 100% rename from FoundationDB.Client/Utils/StringConverters.cs rename to FoundationDB.Client/Shared/StringConverters.cs diff --git a/FoundationDB.Client/Utils/ThrowHelper.cs b/FoundationDB.Client/Shared/ThrowHelper.cs similarity index 100% rename from FoundationDB.Client/Utils/ThrowHelper.cs rename to FoundationDB.Client/Shared/ThrowHelper.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs b/FoundationDB.Client/Shared/Tuples/Formatters/AnonymousTupleFormatter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/AnonymousTupleFormatter.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/AnonymousTupleFormatter.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs b/FoundationDB.Client/Shared/Tuples/Formatters/FormattableTupleFormatter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/FormattableTupleFormatter.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/FormattableTupleFormatter.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs b/FoundationDB.Client/Shared/Tuples/Formatters/GenericTupleFormatter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/GenericTupleFormatter.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/GenericTupleFormatter.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs b/FoundationDB.Client/Shared/Tuples/Formatters/ITupleFormattable.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormattable.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/ITupleFormattable.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs b/FoundationDB.Client/Shared/Tuples/Formatters/ITupleFormatter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/ITupleFormatter.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/ITupleFormatter.cs diff --git a/FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs b/FoundationDB.Client/Shared/Tuples/Formatters/TupleFormatter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Formatters/TupleFormatter.cs rename to FoundationDB.Client/Shared/Tuples/Formatters/TupleFormatter.cs diff --git a/FoundationDB.Client/Layers/Tuples/ITuple.cs b/FoundationDB.Client/Shared/Tuples/ITuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/ITuple.cs rename to FoundationDB.Client/Shared/Tuples/ITuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/JoinedTuple.cs b/FoundationDB.Client/Shared/Tuples/JoinedTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/JoinedTuple.cs rename to FoundationDB.Client/Shared/Tuples/JoinedTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/LinkedTuple.cs b/FoundationDB.Client/Shared/Tuples/LinkedTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/LinkedTuple.cs rename to FoundationDB.Client/Shared/Tuples/LinkedTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/ListTuple.cs b/FoundationDB.Client/Shared/Tuples/ListTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/ListTuple.cs rename to FoundationDB.Client/Shared/Tuples/ListTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple.cs b/FoundationDB.Client/Shared/Tuples/STuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple.cs rename to FoundationDB.Client/Shared/Tuples/STuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`1.cs b/FoundationDB.Client/Shared/Tuples/STuple`1.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`1.cs rename to FoundationDB.Client/Shared/Tuples/STuple`1.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`2.cs b/FoundationDB.Client/Shared/Tuples/STuple`2.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`2.cs rename to FoundationDB.Client/Shared/Tuples/STuple`2.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`3.cs b/FoundationDB.Client/Shared/Tuples/STuple`3.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`3.cs rename to FoundationDB.Client/Shared/Tuples/STuple`3.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`4.cs b/FoundationDB.Client/Shared/Tuples/STuple`4.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`4.cs rename to FoundationDB.Client/Shared/Tuples/STuple`4.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`5.cs b/FoundationDB.Client/Shared/Tuples/STuple`5.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`5.cs rename to FoundationDB.Client/Shared/Tuples/STuple`5.cs diff --git a/FoundationDB.Client/Layers/Tuples/STuple`6.cs b/FoundationDB.Client/Shared/Tuples/STuple`6.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/STuple`6.cs rename to FoundationDB.Client/Shared/Tuples/STuple`6.cs diff --git a/FoundationDB.Client/Layers/Tuples/TupleComparisons.cs b/FoundationDB.Client/Shared/Tuples/TupleComparisons.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TupleComparisons.cs rename to FoundationDB.Client/Shared/Tuples/TupleComparisons.cs diff --git a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs b/FoundationDB.Client/Shared/Tuples/TupleExtensions.cs similarity index 98% rename from FoundationDB.Client/Layers/Tuples/TupleExtensions.cs rename to FoundationDB.Client/Shared/Tuples/TupleExtensions.cs index 0c0a47911..562ee330c 100644 --- a/FoundationDB.Client/Layers/Tuples/TupleExtensions.cs +++ b/FoundationDB.Client/Shared/Tuples/TupleExtensions.cs @@ -280,20 +280,6 @@ internal static void ThrowInvalidTupleSize(ITuple tuple, int expected, int test) } } - /// Creates pre-packed and isolated copy of this tuple - /// - /// Create a copy of the tuple that can be reused frequently to pack values - /// If the tuple is already memoized, the current instance will be returned - [CanBeNull, ContractAnnotation("null => null")] - public static MemoizedTuple Memoize(this ITuple tuple) - { - if (tuple == null) return null; - - var memoized = tuple as MemoizedTuple ?? new MemoizedTuple(tuple.ToArray(), TuPack.Pack(tuple)); - - return memoized; - } - /// Returns a typed version of a tuple of size 1 /// Expected type of the single element /// Tuple that must be of size 1 diff --git a/FoundationDB.Client/Layers/Tuples/TupleHelpers.cs b/FoundationDB.Client/Shared/Tuples/TupleHelpers.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TupleHelpers.cs rename to FoundationDB.Client/Shared/Tuples/TupleHelpers.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/DynamicKeyEncoderBase.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/ICompositeKeyEncoder.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/IDynamicKeyEncoder.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/IKeyEncoder.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/IValueEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/IValueEncoder.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/IValueEncoder.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/IValueEncoder.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Unordered.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Values.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.Values.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Values.cs diff --git a/FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/Encoders/KeyValueEncoders.cs rename to FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs diff --git a/FoundationDB.Client/TypeSystem/IKeyEncoding.cs b/FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/IKeyEncoding.cs rename to FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs diff --git a/FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs b/FoundationDB.Client/Shared/TypeSystem/IOrderedTypeCodec.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/IOrderedTypeCodec.cs rename to FoundationDB.Client/Shared/TypeSystem/IOrderedTypeCodec.cs diff --git a/FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs b/FoundationDB.Client/Shared/TypeSystem/IUnorderedTypeCodec.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/IUnorderedTypeCodec.cs rename to FoundationDB.Client/Shared/TypeSystem/IUnorderedTypeCodec.cs diff --git a/FoundationDB.Client/TypeSystem/TypeCodec`1.cs b/FoundationDB.Client/Shared/TypeSystem/TypeCodec`1.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/TypeCodec`1.cs rename to FoundationDB.Client/Shared/TypeSystem/TypeCodec`1.cs diff --git a/FoundationDB.Client/TypeSystem/TypeSystem.cs b/FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs similarity index 100% rename from FoundationDB.Client/TypeSystem/TypeSystem.cs rename to FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs diff --git a/FoundationDB.Client/Utils/Uuid128.cs b/FoundationDB.Client/Shared/Uuid128.cs similarity index 100% rename from FoundationDB.Client/Utils/Uuid128.cs rename to FoundationDB.Client/Shared/Uuid128.cs diff --git a/FoundationDB.Client/Utils/Uuid64.cs b/FoundationDB.Client/Shared/Uuid64.cs similarity index 100% rename from FoundationDB.Client/Utils/Uuid64.cs rename to FoundationDB.Client/Shared/Uuid64.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs b/FoundationDB.Client/Tuples/Encoding/ITupleSerializable.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/ITupleSerializable.cs rename to FoundationDB.Client/Tuples/Encoding/ITupleSerializable.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs b/FoundationDB.Client/Tuples/Encoding/PrefixedTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/PrefixedTuple.cs rename to FoundationDB.Client/Tuples/Encoding/PrefixedTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs b/FoundationDB.Client/Tuples/Encoding/SlicedTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/SlicedTuple.cs rename to FoundationDB.Client/Tuples/Encoding/SlicedTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs b/FoundationDB.Client/Tuples/Encoding/TupleCodec`1.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleCodec`1.cs rename to FoundationDB.Client/Tuples/Encoding/TupleCodec`1.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleEncoder.cs rename to FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoder.cs rename to FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleKeyEncoding.cs rename to FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs b/FoundationDB.Client/Tuples/Encoding/TuplePacker.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TuplePacker.cs rename to FoundationDB.Client/Tuples/Encoding/TuplePacker.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Tuples/Encoding/TuplePackers.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TuplePackers.cs rename to FoundationDB.Client/Tuples/Encoding/TuplePackers.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Tuples/Encoding/TupleParser.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleParser.cs rename to FoundationDB.Client/Tuples/Encoding/TupleParser.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs b/FoundationDB.Client/Tuples/Encoding/TupleReader.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleReader.cs rename to FoundationDB.Client/Tuples/Encoding/TupleReader.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs b/FoundationDB.Client/Tuples/Encoding/TupleTypes.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleTypes.cs rename to FoundationDB.Client/Tuples/Encoding/TupleTypes.cs diff --git a/FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs b/FoundationDB.Client/Tuples/Encoding/TupleWriter.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/Encoding/TupleWriter.cs rename to FoundationDB.Client/Tuples/Encoding/TupleWriter.cs diff --git a/FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs b/FoundationDB.Client/Tuples/FdbTupleAlias.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/FdbTupleAlias.cs rename to FoundationDB.Client/Tuples/FdbTupleAlias.cs diff --git a/FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs b/FoundationDB.Client/Tuples/MemoizedTuple.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/MemoizedTuple.cs rename to FoundationDB.Client/Tuples/MemoizedTuple.cs diff --git a/FoundationDB.Client/Layers/Tuples/TuPack.cs b/FoundationDB.Client/Tuples/TuPack.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TuPack.cs rename to FoundationDB.Client/Tuples/TuPack.cs diff --git a/FoundationDB.Client/Layers/Tuples/TupleSerializer.cs b/FoundationDB.Client/Tuples/TupleSerializer.cs similarity index 100% rename from FoundationDB.Client/Layers/Tuples/TupleSerializer.cs rename to FoundationDB.Client/Tuples/TupleSerializer.cs From 57e58e8b4f1c4681d0758c778e9a2251ec8b565d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 16:18:10 +0200 Subject: [PATCH 126/153] Move StringConverters to new namespace Doxense.Serialization --- .../Shared/Converters/TypeConverters.cs | 1 + .../Shared/StringConverters.cs | 31 +++++++++++++++++-- FoundationDB.Client/Shared/ThrowHelper.cs | 28 +++++++++++++++-- FoundationDB.Client/Shared/Tuples/STuple.cs | 1 + FoundationDB.Client/Shared/Uuid64.cs | 1 - 5 files changed, 57 insertions(+), 5 deletions(-) diff --git a/FoundationDB.Client/Shared/Converters/TypeConverters.cs b/FoundationDB.Client/Shared/Converters/TypeConverters.cs index 37b36a830..0f960ffdc 100644 --- a/FoundationDB.Client/Shared/Converters/TypeConverters.cs +++ b/FoundationDB.Client/Shared/Converters/TypeConverters.cs @@ -36,6 +36,7 @@ namespace Doxense.Runtime.Converters using System.Runtime.CompilerServices; using System.Threading; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization; using JetBrains.Annotations; /// Helper class to convert object from one type to another diff --git a/FoundationDB.Client/Shared/StringConverters.cs b/FoundationDB.Client/Shared/StringConverters.cs index 073a7d7de..101ea1379 100644 --- a/FoundationDB.Client/Shared/StringConverters.cs +++ b/FoundationDB.Client/Shared/StringConverters.cs @@ -1,5 +1,32 @@ - -namespace Doxense +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Serialization { using System; using System.Globalization; diff --git a/FoundationDB.Client/Shared/ThrowHelper.cs b/FoundationDB.Client/Shared/ThrowHelper.cs index 5a5e99610..a6099d00c 100644 --- a/FoundationDB.Client/Shared/ThrowHelper.cs +++ b/FoundationDB.Client/Shared/ThrowHelper.cs @@ -1,5 +1,29 @@ -#region Copyright (c) 2013-2016, Doxense SAS. All rights reserved. -// See License.MD for license information +#region BSD Licence +/* Copyright (c) 2013-2018, 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 namespace Doxense.Diagnostics.Contracts diff --git a/FoundationDB.Client/Shared/Tuples/STuple.cs b/FoundationDB.Client/Shared/Tuples/STuple.cs index a4597dccf..555d30d37 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple.cs @@ -37,6 +37,7 @@ namespace Doxense.Collections.Tuples using System.Text; using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; + using Doxense.Serialization; using JetBrains.Annotations; /// Factory class for Tuples diff --git a/FoundationDB.Client/Shared/Uuid64.cs b/FoundationDB.Client/Shared/Uuid64.cs index 526b28a72..04c1e0a40 100644 --- a/FoundationDB.Client/Shared/Uuid64.cs +++ b/FoundationDB.Client/Shared/Uuid64.cs @@ -36,7 +36,6 @@ namespace System using System.Diagnostics; using System.Globalization; using System.Runtime.CompilerServices; - using Doxense; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using JetBrains.Annotations; From 7d91acc470d06dc0fff5618c6c88be0541a09592 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 16:50:51 +0200 Subject: [PATCH 127/153] Decouple STuple<..> from TuPack by removing ITupleSerializable support --- .../Shared/Tuples/JoinedTuple.cs | 14 +------------ .../Shared/Tuples/LinkedTuple.cs | 15 +------------- .../Shared/Tuples/ListTuple.cs | 20 +------------------ FoundationDB.Client/Shared/Tuples/STuple.cs | 12 +---------- FoundationDB.Client/Shared/Tuples/STuple`1.cs | 13 +----------- FoundationDB.Client/Shared/Tuples/STuple`2.cs | 13 +----------- FoundationDB.Client/Shared/Tuples/STuple`3.cs | 13 +----------- FoundationDB.Client/Shared/Tuples/STuple`4.cs | 13 +----------- FoundationDB.Client/Shared/Tuples/STuple`5.cs | 13 +----------- FoundationDB.Client/Shared/Tuples/STuple`6.cs | 13 +----------- .../Tuples/Encoding/TupleEncoder.cs | 12 +++++------ 11 files changed, 16 insertions(+), 135 deletions(-) diff --git a/FoundationDB.Client/Shared/Tuples/JoinedTuple.cs b/FoundationDB.Client/Shared/Tuples/JoinedTuple.cs index 27f64f4b0..7e4b42f2a 100644 --- a/FoundationDB.Client/Shared/Tuples/JoinedTuple.cs +++ b/FoundationDB.Client/Shared/Tuples/JoinedTuple.cs @@ -39,7 +39,7 @@ namespace Doxense.Collections.Tuples /// Tuple that represents the concatenation of two tuples [DebuggerDisplay("{ToString(),nq}")] - public sealed class JoinedTuple : ITuple, ITupleSerializable + public sealed class JoinedTuple : ITuple { // Uses cases: joining a 'subspace' tuple (customerId, 'Users', ) with a 'key' tuple (userId, 'Contacts', 123, ) @@ -66,18 +66,6 @@ public JoinedTuple(ITuple head, ITuple tail) m_count = m_split + tail.Count; } - - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - internal void PackTo(ref TupleWriter writer) - { - TupleEncoder.WriteTo(ref writer, this.Head); - TupleEncoder.WriteTo(ref writer, this.Tail); - } - public override string ToString() { return STuple.Formatter.ToString(this); diff --git a/FoundationDB.Client/Shared/Tuples/LinkedTuple.cs b/FoundationDB.Client/Shared/Tuples/LinkedTuple.cs index b8aca3537..4909cbb32 100644 --- a/FoundationDB.Client/Shared/Tuples/LinkedTuple.cs +++ b/FoundationDB.Client/Shared/Tuples/LinkedTuple.cs @@ -38,7 +38,7 @@ namespace Doxense.Collections.Tuples /// Tuple that adds a value at the end of an already existing tuple /// Type of the last value of the tuple [DebuggerDisplay("{ToString(),nq}")] - public sealed class LinkedTuple : ITuple, ITupleSerializable + public sealed class LinkedTuple : ITuple { //TODO: consider changing this to a struct ? @@ -64,19 +64,6 @@ public LinkedTuple([NotNull] ITuple head, T tail) this.Depth = head.Count; } - /// Pack this tuple into a buffer - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - /// Pack this tuple into a buffer - internal void PackTo(ref TupleWriter writer) - { - TupleEncoder.WriteTo(ref writer, this.Head); - TuplePacker.SerializeTo(ref writer, this.Tail); - } - /// Returns the number of elements in this tuple public int Count => this.Depth + 1; diff --git a/FoundationDB.Client/Shared/Tuples/ListTuple.cs b/FoundationDB.Client/Shared/Tuples/ListTuple.cs index 781b6ff3f..102f68c65 100644 --- a/FoundationDB.Client/Shared/Tuples/ListTuple.cs +++ b/FoundationDB.Client/Shared/Tuples/ListTuple.cs @@ -38,7 +38,7 @@ namespace Doxense.Collections.Tuples using JetBrains.Annotations; /// Tuple that can hold any number of untyped items - public sealed class ListTuple : ITuple, ITupleSerializable + public sealed class ListTuple : ITuple { // We could use a ListTuple for tuples where all items are of type T, and ListTuple could derive from ListTuple. // => this could speed up a bit the use case of STuple.FromArray or STuple.FromSequence @@ -225,24 +225,6 @@ private static IEnumerator Enumerate(object[] items, int offset, int cou } } - - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - internal void PackTo(ref TupleWriter writer) - { - //REVIEW: this is VERY slow! - int count = m_count; - var items = m_items; - int offset = m_offset; - for (int i = 0; i < count; i++) - { - TuplePackers.SerializeObjectTo(ref writer, items[i + offset]); - } - } - public override string ToString() { return STuple.Formatter.ToString(m_items, m_offset, m_count); diff --git a/FoundationDB.Client/Shared/Tuples/STuple.cs b/FoundationDB.Client/Shared/Tuples/STuple.cs index 555d30d37..b1ededa29 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple.cs @@ -42,7 +42,7 @@ namespace Doxense.Collections.Tuples /// Factory class for Tuples [PublicAPI] - public readonly struct STuple : ITuple, ITupleSerializable + public readonly struct STuple : ITuple { //note: We cannot use 'Tuple' because it's already used by the BCL in the System namespace, and we cannot use 'Tuples' either because it is part of the namespace... @@ -74,16 +74,6 @@ public ITuple Concat(ITuple tuple) return tuple; } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - internal void PackTo(ref TupleWriter writer) - { - //NO-OP - } - public void CopyTo(object[] array, int offset) { //NO-OP diff --git a/FoundationDB.Client/Shared/Tuples/STuple`1.cs b/FoundationDB.Client/Shared/Tuples/STuple`1.cs index 53a1f3e56..97baee083 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`1.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`1.cs @@ -42,7 +42,7 @@ namespace Doxense.Collections.Tuples /// Tuple that holds only one item /// Type of the item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable> + public readonly struct STuple : ITuple, IEquatable>, IEquatable> { // This is mostly used by code that create a lot of temporary singleton, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -82,17 +82,6 @@ public TItem Get(int index) return TypeConverters.Convert(this.Item1); } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T2 value) { return new STuple(this.Item1, value); diff --git a/FoundationDB.Client/Shared/Tuples/STuple`2.cs b/FoundationDB.Client/Shared/Tuples/STuple`2.cs index 0c48ed299..c8ce9e20d 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`2.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`2.cs @@ -43,7 +43,7 @@ namespace Doxense.Collections.Tuples /// Type of the first item /// Type of the second item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2)> + public readonly struct STuple : ITuple, IEquatable>, IEquatable<(T1, T2)> { // This is mostly used by code that create a lot of temporary pair, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -111,17 +111,6 @@ public STuple Tail get { return new STuple(this.Item2); } } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T3 value) { return new STuple(this.Item1, this.Item2, value); diff --git a/FoundationDB.Client/Shared/Tuples/STuple`3.cs b/FoundationDB.Client/Shared/Tuples/STuple`3.cs index ee15aad06..29b420018 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`3.cs @@ -44,7 +44,7 @@ namespace Doxense.Collections.Tuples /// Type of the second item /// Type of the third item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3)> + public readonly struct STuple : ITuple, IEquatable>, IEquatable<(T1, T2, T3)> { // This is mostly used by code that create a lot of temporary triplet, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -117,17 +117,6 @@ public STuple Tail get { return new STuple(this.Item2, this.Item3); } } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T4 value) { // here, the caller doesn't care about the exact tuple type, so we simply return a boxed List Tuple. diff --git a/FoundationDB.Client/Shared/Tuples/STuple`4.cs b/FoundationDB.Client/Shared/Tuples/STuple`4.cs index e6bbb3fbb..31ec0fac8 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`4.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`4.cs @@ -45,7 +45,7 @@ namespace Doxense.Collections.Tuples /// Type of the third item /// Type of the fourth item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4)> + public readonly struct STuple : ITuple, IEquatable>, IEquatable<(T1, T2, T3, T4)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -126,17 +126,6 @@ public STuple Tail get { return new STuple(this.Item2, this.Item3, this.Item4); } } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T5 value) { // the caller doesn't care about the return type, so just box everything into a list tuple diff --git a/FoundationDB.Client/Shared/Tuples/STuple`5.cs b/FoundationDB.Client/Shared/Tuples/STuple`5.cs index 773d1c862..eb5ee695c 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`5.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`5.cs @@ -46,7 +46,7 @@ namespace Doxense.Collections.Tuples /// Type of the 4th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4, T5)> + public readonly struct STuple : ITuple, IEquatable>, IEquatable<(T1, T2, T3, T4, T5)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -132,17 +132,6 @@ public STuple Tail get { return new STuple(this.Item2, this.Item3, this.Item4, this.Item5); } } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T6 value) { // the caller doesn't care about the return type, so just box everything into a list tuple diff --git a/FoundationDB.Client/Shared/Tuples/STuple`6.cs b/FoundationDB.Client/Shared/Tuples/STuple`6.cs index 7e10645c8..2403ea2c2 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`6.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`6.cs @@ -47,7 +47,7 @@ namespace Doxense.Collections.Tuples /// Type of the 5th item /// Type of the 5th item [ImmutableObject(true), DebuggerDisplay("{ToString(),nq}")] - public readonly struct STuple : ITuple, ITupleSerializable, IEquatable>, IEquatable<(T1, T2, T3, T4, T5, T6)> + public readonly struct STuple : ITuple, IEquatable>, IEquatable<(T1, T2, T3, T4, T5, T6)> { // This is mostly used by code that create a lot of temporary quartets, to reduce the pressure on the Garbage Collector by allocating them on the stack. // Please note that if you return an STuple as an ITuple, it will be boxed by the CLR and all memory gains will be lost @@ -138,17 +138,6 @@ public STuple Tail get { return new STuple(this.Item2, this.Item3, this.Item4, this.Item5, this.Item6); } } - void ITupleSerializable.PackTo(ref TupleWriter writer) - { - PackTo(ref writer); - } - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal void PackTo(ref TupleWriter writer) - { - TupleSerializer.Default.PackTo(ref writer, in this); - } - ITuple ITuple.Append(T7 value) { // the caller doesn't care about the return type, so just box everything into a list tuple diff --git a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs index 237bae37f..07e80e6fd 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs @@ -462,7 +462,7 @@ public static Slice Pack(Slice prefix, ref STuple tuple) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); + TupleSerializer.Default.PackTo(ref writer, tuple); return writer.ToSlice(); } @@ -471,7 +471,7 @@ public static Slice Pack(Slice prefix, ref STuple tuple) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); + TupleSerializer.Default.PackTo(ref writer, tuple); return writer.ToSlice(); } @@ -480,7 +480,7 @@ public static Slice Pack(Slice prefix, ref STuple tuple) { var writer = new TupleWriter(); writer.Output.WriteBytes(prefix); - tuple.PackTo(ref writer); + TupleSerializer.Default.PackTo(ref writer, tuple); return writer.ToSlice(); } @@ -489,7 +489,7 @@ public static Slice Pack(Slice prefix, ref STuple.Default.PackTo(ref writer, tuple); return writer.ToSlice(); } @@ -498,7 +498,7 @@ public static Slice Pack(Slice prefix, ref STuple.Default.PackTo(ref writer, tuple); return writer.ToSlice(); } @@ -507,7 +507,7 @@ public static Slice Pack(Slice prefix, ref STuple.Default.PackTo(ref writer, tuple); return writer.Output.ToSlice(); } From 795b65e50680e2b3e2623f9962a7cad83e59a45f Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 16:51:26 +0200 Subject: [PATCH 128/153] Re-optimize serialization of STuple<..> and ValueTuple<..> by having a cache of boxed serializers --- .../Tuples/Encoding/TuplePacker.cs | 5 + .../Tuples/Encoding/TuplePackers.cs | 284 +++++++++--------- 2 files changed, 149 insertions(+), 140 deletions(-) diff --git a/FoundationDB.Client/Tuples/Encoding/TuplePacker.cs b/FoundationDB.Client/Tuples/Encoding/TuplePacker.cs index ffec57bfc..abf20d6e7 100644 --- a/FoundationDB.Client/Tuples/Encoding/TuplePacker.cs +++ b/FoundationDB.Client/Tuples/Encoding/TuplePacker.cs @@ -57,6 +57,11 @@ public static void SerializeTo(ref TupleWriter writer, T value) Encoder(ref writer, value); } + public static void SerializeBoxedTo(ref TupleWriter writer, object value) + { + Encoder(ref writer, (T) value); + } + /// Serialize a into a binary buffer /// Target buffer /// Value that will be serialized diff --git a/FoundationDB.Client/Tuples/Encoding/TuplePackers.cs b/FoundationDB.Client/Tuples/Encoding/TuplePackers.cs index 68f28989c..0664d1edc 100644 --- a/FoundationDB.Client/Tuples/Encoding/TuplePackers.cs +++ b/FoundationDB.Client/Tuples/Encoding/TuplePackers.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Collections.Tuples.Encoding { using System; + using System.Collections.Concurrent; using System.Collections.Generic; using System.Globalization; using System.Linq; @@ -76,8 +77,7 @@ private static Delegate GetSerializerFor([NotNull] Type type) } // look for well-known types that have their own (non-generic) TuplePackers.SerializeTo(...) method - var typeArgs = new[] { typeof(TupleWriter).MakeByRefType(), type }; - var method = typeof(TuplePackers).GetMethod(nameof(SerializeTo), BindingFlags.Static | BindingFlags.Public, binder: null, types: typeArgs, modifiers: null); + var method = typeof(TuplePackers).GetMethod(nameof(SerializeTo), BindingFlags.Static | BindingFlags.Public, binder: null, types: new[] { typeof(TupleWriter).MakeByRefType(), type }, modifiers: null); if (method != null) { // we have a direct serializer return method.CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); @@ -97,7 +97,17 @@ private static Delegate GetSerializerFor([NotNull] Type type) // maybe it is a tuple ? if (typeof(ITuple).IsAssignableFrom(type)) { - // If so, try to use the corresponding TuplePackers.SerializeTupleTo(...) method + if (type == typeof(STuple) || (type.Name.StartsWith(nameof(STuple) + "`", StringComparison.Ordinal) && type.Namespace == typeof(STuple).Namespace)) + { // well-known STuple struct + var typeArgs = type.GetGenericArguments(); + method = FindSTupleSerializerMethod(typeArgs); + if (method != null) + { + return method.MakeGenericMethod(typeArgs).CreateDelegate(typeof(Encoder<>).MakeGenericType(type)); + } + } + + // will use the default ITuple implementation method = typeof(TuplePackers).GetMethod(nameof(SerializeTupleTo), BindingFlags.Static | BindingFlags.Public); if (method != null) { @@ -116,9 +126,10 @@ private static Delegate GetSerializerFor([NotNull] Type type) } } - if ((type.Name == nameof(System.ValueTuple) || type.Name.StartsWith(nameof(System.ValueTuple) + "`", StringComparison.Ordinal)) && type.Namespace == "System") + // ValueTuple + if (type == typeof(ValueTuple) || (type.Name.StartsWith(nameof(System.ValueTuple) + "`", StringComparison.Ordinal) && type.Namespace == "System")) { - typeArgs = type.GetGenericArguments(); + var typeArgs = type.GetGenericArguments(); method = FindValueTupleSerializerMethod(typeArgs); if (method != null) { @@ -132,6 +143,15 @@ private static Delegate GetSerializerFor([NotNull] Type type) return null; } + private static MethodInfo FindSTupleSerializerMethod(Type[] args) + { + //note: we want to find the correct SerializeSTuple<...>(ref TupleWriter, (...,), but this cannot be done with Type.GetMethod(...) directly + // => we have to scan for all methods with the correct name, and the same number of Type Arguments than the ValueTuple. + return typeof(TuplePackers) + .GetMethods(BindingFlags.Static | BindingFlags.Public) + .SingleOrDefault(m => m.Name == nameof(SerializeSTupleTo) && m.GetGenericArguments().Length == args.Length); + } + private static MethodInfo FindValueTupleSerializerMethod(Type[] args) { //note: we want to find the correct SerializeValueTuple<...>(ref TupleWriter, (...,), but this cannot be done with Type.GetMethod(...) directly @@ -220,149 +240,65 @@ public static void SerializeObjectTo(ref TupleWriter writer, object value) TupleParser.WriteNil(ref writer); return; } + GetBoxedEncoder(value.GetType())(ref writer, value); + } - switch (Type.GetTypeCode(value.GetType())) + private static Encoder GetBoxedEncoder(Type type) + { + if (!BoxedEncoders.TryGetValue(type, out var encoder)) { - case TypeCode.Empty: - case TypeCode.Object: - { - if (value is byte[] bytes) - { - SerializeTo(ref writer, bytes); - return; - } - - if (value is Slice slice) - { - SerializeTo(ref writer, slice); - return; - } - - if (value is Guid g) - { - SerializeTo(ref writer, g); - return; - } + encoder = CreateBoxedEncoder(type); + BoxedEncoders.TryAdd(type, encoder); + } + return encoder; + } - if (value is Uuid128 u128) - { - SerializeTo(ref writer, u128); - return; - } + private static ConcurrentDictionary> BoxedEncoders { get; } = GetDefaultBoxedEncoders(); - if (value is Uuid64 u64) - { - SerializeTo(ref writer, u64); - return; - } - - if (value is TimeSpan ts) - { - SerializeTo(ref writer, ts); - return; - } + private static ConcurrentDictionary> GetDefaultBoxedEncoders() + { + var encoders = new ConcurrentDictionary> + { + [typeof(bool)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (bool) value), + [typeof(char)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (char) value), + [typeof(string)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (string) value), + [typeof(sbyte)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (sbyte) value), + [typeof(short)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (short) value), + [typeof(int)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (int) value), + [typeof(long)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (long) value), + [typeof(byte)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (byte) value), + [typeof(ushort)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (ushort) value), + [typeof(uint)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (uint) value), + [typeof(ulong)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (ulong) value), + [typeof(float)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (float) value), + [typeof(double)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (double) value), + [typeof(decimal)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (decimal) value), + [typeof(Slice)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (Slice) value), + [typeof(byte[])] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (byte[]) value), + [typeof(Guid)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (Guid) value), + [typeof(Uuid128)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (Uuid128) value), + [typeof(Uuid64)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (Uuid64) value), + [typeof(TimeSpan)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (TimeSpan) value), + [typeof(DateTime)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (DateTime) value), + [typeof(DateTimeOffset)] = (ref TupleWriter writer, object value) => SerializeTo(ref writer, (DateTimeOffset) value), + [typeof(ITuple)] = (ref TupleWriter writer, object value) => SerializeTupleTo(ref writer, (ITuple) value), + [typeof(ITupleFormattable)] = (ref TupleWriter writer, object value) => SerializeTupleTo(ref writer, (ITuple) value), + [typeof(DBNull)] = (ref TupleWriter writer, object value) => TupleParser.WriteNil(ref writer) + }; - break; - } - case TypeCode.DBNull: - { // same as null - TupleParser.WriteNil(ref writer); - return; - } - case TypeCode.Boolean: - { - SerializeTo(ref writer, (bool)value); - return; - } - case TypeCode.Char: - { - // should be treated as a string with only one char - SerializeTo(ref writer, (char)value); - return; - } - case TypeCode.SByte: - { - SerializeTo(ref writer, (sbyte)value); - return; - } - case TypeCode.Byte: - { - SerializeTo(ref writer, (byte)value); - return; - } - case TypeCode.Int16: - { - SerializeTo(ref writer, (short)value); - return; - } - case TypeCode.UInt16: - { - SerializeTo(ref writer, (ushort)value); - return; - } - case TypeCode.Int32: - { - SerializeTo(ref writer, (int)value); - return; - } - case TypeCode.UInt32: - { - SerializeTo(ref writer, (uint)value); - return; - } - case TypeCode.Int64: - { - SerializeTo(ref writer, (long)value); - return; - } - case TypeCode.UInt64: - { - SerializeTo(ref writer, (ulong)value); - return; - } - case TypeCode.String: - { - SerializeTo(ref writer, value as string); - return; - } - case TypeCode.DateTime: - { - SerializeTo(ref writer, (DateTime)value); - return; - } - case TypeCode.Double: - { - SerializeTo(ref writer, (double)value); - return; - } - case TypeCode.Single: - { - SerializeTo(ref writer, (float)value); - return; - } - case TypeCode.Decimal: - { - SerializeTo(ref writer, (decimal)value); - return; - } - } + return encoders; + } - if (value is ITuple tuple) - { - SerializeTupleTo(ref writer, tuple); - return; - } + private static Encoder CreateBoxedEncoder(Type type) + { + var m = typeof(TuplePacker<>).MakeGenericType(type).GetMethod(nameof(TuplePacker.SerializeBoxedTo)); + Contract.Assert(m != null); - if (value is ITupleFormattable fmt) - { - tuple = fmt.ToTuple(); - if (tuple == null) throw new InvalidOperationException($"An instance of type '{value.GetType().Name}' returned a null Tuple while serialiazing"); - SerializeTupleTo(ref writer, tuple); - return; - } + var writer = Expression.Parameter(typeof(TupleWriter).MakeByRefType(), "writer"); + var value = Expression.Parameter(typeof(object), "value"); - // Not Supported ? - throw new NotSupportedException($"Doesn't know how to serialize objects of type '{value.GetType().Name}' into Tuple Encoding format"); + var body = Expression.Call(m, writer, value); + return Expression.Lambda>(body, writer, value).Compile(); } /// Writes a slice as a byte[] array @@ -570,6 +506,74 @@ public static void SerializeTupleTo(ref TupleWriter writer, TTuple tuple TupleParser.EndTuple(ref writer); } + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + SerializeTo(ref writer, tuple.Item5); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeSTupleTo(ref TupleWriter writer, STuple tuple) + { + TupleParser.BeginTuple(ref writer); + SerializeTo(ref writer, tuple.Item1); + SerializeTo(ref writer, tuple.Item2); + SerializeTo(ref writer, tuple.Item3); + SerializeTo(ref writer, tuple.Item4); + SerializeTo(ref writer, tuple.Item5); + SerializeTo(ref writer, tuple.Item6); + TupleParser.EndTuple(ref writer); + } + + public static void SerializeTupleFormattableTo(ref TupleWriter writer, TFormattable formattable) + where TFormattable : ITupleFormattable + { + var tuple = formattable.ToTuple(); + if (tuple == null) throw new InvalidOperationException($"An instance of type '{formattable.GetType().Name}' returned a null Tuple while serialiazing"); + + TupleParser.BeginTuple(ref writer); + TupleEncoder.WriteTo(ref writer, tuple); + TupleParser.EndTuple(ref writer); + } + /// Serialize an embedded tuple formattable public static void SerializeFormattableTo(ref TupleWriter writer, ITupleFormattable formattable) { From d4ed6f1ec015e20bcf84dcb2cfa9ce215940c4d8 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 16:51:46 +0200 Subject: [PATCH 129/153] Fix VersionStamp tests --- FoundationDB.Tests/Utils/TuPackFacts.cs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/FoundationDB.Tests/Utils/TuPackFacts.cs b/FoundationDB.Tests/Utils/TuPackFacts.cs index 664b40b47..a1e259390 100644 --- a/FoundationDB.Tests/Utils/TuPackFacts.cs +++ b/FoundationDB.Tests/Utils/TuPackFacts.cs @@ -706,7 +706,7 @@ public void Test_TuplePack_Serialize_VersionStamps() // incomplete, 80 bits Assert.That( TuPack.EncodeKey(VersionStamp.Incomplete()).ToHexaString(' '), - Is.EqualTo("33 FF FF FF FF FF FF FF FF FF FF") + Is.EqualTo("32 FF FF FF FF FF FF FF FF FF FF") ); // incomplete, 96 bits @@ -730,7 +730,7 @@ public void Test_TuplePack_Serialize_VersionStamps() // complete, 80 bits Assert.That( TuPack.EncodeKey(VersionStamp.Complete(0x0123456789ABCDEF, 1234)).ToHexaString(' '), - Is.EqualTo("33 01 23 45 67 89 AB CD EF 04 D2") + Is.EqualTo("32 01 23 45 67 89 AB CD EF 04 D2") ); // complete, 96 bits @@ -757,7 +757,7 @@ public void Test_TuplePack_Deserailize_VersionStamps() { Assert.That(TuPack.DecodeKey(Slice.FromHexa("32 FF FF FF FF FF FF FF FF FF FF")), Is.EqualTo(VersionStamp.Incomplete()), "Incomplete()"); - Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 00 00")), Is.EqualTo(VersionStamp.Incomplete()), "Incomplete(0)"); + Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 00 00")), Is.EqualTo(VersionStamp.Incomplete(0)), "Incomplete(0)"); Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 00 2A")), Is.EqualTo(VersionStamp.Incomplete(42)), "Incomplete(42)"); Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF 01 C8")), Is.EqualTo(VersionStamp.Incomplete(456)), "Incomplete(456)"); Assert.That(TuPack.DecodeKey(Slice.FromHexa("33 FF FF FF FF FF FF FF FF FF FF FF FF")), Is.EqualTo(VersionStamp.Incomplete(65535)), "Incomplete(65535)"); From d34ee9d6e4de5a15d6ff410c8420146cef279e59 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 16:52:02 +0200 Subject: [PATCH 130/153] Re-enable tuple serialization tests for ValueTuple --- FoundationDB.Tests/Utils/TuPackFacts.cs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/FoundationDB.Tests/Utils/TuPackFacts.cs b/FoundationDB.Tests/Utils/TuPackFacts.cs index a1e259390..8d6934975 100644 --- a/FoundationDB.Tests/Utils/TuPackFacts.cs +++ b/FoundationDB.Tests/Utils/TuPackFacts.cs @@ -26,8 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -//#define ENABLE_VALUETUPLE - // ReSharper disable AccessToModifiedClosure namespace Doxense.Collections.Tuples.Tests { @@ -2157,8 +2155,6 @@ void ITupleFormattable.FromTuple(ITuple tuple) } } -#if ENABLE_VALUETUPLES - [Test] public void Test_TuPack_ValueTuple_Pack() { @@ -2202,8 +2198,6 @@ public void Test_TuPack_ValueTuple_Pack() } -#endif - } From 540a6ff6383f42c2777a21c186c02dfb5ca5fd40 Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Fri, 27 Apr 2018 20:15:45 +0500 Subject: [PATCH 131/153] disable signing FoundationDB.Client.csproj on non-win problem: dotnet core doesn't support assembly sining outside of windows solution was taken from: https://github.com/ppekrol/ravendb/commit/20931dd79533986670c034100fd5201fbc6a189e --- FoundationDB.Client/FoundationDB.Client.csproj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/FoundationDB.Client/FoundationDB.Client.csproj b/FoundationDB.Client/FoundationDB.Client.csproj index 3c9e4e746..7512b8084 100644 --- a/FoundationDB.Client/FoundationDB.Client.csproj +++ b/FoundationDB.Client/FoundationDB.Client.csproj @@ -4,7 +4,7 @@ netstandard2.0 FoundationDB.Client FoundationDB.Client - true + true ..\Common\foundationdb-net-client.snk 5.1.0-alpha1 Doxense From 8d4479b7ef083284832ef88947b99473ae52dccd Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 17:59:09 +0200 Subject: [PATCH 132/153] Use a better source of randomness for the VersionStamp random tokens - Use Guid.NewGuid() to generate random bits --- FoundationDB.Client/FdbTransaction.cs | 9 +++++---- FoundationDB.Tests/TransactionFacts.cs | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 2d4ea3970..673bbbe16 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -306,17 +306,18 @@ private ulong GenerateNewVersionStampToken() // Since this is supposed to be a version number with a ~1M tickrate per seconds, we will play it safe, and force the 8 highest bits to 1, // meaning that we only reduce the database potential lifetime but 1/256th, before getting into trouble. // - // By doing some empirical testing, it also seems that the last 16 bits are a transction batch order which is usually a low number. + // By doing some empirical testing, it also seems that the last 16 bits are a transaction batch order which is usually a low number. // Again, we will force the 4 highest bit to 1 to reduce the change of collision with a complete version stamp. // // So the final token will look like: 'FF xx xx xx xx xx xx xx Fy yy', were 'x' is the random token, and 'y' will lowest 12 bits of the transaction retry count - var rnd = new Random(); //TODO: singleton? (need locking!!) ulong x; unsafe { - double r = rnd.NextDouble(); - x = *(ulong*) &r; + // use a 128-bit guid as the source of entropy for our new token + Guid rnd = Guid.NewGuid(); + ulong* p = (ulong*) &rnd; + x = p[0] ^ p[1]; } x |= 0xFF00000000000000UL; diff --git a/FoundationDB.Tests/TransactionFacts.cs b/FoundationDB.Tests/TransactionFacts.cs index a4c1f5f15..cc97fc493 100644 --- a/FoundationDB.Tests/TransactionFacts.cs +++ b/FoundationDB.Tests/TransactionFacts.cs @@ -2000,7 +2000,7 @@ public async Task Test_VersionStamps_Share_The_Same_Token_Per_Transaction_Attemp { // should return a 80-bit incomplete stamp, using a random token var x = tr.CreateVersionStamp(); - Log($"> x : {x} with token '{x.ToSlice():X}'"); + Log($"> x : {x.ToSlice():X} => {x}"); Assert.That(x.IsIncomplete, Is.True, "Placeholder token should be incomplete"); Assert.That(x.HasUserVersion, Is.False); Assert.That(x.UserVersion, Is.Zero); From 8c831bd27c001af1f641f8afbcb460abca282d9f Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 18:01:31 +0200 Subject: [PATCH 133/153] Use TuPack.Encoding directly, instead of using TypeSystem.Tuples or Default --- FoundationDB.Client/FdbDatabase.cs | 5 +++-- .../Layers/Directories/FdbDirectoryLayer.cs | 12 ++++++------ .../Layers/Directories/FdbDirectoryPartition.cs | 5 +++-- .../Layers/Directories/FdbDirectorySubspace.cs | 4 ++-- .../TypeSystem/Encoders/KeyValueEncoders.Tuples.cs | 12 ++++++------ FoundationDB.Client/Subspaces/KeySubspace.cs | 13 ++++++------- .../Tuples/Encoding/TupleKeyEncoder.cs | 2 +- .../Tuples/Encoding/TupleKeyEncoding.cs | 3 +++ FoundationDB.Client/Tuples/TuPack.cs | 5 +++++ 9 files changed, 35 insertions(+), 26 deletions(-) diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 418376da4..70706bc71 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using Doxense.Async; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using Doxense.Serialization.Encoders; @@ -456,8 +457,8 @@ internal void ChangeRoot(IKeySubspace subspace, IFdbDirectory directory, bool re lock (this)//TODO: don't use this for locking { m_readOnly = readOnly; - m_globalSpace = KeySubspace.Copy(subspace, TypeSystem.Tuples); - m_globalSpaceCopy = KeySubspace.Copy(subspace, TypeSystem.Tuples); // keep another copy + m_globalSpace = subspace.Copy(TuPack.Encoding); + m_globalSpaceCopy = subspace.Copy(TuPack.Encoding); // keep another copy m_directory = directory == null ? null : new FdbDatabasePartition(this, directory); } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs index 73a7c089a..2e9af0707 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryLayer.cs @@ -156,11 +156,11 @@ internal FdbDirectoryLayer(IDynamicKeySubspace nodeSubspace, IDynamicKeySubspace /// Create an instance of a Directory Layer located under a specific prefix and path /// Prefix for the content. The nodes will be stored under + <FE> /// Optional path, if the Directory Layer is not located at the root of the database. - /// Optional key encoding scheme. If not specified, will use the type system by default. + /// Optional key encoding scheme. If not specified, will use the encoding by default. [NotNull] public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = null, IKeyEncoding encoding = null) { - var subspace = KeySubspace.CreateDynamic(prefix, encoding ?? TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(prefix, encoding ?? TuPack.Encoding); var location = path != null ? ParsePath(path) : STuple.Empty; return new FdbDirectoryLayer(subspace.Partition[FdbKey.Directory], subspace, location); } @@ -168,14 +168,14 @@ public static FdbDirectoryLayer Create(Slice prefix, IEnumerable path = /// Create an instance of a Directory Layer located under a specific subspace and path /// Subspace for the content. The nodes will be stored under .Key + <FE> /// Optional path, if the Directory Layer is not located at the root of the database. - /// Optional key encoding scheme. If not specified, will use the type system by default. + /// Optional key encoding scheme. If not specified, will use the encoding by default. [NotNull] public static FdbDirectoryLayer Create(IKeySubspace subspace, IEnumerable path = null, IKeyEncoding encoding = null) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); var location = path != null ? ParsePath(path) : STuple.Empty; - var space = subspace.Using(encoding ?? TypeSystem.Tuples); + var space = subspace.AsDynamic(encoding ?? TuPack.Encoding); return new FdbDirectoryLayer(space.Partition[FdbKey.Directory], space, location); } @@ -932,11 +932,11 @@ private FdbDirectorySubspace ContentsOfNode([NotNull] IKeySubspace node, [NotNul var prefix = this.NodeSubspace.Keys.Decode(node.GetPrefix()); if (layer == FdbDirectoryPartition.LayerId) { - return new FdbDirectoryPartition(path, relativePath, prefix, this); + return new FdbDirectoryPartition(path, relativePath, prefix, this, TuPack.Encoding); } else { - return new FdbDirectorySubspace(path, relativePath, prefix, this, layer, TypeSystem.Default); + return new FdbDirectorySubspace(path, relativePath, prefix, this, layer, TuPack.Encoding); } } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs index 22d199562..d183be738 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectoryPartition.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Directories using Doxense.Collections.Tuples; using Doxense.Serialization.Encoders; using FoundationDB.Client; + using JetBrains.Annotations; public class FdbDirectoryPartition : FdbDirectorySubspace { @@ -39,8 +40,8 @@ public class FdbDirectoryPartition : FdbDirectorySubspace /// Returns a slice with the ASCII string "partition" public static Slice LayerId => Slice.FromString("partition"); - internal FdbDirectoryPartition(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer) - : base(location, relativeLocation, prefix, new FdbDirectoryLayer(FromKey(prefix + FdbKey.Directory).Using(TypeSystem.Default), FromKey(prefix).Using(TypeSystem.Default), location), LayerId, TypeSystem.Default) + internal FdbDirectoryPartition([NotNull] ITuple location, [NotNull] ITuple relativeLocation, Slice prefix, [NotNull] FdbDirectoryLayer directoryLayer, [NotNull] IKeyEncoding keyEncoding) + : base(location, relativeLocation, prefix, new FdbDirectoryLayer(FromKey(prefix + FdbKey.Directory).AsDynamic(keyEncoding), FromKey(prefix).AsDynamic(keyEncoding), location), LayerId, keyEncoding) { this.ParentDirectoryLayer = directoryLayer; } diff --git a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs index a9dcd1bc2..045abe205 100644 --- a/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs +++ b/FoundationDB.Client/Layers/Directories/FdbDirectorySubspace.cs @@ -44,7 +44,7 @@ namespace FoundationDB.Layers.Directories public class FdbDirectorySubspace : DynamicKeySubspace, IFdbDirectory { - internal FdbDirectorySubspace(ITuple location, ITuple relativeLocation, Slice prefix, FdbDirectoryLayer directoryLayer, Slice layer, IKeyEncoding encoding) + internal FdbDirectorySubspace([NotNull] ITuple location, [NotNull] ITuple relativeLocation, Slice prefix, [NotNull] FdbDirectoryLayer directoryLayer, Slice layer, [NotNull] IKeyEncoding encoding) : base(prefix, encoding) { Contract.Requires(location != null && relativeLocation != null && prefix != null && directoryLayer != null); @@ -153,7 +153,7 @@ public async Task ChangeLayerAsync(IFdbTransaction trans, // set the layer to the new value await this.DirectoryLayer.ChangeLayerInternalAsync(trans, this.RelativeLocation, newLayer).ConfigureAwait(false); // and return the new version of the subspace - return new FdbDirectorySubspace(this.Location, this.RelativeLocation, GetKeyPrefix(), this.DirectoryLayer, newLayer, TypeSystem.Default); + return new FdbDirectorySubspace(this.Location, this.RelativeLocation, GetKeyPrefix(), this.DirectoryLayer, newLayer, this.Encoding); } /// Opens a subdirectory with the given . diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs index baaa3177f..c7abaa0a3 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -50,7 +50,7 @@ internal class TupleEncoder : IKeyEncoder, IValueEncoder private TupleEncoder() { } - public IKeyEncoding Encoding => TypeSystem.Tuples; + public IKeyEncoding Encoding => TuPack.Encoding; public void WriteKeyTo(ref SliceWriter writer, T key) { @@ -84,7 +84,7 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder private TupleCompositeEncoder() { } - public override IKeyEncoding Encoding => TypeSystem.Tuples; + public override IKeyEncoding Encoding => TuPack.Encoding; public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) key) { @@ -114,7 +114,7 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override IKeyEncoding Encoding => TuPack.Encoding; public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) key) { @@ -146,7 +146,7 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override IKeyEncoding Encoding => TuPack.Encoding; public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) key) { @@ -180,7 +180,7 @@ internal class TupleCompositeEncoder : CompositeKeyEncoder TypeSystem.Tuples; + public override IKeyEncoding Encoding => TuPack.Encoding; public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) key) { @@ -216,7 +216,7 @@ internal class TupleCompositeEncoder : CompositeKeyEncod private TupleCompositeEncoder() { } - public override IKeyEncoding Encoding => TypeSystem.Tuples; + public override IKeyEncoding Encoding => TuPack.Encoding; public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) key) { diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 6a86d6f58..e53047fd2 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -74,17 +74,16 @@ public static DynamicKeySubspace CreateDynamic(Slice prefix, [NotNull] IDynamicK /// Initializes a new subspace with the given binary , that uses a dynamic key . /// A subspace that can handle keys of any types and size. [Pure, NotNull] - public static DynamicKeySubspace CreateDynamic(Slice prefix, [NotNull] IKeyEncoding encoding) + public static DynamicKeySubspace CreateDynamic(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - Contract.NotNull(encoding, nameof(encoding)); - return new DynamicKeySubspace(prefix, encoding.GetDynamicEncoder()); + return new DynamicKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetDynamicEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle keys of type . public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -99,7 +98,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] IKeyE /// A subspace that can handle composite keys of type (, ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -114,7 +113,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, [NotNul /// A subspace that can handle composite keys of type (, , ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -129,7 +128,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, /// A subspace that can handle composite keys of type (, , ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . diff --git a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs index 36eb31a0b..04e3ee415 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs @@ -43,7 +43,7 @@ public sealed class TupleKeyEncoder : IDynamicKeyEncoder private TupleKeyEncoder() { } - public IKeyEncoding Encoding => TypeSystem.Tuples; + public IKeyEncoding Encoding => TuPack.Encoding; public void PackKey(ref SliceWriter writer, TTuple items) where TTuple : ITuple diff --git a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs index 5f1aa645c..c210853df 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs @@ -34,6 +34,9 @@ namespace Doxense.Collections.Tuples.Encoding /// Encoding that uses the Tuple Binary Encoding format public sealed class TupleKeyEncoding : IKeyEncoding { + + public static readonly TupleKeyEncoding Instance = new TupleKeyEncoding(); + public IDynamicKeyEncoder GetDynamicEncoder() { return TupleKeyEncoder.Instance; diff --git a/FoundationDB.Client/Tuples/TuPack.cs b/FoundationDB.Client/Tuples/TuPack.cs index bc8f6ebcc..df9ba6f96 100644 --- a/FoundationDB.Client/Tuples/TuPack.cs +++ b/FoundationDB.Client/Tuples/TuPack.cs @@ -34,6 +34,7 @@ namespace Doxense.Collections.Tuples using Doxense.Diagnostics.Contracts; using Doxense.Collections.Tuples.Encoding; using Doxense.Memory; + using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -42,6 +43,10 @@ namespace Doxense.Collections.Tuples public static class TuPack { + /// Key Encoding that use the Tuple Binary Encoding + [NotNull] + public static IKeyEncoding Encoding => TupleKeyEncoding.Instance; + #region Packing... // Without prefix From 6b73213141fde6c44489a23d9e389dd6f3c7e609 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 18:05:22 +0200 Subject: [PATCH 134/153] [BREAKING CHANGE] Remove TypeSystem completely, and add AsDynamic(..), AsTyped(..) extension methods - Version of Using(..) and UsingEncoder(...) that took an IKeyEncoding are renamed to AsDynamic() or AsTyped() - The encoding is optional, and will use TuPack.Encoding by default - Changed the KeySubspace.Copy(...) into extension methods --- FdbShell/Commands/BasicCommands.cs | 12 +- .../Subspaces/Fdb.Directory.cs | 2 +- FoundationDB.Client/Subspaces/KeySubspace.cs | 91 +--------- .../Subspaces/KeySubspaceExtensions.cs | 157 ++++++++++++++---- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 8 +- .../Collections/FdbMultimap`2.cs | 6 +- .../Collections/FdbQueue`1.cs | 2 +- .../Collections/FdbRankedSet.cs | 3 +- .../Collections/FdbVector`1.cs | 2 +- .../Counters/FdbHighContentionCounter.cs | 2 +- .../Interning/FdbStringIntern.cs | 3 +- .../Documents/FdbHashSetCollection.cs | 3 +- .../Messaging/FdbWorkerPool.cs | 3 +- FoundationDB.Tests/Layers/DirectoryFacts.cs | 6 +- FoundationDB.Tests/Layers/MultiMapFacts.cs | 4 +- FoundationDB.Tests/RangeQueryFacts.cs | 4 +- FoundationDB.Tests/SubspaceFacts.cs | 31 ++-- 17 files changed, 171 insertions(+), 168 deletions(-) diff --git a/FdbShell/Commands/BasicCommands.cs b/FdbShell/Commands/BasicCommands.cs index 1243225f4..4a340e7d1 100644 --- a/FdbShell/Commands/BasicCommands.cs +++ b/FdbShell/Commands/BasicCommands.cs @@ -70,16 +70,16 @@ public static async Task Dir(string[] path, ITuple extras, DirectoryBrowseOption if (!(subfolder is FdbDirectoryPartition)) { long count = await Fdb.System.EstimateCountAsync(db, subfolder.Keys.ToRange(), ct); - log.WriteLine(" {0,-12} {1,-12} {3,9:N0} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, count); + log.WriteLine(" {0,-12} {1,-12} {3,9:N0} {2}", FdbKey.Dump(subfolder.Copy().GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, count); } else { - log.WriteLine(" {0,-12} {1,-12} {3,9} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, "-"); + log.WriteLine(" {0,-12} {1,-12} {3,9} {2}", FdbKey.Dump(subfolder.Copy().GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name, "-"); } } else { - log.WriteLine(" {0,-12} {1,-12} {2}", FdbKey.Dump(KeySubspace.Copy(subfolder).GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name); + log.WriteLine(" {0,-12} {1,-12} {2}", FdbKey.Dump(subfolder.Copy().GetPrefix()), subfolder.Layer.IsNullOrEmpty ? "-" : ("<" + subfolder.Layer.ToUnicode() + ">"), name); } } else @@ -217,7 +217,7 @@ public static async Task Count(string[] path, ITuple extras, IFdbDatabase db, Te return; } - var copy = KeySubspace.Copy(folder); + var copy = folder.Copy(); log.WriteLine("# Counting keys under {0} ...", FdbKey.Dump(copy.GetPrefix())); var progress = new Progress<(long Count, Slice Current)>((state) => @@ -555,7 +555,7 @@ public static async Task Shards(string[] path, ITuple extras, IFdbDatabase db, T var folder = (await TryOpenCurrentDirectoryAsync(path, db, ct)) as FdbDirectorySubspace; if (folder != null) { - var r = KeyRange.StartsWith(KeySubspace.Copy(folder).GetPrefix()); + var r = KeyRange.StartsWith(folder.Copy().GetPrefix()); Console.WriteLine("Searching for shards that intersect with /{0} ...", String.Join("/", path)); ranges = await Fdb.System.GetChunksAsync(db, r, ct); Console.WriteLine("Found {0} ranges intersecting {1}:", ranges.Count, r); @@ -590,7 +590,7 @@ public static async Task Sampling(string[] path, ITuple extras, IFdbDatabase db, KeyRange span; if (folder is FdbDirectorySubspace) { - span = KeyRange.StartsWith(KeySubspace.Copy(folder as FdbDirectorySubspace).GetPrefix()); + span = KeyRange.StartsWith((folder as FdbDirectorySubspace).Copy().GetPrefix()); log.WriteLine("Reading list of shards for /{0} under {1} ...", String.Join("/", path), FdbKey.Dump(span.Begin)); } else diff --git a/FoundationDB.Client/Subspaces/Fdb.Directory.cs b/FoundationDB.Client/Subspaces/Fdb.Directory.cs index e396287ba..da99c542b 100644 --- a/FoundationDB.Client/Subspaces/Fdb.Directory.cs +++ b/FoundationDB.Client/Subspaces/Fdb.Directory.cs @@ -85,7 +85,7 @@ public static async Task OpenNamedPartitionAsync(string clusterFil if (Logging.On) Logging.Verbose(typeof(Fdb.Directory), "OpenNamedPartitionAsync", $"Found named partition '{descriptor.FullName}' at prefix {descriptor}"); // we have to chroot the database to the new prefix, and create a new DirectoryLayer with a new '/' - rootSpace = KeySubspace.Copy(descriptor); //note: create a copy of the key + rootSpace = descriptor.Copy(); //note: create a copy of the key //TODO: find a nicer way to do that! db.ChangeRoot(rootSpace, FdbDirectoryLayer.Create(rootSpace, partitionPath), readOnly); diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index e53047fd2..9618256e1 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -141,95 +141,6 @@ public static TypedKeySubspace CreateTyped(Slice #endregion - #region Copy... - - /// Create a new copy of a subspace's prefix - [Pure] - internal static Slice StealPrefix([NotNull] IKeySubspace subspace) - { - //note: we can workaround the 'security' in top directory partition by accessing their key prefix without triggering an exception! - return subspace is KeySubspace ks - ? ks.Key.Memoize() - : subspace.GetPrefix().Memoize(); - } - - /// Create a copy of a generic subspace, sharing the same binary prefix - [Pure, NotNull] - public static KeySubspace Copy([NotNull] IKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - - var prefix = StealPrefix(subspace); - - if (subspace is IDynamicKeySubspace dyn) - { // reuse the encoding of the original - return new DynamicKeySubspace(prefix, dyn.Encoding); - } - - // no encoding - return new KeySubspace(prefix); - } - - /// Create a copy of a generic subspace, sharing the same binary prefix - [Pure, NotNull] - public static DynamicKeySubspace Copy([NotNull] IKeySubspace subspace, IKeyEncoding encoding) - { - Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new DynamicKeySubspace(StealPrefix(subspace), encoding); - } - - /// Create a copy of a generic subspace, sharing the same binary prefix - [Pure, NotNull] - public static DynamicKeySubspace Copy([NotNull] IKeySubspace subspace, IDynamicKeyEncoder encoder) - { - Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoder, nameof(encoder)); - return new DynamicKeySubspace(StealPrefix(subspace), encoder); - } - - /// Create a copy of a dynamic subspace, sharing the same binary prefix and encoder - [Pure, NotNull] - public static DynamicKeySubspace Copy([NotNull] IDynamicKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - return new DynamicKeySubspace(StealPrefix(subspace), subspace.Encoding); - } - - /// Create a copy of a typed subspace, sharing the same binary prefix and encoder - [Pure, NotNull] - public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); - } - - /// Create a copy of a typed subspace, sharing the same binary prefix and encoder - [Pure, NotNull] - public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); - } - - /// Create a copy of a typed subspace, sharing the same binary prefix and encoder - [Pure, NotNull] - public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); - } - - /// Create a copy of a typed subspace, sharing the same binary prefix and encoder - [Pure, NotNull] - public static TypedKeySubspace Copy([NotNull] ITypedKeySubspace subspace) - { - Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); - } - - #endregion - internal KeySubspace(Slice prefix) { this.Key = prefix; @@ -260,7 +171,7 @@ protected virtual Slice GetKeyPrefix() /// Returns the master instance of the prefix, without any safety checks /// This instance should NEVER be exposed to anyone else, and should ONLY be used for logging/troubleshooting - protected Slice GetPrefixUnsafe() + internal Slice GetPrefixUnsafe() { return this.Key; } diff --git a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index 7afb018e9..9600dbed4 100644 --- a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Client using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Serialization.Encoders; using JetBrains.Annotations; @@ -40,86 +41,90 @@ namespace FoundationDB.Client public static class KeySubspaceExtensions { + #region Encodings... + /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace /// If non-null, uses this specific instance of the TypeSystem. If null, uses the default instance for this particular TypeSystem /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IDynamicKeySubspace Using([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoding encoding) - //REVIEW: rename to AsDynamic() ? ToDynamic() ? would all to make encoding arg optional (and default to Tuples) + public static IDynamicKeySubspace AsDynamic([NotNull] this IKeySubspace subspace, IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoding, nameof(encoding)); - return new DynamicKeySubspace(subspace.GetPrefix(), encoding); + return new DynamicKeySubspace(subspace.GetPrefix(), encoding ?? TuPack.Encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace to extend - /// Custom key encoder + /// Encoding by the keys of this subspace. If not specified, the Tuple Encoding will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static IDynamicKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) + public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoder, nameof(encoder)); - return new DynamicKeySubspace(subspace.GetPrefix(), encoder.Encoding); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace to extend - /// Encoding by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. + /// Encoding used by the keys of this subspace. If not specified, the Tuple Encoding will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) + public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace to extend - /// Custom key encoder + /// Encoding used by the keys of this subspace. If not specified, the Tuple Encoding will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoder encoder) + public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), encoder); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace to extend - /// Encoding used by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. + /// Instance of a generic subspace + /// Encoding used by the keys of this namespace. If not specified, the Tuple Encoding will be used to generate an encoder. /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) + public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); } + + #endregion + + #region Encoders... + /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace to extend /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static IDynamicKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IDynamicKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), encoder); + return new DynamicKeySubspace(subspace.GetPrefix(), encoder.Encoding); } /// Return a version of this subspace, which uses a different type system to produces the keys and values /// Instance of a generic subspace to extend - /// Encoding used by the keys of this subspace. If not specified, the Tuples Type System will be used to generate an encoder. + /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] IKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -127,22 +132,23 @@ public static ITypedKeySubspace UsingEncoder([NotNull] t /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); Contract.NotNull(encoder, nameof(encoder)); - return new TypedKeySubspace(subspace.GetPrefix(), encoder); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values - /// Instance of a generic subspace - /// Encoding used by the keys of this namespace. If not specified, the Tuples Type System will be used to generate an encoder. + /// Instance of a generic subspace to extend + /// Custom key encoder /// Subspace equivalent to , but augmented with a specific TypeSystem [Pure, NotNull] - public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) + public static ITypedKeySubspace UsingEncoder([NotNull] this IKeySubspace subspace, [NotNull] ICompositeKeyEncoder encoder) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TypeSystem.Tuples).GetEncoder()); + Contract.NotNull(encoder, nameof(encoder)); + return new TypedKeySubspace(subspace.GetPrefix(), encoder); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -157,6 +163,97 @@ public static ITypedKeySubspace UsingEncoder([No return new TypedKeySubspace(subspace.GetPrefix(), encoder); } + #endregion + + #region Copy... + + /// Create a new copy of a subspace's prefix + [Pure] + internal static Slice StealPrefix([NotNull] IKeySubspace subspace) + { + //note: we can workaround the 'security' in top directory partition by accessing their key prefix without triggering an exception! + return subspace is KeySubspace ks + ? ks.GetPrefixUnsafe().Memoize() + : subspace.GetPrefix().Memoize(); + } + + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static KeySubspace Copy([NotNull] this IKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + + var prefix = StealPrefix(subspace); + + if (subspace is IDynamicKeySubspace dyn) + { // reuse the encoding of the original + return new DynamicKeySubspace(prefix, dyn.Encoding); + } + + // no encoding + return new KeySubspace(prefix); + } + + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] this IKeySubspace subspace, IKeyEncoding encoding) + { + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoding, nameof(encoding)); + return new DynamicKeySubspace(StealPrefix(subspace), encoding); + } + + /// Create a copy of a generic subspace, sharing the same binary prefix + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] this IKeySubspace subspace, IDynamicKeyEncoder encoder) + { + Contract.NotNull(subspace, nameof(subspace)); + Contract.NotNull(encoder, nameof(encoder)); + return new DynamicKeySubspace(StealPrefix(subspace), encoder); + } + + /// Create a copy of a dynamic subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static DynamicKeySubspace Copy([NotNull] this IDynamicKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new DynamicKeySubspace(StealPrefix(subspace), subspace.Encoding); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] this ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] this ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] this ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + /// Create a copy of a typed subspace, sharing the same binary prefix and encoder + [Pure, NotNull] + public static TypedKeySubspace Copy([NotNull] this ITypedKeySubspace subspace) + { + Contract.NotNull(subspace, nameof(subspace)); + return new TypedKeySubspace(StealPrefix(subspace), subspace.KeyEncoder); + } + + #endregion + /// Clear the entire content of a subspace public static void ClearRange(this IFdbTransaction trans, [NotNull] IKeySubspace subspace) { diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index 9b9c1cd30..c3d95add7 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -32,10 +32,9 @@ namespace FoundationDB.Layers.Blobs using System.Diagnostics; using System.Globalization; using System.Threading.Tasks; + using Doxense.Diagnostics.Contracts; using FoundationDB.Client; using JetBrains.Annotations; - using Doxense.Diagnostics.Contracts; - using Doxense.Serialization.Encoders; /// Represents a potentially large binary value in FoundationDB. [DebuggerDisplay("Subspace={" + nameof(FdbBlob.Subspace) + "}")] @@ -57,11 +56,12 @@ public FdbBlob([NotNull] IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); } /// Subspace used as a prefix for all items in this table - public IDynamicKeySubspace Subspace {[NotNull] get; private set; } + [NotNull] + public IDynamicKeySubspace Subspace { get; } /// Returns the key for data chunk at the specified offset /// diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index b8ab0a3ed..78d4cda08 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -77,13 +77,15 @@ public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues, ICompositeKe #region Public Properties... /// Subspace used as a prefix for all items in this map - public IKeySubspace Subspace { [NotNull] get; } + [NotNull] + public IKeySubspace Subspace { get; } /// If true, allow negative or zero values to stay in the map. public bool AllowNegativeValues { get; } /// Subspace used to encoded the keys for the items - protected ITypedKeySubspace Location { [NotNull] get; } + [NotNull] + protected ITypedKeySubspace Location { get; } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index 8a8cd9473..d1845c587 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -72,7 +72,7 @@ public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] if (subspace == null) throw new ArgumentNullException(nameof(subspace)); if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); this.HighContention = highContention; this.Encoder = encoder; diff --git a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs index 2e5e136a9..c4730edef 100644 --- a/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs +++ b/FoundationDB.Layers.Common/Collections/FdbRankedSet.cs @@ -33,7 +33,6 @@ namespace FoundationDB.Layers.Collections using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; - using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -56,7 +55,7 @@ public FdbRankedSet([NotNull] IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); } public Task OpenAsync([NotNull] IFdbTransaction trans) diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index dc951af2d..439b5e12a 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -75,7 +75,7 @@ public FdbVector([NotNull] IKeySubspace subspace, T defaultValue, [NotNull] IVal if (subspace == null) throw new ArgumentNullException("subspace"); if (encoder == null) throw new ArgumentNullException("encoder"); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); this.DefaultValue = defaultValue; this.Encoder = encoder; } diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index c7efbfc80..2d62e905c 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -70,7 +70,7 @@ public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspac if (encoder == null) throw new ArgumentNullException(nameof(encoder)); this.Database = db; - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); this.Encoder = encoder; } diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index f13d80c54..050f64c76 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -36,7 +36,6 @@ namespace FoundationDB.Layers.Interning using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; - using Doxense.Serialization.Encoders; using FoundationDB.Client; /// Provides a class for interning (aka normalizing, aliasing) commonly-used long strings into shorter representations. @@ -95,7 +94,7 @@ public FdbStringIntern(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException("subspace"); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); } diff --git a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs index 3bf2bf38e..d30df3854 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbHashSetCollection.cs @@ -33,7 +33,6 @@ namespace FoundationDB.Layers.Blobs using System.Threading.Tasks; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -47,7 +46,7 @@ public FdbHashSetCollection(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); } /// Subspace used as a prefix for all hashsets in this collection diff --git a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs index 5baf44136..9b946b15d 100644 --- a/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs +++ b/FoundationDB.Layers.Experimental/Messaging/FdbWorkerPool.cs @@ -34,7 +34,6 @@ namespace FoundationDB.Layers.Messaging using System.Security.Cryptography; using System.Threading; using System.Threading.Tasks; - using Doxense.Serialization.Encoders; using FoundationDB.Client; using FoundationDB.Filters.Logging; using FoundationDB.Layers.Counters; @@ -111,7 +110,7 @@ public FdbWorkerPool(IKeySubspace subspace) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - this.Subspace = subspace.Using(TypeSystem.Tuples); + this.Subspace = subspace.AsDynamic(); this.TaskStore = this.Subspace.Partition.ByKey(Slice.FromChar('T')); this.IdleRing = this.Subspace.Partition.ByKey(Slice.FromChar('I')); diff --git a/FoundationDB.Tests/Layers/DirectoryFacts.cs b/FoundationDB.Tests/Layers/DirectoryFacts.cs index c3fe5133a..6966cc54d 100644 --- a/FoundationDB.Tests/Layers/DirectoryFacts.cs +++ b/FoundationDB.Tests/Layers/DirectoryFacts.cs @@ -571,7 +571,7 @@ public async Task Test_Directory_Partitions() var partition = await directory.CreateAsync(db, "Foo", Slice.FromStringAscii("partition"), this.Cancellation); // we can't get the partition key directory (because it's a root directory) so we need to cheat a little bit - var partitionKey = KeySubspace.Copy(partition).GetPrefix(); + var partitionKey = partition.Copy().GetPrefix(); Log(partition); Assert.That(partition, Is.InstanceOf()); Assert.That(partition.Layer, Is.EqualTo(Slice.FromStringAscii("partition"))); @@ -731,7 +731,7 @@ public async Task Test_Renaming_Partition_Uses_Parent_DirectoryLayer() // should have kept the same prefix //note: we need to cheat to get the key of the partition - Assert.That(KeySubspace.Copy(bar).GetPrefix(), Is.EqualTo(KeySubspace.Copy(foo).GetPrefix())); + Assert.That(bar.Copy().GetPrefix(), Is.EqualTo(foo.Copy().GetPrefix())); // verify list again folders = await directory.ListAsync(tr); @@ -867,7 +867,7 @@ void ShouldPass(ActualValueDelegate del) // === PASS === // these methods are allowed to succeed on directory partitions, because we need them for the rest to work - ShouldPass(() => KeySubspace.Copy(partition).GetPrefix()); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! + ShouldPass(() => partition.Copy().GetPrefix()); // EXCEPTION: we need this to work, because that's the only way that the unit tests above can see the partition key! ShouldPass(() => partition.ToString()); // EXCEPTION: this should never fail! ShouldPass(() => partition.DumpKey(barKey)); // EXCEPTION: this should always work, because this can be used for debugging and logging... ShouldPass(() => partition.BoundCheck(barKey, true)); // EXCEPTION: needs to work because it is used by GetRange() and GetKey() diff --git a/FoundationDB.Tests/Layers/MultiMapFacts.cs b/FoundationDB.Tests/Layers/MultiMapFacts.cs index 048df6424..c46f1cfc8 100644 --- a/FoundationDB.Tests/Layers/MultiMapFacts.cs +++ b/FoundationDB.Tests/Layers/MultiMapFacts.cs @@ -87,7 +87,7 @@ public async Task Test_FdbMultiMap_Read_Write_Delete() // directly read the value, behind the table's back using (var tr = db.BeginTransaction(this.Cancellation)) { - var loc = map.Subspace.Using(TypeSystem.Tuples); + var loc = map.Subspace.AsDynamic(); var value = await tr.GetAsync(loc.Keys.Encode("hello", "world")); Assert.That(value, Is.Not.EqualTo(Slice.Nil)); Assert.That(value.ToInt64(), Is.EqualTo(1)); @@ -111,7 +111,7 @@ public async Task Test_FdbMultiMap_Read_Write_Delete() Assert.That(count, Is.Null); // also check directly - var loc = map.Subspace.Using(TypeSystem.Tuples); + var loc = map.Subspace.AsDynamic(); var data = await tr.GetAsync(loc.Keys.Encode("hello", "world")); Assert.That(data, Is.EqualTo(Slice.Nil)); } diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 9a015c47c..2d84d2f6a 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -688,9 +688,9 @@ public async Task Test_Range_Except_Composite_Key() var location = await GetCleanDirectory(db, "Queries", "ExceptComposite"); // Items contains a list of all ("user", id) that were created - var locItems = (await location.CreateOrOpenAsync(db, "Items", this.Cancellation)).UsingEncoder(); + var locItems = (await location.CreateOrOpenAsync(db, "Items", this.Cancellation)).AsTyped(); // Processed contain the list of all ("user", id) that were processed - var locProcessed = (await location.CreateOrOpenAsync(db, "Processed", this.Cancellation)).UsingEncoder(); + var locProcessed = (await location.CreateOrOpenAsync(db, "Processed", this.Cancellation)).AsTyped(); // the goal is to have a query that returns the list of all unprocessed items (ie: in Items but not in Processed) diff --git a/FoundationDB.Tests/SubspaceFacts.cs b/FoundationDB.Tests/SubspaceFacts.cs index ec6983561..0804e37ea 100644 --- a/FoundationDB.Tests/SubspaceFacts.cs +++ b/FoundationDB.Tests/SubspaceFacts.cs @@ -45,18 +45,18 @@ public void Test_Empty_Subspace_Is_Empty() var subspace = KeySubspace.FromKey(Slice.Empty); Assert.That(subspace, Is.Not.Null, "FdbSubspace.Empty should not return null"); Assert.That(subspace.GetPrefix(), Is.EqualTo(Slice.Empty), "FdbSubspace.Empty.Key should be equal to Slice.Empty"); - Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); + Assert.That(subspace.Copy(), Is.Not.SameAs(subspace)); } [Test] [Category("LocalCluster")] public void Test_Subspace_With_Binary_Prefix() { - var subspace = KeySubspace.CreateDynamic(new byte[] { 42, 255, 0, 127 }.AsSlice(), TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(new byte[] { 42, 255, 0, 127 }.AsSlice()); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("*<00><7F>")); - Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); - Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); + Assert.That(subspace.Copy(), Is.Not.SameAs(subspace)); + Assert.That(subspace.Copy().GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the binary prefix directly Assert.That(subspace[Slice.FromInt32(0x01020304)].ToString(), Is.EqualTo("*<00><7F><04><03><02><01>")); @@ -92,7 +92,7 @@ public void Test_Subspace_With_Binary_Prefix() public void Test_Subspace_Copy_Does_Not_Share_Key_Buffer() { var original = KeySubspace.FromKey(Slice.FromString("Hello")); - var copy = KeySubspace.Copy(original); + var copy = original.Copy(); Assert.That(copy, Is.Not.Null); Assert.That(copy, Is.Not.SameAs(original), "Copy should be a new instance"); Assert.That(copy.GetPrefix(), Is.EqualTo(original.GetPrefix()), "Key should be equal"); @@ -117,11 +117,11 @@ public void Test_Cannot_Create_Or_Partition_Subspace_With_Slice_Nil() [Category("LocalCluster")] public void Test_Subspace_With_Tuple_Prefix() { - var subspace = KeySubspace.CreateDynamic(TuPack.EncodeKey("hello"), TypeSystem.Tuples); + var subspace = KeySubspace.CreateDynamic(TuPack.EncodeKey("hello")); Assert.That(subspace.GetPrefix().ToString(), Is.EqualTo("<02>hello<00>")); - Assert.That(KeySubspace.Copy(subspace), Is.Not.SameAs(subspace)); - Assert.That(KeySubspace.Copy(subspace).GetPrefix(), Is.EqualTo(subspace.GetPrefix())); + Assert.That(subspace.Copy(), Is.Not.SameAs(subspace)); + Assert.That(subspace.Copy().GetPrefix(), Is.EqualTo(subspace.GetPrefix())); // concat(Slice) should append the slice to the tuple prefix directly Assert.That(subspace[Slice.FromInt32(0x01020304)].ToString(), Is.EqualTo("<02>hello<00><04><03><02><01>")); @@ -149,7 +149,7 @@ public void Test_Subspace_With_Tuple_Prefix() public void Test_Subspace_Partitioning_With_Binary_Suffix() { // start from a parent subspace - var parent = KeySubspace.CreateDynamic(Slice.Empty, TypeSystem.Tuples); + var parent = KeySubspace.CreateDynamic(Slice.Empty); Assert.That(parent.GetPrefix().ToString(), Is.EqualTo("")); // create a child subspace using a tuple @@ -176,7 +176,7 @@ public void Test_Subspace_Partitioning_With_Binary_Suffix() [Test] public void Test_DynamicKeySpace_API() { - var location = KeySubspace.CreateDynamic(Slice.FromString("PREFIX"), TypeSystem.Tuples); + var location = KeySubspace.CreateDynamic(Slice.FromString("PREFIX")); Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -231,7 +231,7 @@ public void Test_TypedKeySpace_T1() { var location = KeySubspace.CreateTyped(Slice.FromString("PREFIX")); Assert.That(location.KeyEncoder, Is.Not.Null, "Should have a Key Encoder"); - Assert.That(location.KeyEncoder.Encoding, Is.SameAs(TypeSystem.Tuples), "Encoder should use Tuple type system"); + Assert.That(location.KeyEncoder.Encoding, Is.SameAs(TuPack.Encoding), "Encoder should use Tuple type system"); // shortcuts Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -258,8 +258,7 @@ public void Test_TypedKeySpace_T1() [Test] public void Test_TypedKeySpace_T2() { - var location = new KeySubspace(Slice.FromString("PREFIX")) - .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + var location = KeySubspace.CreateTyped(Slice.FromString("PREFIX")); // shortcuts Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -287,8 +286,7 @@ public void Test_TypedKeySpace_T2() [Test] public void Test_TypedKeySpace_T3() { - var location = new KeySubspace(Slice.FromString("PREFIX")) - .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + var location = KeySubspace.CreateTyped(Slice.FromString("PREFIX")); // shortcuts Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); @@ -317,8 +315,7 @@ public void Test_TypedKeySpace_T3() [Test] public void Test_TypedKeySpace_T4() { - var location = new KeySubspace(Slice.FromString("PREFIX")) - .UsingEncoder(TypeSystem.Tuples.GetEncoder()); + var location = KeySubspace.CreateTyped(Slice.FromString("PREFIX")); // shortcuts Assert.That(location[Slice.FromString("SUFFIX")].ToString(), Is.EqualTo("PREFIXSUFFIX")); From 2cf31c85194013f048c460d66d09573da9c67db5 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 18:05:38 +0200 Subject: [PATCH 135/153] Remove TypeSystem altogether --- .../Shared/TypeSystem/TypeSystem.cs | 47 ------------------- 1 file changed, 47 deletions(-) delete mode 100644 FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs diff --git a/FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs b/FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs deleted file mode 100644 index c9008d342..000000000 --- a/FoundationDB.Client/Shared/TypeSystem/TypeSystem.cs +++ /dev/null @@ -1,47 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace Doxense.Serialization.Encoders -{ - using System; - using Doxense.Collections.Tuples.Encoding; - using JetBrains.Annotations; - - public static class TypeSystem - { - [NotNull] - public static readonly IKeyEncoding Default = new TupleKeyEncoding(); - - [NotNull] - public static readonly IKeyEncoding Tuples = Default; - - //TODO: more? or remove this class? - - } - -} From c3ae90335f0f58512cabfec527b34638cde6d782 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 18:17:47 +0200 Subject: [PATCH 136/153] Decouple KeyRange from the key encoders by representing them as a (Slice, Slice) tuple internally - Add implicit cast between KeyRange and (Slice, Slice) to make this invisible to users --- FoundationDB.Client/KeyRange.cs | 34 +++- .../Encoders/DynamicKeyEncoderBase.cs | 20 +-- .../TypeSystem/Encoders/IDynamicKeyEncoder.cs | 20 +-- .../Tuples/Encoding/TupleKeyEncoder.cs | 20 +-- FoundationDB.Client/Tuples/TuPack.cs | 148 +++++++++--------- 5 files changed, 133 insertions(+), 109 deletions(-) diff --git a/FoundationDB.Client/KeyRange.cs b/FoundationDB.Client/KeyRange.cs index 20fd239bb..a4c72b0de 100644 --- a/FoundationDB.Client/KeyRange.cs +++ b/FoundationDB.Client/KeyRange.cs @@ -38,7 +38,7 @@ namespace FoundationDB.Client /// Represents a pair of keys defining the range 'Begin <= key > End' [DebuggerDisplay("Begin={Begin}, End={End}")] - public readonly struct KeyRange : IEquatable, IComparable + public readonly struct KeyRange : IEquatable, IComparable, IEquatable<(Slice Begin, Slice End)>, IComparable<(Slice Begin, Slice End)> { /// Start of the range @@ -50,11 +50,11 @@ namespace FoundationDB.Client /// Create a new range of keys /// Start of range (usually included) /// End of range (usually excluded) + [MethodImpl(MethodImplOptions.AggressiveInlining)] public KeyRange(Slice begin, Slice end) { this.Begin = begin; this.End = end; - Contract.Ensures(this.Begin <= this.End, "The range is inverted"); } @@ -122,14 +122,14 @@ public static KeyRange FromKey(Slice key) public override bool Equals(object obj) { - return obj is KeyRange range && Equals(range); + if (obj is KeyRange range) return Equals(range); + if (obj is ValueTuple tuple) return Equals(tuple); + return false; } public override int GetHashCode() { - // ReSharper disable NonReadonlyMemberInGetHashCode return HashCodes.Combine(this.Begin.GetHashCode(), this.End.GetHashCode()); - // ReSharper restore NonReadonlyMemberInGetHashCode } public bool Equals(KeyRange other) @@ -137,6 +137,11 @@ public bool Equals(KeyRange other) return this.Begin.Equals(other.Begin) && this.End.Equals(other.End); } + public bool Equals((Slice Begin, Slice End) other) + { + return this.Begin.Equals(other.Begin) && this.End.Equals(other.End); + } + public static bool operator ==(KeyRange left, KeyRange right) { return left.Begin.Equals(right.Begin) && left.End.Equals(right.End); @@ -154,6 +159,25 @@ public int CompareTo(KeyRange other) return c; } + public int CompareTo((Slice Begin, Slice End) other) + { + int c = this.Begin.CompareTo(other.Begin); + if (c == 0) c = this.End.CompareTo(other.End); + return c; + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator KeyRange((Slice Begin, Slice End) range) + { + return new KeyRange(range.Begin, range.End); + } + + [Pure, MethodImpl(MethodImplOptions.AggressiveInlining)] + public static implicit operator (Slice Begin, Slice End)(KeyRange range) + { + return (range.Begin, range.End); + } + /// Combine another range with the current range, to produce a range that includes both (and all keys in between it the ranges are disjoint) /// Range to merge with the current range /// New range where the Begin key is the smallest bound and the End key is the largest bound of both ranges. diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs index 5b735f838..d070347af 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/DynamicKeyEncoderBase.cs @@ -38,7 +38,7 @@ public abstract class DynamicKeyEncoderBase : IDynamicKeyEncoder public abstract IKeyEncoding Encoding { get; } - public virtual KeyRange ToRange(Slice prefix) + public virtual (Slice Begin, Slice End) ToRange(Slice prefix) { return KeyRange.StartsWith(prefix); } @@ -127,49 +127,49 @@ public virtual STuple DecodeKey( return UnpackKey(packed).With((T1 a, T2 b, T3 c, T4 d, T5 e, T6 f) => STuple.Create(a, b, c, d, e, f)); } - public virtual KeyRange ToRange(Slice prefix, ITuple items) + public virtual (Slice Begin, Slice End) ToRange(Slice prefix, ITuple items) { var writer = new SliceWriter(prefix, 16); PackKey(ref writer, items); return ToRange(writer.ToSlice()); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1) { return ToRange(prefix, STuple.Create(item1)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2) { return ToRange(prefix, STuple.Create(item1, item2)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { return ToRange(prefix, STuple.Create(item1, item3, item3)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { return ToRange(prefix, STuple.Create(item1, item3, item3, item4)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7)); } - public virtual KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public virtual (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { return ToRange(prefix, STuple.Create(item1, item3, item3, item4, item5, item6, item7, item8)); } diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs index a4d9b3e9c..ede420b09 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/IDynamicKeyEncoder.cs @@ -231,18 +231,18 @@ public interface IDynamicKeyEncoder /// Optional binary prefix /// Key range which derives from the semantic of the current encoding /// For example, the Tuple encoding will produce ranges of the form "(Key + \x00) <= x < (Key + \xFF)", while a binary-based encoding would produce ranges of the form "Key <= x < Increment(Key)" - KeyRange ToRange(Slice prefix = default(Slice)); + (Slice Begin, Slice End) ToRange(Slice prefix = default(Slice)); /// Return a key range using a tuple as a prefix /// Optional binary prefix that should be added before encoding the key /// Tuple of any size (0 to N) - KeyRange ToRange(Slice prefix, ITuple items); + (Slice Begin, Slice End) ToRange(Slice prefix, ITuple items); /// Return a key range using a single element as a prefix /// Type of the element /// Optional binary prefix that should be added before encoding the key /// Element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1); /// Return a key range using two elements as a prefix /// Type of the first element @@ -250,7 +250,7 @@ public interface IDynamicKeyEncoder /// Optional binary prefix that should be added before encoding the key /// First element to encode /// Second element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2); /// Return a key range using three elements as a prefix /// Type of the first element @@ -260,7 +260,7 @@ public interface IDynamicKeyEncoder /// First element to encode /// Second element to encode /// Third element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3); /// Return a key range using four elements as a prefix /// Type of the first element @@ -272,7 +272,7 @@ public interface IDynamicKeyEncoder /// Second element to encode /// Third element to encode /// Fourth element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4); /// Return a key range using five elements as a prefix /// Type of the first element @@ -286,7 +286,7 @@ public interface IDynamicKeyEncoder /// Third element to encode /// Fourth element to encode /// Fifth element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5); /// Return a key range using six elements as a prefix /// Type of the first element @@ -302,7 +302,7 @@ public interface IDynamicKeyEncoder /// Fourth element to encode /// Fifth element to encode /// Sixth element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6); /// Return a key range using seven elements as a prefix /// Type of the first element @@ -320,7 +320,7 @@ public interface IDynamicKeyEncoder /// Fifth element to encode /// Sixth element to encode /// Seventh element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7); /// Return a key range using eight elements as a prefix /// Type of the first element @@ -340,7 +340,7 @@ public interface IDynamicKeyEncoder /// Sixth element to encode /// Seventh element to encode /// Eighth element to encode - KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8); + (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8); //note: I will be billing $999.99 to anyone who wants up to T11 !!! :( diff --git a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs index 04e3ee415..c4404dc85 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoder.cs @@ -182,52 +182,52 @@ public STuple DecodeKey(Slice pa return TuPack.DecodeKey(packed); } - public KeyRange ToRange(Slice prefix) + public (Slice Begin, Slice End) ToRange(Slice prefix) { return TuPack.ToRange(prefix); } - public KeyRange ToRange(Slice prefix, ITuple items) + public (Slice Begin, Slice End) ToRange(Slice prefix, ITuple items) { return TuPack.ToPrefixedKeyRange(prefix, items); } - public KeyRange ToKeyRange(Slice prefix, T1 item1) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1) { return TuPack.ToPrefixedKeyRange(prefix, item1); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6, item7); } - public KeyRange ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public (Slice Begin, Slice End) ToKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { return TuPack.ToPrefixedKeyRange(prefix, item1, item2, item3, item4, item5, item6, item7, item8); } diff --git a/FoundationDB.Client/Tuples/TuPack.cs b/FoundationDB.Client/Tuples/TuPack.cs index df9ba6f96..2483688e4 100644 --- a/FoundationDB.Client/Tuples/TuPack.cs +++ b/FoundationDB.Client/Tuples/TuPack.cs @@ -528,13 +528,13 @@ public static Slice[] EncodePrefixedKeys([NotNull] ITuple prefix, [NotNull] p /// Range including all possible tuples starting with the specified prefix. /// TuPack.ToRange(Slice.FromAscii("abc")) returns the range [ 'abc\x00', 'abc\xFF' ) [Pure] - public static KeyRange ToRange(Slice prefix) + public static (Slice Begin, Slice End) ToRange(Slice prefix) { if (prefix.IsNull) throw new ArgumentNullException(nameof(prefix)); //note: there is no guarantee that prefix is a valid packed tuple (could be any exotic binary prefix) // prefix => [ prefix."\0", prefix."\xFF" ) - return new KeyRange( + return ( prefix + 0x00, prefix + 0xFF ); @@ -543,14 +543,14 @@ public static KeyRange ToRange(Slice prefix) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// TuPack.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToRange([NotNull] TTuple tuple) + public static (Slice Begin, Slice End) ToRange([NotNull] TTuple tuple) where TTuple : ITuple { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -559,13 +559,13 @@ public static KeyRange ToRange([NotNull] TTuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), tuple.Item1); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -574,13 +574,13 @@ public static KeyRange ToRange(STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToRange(ValueTuple tuple) + public static (Slice Begin, Slice End) ToRange(ValueTuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), tuple.Item1); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -589,11 +589,11 @@ public static KeyRange ToRange(ValueTuple tuple) /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToKeyRange(T1 item1) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -602,11 +602,11 @@ public static KeyRange ToKeyRange(T1 item1) /// Create a range that selects all the tuples of greater length than the specified element, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -615,14 +615,14 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -630,14 +630,14 @@ public static KeyRange ToRange(STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange((T1, T2) tuple) + public static (Slice Begin, Slice End) ToRange((T1, T2) tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -646,11 +646,11 @@ public static KeyRange ToRange((T1, T2) tuple) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToKeyRange("a", "b") includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -659,11 +659,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// ToPrefixedKeyRange(..., "a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -671,14 +671,14 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 ite /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -686,12 +686,12 @@ public static KeyRange ToRange(STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange((T1, T2, T3) tuple) + public static (Slice Begin, Slice End) ToRange((T1, T2, T3) tuple) { // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -699,11 +699,11 @@ public static KeyRange ToRange((T1, T2, T3) tuple) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -711,11 +711,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -723,14 +723,14 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -738,12 +738,12 @@ public static KeyRange ToRange(STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange((T1, T2, T3, T4) tuple) + public static (Slice Begin, Slice End) ToRange((T1, T2, T3, T4) tuple) { // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -751,11 +751,11 @@ public static KeyRange ToRange((T1, T2, T3, T4) tuple) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -763,11 +763,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -775,14 +775,14 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1 /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -790,12 +790,12 @@ public static KeyRange ToRange(STuple tu /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange((T1, T2, T3, T4, T5) tuple) + public static (Slice Begin, Slice End) ToRange((T1, T2, T3, T4, T5) tuple) { // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -803,11 +803,11 @@ public static KeyRange ToRange((T1, T2, T3, T4, T5) tuple) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -815,11 +815,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 ite /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -827,14 +827,14 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 i /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(STuple tuple) + public static (Slice Begin, Slice End) ToRange(STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -842,12 +842,12 @@ public static KeyRange ToRange(STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange((T1, T2, T3, T4, T5, T6) tuple) + public static (Slice Begin, Slice End) ToRange((T1, T2, T3, T4, T5, T6) tuple) { // tuple => [ packed."\0", packed."\xFF" ) var empty = default(Slice); var packed = TupleEncoder.Pack(empty, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -855,11 +855,11 @@ public static KeyRange ToRange((T1, T2, T3, T4, T5, T6) /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -867,11 +867,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -879,11 +879,11 @@ public static KeyRange ToPrefixedKeyRange(Slice prefix, /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -891,11 +891,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 item2 /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7) { // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6, item7); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -903,11 +903,11 @@ public static KeyRange ToPrefixedKeyRange(Slice pref /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public static (Slice Begin, Slice End) ToKeyRange(T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(default(Slice), item1, item2, item3, item4, item5, item6, item7, item8); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -915,11 +915,11 @@ public static KeyRange ToKeyRange(T1 item1, T2 i /// Create a range that selects all the tuples of greater length than the specified items, and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) + public static (Slice Begin, Slice End) ToPrefixedKeyRange(Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5, T6 item6, T7 item7, T8 item8) { // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.EncodeKey(prefix, item1, item2, item3, item4, item5, item6, item7, item8); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -929,14 +929,14 @@ public static KeyRange ToPrefixedKeyRange(Slice /// TuPack.ToRange(Slice.FromInt32(42), Stuple.Create("a", "b")) includes all tuples \x2A.("a", "b", ...), but not the tuple \x2A.("a", "b") itself. /// If is the packed representation of a tuple, then unpacking the resulting key will produce a valid tuple. If not, then the resulting key will need to be truncated first before unpacking. [Pure] - public static KeyRange ToRange(Slice prefix, [NotNull] TTuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, [NotNull] TTuple tuple) where TTuple : ITuple { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ prefix.packed."\0", prefix.packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -945,13 +945,13 @@ public static KeyRange ToRange(Slice prefix, [NotNull] TTuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// TuPack.ToRange(STuple.Create("a")) includes all tuples ("a", ...), but not the tuple ("a") itself. [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -960,13 +960,13 @@ public static KeyRange ToRange(Slice prefix, STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' /// TuPack.ToRange(STuple.Create("a", "b")) includes all tuples ("a", "b", ...), but not the tuple ("a", "b") itself. [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -974,13 +974,13 @@ public static KeyRange ToRange(Slice prefix, STuple tuple) /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -988,13 +988,13 @@ public static KeyRange ToRange(Slice prefix, STuple tupl /// Create a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -1002,13 +1002,13 @@ public static KeyRange ToRange(Slice prefix, STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); @@ -1016,13 +1016,13 @@ public static KeyRange ToRange(Slice prefix, STupleCreate a range that selects all the tuples of greater length than the specified , and that start with the specified elements: packed(tuple)+'\x00' <= k < packed(tuple)+'\xFF' [Pure] - public static KeyRange ToRange(Slice prefix, STuple tuple) + public static (Slice Begin, Slice End) ToRange(Slice prefix, STuple tuple) { Contract.NotNullAllowStructs(tuple, nameof(tuple)); // tuple => [ packed."\0", packed."\xFF" ) var packed = TupleEncoder.Pack(prefix, ref tuple); - return new KeyRange( + return ( packed + 0x00, packed + 0xFF ); From eb00e3cc41418f40eeaf5092f0a906b244a86b4e Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 18:48:00 +0200 Subject: [PATCH 137/153] Refactor Tuple Key/Value encoders --- .../Encoders/ICompositeKeyEncoder.cs | 217 --------- .../Shared/TypeSystem/Encoders/IKeyEncoder.cs | 26 -- .../Encoders/KeyEncoderExtensions.cs | 414 ++++++++++++++++++ .../Encoders/KeyValueEncoders.Tuples.cs | 213 +-------- .../TypeSystem/Encoders/KeyValueEncoders.cs | 226 ---------- .../Encoders/ValueEncoderExtensions.cs | 133 ++++++ .../Tuples/Encoding/TupleEncoder.cs | 208 +++++++++ 7 files changed, 760 insertions(+), 677 deletions(-) create mode 100644 FoundationDB.Client/Shared/TypeSystem/Encoders/KeyEncoderExtensions.cs create mode 100644 FoundationDB.Client/Shared/TypeSystem/Encoders/ValueEncoderExtensions.cs diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs index 8e5efa1cf..db16363f3 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs @@ -29,7 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Serialization.Encoders { using System; - using Doxense.Collections.Tuples; using Doxense.Memory; public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2)> @@ -76,220 +75,4 @@ public interface ICompositeKeyEncoder : IKeyEncoder<(T1, /// Read some or all parts of a composite key void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); } - - public static partial class KeyEncoderExtensions - { - - #region - - public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2) - { - var tuple = (value1, value2); - encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2) - { - var writer = default(SliceWriter); - var tuple = (item1, item2); - encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2) - { - var writer = new SliceWriter(prefix.Count + 24); - writer.WriteBytes(prefix); - encoder.WriteKeyTo(ref writer, item1, item2); - return writer.ToSlice(); - } - - public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, T1 item1) - { - var writer = default(SliceWriter); - var tuple = (item1, default(T2)); - encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1) - { - var writer = new SliceWriter(prefix.Count + 16); - writer.WriteBytes(prefix); - var tuple = (item1, default(T2)); - encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2) items) - { - var writer = default(SliceWriter); - encoder.WriteKeyPartsTo(ref writer, count, ref items); - return writer.ToSlice(); - } - - public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) - { - var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out var items); - //TODO: throw if extra bytes? - return items; - } - - public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) - { - var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out var items); - return items; - } - - #endregion - - #region - - public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3) - { - var tuple = (value1, value2, value3); - encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3) - { - var writer = default(SliceWriter); - var tuple = (item1, item2, item3); - encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3) - { - var writer = new SliceWriter(prefix.Count + 32); - writer.WriteBytes(prefix); - encoder.WriteKeyTo(ref writer, item1, item2, item3); - return writer.ToSlice(); - } - - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3) items) - { - var writer = default(SliceWriter); - encoder.WriteKeyPartsTo(ref writer, count, ref items); - return writer.ToSlice(); - } - - public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) - { - var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out var items); - //TODO: throw if extra bytes? - return items; - } - - public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) - { - var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out var items); - return items; - } - - #endregion - - #region - - public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4) - { - var tuple = (value1, value2, value3, value4); - encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4) - { - var writer = default(SliceWriter); - var tuple = (item1, item2, item3, item4); - encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) - { - var writer = new SliceWriter(prefix.Count + 48); - writer.WriteBytes(prefix); - encoder.WriteKeyTo(ref writer, item1, item2, item3, item4); - return writer.ToSlice(); - } - - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4) items) - { - var writer = default(SliceWriter); - encoder.WriteKeyPartsTo(ref writer, count, ref items); - return writer.ToSlice(); - } - - public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) - { - var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out var items); - //TODO: throw if extra bytes? - return items; - } - - public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) - { - var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out var items); - return items; - } - - #endregion - - #region - - public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) - { - var tuple = (value1, value2, value3, value4, value5); - encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) - { - var writer = default(SliceWriter); - var tuple = (item1, item2, item3, item4, item5); - encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); - return writer.ToSlice(); - } - - public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) - { - var writer = new SliceWriter(prefix.Count + 56); - writer.WriteBytes(prefix); - encoder.WriteKeyTo(ref writer, item1, item2, item3, item4, item5); - return writer.ToSlice(); - } - - public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4, T5) items) - { - var writer = default(SliceWriter); - encoder.WriteKeyPartsTo(ref writer, count, ref items); - return writer.ToSlice(); - } - - public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) - { - var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out var items); - //TODO: throw if extra bytes? - return items; - } - - public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) - { - var reader = new SliceReader(encoded); - encoder.ReadKeyPartsFrom(ref reader, count, out var items); - return items; - } - - - #endregion - - } - } diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs index ec9c3fbde..329c827aa 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/IKeyEncoder.cs @@ -49,30 +49,4 @@ public interface IKeyEncoder : IKeyEncoder void ReadKeyFrom(ref SliceReader reader, out T1 value); } - public static partial class KeyEncoderExtensions - { - - public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, T1 value) - { - var writer = default(SliceWriter); - encoder.WriteKeyTo(ref writer, value); - return writer.ToSlice(); - } - - public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, Slice prefix, T1 value) - { - var writer = new SliceWriter(prefix.Count + 16); // ~16 bytes si T1 = Guid - writer.WriteBytes(prefix); - encoder.WriteKeyTo(ref writer, value); - return writer.ToSlice(); - } - - public static T1 DecodeKey([NotNull] this IKeyEncoder decoder, Slice encoded) - { - var reader = new SliceReader(encoded); - decoder.ReadKeyFrom(ref reader, out T1 item); - //TODO: should we fail if extra bytes? - return item; - } - } } diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyEncoderExtensions.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyEncoderExtensions.cs new file mode 100644 index 000000000..8b3106dea --- /dev/null +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyEncoderExtensions.cs @@ -0,0 +1,414 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Serialization.Encoders +{ + using System; + using System.Collections.Generic; + using System.Linq; + using Doxense.Collections.Tuples; + using Doxense.Diagnostics.Contracts; + using Doxense.Memory; + using JetBrains.Annotations; + + public static class KeyEncoderExtensions + { + + #region + + public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, T1 value) + { + var writer = default(SliceWriter); + encoder.WriteKeyTo(ref writer, value); + return writer.ToSlice(); + } + + public static Slice EncodeKey([NotNull] this IKeyEncoder encoder, Slice prefix, T1 value) + { + var writer = new SliceWriter(prefix.Count + 16); // ~16 bytes si T1 = Guid + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, value); + return writer.ToSlice(); + } + + public static T1 DecodeKey([NotNull] this IKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out T1 item); + //TODO: should we fail if extra bytes? + return item; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2) + { + var tuple = (value1, value2); + encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2) + { + var writer = default(SliceWriter); + var tuple = (item1, item2); + encoder.WriteKeyPartsTo(ref writer, 2, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2) + { + var writer = new SliceWriter(prefix.Count + 24); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2); + return writer.ToSlice(); + } + + public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, T1 item1) + { + var writer = default(SliceWriter); + var tuple = (item1, default(T2)); + encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodePartialKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1) + { + var writer = new SliceWriter(prefix.Count + 16); + writer.WriteBytes(prefix); + var tuple = (item1, default(T2)); + encoder.WriteKeyPartsTo(ref writer, 1, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2) items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out var items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3) + { + var tuple = (value1, value2, value3); + encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3) + { + var writer = default(SliceWriter); + var tuple = (item1, item2, item3); + encoder.WriteKeyPartsTo(ref writer, 3, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3) + { + var writer = new SliceWriter(prefix.Count + 32); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3) items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out var items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4) + { + var tuple = (value1, value2, value3, value4); + encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4) + { + var writer = default(SliceWriter); + var tuple = (item1, item2, item3, item4); + encoder.WriteKeyPartsTo(ref writer, 4, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4) + { + var writer = new SliceWriter(prefix.Count + 48); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3, item4); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4) items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out var items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); + return items; + } + + #endregion + + #region + + public static void WriteKeyTo(this ICompositeKeyEncoder encoder, ref SliceWriter writer, T1 value1, T2 value2, T3 value3, T4 value4, T5 value5) + { + var tuple = (value1, value2, value3, value4, value5); + encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var writer = default(SliceWriter); + var tuple = (item1, item2, item3, item4, item5); + encoder.WriteKeyPartsTo(ref writer, 5, ref tuple); + return writer.ToSlice(); + } + + public static Slice EncodeKey(this ICompositeKeyEncoder encoder, Slice prefix, T1 item1, T2 item2, T3 item3, T4 item4, T5 item5) + { + var writer = new SliceWriter(prefix.Count + 56); + writer.WriteBytes(prefix); + encoder.WriteKeyTo(ref writer, item1, item2, item3, item4, item5); + return writer.ToSlice(); + } + + public static Slice EncodeKeyParts(this ICompositeKeyEncoder encoder, int count, (T1, T2, T3, T4, T5) items) + { + var writer = default(SliceWriter); + encoder.WriteKeyPartsTo(ref writer, count, ref items); + return writer.ToSlice(); + } + + public static STuple DecodeKey(this ICompositeKeyEncoder decoder, Slice encoded) + { + var reader = new SliceReader(encoded); + decoder.ReadKeyFrom(ref reader, out var items); + //TODO: throw if extra bytes? + return items; + } + + public static STuple DecodeKeyParts(this ICompositeKeyEncoder encoder, int count, Slice encoded) + { + var reader = new SliceReader(encoded); + encoder.ReadKeyPartsFrom(ref reader, count, out var items); + return items; + } + + + #endregion + + #region Batched... + + /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params T[] values) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); + + var slices = new Slice[values.Length]; + for (int i = 0; i < values.Length; i++) + { + slices[i] = encoder.EncodeKey(values[i]); + } + return slices; + } + + /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable elements, Func selector) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(selector, nameof(selector)); + + TElement[] arr; + ICollection coll; + + if ((arr = elements as TElement[]) != null) + { // fast path for arrays + return EncodeKeys(encoder, arr, selector); + } + if ((coll = elements as ICollection) != null) + { // we can pre-allocate the result array + var slices = new Slice[coll.Count]; + int p = 0; + foreach(var item in coll) + { + slices[p++] = encoder.EncodeKey(selector(item)); + } + return slices; + } + // slow path + return elements.Select((item) => encoder.EncodeKey(selector(item))).ToArray(); + } + + /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] TElement[] elements, Func selector) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(elements, nameof(elements)); + Contract.NotNull(selector, nameof(selector)); + + var slices = new Slice[elements.Length]; + for (int i = 0; i < elements.Length; i++) + { + slices[i] = encoder.EncodeKey(selector(elements[i])); + } + return slices; + } + + /// Transform a sequence of s into a sequence of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static IEnumerable EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable values) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); + + // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed + + if (values is T[] array) + { // optimized path for arrays + return EncodeKeys(encoder, array); + } + + if (values is ICollection coll) + { // optimized path when we know the count + var slices = new List(coll.Count); + foreach (var value in coll) + { + slices.Add(encoder.EncodeKey(value)); + } + return slices; + } + + // "slow" path + return values.Select(value => encoder.EncodeKey(value)); + } + + /// Convert an array of slices back into an array of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params Slice[] slices) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); + + var values = new T[slices.Length]; + for (int i = 0; i < slices.Length; i++) + { + values[i] = encoder.DecodeKey(slices[i]); + } + return values; + } + + /// Convert the keys of an array of key value pairs of slices back into an array of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] KeyValuePair[] items) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(items, nameof(items)); + + var values = new T[items.Length]; + for (int i = 0; i < items.Length; i++) + { + values[i] = encoder.DecodeKey(items[i].Key); + } + return values; + } + + /// Transform a sequence of slices back into a sequence of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static IEnumerable DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable slices) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); + + // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) + + return slices.Select(slice => encoder.DecodeKey(slice)); + } + + #endregion + + } +} diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs index c7abaa0a3..137cd8b86 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -44,233 +44,30 @@ public static partial class KeyValueEncoders public static class Tuples { - internal class TupleEncoder : IKeyEncoder, IValueEncoder - { - public static readonly TupleEncoder Default = new TupleEncoder(); - - private TupleEncoder() { } - - public IKeyEncoding Encoding => TuPack.Encoding; - - public void WriteKeyTo(ref SliceWriter writer, T key) - { - TupleEncoder.WriteKeysTo(ref writer, key); - } - - public void ReadKeyFrom(ref SliceReader reader, out T key) - { - key = !reader.HasMore - ? default //BUGBUG - : TuPack.DecodeKey(reader.ReadToEnd()); - } - - public Slice EncodeValue(T key) - { - return TupleEncoder.EncodeKey(default(Slice), key); - } - - public T DecodeValue(Slice encoded) - { - if (encoded.IsNullOrEmpty) return default; //BUGBUG - return TuPack.DecodeKey(encoded); - } - - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override IKeyEncoding Encoding => TuPack.Encoding; - - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) key) - { - switch (count) - { - case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; - case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; - default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); - } - } - - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) key) - { - if (count != 1 & count != 2) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); - - var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); - Contract.Assert(t != null); - key.Item1 = t.Get(0); - key.Item2 = count == 2 ? t.Get(1) : default; - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override IKeyEncoding Encoding => TuPack.Encoding; - - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) key) - { - switch (count) - { - case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; - case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; - case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; - default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); - } - } - - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) key) - { - if (count < 1 | count > 3) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); - - var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); - Contract.Assert(t != null); - key.Item1 = t.Get(0); - key.Item2 = count >= 2 ? t.Get(1) : default; - key.Item3 = count >= 3 ? t.Get(2) : default; - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override IKeyEncoding Encoding => TuPack.Encoding; - - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) key) - { - switch (count) - { - case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; - case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; - case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; - case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; - default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); - } - } - - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) key) - { - if (count < 1 || count > 4) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); - - var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); - Contract.Assert(t != null); - key.Item1 = t.Get(0); - key.Item2 = count >= 2 ? t.Get(1) : default; - key.Item3 = count >= 3 ? t.Get(2) : default; - key.Item4 = count >= 4 ? t.Get(3) : default; - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override IKeyEncoding Encoding => TuPack.Encoding; - - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) key) - { - switch (count) - { - case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; - case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; - case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; - case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; - case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; - default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); - } - } - - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) key) - { - if (count < 1 || count > 5) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); - - var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); - Contract.Assert(t != null); - key.Item1 = t.Get(0); - key.Item2 = count >= 2 ? t.Get(1) : default; - key.Item3 = count >= 3 ? t.Get(2) : default; - key.Item4 = count >= 4 ? t.Get(3) : default; - key.Item5 = count >= 5 ? t.Get(4) : default; - } - } - - internal class TupleCompositeEncoder : CompositeKeyEncoder - { - - public static readonly TupleCompositeEncoder Default = new TupleCompositeEncoder(); - - private TupleCompositeEncoder() { } - - public override IKeyEncoding Encoding => TuPack.Encoding; - - public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) key) - { - switch (count) - { - case 6: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5, key.Item6); break; - case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; - case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; - case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; - case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; - case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; - default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); - } - } - - public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) key) - { - if (count < 1 || count > 6) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); - - var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); - Contract.Assert(t != null); - key.Item1 = t.Get(0); - key.Item2 = count >= 2 ? t.Get(1) : default; - key.Item3 = count >= 3 ? t.Get(2) : default; - key.Item4 = count >= 4 ? t.Get(3) : default; - key.Item5 = count >= 5 ? t.Get(4) : default; - key.Item6 = count >= 6 ? t.Get(5) : default; - } - } - #region Keys [NotNull] public static IKeyEncoder Key() { - return TupleEncoder.Default; + return TupleEncoder.KeyEncoder.Default; } [NotNull] public static ICompositeKeyEncoder CompositeKey() { - return TupleCompositeEncoder.Default; + return TupleEncoder.CompositeEncoder.Default; } [NotNull] public static ICompositeKeyEncoder CompositeKey() { - return TupleCompositeEncoder.Default; + return TupleEncoder.CompositeEncoder.Default; } [NotNull] public static ICompositeKeyEncoder CompositeKey() { - return TupleCompositeEncoder.Default; + return TupleEncoder.CompositeEncoder.Default; } #endregion @@ -280,7 +77,7 @@ public static ICompositeKeyEncoder CompositeKey( [NotNull] public static IValueEncoder Value() { - return TupleEncoder.Default; + return TupleEncoder.KeyEncoder.Default; } #endregion diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs index 213f46e80..8bb34ba03 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs @@ -29,10 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Serialization.Encoders { using System; - using System.Collections.Generic; - using System.Linq; using System.Runtime.CompilerServices; - using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using JetBrains.Annotations; @@ -278,230 +275,7 @@ public static IKeyEncoder Bind([NotNull] Func encoder, [NotNull] return new Singleton(encoder, decoder); } - /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params T[] values) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(values, nameof(values)); - - var slices = new Slice[values.Length]; - for (int i = 0; i < values.Length; i++) - { - slices[i] = encoder.EncodeKey(values[i]); - } - return slices; - } - - /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable elements, Func selector) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(elements, nameof(elements)); - Contract.NotNull(selector, nameof(selector)); - - TElement[] arr; - ICollection coll; - - if ((arr = elements as TElement[]) != null) - { // fast path for arrays - return EncodeKeys(encoder, arr, selector); - } - if ((coll = elements as ICollection) != null) - { // we can pre-allocate the result array - var slices = new Slice[coll.Count]; - int p = 0; - foreach(var item in coll) - { - slices[p++] = encoder.EncodeKey(selector(item)); - } - return slices; - } - // slow path - return elements.Select((item) => encoder.EncodeKey(selector(item))).ToArray(); - } - - /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static Slice[] EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] TElement[] elements, Func selector) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(elements, nameof(elements)); - Contract.NotNull(selector, nameof(selector)); - - var slices = new Slice[elements.Length]; - for (int i = 0; i < elements.Length; i++) - { - slices[i] = encoder.EncodeKey(selector(elements[i])); - } - return slices; - } - - /// Transform a sequence of s into a sequence of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static IEnumerable EncodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable values) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(values, nameof(values)); - - // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed - - if (values is T[] array) - { // optimized path for arrays - return EncodeKeys(encoder, array); - } - - if (values is ICollection coll) - { // optimized path when we know the count - var slices = new List(coll.Count); - foreach (var value in coll) - { - slices.Add(encoder.EncodeKey(value)); - } - return slices; - } - - // "slow" path - return values.Select(value => encoder.EncodeKey(value)); - } - - /// Convert an array of slices back into an array of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] params Slice[] slices) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(slices, nameof(slices)); - - var values = new T[slices.Length]; - for (int i = 0; i < slices.Length; i++) - { - values[i] = encoder.DecodeKey(slices[i]); - } - return values; - } - - /// Convert the keys of an array of key value pairs of slices back into an array of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static T[] DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] KeyValuePair[] items) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(items, nameof(items)); - - var values = new T[items.Length]; - for (int i = 0; i < items.Length; i++) - { - values[i] = encoder.DecodeKey(items[i].Key); - } - return values; - } - - /// Transform a sequence of slices back into a sequence of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static IEnumerable DecodeKeys([NotNull] this IKeyEncoder encoder, [NotNull] IEnumerable slices) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(slices, nameof(slices)); - - // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) - - return slices.Select(slice => encoder.DecodeKey(slice)); - } - #endregion - #region Values... - - /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] params T[] values) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(values, nameof(values)); - - var slices = new Slice[values.Length]; - for (int i = 0; i < values.Length; i++) - { - slices[i] = encoder.EncodeValue(values[i]); - } - - return slices; - } - - /// Transform a sequence of s into a sequence of slices, using a serializer (or the default serializer if none is provided) - [NotNull] - public static IEnumerable EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable values) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(values, nameof(values)); - - // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed - - var array = values as T[]; - if (array != null) - { // optimized path for arrays - return EncodeValues(encoder, array); - } - - var coll = values as ICollection; - if (coll != null) - { // optimized path when we know the count - var slices = new List(coll.Count); - foreach (var value in coll) - { - slices.Add(encoder.EncodeValue(value)); - } - return slices; - } - - return values.Select(value => encoder.EncodeValue(value)); - } - - /// Convert an array of slices back into an array of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] params Slice[] slices) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(slices, nameof(slices)); - - var values = new T[slices.Length]; - for (int i = 0; i < slices.Length; i++) - { - values[i] = encoder.DecodeValue(slices[i]); - } - - return values; - } - - /// Convert the values of an array of key value pairs of slices back into an array of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] KeyValuePair[] items) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(items, nameof(items)); - - var values = new T[items.Length]; - for (int i = 0; i < items.Length; i++) - { - values[i] = encoder.DecodeValue(items[i].Value); - } - - return values; - } - - /// Transform a sequence of slices back into a sequence of s, using a serializer (or the default serializer if none is provided) - [NotNull] - public static IEnumerable DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable slices) - { - Contract.NotNull(encoder, nameof(encoder)); - Contract.NotNull(slices, nameof(slices)); - - // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) - - return slices.Select(slice => encoder.DecodeValue(slice)); - } - - #endregion } - } diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/ValueEncoderExtensions.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/ValueEncoderExtensions.cs new file mode 100644 index 000000000..e64f59a77 --- /dev/null +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/ValueEncoderExtensions.cs @@ -0,0 +1,133 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +using System; + +namespace Doxense.Serialization.Encoders +{ + using System; + using System.Collections.Generic; + using System.Linq; + using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; + + public static class ValueEncoderExtensions + { + + /// Convert an array of s into an array of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static Slice[] EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] params T[] values) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); + + var slices = new Slice[values.Length]; + for (int i = 0; i < values.Length; i++) + { + slices[i] = encoder.EncodeValue(values[i]); + } + + return slices; + } + + /// Transform a sequence of s into a sequence of slices, using a serializer (or the default serializer if none is provided) + [NotNull] + public static IEnumerable EncodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable values) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(values, nameof(values)); + + // note: T=>Slice usually is used for writing batches as fast as possible, which means that keys will be consumed immediately and don't need to be streamed + + var array = values as T[]; + if (array != null) + { // optimized path for arrays + return EncodeValues(encoder, array); + } + + var coll = values as ICollection; + if (coll != null) + { // optimized path when we know the count + var slices = new List(coll.Count); + foreach (var value in coll) + { + slices.Add(encoder.EncodeValue(value)); + } + return slices; + } + + return values.Select(value => encoder.EncodeValue(value)); + } + + /// Convert an array of slices back into an array of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] params Slice[] slices) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); + + var values = new T[slices.Length]; + for (int i = 0; i < slices.Length; i++) + { + values[i] = encoder.DecodeValue(slices[i]); + } + + return values; + } + + /// Convert the values of an array of key value pairs of slices back into an array of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static T[] DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] KeyValuePair[] items) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(items, nameof(items)); + + var values = new T[items.Length]; + for (int i = 0; i < items.Length; i++) + { + values[i] = encoder.DecodeValue(items[i].Value); + } + + return values; + } + + /// Transform a sequence of slices back into a sequence of s, using a serializer (or the default serializer if none is provided) + [NotNull] + public static IEnumerable DecodeValues([NotNull] this IValueEncoder encoder, [NotNull] IEnumerable slices) + { + Contract.NotNull(encoder, nameof(encoder)); + Contract.NotNull(slices, nameof(slices)); + + // Slice=>T may be filtered in LINQ queries, so we should probably stream the values (so no optimization needed) + + return slices.Select(slice => encoder.DecodeValue(slice)); + } + + } + +} diff --git a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs index 07e80e6fd..109d8eee6 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs @@ -33,6 +33,7 @@ namespace Doxense.Collections.Tuples.Encoding using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; + using Doxense.Serialization.Encoders; using JetBrains.Annotations; /// Helper class to encode and decode tuples to and from binary buffers @@ -986,6 +987,213 @@ public static bool DecodeNext(ref TupleReader input, out T value) #endregion + #region Encoders... + + internal class KeyEncoder : IKeyEncoder, IValueEncoder + { + public static readonly KeyEncoder Default = new KeyEncoder(); + + private KeyEncoder() { } + + public IKeyEncoding Encoding => TuPack.Encoding; + + public void WriteKeyTo(ref SliceWriter writer, T key) + { + TupleEncoder.WriteKeysTo(ref writer, key); + } + + public void ReadKeyFrom(ref SliceReader reader, out T key) + { + key = !reader.HasMore + ? default //BUGBUG + : TuPack.DecodeKey(reader.ReadToEnd()); + } + + public Slice EncodeValue(T key) + { + return TupleEncoder.EncodeKey(default(Slice), key); + } + + public T DecodeValue(Slice encoded) + { + if (encoded.IsNullOrEmpty) return default; //BUGBUG + return TuPack.DecodeKey(encoded); + } + + } + + internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + { + + public static readonly CompositeEncoder Default = new CompositeEncoder(); + + private CompositeEncoder() { } + + public override IKeyEncoding Encoding => TuPack.Encoding; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) key) + { + switch (count) + { + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) key) + { + if (count != 1 & count != 2) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be either 1 or 2"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count == 2 ? t.Get(1) : default; + } + } + + internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + { + + public static readonly CompositeEncoder Default = new CompositeEncoder(); + + private CompositeEncoder() { } + + public override IKeyEncoding Encoding => TuPack.Encoding; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) key) + { + switch (count) + { + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) key) + { + if (count < 1 | count > 3) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 3"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + } + } + + internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + { + + public static readonly CompositeEncoder Default = new CompositeEncoder(); + + private CompositeEncoder() { } + + public override IKeyEncoding Encoding => TuPack.Encoding; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) key) + { + switch (count) + { + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) key) + { + if (count < 1 || count > 4) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 4"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; + } + } + + internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + { + + public static readonly CompositeEncoder Default = new CompositeEncoder(); + + private CompositeEncoder() { } + + public override IKeyEncoding Encoding => TuPack.Encoding; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) key) + { + switch (count) + { + case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) key) + { + if (count < 1 || count > 5) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 5"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; + key.Item5 = count >= 5 ? t.Get(4) : default; + } + } + + internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + { + + public static readonly CompositeEncoder Default = new CompositeEncoder(); + + private CompositeEncoder() { } + + public override IKeyEncoding Encoding => TuPack.Encoding; + + public override void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) key) + { + switch (count) + { + case 6: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5, key.Item6); break; + case 5: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4, key.Item5); break; + case 4: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3, key.Item4); break; + case 3: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2, key.Item3); break; + case 2: TupleEncoder.WriteKeysTo(ref writer, key.Item1, key.Item2); break; + case 1: TupleEncoder.WriteKeysTo(ref writer, key.Item1); break; + default: throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); + } + } + + public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) key) + { + if (count < 1 || count > 6) throw new ArgumentOutOfRangeException(nameof(count), count, "Item count must be between 1 and 6"); + + var t = TuPack.Unpack(reader.ReadToEnd()).OfSize(count); + Contract.Assert(t != null); + key.Item1 = t.Get(0); + key.Item2 = count >= 2 ? t.Get(1) : default; + key.Item3 = count >= 3 ? t.Get(2) : default; + key.Item4 = count >= 4 ? t.Get(3) : default; + key.Item5 = count >= 5 ? t.Get(4) : default; + key.Item6 = count >= 6 ? t.Get(5) : default; + } + } + + #endregion + } } From d5b9f46129c7eaf3b78f62fdb85026a358c28ff8 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 19:18:58 +0200 Subject: [PATCH 138/153] Decouple KeyValueEncoders from the rest, introduce IValueEncoding and ITypeSystem to tie everything together - IValueEncoding add methods to get value encoders - ITypeSystem combines an IKeyEncoding and IValueEncoding - Rename all IKeyEncoding.GetEncoder() to GetKeyEncoder() to not conflict with value encoders - Refactor all sample layers to be more modern in the way they are configured. --- .../Encoders/ICompositeKeyEncoder.cs | 113 +++++++++++++++ .../Encoders/KeyValueEncoders.Ordered.cs | 41 +++--- .../Encoders/KeyValueEncoders.Tuples.cs | 7 +- .../TypeSystem/Encoders/KeyValueEncoders.cs | 132 ++---------------- .../Shared/TypeSystem/IKeyEncoding.cs | 17 ++- .../Shared/TypeSystem/ITypeSystem.cs | 39 ++++++ .../Shared/TypeSystem/IValueEncoding.cs | 44 ++++++ .../Subspaces/DynamicKeySubspace.cs | 2 +- FoundationDB.Client/Subspaces/KeySubspace.cs | 10 +- .../Subspaces/KeySubspaceExtensions.cs | 8 +- .../Tuples/Encoding/TupleEncoder.cs | 16 +-- .../Tuples/Encoding/TupleKeyEncoding.cs | 42 ++++-- FoundationDB.Client/Tuples/TuPack.cs | 2 +- .../Collections/FdbMap`2.cs | 48 +++---- .../Collections/FdbMultimap`2.cs | 40 +++--- .../Collections/FdbQueue`1.cs | 33 ++--- .../Collections/FdbVector`1.cs | 52 +++---- .../Counters/FdbCounterMap.cs | 35 +++-- .../Counters/FdbHighContentionCounter.cs | 5 +- .../Indexes/FdbIndex`2.cs | 49 +++---- .../Documents/FdbDocumentCollection.cs | 22 ++- .../Indexes/FdbCompressedBitmapIndex.cs | 29 ++-- FoundationDB.Tests/Layers/MapFacts.cs | 2 +- 23 files changed, 434 insertions(+), 354 deletions(-) create mode 100644 FoundationDB.Client/Shared/TypeSystem/ITypeSystem.cs create mode 100644 FoundationDB.Client/Shared/TypeSystem/IValueEncoding.cs diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs index db16363f3..4a720cf3d 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/ICompositeKeyEncoder.cs @@ -29,6 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Serialization.Encoders { using System; + using System.Runtime.CompilerServices; using Doxense.Memory; public interface ICompositeKeyEncoder : IKeyEncoder<(T1, T2)> @@ -75,4 +76,116 @@ public interface ICompositeKeyEncoder : IKeyEncoder<(T1, /// Read some or all parts of a composite key void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); } + + /// Wrapper for encoding and decoding a pair with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) items); + + [MethodImpl(MethodImplOptions.AggressiveInlining)] + public void WriteKeyTo(ref SliceWriter writer, (T1, T2) items) + { + WriteKeyPartsTo(ref writer, 2, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2) items) + { + ReadKeyPartsFrom(ref reader, 2, out items); + } + + } + + /// Wrapper for encoding and decoding a triplet with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) items); + + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3) items) + { + WriteKeyPartsTo(ref writer, 3, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3) items) + { + ReadKeyPartsFrom(ref reader, 3, out items); + } + + } + + /// Wrapper for encoding and decoding a quad with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) items); + + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4) items) + { + WriteKeyPartsTo(ref writer, 4, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4) items) + { + ReadKeyPartsFrom(ref reader, 4, out items); + } + + } + + /// Wrapper for encoding and decoding five items with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) items); + + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5) items) + { + WriteKeyPartsTo(ref writer, 5, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5) items) + { + ReadKeyPartsFrom(ref reader, 5, out items); + } + + } + + /// Wrapper for encoding and decoding six items with lambda functions + public abstract class CompositeKeyEncoder : ICompositeKeyEncoder + { + + public abstract IKeyEncoding Encoding { get; } + + public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) items); + + public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); + + public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5, T6) items) + { + WriteKeyPartsTo(ref writer, 6, ref items); + } + + public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5, T6) items) + { + ReadKeyPartsFrom(ref reader, 6, out items); + } + + } + } diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs index efeab16ff..56b09d110 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Ordered.cs @@ -30,7 +30,6 @@ namespace Doxense.Serialization.Encoders { using JetBrains.Annotations; using System; - using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; @@ -85,19 +84,19 @@ public void ReadKeyFrom(ref SliceReader reader, out T key) #region IKeyEncoding... - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - IKeyEncoder IKeyEncoding.GetEncoder() + IKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1) != typeof(T)) throw new NotSupportedException(); return (IKeyEncoder) (object) this; } - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); #endregion @@ -134,19 +133,19 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 #region IKeyEncoding... - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + IKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2)) throw new NotSupportedException(); return (ICompositeKeyEncoder) (object) this; } - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); #endregion @@ -187,19 +186,19 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 #region IKeyEncoding... - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + IKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2) && typeof(T3B) != typeof(T3)) throw new NotSupportedException(); return (ICompositeKeyEncoder) (object) this; } - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); #endregion } @@ -242,15 +241,15 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 #region IKeyEncoding... - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - IKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + IKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1B) != typeof(T1) && typeof(T2B) != typeof(T2) && typeof(T3B) != typeof(T3) && typeof(T4B) != typeof(T4)) throw new NotSupportedException(); return (ICompositeKeyEncoder) (object) this; diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs index 137cd8b86..acb13ea7b 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.Tuples.cs @@ -29,10 +29,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Serialization.Encoders { using System; - using Doxense.Collections.Tuples; using Doxense.Collections.Tuples.Encoding; - using Doxense.Diagnostics.Contracts; - using Doxense.Memory; using JetBrains.Annotations; /// Helper class for all key/value encoders @@ -49,7 +46,7 @@ public static class Tuples [NotNull] public static IKeyEncoder Key() { - return TupleEncoder.KeyEncoder.Default; + return TupleEncoder.Encoder.Default; } [NotNull] @@ -77,7 +74,7 @@ public static ICompositeKeyEncoder CompositeKey( [NotNull] public static IValueEncoder Value() { - return TupleEncoder.KeyEncoder.Default; + return TupleEncoder.Encoder.Default; } #endregion diff --git a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs index 8bb34ba03..a3b8c84b6 100644 --- a/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs +++ b/FoundationDB.Client/Shared/TypeSystem/Encoders/KeyValueEncoders.cs @@ -29,7 +29,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace Doxense.Serialization.Encoders { using System; - using System.Runtime.CompilerServices; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using JetBrains.Annotations; @@ -75,19 +74,19 @@ public Slice DecodeValue(Slice encoded) #endregion - IKeyEncoder IKeyEncoding.GetEncoder() + IKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1) != typeof(Slice)) throw new NotSupportedException(); return (IKeyEncoder) (object) this; } - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); } @@ -132,130 +131,19 @@ public T DecodeValue(Slice encoded) public IKeyEncoding Encoding => this; - IKeyEncoder IKeyEncoding.GetEncoder() + IKeyEncoder IKeyEncoding.GetKeyEncoder() { if (typeof(T1) != typeof(T)) throw new NotSupportedException(); return (IKeyEncoder) (object) this; } - IDynamicKeyEncoder IKeyEncoding.GetDynamicEncoder() => throw new NotSupportedException(); + IDynamicKeyEncoder IKeyEncoding.GetDynamicKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); - - ICompositeKeyEncoder IKeyEncoding.GetEncoder() => throw new NotSupportedException(); - } - - /// Wrapper for encoding and decoding a pair with lambda functions - public abstract class CompositeKeyEncoder : ICompositeKeyEncoder - { - - public abstract IKeyEncoding Encoding { get; } - - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2) items); - - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2) items); - - [MethodImpl(MethodImplOptions.AggressiveInlining)] - public void WriteKeyTo(ref SliceWriter writer, (T1, T2) items) - { - WriteKeyPartsTo(ref writer, 2, ref items); - } - - public void ReadKeyFrom(ref SliceReader reader, out (T1, T2) items) - { - ReadKeyPartsFrom(ref reader, 2, out items); - } - - } - - /// Wrapper for encoding and decoding a triplet with lambda functions - public abstract class CompositeKeyEncoder : ICompositeKeyEncoder - { - - public abstract IKeyEncoding Encoding { get; } - - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3) items); - - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3) items); - - public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3) items) - { - WriteKeyPartsTo(ref writer, 3, ref items); - } - - public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3) items) - { - ReadKeyPartsFrom(ref reader, 3, out items); - } - - } - - /// Wrapper for encoding and decoding a quad with lambda functions - public abstract class CompositeKeyEncoder : ICompositeKeyEncoder - { - - public abstract IKeyEncoding Encoding { get; } - - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4) items); - - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4) items); - - public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4) items) - { - WriteKeyPartsTo(ref writer, 4, ref items); - } - - public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4) items) - { - ReadKeyPartsFrom(ref reader, 4, out items); - } - - } - - /// Wrapper for encoding and decoding five items with lambda functions - public abstract class CompositeKeyEncoder : ICompositeKeyEncoder - { - - public abstract IKeyEncoding Encoding { get; } - - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5) items); - - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5) items); - - public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5) items) - { - WriteKeyPartsTo(ref writer, 5, ref items); - } - - public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5) items) - { - ReadKeyPartsFrom(ref reader, 5, out items); - } - - } - - /// Wrapper for encoding and decoding six items with lambda functions - public abstract class CompositeKeyEncoder : ICompositeKeyEncoder - { - - public abstract IKeyEncoding Encoding { get; } - - public abstract void WriteKeyPartsTo(ref SliceWriter writer, int count, ref (T1, T2, T3, T4, T5, T6) items); - - public abstract void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1, T2, T3, T4, T5, T6) items); - - public void WriteKeyTo(ref SliceWriter writer, (T1, T2, T3, T4, T5, T6) items) - { - WriteKeyPartsTo(ref writer, 6, ref items); - } - - public void ReadKeyFrom(ref SliceReader reader, out (T1, T2, T3, T4, T5, T6) items) - { - ReadKeyPartsFrom(ref reader, 6, out items); - } + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); + ICompositeKeyEncoder IKeyEncoding.GetKeyEncoder() => throw new NotSupportedException(); } #endregion diff --git a/FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs b/FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs index 682788d93..0d7e4a3b6 100644 --- a/FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs +++ b/FoundationDB.Client/Shared/TypeSystem/IKeyEncoding.cs @@ -26,11 +26,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using System; -using JetBrains.Annotations; - namespace Doxense.Serialization.Encoders { + using System; + using JetBrains.Annotations; + /// Type system that handles encoding and decoding of differnt types of keys /// /// An implementation of this interface knows to create different types of Key Encoders that will all use the same "binary format" to encode and decode keys of various shapes. @@ -43,14 +43,14 @@ public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTyp /// Encoder that encodes dynamic keys /// If this encoding does not support dynamic keys [NotNull] - IDynamicKeyEncoder GetDynamicEncoder(); + IDynamicKeyEncoder GetDynamicKeyEncoder(); /// Returns an encoder which can process keys composed of a single element of a fixed type /// Type of the element to encode /// Key encoder /// If this encoding does not support static keys [NotNull] - IKeyEncoder GetEncoder(); + IKeyEncoder GetKeyEncoder(); /// Returns an encoder which can process keys composed of a two elements of fixed types /// Type of the first element to encode @@ -58,7 +58,7 @@ public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTyp /// Composite key encoder /// If this encoding does not support static keys of size 2 [NotNull] - ICompositeKeyEncoder GetEncoder(); + ICompositeKeyEncoder GetKeyEncoder(); /// Returns an encoder which can process keys composed of a three elements of fixed types /// Type of the first element to encode @@ -67,7 +67,7 @@ public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTyp /// Composite key encoder /// If this encoding does not support static keys of size 3 [NotNull] - ICompositeKeyEncoder GetEncoder(); + ICompositeKeyEncoder GetKeyEncoder(); /// Returns an encoder which can process keys composed of a four elements of fixed types /// Type of the first element to encode @@ -77,8 +77,7 @@ public interface IKeyEncoding //REVIEW: rename to "IKeyEncodingScheme"? "IKeyTyp /// Composite key encoder /// If this encoding does not support static keys of size 4 [NotNull] - ICompositeKeyEncoder GetEncoder(); + ICompositeKeyEncoder GetKeyEncoder(); } - } diff --git a/FoundationDB.Client/Shared/TypeSystem/ITypeSystem.cs b/FoundationDB.Client/Shared/TypeSystem/ITypeSystem.cs new file mode 100644 index 000000000..ccb10d5cf --- /dev/null +++ b/FoundationDB.Client/Shared/TypeSystem/ITypeSystem.cs @@ -0,0 +1,39 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Serialization.Encoders +{ + using System; + using JetBrains.Annotations; + + public interface ITypeSystem : IKeyEncoding, IValueEncoding + { + [NotNull] + string Name { get; } + } +} diff --git a/FoundationDB.Client/Shared/TypeSystem/IValueEncoding.cs b/FoundationDB.Client/Shared/TypeSystem/IValueEncoding.cs new file mode 100644 index 000000000..069517dfc --- /dev/null +++ b/FoundationDB.Client/Shared/TypeSystem/IValueEncoding.cs @@ -0,0 +1,44 @@ +#region BSD Licence +/* Copyright (c) 2013-2018, 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 + +namespace Doxense.Serialization.Encoders +{ + using System; + using JetBrains.Annotations; + + public interface IValueEncoding + { + /// Returns an encoder which can process values of a fixed type + /// Type of the element to encode + /// Value encoder + [NotNull] + IValueEncoder GetValueEncoder(); + + //TODO: DynamicValueValue! + } +} diff --git a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs index 49e00ef56..577546ba1 100644 --- a/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs +++ b/FoundationDB.Client/Subspaces/DynamicKeySubspace.cs @@ -55,7 +55,7 @@ internal DynamicKeySubspace(Slice prefix, [NotNull] IKeyEncoding encoding) { Contract.Requires(encoding != null); this.Encoding = encoding; - this.KeyEncoder = encoding.GetDynamicEncoder(); + this.KeyEncoder = encoding.GetDynamicKeyEncoder(); this.Keys = new DynamicKeys(this, this.KeyEncoder); this.Partition = new DynamicPartition(this); } diff --git a/FoundationDB.Client/Subspaces/KeySubspace.cs b/FoundationDB.Client/Subspaces/KeySubspace.cs index 9618256e1..78d904c22 100644 --- a/FoundationDB.Client/Subspaces/KeySubspace.cs +++ b/FoundationDB.Client/Subspaces/KeySubspace.cs @@ -76,14 +76,14 @@ public static DynamicKeySubspace CreateDynamic(Slice prefix, [NotNull] IDynamicK [Pure, NotNull] public static DynamicKeySubspace CreateDynamic(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new DynamicKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetDynamicEncoder()); + return new DynamicKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetDynamicKeyEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . /// A subspace that can handle keys of type . public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -98,7 +98,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, [NotNull] IKeyE /// A subspace that can handle composite keys of type (, ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -113,7 +113,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, [NotNul /// A subspace that can handle composite keys of type (, , ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . @@ -128,7 +128,7 @@ public static TypedKeySubspace CreateTyped(Slice prefix, /// A subspace that can handle composite keys of type (, , ). public static TypedKeySubspace CreateTyped(Slice prefix, [CanBeNull] IKeyEncoding encoding = null) { - return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(prefix, (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Initializes a new subspace with the given binary , that uses a typed key . diff --git a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs index 9600dbed4..54e5f3fcd 100644 --- a/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs +++ b/FoundationDB.Client/Subspaces/KeySubspaceExtensions.cs @@ -62,7 +62,7 @@ public static IDynamicKeySubspace AsDynamic([NotNull] this IKeySubspace subspace public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -73,7 +73,7 @@ public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspa public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -84,7 +84,7 @@ public static ITypedKeySubspace AsTyped([NotNull] this IKeySubsp public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetKeyEncoder()); } /// Return a version of this subspace, which uses a different type system to produces the keys and values @@ -95,7 +95,7 @@ public static ITypedKeySubspace AsTyped([NotNull] this I public static ITypedKeySubspace AsTyped([NotNull] this IKeySubspace subspace, [CanBeNull] IKeyEncoding encoding = null) { Contract.NotNull(subspace, nameof(subspace)); - return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetEncoder()); + return new TypedKeySubspace(subspace.GetPrefix(), (encoding ?? TuPack.Encoding).GetKeyEncoder()); } diff --git a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs index 109d8eee6..ce40f4bfa 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleEncoder.cs @@ -989,11 +989,11 @@ public static bool DecodeNext(ref TupleReader input, out T value) #region Encoders... - internal class KeyEncoder : IKeyEncoder, IValueEncoder + internal class Encoder : IKeyEncoder, IValueEncoder { - public static readonly KeyEncoder Default = new KeyEncoder(); + public static readonly Encoder Default = new Encoder(); - private KeyEncoder() { } + private Encoder() { } public IKeyEncoding Encoding => TuPack.Encoding; @@ -1022,7 +1022,7 @@ public T DecodeValue(Slice encoded) } - internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + internal class CompositeEncoder : CompositeKeyEncoder { public static readonly CompositeEncoder Default = new CompositeEncoder(); @@ -1052,7 +1052,7 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 } } - internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + internal class CompositeEncoder : CompositeKeyEncoder { public static readonly CompositeEncoder Default = new CompositeEncoder(); @@ -1084,7 +1084,7 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 } } - internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + internal class CompositeEncoder : CompositeKeyEncoder { public static readonly CompositeEncoder Default = new CompositeEncoder(); @@ -1118,7 +1118,7 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 } } - internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + internal class CompositeEncoder : CompositeKeyEncoder { public static readonly CompositeEncoder Default = new CompositeEncoder(); @@ -1154,7 +1154,7 @@ public override void ReadKeyPartsFrom(ref SliceReader reader, int count, out (T1 } } - internal class CompositeEncoder : KeyValueEncoders.CompositeKeyEncoder + internal class CompositeEncoder : CompositeKeyEncoder { public static readonly CompositeEncoder Default = new CompositeEncoder(); diff --git a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs index c210853df..f7df06283 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleKeyEncoding.cs @@ -32,34 +32,56 @@ namespace Doxense.Collections.Tuples.Encoding using Doxense.Serialization.Encoders; /// Encoding that uses the Tuple Binary Encoding format - public sealed class TupleKeyEncoding : IKeyEncoding + public sealed class TupleKeyEncoding : ITypeSystem { public static readonly TupleKeyEncoding Instance = new TupleKeyEncoding(); + + public string Name => "TuPack"; - public IDynamicKeyEncoder GetDynamicEncoder() + #region Keys... + + public IDynamicKeyEncoder GetDynamicKeyEncoder() { return TupleKeyEncoder.Instance; } - public IKeyEncoder GetEncoder() + public IKeyEncoder GetKeyEncoder() + { + return TupleEncoder.Encoder.Default; + } + + public ICompositeKeyEncoder GetKeyEncoder() + { + return TupleEncoder.CompositeEncoder.Default; + } + + public ICompositeKeyEncoder GetKeyEncoder() + { + return TupleEncoder.CompositeEncoder.Default; + } + + public ICompositeKeyEncoder GetKeyEncoder() { - return KeyValueEncoders.Tuples.Key(); + return TupleEncoder.CompositeEncoder.Default; } - public ICompositeKeyEncoder GetEncoder() + public ICompositeKeyEncoder GetEncoder() { - return KeyValueEncoders.Tuples.CompositeKey(); + return TupleEncoder.CompositeEncoder.Default; } - public ICompositeKeyEncoder GetEncoder() + public ICompositeKeyEncoder GetEncoder() { - return KeyValueEncoders.Tuples.CompositeKey(); + return TupleEncoder.CompositeEncoder.Default; } - public ICompositeKeyEncoder GetEncoder() + #endregion + + public IValueEncoder GetValueEncoder() { - return KeyValueEncoders.Tuples.CompositeKey(); + return TupleEncoder.Encoder.Default; } + } } diff --git a/FoundationDB.Client/Tuples/TuPack.cs b/FoundationDB.Client/Tuples/TuPack.cs index 2483688e4..6788bab1d 100644 --- a/FoundationDB.Client/Tuples/TuPack.cs +++ b/FoundationDB.Client/Tuples/TuPack.cs @@ -45,7 +45,7 @@ public static class TuPack /// Key Encoding that use the Tuple Binary Encoding [NotNull] - public static IKeyEncoding Encoding => TupleKeyEncoding.Instance; + public static ITypeSystem Encoding => TupleKeyEncoding.Instance; #region Packing... diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index f4a7eadc2..9b3fdebe3 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -45,19 +45,17 @@ public class FdbMap { public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] IValueEncoder valueEncoder) - : this(name, subspace, KeyValueEncoders.Tuples.Key(), valueEncoder) + : this(name, subspace.AsTyped(), valueEncoder) { } - public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder keyEncoder, [NotNull] IValueEncoder valueEncoder) + public FdbMap([NotNull] string name, [NotNull] ITypedKeySubspace subspace, [NotNull] IValueEncoder valueEncoder) { if (name == null) throw new ArgumentNullException(nameof(name)); if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); this.Name = name; this.Subspace = subspace; - this.Location = subspace.UsingEncoder(keyEncoder); this.ValueEncoder = valueEncoder; } @@ -65,16 +63,16 @@ public FdbMap([NotNull] string name, [NotNull] IKeySubspace subspace, [NotNull] /// Name of the map // REVIEW: do we really need this property? - public string Name { [NotNull] get; private set; } - - /// Subspace used as a prefix for all items in this map - public IKeySubspace Subspace { [NotNull] get; private set; } + [NotNull] + public string Name { get; } /// Subspace used to encoded the keys for the items - protected ITypedKeySubspace Location { [NotNull] get; private set; } + [NotNull] + public ITypedKeySubspace Subspace { get; } /// Class that can serialize/deserialize values into/from slices - public IValueEncoder ValueEncoder { [NotNull] get; private set; } + [NotNull] + public IValueEncoder ValueEncoder { get; } #endregion @@ -91,7 +89,7 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var data = await trans.GetAsync(this.Location.Keys[id]).ConfigureAwait(false); + var data = await trans.GetAsync(this.Subspace.Keys[id]).ConfigureAwait(false); if (data.IsNull) throw new KeyNotFoundException("The given id was not present in the map."); return this.ValueEncoder.DecodeValue(data); @@ -106,7 +104,7 @@ public async Task> TryGetAsync([NotNull] IFdbReadOnlyTransactio if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var data = await trans.GetAsync(this.Location.Keys[id]).ConfigureAwait(false); + var data = await trans.GetAsync(this.Subspace.Keys[id]).ConfigureAwait(false); if (data.IsNull) return default(Optional); return this.ValueEncoder.DecodeValue(data); @@ -122,7 +120,7 @@ public void Set([NotNull] IFdbTransaction trans, TKey id, TValue value) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - trans.Set(this.Location.Keys[id], this.ValueEncoder.EncodeValue(value)); + trans.Set(this.Subspace.Keys[id], this.ValueEncoder.EncodeValue(value)); } /// Remove a single entry from the map @@ -134,7 +132,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey id) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - trans.Clear(this.Location.Keys[id]); + trans.Clear(this.Subspace.Keys[id]); } /// Create a query that will attempt to read all the entries in the map within a single transaction. @@ -147,7 +145,7 @@ public IAsyncEnumerable> All([NotNull] IFdbReadOnlyTr if (trans == null) throw new ArgumentNullException(nameof(trans)); return trans - .GetRange(this.Location.ToRange(), options) + .GetRange(this.Subspace.ToRange(), options) .Select(this.DecodeItem); } @@ -160,7 +158,7 @@ public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTrans if (trans == null) throw new ArgumentNullException(nameof(trans)); if (ids == null) throw new ArgumentNullException(nameof(ids)); - var results = await trans.GetValuesAsync(ids.Select(id => this.Location.Keys[id])).ConfigureAwait(false); + var results = await trans.GetValuesAsync(ids.Select(id => this.Subspace.Keys[id])).ConfigureAwait(false); return Optional.DecodeRange(this.ValueEncoder, results); } @@ -172,7 +170,7 @@ public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTrans private KeyValuePair DecodeItem(KeyValuePair item) { return new KeyValuePair( - this.Location.Keys.Decode(item.Key), + this.Subspace.Keys.Decode(item.Key), this.ValueEncoder.DecodeValue(item.Value) ); } @@ -182,7 +180,7 @@ private KeyValuePair[] DecodeItems(KeyValuePair[] ba { Contract.Requires(batch != null); - var keyEncoder = this.Location.Keys; + var keyEncoder = this.Subspace.Keys; var valueEncoder = this.ValueEncoder; var items = new KeyValuePair[batch.Length]; @@ -203,7 +201,7 @@ public void Clear([NotNull] IFdbTransaction trans) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.ClearRange(this.Location.ToRange()); + trans.ClearRange(this.Subspace.ToRange()); } #region Export... @@ -221,7 +219,7 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action { foreach (var item in batch) @@ -247,7 +245,7 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func { foreach (var item in batch) @@ -272,7 +270,7 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Action { if (batch.Length > 0) @@ -298,7 +296,7 @@ public Task ExportAsync([NotNull] IFdbDatabase db, [NotNull] Func handler(DecodeItems(batch), tok), ct ); @@ -324,7 +322,7 @@ public async Task AggregateAsync([NotNull] IFdbDatabase db, Fu await Fdb.Bulk.ExportAsync( db, - this.Location.ToRange(), + this.Subspace.ToRange(), (batch, _, __) => { state = handler(state, DecodeItems(batch)); @@ -357,7 +355,7 @@ public async Task AggregateAsync([NotNull] IFdbDatabas await Fdb.Bulk.ExportAsync( db, - this.Location.ToRange(), + this.Subspace.ToRange(), (batch, _, __) => { state = handler(state, DecodeItems(batch)); diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index 78d4cda08..63d3f0a39 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -57,36 +57,30 @@ public class FdbMultiMap /// Location where the map will be stored in the database /// If true, allow negative or zero values to stay in the map. public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues) - : this(subspace, allowNegativeValues, KeyValueEncoders.Tuples.CompositeKey()) + : this(subspace.AsTyped(), allowNegativeValues) { } /// Create a new multimap, using a specific key and value encoder /// Location where the map will be stored in the database /// If true, allow negative or zero values to stay in the map. /// Encoder for the key/value pairs - public FdbMultiMap(IKeySubspace subspace, bool allowNegativeValues, ICompositeKeyEncoder encoder) + public FdbMultiMap(ITypedKeySubspace subspace, bool allowNegativeValues) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - this.Subspace = subspace; this.AllowNegativeValues = allowNegativeValues; - this.Location = subspace.UsingEncoder(encoder); + this.Subspace = subspace; } #region Public Properties... - /// Subspace used as a prefix for all items in this map - [NotNull] - public IKeySubspace Subspace { get; } + /// Subspace used to encoded the keys for the items + [NotNull] + public ITypedKeySubspace Subspace { get; } /// If true, allow negative or zero values to stay in the map. public bool AllowNegativeValues { get; } - /// Subspace used to encoded the keys for the items - [NotNull] - protected ITypedKeySubspace Location { get; } - #endregion #region Add / Subtract / Remove... @@ -102,7 +96,7 @@ public Task AddAsync([NotNull] IFdbTransaction trans, TKey key, TValue value) //note: this method does not need to be async, but subtract is, so it's better if both methods have the same shape. if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.AtomicAdd(this.Location.Keys[key, value], PlusOne); + trans.AtomicAdd(this.Subspace.Keys[key, value], PlusOne); return Task.CompletedTask; } @@ -115,7 +109,7 @@ public async Task SubtractAsync([NotNull] IFdbTransaction trans, TKey key, TValu { if (trans == null) throw new ArgumentNullException(nameof(trans)); - Slice k = this.Location.Keys[key, value]; + Slice k = this.Subspace.Keys[key, value]; if (this.AllowNegativeValues) { trans.AtomicAdd(k, MinusOne); @@ -143,7 +137,7 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var v = await trans.GetAsync(this.Location.Keys[key, value]).ConfigureAwait(false); + var v = await trans.GetAsync(this.Subspace.Keys[key, value]).ConfigureAwait(false); return this.AllowNegativeValues ? v.IsPresent : v.ToInt64() > 0; } @@ -157,7 +151,7 @@ public async Task ContainsAsync([NotNull] IFdbReadOnlyTransaction trans, T { if (trans == null) throw new ArgumentNullException(nameof(trans)); - Slice v = await trans.GetAsync(this.Location.Keys[key, value]).ConfigureAwait(false); + Slice v = await trans.GetAsync(this.Subspace.Keys[key, value]).ConfigureAwait(false); if (v.IsNullOrEmpty) return null; long c = v.ToInt64(); return this.AllowNegativeValues || c > 0 ? c : default(long?); @@ -172,19 +166,19 @@ public IAsyncEnumerable Get([NotNull] IFdbReadOnlyTransaction trans, TKe { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var range = KeyRange.StartsWith(this.Location.Keys.EncodePartial(key)); + var range = KeyRange.StartsWith(this.Subspace.Keys.EncodePartial(key)); if (this.AllowNegativeValues) { return trans .GetRange(range) - .Select(kvp => this.Location.Keys.Decode(kvp.Key).Item2); + .Select(kvp => this.Subspace.Keys.Decode(kvp.Key).Item2); } else { return trans .GetRange(range) .Where(kvp => kvp.Value.ToInt64() > 0) // we need to filter out zero or negative values (possible artefacts) - .Select(kvp => this.Location.Keys.Decode(kvp.Key).Item2); + .Select(kvp => this.Subspace.Keys.Decode(kvp.Key).Item2); } } @@ -204,11 +198,11 @@ public Task> GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey [NotNull] public IAsyncEnumerable<(TValue Value, long Count)> GetCounts([NotNull] IFdbReadOnlyTransaction trans, TKey key) { - var range = KeyRange.StartsWith(this.Location.Keys.EncodePartial(key)); + var range = KeyRange.StartsWith(this.Subspace.Keys.EncodePartial(key)); var query = trans .GetRange(range) - .Select(kvp => (Value: this.Location.Keys.Decode(kvp.Key).Item2, Count: kvp.Value.ToInt64())); + .Select(kvp => (Value: this.Subspace.Keys.Decode(kvp.Key).Item2, Count: kvp.Value.ToInt64())); return this.AllowNegativeValues ? query @@ -233,7 +227,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.ClearRange(KeyRange.StartsWith(this.Location.Keys.EncodePartial(key))); + trans.ClearRange(KeyRange.StartsWith(this.Subspace.Keys.EncodePartial(key))); } /// Remove a value for a specific key @@ -245,7 +239,7 @@ public void Remove([NotNull] IFdbTransaction trans, TKey key, TValue value) { if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.Clear(this.Location.Keys[key, value]); + trans.Clear(this.Subspace.Keys[key, value]); } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index d1845c587..aa6d8c997 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -32,6 +32,7 @@ namespace FoundationDB.Layers.Collections using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Serialization.Encoders; using FoundationDB.Client; #if DEBUG @@ -49,32 +50,26 @@ public class FdbQueue // TODO: should we use a PRNG ? If two counter instances are created at the same moment, they could share the same seed ? private readonly Random Rng = new Random(); - /// Create a new High Contention Queue - /// Subspace where the queue will be stored - /// Uses the default Tuple serializer - public FdbQueue([NotNull] KeySubspace subspace) - : this(subspace, highContention: true, encoder: KeyValueEncoders.Tuples.Value()) - { } - /// Create a new queue using either High Contention mode or Simple mode /// Subspace where the queue will be stored /// If true, uses High Contention Mode (lots of popping clients). If true, uses the Simple Mode (a few popping clients). + /// Encoder for the values stored in this queue /// Uses the default Tuple serializer - public FdbQueue([NotNull] KeySubspace subspace, bool highContention) - : this(subspace, highContention: highContention, encoder: KeyValueEncoders.Tuples.Value()) + public FdbQueue([NotNull] IKeySubspace subspace, bool highContention = true, IValueEncoder encoder = null) + : this(subspace.AsDynamic(), highContention, encoder) { } /// Create a new queue using either High Contention mode or Simple mode /// Subspace where the queue will be stored /// If true, uses High Contention Mode (lots of popping clients). If true, uses the Simple Mode (a few popping clients). - public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] IValueEncoder encoder) + /// Encoder for the values stored in this queue + public FdbQueue([NotNull] IDynamicKeySubspace subspace, bool highContention = false, IValueEncoder encoder = null) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - if (encoder == null) throw new ArgumentNullException(nameof(encoder)); - this.Subspace = subspace.AsDynamic(); + this.Subspace = subspace; this.HighContention = highContention; - this.Encoder = encoder; + this.Encoder = encoder ?? TuPack.Encoding.GetValueEncoder(); //TODO: rewrite this, using FdbEncoderSubpsace<..> ! this.ConflictedPop = this.Subspace.Partition.ByKey(Slice.FromStringAscii("pop")); @@ -83,19 +78,19 @@ public FdbQueue([NotNull] IKeySubspace subspace, bool highContention, [NotNull] } /// Subspace used as a prefix for all items in this table - public IDynamicKeySubspace Subspace { [NotNull] get; private set; } + public IDynamicKeySubspace Subspace { [NotNull] get; } /// If true, the queue is operating in High Contention mode that will scale better with a lot of popping clients. - public bool HighContention { get; private set; } + public bool HighContention { get; } /// Serializer for the elements of the queue - public IValueEncoder Encoder { [NotNull] get; private set; } + public IValueEncoder Encoder { [NotNull] get; } - internal IDynamicKeySubspace ConflictedPop { get; private set; } + internal IDynamicKeySubspace ConflictedPop { get; } - internal IDynamicKeySubspace ConflictedItem { get; private set; } + internal IDynamicKeySubspace ConflictedItem { get; } - internal IDynamicKeySubspace QueueItem { get; private set; } + internal IDynamicKeySubspace QueueItem { get; } /// Remove all items from the queue. public void Clear([NotNull] IFdbTransaction trans) diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index 439b5e12a..f3879a705 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -31,6 +31,7 @@ namespace FoundationDB.Layers.Collections using System; using System.Linq; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Linq; using Doxense.Serialization.Encoders; @@ -60,39 +61,40 @@ public class FdbVector /// Create a new sparse Vector /// Subspace where the vector will be stored - /// Sparse entries will be assigned the value Slice.Empty - public FdbVector([NotNull] KeySubspace subspace) - : this(subspace, default(T)) + /// Default value for sparse entries + /// Encoder used for the values of this vector + public FdbVector([NotNull] IKeySubspace subspace, T defaultValue = default(T), IValueEncoder encoder = null) + : this(subspace.AsDynamic(), defaultValue, encoder) { } + /// Create a new sparse Vector /// Subspace where the vector will be stored /// Default value for sparse entries - public FdbVector([NotNull] IKeySubspace subspace, T defaultValue) - : this(subspace, defaultValue, KeyValueEncoders.Tuples.Value()) - { } - public FdbVector([NotNull] IKeySubspace subspace, T defaultValue, [NotNull] IValueEncoder encoder) + /// Encoder used for the values of this vector + public FdbVector([NotNull] IDynamicKeySubspace subspace, T defaultValue, IValueEncoder encoder = null) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - this.Subspace = subspace.AsDynamic(); + this.Subspace = subspace; this.DefaultValue = defaultValue; - this.Encoder = encoder; + this.Encoder = encoder ?? TuPack.Encoding.GetValueEncoder(); } /// Subspace used as a prefix for all items in this vector - public IDynamicKeySubspace Subspace { [NotNull] get; private set; } + [NotNull] + public IDynamicKeySubspace Subspace { get; } /// Default value for sparse entries - public T DefaultValue { get; private set; } + public T DefaultValue { get; } - public IValueEncoder Encoder { [NotNull] get; private set; } + [NotNull] + public IValueEncoder Encoder { get; } /// Get the number of items in the Vector. This number includes the sparsely represented items. public Task SizeAsync([NotNull] IFdbReadOnlyTransaction tr) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); return ComputeSizeAsync(tr); } @@ -100,7 +102,7 @@ public Task SizeAsync([NotNull] IFdbReadOnlyTransaction tr) /// Push a single item onto the end of the Vector. public async Task PushAsync([NotNull] IFdbTransaction tr, T value) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); var size = await ComputeSizeAsync(tr).ConfigureAwait(false); @@ -110,7 +112,7 @@ public async Task PushAsync([NotNull] IFdbTransaction tr, T value) /// Get the value of the last item in the Vector. public Task BackAsync([NotNull] IFdbReadOnlyTransaction tr) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); return tr .GetRange(this.Subspace.Keys.ToRange()) @@ -127,7 +129,7 @@ public Task FrontAsync([NotNull] IFdbReadOnlyTransaction tr) /// Get and pops the last item off the Vector. public async Task> PopAsync([NotNull] IFdbTransaction tr) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); var keyRange = this.Subspace.Keys.ToRange(); @@ -162,7 +164,7 @@ public async Task> PopAsync([NotNull] IFdbTransaction tr) /// Swap the items at positions i1 and i2. public async Task SwapAsync([NotNull] IFdbTransaction tr, long index1, long index2) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); if (index1 < 0 || index2 < 0) throw new IndexOutOfRangeException(String.Format("Indices ({0}, {1}) must be positive", index1, index2)); @@ -199,7 +201,7 @@ public async Task SwapAsync([NotNull] IFdbTransaction tr, long index1, long inde /// Get the item at the specified index. public async Task GetAsync([NotNull] IFdbReadOnlyTransaction tr, long index) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); if (index < 0) throw new IndexOutOfRangeException(String.Format("Index {0} must be positive", index)); var start = GetKeyAt(index); @@ -228,7 +230,7 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction tr, long index) /// [NOT YET IMPLEMENTED] Get a range of items in the Vector, returned as an async sequence. public IAsyncEnumerable GetRangeAsync([NotNull] IFdbReadOnlyTransaction tr, long startIndex, long endIndex, long step) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); //BUGUBG: implement FdbVector.GetRangeAsync() ! @@ -238,7 +240,7 @@ public IAsyncEnumerable GetRangeAsync([NotNull] IFdbReadOnlyTransaction tr, l /// Set the value at a particular index in the Vector. public void Set([NotNull] IFdbTransaction tr, long index, T value) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); tr.Set(GetKeyAt(index), this.Encoder.EncodeValue(value)); } @@ -246,7 +248,7 @@ public void Set([NotNull] IFdbTransaction tr, long index, T value) /// Test whether the Vector is empty. public async Task EmptyAsync([NotNull] IFdbReadOnlyTransaction tr) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); return (await ComputeSizeAsync(tr).ConfigureAwait(false)) == 0; } @@ -254,7 +256,7 @@ public async Task EmptyAsync([NotNull] IFdbReadOnlyTransaction tr) /// Grow or shrink the size of the Vector. public async Task ResizeAsync([NotNull] IFdbTransaction tr, long length) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); long currentSize = await ComputeSizeAsync(tr).ConfigureAwait(false); @@ -277,7 +279,7 @@ public async Task ResizeAsync([NotNull] IFdbTransaction tr, long length) /// Remove all items from the Vector. public void Clear([NotNull] IFdbTransaction tr) { - if (tr == null) throw new ArgumentNullException("tr"); + if (tr == null) throw new ArgumentNullException(nameof(tr)); tr.ClearRange(this.Subspace); } diff --git a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs index 91e87daba..50634f263 100644 --- a/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs +++ b/FoundationDB.Layers.Common/Counters/FdbCounterMap.cs @@ -43,27 +43,24 @@ public sealed class FdbCounterMap /// Create a new counter map. public FdbCounterMap([NotNull] IKeySubspace subspace) - : this(subspace, KeyValueEncoders.Tuples.Key()) + : this(subspace.AsTyped()) { } /// Create a new counter map, using a specific key encoder. - public FdbCounterMap([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoder keyEncoder) + public FdbCounterMap([NotNull] ITypedKeySubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); - if (keyEncoder == null) throw new ArgumentNullException("keyEncoder"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace; - this.KeyEncoder = keyEncoder; - this.Location = subspace.UsingEncoder(keyEncoder); + this.Location = subspace; } /// Subspace used as a prefix for all items in this counter list - public IKeySubspace Subspace { [NotNull] get; private set; } + [NotNull] + public IKeySubspace Subspace { get; } - /// Encoder for the keys of the counter map - public IKeyEncoder KeyEncoder { [NotNull] get; private set; } - - internal ITypedKeySubspace Location { [NotNull] get; private set; } + [NotNull] + internal ITypedKeySubspace Location { get; } /// Add a value to a counter in one atomic operation /// @@ -72,8 +69,8 @@ public FdbCounterMap([NotNull] IKeySubspace subspace, [NotNull] IKeyEncoderThis operation will not cause the current transaction to conflict. It may create conflicts for transactions that would read the value of the counter. public void Add([NotNull] IFdbTransaction transaction, [NotNull] TKey counterKey, long value) { - if (transaction == null) throw new ArgumentNullException("transaction"); - if (counterKey == null) throw new ArgumentNullException("counterKey"); + if (transaction == null) throw new ArgumentNullException(nameof(transaction)); + if (counterKey == null) throw new ArgumentNullException(nameof(counterKey)); //REVIEW: we could no-op if value == 0 but this may change conflict behaviour for other transactions... Slice param = value == 1 ? PlusOne : value == -1 ? MinusOne : Slice.FromFixed64(value); @@ -114,8 +111,8 @@ public void Decrement([NotNull] IFdbTransaction transaction, [NotNull] TKey coun /// public async Task ReadAsync([NotNull] IFdbReadOnlyTransaction transaction, [NotNull] TKey counterKey) { - if (transaction == null) throw new ArgumentNullException("transaction"); - if (counterKey == null) throw new ArgumentNullException("counterKey"); + if (transaction == null) throw new ArgumentNullException(nameof(transaction)); + if (counterKey == null) throw new ArgumentNullException(nameof(counterKey)); var data = await transaction.GetAsync(this.Location.Keys[counterKey]).ConfigureAwait(false); if (data.IsNullOrEmpty) return default(long?); @@ -129,8 +126,8 @@ public void Decrement([NotNull] IFdbTransaction transaction, [NotNull] TKey coun /// This method WILL conflict with other transactions! public async Task AddThenReadAsync([NotNull] IFdbTransaction transaction, [NotNull] TKey counterKey, long value) { - if (transaction == null) throw new ArgumentNullException("transaction"); - if (counterKey == null) throw new ArgumentNullException("counterKey"); + if (transaction == null) throw new ArgumentNullException(nameof(transaction)); + if (counterKey == null) throw new ArgumentNullException(nameof(counterKey)); var key = this.Location.Keys.Encode(counterKey); var res = await transaction.GetAsync(key).ConfigureAwait(false); @@ -163,8 +160,8 @@ public Task DecrementThenReadAsync([NotNull] IFdbTransaction transaction, /// This method WILL conflict with other transactions! public async Task ReadThenAddAsync([NotNull] IFdbTransaction transaction, [NotNull] TKey counterKey, long value) { - if (transaction == null) throw new ArgumentNullException("transaction"); - if (counterKey == null) throw new ArgumentNullException("counterKey"); + if (transaction == null) throw new ArgumentNullException(nameof(transaction)); + if (counterKey == null) throw new ArgumentNullException(nameof(counterKey)); var key = this.Location.Keys[counterKey]; var res = await transaction.GetAsync(key).ConfigureAwait(false); diff --git a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs index 2d62e905c..e8a17ad02 100644 --- a/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs +++ b/FoundationDB.Layers.Common/Counters/FdbHighContentionCounter.cs @@ -33,6 +33,7 @@ namespace FoundationDB.Layers.Counters using System; using System.Threading; using System.Threading.Tasks; + using Doxense.Collections.Tuples; using Doxense.Serialization.Encoders; /// Represents an integer value which can be incremented without conflict. @@ -56,14 +57,14 @@ public class FdbHighContentionCounter /// Database used by this layer /// Subspace to be used for storing the counter public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace) - : this(db, subspace, KeyValueEncoders.Tuples.Value()) + : this(db, subspace.AsDynamic(), TuPack.Encoding.GetValueEncoder()) { } /// Create a new High Contention counter, using a specific value encoder. /// Database used by this layer /// Subspace to be used for storing the counter /// Encoder for the counter values - public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IKeySubspace subspace, [NotNull] IValueEncoder encoder) + public FdbHighContentionCounter([NotNull] IFdbDatabase db, [NotNull] IDynamicKeySubspace subspace, [NotNull] IValueEncoder encoder) { if (db == null) throw new ArgumentNullException(nameof(db)); if (subspace == null) throw new ArgumentNullException(nameof(subspace)); diff --git a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs index 146fb9812..9035effb0 100644 --- a/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs +++ b/FoundationDB.Layers.Common/Indexes/FdbIndex`2.cs @@ -32,7 +32,6 @@ namespace FoundationDB.Layers.Indexing using System.Collections.Generic; using System.Diagnostics; using System.Threading.Tasks; - using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -44,33 +43,31 @@ public class FdbIndex { public FdbIndex([NotNull] string name, [NotNull] IKeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) - : this(name, subspace, valueComparer, indexNullValues, KeyValueEncoders.Tuples.CompositeKey()) + : this(name, subspace.AsTyped(), valueComparer, indexNullValues) { } - public FdbIndex([NotNull] string name, [NotNull] IKeySubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] ICompositeKeyEncoder encoder) + public FdbIndex([NotNull] string name, [NotNull] ITypedKeySubspace subspace, IEqualityComparer valueComparer, bool indexNullValues) { - if (name == null) throw new ArgumentNullException("name"); - if (subspace == null) throw new ArgumentNullException("subspace"); - if (encoder == null) throw new ArgumentNullException("encoder"); + if (name == null) throw new ArgumentNullException(nameof(name)); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Name = name; this.Subspace = subspace; this.ValueComparer = valueComparer ?? EqualityComparer.Default; this.IndexNullValues = indexNullValues; - this.Location = subspace.UsingEncoder(encoder); } - public string Name { [NotNull] get; private set; } + public string Name { [NotNull] get; } - public IKeySubspace Subspace { [NotNull] get; private set; } - - protected ITypedKeySubspace Location { [NotNull] get; private set; } + [NotNull] + public ITypedKeySubspace Subspace { get; } - public IEqualityComparer ValueComparer { [NotNull] get; private set; } + [NotNull] + public IEqualityComparer ValueComparer { get; } /// If true, null values are inserted in the index. If false (default), they are ignored /// This has no effect if is not a reference type - public bool IndexNullValues { get; private set; } + public bool IndexNullValues { get; } /// Insert a newly created entity to the index /// Transaction to use @@ -81,7 +78,7 @@ public bool Add([NotNull] IFdbTransaction trans, TId id, TValue value) { if (this.IndexNullValues || value != null) { - trans.Set(this.Location.Keys[value, id], Slice.Empty); + trans.Set(this.Subspace.Keys[value, id], Slice.Empty); return true; } return false; @@ -101,13 +98,13 @@ public bool Update([NotNull] IFdbTransaction trans, TId id, TValue newValue, TVa // remove previous value if (this.IndexNullValues || previousValue != null) { - trans.Clear(this.Location.Keys[previousValue, id]); + trans.Clear(this.Subspace.Keys[previousValue, id]); } // add new value if (this.IndexNullValues || newValue != null) { - trans.Set(this.Location.Keys[newValue, id], Slice.Empty); + trans.Set(this.Subspace.Keys[newValue, id], Slice.Empty); } // cannot be both null, so we did at least something) @@ -122,9 +119,9 @@ public bool Update([NotNull] IFdbTransaction trans, TId id, TValue newValue, TVa /// Previous value of the entity in the index public void Remove([NotNull] IFdbTransaction trans, TId id, TValue value) { - if (trans == null) throw new ArgumentNullException("trans"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); - trans.Clear(this.Location.Keys[value, id]); + trans.Clear(this.Subspace.Keys[value, id]); } /// Returns a list of ids matching a specific value @@ -147,43 +144,43 @@ public Task> LookupAsync([NotNull] IFdbReadOnlyTransaction trans, TVal [NotNull] public FdbRangeQuery Lookup(IFdbReadOnlyTransaction trans, TValue value, bool reverse = false) { - var prefix = this.Location.Keys.EncodePartial(value); + var prefix = this.Subspace.Keys.EncodePartial(value); return trans .GetRange(KeyRange.StartsWith(prefix), new FdbRangeOptions { Reverse = reverse }) - .Select((kvp) => this.Location.Keys.Decode(kvp.Key).Item2); + .Select((kvp) => this.Subspace.Keys.Decode(kvp.Key).Item2); } [NotNull] public FdbRangeQuery LookupGreaterThan([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool orEqual, bool reverse = false) { - var prefix = this.Location.Keys.EncodePartial(value); + var prefix = this.Subspace.Keys.EncodePartial(value); if (!orEqual) prefix = FdbKey.Increment(prefix); var space = new KeySelectorPair( KeySelector.FirstGreaterThan(prefix), - KeySelector.FirstGreaterOrEqual(this.Location.ToRange().End) + KeySelector.FirstGreaterOrEqual(this.Subspace.ToRange().End) ); return trans .GetRange(space, new FdbRangeOptions { Reverse = reverse }) - .Select((kvp) => this.Location.Keys.Decode(kvp.Key).Item2); + .Select((kvp) => this.Subspace.Keys.Decode(kvp.Key).Item2); } [NotNull] public FdbRangeQuery LookupLessThan([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool orEqual, bool reverse = false) { - var prefix = this.Location.Keys.EncodePartial(value); + var prefix = this.Subspace.Keys.EncodePartial(value); if (orEqual) prefix = FdbKey.Increment(prefix); var space = new KeySelectorPair( - KeySelector.FirstGreaterOrEqual(this.Location.ToRange().Begin), + KeySelector.FirstGreaterOrEqual(this.Subspace.ToRange().Begin), KeySelector.FirstGreaterThan(prefix) ); return trans .GetRange(space, new FdbRangeOptions { Reverse = reverse }) - .Select((kvp) => this.Location.Keys.Decode(kvp.Key).Item2); + .Select((kvp) => this.Subspace.Keys.Decode(kvp.Key).Item2); } public override string ToString() diff --git a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs index 1a6153061..d85d118c4 100644 --- a/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs +++ b/FoundationDB.Layers.Experimental/Documents/FdbDocumentCollection.cs @@ -42,26 +42,24 @@ public class FdbDocumentCollection public const int DefaultChunkSize = 1 << 20; // 1 MB - public FdbDocumentCollection(KeySubspace subspace, Func selector, IValueEncoder valueEncoder) - : this(subspace, selector, KeyValueEncoders.Tuples.CompositeKey(), valueEncoder) + public FdbDocumentCollection(IKeySubspace subspace, Func selector, IValueEncoder valueEncoder) + : this(subspace.AsTyped(), selector, valueEncoder) { } - public FdbDocumentCollection(KeySubspace subspace, Func selector, ICompositeKeyEncoder keyEncoder, IValueEncoder valueEncoder) + public FdbDocumentCollection(ITypedKeySubspace subspace, Func selector, IValueEncoder valueEncoder) { if (subspace == null) throw new ArgumentNullException(nameof(subspace)); if (selector == null) throw new ArgumentNullException(nameof(selector)); - if (keyEncoder == null) throw new ArgumentNullException(nameof(keyEncoder)); if (valueEncoder == null) throw new ArgumentNullException(nameof(valueEncoder)); this.Subspace = subspace; this.IdSelector = selector; this.ValueEncoder = valueEncoder; - this.Location = subspace.UsingEncoder(keyEncoder); } protected virtual Task> LoadPartsAsync(IFdbReadOnlyTransaction trans, TId id) { - var key = this.Location.Keys.EncodePartial(id); + var key = this.Subspace.Keys.EncodePartial(id); return trans .GetRange(KeyRange.StartsWith(key)) //TODO: options ? @@ -76,9 +74,7 @@ protected virtual TDocument DecodeParts(List parts) } /// Subspace used as a prefix for all hashsets in this collection - public KeySubspace Subspace { get; } - - protected ITypedKeySubspace Location { get; } + public ITypedKeySubspace Subspace { get; } /// Encoder that packs/unpacks the documents public IValueEncoder ValueEncoder { get; } @@ -102,7 +98,7 @@ public void Insert(IFdbTransaction trans, TDocument document) var packed = this.ValueEncoder.EncodeValue(document); // Key Prefix = ...(id,) - var key = this.Location.Keys.EncodePartial(id); + var key = this.Subspace.Keys.EncodePartial(id); // clear previous value trans.ClearRange(KeyRange.StartsWith(key)); @@ -123,7 +119,7 @@ public void Insert(IFdbTransaction trans, TDocument document) while (remaining > 0) { int sz = Math.Max(remaining, this.ChunkSize); - trans.Set(this.Location.Keys[id, index], packed.Substring(p, sz)); + trans.Set(this.Subspace.Keys[id, index], packed.Substring(p, sz)); ++index; p += sz; remaining -= sz; @@ -167,7 +163,7 @@ public void Delete(IFdbTransaction trans, TId id) if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); - var key = this.Location.Keys.EncodePartial(id); + var key = this.Subspace.Keys.EncodePartial(id); trans.ClearRange(KeyRange.StartsWith(key)); } @@ -182,7 +178,7 @@ public void DeleteMultiple(IFdbTransaction trans, IEnumerable ids) foreach (var id in ids) { - var key = this.Location.Keys.EncodePartial(id); + var key = this.Subspace.Keys.EncodePartial(id); trans.ClearRange(KeyRange.StartsWith(key)); } } diff --git a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs index f1981697f..7c9c66455 100644 --- a/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs +++ b/FoundationDB.Layers.Experimental/Indexes/FdbCompressedBitmapIndex.cs @@ -47,30 +47,29 @@ namespace FoundationDB.Layers.Experimental.Indexing public class FdbCompressedBitmapIndex { - public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] KeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) - : this(name, subspace, valueComparer, indexNullValues, KeyValueEncoders.Tuples.Key()) + public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] IKeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) + : this(name, subspace.AsTyped(), valueComparer, indexNullValues) { } - public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] KeySubspace subspace, IEqualityComparer valueComparer, bool indexNullValues, [NotNull] IKeyEncoder encoder) + public FdbCompressedBitmapIndex([NotNull] string name, [NotNull] ITypedKeySubspace subspace, IEqualityComparer valueComparer = null, bool indexNullValues = false) { Contract.NotNull(name, nameof(name)); Contract.NotNull(subspace, nameof(subspace)); - Contract.NotNull(encoder, nameof(encoder)); this.Name = name; this.Subspace = subspace; this.ValueComparer = valueComparer ?? EqualityComparer.Default; this.IndexNullValues = indexNullValues; - this.Location = subspace.UsingEncoder(encoder); } - public string Name { [NotNull] get; } + [NotNull] + public string Name { get; } - public KeySubspace Subspace { [NotNull] get; } + [NotNull] + public ITypedKeySubspace Subspace { get; } - protected ITypedKeySubspace Location { [NotNull] get; } - - public IEqualityComparer ValueComparer { [NotNull] get; } + [NotNull] + public IEqualityComparer ValueComparer { get; } /// If true, null values are inserted in the index. If false (default), they are ignored /// This has no effect if is not a reference type @@ -87,7 +86,7 @@ public async Task AddAsync([NotNull] IFdbTransaction trans, long id, TValu if (this.IndexNullValues || value != null) { - var key = this.Location.Keys[value]; + var key = this.Subspace.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); var builder = data.HasValue ? new CompressedBitmapBuilder(data) : CompressedBitmapBuilder.Empty; @@ -117,7 +116,7 @@ public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TV // remove previous value if (this.IndexNullValues || previousValue != null) { - var key = this.Location.Keys[previousValue]; + var key = this.Subspace.Keys[previousValue]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.HasValue) { @@ -130,7 +129,7 @@ public async Task UpdateAsync([NotNull] IFdbTransaction trans, long id, TV // add new value if (this.IndexNullValues || newValue != null) { - var key = this.Location.Keys[newValue]; + var key = this.Subspace.Keys[newValue]; var data = await trans.GetAsync(key).ConfigureAwait(false); var builder = data.HasValue ? new CompressedBitmapBuilder(data) : CompressedBitmapBuilder.Empty; builder.Set((int)id); //BUGBUG: 64 bit id! @@ -151,7 +150,7 @@ public async Task RemoveAsync([NotNull] IFdbTransaction trans, long id, TV { if (trans == null) throw new ArgumentNullException(nameof(trans)); - var key = this.Location.Keys[value]; + var key = this.Subspace.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.HasValue) { @@ -170,7 +169,7 @@ public async Task RemoveAsync([NotNull] IFdbTransaction trans, long id, TV /// List of document ids matching this value for this particular index (can be empty if no document matches) public async Task> LookupAsync([NotNull] IFdbReadOnlyTransaction trans, TValue value, bool reverse = false) { - var key = this.Location.Keys[value]; + var key = this.Subspace.Keys[value]; var data = await trans.GetAsync(key).ConfigureAwait(false); if (data.IsNull) return null; if (data.IsEmpty) return Enumerable.Empty(); diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index d95437903..be15b1540 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -189,7 +189,7 @@ public async Task Test_FdbMap_With_Custom_Key_Encoder() { var location = await GetCleanDirectory(db, "Collections", "Maps"); - var map = new FdbMap("Firewall", location.Partition.ByKey("Hosts"), keyEncoder, KeyValueEncoders.Values.StringEncoder); + var map = new FdbMap("Firewall", location.Partition.ByKey("Hosts").UsingEncoder(keyEncoder), KeyValueEncoders.Values.StringEncoder); // import all the rules await db.WriteAsync((tr) => From af009ac31250685937e01f790ca5eb4413361585 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 19:31:40 +0200 Subject: [PATCH 139/153] Move TaskHelpers into namespace Doxense.Threading.Tasks --- FoundationDB.Client/FdbDatabase.cs | 2 +- FoundationDB.Client/FdbOperationContext.cs | 2 +- .../FdbRangeQuery.PagingIterator.cs | 2 +- .../FdbRangeQuery.ResultIterator.cs | 2 +- FoundationDB.Client/FdbTransaction.cs | 2 +- .../Shared/Async/AsyncHelpers.cs | 1 + .../Shared/Async/TaskHelpers.cs | 2 +- .../Expressions/AsyncFilterExpression.cs | 2 +- .../Expressions/AsyncTransformExpression.cs | 2 +- .../Iterators/AnonymousAsyncGenerator.cs | 3 +- .../Async/Iterators/AsyncFilterIterator.cs | 2 +- .../Async/Iterators/MergeAsyncIterator.cs | 2 +- .../Iterators/ParallelSelectAsyncIterator.cs | 1 + .../Iterators/PrefetchingAsyncIterator.cs | 2 +- .../Async/Iterators/WhereAsyncIterator.cs | 2 +- .../Async/Iterators/WindowingAsyncIterator.cs | 2 +- .../Linq/AsyncEnumerable.EmptySequence.cs | 2 +- .../Shared/Linq/AsyncEnumerable.Iterators.cs | 2 +- .../Linq/AsyncEnumerable.OrderedSequence.cs | 35 ++++++++++++++----- .../Shared/Linq/AsyncEnumerable.cs | 3 +- 20 files changed, 46 insertions(+), 27 deletions(-) diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 70706bc71..96fbb50ca 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -35,11 +35,11 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; using Doxense.Serialization.Encoders; + using Doxense.Threading.Tasks; using FoundationDB.Client.Core; using FoundationDB.Client.Native; using FoundationDB.Layers.Directories; diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index 879bb3c0b..648cbaacd 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -33,8 +33,8 @@ namespace FoundationDB.Client using System.Globalization; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; using JetBrains.Annotations; /// diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 4f807672e..9bf6a6553 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -36,10 +36,10 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq; using Doxense.Linq.Async.Iterators; + using Doxense.Threading.Tasks; using JetBrains.Annotations; public partial class FdbRangeQuery diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index fd779a7a8..f0f3823c9 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -36,10 +36,10 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq; using Doxense.Linq.Async.Iterators; + using Doxense.Threading.Tasks; using JetBrains.Annotations; public partial class FdbRangeQuery diff --git a/FoundationDB.Client/FdbTransaction.cs b/FoundationDB.Client/FdbTransaction.cs index 673bbbe16..62a6f7327 100644 --- a/FoundationDB.Client/FdbTransaction.cs +++ b/FoundationDB.Client/FdbTransaction.cs @@ -36,8 +36,8 @@ namespace FoundationDB.Client using System.Diagnostics; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; using FoundationDB.Client.Core; using FoundationDB.Client.Native; using JetBrains.Annotations; diff --git a/FoundationDB.Client/Shared/Async/AsyncHelpers.cs b/FoundationDB.Client/Shared/Async/AsyncHelpers.cs index c18a556e6..235ae3953 100644 --- a/FoundationDB.Client/Shared/Async/AsyncHelpers.cs +++ b/FoundationDB.Client/Shared/Async/AsyncHelpers.cs @@ -35,6 +35,7 @@ namespace Doxense.Async using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; + using Doxense.Threading.Tasks; /// Helper methods for creating and manipulating async sequences. public static class AsyncHelpers diff --git a/FoundationDB.Client/Shared/Async/TaskHelpers.cs b/FoundationDB.Client/Shared/Async/TaskHelpers.cs index 995762043..18220a255 100644 --- a/FoundationDB.Client/Shared/Async/TaskHelpers.cs +++ b/FoundationDB.Client/Shared/Async/TaskHelpers.cs @@ -26,7 +26,7 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -namespace Doxense.Async +namespace Doxense.Threading.Tasks { using System; using System.Threading; diff --git a/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs index 0e9e67de7..cf064d60a 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncFilterExpression.cs @@ -33,8 +33,8 @@ namespace Doxense.Linq.Async.Expressions using System; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; /// Expression that evalute a condition on each item /// Type of the filtered elements diff --git a/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs index 694d43053..c138b254f 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Expressions/AsyncTransformExpression.cs @@ -31,8 +31,8 @@ namespace Doxense.Linq.Async.Expressions using System; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; using JetBrains.Annotations; /// Expression that applies a transformation on each item diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs index 89153e9a2..966bf96bd 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/AnonymousAsyncGenerator.cs @@ -26,14 +26,13 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion -using Doxense.Async; - namespace Doxense.Linq.Async.Iterators { using System; using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; /// Generate items asynchronously, using a user-provided lambda /// Type of the items produced by this generator diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs index 356c46be1..b1b7919fd 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/AsyncFilterIterator.cs @@ -31,8 +31,8 @@ namespace Doxense.Linq.Async.Iterators using System; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; using JetBrains.Annotations; public abstract class AsyncFilterIterator : AsyncIterator diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs index 9dda44927..f58c96ee4 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/MergeAsyncIterator.cs @@ -32,8 +32,8 @@ namespace Doxense.Linq.Async.Iterators using System.Collections.Generic; using System.Linq; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; /// Performs a Merge Sort on several concurrent range queries /// Type of the elements in the source queries diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs index fe29ad7b0..7e82e3966 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/ParallelSelectAsyncIterator.cs @@ -36,6 +36,7 @@ namespace Doxense.Linq.Async.Iterators using System.Threading.Tasks; using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; using JetBrains.Annotations; /// [EXPERIMENTAL] Iterates over an async sequence of items, kick off an async task in parallel, and returning the results in order diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs index 67b6bab2c..344fc493e 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/PrefetchingAsyncIterator.cs @@ -32,8 +32,8 @@ namespace Doxense.Linq.Async.Iterators using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; /// Prefetches items from the inner sequence, before outputing them down the line. /// Type the the items from the source sequence diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs index 104f69272..eebec9391 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/WhereAsyncIterator.cs @@ -31,9 +31,9 @@ namespace Doxense.Linq.Async.Iterators using System; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq.Async.Expressions; + using Doxense.Threading.Tasks; using JetBrains.Annotations; /// Filters an async sequence of items diff --git a/FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs b/FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs index 79024ca42..0c3100777 100644 --- a/FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs +++ b/FoundationDB.Client/Shared/Linq/Async/Iterators/WindowingAsyncIterator.cs @@ -32,8 +32,8 @@ namespace Doxense.Linq.Async.Iterators using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; /// Merges bursts of already-completed items from a source async sequence, into a sequence of batches. /// Type the the items from the source sequence diff --git a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs index 93c543797..ea5629ac3 100644 --- a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs +++ b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.EmptySequence.cs @@ -31,8 +31,8 @@ namespace Doxense.Linq using System; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; + using Doxense.Threading.Tasks; public static partial class AsyncEnumerable { diff --git a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs index adf835a6d..34594ec52 100644 --- a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs +++ b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.Iterators.cs @@ -33,11 +33,11 @@ namespace Doxense.Linq using System.Diagnostics; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq.Async; using Doxense.Linq.Async.Expressions; using Doxense.Linq.Async.Iterators; + using Doxense.Threading.Tasks; using JetBrains.Annotations; public static partial class AsyncEnumerable diff --git a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs index 6ba01898d..5031beeec 100644 --- a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs +++ b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.OrderedSequence.cs @@ -1,13 +1,31 @@ -#region Copyright Doxense SAS 2013-2016 -// -// All rights are reserved. Reproduction or transmission in whole or in part, in -// any form or by any means, electronic, mechanical or otherwise, is prohibited -// without the prior written consent of the copyright owner. -// +#region BSD Licence +/* Copyright (c) 2013-2018, 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 -using Doxense.Async; - namespace Doxense.Linq { using System; @@ -16,6 +34,7 @@ namespace Doxense.Linq using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq.Async.Iterators; + using Doxense.Threading.Tasks; using JetBrains.Annotations; public static partial class AsyncEnumerable diff --git a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs index 937da6b1d..a209cb7b5 100644 --- a/FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs +++ b/FoundationDB.Client/Shared/Linq/AsyncEnumerable.cs @@ -26,17 +26,16 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace Doxense.Linq { using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; - using Doxense.Async; using Doxense.Diagnostics.Contracts; using Doxense.Linq.Async.Expressions; using Doxense.Linq.Async.Iterators; + using Doxense.Threading.Tasks; using JetBrains.Annotations; /// Provides a set of static methods for querying objects that implement . From 3a392b07f978c4b6d3b10691b8cb35ce766ab7cc Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 19:39:34 +0200 Subject: [PATCH 140/153] Do not use direct Slice constructor --- FoundationDB.Client/FdbKey.cs | 4 ++-- .../Filters/Logging/FdbLoggedTransaction.cs | 2 +- FoundationDB.Client/Native/FdbNative.cs | 8 ++++---- FoundationDB.Client/Tuples/Encoding/TupleParser.cs | 4 +--- FoundationDB.Layers.Common/Blobs/FdbBlob.cs | 2 +- 5 files changed, 9 insertions(+), 11 deletions(-) diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index 3d16fbfbb..dc71ae4d9 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -83,7 +83,7 @@ public static Slice Increment(Slice slice) throw Fdb.Errors.CannotIncrementKey(); } - return new Slice(tmp, 0, lastNonFFByte + 1); + return tmp.AsSlice(0, lastNonFFByte + 1); } /// Merge an array of keys with a same prefix, all sharing the same buffer @@ -162,7 +162,7 @@ internal static Slice[] SplitIntoSegments([NotNull] byte[] buffer, int start, [N int p = start; foreach (var end in endOffsets) { - result[i++] = new Slice(buffer, p, end - p); + result[i++] = buffer.AsSlice(p, end - p); p = end; } diff --git a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs index 23b12cbce..4fca11c33 100644 --- a/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs +++ b/FoundationDB.Client/Filters/Logging/FdbLoggedTransaction.cs @@ -114,7 +114,7 @@ private Slice Grab(Slice slice) int start = m_offset; slice.CopyTo(m_buffer, m_offset); m_offset += slice.Count; - return new Slice(m_buffer, start, slice.Count); + return m_buffer.AsSlice(start, slice.Count); } } diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 1f865f658..3fde8d049 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -309,7 +309,7 @@ public static Slice ToNativeString(string value, bool nullTerminated) { result = Slice.DefaultEncoding.GetBytes(value); } - return new Slice(result, 0, result.Length); + return Slice.CreateUnsafe(result, 0, result.Length); } @@ -696,7 +696,7 @@ public static FdbError FutureGetValue(FutureHandle future, out bool valuePresent { var bytes = new byte[valueLength]; Marshal.Copy(new IntPtr(ptr), bytes, 0, valueLength); - value = new Slice(bytes, 0, valueLength); + value = Slice.CreateUnsafe(bytes, 0, valueLength); } else { @@ -785,8 +785,8 @@ public static FdbError FutureGetKeyValueArray(FutureHandle future, out KeyValueP Marshal.Copy(kvp[i].Value, page, p + kl, vl); result[i] = new KeyValuePair( - new Slice(page, p, kl), - new Slice(page, p + kl, vl) + page.AsSlice(p, kl), + page.AsSlice(p + kl, vl) ); p += kl + vl; diff --git a/FoundationDB.Client/Tuples/Encoding/TupleParser.cs b/FoundationDB.Client/Tuples/Encoding/TupleParser.cs index a2e24b5f6..96b71d58a 100644 --- a/FoundationDB.Client/Tuples/Encoding/TupleParser.cs +++ b/FoundationDB.Client/Tuples/Encoding/TupleParser.cs @@ -883,9 +883,7 @@ public static Slice ParseBytes(Slice slice) Contract.Requires(slice.HasValue && slice[0] == TupleTypes.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 diff --git a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs index c3d95add7..53ad20f24 100644 --- a/FoundationDB.Layers.Common/Blobs/FdbBlob.cs +++ b/FoundationDB.Layers.Common/Blobs/FdbBlob.cs @@ -274,7 +274,7 @@ await trans }) .ConfigureAwait(false); - return new Slice(buffer, 0, buffer.Length); + return buffer.AsSlice(0, buffer.Length); } /// From d7f37264f243cc2d3340aa029ca3bc811dd833f9 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 19:57:56 +0200 Subject: [PATCH 141/153] Cleanup code --- FoundationDB.Client/Fdb.Errors.cs | 10 +-- FoundationDB.Client/Fdb.Options.cs | 2 +- FoundationDB.Client/Fdb.System.cs | 3 +- FoundationDB.Client/Fdb.cs | 61 ++++++++---------- FoundationDB.Client/FdbCluster.cs | 37 +++++------ FoundationDB.Client/FdbClusterFile.cs | 22 ++++--- FoundationDB.Client/FdbDatabase.cs | 37 +++++------ FoundationDB.Client/FdbEndPoint.cs | 8 +-- FoundationDB.Client/FdbError.cs | 2 + FoundationDB.Client/FdbException.cs | 4 +- FoundationDB.Client/FdbIsolationLevel.cs | 2 + FoundationDB.Client/FdbKey.cs | 11 ++-- FoundationDB.Client/FdbNetworkOption.cs | 2 + FoundationDB.Client/FdbOperationContext.cs | 40 ++++++------ FoundationDB.Client/FdbRangeChunk.cs | 1 + FoundationDB.Client/FdbRangeOptions.cs | 2 + .../FdbRangeQuery.PagingIterator.cs | 5 +- .../FdbRangeQuery.ResultIterator.cs | 7 +-- FoundationDB.Client/FdbStreamingMode.cs | 2 + .../FdbTransaction.Snapshot.cs | 62 +++++++------------ .../FdbTransactionExtensions.cs | 1 + FoundationDB.Client/FdbTransactionOption.cs | 2 + FoundationDB.Client/FdbWatch.cs | 36 ++++------- FoundationDB.Client/IFdbCluster.cs | 3 +- FoundationDB.Client/IFdbDatabase.cs | 14 +++-- .../IFdbReadOnlyTransaction.cs | 6 +- FoundationDB.Client/IFdbRetryable.cs | 4 +- FoundationDB.Client/Shared/Tuples/STuple`3.cs | 1 - FoundationDb.Client.sln.DotSettings | 1 + 29 files changed, 175 insertions(+), 213 deletions(-) diff --git a/FoundationDB.Client/Fdb.Errors.cs b/FoundationDB.Client/Fdb.Errors.cs index 7a9b78a77..b0ed1117a 100644 --- a/FoundationDB.Client/Fdb.Errors.cs +++ b/FoundationDB.Client/Fdb.Errors.cs @@ -51,7 +51,7 @@ internal static Exception KeyCannotBeNull(string paramName = "key") internal static Exception KeyIsTooBig(Slice key, string paramName = "key") { - return new ArgumentException(String.Format("Key is too big ({0} > {1}).", key.Count, Fdb.MaxKeySize), paramName); + return new ArgumentException($"Key is too big ({key.Count} > {Fdb.MaxKeySize}).", paramName); } internal static Exception ValueCannotBeNull(Slice value, string paramName = "value") @@ -61,7 +61,7 @@ internal static Exception ValueCannotBeNull(Slice value, string paramName = "val internal static Exception ValueIsTooBig(Slice value, string paramName = "value") { - return new ArgumentException(String.Format("Value is too big ({0} > {1}).", value.Count, Fdb.MaxValueSize), paramName); + return new ArgumentException($"Value is too big ({value.Count} > {Fdb.MaxValueSize}).", paramName); } internal static Exception InvalidKeyOutsideDatabaseNamespace(IFdbDatabase db, Slice key) @@ -70,9 +70,9 @@ internal static Exception InvalidKeyOutsideDatabaseNamespace(IFdbDatabase db, Sl return new FdbException( FdbError.KeyOutsideLegalRange, #if DEBUG - String.Format("An attempt was made to use a key '{2}' that is outside of the global namespace {0} of database '{1}'", db.GlobalSpace, db.Name, FdbKey.Dump(key)) + $"An attempt was made to use a key '{FdbKey.Dump(key)}' that is outside of the global namespace {db.GlobalSpace} of database '{db.Name}'" #else - String.Format("An attempt was made to use a key that is outside of the global namespace {0} of database '{1}'", db.GlobalSpace, db.Name) + $"An attempt was made to use a key that is outside of the global namespace {db.GlobalSpace} of database '{db.Name}'" #endif ); } @@ -82,7 +82,7 @@ internal static Exception InvalidKeyOutsideDatabaseNamespace(IFdbDatabase db, Sl internal static Exception FailedToRegisterTransactionOnDatabase(IFdbTransaction transaction, FdbDatabase db) { Contract.Requires(transaction != null && db != null); - return new InvalidOperationException(String.Format("Failed to register transaction #{0} with this instance of database {1}", transaction.Id, db.Name)); + return new InvalidOperationException($"Failed to register transaction #{transaction.Id} with this instance of database {db.Name}"); } internal static Exception CannotIncrementKey() diff --git a/FoundationDB.Client/Fdb.Options.cs b/FoundationDB.Client/Fdb.Options.cs index 216b724a4..2b4d25b5f 100644 --- a/FoundationDB.Client/Fdb.Options.cs +++ b/FoundationDB.Client/Fdb.Options.cs @@ -72,7 +72,7 @@ public static void EnableNativeLibraryPreloading() /// public static void SetNativeLibPath(string path) { - if (path == null) throw new ArgumentNullException("path"); + if (path == null) throw new ArgumentNullException(nameof(path)); //TODO: throw if native library has already been loaded Fdb.Options.NativeLibPath = path; diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index bacb2e3d4..fd66c65be 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -34,7 +34,6 @@ namespace FoundationDB.Client using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; - using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using FoundationDB.Client.Status; using FoundationDB.Client.Utils; @@ -348,7 +347,7 @@ private static async Task> GetBoundaryKeysInternalAsync([NotNull] IF { results.Add(kvp.Key.Substring(KeyServers.Count)); } - begin = chunk.Last.Key.Substring(KeyServers.Count) + (byte)0; + begin = chunk.Last.Key.Substring(KeyServers.Count) + 0; } if (!chunk.HasMore) { diff --git a/FoundationDB.Client/Fdb.cs b/FoundationDB.Client/Fdb.cs index fd1b13a5f..273d4a097 100644 --- a/FoundationDB.Client/Fdb.cs +++ b/FoundationDB.Client/Fdb.cs @@ -30,9 +30,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Diagnostics; using System.Runtime.CompilerServices; @@ -40,6 +37,8 @@ namespace FoundationDB.Client using System.Threading; using System.Threading.Tasks; using SystemIO = System.IO; + using FoundationDB.Client.Native; + using JetBrains.Annotations; /// FoundationDB binding [PublicAPI] @@ -115,10 +114,7 @@ public static int GetMaxSafeApiVersion() /// Returns the currently selected API version. /// Unless explicitely selected by calling before, the default API version level will be returned - public static int ApiVersion - { - get { return s_apiVersion; } - } + public static int ApiVersion => s_apiVersion; /// Sets the desired API version of the binding. /// The selected version level may affect the availability and behavior or certain features. @@ -138,14 +134,14 @@ public static void UseApiVersion(int value) value = DefaultApiVersion; } if (s_apiVersion == value) return; //Alreay set to same version... skip it. - if (s_started) throw new InvalidOperationException(string.Format("You cannot set API version {0} because version {1} has already been selected", value, s_apiVersion)); + if (s_started) throw new InvalidOperationException($"You cannot set API version {value} because version {Fdb.s_apiVersion} has already been selected"); //note: we don't actually select the version yet, only when Start() is called. int min = GetMinApiVersion(); - if (value < min) throw new ArgumentException(String.Format("The minimum API version supported by this binding is {0} and the default version is {1}.", min, DefaultApiVersion)); + if (value < min) throw new ArgumentException($"The minimum API version supported by this binding is {min} and the default version is {Fdb.DefaultApiVersion}."); int max = GetMaxApiVersion(); - if (value > max) throw new ArgumentException(String.Format("The maximum API version supported by this binding is {0} and the default version is {1}.", max, DefaultApiVersion)); + if (value > max) throw new ArgumentException($"The maximum API version supported by this binding is {max} and the default version is {Fdb.DefaultApiVersion}."); s_apiVersion = value; } @@ -182,7 +178,7 @@ public static Exception MapToException(FdbError code) if (code == FdbError.Success) return null; string msg = GetErrorMessage(code); - if (msg == null) throw new FdbException(code, String.Format("Unexpected error code {0}", (int)code)); + if (msg == null) throw new FdbException(code, $"Unexpected error code {(int) code}"); //TODO: create a custom FdbException to be able to store the error code and error message switch(code) @@ -252,7 +248,7 @@ private static void StopEventLoop() var err = FdbNative.StopNetwork(); if (err != FdbError.Success) { - if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", String.Format("Failed to stop event loop: {0}", err.ToString())); + if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", $"Failed to stop event loop: {err.ToString()}"); } s_eventLoopStarted = false; @@ -276,7 +272,7 @@ private static void StopEventLoop() if (thread.IsAlive) { - if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", String.Format("The fdb network thread has not stopped after {0} seconds. Forcing shutdown...", duration.Elapsed.TotalSeconds.ToString("N0"))); + if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", $"The fdb network thread has not stopped after {duration.Elapsed.TotalSeconds:N0} seconds. Forcing shutdown..."); // Force a shutdown thread.Abort(); @@ -287,7 +283,7 @@ private static void StopEventLoop() if (!stopped) { - if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", String.Format("The fdb network thread failed to stop after more than {0} seconds. Transaction integrity may not be guaranteed.", duration.Elapsed.TotalSeconds.ToString("N0"))); + if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", $"The fdb network thread failed to stop after more than {duration.Elapsed.TotalSeconds:N0} seconds. Transaction integrity may not be guaranteed."); } } } @@ -301,7 +297,7 @@ private static void StopEventLoop() duration.Stop(); if (duration.Elapsed.TotalSeconds >= 20) { - if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", String.Format("The fdb network thread took a long time to stop ({0} seconds).", duration.Elapsed.TotalSeconds.ToString("N0"))); + if (Logging.On) Logging.Warning(typeof(Fdb), "StopEventLoop", $"The fdb network thread took a long time to stop ({duration.Elapsed.TotalSeconds:N0} seconds)."); } } } @@ -321,18 +317,18 @@ private static void EventLoop() s_eventLoopThreadId = Thread.CurrentThread.ManagedThreadId; - if (Logging.On) Logging.Verbose(typeof(Fdb), "EventLoop", String.Format("FDB Event Loop running on thread #{0}...", s_eventLoopThreadId.Value)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "EventLoop", $"FDB Event Loop running on thread #{Fdb.s_eventLoopThreadId.Value}..."); var err = FdbNative.RunNetwork(); if (err != FdbError.Success) { if (s_eventLoopStopRequested || Environment.HasShutdownStarted) { // this was requested, or can be explained by the computer shutting down... - if (Logging.On) Logging.Info(typeof(Fdb), "EventLoop", String.Format("The fdb network thread returned with error code {0}: {1}", err, GetErrorMessage(err))); + if (Logging.On) Logging.Info(typeof(Fdb), "EventLoop", $"The fdb network thread returned with error code {err}: {GetErrorMessage(err)}"); } else { // this was NOT expected ! - if (Logging.On) Logging.Error(typeof(Fdb), "EventLoop", String.Format("The fdb network thread returned with error code {0}: {1}", err, GetErrorMessage(err))); + if (Logging.On) Logging.Error(typeof(Fdb), "EventLoop", $"The fdb network thread returned with error code {err}: {GetErrorMessage(err)}"); #if DEBUG Console.Error.WriteLine("THE FDB NETWORK EVENT LOOP HAS FAILED!"); Console.Error.WriteLine("=> " + err); @@ -394,10 +390,7 @@ private static void EventLoop() } /// Returns true if the Network thread start is executing, otherwise falsse - public static bool IsNetworkRunning - { - get { return s_eventLoopRunning; } - } + public static bool IsNetworkRunning => s_eventLoopRunning; /// Returns 'true' if we are currently running on the Event Loop thread internal static bool IsNetworkThread @@ -563,8 +556,8 @@ internal static async Task OpenInternalAsync(string clusterFile, st if (!success) { // cleanup the cluter if something went wrong - if (db != null) db.Dispose(); - if (cluster != null) cluster.Dispose(); + db?.Dispose(); + cluster?.Dispose(); } } } @@ -597,19 +590,19 @@ public static void Start() int apiVersion = s_apiVersion; if (apiVersion <= 0) apiVersion = DefaultApiVersion; - if (Logging.On) Logging.Info(typeof(Fdb), "Start", String.Format("Selecting fdb API version {0}", apiVersion)); + if (Logging.On) Logging.Info(typeof(Fdb), "Start", $"Selecting fdb API version {apiVersion}"); FdbError err = FdbNative.SelectApiVersion(apiVersion); if (err != FdbError.Success) { - if (Logging.On) Logging.Error(typeof(Fdb), "Start", String.Format("Failed to fdb API version {0}: {1}", apiVersion, err)); + if (Logging.On) Logging.Error(typeof(Fdb), "Start", $"Failed to fdb API version {apiVersion}: {err}"); switch (err) { case FdbError.ApiVersionNotSupported: { // bad version was selected ? // note: we already bound check the values before, so that means that fdb_c.dll is either an older version or an incompatible new version. - throw new FdbException(err, String.Format("The API version {0} is not supported by the FoundationDB client library (fdb_c.dll) installed on this system. The binding only supports versions {1} to {2}. You either need to upgrade the .NET binding or the FoundationDB client library to a newer version.", apiVersion, GetMinApiVersion(), GetMaxApiVersion())); + throw new FdbException(err, $"The API version {apiVersion} is not supported by the FoundationDB client library (fdb_c.dll) installed on this system. The binding only supports versions {GetMinApiVersion()} to {GetMaxApiVersion()}. You either need to upgrade the .NET binding or the FoundationDB client library to a newer version."); } #if DEBUG case FdbError.ApiVersionAlreadySet: @@ -626,7 +619,7 @@ public static void Start() if (!string.IsNullOrWhiteSpace(Fdb.Options.TracePath)) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will trace client activity in '{0}'", Fdb.Options.TracePath)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will trace client activity in '{Fdb.Options.TracePath}'"); // create trace directory if missing... if (!SystemIO.Directory.Exists(Fdb.Options.TracePath)) SystemIO.Directory.CreateDirectory(Fdb.Options.TracePath); @@ -635,40 +628,40 @@ public static void Start() if (!string.IsNullOrWhiteSpace(Fdb.Options.TLSPlugin)) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will use custom TLS plugin '{0}'", Fdb.Options.TLSPlugin)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will use custom TLS plugin '{Fdb.Options.TLSPlugin}'"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSPlugin, Fdb.Options.TLSPlugin)); } if (Fdb.Options.TLSCertificateBytes.IsPresent) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will load TLS root certificate and private key from memory ({0} bytes)", Fdb.Options.TLSCertificateBytes.Count)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will load TLS root certificate and private key from memory ({Fdb.Options.TLSCertificateBytes.Count} bytes)"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSCertBytes, Fdb.Options.TLSCertificateBytes)); } else if (!string.IsNullOrWhiteSpace(Fdb.Options.TLSCertificatePath)) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will load TLS root certificate and private key from '{0}'", Fdb.Options.TLSCertificatePath)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will load TLS root certificate and private key from '{Fdb.Options.TLSCertificatePath}'"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSCertPath, Fdb.Options.TLSCertificatePath)); } if (Fdb.Options.TLSPrivateKeyBytes.IsPresent) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will load TLS private key from memory ({0} bytes)", Fdb.Options.TLSPrivateKeyBytes.Count)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will load TLS private key from memory ({Fdb.Options.TLSPrivateKeyBytes.Count} bytes)"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSKeyBytes, Fdb.Options.TLSPrivateKeyBytes)); } else if (!string.IsNullOrWhiteSpace(Fdb.Options.TLSPrivateKeyPath)) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will load TLS private key from '{0}'", Fdb.Options.TLSPrivateKeyPath)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will load TLS private key from '{Fdb.Options.TLSPrivateKeyPath}'"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSKeyPath, Fdb.Options.TLSPrivateKeyPath)); } if (Fdb.Options.TLSVerificationPattern.IsPresent) { - if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", String.Format("Will verify TLS peers with pattern '{0}'", Fdb.Options.TLSVerificationPattern)); + if (Logging.On) Logging.Verbose(typeof(Fdb), "Start", $"Will verify TLS peers with pattern '{Fdb.Options.TLSVerificationPattern}'"); DieOnError(SetNetworkOption(FdbNetworkOption.TLSVerifyPeers, Fdb.Options.TLSVerificationPattern)); } diff --git a/FoundationDB.Client/FdbCluster.cs b/FoundationDB.Client/FdbCluster.cs index 7d609a905..8204929b5 100644 --- a/FoundationDB.Client/FdbCluster.cs +++ b/FoundationDB.Client/FdbCluster.cs @@ -28,13 +28,12 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Core; - using FoundationDB.Client.Native; - using FoundationDB.Client.Utils; - using JetBrains.Annotations; using System; using System.Threading; using System.Threading.Tasks; + using FoundationDB.Client.Core; + using FoundationDB.Client.Native; + using JetBrains.Annotations; /// FoundationDB Cluster public class FdbCluster : IFdbCluster @@ -52,23 +51,17 @@ public class FdbCluster : IFdbCluster /// Wraps a cluster handle public FdbCluster(IFdbClusterHandler handler, string path) { - if (handler == null) throw new ArgumentNullException("handler"); + if (handler == null) throw new ArgumentNullException(nameof(handler)); m_handler = handler; m_path = path; } /// Path to the cluster file used by this connection, or null if the default cluster file is being used - public string Path - { - get { return m_path; } - } + public string Path => m_path; - internal IFdbClusterHandler Handler - { - [NotNull] - get { return m_handler; } - } + [NotNull] + internal IFdbClusterHandler Handler => m_handler; private void ThrowIfDisposed() { @@ -114,7 +107,7 @@ protected virtual void Dispose(bool disposing) [ItemNotNull] public async Task OpenDatabaseAsync(string databaseName, IKeySubspace subspace, bool readOnly, CancellationToken ct) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); return await OpenDatabaseInternalAsync(databaseName, subspace, readOnly: readOnly, ownsCluster: false, ct: ct).ConfigureAwait(false); } @@ -132,16 +125,16 @@ public async Task OpenDatabaseAsync(string databaseName, IKeySubsp internal async Task OpenDatabaseInternalAsync(string databaseName, IKeySubspace subspace, bool readOnly, bool ownsCluster, CancellationToken ct) { ThrowIfDisposed(); - if (string.IsNullOrEmpty(databaseName)) throw new ArgumentNullException("databaseName"); - if (subspace == null) throw new ArgumentNullException("subspace"); + if (string.IsNullOrEmpty(databaseName)) throw new ArgumentNullException(nameof(databaseName)); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); - if (Logging.On) Logging.Info(typeof(FdbCluster), "OpenDatabaseAsync", String.Format("Connecting to database '{0}' ...", databaseName)); + if (Logging.On) Logging.Info(typeof(FdbCluster), "OpenDatabaseAsync", $"Connecting to database '{databaseName}' ..."); if (ct.IsCancellationRequested) ct.ThrowIfCancellationRequested(); var handler = await m_handler.OpenDatabaseAsync(databaseName, ct).ConfigureAwait(false); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(typeof(FdbCluster), "OpenDatabaseAsync", String.Format("Connected to database '{0}'", databaseName)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(typeof(FdbCluster), "OpenDatabaseAsync", $"Connected to database '{databaseName}'"); return FdbDatabase.Create(this, handler, databaseName, subspace, null, readOnly, ownsCluster); } @@ -154,7 +147,7 @@ public void SetOption(FdbClusterOption option) Fdb.EnsureNotOnNetworkThread(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting cluster option {0}", option.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting cluster option {option.ToString()}"); m_handler.SetOption(option, Slice.Nil); } @@ -168,7 +161,7 @@ public void SetOption(FdbClusterOption option, string value) Fdb.EnsureNotOnNetworkThread(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting cluster option {0} to '{1}'", option.ToString(), value ?? "")); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting cluster option {option.ToString()} to '{value ?? ""}'"); var data = FdbNative.ToNativeString(value, nullTerminated: true); m_handler.SetOption(option, data); @@ -183,7 +176,7 @@ public void SetOption(FdbClusterOption option, long value) Fdb.EnsureNotOnNetworkThread(); - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", String.Format("Setting cluster option {0} to {1}", option.ToString(), value)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "SetOption", $"Setting cluster option {option.ToString()} to {value}"); var data = Slice.FromFixed64(value); m_handler.SetOption(option, data); diff --git a/FoundationDB.Client/FdbClusterFile.cs b/FoundationDB.Client/FdbClusterFile.cs index 7cc33f616..83a89d00b 100644 --- a/FoundationDB.Client/FdbClusterFile.cs +++ b/FoundationDB.Client/FdbClusterFile.cs @@ -28,24 +28,27 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using JetBrains.Annotations; using System; using System.Collections.Generic; using System.Globalization; using System.Linq; using System.Net; + using JetBrains.Annotations; /// Class that exposes the content of a FoundationDB .cluster file public sealed class FdbClusterFile { /// The raw value of the file - internal string RawValue { [NotNull] get; private set; } + [NotNull] + internal string RawValue { get; private set; } /// Cluster Identifier - public string Id { [NotNull] get; private set; } + [NotNull] + public string Id { get; private set; } /// Logical description of the database - public string Description { [NotNull] get; private set; } + [NotNull] + public string Description { get; private set; } /// List of coordination servers public FdbEndPoint[] Coordinators { get; private set; } @@ -59,9 +62,9 @@ private FdbClusterFile() /// public FdbClusterFile(string description, string identifier, IEnumerable coordinators) { - if (description == null) throw new ArgumentNullException("description"); - if (identifier == null) throw new ArgumentNullException("identifier"); - if (coordinators == null) throw new ArgumentNullException("coordinators"); + if (description == null) throw new ArgumentNullException(nameof(description)); + if (identifier == null) throw new ArgumentNullException(nameof(identifier)); + if (coordinators == null) throw new ArgumentNullException(nameof(coordinators)); this.Description = description; this.Id = identifier; @@ -72,7 +75,7 @@ public FdbClusterFile(string description, string identifier, IEnumerable String.Format(CultureInfo.InvariantCulture, "{0}:{1}", kvp.Address, kvp.Port))) + string.Join(",", this.Coordinators.Select(kvp => string.Format(CultureInfo.InvariantCulture, "{0}:{1}", kvp.Address, kvp.Port))) ); } @@ -142,8 +145,7 @@ public override int GetHashCode() /// Check if this cluster file is equal to another object public override bool Equals(object obj) { - var cf = obj as FdbClusterFile; - return cf != null && string.Equals(this.RawValue, cf.RawValue, StringComparison.Ordinal); + return obj is FdbClusterFile cf && string.Equals(this.RawValue, cf.RawValue, StringComparison.Ordinal); } } diff --git a/FoundationDB.Client/FdbDatabase.cs b/FoundationDB.Client/FdbDatabase.cs index 96fbb50ca..491009a64 100644 --- a/FoundationDB.Client/FdbDatabase.cs +++ b/FoundationDB.Client/FdbDatabase.cs @@ -31,13 +31,11 @@ namespace FoundationDB.Client using JetBrains.Annotations; using System; using System.Collections.Concurrent; - using System.Collections.Generic; using System.Diagnostics; using System.Threading; using System.Threading.Tasks; using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; - using Doxense.Memory; using Doxense.Serialization.Encoders; using Doxense.Threading.Tasks; using FoundationDB.Client.Core; @@ -47,7 +45,7 @@ namespace FoundationDB.Client /// FoundationDB database session handle /// An instance of this class can be used to create any number of concurrent transactions that will read and/or write to this particular database. [DebuggerDisplay("Name={m_name}, GlobalSpace={m_globalSpace}")] - public class FdbDatabase : IFdbDatabase, IFdbRetryable + public class FdbDatabase : IFdbDatabase { #region Private Fields... @@ -158,10 +156,10 @@ public string Name /// Returns a cancellation token that is linked with the lifetime of this database instance /// The token will be cancelled if the database instance is disposed //REVIEW: rename this to 'Cancellation'? ('Token' is a keyword that may have different meaning in some apps) - public CancellationToken Cancellation { get { return m_cts.Token; } } + public CancellationToken Cancellation => m_cts.Token; /// If true, this database instance will only allow starting read-only transactions. - public bool IsReadOnly { get { return m_readOnly; } } + public bool IsReadOnly => m_readOnly; /// Root directory of this database instance public FdbDatabasePartition Directory @@ -219,7 +217,7 @@ public IFdbTransaction BeginTransaction(FdbTransactionMode mode, CancellationTok /// Optional context in which the transaction will run internal FdbTransaction CreateNewTransaction(FdbOperationContext context) { - Contract.Requires(context != null && context.Database != null); + Contract.Requires(context?.Database != null); ThrowIfDisposed(); // force the transaction to be read-only, if the database itself is read-only @@ -246,10 +244,7 @@ internal FdbTransaction CreateNewTransaction(FdbOperationContext context) } catch (Exception) { - if (trans != null) - { - trans.Dispose(); - } + trans?.Dispose(); throw; } } @@ -322,15 +317,15 @@ public Task ReadAsync([InstantHandle] Func asyncH /// Runs a transactional lambda function against this database, inside a read-only transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-only transaction on each retry. The result of the task will also be the result of the transactional. /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadAsync(Func> asyncHandler, CancellationToken ct) + public Task ReadAsync(Func> asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, ct); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) + public Task ReadAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, ct); + return FdbOperationContext.RunReadWithResultAsync(this, asyncHandler, onDone, ct); } #endregion @@ -389,15 +384,15 @@ public Task ReadWriteAsync([InstantHandle] Func asyncHand /// Runs a transactional lambda function against this database, inside a read-write transaction context, with retry logic. /// Asynchronous lambda function that is passed a new read-write transaction on each retry. The result of the task will also be the result of the transactional. /// Optional cancellation token that will be passed to the transaction context, and that can also be used to abort the retry loop. - public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, CancellationToken ct) + public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, CancellationToken ct) { - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, ct); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, null, ct); } /// EXPERIMENTAL - public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) + public Task ReadWriteAsync([InstantHandle] Func> asyncHandler, [InstantHandle] Action onDone, CancellationToken ct) { - return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, ct); + return FdbOperationContext.RunWriteWithResultAsync(this, asyncHandler, onDone, ct); } #endregion @@ -604,7 +599,7 @@ internal Slice BoundCheck(Slice key) /// Only effective for future transactions public int DefaultTimeout { - get { return m_defaultTimeout; } + get => m_defaultTimeout; set { if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "Timeout value cannot be negative"); @@ -616,7 +611,7 @@ public int DefaultTimeout /// Only effective for future transactions public int DefaultRetryLimit { - get { return m_defaultRetryLimit; } + get => m_defaultRetryLimit; set { if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "RetryLimit value cannot be negative"); @@ -628,7 +623,7 @@ public int DefaultRetryLimit /// Only effective for future transactions public int DefaultMaxRetryDelay { - get { return m_defaultMaxRetryDelay; } + get => m_defaultMaxRetryDelay; set { if (value < 0) throw new ArgumentOutOfRangeException(nameof(value), value, "MaxRetryDelay value cannot be negative"); diff --git a/FoundationDB.Client/FdbEndPoint.cs b/FoundationDB.Client/FdbEndPoint.cs index d5851605a..1fb34dc9b 100644 --- a/FoundationDB.Client/FdbEndPoint.cs +++ b/FoundationDB.Client/FdbEndPoint.cs @@ -44,10 +44,7 @@ public FdbEndPoint(IPAddress address, int port, bool tls) /// Gets or sets the TLS mode of the endpoint. /// True if the endpoint uses TLS - public bool Tls - { - get { return m_tls; } - } + public bool Tls => m_tls; public override SocketAddress Serialize() { @@ -90,8 +87,7 @@ public override string ToString() public override bool Equals(object comparand) { - var fep = comparand as FdbEndPoint; - return fep != null && fep.m_tls == m_tls && base.Equals(fep); + return comparand is FdbEndPoint fep && fep.m_tls == m_tls && base.Equals(fep); } public override int GetHashCode() diff --git a/FoundationDB.Client/FdbError.cs b/FoundationDB.Client/FdbError.cs index 0068bbe72..f385ee930 100644 --- a/FoundationDB.Client/FdbError.cs +++ b/FoundationDB.Client/FdbError.cs @@ -29,8 +29,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using JetBrains.Annotations; /// FoundationDB API Error Code + [PublicAPI] public enum FdbError { /// Success diff --git a/FoundationDB.Client/FdbException.cs b/FoundationDB.Client/FdbException.cs index 068ad6f04..62d558067 100644 --- a/FoundationDB.Client/FdbException.cs +++ b/FoundationDB.Client/FdbException.cs @@ -69,10 +69,10 @@ public override void GetObjectData(SerializationInfo info, StreamingContext cont #endif /// Gets the code for this error. - public FdbError Code { get; private set; } + public FdbError Code { get; } /// Determine if this FDBError represents a success code from the native layer. - public bool Success { get { return this.Code == FdbError.Success; } } + public bool Success => this.Code == FdbError.Success; //REVIEW: do we need to add more properties? TransactionId ? DBName? diff --git a/FoundationDB.Client/FdbIsolationLevel.cs b/FoundationDB.Client/FdbIsolationLevel.cs index d09c65f6b..98683e0f6 100644 --- a/FoundationDB.Client/FdbIsolationLevel.cs +++ b/FoundationDB.Client/FdbIsolationLevel.cs @@ -29,8 +29,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using JetBrains.Annotations; /// Specifies the isolation level of FoundationDB transactions + [PublicAPI] public enum FdbIsolationLevel { //note: we use the same values as System.Data.IsolationLevel so we can cast to and from, without requiring a reference to System.Data.dll diff --git a/FoundationDB.Client/FdbKey.cs b/FoundationDB.Client/FdbKey.cs index dc71ae4d9..5ba3c7ea3 100644 --- a/FoundationDB.Client/FdbKey.cs +++ b/FoundationDB.Client/FdbKey.cs @@ -35,7 +35,6 @@ namespace FoundationDB.Client using Doxense.Collections.Tuples; using Doxense.Diagnostics.Contracts; using Doxense.Memory; - using FoundationDB.Client; using JetBrains.Annotations; /// Factory class for keys @@ -112,7 +111,7 @@ public static Slice[] Merge(Slice prefix, [NotNull] Slice[] keys) next.Add(writer.Position); } - return FdbKey.SplitIntoSegments(writer.Buffer, 0, next); + return SplitIntoSegments(writer.Buffer, 0, next); } /// Merge a sequence of keys with a same prefix, all sharing the same buffer @@ -128,12 +127,10 @@ public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) //REVIEW: merge this code with Slice.ConcatRange! // use optimized version for arrays - var array = keys as Slice[]; - if (array != null) return Merge(prefix, array); + if (keys is Slice[] array) return Merge(prefix, array); // pre-allocate with a count if we can get one... - var coll = keys as ICollection; - var next = coll == null ? new List() : new List(coll.Count); + var next = !(keys is ICollection coll) ? new List() : new List(coll.Count); var writer = default(SliceWriter); //TODO: use multiple buffers if item count is huge ? @@ -145,7 +142,7 @@ public static Slice[] Merge(Slice prefix, [NotNull] IEnumerable keys) next.Add(writer.Position); } - return FdbKey.SplitIntoSegments(writer.Buffer, 0, next); + return SplitIntoSegments(writer.Buffer, 0, next); } /// Split a buffer containing multiple contiguous segments into an array of segments diff --git a/FoundationDB.Client/FdbNetworkOption.cs b/FoundationDB.Client/FdbNetworkOption.cs index ec9d094ca..704dc79a0 100644 --- a/FoundationDB.Client/FdbNetworkOption.cs +++ b/FoundationDB.Client/FdbNetworkOption.cs @@ -29,8 +29,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using JetBrains.Annotations; /// Defines a set of options for the network thread + [PublicAPI] public enum FdbNetworkOption { /// None diff --git a/FoundationDB.Client/FdbOperationContext.cs b/FoundationDB.Client/FdbOperationContext.cs index 648cbaacd..8756e974d 100644 --- a/FoundationDB.Client/FdbOperationContext.cs +++ b/FoundationDB.Client/FdbOperationContext.cs @@ -145,21 +145,21 @@ internal static async Task ExecuteInternal([NotNull] IFdbDatabase db, [NotNull] try { // call the user provided lambda - if (handler is Func) + if (handler is Func funcWritable) { - await ((Func)handler)(trans).ConfigureAwait(false); + await funcWritable(trans).ConfigureAwait(false); } - else if (handler is Action) + else if (handler is Action action) { - ((Action)handler)(trans); + action(trans); } - else if (handler is Func) + else if (handler is Func funcReadOnly) { - await ((Func)handler)(trans).ConfigureAwait(false); + await funcReadOnly(trans).ConfigureAwait(false); } else { - throw new NotSupportedException(String.Format("Cannot execute handlers of type {0}", handler.GetType().Name)); + throw new NotSupportedException($"Cannot execute handlers of type {handler.GetType().Name}"); } if (context.Abort) @@ -177,25 +177,25 @@ internal static async Task ExecuteInternal([NotNull] IFdbDatabase db, [NotNull] if (onDone != null) { - if (onDone is Action) + if (onDone is Action action1) { - ((Action)onDone)(trans); + action1(trans); } - else if (onDone is Action) + else if (onDone is Action action2) { - ((Action)onDone)(trans); + action2(trans); } - else if (onDone is Func) + else if (onDone is Func func1) { - await ((Func)onDone)(trans).ConfigureAwait(false); + await func1(trans).ConfigureAwait(false); } - else if (onDone is Func) + else if (onDone is Func func2) { - await ((Func)onDone)(trans).ConfigureAwait(false); + await func2(trans).ConfigureAwait(false); } else { - throw new NotSupportedException(String.Format("Cannot execute completion handler of type {0}", handler.GetType().Name)); + throw new NotSupportedException($"Cannot execute completion handler of type {handler.GetType().Name}"); } } } @@ -258,13 +258,13 @@ public static Task RunReadAsync([NotNull] IFdbDatabase db, [NotNull] FuncRun a read-only operation until it suceeds, timeouts, or fail with non-retryable error - public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) + public static async Task RunReadWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) { if (db == null) throw new ArgumentNullException(nameof(db)); if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); ct.ThrowIfCancellationRequested(); - R result = default(R); + TResult result = default(TResult); Func handler = async (tr) => { result = await asyncHandler(tr).ConfigureAwait(false); @@ -302,7 +302,7 @@ public static Task RunWriteAsync([NotNull] IFdbDatabase db, [NotNull] ActionRun a read/write operation until it suceeds, timeouts, or fail with non-retryable error - public static async Task RunWriteWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) + public static async Task RunWriteWithResultAsync([NotNull] IFdbDatabase db, [NotNull] Func> asyncHandler, Action onDone, CancellationToken ct) { if (db == null) throw new ArgumentNullException(nameof(db)); if (asyncHandler == null) throw new ArgumentNullException(nameof(asyncHandler)); @@ -315,7 +315,7 @@ public static async Task RunWriteWithResultAsync([NotNull] IFdbDatabase db var context = new FdbOperationContext(db, FdbTransactionMode.Default | FdbTransactionMode.InsideRetryLoop, ct); await ExecuteInternal(db, context, handler, onDone).ConfigureAwait(false); - return (R)context.Result; + return (TResult)context.Result; } #endregion diff --git a/FoundationDB.Client/FdbRangeChunk.cs b/FoundationDB.Client/FdbRangeChunk.cs index b7b2d09bd..0d39a863b 100644 --- a/FoundationDB.Client/FdbRangeChunk.cs +++ b/FoundationDB.Client/FdbRangeChunk.cs @@ -205,6 +205,7 @@ public T[] DecodeKeys([NotNull] Func handler) /// Decode the content of this chunk into an array of typed keys /// Type of the keys + /// /// Instance used to decode the keys of this chunk /// Array of decoded keys, or an empty array if the chunk doesn't have any results [NotNull] diff --git a/FoundationDB.Client/FdbRangeOptions.cs b/FoundationDB.Client/FdbRangeOptions.cs index 94b760318..3fdb9e8f0 100644 --- a/FoundationDB.Client/FdbRangeOptions.cs +++ b/FoundationDB.Client/FdbRangeOptions.cs @@ -31,9 +31,11 @@ namespace FoundationDB.Client using System; using System.Diagnostics; using Doxense.Diagnostics.Contracts; + using JetBrains.Annotations; /// Container class for options in a Range query [DebuggerDisplay("Limit={Limit}, Reverse={Reverse}, TargetBytes={TargetBytes}, Mode={Mode}")] + [PublicAPI] public sealed class FdbRangeOptions { #region Public Properties... diff --git a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs index 9bf6a6553..02b561401 100644 --- a/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.PagingIterator.cs @@ -34,7 +34,6 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; @@ -52,9 +51,9 @@ private sealed class PagingIterator : AsyncIterator[] #region Iterable Properties... - private FdbRangeQuery Query { get; set; } + private FdbRangeQuery Query { get; } - private IFdbReadOnlyTransaction Transaction { get; set; } + private IFdbReadOnlyTransaction Transaction { get; } #endregion diff --git a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs index f0f3823c9..5bfd38d35 100644 --- a/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs +++ b/FoundationDB.Client/FdbRangeQuery.ResultIterator.cs @@ -34,7 +34,6 @@ namespace FoundationDB.Client using System; using System.Collections.Generic; using System.Diagnostics; - using System.Threading; using System.Threading.Tasks; using Doxense.Diagnostics.Contracts; using Doxense.Linq; @@ -171,9 +170,9 @@ private bool ProcessNextItem() #region LINQ - public override AsyncIterator Select(Func selector) + public override AsyncIterator Select(Func selector) { - var query = new FdbRangeQuery( + var query = new FdbRangeQuery( m_transaction, m_query.Begin, m_query.End, @@ -182,7 +181,7 @@ public override AsyncIterator Select(Func selector) m_query.Options ); - return new FdbRangeQuery.ResultIterator(query, m_transaction, query.Transform); + return new FdbRangeQuery.ResultIterator(query, m_transaction, query.Transform); } public override AsyncIterator Take(int limit) diff --git a/FoundationDB.Client/FdbStreamingMode.cs b/FoundationDB.Client/FdbStreamingMode.cs index 89779b297..ee61038c3 100644 --- a/FoundationDB.Client/FdbStreamingMode.cs +++ b/FoundationDB.Client/FdbStreamingMode.cs @@ -29,8 +29,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using JetBrains.Annotations; /// Defines how the client would like the data in a range a returned + [PublicAPI] public enum FdbStreamingMode { diff --git a/FoundationDB.Client/FdbTransaction.Snapshot.cs b/FoundationDB.Client/FdbTransaction.Snapshot.cs index dcdbc89f9..6f117de1e 100644 --- a/FoundationDB.Client/FdbTransaction.Snapshot.cs +++ b/FoundationDB.Client/FdbTransaction.Snapshot.cs @@ -28,11 +28,11 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { - using FoundationDB.Client.Utils; using System; using System.Collections.Generic; using System.Threading; using System.Threading.Tasks; + using JetBrains.Annotations; /// Wraps an FDB_TRANSACTION handle public partial class FdbTransaction @@ -53,46 +53,28 @@ public IFdbReadOnlyTransaction Snapshot } /// Wrapper on a transaction, that will use Snmapshot mode on all read operations - private sealed class Snapshotted : IFdbReadOnlyTransaction, IDisposable + private sealed class Snapshotted : IFdbReadOnlyTransaction { private readonly FdbTransaction m_parent; - public Snapshotted(FdbTransaction parent) + public Snapshotted([NotNull] FdbTransaction parent) { - if (parent == null) throw new ArgumentNullException("parent"); + if (parent == null) throw new ArgumentNullException(nameof(parent)); m_parent = parent; } - public int Id - { - get { return m_parent.Id; } - } + public int Id => m_parent.Id; - public FdbOperationContext Context - { - get { return m_parent.Context; } - } + public FdbOperationContext Context => m_parent.Context; - public CancellationToken Cancellation - { - get { return m_parent.Cancellation; } - } + public CancellationToken Cancellation => m_parent.Cancellation; - public bool IsSnapshot - { - get { return true; } - } + public bool IsSnapshot => true; - public IFdbReadOnlyTransaction Snapshot - { - get { return this; } - } + public IFdbReadOnlyTransaction Snapshot => this; - public FdbIsolationLevel IsolationLevel - { - //TODO: not all transaction handlers may support Snapshot isolation level?? - get { return FdbIsolationLevel.Snapshot; } - } + public FdbIsolationLevel IsolationLevel => FdbIsolationLevel.Snapshot; + //TODO: not all transaction handlers may support Snapshot isolation level?? public void EnsureCanRead() { @@ -116,7 +98,7 @@ public Task GetAsync(Slice key) m_parent.m_database.EnsureKeyIsValid(ref key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", String.Format("Getting value for '{0}'", key.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetAsync", $"Getting value for '{key.ToString()}'"); #endif return m_parent.m_handler.GetAsync(key, snapshot: true, ct: m_parent.m_cancellation); @@ -124,14 +106,14 @@ public Task GetAsync(Slice key) public Task GetValuesAsync(Slice[] keys) { - if (keys == null) throw new ArgumentNullException("keys"); + if (keys == null) throw new ArgumentNullException(nameof(keys)); EnsureCanRead(); m_parent.m_database.EnsureKeysAreValid(keys); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", String.Format("Getting batch of {0} values ...", keys.Length)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetValuesAsync", $"Getting batch of {keys.Length} values ..."); #endif return m_parent.m_handler.GetValuesAsync(keys, snapshot: true, ct: m_parent.m_cancellation); @@ -144,7 +126,7 @@ public async Task GetKeyAsync(KeySelector selector) m_parent.m_database.EnsureKeyIsValid(selector.Key); #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", String.Format("Getting key '{0}'", selector.ToString())); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeyAsync", $"Getting key '{selector.ToString()}'"); #endif var key = await m_parent.m_handler.GetKeyAsync(selector, snapshot: true, ct: m_parent.m_cancellation).ConfigureAwait(false); @@ -164,7 +146,7 @@ public Task GetKeysAsync(KeySelector[] selectors) } #if DEBUG - if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysCoreAsync", String.Format("Getting batch of {0} keys ...", selectors.Length)); + if (Logging.On && Logging.IsVerbose) Logging.Verbose(this, "GetKeysCoreAsync", $"Getting batch of {selectors.Length} keys ..."); #endif return m_parent.m_handler.GetKeysAsync(selectors, snapshot: true, ct: m_parent.m_cancellation); @@ -231,20 +213,20 @@ public void SetOption(FdbTransactionOption option, long value) public int Timeout { - get { return m_parent.Timeout; } - set { throw new NotSupportedException("The timeout value cannot be changed via the Snapshot view of a transaction."); } + get => m_parent.Timeout; + set => throw new NotSupportedException("The timeout value cannot be changed via the Snapshot view of a transaction."); } public int RetryLimit { - get { return m_parent.RetryLimit; } - set { throw new NotSupportedException("The retry limit value cannot be changed via the Snapshot view of a transaction."); } + get => m_parent.RetryLimit; + set => throw new NotSupportedException("The retry limit value cannot be changed via the Snapshot view of a transaction."); } public int MaxRetryDelay { - get { return m_parent.MaxRetryDelay; } - set { throw new NotSupportedException("The max retry delay value cannot be changed via the Snapshot view of a transaction."); } + get => m_parent.MaxRetryDelay; + set => throw new NotSupportedException("The max retry delay value cannot be changed via the Snapshot view of a transaction."); } void IDisposable.Dispose() diff --git a/FoundationDB.Client/FdbTransactionExtensions.cs b/FoundationDB.Client/FdbTransactionExtensions.cs index e59d39fc2..c57ef0d47 100644 --- a/FoundationDB.Client/FdbTransactionExtensions.cs +++ b/FoundationDB.Client/FdbTransactionExtensions.cs @@ -41,6 +41,7 @@ namespace FoundationDB.Client using JetBrains.Annotations; /// Provides a set of extensions methods shared by all FoundationDB transaction implementations. + [PublicAPI] public static class FdbTransactionExtensions { diff --git a/FoundationDB.Client/FdbTransactionOption.cs b/FoundationDB.Client/FdbTransactionOption.cs index 98c5726dd..5217a754c 100644 --- a/FoundationDB.Client/FdbTransactionOption.cs +++ b/FoundationDB.Client/FdbTransactionOption.cs @@ -29,8 +29,10 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY namespace FoundationDB.Client { using System; + using JetBrains.Annotations; /// Defines a set of options for a transaction + [PublicAPI] public enum FdbTransactionOption { /// None diff --git a/FoundationDB.Client/FdbWatch.cs b/FoundationDB.Client/FdbWatch.cs index cbf17dafc..84f6fa17b 100644 --- a/FoundationDB.Client/FdbWatch.cs +++ b/FoundationDB.Client/FdbWatch.cs @@ -54,32 +54,24 @@ internal FdbWatch(FdbFuture future, Slice key, Slice value) } /// Key that is being watched - public Slice Key { get { return m_key; } } + public Slice Key => m_key; /// Original value of the key, at the time the watch was created (optional) /// This property will return Slice.Nil if the original value was not known at the creation of this Watch instance. - public Slice Value { get { return m_value; } internal set { m_value = value; } } - - /// Returns true if the watch is still active, or false if it fired or was cancelled - public bool IsAlive + public Slice Value { - get { return m_future != null && !m_future.Task.IsCompleted; } + get => m_value; + internal set => m_value = value; } + /// Returns true if the watch is still active, or false if it fired or was cancelled + public bool IsAlive => m_future != null && !m_future.Task.IsCompleted; + /// Returns true if the watch has fired signaling that the key may have changed in the database - public bool HasChanged - { - get { return m_future != null && m_future.Task.Status == TaskStatus.RanToCompletion; } - } + public bool HasChanged => m_future != null && m_future.Task.Status == TaskStatus.RanToCompletion; /// Task that will complete when the watch fires, or is cancelled. It will return the watched key, or an exception. - public Task Task - { - get - { - return m_future != null ? m_future.Task : null; - } - } + public Task Task => m_future?.Task; /// Returns an awaiter for the Watch public TaskAwaiter GetAwaiter() @@ -100,19 +92,13 @@ public TaskAwaiter GetAwaiter() /// Cancel the watch. It will immediately stop monitoring the key. Has no effect if the watch has already fired public void Cancel() { - if (m_future != null) - { - m_future.Cancel(); - } + m_future?.Cancel(); } /// Dispose the resources allocated by the watch. public void Dispose() { - if (m_future != null) - { - m_future.Dispose(); - } + m_future?.Dispose(); } public override string ToString() diff --git a/FoundationDB.Client/IFdbCluster.cs b/FoundationDB.Client/IFdbCluster.cs index 4a9db7589..3322e1320 100644 --- a/FoundationDB.Client/IFdbCluster.cs +++ b/FoundationDB.Client/IFdbCluster.cs @@ -38,7 +38,8 @@ namespace FoundationDB.Client public interface IFdbCluster : IDisposable { /// Path to the cluster file used by this connection, or null if the default cluster file is being used - string Path { [CanBeNull] get; } + [CanBeNull] + string Path { get; } /// Set an option on this cluster that does not take any parameter /// Option to set diff --git a/FoundationDB.Client/IFdbDatabase.cs b/FoundationDB.Client/IFdbDatabase.cs index d2d3ba190..5cc046bb0 100644 --- a/FoundationDB.Client/IFdbDatabase.cs +++ b/FoundationDB.Client/IFdbDatabase.cs @@ -34,13 +34,15 @@ namespace FoundationDB.Client /// Database connection context. [PublicAPI] - public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IDynamicKeySubspace, IDisposable + public interface IFdbDatabase : IFdbRetryable, IDynamicKeySubspace, IDisposable { /// Name of the database - string Name { [NotNull] get; } + [NotNull] + string Name { get; } /// Cluster of the database - IFdbCluster Cluster { [NotNull] get; } + [NotNull] + IFdbCluster Cluster { get; } /// Returns a cancellation token that is linked with the lifetime of this database instance /// The token will be cancelled if the database instance is disposed @@ -48,10 +50,12 @@ public interface IFdbDatabase : IFdbReadOnlyRetryable, IFdbRetryable, IDynamicKe /// Returns the global namespace used by this database instance /// Makes a copy of the subspace tuple, so you should not call this property a lot. Use any of the Partition(..) methods to create a subspace of the database - IDynamicKeySubspace GlobalSpace { [NotNull] get; } + [NotNull] + IDynamicKeySubspace GlobalSpace { get; } /// Directory partition of this database instance - FdbDatabasePartition Directory { [NotNull] get; } + [NotNull] + FdbDatabasePartition Directory { get; } /// If true, this database instance will only allow starting read-only transactions. bool IsReadOnly { get; } diff --git a/FoundationDB.Client/IFdbReadOnlyTransaction.cs b/FoundationDB.Client/IFdbReadOnlyTransaction.cs index 1ed8ac0c9..ccc8c9218 100644 --- a/FoundationDB.Client/IFdbReadOnlyTransaction.cs +++ b/FoundationDB.Client/IFdbReadOnlyTransaction.cs @@ -44,7 +44,8 @@ public interface IFdbReadOnlyTransaction : IDisposable int Id { get; } /// Context of this transaction. - FdbOperationContext Context { [NotNull] get; } + [NotNull] + FdbOperationContext Context { get; } /// Isolation Level of this transaction. FdbIsolationLevel IsolationLevel { get; } @@ -53,7 +54,8 @@ public interface IFdbReadOnlyTransaction : IDisposable bool IsSnapshot { get; } /// Return a Snapshotted version of this transaction, or the transaction itself it is already operating in Snapshot mode. - IFdbReadOnlyTransaction Snapshot { [NotNull] get; } + [NotNull] + IFdbReadOnlyTransaction Snapshot { get; } /// Cancellation Token linked to the life time of the transaction /// Will be triggered if the transaction is aborted or disposed diff --git a/FoundationDB.Client/IFdbRetryable.cs b/FoundationDB.Client/IFdbRetryable.cs index a2a87989b..5efbcb297 100644 --- a/FoundationDB.Client/IFdbRetryable.cs +++ b/FoundationDB.Client/IFdbRetryable.cs @@ -78,7 +78,7 @@ public interface IFdbRetryable : IFdbReadOnlyRetryable /// Since the handler can run more than once, and that there is no guarantee that the transaction commits once it returns, you MAY NOT mutate any global state (counters, cache, global dictionary) inside this lambda! /// You must wait for the Task to complete successfully before updating the global state of the application. /// - Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken ct); + Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, CancellationToken ct); //REVIEW: should we keep these ? @@ -92,7 +92,7 @@ public interface IFdbRetryable : IFdbReadOnlyRetryable Task ReadWriteAsync([NotNull, InstantHandle] Func asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); /// [EXPERIMENTAL] do not use yet!. - Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); + Task ReadWriteAsync([NotNull, InstantHandle] Func> asyncHandler, [NotNull, InstantHandle] Action onDone, CancellationToken ct); } } diff --git a/FoundationDB.Client/Shared/Tuples/STuple`3.cs b/FoundationDB.Client/Shared/Tuples/STuple`3.cs index 29b420018..edcff3d8c 100644 --- a/FoundationDB.Client/Shared/Tuples/STuple`3.cs +++ b/FoundationDB.Client/Shared/Tuples/STuple`3.cs @@ -34,7 +34,6 @@ namespace Doxense.Collections.Tuples using System.ComponentModel; using System.Diagnostics; using System.Runtime.CompilerServices; - using Doxense.Collections.Tuples.Encoding; using Doxense.Diagnostics.Contracts; using Doxense.Runtime.Converters; using JetBrains.Annotations; diff --git a/FoundationDb.Client.sln.DotSettings b/FoundationDb.Client.sln.DotSettings index e627bc3ce..7be8823a8 100644 --- a/FoundationDb.Client.sln.DotSettings +++ b/FoundationDb.Client.sln.DotSettings @@ -59,6 +59,7 @@ UseVarWhenEvident UseVarWhenEvident BE + TLS <Policy Inspect="True" Prefix="" Suffix="" Style="AaBb"><ExtraRule Prefix="" Suffix="" Style="AA_BB" /></Policy> <Policy Inspect="True" Prefix="" Suffix="" Style="AA_BB" /> <Policy Inspect="True" Prefix="" Suffix="" Style="aaBb"><ExtraRule Prefix="_" Suffix="" Style="aaBb" /></Policy> From 69f03599a53264a01289ac53275cc2daec58dac0 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Fri, 27 Apr 2018 20:06:16 +0200 Subject: [PATCH 142/153] Remove usage of Slice.DefaultEncoding --- FoundationDB.Client/Native/FdbNative.cs | 5 +++-- FoundationDB.Client/Shared/Memory/Slice.Encoding.cs | 12 ++---------- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 3fde8d049..460032f0f 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -36,6 +36,7 @@ namespace FoundationDB.Client.Native using System.IO; using System.Runtime.ExceptionServices; using System.Runtime.InteropServices; + using System.Text; using Doxense.Diagnostics.Contracts; internal static unsafe class FdbNative @@ -303,11 +304,11 @@ public static Slice ToNativeString(string value, bool nullTerminated) if (nullTerminated) { // NULL terminated ANSI string result = new byte[value.Length + 1]; - Slice.DefaultEncoding.GetBytes(value, 0, value.Length, result, 0); + Encoding.Default.GetBytes(value, 0, value.Length, result, 0); } else { - result = Slice.DefaultEncoding.GetBytes(value); + result = Encoding.Default.GetBytes(value); } return Slice.CreateUnsafe(result, 0, result.Length); } diff --git a/FoundationDB.Client/Shared/Memory/Slice.Encoding.cs b/FoundationDB.Client/Shared/Memory/Slice.Encoding.cs index 36dadcb97..64057c81b 100644 --- a/FoundationDB.Client/Shared/Memory/Slice.Encoding.cs +++ b/FoundationDB.Client/Shared/Memory/Slice.Encoding.cs @@ -594,14 +594,6 @@ public static Slice FromUuid64(Uuid64 value) return value.ToSlice(); } - /// Encoding used to produce ASCII slices - [NotNull] - internal static readonly ASCIIEncoding AsciiEncoding = new ASCIIEncoding(); - - /// Encoding used to produce ANSI slices - [NotNull] - internal static readonly Encoding DefaultEncoding = Encoding.Default; - /// Encoding used to produce UTF-8 slices [NotNull] internal static readonly UTF8Encoding Utf8NoBomEncoding = new UTF8Encoding(encoderShouldEmitUTF8Identifier: false, throwOnInvalidBytes: true); @@ -616,7 +608,7 @@ public static Slice FromStringAnsi([CanBeNull] string text) { return text == null ? Slice.Nil : text.Length == 0 ? Slice.Empty - : new Slice(DefaultEncoding.GetBytes(text)); + : new Slice(Encoding.Default.GetBytes(text)); } /// Create a slice from an ASCII string, where all the characters map directory into bytes (0..255). The string will be checked before being encoded. @@ -1426,7 +1418,7 @@ public string ToStringAnsi() { if (this.Count == 0) return this.Array != null ? String.Empty : default(string); //note: Encoding.GetString() will do the bound checking for us - return Slice.DefaultEncoding.GetString(this.Array, this.Offset, this.Count); + return Encoding.Default.GetString(this.Array, this.Offset, this.Count); } /// Stringify a slice containing 7-bit ASCII characters only From 5299aace1f8daf0b1465a649187023d8677ca9ac Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Fri, 27 Apr 2018 20:12:15 +0500 Subject: [PATCH 143/153] loading: on non-windows use libdl to load libfdb_c --- .../Native/UnmanagedLibrary.cs | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/FoundationDB.Client/Native/UnmanagedLibrary.cs b/FoundationDB.Client/Native/UnmanagedLibrary.cs index 62659bbdd..681e95e67 100644 --- a/FoundationDB.Client/Native/UnmanagedLibrary.cs +++ b/FoundationDB.Client/Native/UnmanagedLibrary.cs @@ -72,34 +72,32 @@ protected override bool ReleaseHandle() [SuppressUnmanagedCodeSecurity] private static class NativeMethods { -#if __MonoCS__ - const string KERNEL = "dl"; + const string LIBDL = "dl"; + - [DllImport(KERNEL)] + [DllImport(LIBDL)] public static extern SafeLibraryHandle dlopen(string fileName, int flags); - [DllImport(KERNEL, SetLastError = true)] - [return: MarshalAs(UnmanagedType.Bool)] - public static extern int dlclose(IntPtr hModule); +#if __MonoCS__ - public static SafeLibraryHandle LoadLibrary(string fileName) + public static SafeLibraryHandle LoadPlatformLibrary(string fileName) { - return dlopen(fileName, 1); - } - public static bool FreeLibrary(IntPtr hModule) { return dlclose(hModule) == 0; } - #else const string KERNEL = "kernel32"; [DllImport(KERNEL, CharSet = CharSet.Auto, BestFitMapping = false, SetLastError = true)] public static extern SafeLibraryHandle LoadLibrary(string fileName); - [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] - [DllImport(KERNEL, SetLastError = true)] - [return: MarshalAs(UnmanagedType.Bool)] - public static extern bool FreeLibrary(IntPtr hModule); + public static SafeLibraryHandle LoadPlatformLibrary(string fileName) + { + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + return LoadLibrary(fileName); + } + return dlopen(fileName, 1); + } #endif } @@ -112,7 +110,7 @@ public static UnmanagedLibrary Load(string path) { if (path == null) throw new ArgumentNullException("path"); - var handle = NativeMethods.LoadLibrary(path); + var handle = NativeMethods.LoadPlatformLibrary(path); if (handle == null || handle.IsInvalid) { var ex = Marshal.GetExceptionForHR(Marshal.GetHRForLastWin32Error()); From 26667b0848051a8725fe36221a93e09e7ff693a0 Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Fri, 27 Apr 2018 22:23:12 +0500 Subject: [PATCH 144/153] Add platform-specific library closing methods Although FreeLibrary and dlclose aren't used right now, they will be needed later after #54 is resolved --- .../Native/UnmanagedLibrary.cs | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/FoundationDB.Client/Native/UnmanagedLibrary.cs b/FoundationDB.Client/Native/UnmanagedLibrary.cs index 681e95e67..70934b092 100644 --- a/FoundationDB.Client/Native/UnmanagedLibrary.cs +++ b/FoundationDB.Client/Native/UnmanagedLibrary.cs @@ -78,18 +78,30 @@ private static class NativeMethods [DllImport(LIBDL)] public static extern SafeLibraryHandle dlopen(string fileName, int flags); + + [DllImport(LIBDL, SetLastError = true)] + [return: MarshalAs(UnmanagedType.Bool)] + public static extern int dlclose(IntPtr hModule); + #if __MonoCS__ public static SafeLibraryHandle LoadPlatformLibrary(string fileName) { return dlopen(fileName, 1); } + public static bool FreePlatformLibrary(IntPtr hModule) { return dlclose(hModule) == 0; } + #else const string KERNEL = "kernel32"; [DllImport(KERNEL, CharSet = CharSet.Auto, BestFitMapping = false, SetLastError = true)] public static extern SafeLibraryHandle LoadLibrary(string fileName); + [ReliabilityContract(Consistency.WillNotCorruptState, Cer.Success)] + [DllImport(KERNEL, SetLastError = true)] + [return: MarshalAs(UnmanagedType.Bool)] + public static extern bool FreeLibrary(IntPtr hModule); + public static SafeLibraryHandle LoadPlatformLibrary(string fileName) { if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) @@ -98,6 +110,15 @@ public static SafeLibraryHandle LoadPlatformLibrary(string fileName) } return dlopen(fileName, 1); } + + public static bool FreePlatformLibrary(IntPtr hModule) + { + if (RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + return FreeLibrary(hModule); + } + return return dlclose(hModule) == 0; + } #endif } From 3baba6ce67f0e2a61ec3e2b4d99ee1960f87e372 Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Fri, 27 Apr 2018 23:45:06 +0500 Subject: [PATCH 145/153] clean-up static FdbNative ctor to support osx and linux --- FoundationDB.Client/Native/FdbNative.cs | 59 +++++++++++-------- .../Native/UnmanagedLibrary.cs | 2 +- 2 files changed, 35 insertions(+), 26 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 460032f0f..679dff78c 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -49,7 +49,7 @@ internal static unsafe class FdbNative private const string FDB_C_DLL = "libfdb_c.so"; #else /// Name of the C API dll used for P/Invoking - private const string FDB_C_DLL = "fdb_c.dll"; + private const string FDB_C_DLL = "fdb_c"; #endif /// Handle on the native FDB C API library @@ -237,40 +237,49 @@ static FdbNative() { // Impact of NativeLibPath: // - If null, don't preload the library, and let the CLR find the file using the default P/Invoke behavior - // - If String.Empty, call win32 LoadLibrary("fdb_c.dll") and let the os find the file (using the standard OS behavior) - // - Else, combine the path with "fdb_c.dll" and call LoadLibrary with the resulting (relative or absolute) path + // - If String.Empty, call win32 LoadLibrary(FDB_C_DLL) and let the os find the file (using the standard OS behavior) + // - If path is folder, append the FDB_C_DLL + // Afterwards - call LoadLibrary with the resulting (relative or absolute) path var libraryPath = Fdb.Options.NativeLibPath; - if (libraryPath != null) + + if (libraryPath == null) + { + return; + } + + try { - try + if (libraryPath.Length == 0) + { // CLR will handle the search + libraryPath = FDB_C_DLL; + } + else { - if (libraryPath.Length == 0) - { // CLR will handle the search - libraryPath = FDB_C_DLL; - } - else if (!libraryPath.EndsWith(".dll", StringComparison.OrdinalIgnoreCase)) - { // add the file name - libraryPath = Path.Combine(Fdb.Options.NativeLibPath, FDB_C_DLL); + var fileName = Path.GetFileName(libraryPath); + if (String.IsNullOrEmpty(fileName)) + { + libraryPath = Path.Combine(libraryPath, FDB_C_DLL); } + } - FdbCLib = UnmanagedLibrary.Load(libraryPath); + FdbCLib = UnmanagedLibrary.Load(libraryPath); + } + catch (Exception e) + { + if (FdbCLib != null) FdbCLib.Dispose(); + FdbCLib = null; + if (e is BadImageFormatException && IntPtr.Size == 4) + { + e = new InvalidOperationException("The native FDB client is 64-bit only, and cannot be loaded in a 32-bit process.", e); } - catch (Exception e) + else { - if (FdbCLib != null) FdbCLib.Dispose(); - FdbCLib = null; - if (e is BadImageFormatException && IntPtr.Size == 4) - { - e = new InvalidOperationException("The native FDB client is 64-bit only, and cannot be loaded in a 32-bit process.", e); - } - else - { - e = new InvalidOperationException("An error occurred while loading the native FoundationDB library", e); - } - LibraryLoadError = ExceptionDispatchInfo.Capture(e); + e = new InvalidOperationException("An error occurred while loading the native FoundationDB library", e); } + LibraryLoadError = ExceptionDispatchInfo.Capture(e); } + } private static void EnsureLibraryIsLoaded() diff --git a/FoundationDB.Client/Native/UnmanagedLibrary.cs b/FoundationDB.Client/Native/UnmanagedLibrary.cs index 70934b092..968045b0a 100644 --- a/FoundationDB.Client/Native/UnmanagedLibrary.cs +++ b/FoundationDB.Client/Native/UnmanagedLibrary.cs @@ -117,7 +117,7 @@ public static bool FreePlatformLibrary(IntPtr hModule) { return FreeLibrary(hModule); } - return return dlclose(hModule) == 0; + return dlclose(hModule) == 0; } #endif } From 1cbc0733e0088e69d935b68ea0f10ed83f902765 Mon Sep 17 00:00:00 2001 From: Rinat Abdullin Date: Sat, 28 Apr 2018 10:40:34 +0500 Subject: [PATCH 146/153] Simplify library loading on non-windows We keep existing behavior as is for the windows platform and simplify non-windows loading. On non-win we load the native library only if explicitly provided --- FoundationDB.Client/Native/FdbNative.cs | 67 ++++++++++++++++--------- 1 file changed, 44 insertions(+), 23 deletions(-) diff --git a/FoundationDB.Client/Native/FdbNative.cs b/FoundationDB.Client/Native/FdbNative.cs index 679dff78c..bf4adbc06 100644 --- a/FoundationDB.Client/Native/FdbNative.cs +++ b/FoundationDB.Client/Native/FdbNative.cs @@ -235,34 +235,15 @@ public static extern void fdb_transaction_clear_range( static FdbNative() { - // Impact of NativeLibPath: - // - If null, don't preload the library, and let the CLR find the file using the default P/Invoke behavior - // - If String.Empty, call win32 LoadLibrary(FDB_C_DLL) and let the os find the file (using the standard OS behavior) - // - If path is folder, append the FDB_C_DLL - // Afterwards - call LoadLibrary with the resulting (relative or absolute) path - - var libraryPath = Fdb.Options.NativeLibPath; + var libraryPath = GetPreloadPath(); if (libraryPath == null) - { + { // PInvoke will load return; } - + try { - if (libraryPath.Length == 0) - { // CLR will handle the search - libraryPath = FDB_C_DLL; - } - else - { - var fileName = Path.GetFileName(libraryPath); - if (String.IsNullOrEmpty(fileName)) - { - libraryPath = Path.Combine(libraryPath, FDB_C_DLL); - } - } - FdbCLib = UnmanagedLibrary.Load(libraryPath); } catch (Exception e) @@ -275,13 +256,53 @@ static FdbNative() } else { - e = new InvalidOperationException("An error occurred while loading the native FoundationDB library", e); + e = new InvalidOperationException($"An error occurred while loading the native FoundationDB library: '{libraryPath}'.", e); } LibraryLoadError = ExceptionDispatchInfo.Capture(e); } } + private static string GetPreloadPath() + { + // we need to provide sensible defaults for loading the native library + // if this method returns null we'll let PInvoke deal + // otherwise - use explicit platform-specific dll loading + var libraryPath = Fdb.Options.NativeLibPath; + + // on non-windows, library loading by convention just works. + // unless override is provided, just let PInvoke do the work + if (!RuntimeInformation.IsOSPlatform(OSPlatform.Windows)) + { + if (string.IsNullOrEmpty(libraryPath)) + { + return null; + } + // otherwise just use the provided path + return libraryPath; + } + + // Impact of NativeLibPath on windows: + // - If null, don't preload the library, and let the CLR find the file using the default P/Invoke behavior + // - If String.Empty, call win32 LoadLibrary(FDB_C_DLL + ".dll") and let the os find the file (using the standard OS behavior) + // - If path is folder, append the FDB_C_DLL + var winDllWithExtension = FDB_C_DLL + ".dll"; + if (libraryPath == null) + { + return null; + } + if (libraryPath.Length == 0) + { + return winDllWithExtension; + } + var fileName = Path.GetFileName(libraryPath); + if (String.IsNullOrEmpty(fileName)) + { + libraryPath = Path.Combine(libraryPath, winDllWithExtension); + } + return libraryPath; + } + private static void EnsureLibraryIsLoaded() { // should be inlined From 007d667eb960a4e700e3edd7ccf6028d8631d02d Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 12:22:03 +0200 Subject: [PATCH 147/153] Replace Optional with tuple (T, bool) so simplify the code --- .../Collections/FdbMap`2.cs | 20 +- .../Collections/FdbMultimap`2.cs | 4 +- .../Collections/FdbQueue`1.cs | 75 ++-- .../Collections/FdbVector`1.cs | 17 +- .../Interning/FdbStringIntern.cs | 26 +- FoundationDB.Layers.Common/Optional`1.cs | 387 ------------------ FoundationDB.Tests/Layers/MapFacts.cs | 2 +- FoundationDB.Tests/Layers/QueuesFacts.cs | 22 +- 8 files changed, 76 insertions(+), 477 deletions(-) delete mode 100644 FoundationDB.Layers.Common/Optional`1.cs diff --git a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs index 9b3fdebe3..3d214eece 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMap`2.cs @@ -99,15 +99,15 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey /// Transaction used for the operation /// Key of the entry to read from the map /// Optional with the value of the entry it it exists, or an empty result if it is not present in the map. - public async Task> TryGetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey id) + public async Task<(TValue Value, bool HasValue)> TryGetAsync([NotNull] IFdbReadOnlyTransaction trans, TKey id) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (id == null) throw new ArgumentNullException(nameof(id)); var data = await trans.GetAsync(this.Subspace.Keys[id]).ConfigureAwait(false); - if (data.IsNull) return default(Optional); - return this.ValueEncoder.DecodeValue(data); + if (data.IsNull) return (default(TValue), false); + return (this.ValueEncoder.DecodeValue(data), true); } /// Add or update an entry in the map @@ -153,14 +153,22 @@ public IAsyncEnumerable> All([NotNull] IFdbReadOnlyTr /// Transaction used for the operation /// List of the keys to read /// Array of results, in the same order as specified in . - public async Task[]> GetValuesAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ids) + public async Task GetValuesAsync([NotNull] IFdbReadOnlyTransaction trans, [NotNull] IEnumerable ids) { if (trans == null) throw new ArgumentNullException(nameof(trans)); if (ids == null) throw new ArgumentNullException(nameof(ids)); - var results = await trans.GetValuesAsync(ids.Select(id => this.Subspace.Keys[id])).ConfigureAwait(false); + var kv = await trans.GetValuesAsync(ids.Select(id => this.Subspace.Keys[id])).ConfigureAwait(false); + if (kv.Length == 0) return Array.Empty(); - return Optional.DecodeRange(this.ValueEncoder, results); + var result = new TValue[kv.Length]; + var decoder = this.ValueEncoder; + for (int i = 0; i < kv.Length; i++) + { + result[i] = decoder.DecodeValue(kv[i]); + } + + return result; } #endregion diff --git a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs index 63d3f0a39..8ae8ed464 100644 --- a/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs +++ b/FoundationDB.Layers.Common/Collections/FdbMultimap`2.cs @@ -26,7 +26,6 @@ DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY */ #endregion - namespace FoundationDB.Layers.Collections { using System; @@ -34,7 +33,6 @@ namespace FoundationDB.Layers.Collections using System.Diagnostics; using System.Threading.Tasks; using Doxense.Linq; - using Doxense.Serialization.Encoders; using FoundationDB.Client; using JetBrains.Annotations; @@ -44,7 +42,7 @@ namespace FoundationDB.Layers.Collections [DebuggerDisplay("Subspace={" + nameof(FdbMultiMap.Subspace) + "}")] public class FdbMultiMap { - // Inspired by https://foundationdb.com/recipes/developer/multimaps + // Inspired by https://apple.github.io/foundationdb/multimaps.html // It is the logical equivalent of a Map, long> where the value would be incremented each time a specific pair of (key, value) is added (and subtracted when removed) // The layer stores each key/value using the following format: diff --git a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs index aa6d8c997..b29cb08cd 100644 --- a/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbQueue`1.cs @@ -40,12 +40,10 @@ namespace FoundationDB.Layers.Collections #endif using JetBrains.Annotations; - /// - /// Provides a high-contention Queue class - /// + /// Provides a high-contention Queue class public class FdbQueue { - // from https://github.com/FoundationDB/python-layers/blob/master/lib/queue.py + // from https://apple.github.io/foundationdb/queues.html // TODO: should we use a PRNG ? If two counter instances are created at the same moment, they could share the same seed ? private readonly Random Rng = new Random(); @@ -78,13 +76,15 @@ public FdbQueue([NotNull] IDynamicKeySubspace subspace, bool highContention = fa } /// Subspace used as a prefix for all items in this table - public IDynamicKeySubspace Subspace { [NotNull] get; } + [NotNull] + public IDynamicKeySubspace Subspace { get; } /// If true, the queue is operating in High Contention mode that will scale better with a lot of popping clients. public bool HighContention { get; } /// Serializer for the elements of the queue - public IValueEncoder Encoder { [NotNull] get; } + [NotNull] + public IValueEncoder Encoder { get; } internal IDynamicKeySubspace ConflictedPop { get; } @@ -119,13 +119,13 @@ public async Task PushAsync([NotNull] IFdbTransaction trans, T value) } /// Pop the next item from the queue. Cannot be composed with other functions in a single transaction. - public Task> PopAsync([NotNull] IFdbDatabase db, CancellationToken ct) + public Task<(T Value, bool HasValue)> PopAsync([NotNull] IFdbDatabase db, CancellationToken ct) { if (db == null) throw new ArgumentNullException(nameof(db)); if (ct.IsCancellationRequested) { - return Task.FromCanceled>(ct); + return Task.FromCanceled<(T, bool)>(ct); } if (this.HighContention) @@ -145,17 +145,15 @@ public async Task EmptyAsync([NotNull] IFdbReadOnlyTransaction tr) } /// Get the value of the next item in the queue without popping it. - public async Task> PeekAsync([NotNull] IFdbReadOnlyTransaction tr) + public async Task<(T Value, bool HasValue)> PeekAsync([NotNull] IFdbReadOnlyTransaction tr) { var firstItem = await GetFirstItemAsync(tr).ConfigureAwait(false); if (firstItem.Key.IsNull) { - return default(Optional); - } - else - { - return this.Encoder.DecodeValue(firstItem.Value); + return default; } + + return (this.Encoder.DecodeValue(firstItem.Value), true); } #region Bulk Operations @@ -292,17 +290,17 @@ private Task> GetFirstItemAsync([NotNull] IFdbReadOnl return tr.GetRange(range).FirstOrDefaultAsync(); } - private async Task> PopSimpleAsync([NotNull] IFdbTransaction tr) + private async Task<(T Value, bool HasValue)> PopSimpleAsync([NotNull] IFdbTransaction tr) { #if DEBUG tr.Annotate("PopSimple()"); #endif var firstItem = await GetFirstItemAsync(tr).ConfigureAwait(false); - if (firstItem.Key.IsNull) return default(Optional); + if (firstItem.Key.IsNull) return default; tr.Clear(firstItem.Key); - return this.Encoder.DecodeValue(firstItem.Value); + return (this.Encoder.DecodeValue(firstItem.Value), true); } private Task AddConflictedPopAsync([NotNull] IFdbDatabase db, bool forced, CancellationToken ct) @@ -339,7 +337,7 @@ private Task>> GetItemsAsync([NotNull] IFdbReadO private async Task FulfillConflictedPops([NotNull] IFdbDatabase db, CancellationToken ct) { - const int numPops = 100; + const int NUM_POPS = 100; using (var tr = db.BeginTransaction(ct)) { @@ -348,8 +346,8 @@ private async Task FulfillConflictedPops([NotNull] IFdbDatabase db, Cancel #endif var ts = await Task.WhenAll( - GetWaitingPopsAsync(tr.Snapshot, numPops), - GetItemsAsync(tr.Snapshot, numPops) + GetWaitingPopsAsync(tr.Snapshot, NUM_POPS), + GetItemsAsync(tr.Snapshot, NUM_POPS) ).ConfigureAwait(false); var pops = ts[0]; @@ -397,11 +395,11 @@ private async Task FulfillConflictedPops([NotNull] IFdbDatabase db, Cancel // commit await tr.CommitAsync().ConfigureAwait(false); - return pops.Count < numPops; + return pops.Count < NUM_POPS; } } - private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db, CancellationToken ct) + private async Task<(T Value, bool HasValue)> PopHighContentionAsync([NotNull] IFdbDatabase db, CancellationToken ct) { int backOff = 10; Slice waitKey = Slice.Empty; @@ -414,7 +412,6 @@ private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db tr.Annotate("PopHighContention()"); #endif - FdbException error = null; try { // Check if there are other people waiting to be popped. If so, we cannot pop before them. @@ -430,13 +427,7 @@ private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db await tr.CommitAsync().ConfigureAwait(false); } } - catch (FdbException e) - { - // note: cannot await inside a catch(..) block, so flag the error and process it below - error = e; - } - - if (error != null) + catch (FdbException) { // If we didn't succeed, then register our pop request waitKey = await AddConflictedPopAsync(db, forced: true, ct: ct).ConfigureAwait(false); } @@ -451,7 +442,6 @@ private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db while (!ct.IsCancellationRequested) { - error = null; try { while (!(await FulfillConflictedPops(db, ct).ConfigureAwait(false))) @@ -459,24 +449,18 @@ private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db //NOP ? } } - catch (FdbException e) - { - // cannot await in catch(..) block so process it below - error = e; - } - - if (error != null && error.Code != FdbError.NotCommitted) + catch (FdbException e) when (e.Code != FdbError.NotCommitted) { // If the error is 1020 (not_committed), then there is a good chance // that somebody else has managed to fulfill some outstanding pops. In // that case, we proceed to check whether our request has been fulfilled. // Otherwise, we handle the error in the usual fashion. - await tr.OnErrorAsync(error.Code).ConfigureAwait(false); + await tr.OnErrorAsync(e.Code).ConfigureAwait(false); continue; } - error = null; + try { tr.Reset(); @@ -504,22 +488,17 @@ private async Task> PopHighContentionAsync([NotNull] IFdbDatabase db if (result.IsNullOrEmpty) { - return default(Optional); + return default; } tr.Clear(resultKey); await tr.CommitAsync().ConfigureAwait(false); - return this.Encoder.DecodeValue(result); + return (this.Encoder.DecodeValue(result), true); } catch (FdbException e) { - error = e; - } - - if (error != null) - { - await tr.OnErrorAsync(error.Code).ConfigureAwait(false); + await tr.OnErrorAsync(e.Code).ConfigureAwait(false); } } diff --git a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs index f3879a705..8683600a1 100644 --- a/FoundationDB.Layers.Common/Collections/FdbVector`1.cs +++ b/FoundationDB.Layers.Common/Collections/FdbVector`1.cs @@ -39,9 +39,10 @@ namespace FoundationDB.Layers.Collections using JetBrains.Annotations; /// Represents a potentially sparse array in FoundationDB. + [PublicAPI] public class FdbVector { - // from https://github.com/FoundationDB/python-layers/blob/master/lib/vector.py + // from https://apple.github.io/foundationdb/vector.html // Vector stores each of its values using its index as the key. // The size of a vector is equal to the index of its last key + 1. @@ -127,7 +128,7 @@ public Task FrontAsync([NotNull] IFdbReadOnlyTransaction tr) } /// Get and pops the last item off the Vector. - public async Task> PopAsync([NotNull] IFdbTransaction tr) + public async Task<(T Value, bool HasValue)> PopAsync([NotNull] IFdbTransaction tr) { if (tr == null) throw new ArgumentNullException(nameof(tr)); @@ -142,7 +143,7 @@ public async Task> PopAsync([NotNull] IFdbTransaction tr) .ConfigureAwait(false); // Vector was empty - if (lastTwo.Count == 0) return default(Optional); + if (lastTwo.Count == 0) return default; //note: keys are reversed so indices[0] = last, indices[1] = second to last var indices = lastTwo.Select(kvp => this.Subspace.Keys.DecodeFirst(kvp.Key)).ToList(); @@ -158,7 +159,7 @@ public async Task> PopAsync([NotNull] IFdbTransaction tr) tr.Clear(lastTwo[0].Key); - return this.Encoder.DecodeValue(lastTwo[0].Value); + return (this.Encoder.DecodeValue(lastTwo[0].Value), true); } /// Swap the items at positions i1 and i2. @@ -166,14 +167,14 @@ public async Task SwapAsync([NotNull] IFdbTransaction tr, long index1, long inde { if (tr == null) throw new ArgumentNullException(nameof(tr)); - if (index1 < 0 || index2 < 0) throw new IndexOutOfRangeException(String.Format("Indices ({0}, {1}) must be positive", index1, index2)); + if (index1 < 0 || index2 < 0) throw new IndexOutOfRangeException($"Indices ({index1}, {index2}) must be positive"); var k1 = GetKeyAt(index1); var k2 = GetKeyAt(index2); long currentSize = await ComputeSizeAsync(tr).ConfigureAwait(false); - if (index1 >= currentSize || index2 >= currentSize) throw new IndexOutOfRangeException(String.Format("Indices ({0}, {1}) are out of range", index1, index2)); + if (index1 >= currentSize || index2 >= currentSize) throw new IndexOutOfRangeException($"Indices ({index1}, {index2}) are out of range"); var vs = await tr.GetValuesAsync(new[] { k1, k2 }).ConfigureAwait(false); var v1 = vs[0]; @@ -202,7 +203,7 @@ public async Task SwapAsync([NotNull] IFdbTransaction tr, long index1, long inde public async Task GetAsync([NotNull] IFdbReadOnlyTransaction tr, long index) { if (tr == null) throw new ArgumentNullException(nameof(tr)); - if (index < 0) throw new IndexOutOfRangeException(String.Format("Index {0} must be positive", index)); + if (index < 0) throw new IndexOutOfRangeException($"Index {index} must be positive"); var start = GetKeyAt(index); var end = this.Subspace.Keys.ToRange().End; @@ -224,7 +225,7 @@ public async Task GetAsync([NotNull] IFdbReadOnlyTransaction tr, long index) } // We requested a value past the end of the vector - throw new IndexOutOfRangeException(String.Format("Index {0} out of range", index)); + throw new IndexOutOfRangeException($"Index {index} out of range"); } /// [NOT YET IMPLEMENTED] Get a range of items in the Vector, returned as an async sequence. diff --git a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs index 050f64c76..f12879b9f 100644 --- a/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs +++ b/FoundationDB.Layers.Common/Interning/FdbStringIntern.cs @@ -39,7 +39,7 @@ namespace FoundationDB.Layers.Interning using FoundationDB.Client; /// Provides a class for interning (aka normalizing, aliasing) commonly-used long strings into shorter representations. - [DebuggerDisplay("Subspace={Subspace}")] + [DebuggerDisplay("Subspace={" + nameof(FdbStringIntern.Subspace) + "}")] [Obsolete("FIXME! This version of the layer has a MAJOR bug!")] public class FdbStringIntern : IDisposable { @@ -92,13 +92,13 @@ public override int GetHashCode() public FdbStringIntern(IKeySubspace subspace) { - if (subspace == null) throw new ArgumentNullException("subspace"); + if (subspace == null) throw new ArgumentNullException(nameof(subspace)); this.Subspace = subspace.AsDynamic(); } - public IDynamicKeySubspace Subspace { get; private set; } + public IDynamicKeySubspace Subspace { get; } #region Private Helpers... @@ -134,8 +134,7 @@ private void EvictCache() m_uidsInCache.RemoveAt(m_uidsInCache.Count - 1); // remove from caches, account for bytes - string value; - if (!m_uidStringCache.TryGetValue(uidKey, out value) || value == null) + if (!m_uidStringCache.TryGetValue(uidKey, out string value) || value == null) { throw new InvalidOperationException("Error in cache evication: string not found"); } @@ -220,7 +219,7 @@ private async Task FindUidAsync(IFdbTransaction trans) } //TODO: another way ? - throw new InvalidOperationException("Failed to find a free uid for interned string after " + MAX_TRIES + " attempts"); + throw new InvalidOperationException($"Failed to find a free uid for interned string after {MAX_TRIES} attempts"); } #endregion @@ -234,8 +233,8 @@ private async Task FindUidAsync(IFdbTransaction trans) /// The length of the string must not exceed the maximum FoundationDB value size public Task InternAsync(IFdbTransaction trans, string value) { - if (trans == null) throw new ArgumentNullException("trans"); - if (value == null) throw new ArgumentNullException("value"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (value == null) throw new ArgumentNullException(nameof(value)); if (value.Length == 0) return Task.FromResult(Slice.Empty); @@ -243,9 +242,7 @@ public Task InternAsync(IFdbTransaction trans, string value) Debug.WriteLine("Want to intern: " + value); #endif - Uid uidKey; - - if (m_stringUidCache.TryGetValue(value, out uidKey)) + if (m_stringUidCache.TryGetValue(value, out Uid uidKey)) { #if DEBUG_STRING_INTERNING Debug.WriteLine("> found in cache! " + uidKey); @@ -299,13 +296,12 @@ private async Task InternSlowAsync(IFdbTransaction trans, string value) /// Return the long string associated with the normalized representation public Task LookupAsync(IFdbReadOnlyTransaction trans, Slice uid) { - if (trans == null) throw new ArgumentNullException("trans"); - if (uid.IsNull) throw new ArgumentException("String uid cannot be nil", "uid"); + if (trans == null) throw new ArgumentNullException(nameof(trans)); + if (uid.IsNull) throw new ArgumentException("String uid cannot be nil", nameof(uid)); if (uid.IsEmpty) return Task.FromResult(String.Empty); - string value; - if (m_uidStringCache.TryGetValue(new Uid(uid), out value)) + if (m_uidStringCache.TryGetValue(new Uid(uid), out string value)) { return Task.FromResult(value); } diff --git a/FoundationDB.Layers.Common/Optional`1.cs b/FoundationDB.Layers.Common/Optional`1.cs deleted file mode 100644 index 259b65891..000000000 --- a/FoundationDB.Layers.Common/Optional`1.cs +++ /dev/null @@ -1,387 +0,0 @@ -#region BSD Licence -/* Copyright (c) 2013-2018, 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 - -namespace FoundationDB.Layers -{ - using FoundationDB.Client; - using System; - using System.Collections.Generic; - using System.Linq; - using System.Runtime.InteropServices; - using Doxense.Serialization.Encoders; - - /// Helper class for the value type - public static class Optional - { - #region Wrapping... - - /// Returns an with the specified value - public static Optional Return(T value) - { - return new Optional(value); - } - - /// Returns an empty - public static Optional Empty() - { - return default(Optional); - } - - /// Returns an array of from an array of values - public static Optional[] Wrap(T[] values) - { - if (values == null) return null; - var tmp = new Optional[values.Length]; - for (int i = 0; i < values.Length; i++) - { - tmp[i] = new Optional(values[i]); - } - return tmp; - } - - /// Converts a into an - /// Nullable value type - public static Optional Wrap(Nullable value) - where T : struct - { - if (!value.HasValue) - return default(Optional); - return new Optional(value.Value); - } - - /// Converts an array of into an array of - /// Nullable value type - public static Optional[] Wrap(Nullable[] values) - where T : struct - { - if (values == null) throw new ArgumentNullException("values"); - var tmp = new Optional[values.Length]; - for (int i = 0; i < values.Length; i++) - { - if (values[i].HasValue) tmp[i] = new Optional(values[i].Value); - } - return tmp; - } - - /// Transforms a sequence of into a sequence of - public static IEnumerable> AsOptional(IEnumerable> source) - where T : struct - { - if (source == null) throw new ArgumentNullException("source"); - - return source.Select(value => value.HasValue ? new Optional(value.Value) : default(Optional)); - } - - #endregion - - #region Single... - - /// Converts a into a - /// Nullable value type - public static Nullable ToNullable(this Optional value) - where T : struct - { - return !value.HasValue ? default(Nullable) : value.Value; - } - - #endregion - - #region Array... - - /// Extract the values from an array of - /// Nullable value type - /// Array of optional values - /// Default value for empty values - /// Array of values - public static T[] Unwrap(Optional[] values, T defaultValue) - { - if (values == null) throw new ArgumentNullException("values"); - - var tmp = new T[values.Length]; - for (int i = 0; i < values.Length; i++) - { - tmp[i] = values[i].GetValueOrDefault(defaultValue); - } - return tmp; - } - - /// Converts an array of into an array of - /// Nullable value type - public static Nullable[] ToNullable(Optional[] values) - where T : struct - { - if (values == null) throw new ArgumentNullException("values"); - - var tmp = new Nullable[values.Length]; - for (int i = 0; i < values.Length; i++) - { - if (values[i].HasValue) tmp[i] = values[i].Value; - } - return tmp; - } - - /// Converts an array of into an array of - /// Nullable value type - public static T[] Unwrap(Optional[] values) - where T : class - { - if (values == null) throw new ArgumentNullException("values"); - - var tmp = new T[values.Length]; - for (int i = 0; i < values.Length; i++) - { - if (values[i].HasValue) tmp[i] = values[i].Value; - } - return tmp; - } - - #endregion - - #region Enumerable... - - /// Transforms a sequence of into a sequence of values. - /// Type of the elements of - /// Sequence of optional values - /// Default value for empty entries - /// Sequence of values, using for empty entries - public static IEnumerable Unwrap(this IEnumerable> source, T defaultValue) - { - if (source == null) throw new ArgumentNullException("source"); - - return source.Select(value => value.GetValueOrDefault(defaultValue)); - } - - /// Transforms a sequence of into a sequence of - /// Type of the elements of - /// Source of optional values - /// Sequence of nullable values - public static IEnumerable> AsNullable(this IEnumerable> source) - where T : struct - { - if (source == null) throw new ArgumentNullException("source"); - - return source.Select(value => !value.HasValue ? default(Nullable) : value.Value); - } - - /// Transforms a squence of into a sequence of values - /// Type of the elements of - /// Source of optional values - /// Sequence of values, using the default of for empty entries - public static IEnumerable Unwrap(this IEnumerable> source) - where T : class - { - if (source == null) throw new ArgumentNullException("source"); - - return source.Select(value => value.GetValueOrDefault()); - } - - #endregion - - #region Decoding... - - /// Decode an array of slices into an array of - /// Type of the decoded values - /// Decoder used to produce the values - /// Array of slices to decode. Entries equal to will not be decoded and returned as an empty optional. - /// Array of decoded . - public static Optional[] DecodeRange(IValueEncoder decoder, Slice[] data) - { - if (decoder == null) throw new ArgumentNullException("decoder"); - if (data == null) throw new ArgumentNullException("data"); - - var values = new Optional[data.Length]; - for (int i = 0; i < data.Length; i++) - { - Slice item; - if ((item = data[i]).HasValue) - { - values[i] = new Optional(decoder.DecodeValue(item)); - } - } - return values; - } - - /// Decode a sequence of slices into a sequence of - /// Type of the decoded values - /// Sequence of slices to decode. Entries equal to will not be decoded and returned as an empty optional. - /// Decoder used to produce the values - /// Sequence of decoded . - public static IEnumerable> Decode(this IEnumerable source, IValueEncoder decoder) - { - if (decoder == null) throw new ArgumentNullException("decoder"); - if (source == null) throw new ArgumentNullException("source"); - - return source.Select(value => value.HasValue ? decoder.DecodeValue(value) : default(Optional)); - } - - #endregion - - } - - /// Container that is either empty (no value) or null (for reference types), or contains a value of type . - /// Type of the value - [Serializable, StructLayout(LayoutKind.Sequential)] - [Obsolete("Use Maybe instead")] - public struct Optional : IEquatable>, IEquatable - { - // This is the equivalent of Nullable that would accept reference types. - // The main difference is that, 'null' is a legal value for reference types, which is distinct from "no value" - // i.e.: new Optional(null).HasValue == true - - //REVIEW: this looks very similar to Maybe, except without the handling of errors. Maybe we could merge both? - - private readonly bool m_hasValue; - - private readonly T m_value; - - /// Initializes a new instance of the structure to the specified value. - public Optional(T value) - { - m_hasValue = true; - m_value = value; - } - - /// Gets the value of the current value. - /// This can return null for reference types! - public T Value - { - get - { - if (!m_hasValue) - { // we construct and throw the exception in a static helper, to help with inlining - NoValue(); - } - return m_value; - } - } - - /// Gets a value indicating whether the current object has a value. - public bool HasValue { get { return m_hasValue; } } - - /// Retrieves the value of the current object, or the object's default value. - public T GetValueOrDefault() - { - return m_value; - } - - /// Retrieves the value of the current object, or the specified default value. - public T GetValueOrDefault(T defaultValue) - { - return m_hasValue ? m_value : defaultValue; - } - - public override string ToString() - { - if (!m_hasValue || m_value == null) return String.Empty; - return m_value.ToString(); - } - - public bool Equals(Optional value) - { - return m_hasValue == value.m_hasValue && EqualityComparer.Default.Equals(m_value, value.m_value); - } - - public bool Equals(T value) - { - return m_hasValue && EqualityComparer.Default.Equals(m_value, value); - } - - public override int GetHashCode() - { - if (!m_hasValue || m_value == null) return 0; - return m_value.GetHashCode(); - } - - /// Indicates whether the current object is equal to a specified object. - public override bool Equals(object obj) - { - if (obj is T) return Equals((T)obj); - if (obj is Optional) return Equals((Optional)obj); - return m_hasValue ? object.Equals(m_value, obj) : object.ReferenceEquals(obj, null); - } - - public static bool operator ==(Optional a, Optional b) - { - return a.Equals(b); - } - - public static bool operator !=(Optional a, Optional b) - { - return !a.Equals(b); - } - - public static bool operator ==(Optional a, T b) - { - return a.Equals(b); - } - - public static bool operator !=(Optional a, T b) - { - return !a.Equals(b); - } - - public static bool operator ==(T a, Optional b) - { - return b.Equals(a); - } - - public static bool operator !=(T a, Optional b) - { - return !b.Equals(a); - } - - public static bool operator ==(Optional? a, Optional? b) - { - // Needed to be able to write stuff like "if (optional == null)", the compiler will automatically lift "foo == null" to nullables if foo is a struct that implements the '==' operator - return a.GetValueOrDefault().Equals(b.GetValueOrDefault()); - } - - public static bool operator !=(Optional? a, Optional? b) - { - // Needed to be able to write stuff like "if (optional != null)", the compiler will automatically lift "foo != null" to nullables if foo is a struct implements the '!=' operator - return !a.GetValueOrDefault().Equals(b.GetValueOrDefault()); - } - - public static explicit operator T(Optional value) - { - return value.Value; - } - - public static implicit operator Optional(T value) - { - return new Optional(value); - } - - private static void NoValue() - { - throw new InvalidOperationException("Nullable object must have a value."); - } - - } - -} diff --git a/FoundationDB.Tests/Layers/MapFacts.cs b/FoundationDB.Tests/Layers/MapFacts.cs index be15b1540..e1ef92a41 100644 --- a/FoundationDB.Tests/Layers/MapFacts.cs +++ b/FoundationDB.Tests/Layers/MapFacts.cs @@ -61,7 +61,7 @@ public async Task Test_FdbMap_Read_Write_Delete() var value = await map.TryGetAsync(tr, "hello"); Assert.That(value.HasValue, Is.False); - Assert.That(value.GetValueOrDefault(), Is.Null); + Assert.That(value.Value, Is.Null); } // write value diff --git a/FoundationDB.Tests/Layers/QueuesFacts.cs b/FoundationDB.Tests/Layers/QueuesFacts.cs index 45dd0e579..b88020299 100644 --- a/FoundationDB.Tests/Layers/QueuesFacts.cs +++ b/FoundationDB.Tests/Layers/QueuesFacts.cs @@ -74,26 +74,30 @@ public async Task Test_Queue_Fast() Log("Empty? " + empty); Assert.That(empty, Is.False); - Optional item = await queue.PopAsync(db, this.Cancellation); - Log("Pop item: " + item); - Assert.That((int)item, Is.EqualTo(10)); + var item = await queue.PopAsync(db, this.Cancellation); + Log($"Pop item: {item}"); + Assert.That(item.HasValue, Is.True); + Assert.That(item.Value, Is.EqualTo(10)); item = await db.ReadWriteAsync((tr) => queue.PeekAsync(tr), this.Cancellation); - Log("Next item: " + item); - Assert.That((int)item, Is.EqualTo(8)); + Log($"Next item: {item}"); + Assert.That(item.HasValue, Is.True); + Assert.That(item.Value, Is.EqualTo(8)); #if DEBUG await DumpSubspace(db, location); #endif item = await queue.PopAsync(db, this.Cancellation); - Log("Pop item: " + item); - Assert.That((int)item, Is.EqualTo(8)); + Log($"Pop item: {item}"); + Assert.That(item.HasValue, Is.True); + Assert.That(item.Value, Is.EqualTo(8)); #if DEBUG await DumpSubspace(db, location); #endif item = await queue.PopAsync(db, this.Cancellation); - Log("Pop item: " + item); - Assert.That((int)item, Is.EqualTo(6)); + Log($"Pop item: {item}"); + Assert.That(item.HasValue, Is.True); + Assert.That(item.Value, Is.EqualTo(6)); #if DEBUG await DumpSubspace(db, location); #endif From fda40e5c0691453d0c1d4297c4d26df84821f871 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:10:47 +0200 Subject: [PATCH 148/153] Remove old NET461 csproj --- .../FoundationDB.Client.NET461.csproj | 285 ------------------ .../FoundationDB.Layers.Common.NET461.csproj | 82 ----- ...dationDB.Layers.Experimental.NET461.csproj | 88 ------ .../FoundationDB.Linq.Providers.NET461.csproj | 95 ------ 4 files changed, 550 deletions(-) delete mode 100644 FoundationDB.Client/FoundationDB.Client.NET461.csproj delete mode 100644 FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj delete mode 100644 FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj delete mode 100644 FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj diff --git a/FoundationDB.Client/FoundationDB.Client.NET461.csproj b/FoundationDB.Client/FoundationDB.Client.NET461.csproj deleted file mode 100644 index 2abec7a00..000000000 --- a/FoundationDB.Client/FoundationDB.Client.NET461.csproj +++ /dev/null @@ -1,285 +0,0 @@ - - - - - Debug - AnyCPU - {773166B7-DE74-4FCC-845C-84080CC89533} - Library - Properties - FoundationDB.Client - FoundationDB.Client - v4.6.1 - 512 - - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - true - false - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - false - true - bin\Release\FoundationDB.Client.XML - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - true - - - ..\Common\foundationdb-net-client.snk - - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj b/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj deleted file mode 100644 index a7175438c..000000000 --- a/FoundationDB.Layers.Common/FoundationDB.Layers.Common.NET461.csproj +++ /dev/null @@ -1,82 +0,0 @@ - - - - - Debug - AnyCPU - {7C7717D6-A1E7-4541-AF8B-1AC762B5ED0F} - Library - Properties - FoundationDB.Layers.Common - FoundationDB.Layers.Common - v4.6.1 - 512 - - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - false - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - false - bin\Release\FoundationDB.Layers.Common.XML - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - true - - - ..\Common\foundationdb-net-client.snk - - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - - - \ No newline at end of file diff --git a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj b/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj deleted file mode 100644 index 8800e80dc..000000000 --- a/FoundationDB.Layers.Experimental/FoundationDB.Layers.Experimental.NET461.csproj +++ /dev/null @@ -1,88 +0,0 @@ - - - - - Debug - AnyCPU - {E631BCD4-386C-4EB1-AD4D-CABCE77BB4C8} - Library - Properties - FoundationDB.Layers - FoundationDB.Layers.Experimental - v4.6.1 - 512 - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - 105,108,109,114,472,660,661,628,1066 - AnyCPU - true - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - bin\Release\FoundationDB.Layers.Experimental.xml - 105,108,109,114,472,660,661,628,1066 - AnyCPU - true - - - true - - - ..\Common\foundationdb-net-client.snk - - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} - FoundationDB.Layers.Common - - - - - \ No newline at end of file diff --git a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj b/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj deleted file mode 100644 index eb329ccd5..000000000 --- a/FoundationDB.Linq.Providers/FoundationDB.Linq.Providers.NET461.csproj +++ /dev/null @@ -1,95 +0,0 @@ - - - - - Debug - AnyCPU - {FAF14E3F-6662-4084-8B92-E6697F6B9D5A} - Library - Properties - FoundationDB.Linq - FoundationDB.Linq.Providers - v4.6.1 - 512 - - - true - full - false - bin\Debug\ - DEBUG;TRACE - prompt - 4 - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - pdbonly - true - bin\Release\ - TRACE - prompt - 4 - bin\Release\FoundationDB.Linq.Providers.xml - 105,108,109,114,472,660,661,628,1066 - AnyCPU - - - true - - - ..\Common\foundationdb-net-client.snk - - - - - - - - - Properties\VersionInfo.cs - - - - - - - - - - - - - - - - - - - - - - - - - - - - {773166b7-de74-4fcc-845c-84080cc89533} - FoundationDB.Client - - - {7c7717d6-a1e7-4541-af8b-1ac762b5ed0f} - FoundationDB.Layers.Common - - - - - - \ No newline at end of file From 30d88f071c71b8434ec6762161023a0b7d3310e7 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:27:30 +0200 Subject: [PATCH 149/153] Fix FAKE build script - remove .nuget from the package (it was an old v2 that was not able to install packages anymore) - update build.bat to download latest version of nuget and install FAKE4 and NUnit3 runners - Use NUnit3 target for tests - fix path to XML comment file --- .gitignore | 3 +- .nuget/NuGet.Config | 6 -- .nuget/NuGet.exe | Bin 1664000 -> 0 bytes .nuget/NuGet.targets | 144 --------------------------------------- build.bat | 6 +- build/build.fsx | 13 ++-- build/download-nuget.ps1 | 2 +- 7 files changed, 13 insertions(+), 161 deletions(-) delete mode 100644 .nuget/NuGet.Config delete mode 100755 .nuget/NuGet.exe delete mode 100644 .nuget/NuGet.targets diff --git a/.gitignore b/.gitignore index aedd80221..5f7062a2a 100644 --- a/.gitignore +++ b/.gitignore @@ -27,7 +27,7 @@ _ReSharper*/ *.vssscc $tf*/ -nuget/ +.nuget/ packages/ *.ide/ .vs/ @@ -37,6 +37,7 @@ fdb_c.dll # Build tools and artifacts /build/tools/ /build/output/ +/build/.fake # JetBrains .idea diff --git a/.nuget/NuGet.Config b/.nuget/NuGet.Config deleted file mode 100644 index 67f8ea046..000000000 --- a/.nuget/NuGet.Config +++ /dev/null @@ -1,6 +0,0 @@ - - - - - - \ No newline at end of file diff --git a/.nuget/NuGet.exe b/.nuget/NuGet.exe deleted file mode 100755 index 8dd7e45ae75d1a55fc669f09bdef4a49b16a95dd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1664000 zcmb@v51brDng8G0Gut!MyPIS(o1G*}NCG4@y95#t*xf)#h=BYPBJw9HV!#jrhK3m; z!n(u8h=>>v5jjLu0*Hu+h=_=YxyT_RXV61L%mE@s!U)~PGbIx9YR)+sAz)z+RhYxP;P-g@-0v)07#U%5wD zSGY^U`j~?~@5r{CcWrIXJJZ~r_WZVvwxZ{qIL-4yhhBCb;aP-7`Igd2*mvb7q~(`y z?+3(#%U{l0e39o((tl&$$)h9j7Q^>Q3B&h{AOTyyzBJA2Y<>Pa!=HyQH5mJw<#}%# zd-C=;cjbr9<@v@Fl6py9lX9nIBzfN+wR-JT04 z@;c>BbO*I0)1Z{e_u9P^U+eUKH{7YZ@Vk;1yyud<_uuD?tHJh3$30i(_joshw0T~+ z{y>ixgh6iPsCF;!$3S6B)xyEqlG)gE@Oto1P}b4EFoLRo@bo4@Ff57VcA`28H6qV; zNUcIJqM^dLz(;;)XOa|R3K|yT5}%HLA;|{q3F8idUGj*$0zXDdVV~EA(pTnpdaET@ z%H18-AEZ1Th1|&7w=4N!&4Ou7f}t+S_VwULY5Bt-f<=F4ad1asidvweD^OurxJ6{v z%o#ap-jlVzQKoWI-U$?}0i*m-88+Ro2M8djPXrwmRHLL0{b4ZlmS&l1wDZ|9iXMhR z?F0#vIt=2;d?LN`L}YK!s4>C|=LWS)g`}HsXR&rQ;oNpojiwM+yOo&g21?Y{RqDxY zEJZz_m3*-_vCTWXoR6zK_PKgY^gd5vs8YH9khFbkZ76NCHndw(8$lcLp|$BK4AI11 zU$A{~Mvg{wg?K7ip)EfaX;arwgo&(OqE=x$(s(*FL(&ymHABmKxF7ctsS@jrcOcm3 zjl-fDitXqRqf`05o&4O024Dt;hB-_F4nxNrrU8ecat_mg!zho#G~h7Q$YC0A7!r4w z1{{V4I7|Z$qrDxb0kA*#5yc9+BUCUP+zG^w$Ulf?@;rnx@GJxg`Y&iipFfN!^L?Jw z(}lA6pg9?Q6ixV`sR;S$<_o)Ov*A_A7aBBo)3odj&pLGrM+McW24P>>0$X6w7VS?v+67we2K!3DW;p;*pGdy-^e^XqTCk&BlS3}iIxCx6wI-ud0$ zp_qEGaxVHcU3Bm~5&v4uUR#*+yJM1v9b_Pq=`8llTzDNKoDbO5Q^^l3gt%1B-yr0b z_FVL&aJUoZ<@VYtZ30RhR)TUcWl;%QBwG&Ry#y+qoDDUZ}zR5>i?t0ziB zmM&X;;Jq}?z}4iNi?)jV&Xu5QMzj#p2!qmJ=_q$JkaAPOs8motKyndOqZ){$8BZ;f zctJHP-ySa#eyCFaq+G7M(&@LCJL3bv2GCLuq|0wla{dJ+DR*h)R`lGP`}*{`wR);> zQ+1%OOejz2$!{!AsJ8Rrw64!vg3!~y9nmfM&(JHjkdXTmY&QSti=GVmmH7qlXu?#r zoQoGzB0L;Xc7|$6)Xm5tcAyX=NesXcL#bE0AuUeRZzyamxnlUmAi{_SMQ2mb@CO;1 z@CKK_I^`H2#6zFgJAZ=r5|M}npRBv1PB+0-$&I{~?_z%BVELMmA320?zxaA^DBtzB z@gbvIe^)8O_Vi;;Wbo^E9rK~_1yhTHh}vD*N>wr7{YpRhA#yG`2M^vyrM#(rm$Kb@ z0t9m-gcH{On6gd8Pq*qR4okT<(v@lxeedj2L48G79V4uk@zGI;5OIHWtfN!Es{Tfs z7hF)r;v#%TAlwv@Xnu@E_(b`>3UZ%96jBsl_@{|4e4O->Vfoa7Xs|3C zUqKrkSU-_`E^sXHh_{H3wj3y$NF6(iGNkqo%_dEjb|~9aMT!xuX}Drl2H$>7Id~B% zi%oy~(Ud#QQBh({?lP79cuD^%xFc+W1SNvW+I(j+>zC&c8cmtJa>&lh1*BRi(YB5)6o)5hMJ$b0xk~D8=t1Zs_fjO2>jBdO}jwebTD73^hqH zR* zUqOrb%f4u6Fj(c$k1$d=g*1Dha*hwjmh|}%z0moNZsOe?`VrXU>^C2>5cw05iFs~Nvih=k?4{riJMb)cxx#XHkZ0e@7j`+BJNJ8s$B*PfNkqn zWeh~O`_B~$>aBfIY$bDFrt5g5t9igzIuOg|6?%&5?_h;dwooV9h1!K_cdDaVL0>n; zNPYKt(pKmrqnB$l4G`Zuf2KF5O~*^hC?XgV%U!#($xAz%A00+iL{s(C0*E(=7~J~@ zFNM}U$7e`k2N0b%xh*8>F{DB#m5a z&y9C^We5xy{o+})D~%yUxL8x=iBvMS`QB3RoNtMn;d$@%BZ?B%&*5W4B<)9skV53W zXDV+EKr0N+$T^W9b`S2zD#*n(63>| zza-*y^>p^|JmN;A{HZ;pUh~s>X#loh3sOu2V0-9*7JrEJeZBLK@^(hIDPh^K9VY^p zZ2a20n?YX+OgBS}i48cPlu@i?Yg)*ZfWihz+{G<%+q1Y}jD>hIF4H6rY<$XMtUxaN z84{&k3&ZHO7C1>O1nFezUdi7T^S)+LB3dknhsi!1xQUhy;tvz<@N0ieNgBuFjtspG z{0s0b-3Nz(6+~83|e&YnmVIOknsqF^$ zG3HgBcd#bb%B~b$NYZFj*hxxFKDFGf6_JzWEzC8}6*3jwIm)KBiz#&l~D zv2K_c)C}fC7>Z!(`;@7UK&aRML|RQqsd?gw7TRfN)b7ZVxESdXk!qB8%WeBNZ0#mST_^&aT{LG&74O_*?tyC+C2Iv)&5Q_Dz$ zngl`p#!fZBsSmM?NGi(sjP3+sE?>W_tV{g$1K@y*Bh^&b7db)+f)POM}3A;Oal&Em|_}m*#0S|0f!xsVj6H5VsZMV z0Wj)&Ddl1^Cq8!c9x4-E1t8^mJIk6FPvmolz@q|J=1=v$LR`1X0y{+PKk&)MNPIot zIECV+#2-gLbcW}id*Vv~2HHv4I5p1<@HEk`VstuROtde8|KJQF{c3a>$qU73BVT^O zFUD6Bh_2y-q6%xrN}LK<2D+BkJs^X$1L|9md!^kkl=EoLb&^inaIc(ZSzkv^Qn$+d zY2H@8r)WLU^`s*dl#x7>-Mje;{pgBjETbOT$g^8FqpvhRK^5pOyYkg|7V(Na@p+3X z4q1+LZf>QW?vQ#ySqnkL?Bq*+UWYS)d=M#mG?7HmK~ z%{hM3mc6uHANIIPUFtt$CNF!0Co!M|{>XduOIvQNDkJX#jW!^Tj(m)Mg@r}f25)BQ zo+4xrMxWw?xx`5#gr8v|bOsXC5pp>4ZqoZQm<5CHBw;@OG-&6*?u3i+m4v(47nBl| z;&V{2Qe5XV7?x~VU9oLF(T&wow*_ZmctV(u_+Tj=C$?(kO+oK!F zMy)d?n~;xg;IR--BeHQ%o0reGcSnChU)51}3!%C$)gAq1GM3p)y^MeKXYLC7E^y_LR*edlel%gk54tW$;U|b{8 zDZMs`d}f?GTp`m32`~5Fdv_}DY2GivXxwsAXw3to*Pu?@N!<@9^q0Vgb`n0?scKJ- z?;_Zr{6=aheIw<#vrZU?2-o z{j35=FWp)p{ain;La3|JuTS3!;;9+wEa=wCu{_Jx3d+|jHZDe?-9DAi1Iiwri^L>M zZw}*~U^PG&L1$v>9S*AAB@e2vEn5zIeYw~;sHN^-X$neEjZ#5r&2?;GGU#~=ypH|7 z3MPZ~`ib60K#(TBju-)LoN8ZAyJJ3jmA2r}fDcWzsdIbq zr06PXik`kP^%FIuSEkM1)0d-1Rx9m+43qJ0D45y|?ENA)c28={>EaRd*u3bIM7`E(=Q>|mS^`oD~Hh59Kp z$OU6#Y~URVt65Fi4M~s7wMTzN1bq|o-O-<=x^o0vdq0H>gZ2se_z2|GQOM7Ey>JIr zsiUu9v=chj*W>m$-r=i?3>8S{$F~qIG2$O;UuNDA{ZukYpZAcpr*Kq^o6ITRA>Xnu zuL066@^ks_`d1~J+h|~Xy3jinoK(n#DNM4AvW$eyZCnJ$Lo+h^E-C%!arx{XIa(rz zg`{>q5{z$!M>79#E0IC{Ym(hJB|G)if94NMB4^uN(wM@8+QQv_MCk(cl@i`U?CLyB z|8e!vfXZQ*QQ2V{a2QUF!!+Qq<5Nrn4m%;mG~lokQ%nO6Tb^PX0DHSX{C2Qp2_jmE zPp@(oP|Dgnf1Y{$SN@=(LbsLF?m5x?dI`%$D)8A-7X?ng-kY< zVip90V*O)$j+{o~;;?)sBuCJZwtk5G#MKmb)!H-T*{LU$P@gUg>0rXqk*}q!zPOiq z)C1$rB26%L@f0|+tk2lKh$KC_FX&Ra+@pUYD>ZI@0{j4=g3GRjW6M|`i|#4DjO=t$ zP+e4kaSO^``Pa!{uI#}5P7Evc4x=BDxaIjBU<&9mR9L>O!(O3MCINRvW+p3KzK9z3X7(wZ!I#Md=tR;|r0B$guC zB9a|!rP|2ebsBK(rt(k;L=4XTy4=(vfane?6Vo-2LDGKPGnc*T+Og!gu^d*FoVhYnFY?tBe0Ec?UKlVha8p$iGtR=_G3pNitd-x@<#DJphZn_Q!E@k*ZE z#G^;V?HHV=q!N`-t;wPiq2*$UN~mgCRO^+OL?u+G2~`zIr6#02maz!I(WU!Jfww2o zyEN;W!>PE7O6hqKXIXObw*hcF#DW&$yLpgN-_0c=Z(W=2S7ZB#2?qD!ktr|1?&v#^ z4(=n_Ys>W1t|tGMok0eO#CY8U<;I}u5^fB%ai-O=l1zxyGR8vTM|=hea4$g5hH11` zcl~>WI}3HXFZ1E*h9jj3Wx=lgeGu)xF#|?eX0tgT|A0_w%S0&^Q;;y;Bp;ZQFy1UN zh4_aOen7&NK>8{BbH(^UzV&DddLLF7%a~nzGObR_?q5rBb$;GFcjEiyLIl?!qWc9~ zG520UCou)*{4skXe;1orn}^71h_>-+?Zt5;tDg99M0A++gaNglM6J9Ny(jl>>mnI8 zwCr8Z_graT6`pQ#gaar!3p~2XkqDUS#~QNJw#JF@TZjo+78g~=o=AhpSR}eGXH-8(csF?Ga`DHpW^;5BIX9`>`Dxwe z=zKDPRdJ(7;TvBf2GEW3Vl3P^535fotErM@5dAwo&K#YM&8=MtM_rMLFNK7HxN_w6 ztTd11`tulMR^vuaXFy{bznU+Z3jaau^v|T0w0wW#U0AJjj{7^RRR1|qqk?Kwgvk4K zo#!k+qVoqj)nVp<{snzGKYk2y{Kj_ialC%X_ZJ`>PYWV+z|G~xEi|jlR2KuUtCG&& zcJcm7diq>B;h6~IS`pmiaEyN`YzHtmShFVxvvqi%ME;5}>s`2>;m|)2TNHf=hBgq) z7h{IXFrvx(2R{tZEvrWPO1n&~Iaw~<0RuyZ?NECc)RZj=+tsSjeoVyu=v zh$N&t=z|!Z7<;4-V%=oO;Xw=sU%9;Wh9FsemyP=@mpbX6GrWg?pZe6ktc!Z{kcr=T zk1Xx|E9J#|r><->rFsfLbvhyIH5|aAUe0FF@gksOot*mBv0kl@_JzlCO~#zT<@5xCmI=Lqy1FblX8 zqgY|3*j*_VD;Ugjv0N&LD*XZJOxQj;j-1yDwC(g{p_EQmKzyR_K#+S3arU&VQQL3y`I7kw3w) zA0B6^y}A7F5|F>%`BOZ`7a1GNIn^9l@@{Ge;if(WssZX;b`*m6b~1?6rxaduS!($w z5Owqj|9~)9w@5IUTTmlX(EhJJLo^c+)F<-MxfQq3>vBiHe@Yi<88vleW8y}o8P2Mu z_<0x7lO@uQrFt7a=mm1di^;h3*z*2f7efK4l=RI~^dO}1Vv5hP zn@>|Vbn75_qMJs4kN};ysJb^;C)$r_6vw-^5Z0B_fLI!ag%=>!{e#+}4xbFJ;UL-t zdFrxqrBq*rbA)<^bll?z5-5??>7-VQ_yh$j9exhgda4TEMmRdTF zt@pmt-==ESJAVi7DCD|;>eOFQ`&|DwSd5$@lMamYzokyJy3oO};PH!4G~V4}r|AYk z{og@H1=T1ye%2phfBQRf1)qf!A^8tTWTfnmFzP{fXRa7M#dNmuo)%v8kD&gN@Dfy` zw2kFlZGV?vh<*k^{AUO-3QMIO^6_8fiB? z)!P$U?~Flcg$QL0>i;c7f@+kUFy`c4{5q(vcz5Y>+LEq#(-8(wrb6tq=$bcQ zvgjTC7fmN!SSL@pRLXC_Vu*f<@oks>Bik9SUCPc}R0=6A>x+dt9X*WO_$aw0^GkIw zDyg?wN|qWES#I<$ob7x?+4rOKi4VpZv*tWTZ4zxF(;WxbKaGpR3iD(?X4l`XdIx>Q zcr6u}qYG7YQ(xt9R;6XItp5U?kAB66oGUGR_mdB1xtwbtgr;jkC+0OISdI#+QBF;% zI)`@Ruc9H1_q8OTTO|nu)hL&MzSmD%w!^D)U?G_mW6XZo#d*5|-;?ukkuP^3OW7j- z5(sOLk_o4)>Q7SGI0Q?!9qxPcignue-|roDhZ{N$*E2iVQ5wlvls@m2T=W>N%3DOg zkm7d0GD2VVtoeKW_$cJ6wyt&+YMHzYBW-au#okBsI{GE)!a%pmKG{Pjh&ths&hCzc z%ne7iTVW7)5jQwj!TU&8t!+-V-Az(h^j1O^zTF~*jAV)g@dVPUA9zU8nQoJ6Ji;MU zI-mIGAA#?_V`cKAuaj6e*gCp9lX))j($;*qC1SUc>5h4+2G;&YiIO;7;tNV{*Uve4 zx?);Ly*;tx#fb4aRI^*G4{&Hr4~DtspeoYCIrqYUWVmp}&A1b97+pwoy0yklF4dcq)S?wxfA^(I@ zR`NNt^m(z8_p6S&b)8))k~MkEpBY>_dptt*K%cRA(#meK{xL4N||Euh;)VKam;8Suz7r8_itl6Mk*2RPQO6 z__d4Dpd69u&&hiVaQd;+v7CDTjZ<2>HKSfo@1ZqD1=T2dS%EaY^LOz+=kp#7d2=@U zW%6!aA5gom795vY4f+W0E+%WMV-vBidKKnuy}3Qq4^o-Jun-QONpS3#^Iu_Kp9(1r zh*tMEPHidVAEl7jBb`w}HA?qI8t-o*VsLgN^r)a3<;H3hmd!~gf&AFqr_w&~1%@+> zJEIX}YW)?8Al^`64sc^^!S&f`>I`-fo{+CEW#|dmm|BzTQzxGgCe79mM=3-b<#qVSQ*;uG1Psi(1 z^c#eg@N-l$lJT<0+i8?)?S=8YIHA|dt2mBVQN=@@+}cCCtj~D+Q+CH}Ev>;u%wRWY z5YN-K@1lg65fAW$ZNVPM+5u_210-F13}p;+&MtHQ`QI?iof#&#OW5d$k@(uh#O_Y; zx_X_R@_936?mQQky;4z)MdXo-dC`p*0&3MxhQu8StBp^d8Jk$V2qI>$N%<=CyVz;= zyaTmfM=aV^6Ny7NQKf&ri^8B zf0~IqkX`HVN-}jjqOB*j)VgbS$$7y!Z_2NgJZ@&&m)4PW<-@3@JNUJ|G;jf#C$Afy zzw5KImP%X6oB-J#3X5EVsWL( zCFs%Pky>;-JJBoNi**?a|K`rM*v=iiACYjW_ETv>t$-i>mEyVO!LJG5VU6n0EF{XE zaMSkJbqF*a#de3yRZ26W&Ic7Fq4kw_N*g$M##$e6LJ79iDgjHh_T9xU*?3-*p}80 zt1r?R>`}ppE~^My2x_m5Z=W$+&pgFDm@;uXM)%%#5gzB>&j*86d$RSO%iMZT#tV05 z;1I3XpD6AiUG@LJGXuSxr`(JFkFu2gYM4vTTy>0{eqx5&disgkC5v}jkc(6vVn+rq zAlTFC4~vPJ)5W_QpGWxqu*MxpT;@4GU53j%$;X*9gw5)Svnluma+nVzAq-m_@RB?4fsw4-ulH*vqN~Q<0q}eAh@sD?6P^ngmI*E}rGt_4c2YuB}Y>4nVeVL$>wT)KAHcd=QTVXMe4JZGTR%4*LIiPH<-mK5{PUlXHTr zs8RMgK`FubbApm>Ff56qJk9$*oD<{}pjN;Uvmy}8bw>yPTRg7M&}P|32e(V4>tGxm zTtgYtZ55{E$f9ERbMLh86(1!<;>f z<`7QC!tG9!;&Qt4q{)#v;g78r-|3x%j}V@%;{*do!YMfuF}n%Wj%9sg{4kqKMp!@J z&FW@^zZyiW)wq4?-uaXDJpwvoS&vie{$%~Xr1%2?Z0k=*&{?*Qf*N-%t4=UDPRtDx z5Y$=YXII3Pl4%!EyjXf_e&B4F&=ize`>l*V^jjJ4hUh1$L~b2lY}qF1kxgayp4C9V zFll$@RnKW(D(rZ--7R{vn9ZB@u4;H8{p3eug!3-WTq5Pfud6>psP7%;t^Z|uCNL-a zxpEc@dDBtcsh`kSSD}ML^Y!)kkFs)5(!;~j&$XQ9TQd-NygHm9$A-Rlliv}FB2p^Uh!$DQp8$X>K6FJ6fy54IS-n|mQEbk z{>(re-A4C*Y*#7CuplSEl> zk}yMDD)ZW^Fq_~!K#(}2UyO)(8PY;B5Zz+oRuF%3BEycE*_ zn7n(#Vv~2w|JF9*#lGoaOHf-dDP1-l+yORz#QFh~wd&%6J~6V0Dq>>;E;c%os{#^7 z+C?)#6P(j$<8h{Rf*&@9hr+JPx+K#{i8^7ss2-Lylc-f=QDU4p1t>`fiM-`*k(UTB z)B3U?nW&L=xV5BIyXbBraGDgq6*DAvUf@%+?Sl~u>bqjeBBs=Gbe36l@^1gFX>zyL zE8QxR95yC*pA_OqEZ~6TRf=~bEM815?%{2mWUt`&I+TptD{!R<;)5xMQ_9?stY*|+ zPO}>j;nWT-kL%R3kCEoz)VOp^I?vnbnsiL+*1gtu{J+WrQMAZI<@A3NO2EQXAEK!D z^AYdP$J{s9xOVmU%IO(H6>TYbxx(4lUA;4~Gspg2b&=$?Z7JVq9yt%H7EGDEoffc8 z-Kr(WWyb%Cm2qdgF5f;LzmS#_ySJS#p$>`cST;eIT>cM^PcKS`uCx%cX08S0IP-XQ zm4Z5ztM7Y96?CvOJ41|y7zCv=8?4|?Wl|zk(utippR|zf4cek7!!9yZ*-LSx-6`Dy zN>}Ew*%b27Yzuj4Hiq1d6f>ntnxfFs>1*7T)#|jV_mkKcJRlu=sGMtyK=!}!T4cPH zY_%m|D!FPHM`eX;nR`;U&gPBT$zJb!@Hk$1G<8=kSUB);Vgj{b%{Qi|P;;Fz&wMK@ zj~rA#Sa-`}t|MXMR3TOm!8!{z^QhLy@^+C&k3}Zrv$jVH+O#^eOO6PvNzb2*TeM8x z=_q^GeLb~z(>+EP?}YL7O6mP*I16*;u0Hm^Hri+HZgEtNLzAvkl^ z4)wihKe++rb`g?O&Ms~hspMzQD$XqJFJscw0gB*Nl{B5G*4Dsuy7MyBv(0~U1??%9 z=%|xBwWQj`^R!Z|($-Pn%CsnM6xftLid!s7kyPYG@Wd5iYT8rSST1mOMfwLJCDe;g zf|$cj;eiJuOt%ijn&lx>qwnob)pA;fUUxy@{{fR8m(%Pr)4EFt;rAvwd?5O=^CvZc z+`vXtOaoxO^H1=m=eRDFcCG)O?KbXA>c;b9KyeyBbS6~#T44gp>;R|Xa8`0Lfx5Lv z=R1JM_-a0y$bo{;`@{H}R`_6nuWLn?3UVtS1GkVB$25P8^Erg{fMw&*tpvJb)H-6n zMw<+D65qN;*phjzV+w1)lIsBK>bb)^yci|SYE_9FD`;)DJYU-^Pi;Y(HlaUQo~3lj z^9p&E@)zgmYYuZA9)A|7GubcxWV7r^+a0wheZ!hYpGnm)9wyQ>6bN?$`cCT;o6(O^X)u zDf=kSHptn8lc8o!rCi&)$2)seTBvrZgrse1&q_#IrM6F%khDkbFbRo-YxhVzX8_`hq#g7OZhWx>M@1)6n>4&m9%SD`1LTcO_(+Orz+p_Z9i0ZiD)VRSnfZ6A z{YS-uWZt5!mz$c}b$L&pcN;M$Nxw}dpRTJ`^_6hOJ{cJO2(9L|4SIXpH=ZWj&SK-Y zeARvYc094y1KJ>{FF|rKLlLXl5lt?ypz8b){ZFnIYgqysOpBA8a38B8+2%EgT&EK0(nCM>E_vHrz~ zUT987YWJiGTV5w2VM!7udY@$RQcCHDQ%dgql@6JYc}&CAccg13Hrve-<+;xZKF0G1 zr!JC7zXPUfOQdH!91b6L4Fjq-_I~7;u2%F8>i1~-OQR>tWBWG6I5Vuxn3j%~bk%&k zH@!4Ih?e6eiR5XiKE>0?B-ukK>Ha5Nl9IVIwqx63JuRhn$|MJAr2gnG)pB~O0eFC2 zl42SFlXFS^k^z78%eRUu9V{P*@Uh)v<9DQvh)wobKkfP|Jq9b5_wyqkm9O@Zi{*=Z z1V6^N_b&o(a}P57=<1c zRHLHB5OtLrjIB*x#|Rlx4MzplsA!4IH&|;&o~(%u>`03+3LYg-fzNpj-Y_IpOC5+M zO*rp~tWy#Gur%=C-f(eA+}UPcFdH{oiLG#?O)DYdKr!4}_O5LUPIUVdq9<>I+k3~_ za23pQI+E7b3ST4?x~7p%6d{IywwnUK-1HOEc>}9v(g($j)_%m9JhX=WcN8v&sBWMa z-thjzLEfzlNsKd`eUK)}>8+di1P4Kxu&Z52#?J1ww00G11#Ve>~ zO(;fjct`L)I!U6M_fnMYxw8qfp+$?9i58(s;i#Y*6(O7csVBFeUYf=kT(KUV z18*V2qS_92(8b=$NnDW&-dR}NmNtdcFBcVwkx%xwe)#77tz=$(x_3x2ukK|X`988) zv0c5cXesh>&3XyikPodFeV8h9NykbU6k=)?cGhtrahrQ>6$^iydxy{ zjWAw`k3S6zHf^1s+$N*E7~i+Q|C4h^Vzcz!b?k%wIkTr{nObb0KA)KK zIlT1_T#f4UE@-_6H+4V7Nl)Ufytl{YJ$e~qV`83>gj>fjS|G|ek-~X z9n1$#rtlxcIYd!DF83XXUz*8YCahbgK$4w-bJblAflQP7WplQ_bKppmA%pLh9+RPX z5SZraKT{xzl|3mHFxl_jr$1$yb#DLzGJd`+GyN7tV0^{G>o?1m*!86j#tL- zg^caLV337b+M8j3<71DLyL)}6bKvVDT(?>8I(-jU{GtRc<1MRR&9^JY!}_kbj1;AK zHEFsdvI}*G&;cMkQ48YCZ=LDdZn2c!jdM!@Npve8uR4!ZF{*t>)}x>P%yi5)sr z1|{k<{!*Ugt)Qm&_ZLi0dY2qm)<;j%5L$<7$N%X46^5oX3tT%hEvlM(ZO0vCj5^HQ z-09v$@x9q?3*O=UJnl2ziCjmdvnSJi?;Wn2=xT4yUEd+sok+j`V%oHmLrU8e2ImI;Ku$xm%0}lI2ifO=Mx1^W` z9QJv^DAQNdI1RYCTT@H}4*Ob)X#lLk-BF^or>X9*`ulYMQ=yUgxckKqM+m7E@2h)#pZkA|D1(vRJ<|KTrC=;(bYyxY@yF3mC784aR~ zNybLhWKDA}5AL{JF}jR!I?r=^%d51WtDE_hNyDp20*t+1?GHR4m@t&QoFfHs+NjXf8mCQBsEdu zo`cDJ`x#mgA0wIH-op1P;hU4p1=XkspP)ac%Z9vCi>Ja%ATfQDb>f8enXqnMRMs~? zLVbB{Ba6t=Xe3!eVdwqO&HhV+qQ-zRV6s-MN~gN2bCOy9A*Z=HNo9T@dxhR=(a&Kq zxmCT{gK2y1TCyHn{l1Y&Zd;%)5;5bhQ5D@gC^Y_oL=Mj3FI<$?u^`v7^X5?MD@@XJ zwr7w}ZY`b_=UCk!qE+b&^lb|Eg4KMz=`{bYRAsx@@Z4)MU8N}tcPEbmd)#99oY&m) zN`NZ4dno`N=gkSohfx!lhkT9#YCyu%#FP!&fA!B`-J|ufc|a`7c;t> zEE-pf6fUBe9lEVkXmE*40{9X7o30P*HWa@yps&k0TPbZ2QH~FYZ1lAPPD#s^1j%`s zB5|zweH%!IoduVe#e^UksPtA1!ejt9)r@PiSgO)4OZ<-39XD6voJp67!ZdN zhbjkhKpDb$SGntH{MPGfl5Q?n1IeAYa>`v4O#EBXu%R49(5qsb17=d7!;p2WHIy+T zr^heT;h|8~J4MFjz<_{{ESJ6LbuNBYUeV(nIMFB!E#c59mE44h?A1md;jR6FH-Sgy zE7QD}`94Q=fSC*LP`Qc3?_?y>A&j*}UisZ{Afm%1tNTnMZj^_DbhUoY$45`%us6Qh zqI>@Uqx$Jcd{j`45+f8w$B;sPsY2diKNFy5(FJgK5@=bNs@;?d!}VWAen+n>bQ?=m z_v$#KzqC^^MzV6=whS-YZjC&N=%dRi2Y?PCzsouhJ0_D&-Y(CjXJM<}U2v+YOtx#L zp0M_OTDFd+RJz+iok8s zPXtp_Ej`Vhbav|#Y&Ttjz*^H|U_6$axJTyBmgc=in)haUz^I@a6}>}R_E=E}UaW&J zhb5OEjCUwMFDE~1;tkLW6ycqa^aYJ?wXpk~useq|qk?Kw%plAwV#>)Kx=d2mu1F=} z3^+Vxy!b+L58_i4YD9+mmU|ILWM-30@>rv)a!{6ej-GvxZ++)2<6By^?~ulIn)naW z^?%C;!Z|$S{W!U!^9{mST9=tqB5*MCYd#xEB75`(Z zCZ*@qalI1bPm@f_gjipTF+KGUUk_Jr2aN7V;Gm6&g=d_{6;_LF8kB zqAwFO)~`Cmws_}hT|Q4ty3bn?ay_Gvy73LI5p{ZcQ|v(bN>e`B$O%#omV~LoT}6mEj>$4IVG4gm!yD?oC88R6R%G zbf{`t=j4_Fu36k|b`qBuXAzB)%vk+>Zx+nX*ZMz7rehu^ZpA79UW;{3q?EOn^;Ge{ z7%ABAp%c2p2YPJz8XBQf+lE75$^&T}VCU z&!5>Bf0)=Qi$!Dj)g-k=iBh~rpxWIqcYxc@ADe&7?9(%ey8kuF8?CkQ;Get)~%pW}U4^^5M(uf5MX=bO^1xuJ~tSh}^)Jgn(-)SaJTCQP6I z0@ZdSqx$Ij+$TqJ#Te-MiE7J91b31p?N?n=Xzf?qpP#63Z*i5_hUC%1wS%}o?m$uy zHwdP80B(QI0$Ecg^$pzBtov#f0{lOnv%ogqgJR42+jo=i`{Y~pA2a}a;=Z3^8gSVC zDW(C3{UF6O;IK_8rU8fjV~S}2Ox~OLFTO{$es?4D!Pz|e(aF@T{;xcXhpzR0ng?Eq zh@auRkE?U}N^NcjNurPBN%02g1>=8HLZ^J(0x8L&hd5nS)Oq2!O&&UV=-#Q;cfX46 zO`qw)_JpE1qv`_5ODmi~%cs9WSzktf?Cy>pa3wmFBDbbIaV(`Q^pG<7jmV`{M|+T?dUWE}{*@DODT zw~C^txBIA0iRZvyCo`*EDV*$0G6s6n z^_@~TzuC=_^cC(lyoT^a(6Fj42A5Us_^XU((La%5lAqUW#KWNeNe*W-z6bR|UM=b5 zrc%+uV*PgJkX`L_hNf!LYz>*jQi1CHhEzjaEP-Q`uJ-z+px9n28~!;;yO8z=PosT7um%_V+`UR=eypQ_4z-1qS#(x+oyUf)l+_)UC8 z7u}EB>AGUBv5ijRE|qF>aP1)P?7tuCkE^U_(ilT10*R#+xwF7sTD32b&fqAa+$NHX ze!*wziQG0`IMUsdPM+(lJ(os0)9l9CysQCvmuR_Hkr8Zvf?=L2LuS9;yU@W;hq~VZ zp1E^kZkoo#Z@vg7cQfn&r`oaOiEyv|^p{`@?qUYD)lP1`^LO^>>E0*Ezy5~46Fwn7 zfvW)YXPQPn#5>sXJb`ymKd-tofuCd`z;#~SYs2sz)Gwjm#-9Z%PAK?uy5d{VyHflW z02U7)riIdb2;0bq1%ba%Nq$0`l@r}dm)`;y{2c}FuHd0H81}F?8pVhpKUsniitdGt zx<8mAx!XVev0Yw`=XN3@2St5IC;PC`Y?PW?ORLP&i_FdwZronVBz+U*8khkw7n^og zxqhg~6+;O{XSs`ehW-%>XVfMpX3lvTEkD$t z`imjleR3ql<4^kZdHkP*+{DPvQM>1G{1WeYaB&IckD*e}OIN{|7Bz7>Q}+>`FGV`J z_!6uiUfVYjm@{`r5>;=b6)_4e2OcgK6*_@-?JU|`{<1El`~4-i-H&(M@2;LnjS=E!MRyM z7OulBlA$bZhq8$nZuu=Kw=87LkmV_NhNHTHwZ%5w7Agl>)xA;TIpO>%QY2?R^yEbF zrjrvPm*sMM_hs||StAPSTj=-}*_f zYD&^|>fH5{v?X_7(TLTB9g=YPUQC)!Q8`?%5mS?h@1+q{BE)*P9M_D?*ySOn2T9Jk z__xgC8D6Ho-Zml}{a_j)VG+3BOUk2L$=>}v67RP0#b-{p@nwDHbVOXci`H|OEA?%6 zUy%+ueP4Dzq>=ob=tEG~>!iDYajW@m?8raC8JXdGYiN4?)3m?^Ui!UbPTyJ5yN?jB zw*k4-bLf=(9NqJXr$g?nk_%^a{1_P)jf}Su(RdKfsk`_qfI9nk;i#Y*CBJpmkA2IF zg0gED%8=OHb7DR+o3hFsPcM;H?|70qpbQq5Ng3Lu3|y-5VKU?n?-BVL)`wvUt8oil zQ0>*PrK|<=3nV{w^TKE=kaPp5FQGS!@a59(@g(Zw$J62zy4XccDY;NbMe(Y3`~KfB+V~>ZZRch?KpHdceyKmnl_zC?EBwF=-vX-q{=$qTh(voXn(` z^KCpy5z@2Y!%{9q;@xnLI;#BGBX<>a;~7^se1X#F+Hvg&xa9Q`uNws{2kut#h)dDc zk?EJ=O7n}y=O=IZu*r|-t9z>jJ?6k2d2Yzk-~DkD3r&1)n%J#CBrIP_(Zu!AH-Q|7 z^tA{WPDdn-pHQ1r!+%Hh_4>I{K{YCR9||ot%ALj7bxtB1=b7Yc>~y-bwC?8Q_&Hc` zUmokQ5^xnfKTVXZl8D@U=jXhgXhXbrfZ^F->J5JNmJ@ZdHvB>GbPL?)vTd zeL4CAwHuW-iC&P9$UFKs35lGe7bPT8L;$q!V-gaZr2*;=_KOtL0GRCU9>Vw4V{QH~ z(D3sWf&RdSI*+XI*6yS=pZ^+BwkcvWqiUrF0G$c=Rw2 zB`-(+NfG&Z5oajbz>!yYq`RD3p$l|2&ljX)xvWLjNP8`5xresEK-^KtjWigp^Lpwb zaZMLNlXoxkbWy#6vu$M|##w$A6`G#NbVH-kl|f<*7 zA4m4R^z<%@pR{kAysxHzwTGzU=#yktYu}B~uO&1np{)|ST0;LSp_e7}l!VUdBlNU{ zu9MI+5_(lazmd?yenQU@adQL*;NQk*uUT<_B-KON|xNNS3qW7|R zFWYeZ)|W0PVQGdkC0Tm432^tK07-Gig^*nd`{Q zZ~Os><9?TlpPTv#d42!w3ld*V)}Y;F`+0=XawY&B@;7o5J5KWjWBT@Cc*~zV!>9HXzzCaJHVQ6PrVjk^;) zx~BI$j8if}Gx$S=75t-4Blr?$P9M@puk;&co%!Qc(*N%JB`D4Shh4EaL`H zu-AL2O0obbYdkcf&#dVSn{hJv+oSdjyho)$wT<5b_9dyiy5{u`G=o0L@@6=)4($nK z9vY5(M1rq#lNNfz+EW;Pi;qNpZSvd4-u3X11%)tKLhK3YZ_JxS=cNqt#}&xMPw=>7 zF8a2lm-R<%mbX#TvVK9@P9EpZqBa+1r|(SCe@D_YhP)v4ka0x=BnJDp6w?40cX-c3 z`>G3Rbx-DvVPE6B^2y`AezbtNT+B*|0L|y`x#w@@b=@{EN6Y(li7-u|1j$1_W>%^E z_Hs!4{q6bRza4+p>?HwxT=UPzn-tiWi5#{vjJO6*gg3-7TirN_7Amkv_AJ7gomh+| z0*Flz`QIRI$i0spj=B@)C_2!w(7r9d)tE3+Skg(e+?b}zk0o(qWPlabK3U7j+BWXHJ%c!*&&V zyA!(TgwKzaf$Oi~upg?z+FBq&myo$>+Ua@V?Ylo0A@e8o^`m}@5-oLdnj&?P`{>K< zxkXHsy!PDk>hejsfr|(u8|-nLo=0TaS5GZ+5{HK1DDy}8>fy@cA5wyj!jwhQk@CEQ zxEFY*-r&tWWbO$UWF}N6`q6IWx|~i@@14Q+T15&hNDSiMFtq(uhB;2TyN1b97Y+c*a+5;QD7+}IxnDdmvZ;ANANbj zzuKhFMQ?*(>wLhuEO`*-)e+s}W_I$B6biwy=p7cSCo-W|q3rcrwII3pVadqZkihFG zd-LZY8o87A6R^9LJO)-1Ut~YI*!9F$3Mg2d+QkLCZF?Pt#T97azgqgq<9zlRRg3?Y%DgIRPp!@~9;ZCH<>5 zlFHg$tS9mj-)~>Ozed?}VQ_Gca2x+aiQ|7C8XgTGBn>Rmfa=iNB`0PVj}prSoBKOzkBtSN7z=J~20Ny2xEK+17OPXg zP@G!*LUG!(FBD~JU+-cZ;H+Bh8EAE+kM<^uGPk@+(~hiXv?#X^%a;y~^B!g|ELnwq zJ(wVa$T>($Mz#1~B*1C(DGR>L&*pqa#}sVhY)VMU-wr#u+hX{|)TV2II?H{xTloIE z$gn$l3^r;rY7_TJNAGs_YG{BefEIE<p|{RyNj${o>Ir% z&^j>Qg&=2dC3WqdaRl{WP!2l1M3v$>eAXXNYFdncNr>N5iJ#DKS(iydAVG^Yp;M6>a6*N>*To^}@+kdL#=0eN{idWXYg(>e5SJzxmM<-z z0=`x~LEl4B;W)X;kv3?qWlGskf#;p@Y8=vO?}CU$DSl0e*-{u`L;KTTd#tl+1$X(K>;2(<7rc`N=hB{O z@A4y2sFt0)M5}lM2wO&1Q5Dl-x;7}v?>uXc_LN4ACkC9yb0+zwu|DtS5s4JwK%H5XP+hg~_6+qZxx!ohr$1?a1%w zm7{G?$G_(zbYoilJmFxWm>;f55V7oC@R{CAPIcl-j6kZX*q=@F^g&(r97RZgh7_8413$jR?b$laIa`iw+}uOQp%hJ&az zrn&wBTI)u6D0D_U!J#Prj_fq-D2mNmPbkOdlm0%&w_W$PzX#uBelt_K&)hkBi6kSR zmVN@0S;B;sq)=A#!jZp-;3 z#Qt!(y;z7@XW}P*`2k-Ss2@&Ol~LB|Tv~RSJBMkf^1Ha$EWw9wsKVLQI=Pk~rUt!=^ z8Tdm6{x<_(up~R*l?Hx|fj?s4Z3kuZ-^ajL8@Q~>Wy$k8gMO=l&sdtx=OhEa&cL@C z_>8w2{0;m%1K(=kGY&TR8~AkwE=x06^4x0BPdp@>o^#WgeCh^%lYu{O;H5*e`7bo^ zH3t4g1Ao-Ob8pMecOL_nm9s2)t}^IvG4K}*eBNQ%{9^;Z!@&P+;QJh&&Hp?DzsJB| zGjNV@Wyw=Ht^dF{3!!3FU!t%seyCLN2YvkHt-Am zZ2Bh*`YG?ore9{@8w~s*18+YzoBt96A2INI4E$vSU+~WCeAgKGEe5{Z>})>I8T3<* z%ceirz}Fi1O$Pp$fwv!@o$o#dzS6+2GVlis{AB~5b3%4`kWLff#*&(_#60o1D8ATv*fwipl^R)HvORneu;rUY~WohviTov;8z&<&kcOaDcSr_ zFz{;({7D0s!=PF6lvAczxSWX1!e^YCjh|!S4;c8g_h<82W8j+%e9FpfKC2DRIy0M({C0PiJmmyV7Jh=k=PCpLxq+8Ikj?*S1Lw85%J;H5R$^oJVwMFzgv zz}wEs=6|4p%Ws%x$#cY@zsta1H1IibHvf|h{3-)~#K7Cn&gOrBfv+|2+YJ0U1D|nD zcE0kP_gV5h&Y-{3z{|C4KHS)z$^Q`p@2_XmUt-|f4E*48v-#X$;B9NO=}$KB`waZ2 z#cV#kAI!#=8~D`*{+NMR&dcV%%)l=<@J9^1<3ri}4>RzK4g3KE&z+ym|4u&}Ut-YL z4g4kpf84-J7i8zV(7@Li_!kZQQ3KBnXXm?*fv+;~YYbd|YB{UFKWxyKKAcT|sDWQ( z;F}G+Z6urjfd)Qe;P)E%s|LPbBRk)<27af3KUv7;^O8Zo^M%>;#~Jv=27Zr$zhK}q zMzixh+Q8Qt_?-s+oPk%@W#@aCfy=4hEd3ld=i@JKrS+ zzSh8RF>tvVDoa1N8uV`%`0R_b`7AZ?l?HyXf!|`_4;%Q420roQ+2ved;Kv&HS_7Ax zv$EuQwLyQcfj?#7xl6M7?`q&j7&yN!m|4y%4g78cf6~C;Fz}r}k)7|M1}?AvWy$kQ zgZ_R4uUwkVXO)59Yv2<G|bDM#8U6)P2 z#=svj@OfXz=EEg9nfmjhfgkn7Z2H>_yz5KZ^m`ikasyv);I|q0;|AV#eRg^J4g5F* zztF&MG4P)o_}>hC*NxfbTxQ^_4g3-Vzs0~GGVtdOymUi$Id?Pg!wh`2fnQ?aw;1?C z2L8N(*W0qoU%D|HUt-|r8~7at{=9+Dz9~E3Z>vw=Tp;1j== z&3~zZpJ(8=8u(KNUb!th-$M<2*uZZ$@MjHNZpq1#XZ7pZ_yPkz!oXJ=_=N_3je*~3 z;Ex#iHUn?FJ-c4L2ELDhA8p{b?Uc=bwLyQ2fj@8HyM4ppZ{W8W`11z7+Z_gf1HZ+< zpEvN`zG?6`@coSO{T74%83UhoXLi2J4Ez!U-(=vg8u)^5W#_xvz^^y(Ck(uNS2q8{ z3|#K1&g$>-^DtTXI)l$m2L6zNzi8m)Z)fMbkAW{Y@DT&Q-oPI)@aGM@>+bAwE--NU zje;zB9%s;x82C*F{)mCUY~VA#lU<&J4SbD(UuEF;8TfMs-gQrQdG;}Id9gi9o+laf z*BJO?244PdHvgjx{1O9yz`)z?&E|iAfuC>ScNzFg20rh5+4;&3b7#qOwLyP_fp0bN z>V4V#ml^n_2L6D7zhU5m-_OpsZs4~W_%;KdeSbE8E>Oyp=erI1YYhBx1E2DPZ2rd@ z_!S2JQv>hXl+FJL1HZ(;A2jgxf6V5;)WGGJaH zN3!{R(ZF;6oK3&Vz}I$U(?4v`@A}hh`t=6>lz}hYlFjF81OKyuAN^=HpIZ&Q@Uv|C zQw{un1MmHLHlJQYp640#PZ;>Zf63;5oq@k$;42=>=Cj$r=lmj@{$c}v-oTG|Je$vL z20rnZ+4RpD_VXNr{s9A@_Cz-SH3q)Pz^D8wo6l+kzt6zSPiFI3Y2f!5c-OD9`K&N- z`H|qP@#rpt{xt(%vNfB}#RmR}fmi-Do6m9s|Du6EXW;!$W%EDB!0$HjzZtmvTymB? zmp+}1uQ%|`2A+E+o6k}MUvJ=>4LtW7gTH~VH}K5{o_p5dZ{W}Gm`%Ulpuf++Uor3n zzs=@_Z3h0D zfgkq!Z2ngp_;Uun_w(6&)*1Lu4SdEQviZDV$n#8tev^S$U&!Xa#=til__Tk^<`WzE z0|wsvVm6<14E#X@pYiY6eCh^%lhNNFGU%uLF`NGi1Ha9{Up4T9|0A3K6$ZZ5z;}Bo zo6lMU-(=ujf6C^w+`#2W>$AqAn+*Dw4gBCgXY;wrz@Ibleg2Zo=OP1t%)sZooXuyg zfj?m2Q~sLGXQhEJ*dd$#E`$Dg1E29qHlJk%evyISW8g0u`0W48&i6P2zr?`rGw_!T ze9o)c`7Sr`bq0Qifj?v5<^RggcZq?YW8l{t_@f5i_F8tn3k-aPfnQEoBvS;ezAdXGVs?7d>=nM--~8v%Q-gaZ!z#^4SYr}JKqxw z{0akq*udNK+58VQ@beA)P6L0@z;|uW&Ucw1&lLv!bq4-~flmpHd<}eqfj?y6?S*Xq zOALI(!0$2emkoSDF+1OFfHeBy*`KF1pP)dv2g zf%i_#=6|Yz-)i80Ht>a$viYw!@P`e&QqJb{ZUeu;z+W)%y(ee$A2#sUy0ZKGW`lm` zN;d!X2L7ypFYV6ebAy5BrexEfYT%mjJ7@E`+`wNj@I!aW=5wo|pEnrv+YG#aPB#D8!0$BhR}6gN+-&|A z8u(@dFYKDl=LiG8+`xZs;Hx{c^|Kmf=eyRxe{SFlcFX2-xq-iE;79aj^SQ;q3;o&j zD-C?JfzRGOo6m&?e!n5ltp@$FdD;9QFz~@WvgvO#@Y#E2(_dxarTN+PBL@DGfv;SU z&F2XNKWeXR`eTj$zR94Uy>~XBbq4;7fiHPWHlOPayltOs`V|Jg$-rmto6TpPfj?v5 zOZLm=^Q`7?pxerjPQTobE~?0psx9S z-#pJ$oqF%N=bU@)zEt&w-;I}ltOdWz!f2kGJ4=Tkx+fc<(OpdXKT-w_5N|E%=7J#>>|h{5p$2ueb28 zwBWO5$Lqbqf`4woXYLj+=OPRKo(120_joyHTkzK`_{MYM<(y){zp?c1&sq3~>=7@2 zrUgIVg5PGrKeFKC_l(!Op9R0bf<&!!5^^Tvn}KKHIIsyzn=xa+=9Pu!N*qO1@U_Kv)~t5@TV(TM@XIk(REcjg({BsMw!7=fA zzq0squ7&>^3;u=$U+2g1diS^Bms;@U7QExwc=>x+@Cz*XvlhI#Fkb#F3*K)TFP&}S zf7XI`^vCPn--2Ib!9TFz8y^=h|5yutj|ERvWF*Sn<U+>g-y$4$GB^LZO%ljSAS@=5_#p^xPg5PMtzqH`feiAQ#kp+L!f|pN= zm$RP*zs7=pWWgt&9xp$Yj&DDYxA5O)!9TL#@-7W{GxzTAS3{6)O{*%tg93;wtT&s-cY|B9CQ^0}3TKd|7JTkt0>_%{~3 z`ox51@6D;_R7JRt{ z&s`d?x7UInYQfL9;P+eb+`94Q^J5Et?`83Jo?yWrw%~=!uh@$&bu;1^i%XDxW~+Iac1Ecn?L{BaAO`*pnhZ7ldA3;uuw zPh1x-e=7@qwgrE|g0J(Nc=_`z_-z)v<@$Iz+gb4QE%+-Ie7zguJveDbaF^7}3LJr+Fi`*=AsE%?Wl_VWx2|HBsCzb#(x zRu=pu3x2N!UunTNzdc@WzXiX;f`4hjH@zcXeqg~DTjrZLS@>VD;4OE?>s`-+&$8gh zTJXy)_`??bV+%g=u6R3VSn#z!@E&$i$XTkyo6}mxXO;!O z-_m}bZ{dH=f{%PWUhiHOe2E2r#e$D{B3}N17W^s;{+0z_@5y-ihgf^&V)!mss%UEO_Cmc=_8|@KY@KJr?{c3%>DRNSg; z&Zpz;*~fxkY{6f!;N@rH<CGs1>WMMgtI>G_c%@=(LT#nk0C(yI!<@iJ8N!vdIn($uXM#o$7z|B z*h!R>RhNGJSXROd?x`mZ^Fy%5^rm_au2?-sJ^1KbtsMcTb(_@bRT4? zBM~!rh0(preyHvGbZld~?_%Ykw zGfSFt>6oVs!TCM-eCIx2ta$(vA?noKNc&cxqP*D1IU^ zwIiqsKlrH*L0K5FxD8p^nkl(UGi4F9G^0vj2@b$zp|87fTXqh36v{y3(2yJOS7DEF;z$zTZ&$L(Jy!fUmp!&^C>$U90RhLPLvlzcS|u-$do## zEGcC4BI4OnG1qeij4xyhIVpR^ygC%t7hAnTzH`4b8L7~!c0f|qaKzAWf*RY0q&efj z(ZGsrg*KB78`ykN98wqpZxn`<3wd0d5>ZI&m5^JSnx-jf?YJmXg><1!SD{)ZC?Ywg3O0Y~t0P7XY%v4vkAja4}R?2a1S8-As%c_GsL|tnt01wv-fZI0; z3<>pIR4Jq`GY6FmQC$i3EJB91xOyL82_u8zT?8e-fhs6LI*Er~+6^GZs0J6NoS)Dh z!&$q1H=)r;WV=j3;YsVJ49=%ZCgRd`Ik7a|F==JV92wI^L%!8GbDNUxu8bIM(vBfi30 zm+eg+v^oOSQR%QwGB37urC_TUV1hBW%c+If^L4DxryBJl?JRd~WWseDBr;Fea{Iqw zhn2*U(0Pg@%32IHNcZhz73#)tD*KdO?B=xJf>0MAkim9G5EJA(Xl}`-& z2U$El1?3O$45VSgtILN7Q!kwi5ZRM`DP#@M8hU6Xd{<8bHMWl>cN_v=rq*;Jt=VTYyDFidW3|mDbD9EwE)POUqWw7P6&6W@#a-djN(a(6DUV;&?q2;1|*hQ1D&# z0Yv(G0=Py@AA^8evdI&lAkcV%s06h_^>?9l`H#WOim!f;j4S2~`9f~G0;Wzzk_tIJ z3=Yh&N9tB{;ShBcsa+6OZ0(^BGGSER4#r}uUue~6BfN64Z8=i!6o>d3v_ljw9QD8= z0+;8h=wS3kp5*6CrLIC-mowuaqzEDCcN~`05AaV_(ch&YN-d6aEB^NaJq;C*VG)k< z;jD@0t$|Yx@pHkk5gw?Ke;t!roMDE9_lCpv3HmzlyT0( z^FzqA^5TnuWWAEB_JJexy|@G=h`5rt7nDKf)IXWYEFOMRy~o@sExrW7!3c!r(@BM9 zw>zB-(9n4b9cFhrq2}{uvfgxRY7`A?g^+YF%}bktCWtSHZ;-B*HR!V~oek$ZWCuOg z+5c2ScT7q|y!1iUEv$U``6tnexpehJaBiC)n2kb|3Cy{|R@Y%gvNo7a|ro(bSX z)!Bki=*QI%6k}M3HYp#Tln*Z|AFR_((J8Ix47(f-6MBj>Y$<5h@ts}v zG-ahYDd#K%=VP#~(#_P+<=M(E5t(y6GX_~3T)rWhzX=)tCG_qvqld-5)B`0>nGJ|n z>5ro4()_0 zzi)tb=y4|hij!Q2W1uJC`_2`?`Z7Xts~2M^#VM4W4b0}q;d2>_f zp^eZ%W8vbYlrsZ*&|Vc13vNM-tk;W-chp#jFlsz8O^*Z6%j%)VOAvK1(&iNgGsn+5xrAG7CmQB7GXjd z4E>+s6T)CaBTNXvR0pMxsudU{Yb%EWe;Q?K&Bf?TT{L?^H(0LZgI5kVsM8~YjqCKt zV6!?sD(I=x9l^Ldy+$ylP6yp6$^%9x(-pN9&peW=^R2;)3%+#PWG3 z&ti9?tB-^VQ*d1bl)4fIGD!dr^3QXy3gbM_e0ijvFMnyymqe5$57)C~Q2!s5ffpbH zWdOrTS)IF)xss?}$6L4_XLS9Yx*n?QdZ1aC8|iv*u&&6?7a8}@^|<%N#=U=V+^C<5 z?0m`C`MuE2JB*#Tu4X43Tu^_j>v|?u*Rz9lMRvZ-xG&V>J{KGJ`N45lweuBY=d94q z`;49UtY+t{)U~Xx>xo!hPky_u*Nm<`LtPIWU4N4Qw7;a?N-*l+S-c&Kiruc}L|Ow7q8DQLfxlr$$quGCOX5n#6$vr#Yr6wS{&z1=t`(<(L``wxjF9ow4vy^JUxx{ z8HS!gdaPhsKvaU%F2E9fIWSbVL)kSO4{oAzEDQ1( z$ipyKjqFi1?9E`}06&kTPdQc+0`Y;3k1!zwGj#`HDhd{(Hbx6I7Vp1Vmmb|m;s)JtaDp} zMaqK^Py-CBF9s6=*x)q%v5_W{dW+^X5vhF-0-`iMNt3puNkk<`RU2s{f0K2t5&_#Z z30Ot}-=zSeHC|NhThp`^SR_qC!2UyAbs?nG9nXgVBW)H;gk9Mj#)n`ls54=gHHYz` zDQszT7$2grXl(QW>Z+*$W{A!eWOcmBJJt&`Gr`2DbNwZrSvn2J(Pl37p~SxgoU6wt zYLB1e-k0-{#Ag}o6V(GoH378ekKz{0L`u~Oz_VU$+a|9+#lY{;>qI3;od~v^U)!#U z?XP6pmTW{NNSy>WZeDcf)3xt6@jgx7?Z``1g7o+C*qRwW72`f9jVT)*i%AeWfJm3H z4!k1?wizt9EQgp|IRBD#v7|^VQ(Alwst4+Wl${-#R6a|UJ3+(%(HbvYW@ln|Y-0Nx z*=CWAs067~O=5RyVtbBkJCluQjTg0uiN#7g6FaMk_j&T}LSCX0q<6(*Q(|Z1Q4+fw z2ubYjB#>C9^$SOWX+Bn>S9V4N)2bJ!5&b_T|drgC0~4J7UX2LtwX_3?=5>JxBPk5I@V z*aVWbtO@Wj|6F~tu$)R3W?P?hG1@#MgN(a_As{3FalDDea_2M2qXN=wW*b|9N1YA3 zvR>aLI9S53M9u7iwszJTzAZbvFGO2O`BGNBJ(Iu7Io z6XfhT6RJLeTeD4pV{^UOWO7y%Yo}u{*!N#V?koleA==o=jYxe-9V~S3S`K!}41Fu` z5h48m=#e4)H0V(w{SN4kkX{LTjgTJlVat(ghV%^3YlSqIIy*Dr!#o6ZX2XXnA9CTt z1$@Yd57+UbHHt$4ZQ;Y+3>gwWJj92g;lpcuD1;B0kMJ-|9##|w=z!1<{xFw&@Y&u3 zyqVsLvmd%OgMSW$*jJ3f|9be}5dWM-ZH@n3@Q?D~9EktJ@gLwH10@Ijk%Jb^xe)(X z;lB&>-wRMsc)uvTFA49LhWE?E`xW8+Ds$(tp{%~9`Qh4-2e5UCKVs*}cYxyTXuH_`s@A9~0aT>%W8(TzweqFnwPz7i5=P>4Y~;#rr8L zbMI)dx|Qz1U=E=AS$&3psiLP|g#v8V{t!}1Q|Z>=8z6Zc%=jAjbgR@zE~94j;o!l7 zLz6Dnh}v#VjourG6Z{B#14JcAodbK+&u|ITk#sJBntwAL4FhYtH%WPuQsz+#Q3+D# zLO^X!6VqE{I)Y3@B}if5-Cf(GiRo=JX)+O&Aay?N`8h7xL~jX}c#?QCeLA=995$S$ zjwc-$FMoslek7#A7tVH^IWM8fAmyJ!w~A;(?Kgs;OwKQeLEJEUKs8Rj)%Yjt|U- z=F-%)D6Ix0=X<(}yPp@~t(mk&Uz5!YDikOzwSAhB@II5UfJq=KLF!^A;g==}p`R1Z z0MziW$>iF;O@cn4pra{>s067ckWt&OiRnW!9YZFf5~NVzx@(vu*GC;6k?F@|A}T@Z zQkrxbO=7-xOe%1k>HLR24QCF)v51j%E5}9-)e8aSdaehbulqsgg8>Y*TGN%2$TE!m z+Hrtce|j-gqIeQ8pp^@!@wzHaEVxEN6zXcQ zimn0ba`hla~C*Q8n@5iEO51H1o50SZnH)3B(K29owP ztfbzb7`?i|Drq=H)asuw4IElDl|H?QVfA(;A?KFR{T4>m*XC(iv^LM!h8|Jjz{Joa zD;!W5dVqtlP#)G64&W2lf`mZ6M(bxS(1Cf%Y2iHO&E|Q^5s7dK^kW#8(7e`{Krw&k zbGY0|QWi92V?UZROq6xEf&Hl)HXIn)AOvgxhDm_2O9-*cbEf0&!mjpga60V5N!RLI zu*+4PS(q+XNUyIIoc;>$3|6(%;8rZ|FJ()ju87)`$QG{SvDd-arckOK4z*Z-0FydH zqLt^=eV=l^2fifu+|pDkaE1M>xl_mC!Hm;yNAYiLub#O-`mAt&EjR!*e4$ZL-b2Q{ z(yrZH`w?~4=HVKx(YTm+1ZP7308t6ji}46koUKC}kAygd;S>9u4xBx{4q++SwhV=% z=`;dPe7ug@r_V<8;BvHo7+d@lv^hmR_-q~AfR=If2B?q>&Owx7D)!zX{TGcvLhJXQ9ow%6GP6IP4G33~T+{to8l8N{?IaG7oYH zy|Z0k1m|G{vcg-;diD!EsjFFaE(R6(lWQ6%t4;=T6RH(vhoTZ#e(>}L@O93QAeBdW z-d}Jl@&ePy4#v~Jgfvs((iz3$r^e&yE`I^09xc+B4+SS)QINSeQ;k zWh{6sq0K$*;PLe(pz~f@GSk#|(-BYj=W=0KfKHZnaPXEf=LXpnhx>aOXWRx3`H=Uf z4aqU@)irPdChs%@4$~=Ej7Y}E31?lvdHVRQ(8uqC}SEa4wDi+8j)YG)EKkTiGpNT2SiIUtSoW@A&H4?{)js$s_jHIKd> z3*MpBw@%~Or|bY?eTT0GHr!4??On*dMwvX-+~*&zDFB z-;ILf&O;ZRwKj0PeNo0n)sCkNq>p+5a%5W*-n#vGX;s@#g`ni*6 zL9%)>Z>j1jxUo#-WKFpUJKBJYZ220+{0$~AhY1L?rL<7%kP`EKQxQ`XK(`o2Z&0Lx&ci3 zMqr`1gxUsN$zHClrjz=0Jk%C9$vBuYeoGldB}m;^m!W0NpuC`jwn{6Yv$>EX9lA$XHD2CnUk%Yi&B(Js9V94_tVk=wz`>04xa*43I+Uby_e~&0KQIhwa zOGO{sioU!Ok?%PIi#=$$qqAb09Td%bJcrI)3u##GboBN0IwBtJ6Qw3pJ`1J0%UneW(~dDaixNv`kghNzvQ|28B5GZGFk%_|+{ z>6?-+>4#D6B#~1%gi3_K?ckqY1mG%0O_cE_6)F^K71z_{9<`UA%r{0;j|`UEyAqYj ziD>mBo0Ruw!ku;wrrB>uJD_I5ba}+*7c(QY{?zh{sS{ZntUIv;>W%l?e2Wh=?!&HC5`%P)@ z0n6BU`}$fC26u`3(ACPLTgta3Tm2N~V@)<^oq6CLM>%S1sOn`m>WXWm*s1+d-%K4g zC>{c=<65DM-_h}ubZ>P=nrWq+)zkq7^Wa<%*a$bxT4 z@t_jc)M%AZ3r*6)^^#JTuXcrmu=Zma7=f@UN(o{S`YM`|6%U8*tCiXVO=aa z7eVRzupy!U2!~?lY9Vg9p2Go$TQlj#rhIEgvMYSiV-oF6r_^4ss+dd;lMirYabsOF zL%FtOa416cJrF9dr)Mf3BXdImT(3#$KOrovbcNio?*)54lZ62IRtVW zgG!A5+8hkqoC!=zqXfw|Qr2`>jctLP7w%v-HkD|L-!~3dGsd~IKE~nK7o;nTzPunu zIeghdXifHFWz5NpK&q?gX?w@u$#9!@Yg48#ETl5hsY(~vioHOjITQRHF`{;Hw}iSE zAigS*Fm_@B$3U(d(b1UNVw|x$3%W}7<0evYA4G6>n)Sl;Q5}Xx-p!8e2wvXz2At|p*A!;w}(6luZd@zn<&Y65Ps``-`MbM1b$3+0?vH0%hN!{ zwTEvA)!z$hYje_fp?l)H@+|lpGmaU<$V5bPEnBL5gQDxH8~LFKxHKEw4_{)G;j`l6 z?_bjt7J0M!LR|6*eF1Lb0MpL;`XT@^-Ac5+-tCOu&iKQ{8I4W$G4$GQeF@PypxG!> z!6aQQLpVb(gDx@Hv;!-=Kf}P_0p|GcFhoZk$;lXp#+tbv3fo*b1}lMw^OjQcc*7V+ z<1S+y%*oEe;7r!lg+N?jXGfS22K#A*31P5vB1{NiT=)Av?wi2Ji3R9G{JIyk!I9~LIJgt-a*5QJl}G&vhMpX}koX%>4lb8$6hKA4|ZddRKf^xN=!G-x!t2CeKj5&#w^b-Y4 z86E|LnGE7|)$5!Zgk$T3!xf*&&^fZc|JeF@w69XGjI7fmD(lqg0jh!!<4f*jJ5CQ| zpiVvpP1uR8K8`!yU3-GJWc5kjqB&jK*%kb~;18CO+tp9umMiCbc;d*k@cFbX0xf6r zfsxirMu-jEns{V(73QCy(7klHMB@|TTiAfuOY?&Q3+DFK^Whd z>{eBTMswZ^omuY~KNr!LVB6A}B0fYjSC@7QrTv4_h}L+~uj4WJCocHHSkK1HA78(L zM|>UY0kq!wA~Z?rsbS%M7v?>0A`m=GnP1)`Cto0Q^;>w{=+BO*mtBBTH zTAj6F5dU0#F;}M~Rr)}z{DcSl97w zBzjWSU@4!mH}O8~<=zByv1{!ij{1NA!uni&KX?;z-GlYRYzBur-(mijvz5tcIV$Vn>%*Sr@^-Yw)CLy6a|5%q@)6?Z&@!1eCFT48ftw@# zx6CEKZP3lRD0PL6WIH}rhgQm*zcwqzp1J`YbbWr#G4|B|-}+XY)})L&CK00w(Wp13 z`%~(AsKtIXnq>jN!8D(qln z;^@sJ3i?q`<`39O;(F)5#LbE7$GH3MS#ztOkO)4-wfY$@E_;xPun9c~hWj0CMCwbQ zpMxRz0+*~e8XS)P67;xw$MTrpPZ9dxynKZV%M;V~57*34w?bmVX?JZ&Qw{n#ObsqY z_6!h}Aa%E?3b5am_d2J;{@`l}W$*G|+|j#y!&^TeR+aDdGt8j-)U{%Xc?IBLB`%R| ze>0tJxy!9x(qz&FG|5T8B%%_e(5-hTbPGU4<~(8s+Bp28rGUHvv9wgxU_7EvG)xcB`WT%JynoDNbxk5@Sgi5O|VnT74H zQpSAbaVZuyGv)<)kRyCt

?)OXD;)-b_O(O#DTNKgz`CL63TTzY*Ubkr^SQ2As?u z2~p;Fb}>y+=_XNEK~#qk#V-NYMP(YIG7V9gk(rU9)XXSIHQz5RW+^o*Qp0y_QBG<} zH=G`1kZ>#*r{IKfT_8Lm(AKb-?8>_Gj^m}s85wU1ffyKXUKU|O0Au;z4tM^ATV|{7 zOgzd0G6(swaJ3fYgL)7~z!$-r7y)05ThvZvo4j241N(Ry=?)`1yPLQx=gX+@ydPfVxjdjt6fLOps6v4RXj zbXKxPY@F(LwvN;}0On1tZQUS`hbpBR-}BtTRZuY?iP_pUyX|Ouv+4%GFz}FQ!cqMd zV+E}gaIGXAeN#U{lm`r{PzgCt;hguotb_ez3Z9kJB=i%wdrAFQ-mojVjQ5m&hBw^? zk<#vWOq%GdL>=A=@hinIu$Y8VX*P@P&mmyR4Qr`vM-X*6tA+xR*<3zR*%4@39&`b4 z!cD82VZ9#42VKOq)mW;3h^W|&*b_!#=8rieb|XxMTnrguh31LU^iq5YV;JUDU;#M- zBv(g3nhNd9M**?f&<^Rvbf$ovhxA!<^>92w@)6*juw*%VhhfPTr7@V-n)5B@TY@Je z*u=~jKj{+1WP2`Tgsdv_DQ65zd|7WP@Uc7^mM`Noc~8N0AV)OFkszNmsIjrt*w|_` zwPgxzr9xY1BX}PNue_tpZ+MP&Fhnd4%@l^}5l~(jqDSJ!_N0)Bl9g4Xfc5RoQ?!z@ z?1~bG1X0SEEtTKHT5~|JSSS=|ZNV5b%osDw7=v$?7E6UE%{~Pds1Jm%0jM( z{KHzLD3~*Ql}1V6E*U=b8ZI_3#cGjRZ?5l|nKUI$no{^!<5_cmI428nM=<|?WxUk# zZTXLVQPvZa|9($}*7zp>)2rn_vjg9Y&6M#iMU(%T5e;%ANR$6jn{LbtOIpq=H9s|Pgb1^3b!jtqIh;;TXL8G<^Y2b%L@e%8(>yi&;#*A=KanojigKvdq3fYEb z1y4FO<0H>O#yZ=;M~MZfxmX)9Cz_k`!_#ZSmT?WgCh)aDoc%xciCmcJd>Likps7pI+fV% zq<&874pLW>x|7uJNZkc0((%`NKmG@xs7QD7ma6`dx3*;U9=WHgf0DabeHeGf?U*#x z*%0eH+an!rd-Yxhw^#4u?YR4KT`?S^mJYX6eSlE8`XFzm>OfmZKPk)YpQqmxZb6xxyu}4!t2t>2P^3wg(l` z!Gf-y02#h4>IsjYh-YLk7r4^BJK&m;lTwwxz&#ng0YViv5HKWA$S$5fV0W&NUO+fV zfP3XJuvXs21(lMs2E8#*B=j#Z)a|DieZ2`FUvJ82*f^j|eOXh~*TH&Y6+ZXbPahlW zaNOZiaR4FI!p#sh#0sgRzZ|*X6w_WAEoo;c%B zol<8q&(qgNB)6a0(aXt?JNg{R40^#_Wxg#xbj8Y*D@M8fd?;oy=;%YxXe=r@(=p`b zC}au%<5@X#fzJS61J%MZh0Cw2rBrsGLW#LyIU|v3jl^8D#OV+aNhHe@E>JYP$hfW9 zO&R^Gx;qUXIy0l5Mu3z+%x4}1-79Ofl6q<3SBz4(V45$d1bvOuB3U^l2m_>ivEd16 z^)RCwXS%*A?}dyh@L>_87n&S7L?WASwuYyemuJ7h2hC+&cg7}27C3PJ$iX_igCUeT zhysj%JK=vV;78z}x&8#|M{&}+^1K7;T*7$StW@jYK(*c+=9@L^-qDC{NIjK_!XPeu zmZfSv4PaQOOpZzYl}e`m!h}v_1mpHl#bs16T~x3(^Ai@Ah>=+IHUz_IIkQ!kJe)Qf;lwUZ{&yH zcnjCkTI9UtPE5;!eX z>K*6mgq$Z#h)1O2^bI3tNpC^(YFpx}zl#g}`eC@{^aWbd@T5T>D{001Dgs=+6?o-q zQcWW9W)u1@SuA8dDq(?*yC7lnZw^CMMnFh&hrpc#?A@rxnE464wXvc8JkvaPU@dEB z@Xdm6mYoE{8zNfwi{`q8R&B&`y!7|9@nXhdSkQww&Pns`Z zh94?3XEIuyO}B{F;gb%BcRRCQ&wm{9uz0q@dlJHe85ox6KX3&!Duaf17aQ;04PNY@ zCe^kkO&C*eM`~g)26hZEAs8R0olT*4SC#?k9o6&DnN<%D!TcKo<9o1@COhk{>GQha zPG7wM9_j7qOH(g0&M!dEhMeFf6c?`jt;2PSrm|u(317B;4$-S)xX+u=zXn~|0aEq$ zFs`~b(SC-Ru_&7@e84t(4u@m-ZY>d-niEsYdACvVEJ@e?lt8 zHO^_2oo!jML0P{24t13)Aza-HkLtZ>L~UF{?{DMuVzw|?Z+fuabdz4n5`ji9AF20c>iyGT zy`FvvXt<)l`EQa-mXZtE+IZT8MI2|e(}I@d2}EsgWW^|x75J%vQsIz=vO&EZmPS}@$7+0E8{H4l&2HRCOc5xm( z$d0_xG{9=h*dn6;1Uw3b!Rd*hi>=(fQrZZn$pW11*T zB=-M+X7jPoilONKcq$I5bx{|mfTd-a&6szd8fjK9qt#Nc;4(l);qHGGk3!YD8M#Lc zE&ZptmV_=CS{ORm&Y}ETR91-#7+EA37s%uK4S_07AeWp5`Lxg^kdbS=7=fg)%tfD? z%WJSZnrG&nZkWBhw3qvU1G^jg0C)s%cOHmaK8>#hp;N@mYbcNz`Dq@^WKA2HlJf;~ z7&oC00(U!3XB~|DRhUC{L~ZGwu4xqekPc}ST208H_pEUDwxUrn;5W zVG#E3)SC!LUy|o{wwyhgJ~aluS#3vyT|yv$l!YmEA6grHsp@%j zSt3kQyPyHsFa^$%gByl9G_U=fs8LFE_@G9rV*8T7QEF&>|_MCBD%5YlJfIO*ziVx5W2mKKDNjv!%%~agA>E05i<3 zJcvB>YA=wbfY|tG))%lTI2@J^5S1YHAeiNMT&fgiL_!<^+uy@<_cfBs5LL*yu}2 zVgXPOAwKw7=68jbeVDcnrD!Lr6H%AFeqPP@rDXb7`A=P(u^MBCk2rP^aV$kLmcwf; zjHp&XQ~h1tobz+&*jLK93_Yd1<+0gSS>x-d2;;qY#KF=QPah^w>M@9syz|6*Pt&#H zr1szq;l$W=Geqyu7`oJdZceSqL#Ad84x^FI?j^qYHr{GqAIrOGBPsM7&X}q9Ac=W~ zao@T;R7?Mi#q)mj&-T4v>)$>?7xqJ{lA3#fZxJW?PhwcCw zCL2~vMSh7M4V2*A0akM(hGrdu-LmVHKa0f=Ae*V;eE_Z>PKj6MqvT9TRnmcUzd?VY^3=(Fvw@Z zf+_`Jffqe@4J2;I}l-bOuNt|-*vj2({6D% zJH|R`8ao{L8W-!`zo9l;*%!4#PA5hOY`8k%=jA|PXV++FI*R?NP+7T>jOcTCdTa># z-`?H8n63={FTaSksF7OM>qcrTC4|fPpopy}tRgD>*p#OpML`Ku`Sj|kG<6THnd|KC zOg_t`gJ*6}g6xANuYyld_V^ASYCd+L#i0EWny7S9;ZI;s{RGOVET*Rc!6f-HD6U~F zSg)l%Yw76-93Uz|3X|UM{PGbHH_m+ezv=B}^#lUB)6&9T4>RQ*=Q`f$aJ$gj@%|Xm z3mor7&}{2^!Fe$3PN>7+K%5x=4vnNU;dR;`o!!Lu75QGaUpa^jYvRa>a$z9bI@90z zvY&Ve^Y1exi+k`J!2>CD8=o_odoaqGOe99Q^+cwbNEx9Jl_2#bnD9Nv8b*|LzD%8u zs00b0jC!gYLXp?cs~M4Sd5Ck}FQ{1hE6S9`SU>M3(uOyX5t*_7*RIs5%8^K&>0iuR zj6oOHQLc=1;2YxTft*8;R*X^afLoA%DOi|Wz4r4An3R{alW&j8%TS&L+*j~$nWYQJl_+=X{o_*A)jj045!l(W4@QiO-E8_KJLco!4LYuQeE zmA{Hd)*uu6tICIX#+&ZbK&LDDZ|Fn`oG211>oyR;XsSn*f-_LG)tR`+PeG0}KrxkO zD;z8a-nyJSK=RvJtC#lnE_|T{=as&XOZYb)KIOlMXP#?s#<6WBxTy~&guEMvIO_(^ z<9v*k2V>L5*>jE&ZbvGq3uA-?i~MFd8^}#4mE#l77-Ics908n`2Ee_m9Yf{e7~hPl zr@{+nv>v{=wNo7k$Mf^+0LI5}qNHP|=`G$;)wgkL%#mo%T$Z1H23IaaJlW1P-;8us z8!VAL_tY>N)mYat=ZAhMaj5}cG+5<*eboH3m@Uj$x1W0TdPp5$ehw;Zjb(aN#ydVD zUfO`8-F^yeh#S(P&W51qU?9grY;O)(h%v*#& z1Yqw+m=Fg0Ai{(&*oP4&guy`@A5C;1s!h`_EGXUA)a*piNC`<^$ z|5=0yVX)65ObCO05n)0Y?8^uf!eIZ7Fd+=~RfGv)u>VAu5Wuo1h^R+T;`}qt+V+1P zg$rTg{5QgcFxWQ{CWOIOM3@i;TNz1m|ZcW^c9)E#iFTN#2=8wM3pc92C4+cYzlNI3_AK7w^uequ?^RLe4p%A+$2 zJ7|ubv7cc^E}r=jiIE5nn z&qaTZJoIHur)h&)6G>QYoyJ=c&14gtV`Q7&qHBC90N><@elDrf9N|zzz&^f=sj41| z6IMw$0;{G;*Alo^eUj<`=#JX`r0lmK*=QTZCeyEc%V#Ra)r~}lB6^ZG6`}8w&VE#Y zKZ}mAtC~v5vh=jtF;lD3)M{zk7JX+*?a6|IQxsXGxC`Ge>laJ==?nR)kF;yVx(0`F z=tX{YAO-X%$$u^?DEnYv&c2y+tP)(yfibGphahTj9&0sG2~tm^Lf7!2t2)!6WICTr zL?uW)!&)DWeb0fSBc-euT`5LBmQHL~RcmdMco-%AoDzvjka`vp@rLk@2r>&FwYH`h za~VU1%0wl|41wAF@Jt970D}u4vP$i4Kd(`p>cH93pTeA@Xk)W`!_n)7h=y-29fVBP z7lBrPgD{LUbBLpspxX})K+*c@8A0dZl z-S&4{k1$*NE`I6v$P=TnvT}|=MX!k6%;0bvI75=-e79ffNk0L7GYUj6nqF)+2TNc} zl`(MOMjWEnf=&2-9{u<;T(G;$s8ZRcFez?73t%tH{~#aT38}f7i!i)7uP=dI`H79R zz7(XdE+fF9D$a95tgoJN%HwFw{JygZZZ*NJCOF_{$awU&E|Z^Q z(X)XThTvEae;%qZq&U>nzd{-(bAQcrgiAGl;7{S1iyN4}*ZJ}2;9P~QFJpWrbCRvm?NJW#@8FK{ zOr4+jnt0~ibv7QF^>Qug4B&HtN9#$W)OAQD=cujn{el4Xrj(VkI5vlI@Gp@_dD6cD zUui1)S@V27p7jm5dan{2e2{9LmDnO#r;aFVeXtG6y*S2K9fh z+Pk~#%4i^f+1fA&cJXy?3^inNsahbAs1D^#&8L!Rj58BD=1aRKf&Jt(6>=E0_cx)^ z(YgI1T=*m@sK(^M2jP(+(HOyVP|I0X3&EFMhjCQ}O0gFZ;1*ns+NKUb(}C5Ie{*%@ z7C1+}0H9WcsHjt1K;GZcgG41r{hdthO-x6V=~glkl^}(6CGf8RN~qi9hsWQgcaEv&2foHB{uZ5{=&Y5RRPX{`v&6mS2Hs6bno^Okd0vVo>G8$NdZ&ySLy@dSol43FS!Es}6DU6;uE4O;0Qi zW$Tp;^^VUNBWIlPka{f38k=M6lZnWLd^tecV($L2;7NNNmeIaM2nK(k7{1SWH?>Fg zoA1G}30g?Gqfyfva{mavCgBN**erWrSA7Z}^<@#?RiA-0Uk1mV^H(VPoWHAUNg!O# z{ruWIG=8(X6S;Couxat8X%iq0AdhctT=@Yhb1}GkU_r z_=;{Wbs?W9yZJ>a6V8|Q@nPqELtQdqk`S}K9IZ`^sE!dMVVe+#g!_P9Y>)1Nr}$|) z!-C)N`WX_suzr8pRTsgEFvby#VaD=iM)9*c01<0L3p}z8l|dIDd6(yn%(Fg4N)GqI zBwU}i%JWacXReK{3qSQs`KKtXoqdTG8;j!XD~;GXCT->EMkc`jfN`wwjjJ;f<9Oed zch`<<$W<#vta=rdvX`Ylj1u9mv9d;WzX#z}#2v91E0#EKkIH3?OR3Kw=1QVoIa1mT zs$cOW8RhJNzT^z-0h=C0-N+=PT+z?iMZbx+gu2Bz0{&Zjt=+zJdV+5^-wtQ}_? z*oejOLJPiA(Jv75RFW>i){f`l}F_g(Nl$Bo60wBVOn@aHXfwk2Nv zW)}Pi3x1gef6Rhs-FUrQSn&B4{B{feg$19Oh}V071^=l9zukg=X2H9X@p|{P;6JzE zk6G~lTJVV}i{4f7ozYaji;}2b#|3TV`cg`9j*II>IBfJFo2FM0hH@-e@DLE3NP5j+ z*rSnG)1&7u#A{dokU)MjwTgPezi@x1-E2hVfXGi=n9dM3e1sV44P0FOMybq5jo7yc znXe)UoRY7-m?|EwhmpIzhaFGn1X;UG%HpM&6-4PV?PIHqcfc9igI&d`sI-!5(0m== z7?CjVqfUA5oS@NpqKz0n zrC0~?Y2=MyNUq1PL)ia$hw5ELvrL;NK)8PBIfsDvBDbSvNYhaB}6%LFOzTP6pO@#dr%`bN)7F>zqg*O7#C$PkvI{YpC zT{#aa3BR6o*}<{vL#C-gLU(GV{&#K%2KXtYSPs6qWR;|j0UISp=>e@QlI5e%GjG4o6 z{#uZMb+1B7eqaN|fQw>y55xS5h-qB-aR{ah1#;jNV%*>Umfbynst}BfydEC?{k_`m@fKumK}u zJ;kgCcm!C#gJK*=LSj;je-0lOJ+NzuT5lDTah)CM|6q)rybT{B_132Ca_fs4vQXlte|rT>Lj zYBKfnQD1X~iavs!uOYOi@`z*B`uSz!bZ7g9zfuRHaS2{TG8Q61%>NGb zdmDnrP`}^?XUbvTggys%G!;x)-o>8AEfBT(>ZSNl)pEsFgzp0J%=kI+eq_(t z?8lVK&|Ms~s5B{4*V6J0ZOq6E!GFW~&=fcZuLEH)k*tiwhjvED2kfxQla!v(FQ?31 zy@b?ZQK#}Ao%dD4d05c>!q9V@eUg-4xYp;=Gx!?WjWiZKk-a;B zK9BtUEM0mzjKC~T<-nHngV&Ka`E*LZk2}5r|1NJ>>}UBj!;Au=<69j9+=ltGegiCx zzT21IRXCsW^~(_57&vTzfdjiXQyp0I-Ab#EhGGL5QYBJ6|OosPL?6xxTu7;Gjy8U@Lj zhSh*@L2fAuT>T5ap8f||@ZDY+BMyEoD!+VF#Kd~~WUz!U)tZOU93y^xFin}>P&a|k zlN!$x`8>MuyaAs%(h1Gm7|+vp!E>+zg@GSodL8;I6W}s=O`@1W>kl9Br)iJ-4hoz4 zXTu)s+pX|cvJee!OiKCXKZeL;5pNmM2}OUHC-0FK{ON3a@iYZ6`a*6~IOSf}5VK&} z@DD$TfZ?Qg2p>Vn=y%YfFU6)*3HMC!1_UfD=VebO{HVQt6IgrpC4M}DXK4Ay*8K$h zpM?K$Q1%3}k{JK5!86wO>U$)t`&$9OfaR*u<^|s%{m5myA-8-C81RJy=89wlb~Pz$ z#i)$DRi4ArG6t)vEvgUnHU8FI(-}kET(|DV^Mz7&_mied>Th`NAdfIl02%k=dkk!L zP5a3T#(6x44Hq#!3MdX)h$d6-QaF<8IKK$PlAhX@fegd^2u%tPo$2wosO>`gXwUYD z!@sa6F)Mo0Py(W1Np*8n%=&~zy@d|1I+akWikId2J%ezrV}dYVulFY%lcqQq!$B9r zCiAr;tfFVL@TxQ7!@FOHqz7>aJi$mGSE2-?<=Hx>?4i>NYlWuwj?h}3!x4-A>w-Et* zSnuG5j}(A<7nJ7lz4Xw_T=#g<%?>8us)E1g;+BGH6s7)XQQF{v<#VUD9iqpdpgCx?P@% z!?900Tz<|7srUq`=$lyoT2kAJ5-M4EqTKCgL6yO|izZ|TBno#73U&ll2+u&f_$t`G zSDEI(#i{#!gBIC7bF9Z?mB$xEdKgKvaVGDoM9EIy@oS_Zx|}MN_oh>d}bSc#&v0XyJjm9fKwAyfU|N z9)Ob=IfaZm(?p_y5RDx_`d1Al_X7X{YA2k5SZAEA_X6~3& zc0NGp68NNKss?{l;xt;=8u-v(OeWHL5*j;xuY)-u-N|%xC;C(5TJN4PXLLSu8;e<-Kl(RD2OXy9Yf7;Wo7C>DLu7kt8(f_?oZsBu94jr)W*%$o_`iV_;zupEWI(vncK;6gOVj@kSAjWhNs zN6H6%!TtA3q>X;UcDCjH@8fM1ufyf5c{eRHm(Vc++i@qro3cVNv1V_N63M&0M{ zxarBLe%N1<#(+SghCaJ)@E&!8iAsXT8ZYq**}55piP|HfAOBB`+j_6^z4=Y@B2FIID zmdF98x{^<(OJMI9x{RhW5*K`)z+%+?$hF$GP37ePmY4ssgcFq@^)D3eWUvPJL%C+c zV`VrKRS5EEh4?lrmq325KX%jt(>XyxVh78hE+pZ}M2iDU-RDvW83WFjTGgfRF=k-%w zm~S_`@-u(6?>C7#h+?D(BPv0XCfbOz63h*rRf_J+557afgC5=L5JUze*A@8M+Epyu zY7D#|o;Mx~&`hQH`m8zo!5{kLkCrbrU(+;u{oyMmjW@!f;V=lt*7FaI4wgHt2W<}{ zWxqfOv9W)L2onMr>l$ak{4@4I+&m*r^OjOaQEDi2$0)uKhyy)4MVJr7x%}rD75kq8u+e} zP9cmPvm;ChVC3(ZG}(C+{Q>5Ycpq+n($#UeCiG~iGlR{9I)Uo2w0lxh_?ifJ^^bw{ z;lLffOr#coR<8ioE06^Iq8wmPuLO-x*}3{y1h-;X=cMDoocs>Qsu096WKFp$=KdOp0CMq-)kTbgjy|EUCbLNeMib*{ zCx;a$0`9$z+!6+441$b?2;m02?gZS+#WcJh$2iB-y!3A|`Z>i1!P z(>F&dFQD-%q-(pu=~1_TI)nvI63quu2~wZ1KA@Yys(|?fXK(->7BWG{gFQUC`TN+H zY@?i7OHwGAggiPK(eN!)6l526?}-0h;VhohlmQB!S?{*T9fyMowYhvhKp)M62PoIs zUJk-ciY`Nn%;#-+G(=t%Sc>By!n@QPAu*%cSVi-+kaeGRqd2MPoCg}~66mv-NoL6s zwTYR8-t{VC^TG1}XnPYdxr*xV|K@g2cTdkGWRftG1xVOpF@b;}m09c zHJNtwOCN;pfc=J8S+W?%<}s}Os?Dem%KoJvoC^s|qgWHE_3?yiQX=|%Svi-&g|N`P znIEQQ7R>$Lget!B+=#>}7ZHzDnY0-}!mch&c zOnVAyQ=cY3ei^$p4n3`450f;iwQOQG9+w(SxRyXW07a{@<3CF((^9C8LeYjQCe^I% z90L8%NM4H{fO(X*y8)>9qZ2_XE!@%oIo(1Msrj9TF@JW$PFodH^W8j6aq_mh5cfT# zm*4x56V5LGFo1Z?0OI1(`&Bgy*MNxsj(nGZLgi7@b@?8h$* z#S^CGUJuMcy4m~0%Exrw4uth?_N|u&Eq@Bj#$dU#|LJXd0a{GwTeY&D-X8tU@aHxk zg~btL=8jR5BF*i6Yd;}3`L@^z#*BnYy8eeA~73%nO{cL*w1C@X)i8g*I6XHHO$87MGQW1}0Hsf{g_eK>o zA>)wWlydS*<4N^IHWHt6epENYvrwEkF10*Tg*LPGiAa4F#B5d7T5Elo`b+i>OyQV5 zzY|uY!m^M2J1BKd2-MS)_tHuFIUH~A5|`cO5k<42tq+gvE(voV>%8iI1g-r(zXE?N zay=CY zK&2S4U!Q5#Nwrj&x%{xODhSG-`aZ8rcqK>8RrHq)CCr}Y%vYaNYM;!IkS$GK zQ!D6)=Mh0IDEt*vzZL6|0&}gCY$Q>>o^RZ;noa}=(tclo+TTQNK-2`apzs>hY@s7t z?y~KDKZ-n>lT!V~cG6!MDMpmZ8d0V~ApDJJ?W92a@1i|Tv<0=G@OO|9fOzWHJr=l` z-~I_q#=g$I7TPD=#-x$S0|Ub8kix8FhCZ>`RJT@>8}p6T_3dh{VL|b*z+omoty%n} z>2c5OnpDg&gotcJx(qd*P;puloETA&fuHwyO!X?PDblZ#3*EZ5mON|o(^E;-;Zsfi zK}a$ir`J>FEN_!%?ou=eGt9zUx&WnU@9G>HFJ(IC0i_TCFG2}F1N~H2v8P3%*N|56P_noyH~av-Vg$G>z5u5Z?`l3 zn=u&kevOdBastybzz#J0+r7xOYI%GdKBm-Nn9WUy!0PfHe5~$shy|xx+|UJYil@05 zKlC!uIL-zPt2TsjI7u(O6Hl7Q*xng^0@}QjIL~6lts%Z-cD_F8e4uXa{VS5i=7v>D zwoow3Bvq=RfTXG=&ed*-FIm9PGL~i0__#Z4g=bQ{d{q^#ZQhK=!*FLI7@Iv<6L9 zTWUVzYd(qB_6$c(R4_87{a>2S^=nax;D4B6KHe z>c%(fYoQL5!b38(K}l5ax21wl)5urF?i_oPd{JF?1ef=Bt$mfOeQ|648qIeD_BOVg zhoz`VmqesHuZBqP>!RTWxei_?LKTAR%a~q9TA5}=NIRQR^M`0tob;}2>!*7nw7~;j zZe8}4!&9O=BY(tGay0v=%0QzpcTa7snY*Vow#(hq8++&O=4{M7O35yk8<&sTJ}o(V zT4}hQl@?Kp^Ed;IvvT)J;_u}dHan3LlZk3tXE?{Ej;Xm@nwI(j4KL4-PIGsFs)Kv* zr4H`NC+sEn^2*+Pn)~pR)Qpnq-t@*rurSD(qTw^?+gDh_2&nR=uWQLE`)_kNtewHk z;jl9V<9yaI-(+8sQ+(Bhi}<4V#6E66psmvhXOWCrrn#OCNO9|62BdVfz6Z9l^-h>g z{c4FW7clm;&%%)nqH_po?hm;UK`p2ny2)PaAPsKhdQjuqfnYQbQmW!A9tcy3m(^qG z7uIB?|0cX)UpR*dvM)ST1je$8Y2hKjn;U@CTdTGX6H=Fvt5t^;1xv>Ds0vNAyY<$4 z0d$!;MlClsTwe?+v|h3BxLjLanJeJIFc|7E-ySJsaeZa+5M@zN3ks;M=Csc)AZ;L| zLxm)$1?d#PoG@Opkm$GzcYOPtLX^3==mfQ(ZWrC*$`nj(D5ge>t|PN*L_3PUh0MYP zC3G^Q0hh_2TS#F;rEr*-64Zh!noM$XHxAPXWy)UfVeWu@5UEYZFRhN%| zAGURl6kC+&+r`#g4zj&i(x zyILpkjl-&NXt$p=HzSrR)QP}rab8(ARr<=SrUlAv>m-XPE-0aQDtHMw46`(FGcB6a zKCd8ww@U(ts{{nKps=b~kE_GR->|VaY}^eSZ^Pm&^e}*Q%DNP7Oa8v=`8yJF%HL6Z z$=}iX#I0la#9=`BSryzo1IGey9>)&@b#+*A(s8}V=FgyF0;R5?l%N)e#AHYe@bWI7 z6^E3(j%ZAHR^>Jy38;>o0@u`$_v_QuYDh?4R#ceHB_-9vQPs_8o$5&3;X08VH%|~! zKLTpbAbkXIM5&GGHYFvtJb%p-l?l@Sc-%g}AnT36Xr4sah@ch}1|)0gW6>%`-xI%E zr2IC;;jr~-%PU)o!IPEtR3hjZZng!(HIyqeLKYpK@O&w{mru8~hv{s4tJ>SK9dkBx zZhfAy&u;3tm56rFZHz(7ZFm-4iO*b+fPK4Kr}52myVd`7 zZmkTOr@-3(gDi*XhTcmFek{Ub%IKK3%P0$9 zVFOMARzq{bT_l&ct6xFqhtOdx+Xgh<5!xbdDQSXlm1xSKdj?2}S@Ez1_a}=8x9#IZ zBiLZo14a#KXF{lK16nn2;_nhK>42v2QgwM{Tfr8KzcnP9iwdIIRH8YRpb;-7yUrE&E+Hg8JaGBbZZbrCGj?4xe+S-S+%=_!2V5yPRhT%p9p4G1-oOq_jPV{jg!e&}YMe9&XbZd1(6n=b41GM8Vg? ztTo-n_kR?BKsLTxz_Q0-PpNQbyc`x-h~{N(`%B+paj^ZhplmEYfQ0pzCfQPwmnz9- zTh7ZYb+3{1V90XeIbBK^)H?4^h(JzqP9Ow(%H;&KczQ9dLx2bP+%50r+sAsX@4P|EhIrL zD3J1;K%#S!FOit%yF!!s9m%;${~ZiP1!kufm}P_)vwK6g*Q3@-#j`VE6Cb#}{Q9Fi zz<)t`UonGe%6_C=RXhcR4VAp7!F)%>QZ*RoY0MXgg}_S51sIsr!dzokSbk!4ef)uy z^_=-y`*;iwnUjZTQ0pIG2r|0dFrIxs0I2?@2UCGYM&GO1_V)BE;3v1I+0N7oC^8{E&-OJU5?&UzDG|wk! zL{JL~>#BPp$?RT<3ARV_Q}WT7MWGoeaWG?yP`w8322Gn16>;*i7>lDe;%zpLX5;=D z>7b|E&mX1e=j9PYtRJc%Jbu;B&tiLb9 zH@=%wGX$YAwDPoVLL*bFCKrR?h1+dIMYjzVRtL(-B?Qi{CfgG^&zI{hUvzmKL5S2b zA5+dst?NtzlAHE1?}dq9=KPnej%3p`yG7LR0bx~8H?suKXQ_Dry@M+;d)#{H7&_R_ zp6L=l@>FeJ3U_Cu15L@KW4ACRx`jy(?7l*y@jl6tkXAHJ!mI0~4#4jc$?jY+omz+{ z&5zb*tAV|bdf{#M&KQx67Z=g{*{xH#jC!J3(AHmyQ%)`fDfg(q^cY^peBb(4$f|6d zVAjOUu^!v+g0-N7#hM@keiK%9R#%!(4&iFJz);6&RU+ZPWO6rNJ#AjLvC4iPLSY}M z$S?D{-8H{cJ`X4EuZi1y0|+IXT!xVSnUDDrivz;i+>5BCQZyxb9mGo9zN*lC4^Z>H zoQNZWT2OeKnlCHAImtg2p;OPhqAAe%jnaHAO>-vJM9Hg$raZyl6%eyMhdyv$R^HfS zw&SSG2D3SzaObu^N|MQkgfAWI7T~`k{N%%c(tUfk;(Cnct$04hT0p6FtGDeXowltVDwexRLb=U)3jD1$T!nyaQoNf%IUpQFOpC7D$Bu*t-VYj93N}G%-J(S3L60SE(_>YJ1q#Q3 zIotdBjpN4Om{HiFImudjO2**mf5r20X{)`RNa_ufeTdk)Ev0&VhpIqEHLY7!v-SI( zXq;d)x>}zw8m&7O*=A9T8D3(W*z1MkjQAZX@e@151#;r>rLx;CPJ2bmy5*ZPteLIb z?@6lTJJ=NNaiec?NhOP zL;e8&absMa!vj}$Dx&p+q#y{>~gqAjH?Hz_g9)TLy6b&z^CWOa3}Ny{t619*AK4acXb{ZGZjnA+;Z%LN9p^Or zc7-$STM1|KwYI_f?K&l z-H9!!%xJ7`fKm31^}@YBN~o0RH?>>z;a!QM6jIXZFll|w5wvSrfmqM)W?11Iki4&S z-NB!gN6r+ZJ2+QCKE9UiZLYsq{!bBw_w&P2C1> zJDPHCgLg9#(-Vf8Uw4I%kTf$i!TmZ*7j>m|i;@`Dubs2N@IY=d;Cz-M%>({weC9bt z5B*(u>Z!ERc9*M7<*Qfc!xiVc*p;h|2iK0$M_f#Y8aJA23BWwKuKJXnIe1O92N+W? zaKU-HK0o{$Va%5o+b9vYK11ir{P13V@_C9*^Zs2l`oq=AWW~BEZltf!$6i&WS+@Pw z*H?lo)DMH2DaM=!FTZ~nzLno4qUK>0>vaC9FqLf=m*06(soc7)J^f4J%9}W2BNjxO z{}qPmbb9Qavp0%7`5U%ybRl@Nm5qMI`cQ%XPF|8;Zq_+Vp0!Bxn&lb4c2P)WRJ_R> zO^gPaaQzz`om^L*Y3UWBVVg@g*w|eI0$WA(+z7F8&+o=LsNDP_ojvA=b}y4*b)}%@ zqlENM=k}(np%rr02TX{%sWaoqw@YVijy_SKc%3NTB8q~x^%FJ_vv(Dc zt{2i0Aqi?hVXlxqSwOl$NVf_}Pzwqh3hC|w(v3p8O-O=TP}oRFpDG}ITu8SINl*(4 zZwD!C%n!Gb^ikcDI|z4YC9R_BE4_u4mRG(?Ju~%V@_>xEC_b+fy>`!YEZKD0lTPYX ztUmkmPU+Ok6fJ4huO@5@*DF?0v*)DT8d%zxc-X3a&5^8Srj;?}_Bhy?FFoO?j^G|x z8_34$8LY!@2BQRTqp<9?-ZxTB-yY!Uw|8{ra+o*F zc?2<^U;{}!bJonml0T4`4I`%cl3hk@-IGhO`3aa9Je3D#?!sZ6lm~?0v$Sn~QlYwM zJ-HiK@+p3NU`jrX)76Ox5Bm`d7Y&v3m^3~k@DZeO4~{|_&3jY)>*D0IIFtMM@e*AH zDsggu=1&iFl1`4NbTYr_%Fn#-)6_tl3!1}WpUGh60H!^Pe!gY(OSd6it+#LH%Y0kpRLdi!#Q+_|wegJRQGk%JAMqxxy z3o=?G_bGAZN%_{?q)PgQXNA#FJP1mEco6PscyDJWpU+K{$63UwDUabHFmm+I3Ek*a zlh5N$$N6qMgVo(h<6*+={o;D#Zq2xUrUp~xf*-z?SR#y6=KC5!TWVRH5wS+t^!De> zF}?keeYHM44ZbLezF7<&BeNJ~qVNSM(MLQ?I>{rrd#W}^<07D4mGOq%MEFGl#&<71 zoKiO1g6`YpoVDtcUD5I~b4hFh6*dB>j2j5zohKu^SO@PH1YDlspEKO~vBDc=1-SWGL}sj4a3b7&D{4^N5>vyzpr&^} zWjgDg=s42oU0E9G^!oJwI?;2%^j(A%6P?Cc0_)$>I@A+=fON-q(tYjk*+t)Jx_9O2 z#*3dpKH08nDfu!Ynr9mEWMgXb%uq@m2jue%KWSLNFCXHVu=u$a{lqR?53vjjo93dQ5#AIs(3xBJ7xPiWwA;qQl z3%8|MS--Pce5Smf0%c9VW(?3DJqdhpcFc{5U$t&RXSKc#jIr)X04$uJayW6AV0vF@ z4iX3USOzl(FzHEDkM!qj@KnGr<^o<}-SlS#?N6pSI@Ef~QZMULjXAAnSVu^tdR3}YPhLu{PLWa68gfz z-iXti^k35(5fZsWBk8xe@5>`_jc(0(gF8{ zrzts}K36ZZyu#&FmFBMqN|l&%U5VMB%unl!_M!HJuT`zCh}BZEIbrp3@@t~?%+tb@ zOU-ci@;Fs^83&(N_$0sOhZ7U(HaGBxjYf^aai#gcV9>WP`HI=an7fV7fth!jP&}pP z?}Vhi6$+a(CG1x|2^%X^uit})Nt4CQr|ae94}h1{l2c(~5gR2;_tHAsWp}6yfqZ`yg{B7cw2>%3#&f z{=t=mUtA17u(1hMh?bygav1e0-F?uRr5(uoRPmbDxOAOuH(%D9l!!K^lD|=$)&3Lu z6WC_sG5ia5dRIKCsr$)H<};$z=ef*huy)i!XPVNm$;%Y#|wgR0-#OcMVL1 zGY|HzaE(EeeZKL!e>$2j8p&S}O8aTDl1*%5)wRC@9TC)m!d9wEmG(0Qw35(X6PlnF z6c&Qk{zd_*ETq2)Nl*(4TMOx%1*9$^{ar|cT2R-KI0{PFnc6cI#4#RTh`Egz>rF{&pd~s?swl z2x>tlh4yy}NHrn-OH2uBL4nEN9KZ4K5(+oDh$Q;DWFNAe0x1MsVgb6aL;Z>Bsrs!d z<-$HoU-L48WwY>%)+?1^eZs$?V1TsLzFT0Wp7SecTR+b4vjwCcA-y4f1#RmmY!AOI zh%P6+=5m4p6W2NI?-gS8D%QUhOHd06J1Upb!Sqzk&a&zf{C*+Y_?#m_+xl^ieo#P~ zAS9EZpl$t_1P9YI<1Uwq-O0R~>g&w82@`$#?EVGX4@mcmx9GtgbCTB1D^83wWjC49 z&#IY*f2uVAV@~&OSm(a%!V}s})stzI)?oAu>t?G9OAAbP=Q$Cyt)H+HRGLwj;^sQr<>n~2q zpe9}Xq)sK-tng!F)u_DP-pSj6@yfW+o#O>re5HtAfLmK~86ApFV{~?!Ri(}Tbxq$< z^qwoy8xzf>Ao*3IO~}{kl<(4U<03y&*Lazj`RK6})xN|1G*wtcUp=A$hCY(E!<#zg zrJH6mzR)c`{xNF4QJ7$_Wb#`l0+Ws&eY#q`iVj~jvxiti^^PmL04B3px1?4Jvm*AN zYynHf5)Y@w!-}6w5D|_r%RP~wMH;8T(H?7`fRp&aueJW$$oEXO_sPK2-ly=z&}3CU z+WPM2+Z^C0?5t{Q>16R(hxaw75vF+4`4VpiAH`dZZ*z5i!Y+k)-F8pGnwB5qkMmp| zo=eHD;4bQ-1emc)B9)Fvn`_|r^l4Md=b}4xxDfua{hWzEnPnKt7YP7giw}DvYXTN) zYw^wdI>uztD1{f$N3JbvJRf!nk%lf16Zs2rNFMv zJ2_Y9*=$w0NM4}vFY&8_Qe5Xl$|U}HjeUphDI%?5PQky9UUF;s>Q2T(Cp!I0LX z?p#baJ+7xRv@HTUIKH|VYlU(FlSap8Fegm_qpl=jEfdEW^eVJ*7kp{XEa zNFkJDHWd4lw@D1U(1o+rxw$=mtqU+(&oY0+7*HCyHB#~V))mjkVhgNfEwn-IF_U9@YAx_e6tt0G6=v`aPvh^^N<6~;|}+)T-f?e(}n zO(e~_qi%9)VcS=+6paTLOFLVQLd$PPH8#uCd;pPbt}f-|_A# zn}-RK>F@f=L(-6QC?(ku9GZdcYbEc-Gt^B;<><*>%h)H?^l8}|PI@N-H@Lyh-%c90 zynkm3&F|E#m{lfx+gx^?=zUj4(%nQ?dXaJh2w^mLCVERn%&J0_$;x21@~bt)hKj&n zymXhWlTRKeo5?Pq7GN~jda^69+lA--)$?Y)94EU0ZvTw>oXvFKsd0UGpd*4>P{1xX zJ9EF&Dp6w&mCKZPVGr>9Zl}%ZPSCo@WV-bt5yd0#e3aQGV|G0|eJ zbukGcRK~7tp_&~x$j7b?;D@p6vnJWN^()x-F@v_J`nVYh>yIMY9!hBnTAc&g8;a?h z(<;cK>3){3=Y3lAJTc~4-irqqlu^1?)v(kS;y>8;yEHmTZty?w|JvX`&9*gQ7i=LH z$%U}52CTfkFGW?ucPC|Vk;*`G+e#$KZ74ZdrtKJs-BrWXCOk)<))V(uYt4Ig^eXND zdZG)6qp60+S&7{(E7pTB|44^kt3^h>)Vz&y?OC=X1bThi&@sJb+E7X-3$@hr)+|a3 z19+OC8`h-8w5+t%zftX@Nn5%-?J;rKnS#`au_IqQVik@A1DVXg&fy-g-!A_9!=5Ve zime)a;nUyKJdCMBFNQ<}**t_wSeY-__I-C+q?FTqm(XbadK} zYyG3S+QwFogP34mDWei)OGYJOcO1Pn)RXjBRW~}&m3Fo`w+xu8WL6H%ePG0QxEWK4 zsJ<`jD>Mh&WK=r2@Qk=F*id3DOEtz;EqfK8kR69Wt?Q46)onM9q+9j}OfVu#6H+?A zPt7874&5x`m&D0Nj{QiY`5yQj5wxwJu#ZN3)n)aq?|{eV{nmSwTfbHB5w~8kPi0tM zladASGzrA$KKB>ReW4lN10}ld2h`E-C&0b{+P^BKe}HiJSNeilklw@0%_35H)uY$o zg=JA+^FiaK)cRjBF|1!pxBXIq>VcwqfT#*;L3%}VPW#sdq=STXppXQ$AU(P{r~R7( z(!oMHNJxTOP&iOXFBgyw5z@gz64ZjiLCS=z9=PB{Om<8ksz~G^N#qb>B!}`-C`LzK zi4la#Q+5XTBZtjYZwf_kO~QJG0`Orh6sWmwWneSbWkiikIX7xQa;(9DE<9L?!NC?p zMkl9a-o)Ubkhs0jJl6Dpsku0q9rbP;Rb+3VlNtkCcI*(&JTuzV*Dt<~ekb`%k4kAA z)zuSsQ}u_Y;aOS#`a>ikZZSpl$txo#-E#KO9UINYhG{Wb?uYP%t zhuF#a3?c2`7HA$Onun=U32H&%5LK!QgjI(=uR8Qz5W=B=JAI3vCm>D6lZ8-ae9~D4 z#*l+`kV)3WW6l%kj>U!KQGe->Q0hGSY9|k@+nv995h176K^Pn`y^Z#*j^WMgu|0b8 zaJYq0^eacmC3}xfr{6qMumzl1Ri5bc?LKa7HRR8>`W5$R;qguHFf|iW{I)?tzcnpC zv*#d|LLWhv;YfaPe^l;c-F*!of6(#74j?4%{urNdG(VOn=CXgF9BhBi9EgIuN`4P4 zy{n{g3Ei-T3#jrS&L6Wl=5W}bGMG6W_G$(*2Qcom`5mxzt)rgZxE|VZI6*9_0Bt_q z5<;5JpR@SpaGHO~VCHbxUo)6F9QIlUGY2r8Exemi2wikvfs?*eZK0ItHw^PLq#Z?X38h5s!J!%!zhXOci`mRA7K;}Gr}XH*wMS(Q%jHut9TLIhc}@DV|7>r+_9|4^%x>f# z)Os|F|5T@yrZHoqDG_EQoRKDvUdMM(lqS^ACU~S4z+PSIMz3% zwSK;ILmjBuTXGGbaGAf;#OQm8&Pz935req7@#H#Uu=F@VzAi0DPQ*2I0#JEQ5}19l z@F9lmA!!FV`>fY)=Okw!lgZV2T4?cX&#-))HxB1OAOF@czQM2kIWJ$d089YB0I|b~ z&z<=ff;vz8GOp>WhI0Xo?x6%((1MAVR-8ap^UF~yQCJv6mAL1MN)Q= zlwBlxyU9f28OcO4Iv4(G4+}n96q<#7ZDsj8Dq8!3EPo54qi^+jr}R%+UoIqnadI+@ zFjVcM!cJ_N#cT^{3Xsh!OLmvq*1~W9omyy%s&EBGoa=iGZuN&f46iy_U1eOi8a%8D zyb^0??VWy)XK89Np!~ioN!L73`B`3i|(u9f1n315AXe75F>xVas!)XBG<|2?ui231S zx%i%{Z@cjk@um5~h)S(9uNe{d*xm%xSM_i?2;%dxbgh1Ur04 zQ8N6mGBSx%NrcLoCVVStAZ^ ze%3^m^O5P=cZzK`A3J|182(GK-wdp_ z8cq}_Zv6Wry@yy|{t_)@lhm8|#GH-`niGO?e@Z)W^B|&up8;^ARglcgNu4{JrrbGas+1lR{Y(4;d!4>>2i&sa^a5Yc%WbF0aRaC0s^`7mEDXRzk; z@`^@MmE=kgI_XS_*4#83qslhs>`Xe%C#;5+T7Og3>}vg8pJDxK|MV-;381Q|hsV?Y z*2}FFZ3HXqB=t4Fn#O-!5xE)dANnq@d@YNKzDgi21bMqiDE7&%i>P-Xv?_X4B+e5}RD+$;6~VgxPbxRqM8()caU1U-NjLC+d!ZXm`@1ex5F- z+y5kE^EW1e{+W9({go!5Y&~(oE$NKIxV(Q#zajD!#vEJ7;Fi_x~ z>6kJ9F-b*qjM(tCl8U@shISopI|t!O4y{8%VRxh&?xD=xPEz9$#|P+c`Uz+r!Ot1( ze~T?)_b)}i=ezhF{sQ;k)Wc8t84;pk4|B&$-CVrVj5F=0T@Tc;!LO+K)6aL?x@Hg6 zhpzA-m5J7V1Wl!@Y?Bi%e$Z@-&QKA;yP5IL?kaN~)>pw<@jRN|dRzjxdqS`Iq_rs; z)3_>)DQ;2kc@Ae-K1&JOO35LfTyGbR?(1rF*CVlz&}}%9O&igoKLm%(8x(d5VeD?( zI;1=70Au#UJ8ep!oO~WluQv>_y0VGB`d4d@=SAbM^f6WAo>m`wsXdm$XG|Ug;o~G| za-7e-JRM1N&g5&4WE^qH_RYhSI|-yt4$~ZZ=DmZo ztI3(Txm;Te*Eqs!$$O{x4cHyh_|geA;_3ME)fLAV=IvGoCP&*J;C=F;*e*fYZdY+< z@(Fkt{71VNa5hg{Dw*E}d_+(SYJO5)30m8o{H?PvEK=K4q43^K^C{vCqCd2@hyl<9 zgWp#&{Y2UH6a9m(r&x8+{W|p`G}P61iwUTHro9S)jF0b&k54IKK`p5HU0@^QMVxt| zuFUg`^O8U5f*sm?CDL92JAi%RL{#DRa!*(9gz|V^uHosq@I&L0w!s*uz3?A_yFa*o zoZs%f_XGc(yWXbMCuE!X68=C>$T3;#0c&@7>jr^+3v zJ93QC3A_oeIH?(h;>xlmr>^{azxg@(&4MDaizA%_6Ldp*q4#vmhdMp1I&v_(3OGKN zpPDWO=qss{Hn) zDI}Z>sIRs;gLc&MPv2a68?L;h0yfqrk?Y*u%Qi5Awr!+dC~i+8?c^RXR?QCV z`7Q0U`|GwrM!iux(cj^RM`mZP_g&tG{~*aE90L!t)8w|yl6zActFP`fbO(dBPYxE1 z#=D6(5blKwdpIl}m=uMUhA^r3OkQ&X8o2t1xTPN}nceUqiaR_R5U&Za8O?>j^+i4O zIlb|jg7TdE`)&GJ-Q=ekoh$i`%P_Le>ZTgNo7oEMglG)Ph>ddE8#Tfb~ydZLR)DPz!2_&s5)7JVUd} z+$ghXDz*B-$>heGZa`k0$1(@J05cT9GTaQtcWQ=j4#%IC!OY>X=^4x%4x5p|%mHkt z_NsugYw+*abA6;gA|Uml+wRFaT(VtW`IoYLA3|aBvb^%TeD~uEpC?kdML&N67KcA5 zQH0|7>S4E6oSsK6#LzIQ$MY z*K|!w9x}*pf%sBVRzf`SBS1or0`|*e!vsh^N4#{*T_ySik&cwyl0Os1aQk^ek4Me> zFyG#4{}xj~z%26ys!zqF@wOM(A$Qqf z-+T-{(z(PRw+dgv*Pk@#%X(}&J3B>Bz6s^!({0UfeRMUNyJ@t__ON|m;{f_pPq28F zN&TPZ0LLh8W5@TYtbAqpmVJfv8^?lMO6$jbEqr}=eJ~&8bP(BELwfqymoSxN zGm82@tcT}qm92-XcD@WHbAJAbH*1nPOy(C~Prt`5T#cl8()MyDGrD#Ex7081x)RZ70%3FiK-lhTbhPt{u>#&C2Q*cTB`pVFU;YvwKJGG zfMIJc+0x3^q)O_g96k`p;jMLug7x>him7(&?;H`8>A!&SurO@8#zqZ>I!vZ5j)f*q zQ#3QzS*W%juWO)z{f$^0W-BS)l+wrN!Li-{e!R8S?~*f4m2Mu0Ss7YOzf(7#q0jWq z)lpWaZLVFH${L$j^qDz{_qW^zM0)<`fSmvx)9}>i-Gp}qw)BwDLaAg%Gge9d4bSt{ z#rHQJR&85L91i%#wtK2!U0_4TPBnQJR%eeUGEbAgNYV>UnIGrG2`{|C&OHY(`p6Q+{ z56f(TZ1}ug9}YbDXG1@dlaf%t3PWt-6aL-7xB0SwXqI1I}{W zrrEmqh-InzhQU&qo66iXW21p+4zfQP*<++_=74M9rzbJJPCrkuv*2wW*N=v^c^JP^ zAPyuyfN}6;DafRS4S?i_KwN<|dSgciDBiDlXzqsI1Iu;-m>1W}Y)n5#Bre>}P60#F zG*Y#`Dw{@{M>)lsZxy9`vkn`*x$UsgySp|e>b70oZr@~DCf()aZXc7s#F%VX5%zW4 z@g=*hC|xbr`|4Og_tgi=Nz71*%T~jE-EIv{JJQ>$d+g?kN^`Xr<{2`c~<_e@xtBy8kI|!{W|gX}W^GRl3bFaf|8x zr?`tO?&d2^SJ1agck!6G#dO~+E^la*?GX*`o1pE6JydBZ_w9jqaUZ;n?1ViXsI!1< zu-M68%@&Bl#v60M5%rlRqr=SMumu^+91hzmgP8*u>zsE2%NEG_Th{_g7SLvq>a@+_ zu&pzgIe1nJ#8r`m3DtP+L4@Y)z)kn42i!~f$93o z6fK>HOo?`f7N$g`e<%cKEodDlb~zu;6Dp7c34)P09E_v+9#c?G=t}m-0n29rOb(R0 zCuW217`$G7I_EJA!84&x4}T5y4(oE1zH^4&0l1RNx@PkWbhZ~d}wa*0DPQf^4$yg;>*xDBH8KC z3GYPjhb@v#w5#oR(Yi8K-PxcvON{;_f?7~mOm*o`p4Y$^_9xt*ZsBm3=+fvOq_D8A zb56S_mBVZBH>CM=DLhYkPNp&|9`GVzw#gyfO#T}(T`7z17FzEHAH@fbQtiE`A zA5ne^?fqoPs^9Q7=Qw8zr`+iiR))ilk#M@Qeq%IF)%LvE7Rt^sNNX{6Poa!7tm0O} zKgc9^fT$xGtI}7cOJ7oObwgZLe{#wwzPlnB$$U^e^}{#aM26t`^6>Mre60}PXS)CM z6<*Q{x~g#WO6-^zhg>}34z7be7gp5=HZQk=;c5`;b&_lFvhUzMM#A}!uW{sGbyf4>d#SAM^R`eiNLGiA@rl=vmdEa-jlV#Z1Ofe8@-C=^H0vV5g%FWIJA}h ztUo>@CuJrctX|65xMyR+s`(D059n&OR12-dkp`mE;vD}^+Gmg zuM$mI-TUDAIs-k4ZGv!~QQjsF=i|`q)o7>XJu{d& z9JW^mGl#?W&S2(n*ghG|91eR=1~Z4l_RV1CaM*qs%p4BeKZBXWVFzR|b2#k43}z04 zh09eqtgk<$@Am0i1>B}n(K96KUkUp}it{P|4j+`!F$Z*JY_v_p21E2A)vs`&V*iL) zdK^9}C%XW1=I;TTQ)(Yf^lW1DGzfm!Pf!c8!+znvYf# z7c1x~emv+?V}d-v@I?@umpFVzF&^coG&Q_P^`)_;Y(A`z$Awx6Pw=TEza+ddtxw7> ziOD3A{0g84#o-2^$x8)*vY!0901$(wu}XeZfG!dAWC<>FX~eoad-rV zhot^njsExxnP)l*ZR4+OcTQfZ`qDn6P#>NFv)QAGuAmka=Gk_*5yP&feP{vw8#%h5 z7G&r>+|3OiyybgX0sEUdwxAYd*zBoIh<*xRSBanW0Ya)XdcpR43-Q0D_;xZ?Pz$ok znk=V#tAM=>Exe;f2Z9(%MqA;NbR0ppAMh@ch}E{AHe z5IOYBJC&?d6I9X|=GW}*#ra19;_zCR?rzt4O^5F4&{al5SwabS?a|ORfx3Jy8zrAn zqDj}*=U73FT3e^skBivAO+6kQq<1@`}c*Ru58rj1Pv|@2v6cnxz-p{vBTiuG%B_C`W#6S ziPHg;lE-k`u|)609ne0yP_>?gMsp8UEkP|v6=qI20|J?hbw|<;$or$nmZ+Kzv;Vmr>-R5jXoC~H%_k>*q%Q=78!uyb5I3}~mh z#L3^ImXA9n`4frGE>#+9(uWQtuZo3ga)Y+MV|%K0e0Qy!{GRkJU4MWo)%s`Y4nlm6F?FynRfeNWLc?{!WAuK`kf@QyT4K3rOD=((6JJ z)Plm9ASM3*Hh$g|c<4_4iQCTQ{0m2TgP(4<{{N*hIcitK%37MVXx(ViJEI9?jQbJR zjn>`C<%D>meU6TlU-m4Xj%NSC;o1xSII!zLVE!BrYaSe4fIS(M&B4$@0BhNm_Hl(A z{6INa4o@S3T2S(Dyt6AMruJVYv>blH?|k)7)A`%^`EQvR-# zC*9FU_Z{6J8HPWTA6J}}yyC|ztLo)+YiottCYq+B?7Xod{jtoRunXXgqCB6ovV68r zDCFchRt;4vFnggw-tg;CpzFnO|r zieNHSK|84ac!rJ8mSgANpoghO{%leQO{!PS@K^Oe707nzDsIScLg=hK8hq|4$hhq9{|O#&-0f ze&^qO&BZ)w=OdG6_Px?k-4d=4bqA&qE+@Ao<@~ksm_3K6_5BPP=&cbqr~5bg z1U=v&Q{J;_8w0xJ@Qye+h&ZXm5#ooheKR6s1OdH$2!fs&;124%IkVsBChBK1ra?+C zRN`AHGY$N)Dt8%QIETbqUAT0~XN7OGDzMH_rFJO(9@@>S=co(L1;5!1ibbd^LXRSR z5cur0ZCq1od~VGw`!@M54aTdC3qL1SuF@@KZ>Vam@fbYbBHCfRh+0jXx#2>opDK+O z@^4(kPjWsLu5ra^@3r#Yl!x9gFXkVPT%TvNyaqMpl*Yq3(wBtv))c>$Bafjt^H6I7 zm6_{!Q{b+~x*JmGeL^|f9LASO=08aFW|O2SE4L=QqD;Q7%9(vqj{O92@Q%vz$`7SB zn;WH8g6wVhBr_mkVNw0pJ z_<2tJluQ(9`<)7_CGQ3qF--C|R+CxeYT`I{-)yL{)2V*F2&)3_F&nU=<(mFZpRlBR znmruNl(>TV#Kdu26PK5E7sdTknA}s=FPBh?L3OV+1py2&jHDIi?F3*4POmk&%hg>o z2g3#A14D!i6GREgQMM>R^Z5B=8{jKcmT=Tp;)BoSdL-{W!T=ZAgu+Yme|tP=?gG_tge2 ztHn{wW98lbSIE^rx!$!$U%mM(37jne&yimwX3RpvoLSjd><+`*DYw$h!GsA7&3Dy> zn$5U$STC3`d*{s+%*vvYOeO>4yH{OU1s3!PA846o zzoPw4jONY49XEc9$D&S*N~Jx_+u?H06yllK;JbdNz{_AkoC@BnCH-l)y#WDLUg_OJ zzeZ?a0W&1s(yhD?9AGAXmg&u{za9<$JPQmPn0-uK5QixgSzt7sqXB!ZRidzz%0V_M zA8`5JN%;)W*IKQ~^Y3AwAdJ3YRM<|bBDVQW4HU0Xpt6A(PN=G;Up5DHft`}U%wcKz zzSLdN<+?o%)o=;zAB{xbCu!pQNe?ftEWDTSyFFa*e_HIUFQR!ypCYVmW<*?Y%!UWs zRkrZ0N)6410~I~|8`gW8t0J1-TH}#YbV@bp$FnpVmnx#F%>k?GCALVX;!5Sh-2>8Q z^RBn^EZq$KLiB0UELqS(EvF^U@DFB}J@c3#G9C^10oq!a1}(14_%qSRSM4E#>^6+G zKxECwuD;ooW1@vIz2C;%ZFkNj(}@cPE2a7dKUkRsRW-Z;UAL;Lc87oAsfHKGs$B~p zS?K0|pR+EAHf_3{ljd!zZJ!|zjcI=zcM7)2I)Uiv&oG9kgw^=^llIB#IDIyvZlG#@ ztdqSAwux9FNB2Oh`5V&b=&!1Y&T#8QRCpDkBp<~Ez;p(~h%>p?0j4-GjO!g>ii3Yb zCb`i8$MOjRtZQb39M0*W+~g?IRyubkdRyFEvevsyNgi}GQ#Jfmq;GM+35sWcTODAE z2?N~j0BIqTJKZfUL~@tAPv+wpyW0V#X0UvG)&ZuQFuH`9W5Y142G%6vpn3knxAsDvwht)ey0HW1ODPdTlxY|bRgpaa^C z0=}1ZgiW5bPGgbwp55 z3kn|+!RvB<+OZ$Uw9eTMN=_L}75cWjk)2EZto1Ds?Sx*XeQF_*x)NDei3n;zVYV`t z_N9H%<;d?o)q(JD>IP?WMgUYAf7X~aNo^b>ANnKfEvw1fh|5tBFnh;#0p94x!NmM6 zA>sPU6RAdSfhm11Gq+CvX^Q;Lt8$qx^9cGb)VwKwy>_?%{qScGPR0r9HlE+g`=0gi zTGvO$)0>;jvb35^0Vk`oI={35JUt*fL#*9a&ik=wJkp-ciNQuLr>Af~Nc`p?NrI&T zqY_HUmo7$H(%hqve$Qu$kpNj6aK z=Fx7+!(7}wt_Wv$LDw?qmPOkw7(Y*jzumT5;K$9iVUWBC+I%Ulyz~#tOW!UN@D84X zr~`~q!XJGmmHAZ+XQ^83))m!auELyuq%M?^s%7-yC;fvlUO&nDc3 zaxg;is}FQO*5y#V{2+}wmdanGWLX&%mQ5)YfBn$+7Sr;rMMsk`Q@*}<$doLLKZlrT zCxM^#vq!%?c?yg7BdV1_2%38p;2XL79Uo7dV?yi`&?F|Zn-+h28Sv0%q+sd zVQtCzb+v^!{7ud4pK_=3iIOGs1<`uVY57q5Y5}Ki_sl&aoVMd6{VPvTU{7yY@%?7e z@VXLh8A0-t%_Y@8d{HznOljJ#nT^rsd1O_PVB_xvX=IPvYismJ75CgU?y;B1+#K)^Y&e6N!(nG;FmpJpmBGy6uyzJBhr`awVCHbxNCq>9!_LlN<^ZPg zOmp|gkPCDFlI{u0`l>AKOKBMM`a+n$3;$(>@rJJ6g`cH}&~kZCjheXiH_Gn0^ITtQ z_pGZ+)3b8rFaZ)M3?}Ns;dy<~hELKLO%WfvmC3HFn6)*0AIQ~HqoDh@PFEIF9(qid zA|6UMmUI2EQQ;9DVhtp#(RsQW)yR|*5%XnfuD|sA8y_d1rN!e|iO`aDFp#yxnjnw8 zzma`xk#4LqNj3l?=LcRsrLmD{l)W*idvf={|Agdccpfnhs}d&;ef%~hP}*r2;G0b4 z2lq6&r@MQG+^e~JvfNYTwl+#S=VxVY4o~}n3}z08U6{el;joJ`m^mDFaRxJo!!F5S z=5W}h8O$6GyDWp5!(o?aFmpKUiVS8BhkZ~my?vtf&{rg{QcE!EiOrQ)2b}7&((Q}n z0?IWE(RSyl=j^wgh_EHX$z=1PEFE)rIv>tp<^aaJ@;Kte$!{>|r|^*Jl^KpXJl<6q z%p4B8I)jRW!OY>XYcrTR9Clp>Gl#>j&tT?o7~6&3 zCz-=xH)b$%IPBvY%p4B8DTA5AVK-+mb2#jl3}z08Ey-Z!aM-OG%p4A5SJ3%02Qc&- z2kThHunJ`OJO!&fZqM+{;c@TCVCDdZ4d*$)vejug2paa9HD#*vcM9~j8Mam(QC-`J zcwe=)xjrDPXf_J_s|wh~0PSogt)FcrrQ@T=|C+_0yVCfAzFGW>Y zcJ}*N9JVkN@5jU;;P>;%=4>uw`l9JLcUgxcGkWPkKlz)RYJdV^W9~Z?t!at?t<10W zQ_(+TOt>u2i{Y!Ke(b+wWQ>i>>Jc09(UZslYi3?vQn?KsN)ql|{2^TSl(+jA$)6ky znIRf4aPC7BbSMfi}W zMVyB?Q*L$MsG#Rv%$LnB(48?orZ3_uMiHADt z1Y*Tup?*;lb-@dFz+{K=`nsAM0qJ!G(dff{1LI)_<(Td4)moBK+>KtarED_nDf{6V zeg=Iu``*qJ-!Ag9(&jcVPDDx=qwn=n`S|?sXVLn-_R&M(;&{s`|RkzNL4sgP|z zw2dq6YhwrzeVwvvZbn6^r#q+2eP^D@eu27Jrb=xnG~Y>hxJpZ(tNCH8;!ZfvG!^Fv zgMPR^BB%w0kAQ|jSevb{9PeD=(Z=c{f?80x2E2s1sU5RV$Mkx11$;eaZHtkko~s{p z0Zpe#&A+i?JoZ#CKR#W@SNm-j+cwI$MVNqr1rkVgz~J1TIg5$kY~WKDXy>sy)!?1R zKV_b#N*Q=HkXeXfiyl4-pFZc%Ubbv>^(XUhRWH93$?D9W%h+LyW|ybzxy&&3m}0vh z#GNQV@3Bscy3w_cqYbTzgiZf!lX}8N)v=OQv2;93IREYUZS7}N)Qq+O6u0jxNce-2 z@RpLWpcWK92IuWh7Ld*p(gGm~YC+*zkeaJ9OZPl3h?A{AFu6#7y_5UnWFgSzDu`uJ zHuiov?o#Mc3AHb&)y2~)H|Cj@aJdC&u82Ld;g(a_b*V19-1-zOtZ+XyJ@(ze?M-Jn z?2#Wt{UHcu+XZ^Zu^{{8N+mW1c9meVb7mG8Jp*6ZPYo+YGvKe-o+6K%P^)r1YAMok z@EdQ9+j+WyeEVG3?+^N>@;>}&QdsdksGx6kPVrk~;yy>sUv#yfCH+rvzin|pv(j`0 zeXDf8GbU~^-TxGKsl~l#rRfU#R_T6sOx$9+|0(XX7WdwjrYq=MrTe`xaf|7`dECC} z=hTbuD{gaZ8UfcOe*{AS`)S^wj8KlwU+wNXGm8k8DCtj^bF||xMH1khd$--aeY{B-K=jMZsKECp|tK-!u{cs z0Bk&=%`*3T5U8zd%|ZTwJ(R)B;jqtTFmnLo?C5>Kva_SQ8P4bkDEiR(d=|$X9_O=# zIGZCCiz8qf=iw}lIh@WHGMG7lNiVLx{Rb+m!uhj%3VF+Lekh#6`LoeDJ=PaIC!BDL zBxVA$!yb0%RJYrqBGn+C`0m1!45N+EIZ&tU zsZ`@|t0INlg`q2Gf@D#qAMR@Ed`WLD4tFa|CrWDr$yxdo2-d=93AJ%U_x4`|FaHBC z|Mi*D`-q*NDcz0OnVvdfxO%}^#{ zJYyOnwpqD5N3xd8N`Bt6c*gWv63dJ|$0jL#oeRv8tPZ5{N1r&A)P}Yz@O*mCbK%4( zvn&(6QS$Srl;E`#RrnSJxMeDQoiBO;2i`AFWe5ckvalp}E+lboo`fGb@>8i?4pB@6 z)$M%BIHRa!u)pJw-_yl}z9>&R-VE5Ac0Wd5`Po&M$)Y?xWWw3iQR(>ETb9*ILFb2+ zkx{4rO34F&vNAm@Cq|iadUX^rKUoWrr*gh#Y_wMtN>Rk|WXe%eEZLePT*v$(EM<*T z&%m~sbG^V^U@?cc8Vx)AfNB+P6c|1K9PC{k{M;z;_)4-ZeeC@2BIjzdot&ugcujYE zIUm8kP<8Y%D!TJ1hQ;3^iT zyVFd4l1h!O$`7G7IV~CRW-^>5UGVO2j9U?8;0j=m+*fZiF|Q{jAf*0oiT>35TKekp zeiqlM4RAHtNxb#U!(b(v)Dw1=FD=`?-o%u&O3y9yJiQcL7WJO63o$nMnwuB*EbnLC zos#ZOb``Pl^VCsI8XtC(Z^EdIPbg%3m6Z2Y`W)KWfZ;PW50oNlz-?iYx{~D;HYIzP zCp&(5Kg--S*=5#?3~Si5quH@JixW~_R-h0Il_$B3g{CbpvK~(7#9tWP`Y7rcNgq%D z$8;}xgYN{s`V)LIKT2(XzC}29k3@$9XZ|({z8&#tK+!MKc%U#Qt{<^+6)n>^U~Q*= z%W1&7teNz0sf`~-F*Tq51>DF2%LP zyXrJcZ$gxAEbK3$-2+|y$)2N$c2CV<9iraJ<+9mhrL0=d9`+MmyE=9S%6;Xj6JXfV z>(EYn{d^5kwewFuQ5g;GNkZjhFDnq0{*-$27wq_8&|%G^9Bxt{c{4w)0W9vS!}_(G zx=OC);9l()wMDcyB+zo~!)Lb53D9}k0Ba*SX=@Ka_3EeUv$HepU!X8^zC8SA3fEjo zXWB=FSMALA&k29a^S_?z|JC_l&b%g%I4kM6*Jt{E>NYxJH*L|Z3NuR2mH+xI%jZp9 zWjjA#oD*#vZO9n)A~c%6L`1Tmord%{^gLCcVR=ly#i-Zug1@3>PdtJKJ}B^0L! zX}oNT@C8e&nozjMwg%*pBGq_0k2MYC zCy;I4$4rPC#Qha<>FItA4y&S6y(6V{xC>;PK=LHL&HWhNqS|0J$FeEYCxvC5;tUe5 z4(r!C#aiAe{zmB+JH=YggRdK3Qr|KK8_D#Zq9-T=<;&N0j`M5bl-~@e8magGjpXoh zDu>l!1Y7Yn^x=+$nfk`9)%Z~J1a&PTPlFid^2#e>Grhz?*wyfrQG9qBFDs4TrfIAx zqUF|F`jE!j`o^tUd`Lr3x5Xe0!(3kZ-$EL~9-Bspkb3kAY5e#7`iA7(C%m7?>(+{k zv+NAh5i|+*E_~>J>suafh46~EJX{78y$9W8G@&a!J@p+;f7b-k9ptnVhNEJd)yLi- zWZnA*Vbx9HZ6xouil?F6qv4wBS4-fUR_yw-b(`HY@uUjK7w@IyRc6+9PqB_AW$83X z$0tX=29Qdh4tXujUivCiJ#O+RqT=tN`#ul&U(p=!c|aO$;pl@+N0I45 z*prjmGQBHBPTFdEKMP?}Ev8{|jtY|+E)A1&RG3tQX_%ayFxkK8$dUehIzFUTS~fG+ z9l!bhpk-C670RpQ&f0i6Vr-6nr+k)7X_Ma#tWKvqSSfR4<8Hbd+nA^h2Dbh?=Q1>} z)*mwxkuEMp#@qqZW#@Q)Pdd#b=@8KwZq91N3B5&C!{6okzkJ8X5s0AfQt2edz z+ZraFY{K-0c3tKS)NyPs3F~-+i*{tOZ36`w|n4dSxZ|*8g!#T+_q;hq&5v(0$V3 zdDw$D`MaeW__6z?*Bf^6)99|H$kV-r-Fq@EX=;;lK(01rv*By78V~DtAo(KXxqtgf zJmEg{mUc!Zd={5&62`*{VXE?iB)&C6eliXQxDD9g;hB67MfM**ftNp7KA$4i(6e~s zVL6i%ktCZD<~|8GTc_qe8Fz2h%^qZx=ya2kGHJFDYXhtnpl-`zo4y>YQvErPPL*e1 zHlaUAHf9r{{lGXzf@Bk%%wO`<(}*&sEFwP$a@CM)o}f-U6cHW*p=JmulwWqaYi#76 z=l{dno50sql>h&y=iZ#VBq^i`NlSp1rR9)PD0`AZX<22bBD;WqfPiu!m#Pr%C4#Jm zO+0CL&mV@6R)5%W|#X@Bg3IE9cHUGxN+dpV_x( z1VJHsH_4mW&i!KL)ftJq@N#GuqVIz%#rH86B=h6v{QSWA`EBecIp>C0pmRS_4yxJn z9m`0n(W4VU(PbRd)~ZcYty8MYy9*TWtC@^_JLLl(MyA4XCqI0aD2FZs)97hrfTpWk2RQ>{+T7*c^ACwde2l>w;pD6M(DY+;5eQG8?;i~fs zVy8qZ{@MJ#N|{1mEMZUf?ZA?gH98iSM4)_#j>oJpZroDwRB-g?WaW)7#r#7n?PfN0 z_Hl+P%+ao5$8D`@GYx=gyU(z>v`y6tMuyEQ-ap{~n##L_?p{In6bTyC{<OQxx*!UUDlg92%p>m*(@QP|q;V9nNiPmyjc4)6U^a$*oijmgZQiR}6 z$zza)*51%Nu@JrvBQr{wKsEX(9?C;sdU=X-FHP#+?nl=lquola47~`mmhcVuSm<1$ zDiMk^=1NzaiU_*eKJG9OZql8>Zn5_|IMiN@$7btk6y`t?X3_r;JUWFyt;O&&T5el} zjBT}-@ZMet>a+3Qzd1}QWm9`mpQGW~GUfGT^Ts&{k(>GD=8cM12lCy#k%zxz%x0LU zB0)!Sna-8us0v?k^G?OPlJMa$OeURZew{>y-{3Q;;7mT3O(J;ANu!*Oz)e|8;Wx#J zW^Aq<`O+dMJNt{=IgiS6mGb3Uy_;iO?(VtT!Mj()Cz{8oD(LR(sPuV@H^(=Bd+@JQ zCOi(tb$b3$W1~J^quBkRoyC7UX8kE&_rd;#^7Z97U)Kkp{~}-i9_MR__kJeCX})d% z;s27aigzXX@*ne6@vbENf03^torSF;TzKq&=7T5#xJhNSv(zL{T9Q-dWAdb{kK3AK z@%5e?BqzkM*TDv&(ag0 z>e}X2Va4AGCwPl@qN(nj-|PLH^u3v-XR|atTBMT(x2ZgQ6;37IK5GFBjR$kh=Rc9N zb{aKuL{xJM>9O4rXV+vF8b8kkcq^OGHu4Dg0g{V#cT4vP%=PpBKcqtM_3xt$nuxP+cUJP+rAjjRRKI+AZ|0 zNgpr$!B67IjSRl}TG|vSZmHq-BRSjJ0KUw$idAT$BrSq7i*NR=IU!e6J#*Uc=xv-EvHweDLBpByxPd5In&;tR|NrFSV7tu3BFHjlae^p`GnF_rj9Y{Ejk9DzrvE}49Mb`XNDp;7G*S>S=jV^qdOb-4&nXbWiZXH6|_?OdE-u0 z)(TFdU`)DJ@b7V9ZLQ$H47Uf-6SR`_~o!8av=xM;V zLOHsFo~OVhwv3xw2GLZ$&=TE+(973WQp?eu6Fk{mzRNhGwl+Su{4sC#HJ*ZSPSgqr z6HR++O?JVIAa3HRbtf&{>d$XHf>SbWtV#}hw@_4bs;gA_jYo67S`%qUL^Y?nMc|I& z+FV&3s{Z$j&!`sa znx12Jmq^{BaICXqnjp3!5x@BQ787H&H!6eDU7#tw{#;nQuobsFTQPe-+}2lEi&W{# zGgkW~ntPC#@T=Pag6z$&KTJVI`_Lg?G_xO7b-zd=MuD$5_FLgB&s=uKT?9y_ezzn4HVe;S*lW|okpKFCAt_611;D} zJuKc%2wVE^!fN3+4SQ&u2baLwAO1bkMW4pl_fbFWMQGZ9S(5z*byI6GS8$6uo{Ew5l-eOh^`Q) z=fblUp0ziu?Jd?fu`zl$7jLwZ|4=KLfx((>d~ZPcw>x3m2+JPloA~22M_e%Ko5G|t ztnYvZ4bB){DNzPBk>xnYKNHRz3J}mSV8ZJ<-`yy`3jcK_!*>b0T6Sew+!wQ!FP44X zrbbS6#hux27e?Ii48l!Fg-iQ=ez<0~fPP9j=~Z|o?@!`b2>(JqQvb7Y=cJh0Oz}Ge zX{Wk0zGZ1F)Hf4C#;r6gvki{ywzqq!VS#$61IVgVX=aUin&^Ki(fu?i)p1c-ef-84 z8TUv#x`a((ta#je8kW4#mM~PE)|$@C{H~((4=d(@?XlY1oPNl0H2U4>Zi=O=wvE5w zBWcbT;##`~G)F`=r|`#kZS6k}7P*MpYHyQ>_HuXyqdU!Kokq;R=6G!@UN_@BB5FgQ z@V8`fg|}w~z_mmZ{zS}eS}|tz#1Ho(n@Zum_;36z$8?dH9xbM#np5}?ob326@80Nn_*rT z`-sX;5)hH|nD zsyT)CD{4E}I#Jbvjv>T((&im;g3FU$^dFuNy`zb2PF?Jk2pIl`OhtuPE55^PshIA< zI#0e_g==&*pZ3al*NQ2XWf_~Zs_z5f$&?#SlvKDekM+@J+y0^C&9*l0#M82Jv?)?d zLuU87-?7w7x9!c&^UzJ$*m(BCwTRI+o@bfCo{o$2MEe}#(b}OM%-6!Lablj}juGQb`{Z_C_XQEzncFRq zpTsZron+TISnEdJ$e>Ef-$sk{R#bIF^>VshhZflP0+eZH4SI(5swU@(nFWzr_V7K5$>VM5f(ccK_D1Y6PPF%BJM+^cENl!-x3DYWXn7zIGO^JW+r3UrxVxA6}4c7PVV`2Cbz}3&7gf( zv}yYeZo4Q(;s|$kQS0ujXsNzV>C1aBd}qMdy;tdnd*C6~qS}KKt9hbC>(Oo4YY5R0 zrDP7|_lhjb&|9mcUcxv>H&W~8#II~Rj8$eNyZ%_t>#{Z6sX@3e!ION@n0kHOS2rwm zWrjD8rWRRFZJ)|5#Hl3|C>Hg|h~AM5_p^|~QSZ;`+_3)QZeJs#`kMfuyFMp#oFv}# z&TbRMbH(hrIZn^0=@@#L0+YOJ93Dm_KFhg>=(iXiY1W|rnmD(FN6A{7NRt}mpuLwH(1bCCQGVEfi_?FC-?tOLW^m!( zNoZzpGzll291=yp|4#gv!TDK{(98gha-$?fWfG(OP=2xG;`LfyoQ5238WFnFtENzb z+?Rv<){d8~x{4jo6S}=Ertr>z%G4@WQVaEMR8sx=*8FfFOSb4363c5;`1Ja0#OCOw z=)gI|+)xU65*FI1Rbf`Ui2@e>l+FY~hX|h+;Q~8-p$Nc8}ja-I7B?r%KFj(8ELT{4J%E-djuMXc`>V;%%#PRA_kUR8DgGWF)RZ zsE&wgPBtM3*-V;SUr2!37m=`)H`kG;tzVmU<>+W>AEloud)ttg#APq&fQ$_YxScJcXz-aKJ^7|DL z9)e&) zo{@W^R*F*6Fxb@JAO0t;gi{h{M%QpNjr$%6cIQxXbOU*IHM$)~+V~c#P}25v&hPX_ zh-;R~^?u~)4r^(nrZbc`#^)qm)2M|V9U68um@Q*Vipz3(?XXH%743ln9y^MCJ=!=_ zW!EE%s$tZ9l_*!C^x~om!d{s5q3Y)~Pg&dKkkoydNCxCVF^;&Y>J zyW420;mSRMJ8fF#yr)>W!=`0xl3QK4ZxOC!hse_A8|~1!Mw=MixMP~(vyRDgAj}4| z#2u&G>O#_8+Okz{EiwED<6Ebdsi&@f#V*XTupo2X6_=E=Rvn^sgo?HP(H-OyV>uOW zboxRMPt$=r$s@gcA-(0g$W!aKwz5!POl#%W?j%%Qq2b%W`}@Ye*e0(4{UVq~lvWzRJ5 z*b8PBxHR3=ycHX$D^IZ-7IxM-ifT^bt6*d27V;<2-MSa^vmb&MV@DGq{q3CJW_thT zsC;$B@G$44k$Hb8UQ*b3o?B?auyq0Cj89$+2)Hc?FerN1IE{|@Va|7}0(mLmv;$?1UFtgDdDV`F&Qp!z$&M{x)6070bk09-pB<*HE zPH5GHW(H_Fhpbfg5uc9A^ZX*5q;$JX@lEh|`k&`oI?u(uNH6sha&p2NcWACbbPtZn z8aG)Mo`KkL+qbHBGVyJg@*3O|Yi<=dJdX&R7R^um){x)&l zhewN)h-tnFX^ZydWDVNJu^W1`c6V*Z!gP`w>hW62<0|9lsP~eVdMm)iLYloz_LlsV z^{I3|Iw5}&eHXbYE7RSJmCi!=Y5j0ZQa?zyN{^DE6t^)-mbh;JtH5d zCi!Rv0ED(mLNkN%)E`zD>PcEAR!#hv0YA)RFN6PN^Vp8WuNhn%(-N8)9IZ2X{Z#QK^%GXM&7OrJ2o@_%7*9x@XwE<;=z*@hv;WGR{M`6h9G(n zGY!>v;2At}+vVpta_!a9g!LC#8$G#{eI-qqsOFSQnY)=Cw=9=U(y~}VGHKfTKZGS)PZM4C3YJaKDD zTDGtTl(#E$p_96+u0y*dUrJd$rm>ZKTY@!Bjdd88O#ZtO4S@oxfl}Q2z)XK`eKu}I zkAFQ;$r`WwWlMv8p>ZT&@%}io*ZK^MjBfQ4T`0opBD93xkqBwux`&G*iDUf!Udw99 z{@#RzNG|YvB8{iuxEEvYuD-5r;!J)%fcCi;d2B3V2IPPy8MXE&sX1>D-hr^mwzT!l z+a>b4gq^K<`ikJEc&klk5k1_J;bCJ9LMQFZ&*Y>`9TC->;_QtvXfqvSTPTkiU)ade zottoaAcF870p{ADR@*<22u{Sj_vl~WF?s4>OY4xZ zV{Kppb|1@O$9*@;SHptxpz{cu5N?ogwdwJkWP8m?{Q7qMxcDQLZ>A$?9&{@!R#i7 zD2cbTOf&K0yxVI*2l8CdrUa<(pgeG|W~`elTYpp8vNwZ8ZRiv3N)aTFdX-1^j4zLe z*y$hD8Q!c=@vxWSa?j1BdaO^4oAq_o(AJ3gR0_%^tc;|6kb84%8q22Y6bhx{%oAM~L1br0YRd%rJalq8B{MifoLdCIG4y2(4NM#F3 z^_{F}8fK<>$5D4y%#9Bu%!q#G*yt_ah)>T5rt$sWw~MTP_*(=Dp8&1KdblSo_VdEn z&IgEUPVQZ2wNpDH=eCk_mXovo9tk$Augi2@wH(lmpzuj$gLAT&kZ$d9(k11p;Atea zNPO3c`R-zT^^F_qIHUGRGpf*-os;?nN&PD|exjOFxEu+HbzQ~!cV)@($`Zpw_g|l8 zY-Ac<$&_B&gAf@OBLm9+RWTUew+SGIyH$63WXuZ>tS#Eyt>m%2pju3uQuH)=)E`w; z_2Z*RFu1kZO&Npf&eD3P%L4b9J3}PwmKg+}8>=CHOmWMetA9>k%y!7gS=i@=ber*B zN?cW2-z~UGLk0f5qE;u;eF5O2n!Pb62iJ>&%PNwn=42H~b@-!hEVb15v^7*K;PgHXX z|KQl0Jb(1lVxVmD8-2OJuPCsMF+?>dtG+y}sPt^xR#>c*rX%|P-~v#xAsqJ4Q6g*RBONI6aE#(r1B{do)+P6 z5L!#_S^zViU|8YpVnErXAOl=F+q~yU=T5?G=ydSTotCvyR~| zdxiId`DMe>=JbiJqgW3ABzlXK)s2ebvsgP?Na}})*{Pw3o`Ytt?*2@mCzGmn|1VBA z{jy%E>Dw}SU{+G6e^=?Y1y51UDO{667d~&L$ohU;&tzWn z5PrvuA5>Q9h`gP_eMWk?MW7^yTMjNJ=INF%jy|FqljdEEM}*#cR7RIx1AYuos}KV2 z)I}Fj9Qzu)*_57(N%bYTcH>RSBsnzj+XWuzJEtNTIW8MTPi^F`#{YAU|11dcBL>>+ zidC@K!VA$xHz-!E&Ix#ef4gvemjQQi8{64nd$FrKyb>0(+-#rp2_SH3$Bk}ZeEJ=ZgpWQb83mcRIc zA8jA>tlHAQ6~CM%na@v)8zY+_i|9~2f9Yj>+R6jdu#|ZOds-nH0Rfu{SNi;=pTpU< z$Y$YT40|m%p~2MAq8lA-;k%?w3j*O>3Q}WkuC(>gDZ0`osySI{>ufQdrz(PHcD zd#YTxb@tH5V>QOdUISN180W)%HU?amACoa1GIuIve^!Nk!BrfUNy)mG_#1?jYxU#| z+iwCW14ql|x(J)rg{s#_zVA(#+6FdODb)8+t}nfUJYagG-kjZj(Mr)LVH5q8IL%Lg zcjjh`W|@=URKHnUc`~JC%dF)&fY!z|o^@NJWUWuE_e}AxM3TPsi+5g}U)u~}R$-cKF`E%ih6gI=ja?YE_V7{4qEuOI^Y z*3FaZVU+{I79zX^4B?BEQwUp%@UjRmIl@*V{9S~XAvEUYQanQ`eg&ryQOzl2=$ork z1v`iTKFQe+A3^^CC(u()ARa)7CsMBvFnrZQySB3EsR7!Tu=dtRwMctOOIga-kHA}m zpZ)L?FInlHa9j5j{=4hLosjODJjkQ}Tvz(O)65 zcjGh^<)o>-?WIE7*eF*XR#$nj8X&4U*={LAGN@ zyo14{5WYtFa1%beh2qC~Y&-fXq;}~(h5HiadeSCzcynk^-w&@EuOL$%sVR(47_Pq# zXP%yYgCDy+>8`oZ4Rw&>QuHdN{DQxd_O^cYp&<7UeDW-y9Lmvad>3fQ5gw;B-Nh{K zRBi-@9oQ@v8dTYI(z}NI*i1!Kb8<74H{sdtP81cp>AH@9V{HL~Y6KY6k3iB6&R4ZW zZxCPoNVAr){zKLeVy$!p!y>!&jyEB9P^Y(6B~z#9y2KJkQ%h0AY6~m#2dF1>$=LKW zErUuN^Nc@|wSAm^NpLIJIm<(Y8yab?9v?+Ces~YfuC;D@zJ8(NIa()^e*)R2l5x2% zcbzxvLXy$?;69lW9EeF2P~)SD?I36Hxyo6(Mtjx#>FLCOItElozFKMAU{p zmP(X?O=xbDa=@5((;SPn#KM#@h-ywww}LuxM6EG7UpBvfRMCNFr0R)Q# z!CR8}EhI*S|1fz`n0WG{&M;{|TI|gD;ljWs6gFjG>Xgn##?3oy>d;y$xQWBRh+8rz zZRmcOj}Wg!)0jFLRY-Rr$?YyOLRMv7bxO;iCcK5lW;u9frU;8_PKNNRQ&3ly#&b0F z>PW>CKijI^&2$#6ytcd6bNdQ2IEEqHzwppk=vj25Y>mxx5zbPC?~slWQOzlQ8{dsB za-P?nsOA*DBTrlAJgt*@64ji-f8&WU+zR>KD(81y`L)JNRCCHTMuq8%>LzG^ zF35TcvO+S5+R!I_S9}L@o@UF_yYeKeIfb0yQDehlySa_QoZmUA45FG-PKKddwQ}$w zwJi>J=5X;`%>8mN!7Fj%ata@nf${FioG+QXjkoLU+8n~^FTN!DCygn8NtAi}L)dG+ zi$vv&cngZR7{{B~TgJ(8P@ajH)&lc^|B}L&$NAT~xz?OMsB)#Z4TI<_$yo}Gf#h}+ zZagTix6U^JJ?o=;<1;`sMI8mx*;?&ih~L08wzd{qOoM#MUE=r=mWGS?RJ{ZQ*9HjyFc{n7A=&^^GM0AN2R$->uK-O$St`ddEnH#)4cXbFe0;6&Os9 zh-yxuPf1|WZ0npiTGgOd0g(|=&8e|X&V7O0t8y3BoWg<%)WV#nt>vjzoS2xRb+fs|sQS@kzbC zupyq@x)-Ac&c8CvOevaQb)#UIW15w4BgnDRsf{O9aMb7ak1JCa@9NIQ`rcjM6IZ$Y zW9j|a40cpM`ljfsq^-JqJqWFS^hGQ^rHJ7{wcC`c+phI;96k3;o#q&n5B;^bdu!3! zPhm}XXq|VboEXkSEDBIBcNuiP;aKBe4A+PJzlk8>KXt0(ulk`=rpLjrw%GbCzkUoo z4nyz{>8DsfRzE9(?bKySPfai^cS}vUm844g5ba$ZFFxDHe2#N`j&}@p$QZ~W$AEC+ z(aTvC$?{=%xLZF|*ywgp(*>^=h^lS5&I^w@UO8s{yBz1uDqi~<$=-g+TW5o0ExhV& zOZbl%cG1^W?z9d?s}RnIdvY>uV>|Fxm&f127cgj zPFdti9t#^f-=;!g1b4H617##n#V&B6-4@n{cUgO3xU}0{<0vP$&XFV{-Z`m=4nZbE zoH-{#YmUYs7w;8p;RgU43csT37WtnVwCd z53;sM*{t-aPl8vt1#-#G-L%>&v~jW9su*n(QFi)K+51*+ETRFG4%gxBFvKIS#$pk5 ze@;aC*B;#V{GLquzvK-;l?xUI@l)Ioea zZT@(w(}g~LZHfNl+IDytksyuj`6L@{J3y^XBh-kf<}_l0nf$eX>u^BALv6>F(3c`# zV+Xur^g_?%{0ExQPijKHqzV1*CiGXD&{vy2xjcI`p)YMh z|56irVMY`FP3TuQp}*CHzGU^u`Crn6{y-CYNAKi(wrxT`r3w9}CiI7!&`WDf9`AxC z^rM>4&uK!xtqFa#H7AdEnnT^lvtyzt)7l{;bLQ@6m*QUK9Fvo6z5CLf>TV$>Xgxq2JkrURq~zKKnPJ zU)zL!e-nCX-O2fH(u7`bLchHUy=A@0`7drlzpM%Up(gZ>*^~1>xC#BDCiDlJ&|BwB z&VQ>W^n;twZ)!sST@(6@^(T+_;3o8+G@-8+PR{3;CiJVD(C=+RpR&Q^{I_XBKcxx% zh9>j}o6t*rlgGPT6Z%O_=s#&f@0~k2|NWcLFK1ga1}xVV(_5xwmdJGfh}M4b%m z{iPS6rqMR$wTPF!rL({0=UQ-L*>fpNHacd)25R&;CvbRfkfq7ap#L6%OydlgY}{pa zuM^$|(Ju`v;>J!4*ptb`Pc@n7(quwZb7C!LPnOH3mBagR)S62rDxp$j;COg+l`yI6DwOu z=?*^(gZw35TU~Oi0NlMOkPKS~(2523mc;;)TOW%#WD5z}?a{k5?qWq>%!R91EU6h6 z!)NR&TfFvinm6_QM32yqy{XHbxp)4W+E4R7Ze_KrwRRw-4V?}328E{FQa@dVw!9)Z zC@F-t*4bWZyeF6GKO=>1ty5HUYV4A8e^BnW)+wqvxwXz+bDn-KPqx-6YC|8l*122G z(=X)7);dLP=woZ0rp;gxb6IxXP9)C(h%BIDu%kA9#Z!A^BHNN@sXVNdVzuSY%?ax$ zt~R1@U|@aaNXZ>h0f~bXB$Db_p|!nt3-reBxn%!R$+ikBsyW4ht=5)LtBa4--1ay@?XLR1 zjPT`QIS;6bD3(lr2FZB8lJR#iYArSP$VK&#qFPODgQ(^d2CODiYpraxtq4A#pgaJF zzJ!j}u%z|qJ#!qEi31NMR!2lNr?8Bt&Z$n2Sn`hc4hRB2x|IgS-umoZC8pStY{x1S z7LsOJaxBN%V?JXR`fbU%YD><87wKsNliA%>1^S5eJ{bcPO9zLZ4l*cF}w#q`R^pou{tvB{ZY7ihf~1T=JyPKbss?MU>LpK z$o>oX=RNDRjS{^J+H?%uIUPg454W?Sa~3V7|2RH$(7H*w80AhOoV98~@a=_N)>Ut% zBXMh4PVew3Y6qPo=Yi#FhYHXa`4#i2^&qV)spUS^+Dz`zT7%9pCS+ju(a-jq18)|j zVf;GOqcBQttq4KZ*Grdb>ZN0 zZvF9DxnniW${ssz%t2SXV{mN{(NI-a^$LD(v@RyM?#p@p^Ra}FQmB>RbNrbbCzcg) zBsrdj<3JZ4ec#O7u|VTdwXjHqQKy`edo< zeok3-dX~0<3293SzFS0fYpHB9L~DVMsgzS1lCr3Cpd%T-30N%|$FDHGHWNOzS$tTa z)v9c~N8)7rgr(yN8Y2TA?cW&Psrm3%Y7~wZtbRxX5MTFA)_T!afI}u*tG73@ogseF z+Qhbo8xWlQQG_SvEf}^$>p&mS&O>Zp7yGgk@sQI+t{u|Yc|DxD+~v|-gg&SSdNu*V3hk~R{!YSIF(!Kf1#dk#e^TL(vhaTWY=z&;g+E&1 zZ&9&XM+X}e=azo$aHxfd4oHPoccQO=$64^`{|JUcL|j$VI#7s+tLi&->MAz{c5UV; zxyrFZgi7%$*Y*2#Rv{?eL@k@5SJIawWqYMdhZCx{wWk-8#z}`&oncNP0uv%n3|^#t(}@@4DFMe zM+8fjH8ZD2AySM8Y{N#n1$93}`7c}q$2r6o-vxEI1eBFS$5SF#onlIc3TBC0C!Oal z4BblIjtQCpxZ&sRFwXCD*{E~q6cz0QNG(^8>ib9#0w|wW51%thbVr3&z&qFEcov#; z8f<{vlU`5pTw7Q2+Php^mxMoWO!&V4I=uP{GS)j^;kiR$0u)2CK6%5h=xv9){`_I3 zAeVf5>mmFDVeBkdZ9}5%tIg9lN8QM%8|&-RuJZI#3Z!lubWsjDm67|YFE0n6!Smfq5ZIFBkFOOM1Yy>5vMQZ1mpo6VK( zAi4%q5Q*A0aZ}IUKLrq<8?2ohCm5H#UEzHJylcm(9vU}vL^xh0V^%+hnz>xl0zaBT zJM-j3=2ZC)sme-&udv`)vCrv>>FaTzLE$j3H$RE9Xy=$ORC;PRATlyvOt{I5MOrS} zI$e0~`L2rZ@Odj`x`*mVUE1D4KeKBWiXQ!tcM<#x;fu=lLicgq(bmK?C$RHr6o;1- zn%}=07uTj$yd4O4G2v#ebwMY&hn|=2nQKLN&{(zx-KEC+C?v@Y_7wuw(BoVk5!IZ+ zHp&*aN>-TC2*;C2Hass;TEmx37W>h*_2vW)+jDKA+aLY3;$KlaQMti+<594;dhPgC zy^G*?srW@(!eD@cMT2HWoe}nJWON+POlh z8-h|ra68uzALbz1z@GAIhZvjO-RwEQ4c5qGCUE2oqe$%@bCgkMQ`>sldpgmhAvK&~ z0q5+32A*7Aw0NgrzD#-H#!9(M6t2lt`{Pt!+jR&wURVMQR}qq=QYZZf}U6Yv7l5b$;!~M13RU5C$Brg(9li?}1RemyexmaQfOL z%X);pmay@mlqxDs-X`KnTQt|tmA&@WlJ-z7GP>QBmaZjehnvCtV@XeSxt0m$bURIc z%u`gDyn?sNG_QQ8$y_y^=GPN$=wAAi5!IxjM`Thx8+t}2UF99xnHn>4s7woF(skaU z*W~&EnckJ@aG6S*Vmd;mwPiX|ro}RSP^KegI!dO?WjY$utPj(?_w;mkcdMu8Z`O~( zyJ$XYC_k-^9D|SS7MCh^a*In<*qNtW{dVTs=RkJkLxjq1blCBLXh)inXlFhn$Ksll z8K>*%bHd$pJ*}eYoRy~^j~V!6wNQ^_C4-i*1MeIw@3c@lgE4%ZnCqN{8AuMa;}e=0 zpe1^tm0UkmTVaz8dL=P(Kkg!`Ihr0myC)de9o_ApiPxGF*T!oq%Qq+;(P1Q#$6;nn z@lIUbCQC+? zSdSFdzCbMD5sa)`EvAyGNNX#**-5xC$Jgnux=DP8?kC&hxvvVmPs^%F@63v8Nz>V& z;*F<*ZWv=az9@l=qi6?i4rmNc3>N^4G8)weDwj7-U ze(jYDkDb2I%F_wd?;#6dtvU|f+XSG})vs}OrTZp2lPQt>SrD>ILdmVT zur6ky=wxYpl=wT#&WqjJuzaL`^n!{TNVw(dmT*n-HeemJrN&@38;|o=IC=a9<#7(? zn+5Y0@Q53One)PPwBTep`-Sn2vvQ8NjC0J&@LUSL$_T5Orz3VUPTWdm?iT1HqMDP< zXbO1e9q3iXuy^C;v19N z{pOL?j43wOXz^6aZc!exjyDVW2iJz+hDawNuM^cPwi1xS`AEvYjct)$*C0 zZK^Hn_0>>XZ7^R?YF!hrq8aoBLu)dKis|$zNuSU=e+}E8{fhX+dP&s#xX!$W$5R`p z(K1Fm(_vNn=ix-FgUXm{Yoml75lT0DUWo8+z!ieT`F1MvoegfZ~~gsUdRa8Ky7QEY~z`4%_8Ry0BQSq=KJDfI z5T}N@wLK{;ZH16JFn3S|xM&-0wASV1%AaLoeJ}j!PGu*;)Rw52bzy?Anh5(qz}zkK zzH;or#A5o2fX&0Xwf&v*44Ds<^XizwUOuMPTD#g%YurJvhHF5DaEJ(pim;~quEl2w z+jnXt(9kb&`$-1dqnt<>RL)8}wrwsG=}deZH^#i;=XRZOm?ek9;mXR$+W(PdK`gtL zbhISnkak;Rfl6igCAG?r)G@g+OLzDwretp9u$T%hfN(Pa#}~6b)#_v5+dXXlPyb zXk-D2pJ+2B)uY$#y@k(j!6#`4)p9bx3AdzRrFULbSPS7?epa30yWhCQDp*SvZy59w zw$aGiH!nI;33MBsYVq>E-K>@t(+wlymb{NR3a(L?F$5Ba5JM(wrwaD%)s;r)(9_o# z&Z3+WafP|+9@cYSX=RTRNmYWj(Av5a2^`)mB+|DJrHdM}_ieCn0kfW@yU$TvGtBZ_ zEV2V>OF}i3%{jP$JpbYXGGuyHd$)`cUemA3_jd16IDJQX5ks6n9_NN|^j(~)q%0zA z21Lhc4SRD8(~mp2ulqPHl&QUkxTyf#gwFv0uXH%>T|POM9n z(Waj@o$8G%1|hjFBwgL_oxh&PJ3QX)mUf{TmxdmmT9LJ6eKgu0ST1*QMJE$lbLF7+ z31aIk^&bvzj<2;u+={8zuT-5jsX798u7&l*GZMKpj1MK59vq2f6C9lOkW?L|VTCud z>+Cf*uk;+y9v=gl_gQyFWb7G`AbJrLoMe@*&T8#UN!Hr`S#fUVyq!sQ6`WlfI;u{x zR+f!Y$#`npNyfntzaCQQh zL6sJ-_3#x8i%})ydau0`l9O!uLB}1a!E;~Pc?wsMHd`nda&D2HguaMk@r}*<@--VeeR%r|W~xpfgB^5w_6chNrG|aBU{{ zyl9WA=n0F5tQt^JN3s9FhOn94t<=PJt)1?Q0N+#EtwN%LzaWA!6sBx84ZqSdHiCmeRnzw73lDFS1e(PoYWX#)dmAgJMzX!+JACX<3*!`Jt_UC1PD|4@Gfqf>Q*#B0t z=iGN5=YE*%`o#V}BD+4Z{Vdt_iS1X)u1{?LlI;4#_PfW~ACq05*!|^k_LePi--=J{ ze)u^1CFAV3$^KO4zp~X>`&Q%Z`^c_O%>T@B_OFk#|3db+GylE)W9>W3u20PGn6dT| zEgfeCytOoRmOSWV0ZK!c%AOX8($JN%zw}OA;7UU`$^NCx|AQFa%v2SmG@^CqECMFR zRwI6$1SnhK9#XNk2+NlFLn&v#U!>{316Otw;H5k7Q0&s%-O95sF=s@R-#qR zW4j9Tkcx3^w-WM9#@|ZBTbW&%KeWm~>R*{VG%vGT4(^iKl{rI4Wp>MvOH%uY))6v* z+A9Y3P%2=>z#h6S2~aVxhi*^pBbwwp1}D%Ms?#xy9VrrtS=A8z2}@J9PRM|y+p^eZ z-Rv`*h$p-#*JCf7`tPW%$?<1kV^YU~D;+Fcsff716&JW{q}U51 zk8TU%yi{W^G5--w1Z;(-HYE8u89a0s39fyGA~p0zVv1^#rgdoF!KO~1N}cXbOi8HO z3zAR=C#J;d&V(@ZN@7Z!mTaAbswbwz>DGiWG{Nby)Tyvd96k;;qV=g+42e!3_pv$W zNFCSO$-;mknyVVBKMkX6`zu;14BAFCTZ`u(fsG%2NdFgaM`^~yIAR)W2+?UwZQUC( z+C)Ew@vbAexlmVM5Y_QPU+U&EEyjx__;I{;)RctdD|5&3;;0S1q)m^+iv!1bUTkG~ zKY~~792en!Q;_W74^*>Gh+VGcjh$8l`46@1>> zJOxX8C0d_dS8S(Z!KV2ux*2ZwCig6caSGR0_|q`4ruQXzDu#90sEbQCk8IK2nTwB~bLAAal!7G)_&?{9!-~)p3Y&xT``?+XmhvUtAmr z_k=PynAmp4HivmegNRMJGv*Hgwk@{wbW#cJC^J;W#Xwv?2+Ea#>fmnarJ3T2Zf|Ghqw8wGVUaguZvWch|xxwr0ptjjnJPBF10u{OR6_735bbI(A14QQ_PPfx8$h0uIboYv=L_Lvj=sozl|fC5QE@v8`xn zp3}bsyVJ3!t4c?Zm}!;he1Z?SDG!lEpOOSRKq!k+=&C@t0TFaoD;?DdvARb|2iHrd zPGhh3RCqK(d@?!xcuv&iZML&Jo8mLuer}IsPE@w@)Ay)u&?nfL-|JPUf?@rb+|e~~ z3@NJA1}eyfsvw+Tt40^_DOu$wi_>;Ob^BiNyC~-OIq|csA@s!xeIWtEJ`$OQD|wd? zj^(wE0`Cn3`%`Xe7s=Zmn5xBzG)~(sI*=d*zi}o_Ua~cNU=nadRC6*GZD^_gWX|6~ zsXtN8$^5lXE0n{FiMVeXx}D9Crmf_igPD!Yj@l)0+HN3iD9yRik{GujpH>ntfkor2 zT!aTJ!cRjT5!IZ+xj+CWmny)eO6-OZ8fWK1y(~Y+*fqC-L5ihfCS5)R`Y@jqe@c?Raf+R;_IM{T~p< zpNTQPDFb&{W^0(OSq@nwqRzaYluxj{OjsVG-t165S@{a=eFKhKuokx;NhTPpE!Cx) z(E_uKS0#PnCmB@4O21g|VEvp^3TJ6@7w$s{AKQENTX0pi94^7;){>;lpA~?rxe%%5 z#CyQkcB9oT@~6VMUhXQ<@z~4o8pZ>eBS|&Tsdahk8GU ze;h7uJEBu*X+_kxdm zB*sZ(yIezCFktJWnN}c`6g&J9H4Nf#Q@C(gpt3;S)y!G81U{3S20c!&;9%QIpwH*% zO^q@~PP#EoS$DPU&$naDu6N{VDTBe;v{J4rSFmqLd+HXyeXDf^SNF8FWeYjtdxGul za)_GtsT$EbkwU6YwR`^aoZB7GOy_nd(sY;T^Mny@th3xKBfkDBgN}~VBrrdn!vu%*D*T0 zzF|&^BQR;BZ(^K?*Q3Jg&5ixLQdlyc(YFYa;gxZWzHJT@@%j#Q@cOQzGQ30(UPejq z`gJqBbf$#eq(rDn8WLPu<(Z!O<>>Xc8m_MJIqOO1SV~C zC&r0*Jtn+9{^BuzPw~olM&Bn$hF8Wh`hhu2#OsI9!RszXWq64oyo{3I_1hd?59JY& z@6n5Lh&+>b6_3%2bFM4H>vx71Hzv~Z^&?pL!-^)bZ*(_Chu4qINpS=wZS)h26Y=7r zUB|5N-M0V7Q@k>s(LDso@X9zw_nN~*yzYYzUiT|1!%GC=Wt0T3CvteTFU+-H@;%Be zkF*5!=3T{Ol=~g2>&ozY((w9B4zHiW!XH*NfqkO~Fgm<`W=@JDFlnO)F;2v5x$r8U z_}gPsyfU8A&k2&@m2r%IVGa}V`XzMmdPq?jULpuDqa=9!K8M$s@+IG+m*o&ykjG0r zMn9W#T^U|a8D5v?@LC27e^}83_KhCK=UcII1#U>h1aj!7JVeeE8`hG zN{|e%jAQg`bC`(NZ=i$MV~Wb~53U+{{Qz(9*oCc#Lu%BXwOF zUVk*axS5gG$KS%jA67JheWTxDba*{(PKqNiX`?4FPQ>dO;dMpfo9m@`Wjv!N36kNJ zag3Im!$iD(4;{RoQdEYQ2*S%K30{B7;dNOa5&0gyI)})Y@~+}BdQHxCWq3VncySvf z#p`KU_``}Ouy6DSj1I3qnv>!POxoxfj1%#CPI&D%`|%&8cx61JKM^FuE8`eFYYr3f zdJZ~x{aH~NULpuDqa=9!IfvIz@`%XyD7RJ867+E1RXj$ylajiw46nZ!UfkkH@%jra z{9#2C*f;tsMu*qm%t>(sCT;XQ#))|SRd}sCf8-A-UK!8m1%hOFWgMdy&0!*5FF^;d zmlc)aC4%rWN`lwla(F$TM?}6yx!sW>vhlXL-atG?xwnzJt_-i|4X>MWc>Ns~{;;A6 z>>Is;(c$%~IVq07q>Wy~I1#TGgxBFmJ@dm9uZ(B(IzckLGLF$3<}eYje?SMXHx-rP zC4%rWN`lvmIlRV{FZmw*d=8O)@_32ID0epgYrI}EyuOgb>z}ajhZRj=-{>uj4zGWi zli~>p1j+EqI7a_AhlzNtfDT^oDk{TE1mR_r1h2p6 z@H#S&hq|Mjyt438G=Y7iDG(f9zBwt5z@&`|5GLaF zs_^>eZ;#5_uZ(9@BuIu=#xZI!hlzLvinLfSDJsKD1mR_r1h3a}c#SDv@;&8Do;Z-#!#SxgaQ7guYc)cOK_WH`1Gt=^w z@r>FClHrwcjM~j%B3@IWgV!pG%J33Fco`+Z>mND1p2{O4-=mz+O%YjpQLgLN&n zSH>|~%^W7;)eRlIdK8u6C4%rWN`lv0IlQ*WBO>3UoUTpr+9&TS9;2KCOBK@LI>56h~mvM(bjnh}R0?wV-n6uT#7-p3!;)$?(cJMzhUfB3^T#gV*|s z%J33Fco`+Z>)mE}$@l1Ya)?}%$4fj$Iis1PxiY-GqSn8@m%}TBg+Hul0{ccAV03u( znUmrOOxkEJ#)){buGlf_rNTFFN%6{fMjH|&!z<$$%`=CIcx?n7yf#);hL;G!%P0w6 zK5-`Ng=5N>e2;Q+GezX9dA!79lyiQm>&oyd7+!bg@Iv7Vcqy8|zR@Na9bTK7li~@J3hrL;~8yEkPNSkW3+`iOvGzT=-{=LqB6Wh5MD+}@M_87b$1>S z`5xtjUW&*Qc~|im<(ypVx-z^1!;6yzY5T?fGVoF~fqf(PfnvN^*TzY41on*=X2|%cx61JtqGFhm2r%=F^7qGErbqU+bSxe2;R%B1L5O;@o&bJVx33 zPhD4rSDWF*u5^mmdtl)YE1JN*(JmMrUb~u;;s{LIXg7=#@oE=dr@qzlyA-dCXS6#( zGQ2X5(H`b75wAU=gV$b)%J33Fco`+ZYibU!fjlDeJ<2Y5ipVZ`SMeBSFFbW!8D6Ux zUhGV#c6_w#7g77j*f>%cluOst_$oD8a+$ka#GrZUhPVqVb z7XGlJ3G5pkh|%G7kU1%iz@&{1#yAnLPT{q2*J&LoUK!8m{RGMI$~Z=cn8QT84uuY0 zhbbz{-&>>l$rP`QXY@gWWO!v9qod4WB3?&B2d`rkmEk3V z@G?q*S5FSF<#|Npdz4+_6p?@DUBzRRz01^fWq3_Dyx4_I@%j)f{9#2C*f%;Bqr>Ys zb5b0IubOnnrOQT8rV*OlSbYk08>nc{UKEc{_b6WBL838TYn$ea{MVA4i4 zj1%!%LwK#_Z#F%}E8`iROppw(jAOLa946xRQRv|HF-2u~i6Fd;lHj#w4zD%0%k_Zr zJ<5(~ipb%4SMeBS?=p2=8D47{UhG1qczqle{;;A6>>Hhe(cyKfIVq07q>Y9#PQ+`b z@OtXCvolySp3!Lp$s96{QQaIS;?;l-UZ*Q6!%GC=Wt0T3SvkDU%OfJ+qwI*Lh+Lm{ z6^~K&E>qW);kCBm#V%xu*9a{9VMP2Av<$eb9BA;Uiz8?126U1l;+OqP^X?o~8c@ zmwqYl3eSOoi!87xqjM#IE>9-mbR=`O()hr}=boR|p)6c<9wF=|p$n6pZ8%@Ni}g>5 zqvgE_%6p?Ec|RwYwr}TiSH4Ht0Zr5PaNbosMwjPYSC;qdTiV%)O!Iye7XGlJ3G5qP zfYIgsh32F<0+Tkn2;;=O4~5IIbGQFmic7{bx|kqY-e(-6OUxn3`wXv7LkF)*6_w#7 zg77j*g4YH)yvDS}@;%B9Xo}a%dA!79l)cQ!t}`da z!EHdk(e->L;Jdz2l}6p?-NuHrGuUS{gLGQ3c&2VU$%rg(iG7XGlJ3G5r)jM3rs1#?myfk_+P zf^i~VJn&3?yz~plq~$B)8GVr;8D1I3=u74>5w9;p2d`TdmEk3V@G?q*7tcbcYg4D^ z5s~jvc0f}^F3P)#$0&Q5sq4z{+RX4`Co;wBE3oi~6-{8@=&KkVUSBgO#Sxga(QOzf z;N;;uf2_YuXaP^^8HGrZBaaPP}#SEaQ33gNAs5p;)UhZSLX z3~USb#w^+l5-wM#A$0C@70;jId0PleyXHFhg2XC>i;zINeCNT!Zk0O?RHr~5qoi{g z+o`>#`8RqWUK5>E`*wT5sX@;1+lW0VwF!>T0Vy9;&IQro3R>>T`8Q@|Q@>nu%o^8m zk~8`ae00V5yZjDpkn%eeeon8DDddScawi83BUQ(s62CV4IoK?f)HmzXH?3vGYBEUm zh2gDp3uUmy(;expNTWY2QuKYCrAT!rL`lou zrm^D>a8!BJTbu_e68`#?3Jyf$G{+xSj0K9VNb}#M$^A-1BdVUQctmY{{f=iOQ?_kIV4tjmM$bTcT#z zFWeI`hMs_fxnysN8h!Q-lhZ%agnn-m`ivbX=W}cm`khVa?K@4*XWu6D%bU=jZ9?B- z=gIk>(}ezqCiJ=QnVe6(3H^~K^clNM&gaY~^k0hEhMUlzYC_*}kIDI5*@XUP6Z-ypPR{4nCiE$LP4555CiJ_T(0lftoX;su=#Mm^ zFJ3Y^pNpE%9~n>Ay9j&Ht_-T}nz>f!i#njV>&58Y^6g@A&{yM%CHkRq3Sc*k z6~IP6qBSec+%POH`%x9U@)0XRDGjLGi9Pz! zxp?P!dzdXKh>ogAwWsd>REM|s;mQ4}il_U8TO)6GwO=!RKj0C8^9XMGFr&LkYtX~A z-SOP?>G-a2i?<7&x3O@()UVl%ud+SU(#}N|v{{A+@&+}^)cIEjY??d7ijD(`dJe2Q z&1rTYXVZJqzV)*llcbS$1Vk`_v;F^ zOBnFgc2JfTp>X^Kh4XC2=CuvvtB z8lNpEIi@bK;LOowqzC#F(4(Kjv9*5@-Wf#w0t-(XMZdIY22@ZS8s(^+5@d?AU?P0V z(L;EOE`?R45G}(@_=owK_uX_5#=*Gc%5R3LW$y|`0LjG`%_%WQK z-|`u>Tl5BdLRMu?{Pb8famBuUxvF4g=|*I!E__?1Mx9ecfG2x8gRag}E(1Hu{hKpM zL6fp_u(J|y5&L&C>3;V64GZNNRLjQK^ArJ(dIYCgsH0>BxT6&fRcZ`<9#lJvOkuc1 z8wJ&22&2adHpg9pE)Bd#uEAhdUXqxmi$@wpQ5ic^B$bf4!90~bB1Qe&QIB&eW-zF> z1=*}UveME~F?H$|q<^mbho?(G)Ka8D(c^wLH0Q?#;l=Pg2r#OOpA?11+5+{$q)P4J zkMug5TBm8{72R3ERgxB?PEhUFj{Kv#$omlg@~J=3>-;a_|CjtrQ2mMiPyTl$TeicJ z`0quR)cMf2qpZf|@d={uS-2D;3L*2AatBChDYODW%d8uXE?`_Ib(uVd*wsmgu#(-~ zzI+xxNy+`@Bqg_pDL!3R+#k&J_W!68b)r7b)cSMJaW2s8$o_=Tv4S#PR)vX`}L;kHAD#gkwWE$371m)=6ORs9}I zt6y@ZP>CwFEmtHsJ%!V}Ctae@9>uZJxV7o()AKfIAGguAE=}kauyQd6$B+8fE7#Qq z`n9LwR96jg%MrMLAz7FRSEX#)!!-4$<-s4;PfZh``X4aiSN+ei#J8iRLquMo*Z~g% zy1D31A$7O1oT^haF$E+@Uo5IdSU&p1a*iQWzABtrc; zw%oCh@lSYC8%(pP>}+lIMCZ-GxLfHi)}FgaX)$<`;0|WAm>LfVD7i()X38ppsvfS zEvDf;)rCe8AN>0M58=K~eoF_K8z1guc`A9L&uL^)aB9?sC0Fz!Qp8}U_UI*uya+ZZ zh=%q>WsX*s#Gnw}00(56LSw4+h0XD?GyDsBd`xAUL7N`8Yb)C{**H*zZCGeej=2HS z?|P`Q;=%mdqJ5bdg;8Svdu-p*xfh~W9DzBu{PO@}^r|SS`)kfUwZA^W{~MxQ=+Y4} zF6QE`v5NkILrlI|fbW#XF|`=I>G&M#XqIWQUzGFey+SAdKb^N!{^_*p6}@AW)c?Pohg^DBh;m#U`Hh+UK=JO_uwKQ$OyZj2tX__nFH+*JLMiOixC<^o zCR@=6sXGs!D8M>?clhg7k~IK7^7zuyCe6Z7ozM*LB@Nm?Eg(( z)%E$|_wf}4B%s=_b|P>iodHvX(6(G-u2|8^v&+?R1vYv?RkbQ-JUj7+#V_AJ_jt3n z_T-&)x$xV>;PeJKx&G|rG8vzf8Qe1qVf03&`bqt9Wl6H?);4X3dE1QJ^>tC5n?2;_ zjIoEOm87%jl^ivxR~DEtqv;s|*XJel9sckOih!XgafxKg2Z&4Na2Qr~d2rs;hO~Q! z;&&Ir(OCW(KLa~MbE}#sVx4gTV*}}U_QQLyO3A#F{rA{LS@)fCV=z4?J(ZDc-Imp} zpFpBKxi+O_+9!7$h6K0coM zKv{hudO|+XcVj-jiNzI%@P3SQtcUT#pJHCVka$p&fNlfj;ea*_9Lo+`T9Ly2;Srs| z$0nSKqfwUQ+gpAEpVEMqYXdjeiA6e^(D7P1LvO?u$a>`Z(_*$4{uAMk%6v*CTH-Ts zK4+&syTe-MQ<~9YC>Z-%O1m7MU1RaQk1+AvhnEz+b8qw(qVr*j<~|QT>q&b0S1a_U zY{(hATygHQV&A8=`g8ENJNeno4$2*{By-Z5+-6h1a#oYu{YqU`6wOIxYI z--(Rj1AP4GcD{@&(T`0C9)M>5Z#$YB#+90CCAsiXe7S0*htX`p#p`H6R3QPb29>4< zQ8lv}=u{q8Mn6Vym)(@nIC$B4u{y%q^Mg)~lIh&w8ELSj7P#ihmh`R4PUFdt=ykkS z`!$1fmDY%_i5Qr&7{6Q!e~z`=-m7!ySM#XJ`hBjoY%ylEjmT;LU3gbem{c(8YdHy}) z`Lgj;@kn^eTsLx3Zj7hG_`~`pJbUN&cvr!HZ!q@5y9s8Rbe+)_$=s97g`pp;?VZ>i zK8z1=l1FpxS!CJFdu$?g+qziK7+;HML8~@t<|QV~G?FFf163nHRGs(diR#DvgkktA zMM(N>jD4`TZX;Qj3p`D2mq_kHx;9yKYm+TWuo$;hyc&Mrt8!#gy9Z{A$lFTcA6k6f{;@(?#&0iWZi}%+jZ^y3~Er?GO!U}$D(J^I+$Hm36DTXS5s)Kiw*@E>rYWM zm~koZvU5qV82a>`W?1Nct@d`UC{U-Ft-*L7rOUh|6KL?$5|w1V`4GtHtKd z`$?{Xf7-ur3%8Jlq2-zrY6koQVeFAdl@a}Qc6t@HpPF~i!oI(yNg*e}(hq+__;}6M z`9GkEf5X!cA4~j`hhnO83~%dK^LFC(4z%!s+0~_C8Lb1irps1Pt9nK0PjG502XrYr zP*U2LYYs<8!rHc_DOrDj>J1Edtx8i8$3g%5IClJRYpJGnQxNrWsU zfuIBwY{McVXc7d%A}T5(D&hu$1_Xh|dxHy2M?{ukbAS=tPzj>rxS=A5EF!o8gNg_P zIsq3HL1%Py6qiwazu&3eZV1lz-uKT7-0rGVr%s(Zb?Ve!O}TNp^(v+TnKN_d8I*ZM zCfjj$cH#mM^X|M~fHTWLQPX&^Hy;Bf>j`O0eC-#@o22gFNsI63MAgky8RRJ zr0XKWQ-ev6uJfZ1L*Y^WRA$tjn8^QX@@*aj>Vzw?=!r&=7Zl=~3BLfOMwEut7>XR4 zbXNH3Rt~^asxN>Fm6m!|-0=hr-^JCg0bRq&1xEQNAehL53*9UNWS%p#Y&5{pV4f4U zBOie^?b9GYUMD4eHmvtI0ko_ekkhRKIO8Y7_E?2*RnVHIOG_O9lVZ^fq~}{jKz45% zyGsTY(FO(;5vwWCaC0MP-*EY4Gy2jL$LO^c!`lYn=6JXNNX-}FLP{KQb?1u=>v#mo z$R;mR#%>NSd6ZdPj%_gMiK`kZC_u%Ke=J`V%~9hUxl@ze;tovoQZP}Hf{~Dp9W0aX zQ{x(4Td9?PsOF$_?Pu+j6H_=Zit#}VJvsHKn_nS;HmZ2gD9~q}0IuA!!FP?7zOhpa zSSwuK<~m_J3bc=z^fv}Pp8aj;js)v3t!N^mVqkNYQc<`;E`yFhr`Hf zh?iM52x#F#minBn3av$jteDAGk{&@>@@9Ye!c4aG)Z-T-?QGp9RowK~ifQ2kzFvVY zl_8@l(4*$K!Qj`BLBN(GgRHx4WIyeO=T)DPeGu%zkw0DPk=*H6KZh9Kih&Z*o&+|8 z7TT=oHNR43obQg`1ddx2C9t79HknWf0%BgrrrMu@FcdPzXJB;mUtrF#aCC%n#d(le z`5g$3SU5Tni@!&iZ-YyP&{!T7(i$9OMy)n@6;_@EjA(&%GCURzOQ=s*NslBo48GW< za`(@;#Z2gvN+@2bIgmn={XlM=ZmdlP@oUMqD-l-Y?KpTH+La_#*D?_>feRP>vJSfd zdg=fvHN*S{WO1D#%w-Ip45V_M2}6l%JuTc=*H%r*ie1O}0pH{KC(tLte&(hgX~LBF z6$AI1wQaSEDrMuT|D0#wA8=C(2I?jU`Klv=nx+~0#qt#`!x$iChp!VTJIUz>*KPSx z+U>!F$pC49%wS03VgCe7y}(I8!S0k;7v6O6CD8OMxdVo+e_Ycn<4H*DV^3_<{)1ps1?kJ%u@Sv z%o$qS&g%0@)R9|3<2s~mOqdKoKyDoYz-RxE&IITWZsAmH;QRFCU$Tg$YlfTuA!Cvw z-DZH1IE^?WA|(J<7Q|6zLxC+{7H*#@s%=BE*MkpyURZr%eLVn8nq;rx=d^w8u-i{_VXpocvm`Eb3uo|!R@_(+KHn~&59?8eH@gP%wO z7fL{q7((fEK~hF!ev7u1Y*mnOi&HDvRYa;SZmx{LWp*Z_Q82o|LPL)`5G8NL*yBhR zt609GH_#0VD>FY!jkhs}cNq|HfoXDgLdh{h2HJ&JI0oY28*1!77G2?_(uaj=_^d8? zxCJ8#NdF_NOeRS3RY1NXRzC@axmWr~1uBICMLMeDMI>Gw#~(6wYOKv!thSG0&Xwhy zefypgyF1YJ9b1;}X+aL~?<8_<)5**NzI*2#58g(&UJ0EdJ zh#aXr6N&^hITbHnXr`Mgc){z^}9wbZ2)WYL)}Ul^)jp zV$_4H039j%=T<%{0=^!K9s!JqfmdQ4)IBCW7!QJ;Ea(}RJDXJ_Oue{Fk)g#e$T>{R_KLR3`}^7 zbo@rVOn4(zy+DGJ7TUaZwyZ3*&W8A6>~*szvxmNjA>W7}&HQ;?f3_QuL=KW+qg;%) zYqui7J+TQLT?;1)UDcH{olfc3IT>`uKS#&K$)eM2iKZUbK8+s3Izl@dfg1%zeT2?u z(>D;a-*h;~37#WBu!3+xxE{pxtX|;U>MiVZEX>rrv2q9Ss*)G5jFn@d@JH?f5hqot~eCR108Gwf-ft* z@QSn99pnrg`$6F!cL31uy~ANF)v%xLGWH)Z_BD?Oee8LX>JD#9<)!^bXo1;E{IbK! z9gncL-eV9hY4s?33%g0 z)xe9n<+}v$&{-^g>*^N79jD^9MQFun_zkNsxPi60hUtlvT4Z+}08f5qnQD(+$#rc6 zV0w$}N}6m8Lphv6Z-A7fMn|l3gi`yrqB3oO`KNOMXoL58e0y^~QH|!!56^#A4-SyhHV7< zfpL{Kth6MKnOwE`Os$rKLVG}@?+bT3xC_a>s(ajHUIKRlG!&EUZjn#S4fmiegC3)4 zW#ggJj7WZ~1z)}cdPS^E`n(ZYaL$aj$hhKcPuW?1Y^C_4J&v|0>DmTw*_wtg6{=kc z2KMbqWRx`W1qQDujg&Si<#xVE!KsfnQTg4f>bg5A%xaHKAS@UvGR%vZoqSb&^QD%H ztrZ((79dNB^MP$$fCp@?VB2D~#1@-Fp*I-x#?B#X|LG z61or%td(nbn4(35QnklGmBS{?HBxKTmVi{Vlx|UXbg=1WQ2>W-O!iAZzP}-#N;ljZ zB#lr%Lp5~*YpU9%L557`=r--IVWpD<39vtm@rV4W7E7V%A}s47@Mu^A@O10Z2yC(w z6CQ+n)eN-Mdf^1IgTBv8Mp(VY2i0M;l7!1#PW;oyPsvVfI|K8a ztTee3F!i^`;nTZn-e3BIrgp_0-aJ8`*tT1QcPVtyjc!s+&8A4SG^8Qu6b%GFnPq26 zx@^3n(u6g$U~$y3sfYj+So&_N-bKB*(e`hk%f+bXX(aF4fF7WEEBHWL#B5=wBX@qF zBF4J1ixGu!jS1YHP(i(dmG-ukc6vuw?E3Uh2J&spmu}{=%udH6RGO?wNu{aEGE@5l z;!Q-nLhLD4`V1X*_LD7{{&V-&bd+-*~VRfH~)0A z3(}P-NHsyxHUM(kQGl}aAO8~l8yEc&hKOpr0PfQidl^GTqaE?;nXX%%DL?B>d7dH9 zZg`q!;i1YD(Bz?@b)^0Xk@Pgp}1-dPj|0_0tmv#ke$2Nfl$dSjF4O@z+4F$@ zL`)rwixe8}b_ZG4RFhbq4A^Er02-5!hLKvyj>j>Q>KY_9^#3X;6a{mn$O_x3-Kj~6 z?O~v$mcG!)x+f7Y1@BAMDy+w?s#mGnSVVNds}vmuja1?aHcU#7D=rczWn0mQPQxJv zHdL^ruxiB+-9qOn65GrE4H`c|$X5vY3L)Q-GBNuCuTJWP8ateysbRD$@3C<+VXrvo@?d#o;12;3stto zJNq`eriA(uLo%{1HJM(4am-`(Ljj=+#<_xA?Oh;Eh8*2$%K180Osf(JWtQEG(5S3Z zWA}grwB8u+K7;jH_@Kn zq}Is2ybsa~3(elhrOK;Xo`yWUaD01&6$>#u3c|~_Q8BYc?RhduE$_bx?cF6 zoRJtzE^)pSLy$utW0->g(ABE0(~6cMRi@Qs?C-P$pmY&vbJqX&v1Fej3HWnjBXpbb z{r($0GT2A@2qUgPPezYS^&Klr7LJAQRP!>Zk`X@*@=z&Qg9Ol9078iXtZACr%q`{w z+J(|mKHa3#RAyVU9{KHI$H;5=YVx()5V`4F_hEh%245&jc^f z^_YBPS@MKCG673q>=ki3!=>AjKbvlg4Y97ASCU;fj2U&q#K|y}Khs#{3h4}>7!%H? z3d`unCSW`La_rr(+&D{d%_Bm})&9w3B zqZWIit@AR<*`DEO%WM!+E9f$}n~#O(YUoQ|x}KA6+HjSzqp#a63WQAzEGa*S3VQ6s zF3pKq+`6@w4J3~Cq^so-eA7MMyoJUy>Owh3H0Yjt2AZECFI zOxj^GgKr%|)G7RmsOq+7xy#*J3VK{RTIR}YylKt%wd=d3oZ1)Yy7s#GfBQ59RXjuz!gzr-6}+O)L^xKxMnct` zPIRk+6WJN&7$T`sGUITUwG1f4?}MaOe-7PHDb%GA+4Coj4u_}{5M?i7XnSR@!W>I- z`6|tPNv=+@W8H}mZ$4S}HkNAVCA4gn|5EL$X)o`!dy~ur@ZCJtfc{kEQEjpd z#$K_q-UtZ~nxmpVP*heAWcmtZI$g8_ORnavWESIHwXY__yqSJ@L4w8De&z2Cm_bqP z$?T?Oou@TceCKGFjZkB-?gj)aqE74q`#d*5^}$EQR6e@nFi@&!mdLF143sK2M|m3* ziaX|MkARhF%qO;TLY`ddoCA<6z!@NvWys)8wwcs)0R4Jv(&& z$b>@0ID`VgirqRdt(*-MKD1nTnQJI2Xb=E6#Rs6*LcrNE4})S8y8$w^jj=M6Flo}0 zSymo+vdTE=rv^NiLw>4)NH5K0!RxE#q^b>}>bPP>H(Cky&)pM?{2;1~?Y(|mIoV7v zXAM;3F9mtECsq1>N-BzGMDmng5g=c7Y~*MNZP2mNB5qHBO&w-Oy8*LQwRY)It2=$D z12SJ!sJ->6H(!)0V~iP6c5dUvE~#=F)({FaqRzv)=1YORd7h|zGY~Nlh33s&1E^j5Dx3j~xu@2fAk@eZPlBfZ~ViP!8j|hsElx zLow)g*P#Z*yFtk&bC*~AQMy=5A-7=9XSo3KIyRPb4}XB%uylpZfXD=c%KJeQhIDo@ zH`NiIh;=1WEpIInA_~u(Uj>kPHy#n|YI$CRr`1Ij8IcU@bb5@5C^NJyn8RQiSnny2 z#K00^6$)lJ;!v=z9!wm7p^qz^sbUr4sa9L152M{MPG>16k509dK(3RV133S5llBX% zwQ#}^&V&aOsM82*M3{0TgpD$^?&t*D3r*&QvbzqEW>R(`x|lmsbX9CN4({5z`6p0~ zsdai-)nbMS64aa<;G?o055`YrB5n}89YSel1pQFAyc%Nvt`PfSnOGf3N~$$78I0sq zevZ>dfu&hT=|Z3Xhb(kT&cbPax)2Z3HQZh5N-r8L+cPBFTEk^~rUW(^uk82Yl}jMk zolWFwHu7qC>#|t@L}WyjDcAKtk!f|&zo3T8_>g;lImbMQ@e;Tyfm_OIFg7e_Mb(&Jwvjd-vk{o@6d)s&BYg&^%*A1y=oj zkAXhVY_j3bBS*E9fOHq*$QGo+?kZAsgOODEw$^OK`6J?}P2%bdjBdVyIOakSPX_bk ziEgf7;35g+7JgYnz^x5(MnD|^v4$bnZM-8IV}TT!eUmuS)m7(b6#p1YJ_|gId9o!R zU<(=M84r)AqY}U0D}y|YHwFWL-tXljiB<8)%B9ET=>Nhb<>m1RoB05z~#sWt}g1;3ZZdPG)R zyiGdQo;h|KR8jrEB=5@NA<92F&js2E=r8?={Mcu~TvueldyFF+!Sw7^$=SUI#=xCb z4jH8Id5+N3?$v(W-JD#XzLje{^dT&E8?y%nh6X9XM zYc)8JsYb!UWG2Q;oIXZJj~!m?Iy}l`4nj&%1{~_Jn_SvqT^|c9_bFR)#*ev zu&TJ#>-oBikz5Uvo^Pre9_ywsO%*BV&MuE97|W;3z0X)u@bT0t2nOJx!&>Gjpu6Rw zEH=tt`m0&mf``1dGV?89@ti<6ub(3hqzYJf4<-%;>*2w~phTDy-=XippIS|JsK)t?A>#o5g?f z;pPaEdz$15%;kLHxEYhDzHid)o`_X(0(j8vUUbTGf^K4(vaQ1~bU6uEh~&Arqp4j2 z9l94hbC1XiAclchv^*byQR^9a(H&)>MXe{`LG|_(ExU}<4oP>qgfz_^6a}i5$5CcZ zY6K_vK?nn81P9{tON^5G) z_rpsr_reL20EA!QhhOQ36D9$$L0Q_>AbuaplM5Tt60_nPK|y3jD==Gy4dnKDh$k&y zG0#(}WqY^9Ix3E<1Tsig`nFWf>>BZw?X>anyTOgj?3Wti7N`OsvEq(xv~JL|`$f+Z zqrg$7o*F5(M#B}0H0j<+IEq*|!jCgEa%zCv!J_Hb7=Y5PUifrgx-}MV|8gv>`ZwwL zk}h)riE#+Y#*iAXg6=yJGov8G`VBorYdk=0a40noH(A{TcV1fA?Fi3JGvo9{GIX9^ zA^nKDluCUB9WXhtu#_3&fdcBIoIuD+Ps4$hP1q@!M@3?PA8!>)u8Yh-Tpm^M36VHp zihG^}$xB7@2_#=JfvQOoxJ$I)XaC#{n3DYxibRH76q$5C*%WPIVG84ESTD&p*P~9M z0&#BGft(+JoMmP_W|Uj-V?b1m)XbNGmKjxB0nP@zgr{Ofc3G+A8$e8*Y%G64oRzR) z*=4ig_HJb3!L*lHePNdsm2(lAE`pY+)O1(Arh#1AhdyH$EL<;J0D(j5wllL~J8FTl-DXW^|*bfdtynNf+Mk|q+?0_BjA zo1rGmCLq*hg>#w`2o(1`INPXBE2FYNnJ$G9=jx>FMaYcCpNEd9)6r^0Puc@j_s`Lm zeD0lBjmfL}p6+Tu`yhP2@l$vLip-Ix@_PQn=pM+UfHS<$s zwO-B4DnMh9s;XwR?2kamjjF1do)JM2sH&e-HPcnqe9fzx7SH-fW#oz+>LQO06?AV8 zPkvr3@j9z2rubE?za_`Y%E@eIqTP-qZbl?$74E{z$>iteQ){sA4>p50aCs0m%ej$| z#xSrTFT0E#znqMm?BJYI-f{VPjj$j<#m>pj$r7H^6nwYDRYWx7CiW}9~ zvgKqRiM#=ki{kN~HBr920zYa_fnp>N8O+N851k$g^XXp$vjhrGeiFr+PnjS5Tb3r4gsqk=|SJ>8V%(qfDyu?&z!B7p( zi$ER4R9F4#nd)egsYbm(A-mQ?nQHDx?b#g02VCvK?@>4=vg*OjKa3fqKg#t1F{p*2 z1Zclg{QVN~e?X*@S<>+z>7wq%8xMwydKvFP7rT$~UUX3|@vG@#?((Hikw2L|hy2Ox zAEUGX<^vyG_gmAn4sqVw9@e|z zMx{t&n%vDAa%~3h zTMC#!pCI5fQ|`SWp9XihL@addBf5PV^Q3m+oD_tiZH**z8Q_KP=g8E~AVK^!Kod0} zRv}A{l8B2z${~i``3{jY&z*DT;$i4#;!W5P-dFmf+*j(X#>*U-if-DG$Yz=s9Dt8m zT`{)t=eT?tr-8;2(%9nCcuCP%6QZ$|G$uoY5u0G^ETN?44sd^)S)C2x7|t+A2bC0D zwM}f#!1jP^0pM6AR>*yasd5;o7HP@7CpJTkNYqTvV~31+gWiz_ZiPWwvbkjnDDz%} z4DD>tS<1R7@vNCyD3pJLbWHk@5$ij+IU#yGVz|}o3+eTEX9F!DV6Bx2kSA3Kajcqs*s>dsnvqBeu93uCb83p2Fw<}}bZ_rk#I z=q^Z86Fm~yM#={k7KHu>mu?1x|U8Ds40$s+doIN|_L zfDx1DU>Mz+20|9@7~md?VvJ)rce;#a_!i3$D>ER5jkr~4akI)eK25hqz?~h4UYN&g#F$z=Vjy_%Jnw0hqJq2M9DK>Yg3`* z$#tPtNcvHhnKb}n!pYwmAeVV`smR>{G7k~evhq}X-Ei@QRi@HvI{WJJ9KL@h)2V`8g#{#EPt*wlK zv&E50wsKf-qEA=%p%o;)L+K$~_Ba3?KEiqXC-sE;ah^xa$0=Kzk+IPcZKjrnAD3n4 z;a9c*550+^t*F%u!>~opr{*7t{1dz;{tTs^OsE7|`v7xpBr}B)udpTIKz1jFVD{IVO6MSXM;7(TZC`YVz>4QDL2BnlVK)w!#y$Kp40Tw z?x)+g0x8w%-55FhdkS;rh4s?y+epCr6q1(KIp7)bYzg}sQRKrUK|HSMpi$PI6doo4 zO}8C9V}+T8S%uk=D2_^;Bv#ieC)KW_t*jwU^I8??@xeXSw}9SLpoeO3J8)n=cxRM7 z8EzYU53!|R$9CvUw(!qr?c++XMVa0m-Ns~HH?Ag8J3 zizwtw6&TGoUxF{WO&g-ycOYH!RYcV7DRj<)fc-v51(>`OP_LxDd_YfTOC_Buf$WO- zac)*&i8(+pao`q&r-exPWx_iodLWbh^T^Hc5zE^McrJ-xH-)+a!LaBrP=}y~szb(d z%7=A?`f@QMJq_6gl>pxk15MYC%!D(rBlP)pq|53pV5#(F+oSP2p-ZZOV=Kp@M5?qGade_bz@p=%~Mj-vC4^8;A!?g`{tjzBL+k zD&HIeVaX<7e0|M>T0;S>L0g;ZZmsJG%6hy8J@Wck=8Y(K=MRVpgGAzDkV2`9YA_W- zRVugDPv!0)6(1{PKU+|XbugsN1qg&;XEPXwv>SfNy!Hkisypr;grZHFrg0_E!b z8Z`1=BCSPWw>QB?LPA57n zYlskv-#j_Ko{wFEEcmGTA?&jGZ3?E>=dw6$KFj|o8Ku)|j6O+C4X9aDgM`p$gCHb4 z7vZvRDdbnvOWYc-Vopa)HQR(CL(I)AJIQ){r<90L>W;7w%g5JaS#fqpa>^Aq(J!-Z z@sXdVj5kL>G1Xc!ucTBgG7C%9$CRAAn8|qNXD+I46A*vSi<&3|tIi*Zq}zNcB|jmV zmrz^DI#9o@6j;z!O0@{gghMM4*nRd<<^y6CAZ1j3&+frqE&w>O3ue7QCOFJ~nZO8z z{|Vu8qg}ZByJsY;Xo(;PPIIay_1ZQBw=<{t@ zJJ*Ii0HDJMY}kEt-%EGMhP75UEK^$yKAsgDMnbS*Fi~Q|pj%qmWISM}Bn_K1tm`2T z(Ib7*{qURrz`&y73Zlh-sB=qCm3#DWN#)LE-hhOAf+ybSeHTfU@IsJ;>0)|fy67nI z`uPm3U#0>J%2XH)l_{mB#!_q-o~tL*hY;$@^g%cSnbPOWw6iPIM*wvAfJ~Rr{V?4j znYO1)xrg=u(t2KGNMMz$Fi{{r<{HQ~~%x)jXTtX7K|xk%D0 z%-k#WU#0G*+Y_`ZZo#ZLj;$bz1)1)KwNCKVd1DbYzuMlt>h1*eOG(o&E8cNV8@eJS z6CC!ZrlDb!U#Uxz354#aDALl7Po^1^wFoa%)+|llQ{akKWae%e74dOE8`STc4bw#C zD=>_Lj2=T=!#o85cT2I~mw2I`&inVC9}NuQQ3Z-! z|1Q+iTuHlwpLSC(ZC7(eI4b8onZoz@$OMiJLD0VDcB7U>OF559G4t;_NR~gYNb?Um z%$z0Z@F{?=3Y8Ib82vb3hY?%z^q9$o^tic}*3#Ewp&RJ2@Eg)&YM9dFASbRKOK~ba z_S%UT>*?nT#CG-bc{l_8q|evSbHN_<^CbWsKHy$S_X~7~^s^@rp`STg9JyO9c_*Qe zesZzz3FO?wyQ~rq=%=Km+awm!&o%HuHyBu!LPXsrNa^P)2I@9nNopYg~Qp3QW% zRG5)3mDTU)DHA-Fo(kPSPle-J44o(6ew^;C*ASx?v0)6?~c?ds_|I0HSU z&)3t7!5;N=6MzmMaBradWx7Lp+7F1((?(hZvRNy6C!tW=yBEds3VvmKSUyc0Nk_Ly zD5RsW!VBGCpxjr3f+R@k=qn7=ZN8L_ZY03wGu|YZ%0eA=jh8?<)2X24=QCNst-0oy zn(OKg`g~ov66{e| zwgBkx0rzIQ-=sUFE0+Kf{p_eV3Hg3m@=Zd}6|e6*1Oa8&;sHI8G;GqaIHZ$>?UHuT zvhJL^&{fDdgveObz0&?YG|J|?eC)tO^=)4R38)t7N#B+Zzi-Pxzf8z1+8==hG8INb zGNn|4zU_v3GTVkwS7v{OGmsg5zRZTZGJ6+5hY!f?ZMxs0J0!EKC^PnLn;_?}h|EX` z(@!1i-+ur-+EJ*b;+t?an(4ERsOI1}A}2o%aYS zaCQ=Xq(T>vO^B=+JKKq3Gpf`_-6lYFEWFMF15F&?jyPM1+*DFPRl0~2-SqG|S%l%M zt9&VsZZoaBMs=Er-P_2!%c@kjK2qJT1Jti>*TN&!Ey-4M6D@xsTTLeXa^{>r&BDfz zJR7prx=op7MvH<^`mtwN7GWNQJ-7Evo=`*8+Cp>f&F2E}RF^|@>B{bSZNYMx zdPY7QX&ef_$}+P41jJYwHvvVgyWvK*VB&GXE*5#Vr=_~LyQmEF_ za&_}wB6C6o5|BB~bRgk-jeF4(1xcz``$vyn@NqBKQRN!ds>XP0KISaQR^|-O27%YW zV^FBNy&I+K+JL{o8Q1{&d>ddvX=pe10O;@$uJ<<}5cJdJey7((PZ>F6E7=K>Z)?I< z3SA*&T?L7|!_bz%KwT$k0Xw_kf+`VPEFm4Cxf=%ib;PxW0t@t45(={FFsI5EQVws_ zlf&N;>dN5*I0HG*=gZ+1R}OUmI(&rXFkZdT`L)+_Nv{3!B%kiB0zU1P7?P)lbrWK8G>Xo zBFIpBvl~i}z#6n&4WxIkpPo$63C~1#C+I@y6*ZXN+=kL4@EG}<>8FQ1-mW-^zBEIf z9SHK*G&wyein|`h+}GC=fdzUZ0tx8}H6hTG&Gq!;Lxj3|vJcKcPw4aYLr>TU&}t9EZ!La}pduV_uE7txNwqQ@+osM)S~lrgi+q{Z zB7zZScTihVo$3V7R3PzO|#bMrsJ-g*4lENYgAyhielZns_ zdiu>o>g%__0{s?c3F$XAIGE#NAD4S*wn7gR^S}o#Gc&(M`qnIXVDdi(x&q8mfKPnD zT?Bv`Z(WUc^{@5n%V)rL>&vHb2K9wLzrM_L>&pQE9X`VKWs0gVnX-3mo75LZMt$MT z3hPUM{GP$D>{&ceKZLeTI@UMfQT4jZ8D`!s`qdOnLv!PuF(j7*5Yfc_G?x1j1(kvE zr83are*Dy@Rx_+Y<9JtCaq+=7FYzSx75YWV^!@%l1M8Qiz=E=rjD^aQ(hbUz>k*$b z-G3v~C>L)%0qe{M0XEDSS0Ms8_M9CO_+wb|b#I>GoDs|Mt$OA74KST`Vfe5wIp zaTdxZM%p4z&uB|}!T&Dugxf@Oyxt5KqYbO6?=KO5BlrvPwOE`Oh^Y^x;F~-kZZ^f5 zpWLbGuBD^QW2MjQ@!J|<&wDPg@wpY?*?&Mibe%KdG=D`6biNa=1D>G!hXFFoVM_Ns z{oYZpTz>+RTdqIC8I&u1C|5Sk1&QTux&8-0=Vv^^<@z|wwHS9;VY`L)o|G#ihwK!_ ztlBRiRF)41?3B>PM|^-~UJL%LN@UeJ0xyb^fo0sj3w`VIV%afmh9NZVj5v zD8CYFt0$>hPhjcM9ptLMqz{#T0hW+`K)gGQp!EREpL=*K2m!xt2&cfK_|_?sW-96$ zTQ^zaE3ow(l9jt8Ts-Foh80O(3bVX*BgwYPQ%_dvVi_zy3-J|Xxuevl;o;TsNn_=5 zNR7YHSef*Fz4q3Sc7|ksjCm?1cC2|Sp&8T*;Wkt+s3~e39OT0t2TL&p^Hj8>yXxs# zO2pSQ4bDK%==1e#HQ3{L%K*^P@d)eLa|o32+&Iwxo8*;|L-ylL1eD3DNfSrt;(~o( zh*mhMaFuc3i87+w`Hf{%D2QS@+!zc~t3Gro-1dJB*%Zd}Wlsh%A0@Crc0wv7JLX>5 z)@#7P?s~FW3>|dq;Dc}mvZ2qH%|q}|HV@x4eoDH9&SSvDEYqL)n)EoHN1CK#9IsFMOW?+O zFFe*0csNhu0f91h*-J{$1X)ip@M#9xBw5jiVK&*uI+i?1+KADcqZZIV#*t!LpZIEyDUQ1(pGk-$QgH5Wu4Kv*(u3*S9i zNqOItZayc(g!`8DIc(~2*n}ZUdj9O!^N7gVukbp>I7pelP<>`reVy{^^ZTLlI!1ja zUk%o0+)qoIb68;1hd?)_a6`ID?NjwyLL}>My%s5`dOgV7M`^x>YKV>6Z?HCs?&njv zxyM)>NDK9$i3byhz|`J9?rXae{tu8h!&-r~iz`(Dz5usjt;Ew>g@=1J&5HqaHJokq zNiX4LuEC>14o_&6Zm);KdJW^e zeDmL!Tr1~h2p9S~pygPJ#ofVNbVYshk&`6736GJ^75fC4=W=Nsiv%yPiHxmO{OU?+GmaVS^|4<-(Qsj{r_ zbiyeiiiUnQ9oBw~=vo|ay$s3no@9Z23>fv$^Pf@A&o~SpqVQb>l&A+7-LFA z19v#evlX7^PSwT1z6ABX_&Szzyce2x^J4^Ivp7~wK%KJjaP|t$rUr33Y{x`Zirf{5 z_D0d%C-{|D{vOQ_1?PnZahk@$8sc00s(!><_g()dW?e|7cmrSY-gjJW3a za6S>7J->^yUvMgiG(@Lad-8)gUgRH;Ks4*>7b5PXPX-HEcpEsqGb zCc$a#2$v<|VFxkjr9g#~>2(r#s)F~zZEy5j8=z73_9yo@@S()h5h`aN60cC_Jifd) z@6LV%y7ejZYW{Qqj`^n)9wlsNgJ}QXBW;w5bZ)YzG=I~fph@t z?!m;NU_CsTH~^y^=fph!JTqY8Dn&L82487r8eZ=HG*?eBPOk(Zt0jg&q4LHUi-RIu zkibAXy&ct>hOq1)jLxISU*o*^y%>KQlt#s;s(3o^yho=ua1;r-sKJX$u$wAQ7$xa; z9|RfJVhq4SJaf+BY$o@uw};(+9i=R?6o=5T-|%4KP_Q>Wm^eaddA!1oZUzog*n*eY z3#&AKKKkkQ#V}^NJpfO0D@pnLu?|2-f`%O{V+eKomD}J9`jzzgV|E-FmVRY9-VPrq zhzh#j0v3zO)19~BhdpeneT;VRbJ0f<3XSvLK)@sTm2JcWJt0Xa-+710q4I#JU4lo= zb^yGyBk#hU*n!7nLM6!TjU1bO@NnJ(%!^u#sEM5bIJ*d)^fx?@Gk+a?SvRjWE*)O$;W1fR7l*=5gEs_zItd9|-uC`SyIG+H3?lcQU z#ye6vHWRe=Gf0S8pTb$t?ICQr3D37OuHQeUQlK3tOfpU#3Hp~pHc&%{S{Tfyu-^1S zL7JZl*l$D7?q7}SWKR^sd=;GLxyY8U_v{nX-W(LY8zNQm6!ef{d$~;5CSQH`9rlKs z7V(#xtEJ;aREl4xknXh->i`l?e9qM4x;>O>Uc-k{2VbBQ29gYATonx>Z3kru_ zFJV;-`+-M>ys$+`bA*JAl;;id93{`uc$zWH+=Ofj=V&;DJ%l_a{?27%oJf0v=Yw#V z3HZ*N>a5V_&{?56E+}K>J7%3nad~yMB@`IrzBE}w|3YMT5LvcwgLnlY6SxtZzuh{I<(@$=yxzj5CkX zqw{zg68MsN-0S9Xi^|v5P`>stU;h9L5u0G^e94_}izI)P7is;2v_5oc{Z-N07NYf0 zK#Sl8X_5EagzJg&w8{H->HwHmo}Lc=Q{?FY!?6CF4z>o%!$EtJV6$zrL;?Satg+q% z^X(o?90F5qE82`Z5Qbw)Ovt>9hnX6Jd3j-er%Hrr z5@7%$B)(Ei@jjcAL7RU#;lywY_vzuLD$K9DPVRM2EI?ChEs{=}>RYF3qY&pS$R{uW zy8Q=GaND67a0cxVeSSN1IWo?+XC{CSA8^m6dzNy~rF)KY|B>#y=%x(azRdtij%Qdi z(E5BWWyAzR`*7X_av4X0XqzN$oTUJUntogW7v>iTFME_YIL*L7%>T1#y*4qXeIw=|iDH@9v4ZfU(bT<`5sNhPXs%V&zWQ3kj!S)-mrb zq-f=Jgoe0ruV7&2L(D8;8qJcJAxJUvAOm%qFU3r;$BfJ$sc4vvWCV|yDkSSyhHK%G z$`GE5yAOrIUr6Vf(~!=mX}2M>{|=R{tMh39d%rSPFWqKQ9K9Ef14(y&pVF!GcBIj* z;$)>aYxv9uGob1aGO254X=yzbtGZUAwZG7FkI~X@Qf{;n!hcY=M9)HXi#0&iEeR2o zaqE_7kE&Y}UByzR>xN4~d*QAIi@@D*k$rOAq&azA{FB&Erkq9n!qwZh@XCz*_{WyYgdn205 zsI2Zqt-oM4pkQ>Ri`5fY4duWmi%D{FvmCT>&caX31*?3iZU6n+r-kx*!yXd!^de9K1T0aGaHD9Q!a-OvE%IYiKHKFj+ON4c#W48mpZ zMAkzsT-V;GCtqzjR`DgMF?Deg64r= z2!V%Un#RRm83O+WS39`iRUvTCuYB-}A@J=1k<|p}J1_BNGkrB{bRjvmDM$Cr6?srW z{Y0sRr%8;ac7)Q}sJ%^3qHxu>kYq;^5tl{_l`QRA^!B~2Ny{O+iAZ}5FmRy7Phf;3 zAi0~wFtY@*sc0wx+5AStrRhltsAH-Lm4K*+y;MeVB49~MZhb=HYb9~YOy#aU6UWIG z0-Cc-6at?$6W9)N^Me|xxRDE-bwYyzRy0%wMBtMcX3j%>3uWLhzJz2q9TAsiBxS(e zRu|5|BcTj1hRQ&JvmS2UW-+S7xdTV}^+^mU`rdWCiUFk< zIDaHbsp3*%Q8{NaiSeu?PZFa@l)ez(HEc&-R2^`&AGRh|@y#z>z`7ZgvH(c5Lb{n? zvx2W1Ca*I?a2kKa;UC%(CJ|~2)mgrzZE)b~p|4AiAVj+{x~3L@@YacGWA(Rm1)7ZAzLDcql)%5ddm7BJ?6oWq9cUzZk$UKK;7g z3{Qoh#=KD%&jwfjK{*DMMYkgebldJIoI%@7pWk-31TnVVt?=dt+mhyW(@DR$<8c~S z|9=r(XXMbh`Dr9vR*X;-n$T60)KgeL63IZ6L<|pW0P^Q#!iClJEWQfEk@%hDSnEtD z8zE7B5)sVBs6=Q67{_{$E%tGYRwU8n(|rgeNIqZnDAsO@okqmaZN8kwbT<`jY__=r z%(`*iSTuv}*Xd>s@yuKmlZjQQVx7;*gUiV$)aJ9CRUV`%CSlffX2Gv8tc(2|N{os; zjB)%eJO!X3TX>Sz{Lg~)_zT%qvK_Lms&DpdNdBR*HDbzH+ERQE7}0)rjKOSrV+>|i z*ROqW)=jaXV>Y$@dCHCYM_%Sou&jT9ZI;3g)Xkx`DcdX|HcQX7%`8vzAxOGZ zdpRg^mYRH~%PE9QxA}A<&?)BAuM1cY`~X$UFh4-62}V$_i-06`SRCLDu%;eN90F5y z?@D~_#4xXfEyp6%NU-AUWp!s9ad>sEKjyynAw;)7#4~x$wjXx>0vN|_G2;yrV+`cS zt9wW4)xA%F=+?dca0YdcKELigho+Eq?=t`$KA1Ic&guySqVDBt*P?`fld@;z(Egm$ z5x_n1_@1Cv)&URHL7}fG%4jOS*1(3fU}E49(=QaXvd<6~+DwodM;~AyHtO;vPqmfq za|PxrGzD=`u*kJLY$HGo+s;sg>xCtjAxF+9j8-HJV6tF3FhmPccM08~UP%Fl>J>Ff)hh{+(sHYb6kKwj2lHKDB7$Kx zLrriSB2TxWzyCm(jFb%PespnywU?!y>Vd*@)fv)*-YGg6bnSoSM(Q29O$CV6)6gC zTq;caRrs6+*z>jJWtJtj)5!b*=ST0mPpJuP2?-|Y2>T00C_l&O6ElI3ekApYg=U~1 zlJTG@9F~OA4+)V>yZRydPu7niy8Qz)ZU2a;`3(0FeL;)r=?KDqlJK7!4(A$O3fE6o zA}VCSwf-*1V=D6~rWW5=D2DDiI6#1Yk%Yey?13mXPaz8K9pip)|4n*Aj0q*a|PhIuTFjBs_Gx zHQ`4hZSYE*jE6ap-Ab=rartd5^Ze*>SPJyIk5#XO;@q(XJd8(NMZuvB<{sQGQ=(V$4 zI;o2qtuxW(q&{{?>?8dGChXC#L7xSFs~9_T466m6a&5W!C~l?=?=SK2pGVJK@^~*$ z-AfN-y37)txzk))7fmWY(3nYv9CZTMIunH#==x>PZ^DwMI#3Yw1G6Eoz&K+Y7m(a`@=Q2`c9K57 z9~e;!)HW17)!`%DHvR_1gMCqFgfMJGe}j?HHezZ}RvFI1+dE4vbcK+0Aw-mXmYAdk z?3@7?T1#Tfh7USWwOk#r8SJ-XWCr$0U_m=12?beonA1}3(Zrm0s&xuPt=nDcEQOQy zjc~~a_iA9Yo1+^AX13&9DtN?dOH30t__r0?z_sSCz->(2LYTM*Fz+3aD7Z7(x(`09 z9iqq;+%CYd+Dpnz%&6mF~s1j!)BIKT- zaT1V!hE)cTe4$o3E#7+QlHlUK`6pN+S}^Y%o=S&%6}p4AVU;7kh1(%wiLN9LnqGa1 zd-brA{9w*4CiT+fhUYLI_qCFre8?xU-`$__4&vax+wFK^!yi3IBA;@DMlG6H^Aa{< z;Wl@QVQztnmn%!o%XxLXH(~{rJm8D>$6kx{_-Dv1khJ#cb+ zz_R2Bs|jSMnkgC47S1g|{l(Nw%`BwyV8oh%mu{a6TGn`$>Uc_#jS~tYEJya~&LbA4 zCsbWm<&8Rf7K<0Qt{dL?UgjY&x?WlPHunNV7U2YTo8tzaXDI-WfsugzfRJksd%zjk zL;C!2LnS=y+4RKQ!Ce(}_olm#a-U6iU%IJNZeGrTA7h9{+IcBhG?aQrqMJIIgz&W8Y(}_O1FnlmBnnL zR}1<#Blv2-3=5+kwaZDi^(F>NFvcm3G@d5ppJ72HG3ye9l%JREcP*?WbUl@V7PkB( zLAsRq+}4Rtg2Cys<6|AY_5hp0Q93p)g0XsIe6Mb@y&DLc{PRrmbyG+jnG<1tAK9<0N|dop;J*1rWHG0oofljD(wduRRPN?23fN`yGed~&=1=jg#c!0 zVrIS!oID-I>5;pTgJjv|RC(QHxQJRHOyEul9ulndWMU#n?7+cFL)u5B3$XJA|C z^KHuoZeMFWfDS&mk<`~Z8-cJPnc5EUfN6HL-HaTXbA$cVM&pO+Os#AQ9+-;|`nV(> zm|}a)GmuY?j$%$6Fw{k2s2&>bf`{80#K4;vsNxmi=6J$U&123O!iyvlrX{FKE(tfX zROgktc_FxUMs@>{QMuI{*r@Z)nB0* z)L#)&sQ$9FRQ;6@5sO=YMMA3ndh<`s>y`Bc#CFTN8qT1s>GR9_GO)+C;}!s&oAH4A zHo9-6JJfdc2O|1N5p5!beZ1tGghGAnOAt^tM3uLs(Zrbu42Yj}af7}*2`U`S@oh{6 z+k_PKV3!xD^f(D(7CIy%U2mbL@J@X;)CVV?UnXQ0V->>Y3`4s?-UHbRiI8k5ouCeG z#XbE1)J(#|+gd*u8BXNGTumlaf>b}meBG_17DuW6c!#p-jWqd(EBX-!8%`N>a){_P zK2bs?ND7T2(95Gi7(g#CT^I!V8YD@IaGkHusC!E zuijhrUpO8@sJ|*ehz@nY&^ocJ_>=baWU=RNT-G9IwObr zNm~){6neL4^fg;M@gY*_}ki4drlKFXKaI&L4?m^I0ZQxcNTF>D~|AIk;`&7hu&z(Vzu zrKxP7gb44h4HQ8p_hA+p{Der%9n$BDWUc@}fz`=7+7Hx`5Ez#)gxXB3_9pem{{#+P zeOdr#pilJq`ZNhyq&_VK(779rus%&dAZ%x$-0_8#QM4J141G!ii~OO7tUZa}SNL6w zj$;|Wa0|VXG_W-n*w&})JDdPsokj3MljNns4j?S?F&@r*h80Rgv7JIeP_0x<*EFeo z079-B2c zIp*p2l4zEk1PWh`g>1^n*JTzK+IWElx+tOx=^`~D$xht^Ktty_?`;r{KHz-_IO_Z- z;P*EK$DZSc=wRM$IHp{!>tr{@V#aI=i{rOL$>9$k70x+|he+6(LVG_6WX#K@1_XUd zjAtJvN~(E5QeTzh>b58o{u$Pls97gevi&fsJ8>@nuI7XGEI~|hs_#B<#wR?Mh8?dZ z26`M~$A@G8sj%}X0ETrnXjs?afgRss+gMZ@pN5rEH~ zC;})`3Bj9ijMEKsD1uVWMXZu&&+r+Ykl$;c_uJ79mVmJH44v`Iu=?yQr&E2S z6AK7zV4na)*6yFA1L(*Du>>X#z;JJ-`Gn*|WVM{mR(i1yYlUxa2jMmol=$L6eD+z` zZ99=R$uR$d_}GIBESM&&-pe<>V2ET+kMnvt?yuBnQBuiSJ-tZLN91RiI5y29wqfCJ z&tOh4Q@anSCyCu7@nQ~*wNfJ%X;IQuW)xeT+Lf!9krv&%7U5lX1s;l#ieYp%byE@E znpTQgkd$QC6TW6^;ka-!3(He#ic+x=ZBb&af_fLK?}duTtOgXzXc4Q#9GHf^X4FUQH4CX_DWFtOK>32w@%!-X@1HDuQ4`(KFVn?< zj1W!>U@|!0p{MLz5*I2ix43~YHnpgzL}4ox&49lz-UTl6D++wIE18F4g)CmD70^#OTrRAvfna z*ONle^Y9=*hr7Ilv0V>#Y!l5Ua^V%$98y^!5I)^Po#jDUXO&LnYk!2>qztVB?8A%c zRzA1pVHGx3!SjNK0$AUmPZo<*h+-;CBcfZEa0j2nuyi<$3AkTD&y2dgAW2%fjE1=q z;>02YIF6dD;MUE4h#-1uSU-U{dg9g%5Q1U-hd!6Tbg}8^xr4u4MBqNnJW#=jfl)xt zN;j0mrFzXrG)M!>6iU&^waz zplzX!Q)~H$%k>e>zM6L%Dd#ofTn(sv+YY;x>Es9QT!bqF+^gW^ZoK4k`?K&d0>r_V zWRe~t=;9nobWD;U(;nabborPiE<6FQ`4K-c=!ZeyUk3E3@ef8=N&VH0Kyv%5ufQ4f zSLs84l_gu?eX`Ns1mIEMZF~p1BkNFC0v97U*g{4QjrF+%;H{?#T_I$B0*;f{(@0;y z&ITr5=nz{vrgVhH@(ixu9}`&6ACrVa{W0b=kWX?szKSq=f(TojH;eN%K;3c_rz%H5 zNCddmEEsHVROQ&;+q7Yhq`iKfiL!st7b0i}*`!)GLR00FbTQp#a76BvmamSO(-23t zh~X3VIL8*NU{zI7%rS^p8kJkDq4%^Eo*oR*?KhZ#F|yHQGr1m1lQ+we2b9MnSeiiSD%q0G@K=W+8&or=J&=W_!Mxx0Yt5s>^v})|->a7if zYn*0@uNUA$ack)WZ4CYm#YeZ}c%BXX@d#)CNz*zWP2@sENQ4NGOf|j~StZG^l`8x1 zk7bnlpBuByq~Sz1fGUL z7MDUoOF^#Fe*JD&u@JgZN_i)cI|}YR zDPaq43h;U<0cK$sw53WL3G@_m(6BKRs!FaKHg23%T{66?WZZ-?V@j-e$*^%HRb$6j z-&T^Sj*l8QvO`KMg*kRYLhy!HRo_xoHLk?&=m7Wn68ki#L&|B%IKxVA7V0D7*4UEj z8>&iL0lQU6)wl^`tK!3|QSUmWv`CIJGF~;Tx+)HAFaYchDQ$lT0en{7G-247l4?sK zH%~?$QxI#3vX}Ar8kU9 zl#GvCqpPl~E{Ru-sESvOgJ~-1KjECJ>XPxpuDfyA$SN~&3fZkL85XZ989OSGfaFW8 zaeyK6b=KJN6Cj%oC1;NtHhfH#8+Cw+S^|4hZ6O*&alLiRxG~nS>k$uTu%x6f@;8=Y zqsEP};$ubf?Mf0=RV6o6SC6ka_0*A8hq0rsi(3h6M0E$GbLxm&#*7+w9merI+R?BGW6IQF$yezEf&(Ww~YV)E%1Jhr|{=m;8psr$rlxEoHC?%irpjomoqG;8COSH^Fg@+N-SQ#Vd|;N;Fz7vooY zO6RF(w(UIi(bGCl9dt(LsV#bUo_bH;&Qn*O+j(l*HJzvCUyJan&QsqW+j;6)M?Raj zPd_kiP|<;DJB~jvZ5MuzlpL7$aF+wq+Vwjyt@{P|RUMdiLCt|_*W7+!TJ)|1)1H~T zZ~9x0?3-Tq*uLpcJ+W_k|0nlNzh~9H>2JKUZ~Ez5_D%1#b>H-}@oThi-}H6g?wkJ1 zkNc*-^WN|oXO2EJW5d`(GxpweXvT_phi1HhU$46l&3J3^p&2!`hh_|Y5x?z+X0$(a zXvSGz9GY?9r$aODow$8wuUXq?UNU?8%wfy$Tfco~V#D^C+I!n){&H~p%&lK;pZOMk z=Y6w%=ElgoGq1~hcjg0cmdrZ;!Naq^ef02bbH(A=<*N?QKJmrFv!7ghc=nb}hiAXP z`S9$~ZylaJ=##^`?s&gw&b+#!IV(Ocn)B++ALs0`&zjruU%lsM{MdW$7r*wN`yZ{( z+>cZG%q`CAGj~*BpShXG_nG_n$$jQ#bm}wr?z8&L9p1gq+$#Kf{bkV~Ynr#2ckZDx z<_)-M%e+%=+cGa!vt?fK9b4wjyl2b2whwKYm-5J#d7B?c*orOlx~|+Z?}hbS=1qBd z%e-m$UA%tx{NAtbp1)@6?)mHS+xFMp^Z&Jb_xww~*ge1e``z5?7t}4@d2-$2>~?jFm!Dp@cv@NA;*JCB7N=ZNw>W~|1w-o=KYc^p;te;}Eza4K zSn^ozswFSxty=QSNvoE;-fq>B!56Jsa>~F}OFH29dey2WyKh*v*u%jWE*Z{yeT zt?Z?(c4jXvIheh4)0f#x`)S85{q>*2mt8#jhh?|j^TV%%${@a!Bb^Lbavwgo^`QTH7R=q!Q%!@OB?)y^O`tEBBX4kI$WkK!QhZfbY zUGQA(+J*RCx3YHaKi1c-J+h&8?a-~YYqP$oU3>hu@c&f1_RW-~Yim|dSXVT}SlImp=5GA%oZO8!Uy!@;?t!@*n_mw9O}QH*x8-gu zoRquq!dbZ+JLI+A)T5yFrs|VhZ@Tc*)|)Omz4fNnWvw?oeL?F@+b?dtsdixNO^@Ms z;SH@f4ZX4TrhnYtdehrePJAtzxbn44AH`q0wC=*!Z@RS0W_@s%&GWA8vbpOWT{fSE z-(37U&gimv^{g(N4=wJpId=)dmvz~^^rAo%6xH4-pCzWuSo3JIt0H{t9NW& zV(-}ce9exn&o0=pb>RIwwtnyo{4egm~vGv{ec5I#Ye#>nmK5n_~mQPx4 zTla0tZTB>3wXNv*R@)AhwA%I+eq&E=we6(RR@vlC=jqnZYcD=W{|L%-8`tL5<(tr0C@AluV@9)3+;8*>3&-%Lm?jOGc>_7c?pPzEk z?puz#X!jlXO~>!e=@WLJIW1$)#CaKe9Q@9opRwoWMHzcuyDwwU-yX==bL7d4Jv*O; z`-O}>pT3o`=lks$dzSA7{sTYk`LfHp4;FM=_rd1#*M0DQ|8*anf9bjpMqRb;gILwN z4_+O=?t{Y<*L_fFulwK}{JxyL?t|N=totDMvcq+Wp@-{U8hyC#W&Hkl#Pv5oT zv-iGfb6|DP??1otobNv`9{m01JFfcv^NWXn|M?*NPOkd?^T)@0|M?mVzsJA-{LW{; z|9tbt?>~Qk)AyhE`sRg$U4MGv;INF92gk=&9(<%^<-q}6Rvzqi*2;q)_FsAM(8Vhc zeudvBm#jQE`i7MUD{uNg>|J+26j%41O%xkoiNQh=SH+H~V6X%Wid~e5D2WtX?m{c+jq_tZEkt4sAKnYMQ**$6*cdD zuBbKejr+Nx-^QFP3JN?|RQqPF;{4bu{4vyuDV@Rsg(8`)sr+5PVIA?|nY4|l(7`=R^YZ*&(+mc;y7a(Zb=$>(cJN(y(B zl>EKBq$FT}NlAy3B_)5HE-Cr?`;wB&Kb4f6{-dO1>|Z4%s{17+en5A3@!pS~;=Pp` z@!qIl@m_A2c<&;xB|^N{cB*)8^x}t zKUdrJbbHNRPe;_+^>k^&T~ANEv+Jotn_W-)wcYh}08rOy*V7|zyPobGv+L=ayfi@w zUasiXEK~9R26MB5ZRTcmcA1+s-D_?ZaMIi?>1%T{;jFpYsUOVEqJPETubG?um2YmA zc-`D=8ZiB$RgJ}$tZMlEZdGGqfmMyi){Sb^ey33l3qZ7KRKu@*qZ%hWH>#2HZlfB$ zeHzs;b8b}Q!1v8-=6n)cvr9s3&6Y{AHCN1wt+^W5K0mhRf}OE79d^amY;q{J=HM)p zIU8H^^$W2z8(fO5dHVNeb%I@fsx$pkQ}e}_o0|JwX=*;Pu&Mdux2(-;H?uak0K`_- z=6)Tl%};i*HcxT1HuvpoZEn`j+T1+nj>X&O?^tyD>5j$4t9LAF{B_4--rsjDb_jPZ z?3>-SNCc(esccR{xz&)#`intyVw8rCR-DpxTgX_3wCB zs~_i6t^Q2lo5*VQr^ZyPUmTC^ZRh$qX7Tl#IGH#6EwOvUeaYP$_FCV)VV8~F8#)4g zQ@c0(CB1vYZ;y3v7;~X}!(BJJH#}O@z2UXT`2O`C4No*_WtG>ZmDN4_R#wOQwX$jx z(8_9DL@TR+s8&|NfZr#rtS(G#Wp!d&E2~@cTUli-Z)LUWi&(4P2V<@FpNqA!zYuHn zF7W!LSgSQxVy&JQ$66%|aaP;kh_lLQ7-zM{I?ig6O`O$2m;A<;J@Xs?284W=-#Bc1 ze&Z{u{Kn@a^Bdp!IKT0@#rSSXe&gD!^BX(w%x~YB!n*EO|Q>6$JG(=~lOPuDbTv99U1WxA$Uw(6SR-i|VRbWN?-q*}iPY+IXZ zePDg6b*r>g>x_z9r4S8&J~xdtl?>lICkClr*>1l{D`y0&`264^J&=K4D8q^BVh0n*SWW zyoGb~mNqLL-E5X`JZS>$$yH49&{^GPvkHe>JI(&87X5op`HcNm_r%v1SeS+_+ zpRuW9amJ>pO+d(B{EEg*G3X-LM%_>xRt;;G4QPY^F81 zVKczyhRyYMH*9_ke`NCu@L9woo27Fe*>wH%kch@J+k@fL51>)1AZ+r+jR1QglEwu!QfZPP?k(6+iFt6js_vfAa>&T98tgRFL+49sdbdvI2} zxxl><_;MxA$YM$+HntQgl zs^{5$Pz%rYx(=T0Gl481rJHAa?I_RoPeyyTk8bbUVOb~F4)q*dJN(eswL`!F*A9OV za_z8asB4GCK%T2>2j^M%Zh>ou+RI%#Sg%0&wRLQJZmwgyVtXCi$NTHp?mbw?w#%1w zY#o6bnRRTFPS>&ZxmCwD?Vmcf2cM#xnYr!60nIxO$@J{B&U{4Y2B%hZ={Uj6-pR|% z-qPR9{z0gjz4>%A`<+1Rm1g#fHk#RQ*koqE8L-@IX0JVHX8+bVX7-Oe^tNyQUT^!( zPQC44a{)qn+xv*U?bpWjw%?lA+kPidFR8b^u(7xOhSc8n?GN|1kLvB=APn?yXzA+V zU_Zjc;YzrN!@?+h7w6&d_C^nf&r&@cwgcv2FJIi;JaxX98V-~aGd!d$MNAmIgbAT zHJ|1N#?6m;vFQP3^;ctN+irwY0?`@Nvs{ObkXRD}iI{DAHE3c6YRQ_$_lh$-EL z8B@B~pE;#_%Xz@TDc$!Rn$o@Bu_@hqUzpN8>Ee{`^ME=xr*u!eGo}0b`%}6vONr?5 zMV~o6uGQGs^RJp4d)BD8vFFjo8+$f&*w{1BabwRQVEPz*|KY}-OU7^P*;=)+=bs@P zdrs1A?CE9h?4+yf>~y}dvr`irXQy|7KU+CFt?2FS6zbya)L@vi(=az@r#LTXrv*MJ zt8sR^J;1fkwo$Hq4tu!vIThmC=Zwy^&-zI0&vfnc(=69M?dH1n8NJZ8Pm4vaeOdwj zYh3$yoU-V5<($%`fm5`Lr(3kkMBvu@(Jm$K(JrbFqFvmz(Jqriqg{TQgYS}~U1l$i zcKK>^v`bD}v`f#zbpt-@v~2K^I+nw`*0UVGrj6zBC%r9)r}VQN{+5g7@ZEzfhaUhG zL-9Gpa=2ZDy48~e$Xs= zwoM-C(K~tMgFeY4Pxen9d2wL!Nb}LjBmWwoJW@UV*O94B2DyD#oZ;r;@ZtMG&F_!; z_Q(0|zy31c{monR-M8PL@1FHwzPsHU3*3h{T;Sfj{Q~#xoff$7>9WARufqcOLBQOB z3*4t=wec8uCVi~>YWi65TKd>N_tM91u|7EVr#1)2-e`Yt>`frw_Tbpjoez#3=y-5! zNWX()U4|VT+p5XQ@n*`C;~OJXh*L@Pse&?0CNOSFRMD9UXfOwUJaF=Ui({ndR1%h>DAlG)61rp zr`NlE@%I6qUVplJdM)3+#_PRZYrN{GukpH^vBs-M)*7!4r`LEb{C17k5@6GJYrOiJ zt@Y|*xz?+W)mpEnjn;Y%+&@Zn?e17rjfZ1ZR*%N2Ce-m%J?!YIO6u&Xng?X}^HfC) z@>E?L?y2&7-%~Zs-BV@m>8Z+zTCLg|zgjhW#%k5dS*umG)~!~3mbzLM|JiEQ;63=; z;nk|c8LL%YPpnqGc5=0!CI@?m zE)Mo~UJ~p*5J+AZ?A>f(b@aRB+0oC;yQ5!9a7Vvl zAS}6~Uji^4_}^lbS<%sN9Xz)ZLFfKT5~5BSA9Js@g&dO*#j^nkj+ zfEDQh_t&Hc_-;xM$WKcTsP0vjju=0uR$I`J<~jb=4|r_N(S2O z^a$Fw+aqY;K98Wf$322(W_tu}KI0LT)h;~f$9He(&WBorS?-)BYF%%M!+mav6Z~$8 z-v-?2T z=d`knT+!At@^7W4_BdY;#9NH4~!HF$V z9|4_CZi!lVZcEhO+$~YN^R`5dy16ClM@3rH-r8wVa~h;Yec3WCDz|l7RO0v5qdVWI z9=+gp_2@_cRF8fFm_Mx^9aYmTdePfv(RY9}d$Z^TJwE-sl*)aB2NyFH0E#8c6imRDD zaR+NGQ0q_pYB08exGwe=up78iEC{QBvA{P)cuEx8*7*M8;ev1sFmn|IS70&l6VPIW zAZUQyK<|-w8zQ#Tu{{eka8?MJxQ|Q(R`gZ~g+O=wHfSq+zKHEi;3>-N22{WZ{Jh#- zJUS8z>;XCfw{X9;1Maihf$diO_)#0c9|*$d4!{lUTj2NNEL$mr3BY!sYfFWY2z&>G z+9-r$KyzSuYlScrc#JXy*a{JX&tj z`D40dC{MY!7VT#P%a#3y|0jZ3phO6@)ZkBG40`*{?!tf&IV=TS0IKt^iN) z*~Lx}<^Wqd;VwSlJM4R5n~fJzHAI=89^yeAzy?qO!G8-v$pb;yhV6BrGq4|@qW~x1 z>3u;Mjaxp$w&4cGR6$6`4J>njF2F~?rPa8-0o!fZP61j1Zoqxq7jp_&1ynD@`2-df z2*QWJSs)UhTin38#?}nJQHbqgd>#jU18DXM!k0j!{rL4K!1oLE8_)pX)%+YhdvRWY ziP+x{Sm19lXyf{(3gHIefc>txPqwJBLYRW>25dh7>H#P4*%M{z0)Hhcgx=E>f)BRa zfj$We!8abiPXK%urx12vdmqTf_f7HlBy6>~y#LK_IM+4dQ*{+W4U`)WOo>$p#{tb$ zg|u!};l!!{k5i2Y4KQ^2gVLWt?45WWJw$Nn&2 zKA`TP5blkEua3nVRse4x%me)kI04&$D?kRyj0743Gl1z<`0-Pq5%Bw)@O?lBR0F&+xYp-~jfcfiBpe z)IuR_1FkejA7J~vQX!ZFr@8f{RfW>5-hi>2iHUjT-#|`qpS?sq3 zo;pG+*cRctcCHdLhSDby0wAkuRabmozkdW=0O~$LU6S!5(ItYA z3KRl+?;(Z&w!kNU;hqrSFYLSA$L$5cQGA|>tpo0}aPfza1E+EC!FWH!KHwto20qUP z+<>F_dpz(M=&nJW%!F3Y3W6F)2ljk}`T%u+ZRap10M&qv-y&WBPq2R<+jlQP$3SiL zOD49rE~0ONEr84S=)Vgn3(Ud(cR&+-|1EAUnU5F}xe7k)gu99wqx}G0A1yp;gxCjk z!hZKAxb+fnZi;h&tvR-ru-yRIE=CMqC5U53Rh3c!xKlSpc@!zk(kYjl^~s`+-s5Q(ziyL%D^&Zv=W^|Li!7bHIndA|T>J zjB~&)pc-)I1BK8F+lAPE04RYS$Dm1_Lg))Ue*)|We!*v3;12dTVB0thu>eQ`+<|H$ zVlI&Gjq?iF0b#%<75o%v3QPosd7-a?uO_08u>Bs0d|wbcj>30tg5U?dh0ossz1{Ip zJHFpGS`d=3y@$`sfsy#!G7z!lXB^8O@yHGF39xxzA^bB6F&W$AKz;0c0lR=z?h3&b zC<1zpMtgQZpK*v^4F%!njtXH3Py-kN6d^uu0_<%S!k0i4um>-p`x^)VS^`&lAm;Z3 zC$`Ig=G}1~yD5Y@Kwsc5N5oIy0KN|bUPGA>_+`cd|Fy(+32+Xmf$tCN1mO?(>R;Zd!(sH_ z3Ox9>9Q_G2#r}QZ81O#OaV0bc)LMmp$95&q7yFLm;Z{A6o7}~C1egQvz|=d?*lp+n z+iYMf_PqgX;KHARFc(Pr74-+Qf5G|2_A8vLIN&vWo(A*>hW#c8$v|!36v~X6i`bHk z;{%t0v-ms@coV3VA_xP4C%~6LBG40kb@B}Ijd9T5hv39^J+^K@9`>JpfPV0V?tuVc zCq7RA%mLL&)MKI`gd@h)Mm#?CjzSm&BmrhX_S?`B&>wgW`06c%5C`-D{;^gFQx%8@ zz*(RK2o~_fAutd)3?SqPe*m@cx3K_*ePMM!h2S^{^+%z{z)+wrFdf*A@2>-sf%Ag} zVHq$1XaT%a6L~RmoB1DMoCj*7-h+TVoS#4-4VX9qWr0<|4}d$$SpjixVeA2J0L|XU zd9=p&*dGR1V1Hv%^fk~A`;owQpdDiVqT_<_DnOI;Ta0jT4)fnU_f34uYt|9p{40DeG@5A+ULW4{*s=%^o#53~jP9);gzK>tV3t=R4a zoU#8m>N*?SxA55)+WQXWW&zgNZ(9Q0A`ZUO7J9`wd~YP?96$!}5AbLNd{gyASAw@9twe6yH6;FCqsGQwTo+K7$p4^AOBY1}TK;fEDl^zDq%wX25Rzt^EbG z@jPNKFdwM<9rBZ0#8aRx@KX-XJ+Kwuj|c7p6Y493G@t^9kE9V%m5Aosh?pi-G4mb)5+%}9@ z_e2olfiHn5eBb*Y-{p8Amjf(R&;+(WBW8@jb|^k)0W*PgjQhO-1@H;* z&Pnv$3Dgnz31|Q$XCpVyLVd9P7Kp=sFW^xoVktg1euS9$Cvw_W(6bF<9`G&j7CvhL zx7HZbfY*U(ZJaY1!8?72yq$PYxo?E zZ7M#G1MUD$@VQz@Mm_}6$0&qJK(+_;jcrS8zr#2uV!H<8hb!V1D(!> zPhooj$WKB{237(mXF<$3Y=M40JQJvn@15aSmP3%ceTsA67cm;>xHbU3?}NT5f_{;Q9Qy?I#r9onqk$Cc z-;PGifv=AOmI9yQb92BCSk?%A*b@219GnB-Z2%XGgdb;PjRhDD)CbPZLM#Oa1J!}! zNmx^U1AX2Ax(1d5_*qHeYv6NyKMn8z`ZmOR2T&jQ1E24O!&f8Vhu9X1$dQ45*iXfG zo{@;B*dF;9Z3b!sd+?ohC*lTh1gNzOzfT5i1ZM9>-LdV3?a@8ZBX9}(ld(Ox9(@AD z0NLv>o?)AW?Q7T$!*=mn5IH+f3#e7k>aMZbygR&=hLiIDF3Y*}ECuURv`n=*MXz*B#N;qFxu>c%0L@27P zSRkse6Ck*X4;qT6mI4L4%8QA3)aw1pis{g6VS=u*Vj^l5C@e*>zNJsS3;cpTPvnhS zsRcLev7Td3nW+IceA2=p=)oxsdSR0A1xNsFEaF2(ao{{3GKv=c zsv}P?Ep+zCB6~8G4H0S~2?cB_DqvLkiK-P81qD)&N*S3t@zRXS3X<}sR9=u&l~s8` zQcQiMMwQf&)ZkxvLGpCGub?tU@?s@2w@MC89vV}Xg3S9aR9=ug>+M>V9-6#n4+`3r zb_zDC_+a>4v0&-%U3pRZOaj z_50FJ?hgNFPkFQK$^{6+o(|aP&X~v|K zBx%N^mW$GiNilEKm(*rb%_wQcq?`rPj7dFgby0$okb>By7D>iz>=+}>n3S|wnlY(K zz^vVl2_r=CgH|RXCn-JK4b#tLL}V#OW%YuIqiH5aB`uI@DhijSnKBjOS6xl7Fli`J ziYY0mNQx=xXY892DopChkzz{Pv9*$LR8r1bDW;?w1(v8x_9hy80;QOeX1{jTQNqJ6rouA@>OCiW>kh{6BCMyf_>GCGX;#PYH{-VIaMuAo_@Tl#mT!LSG72K zbXQyhs8UCg7kgE;IC<`psum}2JyZ4KEUuebSF!Hol}=SHPM)}^;^G7F!&@r-TW{D3 zRG4rY#qG=Y$d`*7J=5VW31*_O%5*d4snxMqYEmW1V?#_eBhS5Lsu_7OyHsLQCCQVw znrcQK-JqF?M@vSj%!opyRtj{U3`|WB%!c= z!cca=E?wB@^@mJ15`{laHDVs$wuMR6BY#ga&4|4HvS~)->u0h`f6zGjc(M_Jaqrw=jzOIj1o=v%ma%{pM>JbvM<>=E6JEt65mmhu_5EJlVnV)8PG|hF;kCQ zXGzASBKDA+1m7k#ZLl|}iRJk0&6X@HdVaS;*=kC1FdWeoBcqzOOE(sUZ>1VDH5E%U zCN)`fmDFZZldUvkQqu@&#-t{#G-FcJVrj;trnAzFNlkxCHD+q6k2|0Bf-ucJNll%l z8Izhmf^lD_nX-?}uG!Lk9Q3x}vegw0lEHFGjA~oPk4zMH@gietJjI8M)LFuZjMUn~ zk;@4n^$y`fMrw}cLq_Ue!-tI2ex45*slP@yE+>HIU2XV~QFQ3Thm4}dY>+vXFZ9Z< zO0EQfiyQ)Dl-SRQNEFWTAY#!WUp^5th$6r-9z-k`_*p&?DgUv2B2v9w4?Oo#zGyx4_kgl@%Cj>6p2-e@c|YmqM#azL zK?A+Zr(tTZ)>CGqNb5u8(vZ^E%cUWm-;_&3D)0QB%)TLwhsmWOh3CknA$>Q(!(Oty zn8tP=xiqBh4RUEn+0A<4Ca9`N^W{rrDi~If2s{r7FB=-wyj327DBO`lz!co0x8x>} zcH^ZJkZMm!Cm_An?<2Vhq|^_j6OcyNOD7<87D^`|U3Td!xerK@QPK%Wi(g46AQjef z#+AgXsL&m&$Iq{WybrS~4a=p?j0&7C$6ORLq?tM3}!dSijm8`qH|wGgiQY zg#9RBQ&9njLHj=KR>FxAbIY{ft5Cfd^EQ5 zg5-@iDlbTW=QOC&LsLADtGpoj(y_`5lGiv6##0GpI|Yl{|DN&aV9?bR1EW`cDa}w6 zY7CKZMCNtFOgAJi+-SNXdF2Ds4arMghnn0*^4fi-8X#{q#83NZj@$B8vIq7F{!V~7)fm=-Hn!JOp2Q) z&6u=yTADGbtom36t&u65Vl7_}gF(n15q?nR^f~1&|dOne2O4^wr#gvq@K#D2pW~DS!rkXS< zrlgtArI?apvSDgJ1Tz@+L+JEoo2uMxcn)SgO*bqG$$3= zj+baoO6oB|qB*Il|3?zdNl~LFN;D@`b@P&FZcvs=qB*H+skdZvrm(MkB$|`TvVA3* zlhSvf%m~DmYU&%8DGcO$m zOnME8OPLGguAm&lm(myk(q0ucUN9B-N@~p_#pqW-P0=I!RZvqzu?u8qfo z$b(Dh{GE@Yk>9JJrU-TQRZttER>&*p5V{z5{8dm>bnBpf1+}J#_baF?O#4f}%FY;i zx`j$G!x2q4V>1|NH&j6 zH)DgldzgupBzIqFsu{U*ClNQ3S2WU&#IP-5p_Kj0-sh?-=Pus4AR5kt$f%Y_d={BfEuiNNRhS;Ag*QLJ61=HK#^|wJy0IwKijjC^ z78N3-85<(N6iG)WwX@%Plh9$LY}Hgr#)gzRR+2F(mi;W4gldyIec~h;lM*fBB^oo; zZI@Byv}&WV*6kAiw8W5i~oH9h~%R;bljrxt~SFz#ECswy%zs;k;G*(9Pc zSS|@u+YEUmq`F)3NJxDjPM6guq{0k&B&5bhGi23-RH>CmLh8IPkAzg}Hd9t}Xrj4A z9to+oc9N``ka{fU)URto`7PYJPMYBQGjZcs;oaeN}m_n4E2SP?4N{c~Fs@_wk@2Ip60&MRI<3 zE{AR>IS2EgB02BlK}B-D&4-G~xqS+Uz9KmX@Sq|&=Yq{}qaMid2RER;<|F<|r}*+dipS}c}LMEXA~ zpNQ1|`x4nTBJFSZlmih{{wGUi6B#Dr%VZOg>X$Faosd;i{mb_jyRWEZM(|wkFPj>b zzFjVbC{$l5s}@Y%AIYI0T_2D`L5i-oN=8FS%NjWpq~hapC`iB7t7SBVlq<@iAkBUw zkAkVS^%@xsA)Us`p&*4`21VDZDYWz*%U#x1c|b31XjJDoc?6;`O%4H5<__ruq{|`839e;j)P%ohYo}N5`VZ4PJBC9nQbLibW~77m2TklrQh~pzW~6}?rkargem2#N{Qu2ECbo^n4_8yo$oFTP zYDS)a2WH*NcIC=nQ{MV8Cb3eDjjC{!Pa+Coa!HtCK9xs8`uS2G390F}JQC8@+ec)b z9a7p*c_gH}2zex=!maX1NRvOvC1K-J^$b}ZOL}#bM?&hI0uslv1^E18N!fW_DolGz zGc_vrgj@l*}3exBTITWPOA3)K$GMT-6IkU!>Fc^wW2US|$0c|+&mM{~s;ca4(H%79 zTsbg>#;B|mo^*K0K0i99z+zr>q{?>LocfNG8qAB1)VqTh9Vz-YFFI1W;|WfEN6L@p zMMu%#7(Y4|AznMlsj(C_`tqWqNU|ApEk@!Ic;f>(6ZKajjX%#k5LYMJk7NFUV(lw0 zW{d=*zc!(`C>*JJai(PJQ&nUqr7o#zaZ+M~(^X_Ar7f>&aZ-}q>no_e^d#mN(+&;5&?y=f#wV7J1spE}mzC4}rTF^v#`Pr|oY zhb`@q6%{vnW@jm8qVR#~X3SF)Of@5qJ!YyId2aO_6T5^w*vV8g^5hg#&B&vVm}*9z zT`SkbwvmT>m~O^AeUYhVPH6zb|{X0QWmUiVA(=XQDQ9>6KYFk+$qnAfjy|5^p zsA^&6(Y4N3sT$|Cg~?~{R;@63YPSnjsx$d#c-0D%S01ifVe-W{zpqj? z=z395)e4i}eNnZ-=#7q-BpZuDfmCDWtt&1| zIx>0h@E;@@lQ;h(&6vD<_m7f}Oy2JOlO$vE{)RtGGA1>gkY-Hk`1lt|M<%tj|5cJP zspn^D#-yf{-!Kz53PUBZAA&7?Ctp7(jm^Ya8N4LFoO}5C{*Fn4d>W(TKITa$3ZL<# zW2(H&i;i?!{|cwRBjvjCq9ZLY=0!*9{)HDE>Ak@pocfL;zyw}&6cbkRqhnd!WnOd? zHyT{!)OQp~0zro#S-)Bw;VT3Br43T@X+H{bcP%5V*)8Fh-9eK18C>@*0O4 zv54U*pNL|`YWYMIC5q$|QGDo|C%ca*GR&4wL^0und?JblUGrr(jp9HG4p^5+k!Z`-dJ)Wenr>!PxI=*qb5Rhbn=>_UmugNLu2v|q@}%6cQq4)P zTcw(lO7BZGCvEn=A+hbG$URccNr!cdBp#iz^odfEx7UTY|iX_8sy?Fj6 z_p)_21tf#zk{DHZnje`c)W6B8W=x?Te8@1K9dg_#einFxzvmz!csnD6d!)!LuQ!Q->D?IFC8rg;Jx~+Nm0nn>BW@B7usp` zxq30^MpZp-R7PivAf4{M0$Ne{h(9fhD${t=QhfQAH!Ve)rX`&FmST<b)5-{p(X7s6E<|`61P>w>sXmcUL~-e$d?JcM zum4YWA5qM4kxxVsCQd#P#gk+5i70wBej>Y%C{}p!AYwVta`{9Q2hPhUBGo(mgXfVd zN=n%6U@p+RAFif^L;Ipodd?Omgo9$BJPM=of96Fc3ca3ksTEWIr#z@g|4jsi{Jx?H zu#5*4#eh}{4z;2vaF7QT#esUSai|qVg8e+GC>Ff;I)_?Oo{-6disC`fY8+}s5utu{ zh2U73oJS!Rgb2<6`$_94%x>8Gmlngc(qK}ysWAc+$fXbkdox+JV2Y2DLqSTL|=A?aLW&aKI$`h0$b6azl6;8sJ52$%Q~ zQhexP$*qPIEw1t-q}UNqms<@flH~Ctq_{G+9=94&l&M|6l9YnhH$yQu&|y|US7_Kw z;otXEg@LNV#nh@(86(Y)yeUPYbpx)oWYI>)lak`ik31zDS9wxW)bVS` zv6d8fzUN6vk!Q@C9BWCj=R3ZXY#m~d6~|gq{P~I}B}JgnMv98>HF1Z;g3%oIV`$~Q z*Y$UhRCupRRY;6c=hMdA$V9=m2`4fZX^!wAqZkv{luOMhy0}|&A)~lr{T3H8iYR$} z$S9U1zRjg(6h&sd!-b6E$H-=!$XMiPuH-^SG2;h5WE3qLv`|!hGv{c;2^J~J&QZRH zd4-RlRBUdH5G`ABpb-U?d>R%NHpr!+c<{Ge8j1uXZDjTh#enT{X-NI%tz_1QbnhjX zh7|v+TpH54b8DH6qS?uExiqBjV!1S=?8r8Xil6Ct{={WD+6F;|S0*bXFsl2Vwp@sy zZyrQU;Y;Kbk;ZSxCnA;iYbU#pNasuB6Oq!d$tNPM4{R^HX{7dT@`*_APk9iraeYh& z*-azOua{3ms<*LK2wf{$oO~|Y(yu6+48sZ^0V~hYsPcL82t;AK90I25ucZ@^nhT^8 zkcw+`l-!A=-qz9yNVP7~2}rHJ(g{eV3DOBjotxwkFjamjoq*K%n{)zFp_N@FbMP16 zv7s&}w{`?cg>&$#kQmkJ&5ukJ=I|n8sy)Dmj1+u_4;iVsQztGbfRwG{Lq;m!#)pg) ze}NAfso$zImp-FNFqRh?iwYa~kWqyAnGYF7iveA#6fKN*43?d($AhE_(V}7!W3<@E zk4zMP;YG%xMRj{F&7^2y&xeeng_aK)MT^aR$S7J|6ho$-&rLTZrCc%H zkW^vcO@AT3Y$s=T*{M)1_BPRe5Q3lKsUvo0{-koU?{Of+TgS>Gto&zes&&_o=+cah z@|z={L=>9zkXaKZ%T4k~NTTjNWz~e_+TuM~BqZHm<<#S3Ak72}%0hUb09?-uLB^ zkkmK#mQ@p40Qc%6i-bZzqrS38C>*qR)?d~tOMs(s(@ih}ffm{p%Mt%CFvn#rspdun z_v*)iMiiFGr(r5Cl1oE69^GGNqe#hT<ivVro?MpusXJaC5Fa z3Z~@W6FUz4I zl{yZW(GXJT4p87FzZJ{u<=>k62S(+j+s|eFvXN1h#jcW#Md8O05{;P>zaA;cm^8TG zO_DLGZ`1pdj7fJ3M@cd!#l^WxGA6BsjFx0fDjV-1$(Zytbc{q}rmXH`B^i^ZT6;<| zCN(wuKq1&wrl!)Da$ozf3PWUR6Qhz^OEndRkUH2 zjdE)Ks;EU_@T;L_G7NeZ)FjchuY#K7d+}9JleDcSz5)*+*^hb^)D#k?y$Wgy7sp=( zHH8v$uUF95w6N>^DyS(0MZOAZ3Qw0nU11`?-qO+^0m%nfQq_Vn^fXdQFcXDQrkk;l zG22u#3I~@=H6y7v^)|6>Bx65Q%}An|rkaubEPPC?BuOjLR5OyzZ>F2Ek+++#iIpUG zk2TedT=@+(mSrj$X-nVo%O2}1{VLcfVKOKN%A+tUWC1TKQP|0ciYe)99#o{RJRVe} zxTid*NQG_uIGjOJ<}e;qq}E^_RHWdIJg7+3r}$7YrRVXWBK23F#NiB5MCb~t?iIZyZYHW-GtL2l3!WFqBO!>|HWi^L1KT{qFsl7xV3F&-{ zMpko3;k)ILkhW_F$f^mcdVxF=(sPkq5~k#SfwG!I8r~$2gw)$7sFLxV-D4C4^%{Q~ z`QPLD8cxv`RB{O~D!f22m`kmgs&zc5NZY4*P?5qrhHz*g>3k^?=`L~+38 zW7$L$1?-|^6HyGXkCshD5y0*f*+iuO`Z2PJNc~|`WD}A06Q*(?V#=QqE1QUPuZ@#U zM5@0Mk4H4BCfd^XUwo5L<#18j(5Ui%i>-qK8sN^CeoCWR?b`X3MArY4e6W3Z~B3IWlTN`mB{KgMt)#85I30lUe1D*?G@Z2t(!57?nGl zC!Hwl;z!36eTf$xsr!u-PJKs8cjZM#s*mGEM-gBXFFJ|_r+Lv)WcZsG9Yu+j^EjPZ zS}vQ&i;kklTwZh(Ns2+&x3UmW*<8EXe7uH0E{QRUMDZgNg>$^fSOjUkfJ-weekAiD zqv%m*A(xs_>zjA2Ny?Z5DB9CPj^l<&4#8&Z7Zl`?5a>)YkiFqQXRC9^gLeaodGW&5wj?J!kWcIid5 z|H0UVB%-vjQPuU=$R-hmwsJ|BqMhWCkepp@RhI95N54tX1H8Z1P_scOC zg?rM?nM$44Nvu5SGfk>FDbsen#G{iY7f3ZHHQteGPC6X2L1N`ef$OE3llE$Flz4QS zPe)2MC%ye3)tr{ZKjRzH}_X-axQu3fJ92!Wf-pqrF6y7L}L#;^d^Y~D)@%AwfDvAQb zwsL47MT8rms&Eh;fmpy^nB`LDO=s@g^wWUy14_l_#z^4LgGLmhfylON?rGLhgP816H(J|#W-p;A96aiGc=qM6=%ZrX8LcJZF z8cUI394|VG5UY66QKTs5MMn{%<4#U}N3(*Nyyz%`Wb&e;NYZ0hC9{SX!$aA5LLw+C zoHbNVVT>Rfc~OZ%1|KRGIWF;_qNq{AgNh0Cblk2;g0;aC6F z;V5Db`Qkrn9Yxj8{;R`L?l)iBKGYp-i7#gE>%XC=;c)C#z0gI@6r4vw$d?cNKqMq|X$$da^Y_@a)icC4u z2`JWB9g^GxiXIcB6Hq)zl|#T3en&b1X}0@e$$da7oC5;fnpFNG4O5{S`Buftu{ofr zaIL3mG)9HiKf;j?uh`;8$J9HQ7aeIij~5*&+arTh-;vHY@uDNuKj1}2F`(yBPK~9A zFq;=0#fR_t(XnXp-Z4&%rPvYAi;f~moiFh)q&uFQQsY$vrC*%utY0lw<7yE5Eu|>D zz>{4i_Ah7M_#+Sv=RstQCR6wlio!acge;h8CLfVhvH7h2$>p+~oNlTr|K&g9zR_Qzkz#68N}OB@ zQP?7nf~n`M917CWJvkJls8;7>bSG(Rgd7S|S(F?K(%VKk6r{X7ITWNpt8Zm=Cyi;n z29_(V^Z8XX~v|r2H#0)GpTHhG-J}&3aQ3yocmRpF=?vBc}Z<1HTl8V zt}E@jT0Hp8g9 zsYXWC2;WOK76lKf#!NX|q#2WTnq8DsZBox#X~v|V>X#%PnG`f#nlWkU2WiHnBGqL{ z)g~R~Ni}9la{EEjkx5HmNi!xjh5v}Ow0vO<#Ji)|Z^qNh)BUio!I~(0s}%cJgR3ne z!gWv#lt*Dy+VP(_QHjFlpSe&m6>5IrKt-BN`IQ3|DfGc_9H>aIO|EdDBK3Ox!GVgj z?0l626)8LG8V4%U`I9^@R7~})u5+NG7|^bO0~JMt$v3ccQ@)&sVB9TxjNMX%E4We& zjp{!pk3bZDmP5dleqTBP>HCdh$$dbo?k1gpw5*a&U>JX;6Oe9Gr4x`^Pe~^rjoy(% zz?9kSrsUH|dh9QqfK-?S0=x3XhXwsS5Q>j4&n+HuH3N(=7-2uc1C_1#(m3=U=oHa@<>R%W2(t& z4ykvWJQ7lG)9SKnLh22WM?&iTMIH&Mx1X7;=8$@q$|E85-j+u~>P@ckT15+5FFtEf z_V{}R1TGbgzt105D4W2j;x}q?Argf?JcyW@{pAypqSwhMB2^d4Cn9CHt|hy(N!`Bk ziAdpVJP&T*1yX)G^#va z9)TzvlS9B%{X{wesrfx~$$da7j*?D5>OCNxfK>aJbOKVVgN5WiAeBy*PC)A1C5M2` zlW$5VAT@Tdl-viTLJbHixGp^gb6y=P%GUGBoBeJ9MFrQTt3qK^=2>1;qVOjlDyG&3 zbvZPUblaB)6)8A^2bE!l%!7(loyUWU^xd=`hi)gOkK{o`nxDsqimCq`4=RcSuh-|$ zSB8iHDuwlEf3?!tZHO{Z9i{Zs>XafrhpBZDYMq^;Y5uCYd9&B&El<0?Vw%2G-mV#i z3pZblOHjOh?Z_ea=jVUB{>6s8BTEYB?qh#`b68O3j;y(QqfnjJK&fcyqSmqRI=cJ# z2W1`!3sVOv{W6c}l)mZ^rMuy<*8PK25mc*Cl}@EZ8CD~WuNHMu4p%Ar2W4*6Gy7(4 z_0?oxnnG{2l6&Ay>@ti-oejdQLuaoEEIh{R0U?P z&pwLU;53Ws5K*fP&0e0lBTyAh6;ua>WbRN#hw4;5+K9~c?1Q&T?;U~JyX+MGUDT>T zQ4^t2>)->%CgIeLL4WGJ)fzDz&5F>5DBVyFMaW+?>M-p0Q~Rm?gX|OoT-0IyDsN4Y z27SaFhI&L5tf30Aj#39{*+~giO%7L!=pFK{Fj39A2fd?ZG?`n&R6eMUT8aJ@bt;Xh z4hDsxUYTk5)>m(9=;BaqxK8P#7Kw$?sO=Pk^e2ca1EMzSAeFaPr;65SL!O^B6^Jz8 z_3`nCIrh+G)GQ4B_pCl+uf{Ker(R9?{A%KwtBLq`7W){#_G&`>)%ee@#>HPvTyQlp z@oK^z=EVu=`0KUw)V!pnc{5XScCIEYygqGx-t2>{aQxD%@kg)5FK6d=Z1L(ZlttU7 z7p*(^g2UfTPGkPQB~DrN`GTT#+m*$8mldUMFWP)WS)8!Fc*WM6OO~;}7B87yG;dnb z`qj$fX`71IExEa5t5G4&W23SYvZg~My3EfbL$pEaN!ba^H=@;Iq)Hd93em7G2-K)W z{9Wk_>4@q{a2b^JQHN-tpx!u_Dx3nGNP}Oi*eO7nQfJXe`@%^Kd*LJo=d8(1$T>tJ z%h?S{B`9-_{#s?UuTC4FE+u`a_66ztlIA1TA-+uUOlJMH!J*-Z zwfZgyQ>uc(lxnnJ%|d}%gjx{&>=eV`Fr~yF9;R11e2W=ktnfudP)`omgzB}$Dy0d= z(X=|vBm@!mS3^{=Q+PZN0nw0dxCj|DcSQX04L6(%M3JC2TAi;ZL?x=3{t;f8kE??< znQ1t=bXve`JW+Ov;iW=?o;4a`@YN~}wPbN6h%xCK%yt-~CRnAfCG!Jgs6d?Z&tDan zm$aJ2g3|g|UQNi*M*=85O|SS=7C90QAb8+!80$9b!v#KXMsT3BPUXrD6dxAPSyeoH z)6MCtl*NbUbP0uVUR z%wSE{av$bOzB=qgho}RX!1Vp1GB_0mi^#BO7C5pGA<+0{C#Yb^qDnXmzs~|lc$mJA zaoY7F@>MB=GuOirLf{(OV675X%oQ}qGW5<6Y&dBww%92aD9Q>Y=@_h!AqKD@a)5;h zeJn`N*@w`OJ2y8eXSXp%9Lr6{K%YQ^x{}3`LkL4N@i`vfVz6hihQ|G)IiKeoRpx-3 zg&CB|$Vq3VUy4R{3U^mD6W)-yKg654f7V(SR5U?>to>?H8K{a1QDw2vq76d43{(b& z2L&N{V7q07o~%6~c8U+&L)AVSKa8nLF)9@QB8%|T1hJ1uU061w@`DVS3$n}t2~e1b z%tJ4Wp30E$Fqqq78yYx?ZP>^6^q-)o{N$N=J69B>#^>At-Nt%C%#w z5u=b=awA$pLc(ZQUpqJ_?`SeR(6qe<&TWhH z*C!ROU-JTInbl;BJ|4QPx#5UV=z%CEVGP;XyKI%nIdlji7-51{0f=^pA=G>oeAJl8 zge${?v@9GNt6t_|>=Yr+A(PbJh&e2uaY6LZu>KCQRr+FB41tg09O)>!z1YV}WOJcv z4aNbTs6?L0vgI%>&VEQG$BP`mx(#R6N2LtX;&dVehA|&k=`grNXj#U=nl>q1rSnzk zke%U=VWHtjgIFoVre|%<{Jd18tmMFx-9t0C>Rk$nl+mleh?@9AykW}BbxL)JPCZEz zChE{+Rj4jAEev6`RI?Fr4K8^@Sw$ce10@a&1fK?giKq`@r-^Q`%>i9r)irIX0Ik3yqcKAf^gzz@JD1`uEdTo`}Tc&d+i*Jv2-Vl4AiN!SF#k1Ng^~z6R6MGFqW%A zF(@KiV`_+2(K*vb;v06t49T6b4KGQ-@X=1;^MY@_;FpNb5%5uccCJ5T%sDbY4>NjV z=9bKjhD4sFfaM&;c$%gua+V@Ke*v&O5WzJ)=UDC>W$rY5a}dcO;k}5lVqJJpcyM@# zuZjsc1j7#EwXfP2vDQb80Idy+LT_V2tWO{jU(`Vwov6{3!%wYLI5;bjq==>Few2gg z19&(DPE-3*Q$a*ZWQ*4hr97JoA`Cx^%}5V1jVM^3#$sceXGJJucs3Qhc60^`=BH23 z`*bHO*v(xX9ij@RWG{Qyv-pTz%#`%`Vg`a7vc}*S=85WH#qa@aP#ca`utdlZ0x7{W zM8|(m`;=IuVA&!@y;3pJPVpWiViPQk>X|!`;~@m$L+Pm>asxJ^u?2=wlfYiaS{U*; zgQ;P{hqy&KAsY_Mn*8+|fUxWX57ihlw-+aQNI;CaAM(VqrhEg-n(}QiCTbKyN=^A* zO!+YRiz`Uo!p=mG(lb0`PR4?Qags+F8>WFR(ehClBPULC?eZK!?*fj!Nn5Uoz_tBmx=7~atx zzKbsngR~keGy@8bEzVoHrf~Y)>lr)q;*QYtt>D0}yra`_UTK^l!~CWD|2&eMw|hq3 zv5f0WHnWAAxLJkk53l%V5CSPS_CS6C+HW0FyHqZr12IZ&k*G><8fGLg+bGze#+M2tu2%)q*|10u~K%502g%xW{gN?G10o1&7Mmr$crd*;pv9sJYfCH#Luwa{6$9TkI3^MyJ7s_Unv%h9(8qL8w$_P? zhcOeq^RT~!8B(pxt#*nJM`=W@4%$>1+GuPcYZ;4x%q8Dv%S^0Al=ft%8CrzzCR3aA z-1z=kyF|=7C$p3X*^O~#nZIRDe$xEBxJ}4k;I+?^7x<3e{)MzgAGctK1uMgf*0!sO zOZD^8Ei7k>--YRCLF%Vh<&d-969O<1eRfh<;-j)vPV$YSFSV35X>k_dG$=|w z;m>@6of>T!-bY!D-uuFXFzFE}7u#@wyQS%}zqAZIU2 zX>w;7k{^7LLGw@j?2^q_Q7QwePGef&g#X)-`y2A3AYToe2SZ`r>_R~17L53Tuna`b z9|c-)V)1^ET0|t)p#Zic!tv3dehS){@V6*PIe?|F{G?9{b{<9iHI#Z5j9FA{!Pdr( zE@fr(!PQXaS)^rU96rq45r$=PWr*5ai;)s)izqLSMiGIX;@wfSO&xPBeC2ag5`P~QWJ~`CDz>4N;;;-*rkTD4%E*th6?IGy~fu< z_|$PP1`kD7GI~k-yD@i3FJ89k#-80p3CsSy^xVTu>y1$WiAT7%G8~nPM3!QZHKyv% z1~DdUnxupqyP_-lFmr3Lz9-b^2(&D$%swO`8rtTOvlgCz7@j}VsIYXs!$WA#ryQkK zhig6Ms?I?BF10?YEI}Bi1uEvzNPjU|P(`V9NKNb%lS0BLJ@;sR03JMgwA)0tQ6oJ+ z=;bepp*=fw>SEu`t~37YLh^ZDz|O;cz^GonxDcWV8IK8r&Z4%S&(3fXo|pXp1HII@ z|KD}b;^<2~^Z#D=|2q2r5LBh0+Ib`!bV4vi!cdBPF*?#bGE{}@eQXiln=R8~62N9C zY@TVSaG)i1%1y4N@1TqYD>HPR5BW*nrWLs0SGbr>BlC9Yr-j|fh{jxIq)%bUh%9ia z@7awUo!G>2ADeb6J{qd_Me3^0XOPMu#qf>L255Ygf%+@?ooG&oEZDGIPHW}fnHi!X z{ZWTRDI<=rdo~K|r;@>jb@p(@2v?TAVI2muX*F8;{Qi)RV==Sq=%!NXuu6`3HJc?N z3o4Z%=~JOeYJFxzlfC}#Y#oWMv_DIVbjt7$Em9U-0n#roB0ch9H=3}nMplTT&$1o; zOwUf?>A_A4mNbKKRTfh?W~R~U?J~DAGySwqi@OCdi_6@Gh3sb(n5bbj8#6UF?PHht zUYyz@*BkOIZwlAX2tqX-aCK1|8j8FQD@Xb&VT#A@%6Z|O_n|!MYWx;s)})^qRG2}@rgs(@fz=MTVi$TuWw zI(iKk^57}Bcc$aBd6u1`NjVvYvV|fdXARbDFGguDt))8Fau zi8b7p(~%}DO5CsGhs)r&dBIN6*hxPgVm(ydi{u=Gt_fE6{{O#l?}IFbK5K5n^ya4u z4>G1CSlBVHylK5fB)3?4!-8dyR)wqgla#HTLql;HPSt9>-kkn5^?VtlEu*=$H;+euCcb?lxZOm7iq^hai@4%gw9CoFv-rublK zAj(d0L{S*GqTtxfyxHptGBPTx9+v*DPuq`6s8sga;grIyd-7(d(H(jD>ki_Ut-OpS zc}L>%w#>;(+MSoNC~wwW`ci-Y*9I2$494VX?jevCMC=Z4=*d& zxrv4#HcITJv!MUey7`Dv6sGd#?J77jD{uPd3LB)nc6c+*4bimxl+6VTmXZ}!GVkb` z>l@>*Z&-__L2 z;tU-F98JTpx`#i|WFG6E7@%VI+#U><7;1tL12Z#%q5Nmo5!vh2xC&yt#$39l z`XROq$Fjw8JG-|A!4(;THhZ}`1S_Z@R{3H`!Yz2q)xr9Js}%#Iu=L?(o#8GuN^UG+{3qzo?cLw4~9ZjP|xr24g(}^}{N#CYbF6v%51GRk$D4h6BU& z2O7XueeieIPIf;;2)l1Enq4ao!NR)Ady-lg?x)5*4!;CbQb%Tnb_H|hV_d$%a5t~AeYj^@F3d-bv( z?3wPV>8@x^^$1mo=(?&dqN@rCRZFERi4wZZNKJPKnE?cOiR_FFsLZahApr>lD$pGY zBtV4(NJt=oP=XM$D`DngyvAtxGJf@-OhCX|_F7){qy77T-`@M26B(IEH9cd`GF@Fo zWW+i9oU_l~-}V3fzjoJWBAQ7OVDf;*CvAqZRFoZ)inKmaDuYg8(}ww%AY|3>3(&zs zq{c>?$VGd+I*<21l^!qT>q+qkX^+hm_A6;O;FJo8@#?@m2xE1fr)v_mO2-me?Qb<| zt;T10ZA$VeKg%-_a5rHu5P?g3vL3#)bPIzl&Rc3DdRMaWlEnYNGQV$(wdt2$C0U~n z7FZ{I9z$gwJ_j&xgb4av4?{BW*-LM{bz$8{m(GB3{zw6LFr(`a8RCHOI)Je^4V$p( z!n!RN#y4Nwb_N-0-#>X_9B|m-3*$!&zwicMUVahyDVH4=asm9pI)eYN^L=@0lG)jH z6?EQFitYc4Y_2B}&P=iX=XMK<4YZ-c$15+NCY*m|Vr%8fuMb{(Y9iRN*LG~U`pV(T zmFJ(ovUl^f4M)MOetL4NG69&erImVR$9CDeU3OscdONPZbl8`Z2KaR1=#^s=l~3P0 ze09S+mW2qB$oC@Xb<<6YnggRCB{1caAXVVM6lV0~KznSo)z?tQ0O;9ReT<|T;I&=@ zPmrGhW6{3Oa#iZJHF9?65GHo{Y({DJ4;q}-#wk6zqM}`%`!1+9%|3*W?R7m^?5{ilEBJmCP1`S-#A%kJs|&`<$;!Mhwe!rcO>2Cv)q# zK;6b6O+@)Y8jhzcEu7$ibzM6lp)_Its*J%Ni~J5S8X>2Z#5WWpIVC-S@=6~PE}sx; zQgsiyDL~ics3vrC{LWW`X&WH&{OM|TA$z1Y>}ec8GcGL4vGEpQ$GKKdWpwg%AC>_b zi$3)9FexSn;8-fDXYzEVZ}LRn`5OCWamNf|1o zs)VQns)J>xz#p0YVE@=-LpweG(mpxeuZI#N@eU^`Uf=BIeLQrhUtse(trjf8VC z#PaxCSNEbxQrh?LKiq1si2}MZraQ!@1dZ?qcmKSnzul;ih2=#6RuSe!cjnbQx1ImB zfypS^@okxaC|dm5M=btS#`6zs5*w9PTxkZ}1nk9A{=-EW1vrP)I1CJdV3Mzi?NFER zSk3-Rf4M+BKACuWdfWDk`%mc={+_?n>7C=3-rO>^<)i$Ud4|Af+8aK%DHhrc*2kL5 zs61%`e5~LtjIC~G^Y7#J)++$6R@9c&F{Z(WN#f(OQf)I(@kOw@DZ7_D`Q!V+%d=1g zb4RX|oORn7dk1iu*28c=Ip;sQ|CZWvp*vA&LxkyiL2|C9dL~7|K)Gmfs7wiP2P^)5 zJP8am;GkT+9y8k5UD#~4Mu>6xTjp_Ph5#5?PdJMz7(Q&0U9s<=<|P!i?7A@ioaX53 zSM6n2$n#gO9C}^=_5*9R2(CS|n?Ldzma9)C4gQCBa@zwkLc}(kuh`bt zz9)y-^?%>-H*yB80B^p2t>H0!u>jCSW9n-exA|uHLILD^i~U^K>c1CmA8bV`!qXVl z?AG%Be%nUob#`E2^+12?vvNDx(fG8?I|l@ojgugmVQxRt-gR+c_`|Klyv zMz)y>ud&%Yna@;K}kE9Qy#%xbpC9~UefSg?EL0QU-?sKqW^roN4Wop8Qw zbTaq~5#=wpn*FKI0a5zyvc3S&REXo=MU|;%w_e=5X?pizP1iTm=`-~J$Ga++an_>8 z@a$J1=@<^Q67F*W*d8MKA1Z8Kv3S1An8CBV220B0;Xku*`w|Oa+gOSN2-l87Zqp#= z-2@_9PM}V9Sgu~sw*5i-Ac##)0&uTpKh77S74|iHs}wi)B$hhX z>%z2NRocVT;(f-EcuhIlT<)q#D=QN!Dlmo!NH0VduNkO?Kb;Iz-AAs_k{k=(q4tdg z|KsK5FZC9cSr>Sxye;ljpsRAJPxlRTcS#4jBy=cvKz+)iVZ19xw^72g;p*NEUH8?G z!@jCCcoQR|%Nv;c#5)@T+O}2^%nzQ8RtHA%D3Ju9a$}O11pLVY6g29e#y2R?T1^+% zKbp4S6$Cv?V8g{~S>ygq*rq3B^-6+HAZ8J<{4rtBCAS&R2#bW3X}T9ZY$kS6&gYL} zP(K+&FX-kv;+3-K~4R3I53c&^qrOmTTOIP|D1O0|OBHoV`WjcNUGZKR` z$OVMxkIXuZ8~&%Gi-UFdwdz-Y_jfA%UjSNkJ)$!I{^nw{4{nSTL}qCQZ_Lk)_4EI+ z{q^+)QO)1m`~6D#BWQ{5R~{ET!F~17rIp7W+}m23-M%0>afnCGN`Ja)4`(^~0w+4m zdujIVH2fUbO4)p!Xt3|4*>@Kpg;VVs7v%PE1Pk9h?syprGMoNq2jKBvFN{7+Z4;XRUA=MEi{yfphWW~QiPgJxxZOxH#qwav+vK?e^& zSUIszFp@yV^`+T@1=u4vtFBJN{FYQKL)=y1rm$0q(Y>enZ7i5eu@)350~eGmQJ&Io zk7H6Nb_*_`UoobS68xns)7 zf)*j^H`H8xb{A%2ALb#sKJr$8WZOfQ2aMzx=Z*|ivjq#=Eee}f5@k}6|2>WV<#+{H z65#jA6R@@TW5qJUC6Vg_+o|2MvIfeWK#@?sjB8#SZCn1zs&YM(HPYmwWUr0=Lvc@h z@2Gs=5^4g|0y>9`5*DtKwc46IK6%=q5uWGTEKvgIbxbhU`kpu_c zIC$l`y)$VruD${~X2Ooxw&CiowO0;oHzWDwiK|a;D8!KNd~~#TxZT%YRcVd&P>eo# z?A&;Bjqom%miHL91x4zkt&u)nt}Out?MM0SY!~M?;PBwd`K7L`qm7uw0y?0Y}0ukIn8XyH#IzaXL)_%CXss*@Tq)O`hTGU3tz z0<@#GF$GSY31O(eHrm4DBP2^0$#!pJkTBGMgEizk&-`*(lPfmymYs0HH}>7rJI)XT zQr+98!@h?<`Dowtx+7o+3{UWsHvGn8QyWiw^1+K~&liu>;<)qjz9VTE-`DCH3P?XRMS2sPdW&a z_-0h;Z?DBeeh7&uXiTtB{#RIlmK>KK70Rx! z9D{O4Zuge`S6<_{PWGM7gNq@{6t{jD&Wv4a;JF}f(L~~q?u&DX5#W+%lp+_%m*T6- zzb14Koe`1X#$bmSdid_2H-MOiQ`{y%&R1$Clu%&6C=~FW?T8jjUr@=81<_~x8IfWq zggc#cXLF}0GSUnC5<^5qNqF?)*-=T75%nONEd6zizp2eu`Yh1!aL`83ntz5o>5n3Z zyk!3hMO0NognOR!pD$PCmv(RO$E?lP4%L8brKVsQxP*=7y|{AfF(G815tQ z3N4k1uWD9BWK@0{zn$KDdg}DHqV>^fZge}oXo0w*%T10y?KC;g?&A5*?dz)#myHh| zTK7Z=XFJ^`ar$l+2T}ic$Nm@vBV$3*P$(mcYi7udXmcQ1{N!c@SWUZ!gh|qGW@Qu% zh;B9q9B$#_@m{dol06|aLW~V&SFjAzwc-oN?l+DsSg^n^ti3P}@y)TR522*P48Xd` z!3)>39AfR^J+s&WR}O5tw(c;y`0B5YM{b4Pr>!UvgW{R(R2+5~3O}0LP*~&7LXBaC z(AwB&U#&tBJ;XQ8w-6r?nkT&q5-4WWWBc}-p};n^H$yr-?$+Xa7_;~A;Pnj!EPyV? z!Mn-}$E~RNTmEvsy|Ae2%@)Y@J+*qIR0r5rKw`-fahJ&iSh%>>Ql>K)kp_wt$X%u( zmOjuE3L+|qa6yxC+n3eBlfb{!=WZ3EyI7^w2)mJB$JMiH!-+7)*=)fo=d-zz; z`zZJk(S+fk1$sPkFAq=dURDQ!C7`+@=Th&{(r*bXOq`rdWOvO@Y#oa{aNwcV$J*fD zP~8-!Gr<^SWLO8A1Cq?lzAHhwhiJ|LQ=rrm&e+YlhF6By7^$!z*vXG9sWn;+7#ndI zFlb0z^xo1&`-E2_-9wRc$N{QKvAc3Fdls0m_h>PMd@!RHZWM7%!VQo|1F9?@g3AU4 zoJkSb&mDfa--ypC)Q>%C5(EFJtl@vSR1Wjq@ z>#w8jz8K#@#C&Oi=Tey@u=AJ zt-G5`GdoEdWNG%hipoZHQTnAcvP;^f*?;!|E1Y~^tg$IrP;sE z?saCmB91C&)C_O{0)~Mdsam;rITAAd2{w#8o+o(gy$E0I{K%sO&=}ne*$AW^MCu%e zmXMAHw-#fb4MF`h!ET1ukzJ07muMBPjh=6zGId78H2XD{*F_6+H1kNAC-|{W#LrY_ z)5AuYdVPHQmEF_3-|)jY2BTI{Pb|&Ua{endjgWZ{i{z-Zq}d{ALX<(R`JN0 z*>bR=%A;S={Vv_k#6a#j?@Q(BX_$&YhXuM-9lKK!K`44h$Bd4@PC4ag$kHpU-&?M+{M2B zaINgAo5JoA*G>T=NS($?3tSUu!VC?pJJ_*5^NNgW53GqgP~#+B*3Ax;@}I>_NM*&b%*TlC@J?e(;PNf`^FP z5T#Il6&z42kkK|&Jl=1onmrjNqlAVGbfXHo=fq?2E%bo$hS+Bbg$05kQHv3wB$Yw1 zc$2393K=2oZTeM1$**Im@^Jo$`kooaeoyix#prB$>_XV@k?7G(z<|s4Be{#~-hwW$ zyiIxv^S*nnOf}mYmqmnh8bv561B^&YBbZB|e@0X+qAO>GRC%o3VGWjZoCyEDGI)y$ zR0;=CgGW$_0pw1^K>}F`-^7BEz$(e5^64m4z<|7=G$-)3rAs5MXFX51ln5+K_jsG3 z*>1rp5y6${M#%Fl?{EJ+?vj5Kbx>Ft5fBvDhN3dm2}M2A!@6ZA`2Wn01D4E6!FotX8a7hbi8W3CMTi| za~-6(Di>C5Wi|WPH@Ucud`%FNN8aJ_607h`(2`Z6lvpw&fKvg~e4KS#%aaRipW42O9FX5!sFo=aYrcs}Ls7)rvvbQOEX4~2)luf2>N;%ktn1!hs^BgXtu=1k(#zHMkHCvbKp@diuq6YWg6$+{%2^CP6E^ZV=T-j`+(z+kT+~^RetHjU| z4Tb)9Ql6niR7)aYJDz;SUXNRY&3690lIp*5qX$+yDp}MSq4_K7? zs3=+-&vwFi&Q7%6;UVqJesx!=W?xyP?l#su#YbewbXbXgUcqC(c&C*SyP$NjHD>FG za!+V^;sfT$Ep<3Kxx+N=XKZWmwb<66+yW&|VA5!+G_k8yvu`rp{><`m^j=gu2`)?0 z)bU)Au`oibFQ@#s@4XqgpGu1w@w1I z5g_CNOOX+0a7YV~v;#W4rWDtTfC{#!csL+C>i1%ah?i8l1Hx=dq7(!{FImyRV0(ZJ zN%p9R3h?ut=g8w4a6e4%ebVC?Q~OVf3U}KZv@e?4{${?tVshxx+O5=5gyIGcTVj;GqrxBIB{*ruoW5a;5!`b8u>IQ0*5I zPJ_Ue#7dGXH(I;|`lWCooL8=q+=tuD0j&)Wk&I-R@Xi%a)GG<3JW+A6i>)<6Bt;CQNB+jjiQ)6dK9 zFNizKZ(TXCm0wA29TYj#p`)QxBxS$W=D*^ts2^0JmTn*z;VcQKzfFd>lXAj~dmB!s zs?2$>RQfg7jM4{bXYZJdbXYrasv#X88SSTiP|S#VZx>QPq#s$0Do{6gY=fmj(3RCO zz?fPf3IO$T1tI8=r1_EwD^lyRM$Zk|vEemF%&f}H%?#y-xF}Plidc1{5$GAKVqdAq zW@}Z>Dg*ow(@vJ>M=~%Cm&Z>9`dwLt?_e%b%c8yn;yTyIwJHkq>TZ>qC$6iIYD>1P z3R)Nr;GR_#sw;tRff|@p0CRpKdvqcBF-n#J;9)i~DJ!Y9Tn-agtp&n`%@h!~no^WE zrW4ZzlWzZQB-Z*dV#cW6b8N1Xa7d|s`qC+h_S-j{y4{T&_kQPUIAwLq9W7bnNS+Lz&(>#(1M_B8SRSbCQq!r_YDz++++_$MS8O$o|aPqL`S9W^J+AOU>pC4|An2T+<^)^&H9dAn9VBt53&8! zk60OoU$hu5fP67l*lc&M3-*@Q!hb;Br~drT27o$qISj$y*=%Z{OLHomK5IdR$2DNy znMs*$P4N1bgxp)mgGH7c#r@Z;6o@+5iY+%K4Y6omOvtyPffcy*UREUVQcA50)!L&T zzmv^KV=Cam%7c~KQ(+1^T^i!2BeUk_r!U?jv6=+c&wOApZN3fkSiMu-a^wm=Bur%- zh~b$MQyOpclRjP`G_#BQk=anGep_5$$WXLqf{e3bDh=wHZKS?xa>HT2jVtybIg?m5+njk-I9d`>ne%CZN3tZ$fLHQHcLk;|axR|1DWrQk z&Mo$1h=~?15UIOLM`Jt1RLkhyDSFkxeTCtQY?a-|g9OLPnpi1i5(MlVDHr%aW3HuR z;7F>~P8Gv4Ad9?I;y1`0XwxEbaT4mfMJA#pGh}4?TpQN$bw&xkab)w+(9+_ON(C_ zv^%ls^vG;wdLZh_yJn?0V*08OL0nN{_kr9hC@HMlb=ksxyb?qRt?3AP)Q~Q5a}x!} z2?>)-?OAB>v%J5yLY;gRz8F@k(bVGyz%HfpA_PrCfawd{RbLwkMEot#2vxJesg3I{ zy*=S7>GG=^rnWsZHL*TLHP9c<6bes78aQ$g`AOpI{AkD2FZZOAykQVQkjuMi6GbEF z{X|cuK6*)7t54TE_>P_W^5wnTr#5c7xbrl9o|F1=l?Aax_v^ikseXBU1J`qH2Wg8` zTJd5BcTSz$ql@jy9#%=|xlPLNr^Y5_qDM!HW) z0$>!YN6(!hyRPoi8UYGGD)A`6OG=Pns1;>#Y}g)H*rGnNHjl|(Zl&peC%YTy{D~E+ z%kx;sy+v^^7R*PevRK9)8lR0hNkDHyd!qXQJh;*ntqu_wgn$t%4dbNZ_-a@o9)LyI zV#X9Vj?Hipv?+3fLchw*9Sh@TEFbaZPz6O#NChgA+RdKL7K?8XyFEN(axyHAXt$~! zN380=kP%uN;rpR!3=hDBLgWA$B4~054Y@=OBDcao15g23O&}^IPGu+_5^rV4?Mlb! zrbM*SsW7FK!t`-VlZX7C*f<@^luk%osTV3oV~otzs_iquBNEGYykE=x%dv~M!;r>d z8UIf2+2_o9E!B!@RdZzfo-huGdl3}*(<+7?Iw+Bh(ymw7hgL~WWu*4qp*IgqM#BvZ zdd5Z)!A>eY8>F zDRD(UW!H{;Ob0TtF1@tl+SWp#shGOnLF!n|#IL=y$$E@OC7j+~n3w`<`AJ$o&CG>g zIriSwUE4)u4##7m{nNGQw_bT=$EWX4IM4ZptG{}_D+|=_?6LaT8iMc0q>^e!Z@X?+ z%%11iUf##tEas}M_g%&Y}YI*QNftz1-ppzl<+QYOuIT`v?7i*LHeU1?%wu=0toOiy+*lrmwHK)N5d^AR zECZP%D*yqF09;#ENADyNCaXhnJrg_C>`pnrW!G1|ls7+(S33RcwbOfEo8EC`YWwRX zU#7Nio!+pm!>o`-Uoic~tNDRpya4oMBu<_F<)%+gZF1Wn*a##bsy2|Uv@zYj6eT-lVh793&9ZB|*i z4cq1Pa&jWgWf+Y0%43A38v`_DU7;p}{BgTq9=xBzcx4Jp|J)5GZS+oDQhCQc;%V%C?=L zqprQU2{od$$F-fHrQiwJadpdEVj0}ApMtI>L&LQ-y<^1MVbwM1u66B`;3U6-=~$B2?Xt$s;(uB|aKtw@6&_3x8+|t%CG}CPbc$N(qw)OQP%Ck?bor z2WV(m@MA0FS!7i;`_6*}?s$^16yuuV&kmN=J*&Kdd-cIU(QYv#ooe%AwS9#%I|NHL zyQMspBL3(m7J!Cy?B=+Yv1UP#V(fal;C;`1V)aJ4awhMNZoATY!_O`zThzk?w;;c8)!x9}9b?n%(BxpkOM<*a+oqT>ZqD3T~at zm^Xqo?rS&b_5ydLibwM1-r>MAPq_1+9DdKe<;}fspQE`q%hszJ)Gydv@Sw4J8rcJP zi?kwk)t;-jcjexalO03AA+W58N84AkA9#am{%dRV%BwW8?X|mjHG;fZpqBOMu^3x1J02g2vk8RzX zAq5tqtR+=P!Ysn71!>BAds{5j7;a0;Ki?mY#0AECsNCE8JAUg4GRsp-7s;=|8FhGV z?z6;MTZQWW{^#uJo#x#31u^aOfBRj%JDiO36YSD|YGmJ116KAK!+M2=-X1yZ$1%4q z-fw3fZFe+M!>L=HJPdrCe;eZzGnrg{iu$eGOUrG9?)F$kX*C7k)uH!TsTji>j?W7iCyu;#(XF2JQo_ufg-b_X^P>W!DwdC>Tamtd zuH#-`PST}xmz8-?(@j73Qp##9l7YW5Q~1Mjd4#8zL3J9gv1#rP5pZZTr1V82pp10b?^@r)PbAn9~D z)D`OfXNfX`^5&PqXjUy&s0VXI*ts)U28WU@K$ob0m38IXjceAt8Vtrnx(f?XSf&HG=j6BOt|X8_JRaxWKwF07aDctFdpR zn8)c}Vm_BtfEw(TJGyfq)CjRMDsH*B2e`-nwmMRv6F3asLQ4|*4VQ|tN(x82Rc&YV zG%0JCc!qDQOde93K&yz;^BC4Fij@K&R`4=cf>@|}3MLT`3ylH=w>AXTnu zKJU-m)J?z6&8fV0U5s14IrY)*3+r}R2(*O1BLrH=YKEXH%f5)noAEpb?AvGe10avC zJI@fqWu&6AF9cJKpTdt>_hyda>nK_3^nQ43uNn6O`-vV~-GUTPULC9j(jxR8_`s}0cIiRc(IDluCV?v2$!vVjoW=XSxz zbM7?IT!!izkR=M$Jq?_f5xH?iX>Uz!FI7J@GK$Bo@NP9bmxP|exlW8tP|6rjgUl0( zQ8GelUEPK^q#&S#b?F;}te0&v`L66T*>D5EjSv?rb-Yjo^hOJ%Ih`-BKEUYKVoY^5 zGDE!Jw2)KQt*LN8yG$L@XXco*|KXDhA9kv(kktU@>cTi-m@9+RmTK`{JzcT>Tav0m7*;a`A2z zL$Ro{8U=;sI%R;T+TGA#!2CZ8Ws7DpJ>fJ8SDN0G zCef&tn8+*liEb6f6os5~Lo;fH85)?!Y!5)UAPvx+8hmHEKuzW3+Lo6Q>o-cDDi|8(6m?*Hi-*UYpamz37JGnHw{(*tE z{ma)IvB1;r!NPkN{X$+Hc`b__rn+T|-xlcEZ@u2gZkC8B{IgHMzl$TUJ6Dlb{ktXm z?l0XEYyTpdb${901V`q^Z2szuNWFS*FVjH?_ZN4$A*G5qKW0&#b1dx+>pFogqfYGqsGr8?qK4zY^EV;zlyq_f4(A>UfwxFAP)ZHEI zD7jLM=$2U=r_ePgPb~b=qk@CTrS+`dOxGj((!$SR#%Fh&f7ViOYF-0VULM^14AyV< zhuzG@WC>E6slcydGI1$U{zUXKnzYr*ouhd2M(=?W5^FO-J1J)z7I-{W$b!Jtsd(f= zqN{!KREqnt=wdHB0|brVYQdz!3CC~fyw=o zdm1Y&kmj|SY%_WSj^R9-1m_aA0`uR%a1A_gjploEz! zojfC1)X<(-)pDd(1RJUh1YYpjf^7(b78x&mV?uJER38kjnuod10##4A0hS3Ip>g@h zQElT(GIno$c(5^fkhpeyph5^tE6#aD)rtToHny*>W=roT0c5CzM2Qmy5cpTIBbKTK zhj5Dl9uj{HOPVNDPA2M)+kXpOHY#?C)0Mq61D>F|%A7j5Q%rQD0FSIMa1N_afv?99 z0$RWR!uUIe=bIq2S&lO4N~++DO%2O&oE3qPCB;e#-Mg-g1BE+Ypn&m?#o{1$UgE3o z(fDw0p~}-`m{oaNPsE(oC0ptVq8P)V<|d%sf_MYr8pud^J5zb)-p9O>+iYpMYkU}qXa8xhbLcu0% z7+g|Ofa7F;w|S;lzzUb{xwC&BfJ1|_DxTd~8QifBlH0*$L~#{!XI_e}aD%s>7i!X; z)(44YoirjVGMNv`JO5W1{0iQG0W5%Y0%{=0_m*SPf%?F{Z>TDAqrV+T;+*|>mCs5X ziB3B}e2JdXRLXem*5_bGJe9d3x%k0ONT0}Thcc^!Tc%EJQwAs|x;sj^IMNeZ9+Iuh zdB;vYI2%b~V7Lj-kczrGuHZ^AFqiO~jEsMtxz@V)-s>V^Jo5zKqw8GhrObQBo%>Bn9lO_J&V2RbX z!$=QZ=%MsniL1C5PA05?ZhVP(Qf>x8VcnU+J;7E^w2>dWd9G)_*(Xc@MYi_OoS9LY zFI3ri8t~@gZoGLWW#i{zNc`WkXqYGiCs8B4kF2q@ok1?p*P$@FSek^LFq0!uM^3(_ z#w0CE#}VxQ4E%_5&4^v5_`F^Lk$CV3DwIeWmBhky33P~dVuu+euA|ff#UEMgv{;5< zlmrBZjO60S9L&GnZW$t0+>XCqLHu;fC~Muuzl1*TI3v!k_!eLn4va7nl~yPmR({ z7j-;OweqlGcu0&E*v2p~gMU=bev{}A0hJcG4?WRWu<`2S^`1SR`# zPj^_F6ya`jz5_ z0#iCQ@*Eat$f{t{4RPzDV=OT>ehz}e|1I<0L`qQ>l!!;+1kvF8f-CC^Wx!>M@}_fh z941G^^^mlZ6Ud;S2d*PnyxbPK5;+yibMV{q8Ls=gUHc}LU6_=-2pU=cv_&YFm$eIrAqgC26CwXCM!F8o89B7P!xIuk>X0F12 zyim2zvTR}rWbzJGOoNfIl6F_s?2o)a*`2Sh1~i>ED6LyC1V$3?aX1c1kb3^1Qp%4Q36y4A6I zY^uh^I5q~#E+EQ{Hzljq*vaW5-dynBLz+Vc;U=Yi5u}^#bqrCOIXdmM0#a< z`))ce<07Y!c5vxq7{c;295r=xOY-zPk@oXuN80x>mCcn94*(XOd@HNot$Y26#1X4; zma>@)cw5;7C#tIc@9CtRDJTW*5(e+=l zJP#*ZpC#@_dcLwT`YDj+SfK*5uTAgQECoKiGOTSD%zO3IcOk9YT*9#x%*Gi&_b>Z{ z&H{8-)op96O@D(1!NNcJOwjhBdJA3$JcDX>_xaC>WskHOg#FG~6cRm6iN-=OjM5rf-uP-^;ISvo7cfQ~NMTdH`s(cY%I`Xt)On>A*CV;j7 z&hndj>!nMFUeO_c{8+8GzKSIjk`U=Vg zX7w4(k0)BpK0eqQJ1#{_*R{&pjz&I=OF794;@rBozwoB_j6z)8m z=d0}Mi^sbV%M?(O)ofnihtZ=r3Xuee-HjS{X>B#24A<(qtvDY{gpTt0h$|{ihdTQH z$l})TAo<|(qZhBOm=9!~VW*{fY4U^{Ekj6zS!lBfubC^Xt=rBXMJ5!{VfewA%3K{B zm@r_86omy{bEv4c7aRD@(KQ77|Q2=CqWj;>lzQnWoTQ$nZzPEZ1R1lmRTNUQ|KA1RUA+$;2DU zfI?w;(_HWDof10P{E$2uKQ(=1+tk`E2`g~PdUt!H*OW!c?no&LI5~!?vNddrVF6-S zMN$o_z@-`EnO=SkIgENABngFZzCV?5-bLFB$AaQT%dPu*ktMc@9D5+jGtKz@ZUJ#4 zVMisRALCu*#k%wiQu5Ry=GAOr3@)o>ju{8ajBAGbl_^&-f4T#L&MIB(S*err8D7*i zILiZ#)ZjX*Va#H@g<)(}S|ww11==DSbC7}FIkkO5M;$L2?$8%|-P@_d?i}II?5zM2otD_=#+z%-ZnXI!k{0I_ zz7{pgRGZCSh11-V0fxU|W_XEL5mHYn9`42zZQ0(-W{O2LN_N&VtX5Jy`Z`B6?9Hq#)RbBi6&f8&~^s=n|30 zh6Eq8K!HH~Bw2xR`}SjFhUBh#E+x`TYZ5efm?D%hlE3dVWVYO|R%{)YQjIA&>bX~n z3BBZS?ogZRm_OXzF<`dTuai2xb-f6EZ^(8-eTWI^CL_LD~h2SUZHeHc5d!ctj_)Eh0U+Ky z3|rHPz>z{Gsk&~c4KxY^N;;EqOjr`KfzWb#+jfAfXpHGi?_c_87aWAEt5$D)ypwAeg z>Jhq{oOOSz4^wn?Yot8{JL57;m*&W7;#Ug)ExOO}B?JMb18VK%JOE)p5mp&V%$@bY z_6Q_n9Hj#QQ7A}wCkV^Q{UewHphyvN-@2a|CN=OqjS({WSV;iutkz4Vy=^+M4>yd( zJWNH893ny$i|@b1L*djAiso4M=V}V+E!NXgw|Iqdto*n%inFkE+Zb?1M)JGK2doK zlOnA1aMAN&6H%ZDuyj7C$Wr)YC(^k33W;H)+TPbHS9WYSzCdGnGOmp)hn^SD(}A@L z4y7o%sG!OnFc#d&j11t?8LtM4)OyF&SE=}*xd#2G64O4iC=JR>F|3;-YI@PjD^^Kcx|+kK#Hj`Q1l0@*?o}CY1=$`3T&ObeV=^r8sPBh7mi+ja$ASb z7Y71RYa{V*97>yQp$21X*PrxjRw8+ z>8VVf8g8zxpWBpN3BFrRsC4o}ZOQf=W8RA$2NNHtYx!t!@mc$5?ZAcYmi8lFp=GTG zzgM~0XX@tLDmU8`>O*|Qn`t-kQ)-Px#A=)pJtAW3WjHjE4kET6+rhSoeD>DnxT(AD z2iyrWfKGMiQT@0`U9~_%!@%oXdh*b3_RcZLf`S_h~X7t>Q}oAE7`{x@?#J01|SN{l5Rf?q0x zGGahtgRl7KT|8BnDDTbj7!OQ7dG?(?oX=j(Omaml1C2h^2f|{RP+Q82heS4M?8fjM z10Kmd`m8We9(~qd=;J1#X<1TTe&}hTCRfe+LuLM*?M8oZW&V=Ufn`k`pZSZ2n?v9q zrIzk0y25|@ARskJzPk+N7q85JB-CAez52E4SF;;;T=mJnF~?Sob7NJ6y?SF+^xTjY zzxzA6JU0n`V^!Q(6~A+1Rgg07Flzq8mld}wyRZz~iARI_5JylfhKgwhD6O#_+LytU z(r+Mu!e1W!FO^rl_~a)aoWNz9+Cy>hVW$|BllPT)ibeGO6pETWz+X7{86EuX$0qmD z=CD`QMb5+Lk$0+*^1`8gmDM>ppN~%KVg*QQEsEqIPT*k0g6tnZV7~L>IZW7#Y3v)t zQSSc945m4o8Vr0v3`4X#OpFMb@eK=MbH4yR`|dKnHw9Dz;Tr?kK(+SIW2~BACl3j# z%m*m>L=zXI*ui41X5Y9Mmb}Qh7VZRSmg3TW|D^kH+5DS$^?UsT^{G^#CI+)vDhx`; zyzMU`aGmiI7KEX$CGM{rb05CLJ@abSKq02s%a%=7wm_wQ$GKlc3NG@spw`yFpKYO9 z!^Qg(h*H~JFlW`GW?Iey49E{87A${on}3U?rupY92>YUIhnFnA`>y#h7OE`X?f2WgrGVrh3jjp%*>?sAR)Q?HK&6FS z2%*%L@r`Jh6mbcK+H5J<{{nN zC&h0zk!;gRcLjn?r|P%+?He%~iDi=?if54bsTxzm`KCWb{2%KEdGep!Ex;}jP$EYn z>nFyD3%9DyNMheta{caMB2$7k@>eRU$5+#ru``ZhoBh>nNu0+>$lqJ1v-IR!mWvu{ zs5Lx$fmesaAOa@e7~ysF5L^LJ z=ZyOrjlVqwn-{&SXL_v1XF5c&&p@Dz1TU?r>2S|*y*4t$x2jk#aV}P~{+Y@63+taV z?(EG48hwj|(pGrcFe#?D>6R@>h}64_W)l2=Q+< zR`A~t0%xD}iSL6Df!3`4N<@@L=EdKL@|lF~6x=5oO)#xZn5FQV!S1OA=ei8SQ1H7-Sxq$s z{_B{EcCv~+@t~B3NlT~Gn%Hyd!n)UJ5GlIVwd?%iokC1(sk_lnCE@euG10s`UM%Km z3Nk0qk}B#e>*DB+K!oAfNuVPWO$QKPZ$+Eq{)g5y>A<1AM!q6nT)_Y-=9UdaMBX^9 zuKxD2V1(F=+tRxKE_G5%#1>&d)QL|s=0}*;;>*@cD;OreK33H=;lDV)4;k^kay~A3 zQWaPjI)-v3h)xsWg=n&r??pCwSy-oml5$ANWRZ-rP;n4h)$IP5l4a3_`?frhJ#TK; zAdE)p12Sx#kJ4w|#220UqS!(r-M1v5jz|-tKan&8GGEpZwWUs=m$!ht8^*Y0eb#^8 z)8A$))=<{qx=;p=z;OXPc(<=b7duJb(DG1O*k3+-dFOs)6@WC5g-EKY{lDS^EffVR zI{flcV4Ux ztR7T?kqJ~5!`3urJln$q^fW?Rk@h9$hS{{N-YD8iN*X6?2`&O!3HGeSMNzWK$pwPV z^p`ny9dV%~un_?9VkjqP2wn(piYB&XYXt@yX!cSn5DTniVBz8D&7oG2DY9Y8EsRzs z@V~^7q*Z?#ncJqNpT;^UKT^hzUBn23R!knEYJRvOSR=pK=q#E-7r^fJ)snE-s2lJn z8EQo;E*_qHdSal>GaR`YbFuxFGoXvID@Q=aXw zTzO$D8L-_~cLJo`Te-S(E#%9XX;)7k=YI`$x#74a#W2RkNs+yL{ObC30mPFt31zFJ zL1tK15u-0-s?P50sjsT6VJ-K_6>C|3g?@NyX2fdl8IwyU`nbtsOyWRImYWqSW2LPb z?V}ydnD%?@j5i{gtK4@GyQt(v#dSBJbzz)8Lp3OY;M}Xw&s&HF_J3Mk0A`T2tggf& zl)asucV^=EEwL3urZVOe>KGSt=0IXF)C6-8l7)XO*2V4}gGEbR0+8{%S+T2}mCSx@ zjqBb=sH*cVM~2eVQw;S>5H(!!{AoYPQ*2x)if`51TE;#$%{^Mp{@l%uBqRIV!+q?a zl>jg%4~PMw?QJZSMKsCVTU!Z1LT!vyx~kd7j_+fmh_;WN$!0DWc`RH!j0LR5aDyyx zOI#4Ise^lg`YrUBR&LBhtQ^XjKIHt%JKmjs>(!aABXTx$TR3zoHBHaH6y3sgG-O~$ zfQcy{6}p?>`&ez6x^`pN(E)B4#0sA%7~D5T_(l?vw_a~a3`$H&EV}z>{a$lpP@LNL zE@Hawh87fY)@~^(f?ygJsUUgwo|QyP@CZ7DVZm_(2Dwr)b(mnzqI(C1=ow{rm@r{Z zVVKVq0ff~xAOealh(zCiWc$9m2Zko!KwMZim^kf-O{_WPBAd)Rqd-TLMl=DG2+8++8V30;AfQ}8Er=$0&sG9@{NC9!g^CJ@v`*a1eXt|^R7 zwe>!kG4vWBX_1S`^><&()hs#cBC z(dB_4cd4s(5HfidKKhZK+Q)>;UBphA@ty86V_Eau9-fRMRx9?Zs5k=zk0HeAou(M< zPw&+-(ps!9XPc-yVYJs}P!K#qX-mh$zlI${Fb+G-z5)ktpr2BF>29|6^p?Y@zfUGk z;o@C-PyQgLl7Ev-GB?96uRl1o=?R5-PrrZ(m4}kQDpdas6^#{uvE`f_!0sVhMM?rY z-IG~ol1PpE~DCdQ*Z6K{Nk&he1w%mOASHx zW{M=Nbzb^~2x7)joZ2?d8}ID*=n%hRV2iHBK|H;FVO?1;J$Uu##MOOAuWZK*D!7dD z1+Tuk@#+&#U)}Id!v3Z56v2qFpiZn6J~>LPB%mI6c+J>~{;b z(e`h{y2u69OWFD7b7|xetPYnD>&W^us;?veQNu?Sqo@oecL1%I!Az*I+BP8O)JS2% zJWd%&?SQ;)VUowC1GS9qXgPfSvy`z((Sy%;k*U+8?!714BnLP zZRWvDin)N+GqPNdcF#{BgByf8j19-@Ul^XRa&zaS-yGUSa!`zm2QFFIty)al5AG%w zzpNiRO*37>&EV;E&LIkBmnNL6m^hvIu(H%e zrEp%iD|2g#h(~T$e|cQ`(HJ4J02J+V%3K22E;H{Uyg5;3ycI*+Go-|iAV@Uv0>|4(E?n!Osii{0e7toff<@M2-&ww4Hc9bOtCB}+ye+o{@ z42FWp%QwNv|#vyDR(``NAG!~T==J~^`m!b}_8*a9967cd^`sXgyg3BVvz zN4E)W{J|@iU)*we?^_HNt{S3AmcFJx-zCE)PwhWLDO_P>iIlZj?h=7B?pfJ{qI#fR ziVvlHb_xrH%aHZ3A~T#NKK3{1AJ-?otyMTc=g!dnecxDpHJ%nG&s+RKtnTd7eNE`z z2WvQ2aq|9|JgFstmYNVz^r*zIV{gbq950W%T=brg>=4W*X(hfAhK}@j= zM5$mpR2K2T)B$paXTWfm!u;dZ2qTvdSsrpNA*}Eq<})6;l_;1IWWV&PN#VC+tnAI0 zRG>g(7Uyz2ySjIKB)6s_n2NDdsE zC=g{^#Ef3Tv$&%3uI||MJ>UP;Y$KaAcS{FG&+Y1GUAJR|i|C7@nTev z(BYTnaBU2mL;{VUL(oi4?O<(ejXH;s2pzNtT8p?_r4I;m-`S09)}gpM+tI1@g(;`j z;$-~G_6YC>xp92$ta))^q~G)t!qf~)O8JqtX3JYFjnx@9ge|^ikgoq@cLz9v(Px&2 z729@oXV#Vx8AE5WZY8B7EDjf(U%AJyco)=?&nw|KLUEX!#P|4DDENtwH)4KDu^B-( zzd>^6$9b;AdWEe`f@f{m0A_J&lfthpRJ#%KkUt)5f^x?g5hCZ@o_T5BARt-%V}*C0fWK9tzT(dj8T!+fippQR1slzhact zG0$Qnr8B0V+154HG3pw zK@k6(i&dRJi$FB2T_VRTNZ2T3sY~!E;Yor6&;9U%iiweG_Fr`daOV%AzvH zslt2Zf@<+LD`YssSP#ns@BQsPtUgQpx(X;`Qno%B4foKFGwaP|zJpCF*LH27{WX<* zCrNJPL=%-OCpKK&@Uj`}TlQaj5k|2NRLa$t4$IG1JmA3EvNiv3NQd0!@RX*UCRHBm zzeCM6STdAt4nv5g9+Mt6!*nXjz6l@c4i)A77&ZF7^kuJKg1s(Cg3H9-iZbRDxu)? z+fgB2&c9!lz)F%JzyZcoV(n2OYFW)z+}9i+gCW5MkR?w9_xH9DsoHFVxHJBvPn^;h zXd@y4AYH7Q_j*7VyaBo?yKILlwMH~rW9{*M2j2^zTr;`>a4nS~+$HPcDkadJ!!gO^A z^$y3Odn&15@CSKb_(t+SbxOXt?F=?X_mtGSS%@n(WFeO051cEC9D;o1`RA|f-F$7s z(JQYX?k4!>Vz;=qW5d-~4$sJHc{zC@5;U4p=^!ujORp( zNb<9uR)}aV#N<*MvF5A)Xcvv+vE!8UijpDF!Irk zE_Kqi=vUJdzEUc+W$anf1H$>*-WyfahLL*c$VE?&JJh(Nzd_uDShYC6tY1%F$T}+? z>c=w1J(e9?;gF=4zS(w!D3$Lpg{w9_F}3Y|yAU5Q|LS>^goj9TqO~7;a61IBc8<$e zFk%bF3QFo7g|!*r(#}1Fv7CHOBR;hUj!nh3mM?C$`bJ0ED2vFuCpALlxuy2IBc7NM z@7ac1H8M!R80FIJ?LE7Tq^3axS|fm@RPE76XldRZ1W2}KOB@8Qu+vW`Pe5wfRQf(448`q-X`OVQ?P`I@6*J$D)X57TX~ZpxCfOA{)spcco)Oq3y8cE50k$| zN@6Z()!N` zxMHxXLPQI1#3sEf#A7WZ$3aJK{2k;%_T@~NAgOl*}#vQ^m}U&*TSW+1rKtI2vw3L zs2L1HE%xH$+dZgF2#0DWXfij#iPuGC{ns5z2z}woDnJ|LiF|NjJ&il?M8pjeUiQ>h zj5(kVPnm)-w7guDCoYWtIx*Qr&18Kf*F9&+NgNe&l22PEbEnq$SW+?xVj3V>a1$Jv zxccGlEBp4^0t-PbD3I_=l#`4J$i#UUoe)r5n@B>wvPq;QLbX2inmI9hcYk{9y@H-; z5tXV#EmG6=I^EwO6}bynBq$7nS?RUI8F_BJrmUk5AR9T{9P1qug3CKK?4m)S_(=vD zm4CDLl{@~;+Lu`Yy#Oz;g=gZmey$gcGlc9)3tK@YC5-ZyY`q%N;`X=eLEP|meWA0Q zxt8+DV%d00>n$wrE*P)FN^b^am$(bww=mNQoD^d>5DGtaMKA}(w1aenLP-3hlTKQO z67&{164m~Qn9mxxi@fh+n0CEIf}A5U`9;5Io4)58RlRCOBTA@rW&-${^&b z^~Q#SK^U)RHmAZ^=1MhjrCPjXgP6r@hC{2_b6IDtk|{DrhQOyz_5{!_YD*4H)NX7% zJScZ}B0Sl5YtVZ>#>B*>12H3$&P~Z404D^$G^~$b7(9DAD^@~scz88$-`_+>$9#SF6aDi`(7`)0L7$Qq9|*()}ELnrrzjF>D$$`){*S3~kwH zWdMZ;%Qgj8QlpkNj?P=XkGNQ?wz3^qZM+SeYz;$ij?x=V$U1lE(5^#ZPQ-rWE?@$= zhjIB~+1Xv^&KQNNQyS(`I=7uL3_FN)!V%nwU-R6NYIZ&&k4CaeYS(8Ug#NgiPl zkW3B<`SY{Tyi7T#C3g4tkAoxNaQa7BrKoEZM!n*^h$-VV9A3bDOH(ntaVVxHYGwr3 z%AMSgHnL|ZdZD_QE-r<>lE|)lP58y#NQ@3_H9MEK9Fr@oBC3%Pq4tO8bb@=SV2?H- zFa=@XLvu6Kd5c^hge5dZsN4lh3|2W~-I21#+8FxR}PlW zDB$z^CjEABY9mZbZ@S2gAa&*TsEe4In7|4OLaF2W9ZvnhsUv5mKH41yDBM5CmG4_F zx>xjg!BC9e{`1TmfLWpr+veJi38m*8XJtsO2*k?mDLDGe)r5K6_S1Kdmg2Tyb;X3C zMRhgmr|*7v^;bs?*mZPAc+_OMg@mE!4qrHaWy?ufUEEn7OHMbF>WBXOEPDAdL`+ba z4W8id(ce!cMq@Q`zbV1jGn)7Kf)y?-W&|30d2E!F9;HS_D~|5ECZx}93}r{*y?P&B z*=U$@m=Ypj>?3`5;CV8o) zgHqxa%B2`5%FbiFd>eVGREkE}hEa6ecRh!5w1Wym(7UVna_M#Arq5owX~v)xRe6Xz z{!T29q##%`%Lx5OA_5Klpe{vB{RA=&IPF=D=d)3?`Z`jiIAY z&!`qZ8ZKNY^is#+Eo3hPn{?#ZoS6d!c;_e{gC(m<%r6vJ!9u5q5K_^R>881rcdOZd zligP%<4gimTwY`z5L7E1nFq>8IN^Z8D+lpETM=sIP7)Jgomvhxn6ABs5b7m#5@fr- za#yW|_>1lbCOtwXep_0ub%Oroy0V{?ZC)9iPD*D28m#1ORrDAiI(VenBt~jE`D!Iw z;I^nElrh7cwJ-$t3&r$P@4q&+@%>;)?mr1ZgB8NGH^Y5$_7iD`yL5KvXk@jzj+Rd_|x?>@1 z)W#ZfgEuIj8Q4Rvgz|^*6fiEg@+GIC0HaxM1;rH>G@F*EudL? zy<*=GEfM_!Tzs%L3=*0s1TW)frI!KbIe}I5s*NJi@}Ksqm4#njM>*IsB7kOa7MXBh z(^)}UK-Fy1EY8itT6!hCAaZf<@#)QLac^Rj)j66irn)3q=E>~d9|xX`0)Uw5?hRK@?$B8=mYNDT zBA`R%H+^U);C8D(*~wN;D{W?7KY|L=+c-!@1A5!ys=>OZd?T(2`S?I(*n(u8^F&)* z493E1+mP)S7DGo3D-`N- zkJRng*wpDCVRn){~9 z*Eah=SNBGATzl@bEBU31+L{4ko2SBa=g+3LbF_FWJeU18(zGjkznZ1lx5D6wh^q37 zk^Tj(+&hI@ z1Lsui>Ezb5r2dzkxZC5KQVAcvEqnRUv5Uu#x>4-5vyS~vV(TU;5n4F~)7W%C_X1%B z92a;WCj|&2&`*iMdZOeu-hBORt3C*N5$dbA;lkL#Nv16kY*A*TxDlk@#>qaK5gTA$ z23j|R*Q0*WfW;FD4%1mS>)l@A7B^p zDsg6Ht?`G|!5}vwC#x){Erop_M6FbEHT%bxUVq}^t{0>BbE^w?k?nGE^Cn0W$&6^1 z=(wpPZ>Hm{@+2Lk6veBjb=+47{nv*k-#WLiUQtyD9;0pqi=6*2Cjh+0ZVXF=SNMqE zY>S&6;AYuRMN`V-?6o#Eufvu5hRK#!vu{3FR{=wf*cGP7;N&UJlkx7)?e+B_XhT~d ze`xmXgLabA$2bWXg@+0!sSZN~{T7ZcV5<@f;@`>c3bWT?ZCqHtT@Dwn*?O?3$1c9` zhUN{^W#XynT|_@g=oajcZ-qe$sG)Z^T-^)E^xaKF1E!FE@>|ly17j-BcB?f$%k781MRdQs@YG@@3GL-d+2nzCEijj z(3HtF5tx(M6%L91v0x7ndOWTj8N zR#?YhUEFSoz&?2o@+h>TL8kEpNKZ;F+sjBj>!=26;j1+NN?Kw1mj^FX-n;4O^zNsp zp4vCH^WfCU6W&~9YTGkY6YI53zch99$*I>~f*zE^r?x-y$!QoupXS(4CSJd|^YkYl z8uj39TFtaaj>bRvCS**!4Zn##i6%;-DD$4&%|@fOf~qN%1PVpl${SHuN>5u=Jn7*A z2NCX@~DJR?2wG6cW?Lk1WgzSf=6p`?QySWQt^91h$C9i0XUpJ;4bbeF6+5ns% zF3tJ6_B|KI-?Q+8?Fk|dWgWt{ut)|;2=WuJ+0GDi*VM+tl)05Q$~UiW*nNeq@zVBk zr2N`56KY5fQQ)z?gxAITD*_vq*lWiMJjM!n^)jIBbHT6K|nPQ5kt)1}J{lNxzm zIK#>y)1;zXbaYQ`KGl#9m=e(DFgAuz|D4NjbKn&QRfyMii2|tXru7gw2;~f7jNc&n zvwjNLGbZfW7udBuZK^#G=6L7y31_F$dgO?-18mCs#>(L)< z5Ma;uJoM1wc}s4MuZ1sr>vFP)S^+2%s|O%EnqyY(S15dBB|s^|d|Ce$H=JqD!ot#} zASD2$LpNqCZ6^0Xq=h}%W+M`g3W0!u#)_3AhH@rJKx@klna2yn?@-vW4?p;mZtVe$ z+q>EvUZJakjqYWc5sGWk<;03PwOE%}R!}Jdy2rt((a)0R=oV`>gji!p-wtrCwj~SM z`X~V^+yUFZD`J{rvCA*NLeMlM2a3G`(S%;g{mH2(y4;l?I<@w+uyEU7pW3uDVFs|> z^<wD&Kqg8bSsBrp5O7*ZQV&& zF6|juY^XC!U~Ez6Nb!V6)31H)UJ)OsZfa+rCcZ**6U*bb!-P_kN&I9`Wnt2-dG|Cs zy0Ws_3rxN?b9$@Uf0sGb8v|j9Ny_=U-|L{bZ%%i3vlLzcZdVp*O@db90W5e+q8MY{ zBQ`&x?Tym+O83jcD0cK{Avi`VH_up-H*5XrXj^~Xx+@wKLHQ0)30%0Eoy?*$4|p7# zT@NFJK8l+vHTY~M{x0O`Z_}Rw$EL?ZF%rSSKahCmbT$R@AIryA0(>(uW;An7eQYbz z9}|MT#2Vdo2zxxWh)aw6TeD6Xp6nXIfV-?l}8 zikum?DtuD{vipGDIyS!95Z%fozR>=0B4yQG0!9JICa+a1p8+Ai%(4pXNC(N+gk2jZ$(di5bp6Uv7=_QB##;@B zc@mJR-K3}SwS+?=USQreq|)JG_1%JH0$O6)Dd)!i+hr zW=FFnX7OnnZZR6IfV)JJ&a43NT{256N}@Jh_Dko%^|Lt6yfR>2#+xxXP;=;K%mYF7 zsq|}6@Rgfe{zciY8-b;XP@R==6m3w~2-fGIRcAf)0z3;7fI---eB-NMyDJ^cddA2DY*B?mw9tqsosjI8>D zXZIMj#?R8wG4jl!B-lR5p}Z~a{CS&az)H45iaQM zudl|%B8))Mft*D*cH&ITTg|6F9K6YWJu7dmRvw8Q9LXTUHo}ASnbCo1VwjfIsnqSE zIOF)xY(V$qpsrt0Q#=thDu**Ul)ULBc2&NTxwHs7Wl3P*Fnfko*K5~Ihw+3zwbwDo zb^oSdsGzt=hNH-&oa;XyqS3M27mTg)UL1o6*@rEW9uXF(G*`7GoQ8mbN4^c@k3VoY zJ+6hO)%FF+jWs z3r>z+T);=-VUz+$iU zg8LjjWhVptcGcWG6Ti*-7UcxDimObO0>>tj`c3|JBj)Wv zl8cl@$bQmM7X3spEq7F^XlXJgPAlf)oSGdj)nY{Dm2vL?8^|!*O)Fdr(L{^;$S7GrCixW zi0oRDO8}`D7=RR_TOf$EXO^5)bS)YLeY`_jxkRRo7C}b2}Hp&Qs#p(|< zMi|NhLPSA&EB6{}1oJk333QK>r(v5FAB5ot1RnPz*>2#3_{)NddSyuk04>1p;tNPE z@?B#bw8QXANSK2mt`o>ZxWzhU5aW>l_2Z}eqi9c=P!EI@@wgxl6?m*>wIr{>7_j`T z3X({#W`kWKl${~WPO;s2il}XIoK6CTwr9IAPBj=Afi8|*_E14C z7tzQutXbfSGt97_>e~2zp{~FgBZ|8M5F&i4q8cm;cKbjdSPDx)&^ED8W6QRn-thAJ zDqb|>-bU3Z^;@GuFdCoTsMaSPF;nI~t{z7q^njupgfk_CN`N>s?C(f#By!_ zU?!#5tBFV}dJ1BanP9!DaT2J5upuCmSjko}sX0dWE!gesu`8K*SspO=Ub91eD=Odo z>Q}$f`|a=CKEJ2u8{eMKU*DX+YdU{o>>IU0uUKv4&v$ZVeyg%_ zWMt@jfAS|iFxoT+h`mvP^(V6pI@|YE2Z^_4`@J$hTA<&!{i`>g_Qt!tv65%qDxYQ3 z-`KY|q?C0eN0Q98kZ#E2k~+Mhfo@nAH*BdJmLv2oH|*rOU5fvAtc#l$Ljy7&s&f}7by@Co{Z?2jjC!2fZm^FD*t#vj;-~$Kp2Zy0b@nX_tzK1fIvh<`TWp2 zt0NfTk0mVl`~}g?Pas&vzN_Hg+p!TFe#Zi#oH{CA&mZNK=&s9I=ag;;@h>xf(bEnO zklbueDWS!G??GNQjs02yS`yY96@9;d>7yM}zuXhPPist(yT5mNd_(?|uPS~2dk-ud zr14W;+B52BSF`#PgbRPUcTL}S1bI_@9&v1t(~Ds<)8dslaDFMvjW z576l78d5g#<)?@I2k9B?5fq+D_xSbm&sqdJwkh-WLldCd67o5e{Zhtex51?O?bypS z?itW_WF0G`lc&W+Q_cSPBJC6D_yrns-&W6;SJI2}*QeKV^VGWC>V$amSbmtlG=Gme z=4+4D`sm_yhU~fCp#h8(TI#QhERq7$7o-HwJ6*J3q>Sck;oiGH&abck5F4-mR-bnAk!j6T(V)xoN8iQN_0b{7T|GF)S2sK0Gfv`i=q#UW6 z3K}0z`^nS4D~2au1zbjw!EQ)nQHDJX85FqH_Av476(onnFT=>Rsv-+L(6H{G9w;sB zK(+fen5{M(Fc~pYC|#ik)xGd}K@ET%f|lJQ9(2Q zp(O^CDG1nS<$j@$>%)|Tt@9aYx=X8qle8E} z{5w&|fL=DDPl~IWJwr+MOhHg$@I(YifrZ_E>f@)?ozt;Dr!io*!txKXr|{hf3VkpB zf6zmG3!Gc>!I9nG_E>ER_{F++3f%qwu=nQiHC^lfxGkoNA%>C~;)p3BJyPSX#*hda zLW9PXs(WN|GLnfuTK7c;${-18MCP zJOdv|F;}ruX(cPwQ_>_R!$Um_->e1f#`YED?^6KFWQ&2O$VO3>o1dR2$CRKjxeTA> z?di|R_QcC&$_Ev;1||m4UBVS>fY65k^1^i>)de^nr0vjiC@X+EC=$1wh!kYcvhuC5 z$|iH?NCJ4giGOwLqSEy3oJ54GkF7{7jY;PEm~t{r42RO4pBhgT?QP;S2rc2#(&bC< z99^nb)_7xdwGYds0V~|B{4L-sAecDsDpe?yrx<97B0vU*3J$2^185t7QQ@4Ll`!SA zNc{>uZBTSRB(w$33Pvzjxn3*zu`C=EFUa%ZJaCj=HjRos1R{Ufmr=H|QCF#cSbX#o zfZpDE)R3AR9zs~Sncl!xsgm{OktIAhRc%D%n>NFcaBrYJfno+u8?ObWFO}B=I1FAA z15O#WMmaT2iKXN1F#weq4mGozR+=)2DpdR683-rc1N#rw98Ed7K@Ng4Ciy@+g|-xg z!q#9!qkYc{HD1}2z(R%ct4xV0*x1<`qv?blE)wK>Pi*M2@Qt=q;$(trRD+%M7az#`BgI0221Cmm*8lXI2@7;no1Km)ev6Ka%!gfL$1aw29eKuY$ z`I+b&Ce;&^JJC#_8{rw~{o{Kh4Yhn5K3f#|iJK96iHheK=Yx-l-a_9i4zYuWsvN(nA2)Qjk6KhT@B_L}AR>V%(S^%1q(vG17)lyxX&n&f} zU+cv+O%>=+vItfIRTo8J0y5`;3^Scmxnxbn){M&7bu~*D)W`-=PBjtEvfFAMW26>b zio(|**s4>PRK_2yNLX5t00A7NsAzHp1aMBz)fdE4HE$$xsEH0$P6twmWTyk+Q`Hx- z$~%nk5Yr%dCb$gIS~w$9gp)xvT`@(CWe+e{=ruEmjMpkarSfFe1uNS$R@eRcFpSv{hqlvz$khDA++W_1kd^3QU2AT@# z4%h`K5nR;k0qiy5avp?qeq8>c`IN(5Tk!(GLopEicTVfVq&DQQT5t#HL;N8je3ilm zWcLJMOtn(i^3S5Ui3 zFE^Buc5UT$@uE2?Hak%`gF!yeYpjrG)S6Bu0l}OJWG-@)BBPFlSq$b_=`CUVGy1j~ z){oJt09W6l^3PZ~VXbW2s{w`F@+T}|1t*5z94oJ?w|Fp(vD!JkG{MGlTs#4XMdPf*r1tK7z?Cd zHRGB*a;CEx>Dyz%;grGt4^k^w(_BShMmDFMWSRkW4(#SysYF6SCcf&JET=4DZe~)h zGs}S{7ZI@w@U0)aj_lGFFO8*4GksTR)Lu}6P4x-79N%F_MIASkJ%?gO+~`metDOlP zIusF9fu|}Tq>|LQxuSNkM=&-ZcbdkSsEk9kxCv!P^W^6WNu)Wm(q_mRRx2QssJwK2 zY1Xz9UcN7TNlhgM`NqOjRO-ztYZ2FT9JabqPNt4E%BkBpeTW!4aXN~*w zO^Sfq)gMZsBq9HG+)=e)pp~8>o|ddn_P|_QQr`sG&}v0We!NXEiG^ylDv`55ne?1X z7SZixaeTm(wx~C_p53x$pBX(0Z%t_i?98LRjg_uc8W-L_GZUes@zRA%PpcW2qLQ9q ziOAIWL88Ep5Jm@f8=~htZBSG=Omlo>a}ZY`9725Uh?ejIC-@0CQP_CO`AAEGRXrOH zVW^6*7N{S{kto>spF0LFOh`)f39VA?71jW{H#0ayu}S3#s+HpEGEQ~1Vu<`Dlpmi7 zX-hUWa)OmoD-aw=hhvo&**dHlhy#f`3}fh`LQiGLc@o@SIwvt)Os}I2Z7Qm_yT$II0+|sC=@+dXxz@E05i!If^kGTG`Z?u*^xkO4nW}CB=4_|b@)vZ3HrpPhstGu%soFO3`}w|th!sw!{Cz68ZF;IOM-kDO zODuJ`4x840=)l}aavc-G2#qOEfvY*uSGBRyNT^(WL54$X@;&a_zWI| z8O6+mU`-e)1yr49{t&gUizlwDumm83645uL zBd6>!^n-Fm5s@TEs$66DFFg`HjB3z0H8piYCU`wqU0!LWyAo0ED;B=UeF~5(70yIb zB$eegp22(oUfy7S1Exc{PJFA0B<%_28!bBKq-AF^?=PZYvgO5pLR?hsLI6*Kqdzv! zFr7dA1{AB|cb2;Z-KW|LK}ZCg*hD!87dcbKuz#`B#0^ZFmwJ0R&>l*L;t&v>$`C(U zlVzsX5Qc%HQ-A{O=i#6>iE6!wn=*8n406G@^x8YTc=n^CN@h$%NOdqW68rAd5Fv!0=V(68kru>I{2vFo-z6Bmb z+2(4!1!rKSXP42ExI`u9L03yJAAE7O4Y453#Y<6KdF*_d6Q$Z5q}v3~#yU7iuu*3Z z2a^?lkX6BmIWYG5Fu?$V z0Z~Dkl|T#vii)MGB2&XFCe|~5Vh>@%1O`hKHic7(+riu>r+=BXTU+B~Z?oZga|{Y> z1_b*;l)|dvozhAxq*39z5J+9JZxjSI=C=sd(A0u`C1(VyO++~IMb$D)LDrE0!T zK|zL!T{I+09y?!r>M#=Bw1zRKu!zMdRvo|n?Pz4k1I&hEB(Mncs3MrEeb9?BNg?-1 zF6XsMlC20J)fgElJ*$dCn%V%~okjQ!iz+v7H=AO*|yJowLM2xkoA`x39p)jgGl!#wz%c<8;XbBR40B@!upa#nrP;-zJsk-)M44J1Q2QxjBlXR(@AkW@z}p*2vC+U($}g6M_%UsXx;9L>@^kJi6x1ci>WV_ z#sC{1OV&XQB}oGmcqe$!D^~qn-I=@~s8V+9v5ZR==odVsyeg>=cvfXFrsHhYGOD(S z0QnKT889FjhL!Bp{e%Fbdc~rKF=B93=bo9lWNID*J~ocV3&BXbpE7ej9);D8gYF-CLZn=z_<`TM!5CYC`x!T0bzRL=hN(sVRu&InO z++Vz@(hyZENt_fWkT|tVFsK+zYavt))d)qU8G!bvhA2~vOeLW|l7YZc4o?jzc0eLW z^Sjn&wUUoK!Dx=iG7#dBhL2em{4LQ-63UD+!`RO%F-k3PHOCtkn~CbAmA1;b9Pqu7 zFf^JA^}Qsgr5AB@|j$R41^rtT94K@b*N70vZFgXeQ@djqi9>cT9O9rh_n%_=!@z<`?yg88}XJmpS?|C{@{P0FNEwDH1hmW_n*WZ28{XI_q z-8}qzocF;q;@4>AcNW*|W00IYv?c#OzCAYQky&UsusXABY z@UV0+S}{0P6}xIrXRC}A{AvtSwQ*1^3wC2U7FBC^OurN%CDSiOERXjPnlc)d<3d*G zVNJ&zzbi3jyy2i`$6(*fk)Ey&YU5dh-)f8GbY#P#C~i2*BxWUOq1&JA|ME2HK3oURxd}Lc0(;FK+|;HK49)n{;$9!Ne-|cV9J&uF@41nu%Dwk zujW-3Xvk_lx`Lyzr7_$@QLZjsoA)2DR%5lv?2b=|^+G1Mw;MBG$*n3+0On{4MKa%( z2%@t^!RNqmo}>;Wx3Q$}abPZhKa^pIuj)ULchW=!VyGFLf5?~WZ6N>x2P>RMle|Na zyMoYyFYpqk=lb@u*38@4zAi2kxhcOcZ_=v7yIrXQUV`zF>?yNd5>9c+Vx17xOqTKtGq)^q5aI z$zzc-pggtSLcYT)>GB$an^5I4B%8pP&BMQ_aSpQ=8SsUPge8&5cacZJ zzoNRr1oLMQm0I!q@XsScfKylk0Be$xTgX1n!<}B;DgCRu%o%(!1`5VRofW_*J)pms zh>^%H1fNHMH&#a=(gc7r2}F8}Kgy&l^$5qaT6$P`!Xrtq4&2$aMztD8aA?m4nlx-K zE29F?<{;Dspc*xY4BE^IX=$?;{`ThJ0UTmHewbs~7R28zY zg%{3CW6g%uS}S<~1*ZdC188bYdi>+bAS|>Tgs?+Q*^%BD!eIcc!Q)SCQv8)6mjwPQ z8+DxuuV7nvDV;*RBangPHL-_IuBr$pBPC-~k5!GMv>u-i^wz`YQS;y+LB=|6pcLlC z=&-NJQ6<-)u%#M|xQ!3FHT8G|@eBdD3Bv=kIm=rQqrSsw#rm#S&o)tQieJDUgX;8z z>Cn1HOuX3?edS5mQG3L!i`sQJDwAkMR9ryvaBFNzS)mDd7o*~DN25-~e%=&5@6q(Z zzuWK!m6JlUU^3i}D^X;H)sK=RxDhtLW6iLMR8drC{(a8O%X&XC}WDC-)M@N0zw^1S=r=Pq`AzB zrP2aM5~d@j_UtfHH@v_FAQxZq#E?#$?+Ln+{QZEC&hM~4%E^I{rqqUz-3K7O0TMK+TvV8By0%CblQ<^{uc}C4Oa4q{mHDy- ztMQHu0GEznOP9kxl*gxWj3kQmFeGNdjfm;4)ReF#XXKcIeJE!~rHA}A`FZmxj;RMm zP?VeS*UbzM(6H#K)M}iL(!70mAc$$UTB*6n2V9X;I81lpu4u^nROZZ74q(SAKNYony!E?o2G-8^u+8?3t!0Nd zV5gz0GgHg6mIgdX*-H<)CU+$GvKX67G9FpsBU0@_Qd>GaQRBxWFW zooZ`v8W2h>k`;cv6yOYHWeng-cyz>YNOhS>iRkY#{wbD%P8vJY;b$VThbTR$9u5Ce z9tj^k;;q?4GEflC!i&Sdga?WUM(T-pU+6RCg92K_-w-v#%B-?1 z81)OsoMHK(=n=~c0=E#L0-orxw=e4)wroyFrSyda$~bUilxO6X=0bp@%8*FD`O$Dob=Zk*{KNLFm0D*SEK&BY~1tE55c~vY6WYBv23vK8F-sj zurSSQEnEy1i81=X06mhRu}Ku9hlB$Y&8xE!aD*wY1?f5-m7!!;mOZU-M2XDT5HL!C zoyZNmtFoFZ(5E(`w#vG;|o*MXtV}N5$NzCgWd03#ABtV61?< zg}D+8$>ctQ6=f)l;tlwtbkp{d*r@V7xj@#IL}$QZRJP+-={D@P2_;99k^D>k zpt!8kq&+lW#t<-qGkCHsfVV+g%Gx6(dCAx*N>bA44h*S_#!A(T zd4kR?B_+)Wk)XW-Tu^7W1V`vR8Qo|8Q7@ezTtb4s5GPQ9TG=v zsd_}*rU%WwFvb?8GI`|XsU-R;!$8ercCs}~ekv61<0PnrvB0WDekA#(2#i6es&_}w zASr$&BAE&d)cNY>&j;xu`Lb{+d;a-&eo`5Ul#%auY<4rURC-cMU<98^n9kRiamvtn zwE2?Oq?ML2f+{kD(1z5KxK>({`G?Ah6BeB$Y zOS7O;orwkokR!NWu{o=zG)yFUMneymiSJn7dpgdr|XGjjV@00rszAeJs2?Ph%8bG=NY)5Wkj8GdGNhL9{ z@RSeIN0PPDJU1Qf^O`g1Sj6zK3Wrr~?2wFJ5;>3{o5W^8@KH>E?9H-1K_FGqIzXDP z5AcPv3L6Rb-%uTxGlCS}NsbZ$@^pj)nS38PdU<^`!8;Iy%OCMQ=p#;;>iv=XY%hcx zhG^boc9sXdp6N=s-Gd7X*#y(Xm+*+9~jr@xZAxi^h zrK%ye-Aw<>qeEk)YK2>*fBm_Fm;tM5h;IddeVEaZs+IrsWuX=IQ!%Oh*T?2hX3k%I z=iCTaJr$RV(A9^9i!c)Y(RIp`{;sj-;hA+ZMiN{?M z%)$kjnmM&p1}==F@oJw0DpO$1QqmK1R4~45?6rTg&ytwa;QTJRWeFbvSEzEUrXzWzov&T579V8?QCs*Bo$}U1@^!Fs60h=C0*No*S?Wb8E-aEYmUjl$ zjwy|tM(#P-(nbV@>hKU`fYn&W446!A5z%%q*wXCeYngwL`xD97#2O{@4^j;Sz{q}y z0$sTixt94t#C(Byr3{n(?Zsdmfe{lYtQn)Tk0>5*Q|0kv#M4Vw&Q2PVz7?5k?k`?c zk(7-s^?pn?3*Z>TE^7}|#xAYMT|>!p#yl6>X*7;=Nre|oaWsi%2ZzCIfFpwUhq&aB zk)u4J(PKx7Ke9vlX$aG4rQw&iT;2z@c@fm+>_3^t*`zOvv=+ooFzK_6>N=`zOSWS% z4&?47bytKfO#NuEo zOZ1`?8QN(X%yScm1yZu@2v$(}$rUKe!t=39_AD;RUQ)7jW680j#J^^UE8mk13mTWm zz2vu{iIgTGuNpn&j-IkLpQ5L)>Sp=UI8Mq|x^vB|#!6GRF&6~Ud|8)lJApoyq#qW2 zTo#9XZuG8fSAypKV4|I@);-3sK;1hmO^OVQ2qxw;a-V^|OnwBs8l=D@qAh!py#2Kr z&g4ekeipyrv~8iY!_RC9E7%AcS#p1P`@sRFVJ!M^VzN>QqGdJ@{85xy9{*bBK!nnv zFzUK~KHOcDU}w@jt#qExRmNvVp$^e+PV1oLe`DSPo4_5kBIz5*CGwFAiZujMd;^%T zD@gMmr}d|(C~MJ;vqzALr`OmFpXfH>z0r1>NlyH~BiUy}_`^eey&(!`34U}q0Vr?0 zW}oDsOrUrtB7Os0#1XOXD7QxD5EJvAi3EHd6b@Q;tU%^ECaNw|S{v*P-e6uN&okaO zjO9RmD7^iiI$n+d@{kB-;D16R*l_VCA`uus*#A|lkyatV3$^qL^fmCkBHW{qj!C2P zGlN)Z5!Ov6GnlBRkHH|yGgd-p{jSDs7TUvm9uk_MNU zk7Vp(^islJ-ErJmK3yxukNH1YNm>}kn!`w&B6&iFx+QAjWFk!?+L4RV*iHz(&k2$ywDJsUrrRWqi zR=5I5^u3TEO&De&rkXd7pb6r1a^^9Ks0KIOj#`aVSk0BqY_nU2(`s^o@tP8Xq3c!S z3=~8LM9}T1b=9j-MC~ZqlMY!Vxb6)HN>+gG8{S05flrKM`^n=NC6H`Nuc?@YLJgJ( z91mo=e3lHXI3ZOoUTXCF5ki7LG;{VLWh-W%(X~gKtwIHMKyzQk@+FA2Ua}JO@(|$R--6tNiIv4 za%OaGW~-9$|AbtNxVVTtdKvR?5;Ya~jiL++c~zr~L{rQm%EP4xQ6q(zx?V9Aq5GYH ziS2+UNVz0v^$gH{o^Wq251VURzMmxLeG zfnIs`i~9f{qx41U@*cF-xgp33mM0mLrFtLI6TRobH47cNTlr^8$~LZ*&kzb8*Z(MT zmzH5micXjEgzfxe$+9?HQnGiKa*3i1b$3GqZFYVZY6+OC8vsSq?XW<=WDA3a6i5!luUrZ*S!A>41f0Auh2Ki-)Ieg_?(Sw}qb> z=XrAYMDU1ds30aOItu zUo}+cV=-0?t#~Ee!4@ic5V2s7Yo%Uey(6H4VDlC?m*+jrYF|d#9;%3OHJr((Ka=V zQaXy=K);&a))=y;*4pB1Na;u)xLAQ}!G)}bl4wQ=2BY4x2O=N@f;|pp0U=X-qDp;K ziEj~CGI#~PZ0aH3=vk-}0ITWJR3505JIOHpsE`92hN@-h0~m~ka2e1hL$#XsXThTb z&54vY;4#2w8qNZpa-m$PaM*%@90729B77mrNS1CqK)#Np*{B0a?iofe0taQ;vZbYI zyRi=|6!_rZ=O+ukA!-t*0FOz#wVP-tJF0x^COA+Ce1leq&xf^Jqq#dr(&6%WMT-%R za6AA!f!Zda_f4H0K+b>x2n0aD?JMm? zK<`1hA;pEl63(=EB(3x@n>cV~h5Il)fi8n_NLw@Afs(Ff0^$)w8CW#>_)D}L3Tqd- zg9e^fmA^vuBi02rm-G6lPGUu#&o0oYlA-Uqma^%I(hQm%uN1VT0WM^TgrL2v2@ zyER}4dRPKM`kC;uYBeL!NkwxHg98_A-{_s(p7F)uDdA%bgU$lO)lc@)idoN_XMS68 zACud#A7%#t6bg$Ruph$3X@;}(Dlq|>BNtpIM7PC|GxAN8r`1Xm90~9e!oV_O=^>o} za6G`L@QlbB;E$*$#xE1gogrm#A<`{~y+qY!Y%S&7N|sW_0A?fQ?nUk7v+?%g70j)i z%gYf_rWRWUY7MFu+dB%yN}`ne0ZJ1sU&hU1+2DZYl&!gU>w8g5F)f!Z2gWPDZv%~a+b0MZ6v*4i$V^eC+>Pj?}rSZYC# z4r>QRPKo`QR*qaN1sqSIei-cc)t0NV3D8ngePlh5uUtV?hCwR@JMtR=;>Eb`jm3k+ zOeG#sO`wi^&LIBaW#8De&CiPP*4H*~sknnqtOC_j0}Dq5Qud*OD!lC6=;P$ZwUkAM z=mkv_1GX?~r~|klw-6Ou@IC3AlnX{^8TyJ>nIkmK5txh{7`O!{nudfm8I{NYtDTz} ztSNFoWd68Z#bh-0Tr16HQib3vu%JnEC9hv7l5#@x^H2gZkO5mL;V}Ro4-yo{@ zi4#NpQTa~%h=@cGu;yf>P}N8j)R037uww&f6T=ZKFQO|)u-FHgQY=}3E~c6U<$F$) z9p8h}b;?G@ih9?p!mi7(*_0ms`1lft7dbM_WcMAaI*mjYCs?` z{|1B_cAHsnNwX3)5obWMU_!B&NTeNnRdjD`h)RyOG79Pz#QCg^P=6F0 z*CS=4=CyoakW|Pzt~80=A5Vw$RDg5gE}0)Z6Ri29R-o(o$L9k!h+YHBhHet$Ae+hl zD#b%9mDY9j_T?Uli6UPRVwcvc54Jtwm|&p^q{btvf#3y;RG^j&^9^xYGpyG@%{0At zfEXmBD}&J#YH$z8n~6|7pu4hkMW)}Vr#rtL+K#=>Q2^E>IW00>7xk1pZf&+Z%ft6Uy2%LhAmmATGAx4a2e`P< zpolCOB_HWRa_Z1P1!=~JaV8}KX^jvf(5uqxLOmk2*^`9}pg)M5OOrfG1Os@&3f%ds z@C!Z(T}%^+fDQS6Slk9cWsEqF%%D$zF+)rt$fCNHPPO`&%QeL4S%M~ry1a;%VCNKf zyo{h~HY^x~{7gS+d~$4v z%}7+WCmj_TcRoG{V^Q;na6>-?KtNfR94`R0HS?!|!ZZ@af-(ud4o5RnBne9(Zj5oN zr7d~1NTt=YeTM@Oj2DeSu&sw~CJ?tgVMDHULp5TkmJTI1>4PH3g&EpI=R+5g+nmM6 zKvzpHZ~4*jWD&uX-xNl{A3l_rXx~dNfgm!#&Iu!e=eQ!6ByYqbABsEskS(E#91$3T z6$Fq2SjKij!_~ut2@GK;yAp95;Y`$OOsW%0p3KXFUqE{7y&Wwoha&#PH!HQOUXly<=Pr{TM6Xi0MHylQCq(Uq{%)LY zxTa=X)motxdTV2V7%PYNu@1AquRC(unHV*&3GAhKFF|M8FpO zfin}4bPsKzdQ9z5UB$$pc$(46ODOxrQ6_OShJ-<`=U$M#s(rw8Ag)=`HVURG1Io;pKfnCmz5^98ODi_Y*YWYI^&(^kpcAzc zVOJi?tVm8oRsk!(nvEw-Ohmkoo)M2u))jC-RQ`I$BC@^iuY{?UW_%oJdtG;|OhWtw zM#8ln1aur_TNkWXeBCSpH(%IH;k11+*SE$G2qC484w9|V1T=Nz9RzAMyiH^JwK671 zUp5=^-cNO|2rg28;r2!kl*pn|cn3NN zg(abIlvpnQXr5rfzl?FejW&K0)$vpU1x7>1=uO% zOmUI@EhluE#{KXKRy-^}wfCR)B`*CLghAOHMayhAQbr2bL(Bp?NBd!@2r^)^6TwA@ zd-Y|qXrpWNB&;$_(k(m&Ri1UJoqtK%+0{LaMIfg=8gTI9DhwNHe z%291{q+`^(oGP$cM_>uEtO^zBFhe8OnuU%qhMKP!7v>;IgMl5@S@JmTAmSNR%iI{z z%)NZr#5st}4p5j-D6(d_grb$;0O!=lB4tAzp**OLe=22kcp8aGc0Jlxse-U}MRlFR z1nN3?&0z#1f++$s=tymW89XAxa26(_ugQPq*8FV(q^!O#ivfvSF@u~H`QjRo22#27! za@PQYTH%qtlwQk{W>vHVGL->1fI|XUO?fN}3d9EJh{ynKLSfgz30y4%hTmAOjEE!~ z7adV|=_#RAaK4lGQq`W0WPWQOc2Q935kRr2v8$?JR->c6k0~NsDu3Rc}yG1 zR-|q=FN|A2IY5Zsh0R` z{nB0)A3y@8zwES~58YU@hRc>L9hI*ZZWX)`_TmskK-k*G#UzNp3saJvhsPpgLU!6B z(-yaaup^1cN`aA+6&O(};(@=&-Kv#3xe$dGtOpscL{M1SwMk`^JIlUBrgT)x;^Pn7 z-PHG|E}rb-tU|fkxu^pcFqe(B#fkP<_z79o8xYFY7LRGOLWCK>y9Ga##g7pEk4DgQ`QTnU}

s0HCb|zhvuu4HkXyo61ZE5M zP2eRul1?{CmsA`GF`Vs*nm|447r5{%8C;A9X z)oQ^Z7?*qaYGl^uKKO;z)+e`O|Ge_+XSbtbC}p9xJDKEWqjhl`W*;hLi9{p{P)pP# ziUyK&Z^_A62$9=S(YK@H8E4?e+tI6OmG6?s+DTn^KN058pNE&$i4^VosY^`>t|%Lk z%tTmmN7gaY6s0-E-ee)IRCUma+hQD7L;=8dF9%p|fi#5P1|#r4n;r~ox0QkuC+j0$2NkqZcUZv3!PL$JQ^8JAOm)A^CG zQ2>CHo8mi~YP*|Ttx<2}G_V{gVM?8ej^t+Ee7R&fGQqJIry7^Oyvw6im>x7+S#%2v zg4WiXP?wTdXI@L-~ zYs(ln-G4>aIJ8ru=_d^x2~=dx5O)j(z{rj<2EeGGXBGn~0IWzlarWi?ykrs1Wu^!z zydn!rwb7Fz4E zCDUCoJ;0^JqW47lHI1p-a{W%^sY9R~lH1{02#eDx!L%Sfqq0HW(3uFuVJjrYN`Jt7 zi3v;}KuluNXf>k^BdsJE#MM8MS!2VfZbcLhx}fqqbIkGm#@owQD-8jxq9o0jDkFkg z+m2w%=S(;@Le-Gf3;X~|Ttvhea`J6lUz(ZDHG*=MMuvP7`Qnute)T5)yXmDKf<))E zAuvBnAS9@AAbh39&0J$+WVFq)BSKpMECQ*-5`rnpc?8wqQ+P}v8my`MTzlkx8ZZ-e2X|27YV4p*uAhBAKonKyhXx2<1{D{7$;uy;MQTV) zQdnF9c5{X!hVxfn=6f z(YYek&L}0JJE(K_z;iSaa51vbU`ug(1~?kPBw(>&Mh&YCkwD!%z~cf?=G52+*kv-lO*}hUYN8(q7y~ijjVJNa z!6JgI3GgI}Hp1^=jBAQ9qU^KC33`N&X)siV3+H!I{1M0@;}ABv6JWK0;z#*qcwdxn zCOSfkiZz37@Kr@#7xgY6?hMhdl`cyEQA*bk>`Jy5a0;U4DqJ0L?rz?$saUeA5|vZMx!9G|ZZc-QWS>y+?}BE3 zez?Dh0ip+YBY<*z~_X}rEQ7NEP~Gk zV8(zN4PVKrSv0xm9A%PYa`DARW(VU3goq{&AMXiUXDjPoUR6CrL3_~&mT>_tz?zyq zwlHKzQ91^3@Uq>H2&rfzBd8sG;A*o1A8V-hIL`!$N8t=+Sx^fU)W{}#fg0s|r0ka` zBZHjxBA}wSX7DX@>C>r{V+9|*x)%tv)Uxfe=GkS+=6e}@wsWS64VCY{1exPWQ)F*K zIYUTYM(-i^MA=i!gY5;KMTPOBT{#rqO<||tUFc>31*aGn;OQe!AR2%bF;;l3Xh#Ly zg=$BuP=9cC&<{xLuYJpAfId0{9$U-;GzX-2Y1UR?;dnMY_B&N`e4`_7U8=0HTq(R- zzWlJVHxeXV_CHFaOS0p5$~p(z)o2S)Go_$p&3)FSEq6x+o{Ij;`RK-^aU<3dgKY}< z#-q)L$mOta?7KLDKX5ph$HV~A9!Q%uq%I+*8?$;L&nbJ? zI94oUmgB9Bn}fWG;2S=3v1uS}L7MEy`mQ*F8@ z80$uArS*RTm*zoBIrUW*uBxD?@j*!3{`ayz7n0sYwDbKzWW$nB7JS3VMahpdNjrtD zZ@jC5au&58t4mf&kX6L0B2cxGy(6naKz!`N7=aukCbNQ+ z6fP7YD3!>fU~ykc=;N&oy;O zlDFN12x}4KNxBNg8d^2I3!D%}f8nX%z$Zl-O1FtB4LWjRfc>c!SzAC<&7BO0bNOdT z)rgjcRM-6ORy5aRI!B?h$2w(s@!!ZY3O}dOE_iOD5F8EqzfwH%q1ObW-v6! z;P9~7z+cauMLHHc54HiJMAn_4WD>R$@!S?(hN=9->XNvlBm@PJq;!4s-7R}@F0ZOo zzCW%!BfdN@7iUR!)8xDbxTZpaXObTZ=?j!sStH1=mHIgft(c zKND>k?(Dt7i$d{vLi|hp#zJ8ym>vSp@Q=XiBsoZ~eD){f4hNzc+p}>m(nxtTU zI|{WXcJZMCh8KfW5|1G3DjAKdVp=fSWCSpkoERxfS(aW5()p0`cPOpGz>2WdK%@>v zN(tbaP@J*r&yDmuvk_Y`s`-E+CM1&F3$wBFQ1J-4&V*4}K*FB|WK4Zuc~ z)$EZU6uXjfYuO67n%CvcEVWGqVnghXxvfzs0ty=2GemA2{RdybsOMiO)Y4>~|9Kz~ zq#Mc}!{TQwjZH4LlfR>2pFg-pB8jdc{}A?-5be*M>ec!3!p|q9I(6!-#1SwN2>>XS zF~M6X9Lir4kg4V!Vy6iV1+7(pekM4+xWzWOpVprK&khxe0zi=BJs5Oin05hn8n^J7 zn6P8KBN@Ni;F$vl>iT>3AMn=OefxRa|Ksg0HE$LUttHS~zB{tn>Y`J-8{!}Q&&|=9 z{@?3$r>LfnuSIqE{aVyd(l=4pUi&7hL91_~mRo%jRf!{X@HbJ>IAU=;AMs7pPh-A` zdT+uvQBfVAMg2B;dUVn6_o5p$tcZT2aYb~mmKD)vb`{YH_7%~Y1M%6RBKpYSis;c! z710AGS41zu5r@OJ`&Ww-wv3DU*PZy7t0nO<9UsTXYWOW3hEzaYP)8_5A2qZ2Hn;v6%_S zV)x*%SaB@2&aPvzpYAyp+cW=IY`rfMmi%^bVcZ`$mLFOew>o!WoOEJgT+Zo*aV@Sa zj9c(vVcf&2g>gNfE{xNee;hZv*~f9KTYen(`o@cK^=w?@U$=FQ@9~yv{P%CW#t#a2 zjZc{C8lQ-x#{$>*g9}~bM@GBGH&1qrk6Yy$zkQu+{HiA#6XyM~F`-?(w1io&r6tU7 zl9uq@>uCusI;ADF!BJ+NmN0y5TEf!lX$k8;#P2?732&#FC*Iy=p7`r-^Tf~3;P~D= zamyp~#0kHfC*F{nB>LBHk{E!)scVzOf7&!jEbrAMaa8?+#I79+68l>fB)-%ZBu;ZF zNZd81AaOU2E8`0ieI^wo_Vz4D+~HG@cqX7Aan$UBM9WWNmLH9YS>7%&X8D4(G0Xd` zk6FGiJ!bhK9C4qepDX8G@xca!zweor1#K6FKI`}7s- z(>tzmOE;{s-C4wrlHbNQ<*~*#Wmsq1luqq|JW&Nm}%UlJQ!O@kScmV@EjFLsf$zuLZ(csheslU%`_2FMv)|k_*nacg2>Z>k z3+y-Br`T_n*4l4wfW!4u`^{Oq>^Gm>W54;+Lq|3r{XNQX97n>_C_|h2(T2Ge(T3LT zq78>M(S~|mqYXm_L>syej5hQi6m4iTG}`dl=@0~Dcem!C6Q+2{% z|Jq4Ik;O^Fv~DL2eY7VH<^xU|hK)LD7(e=?Ve6%@4L^))^68DpZCl?zzH9qCGqScj z;JEP5tnDN8S=%26W^F$?KWqEIkF&PBFUs0}Z846ttnJ$mW^F%^o3%aYY}WR}7cTAi z{XkmAqP(<>t@&vgIR$AMy2`YSmcQV;$7vax-q@7UtK+5&I~CVS7W1!}cC23)_1FM{;@C-a5a8?X_qazBi{? z_}+?E;d=*l3E!*f8ou|f9^rdi+J*0J8niLH!<>!TKSgfLz81AHyFtRn?B#1WW>?|} zJ-9JD8b>US=f`nQ{>JS0E^N#``lw#caU2O%^>W(Osh=~~ynasW*X!pTYF|I6UZ?sw zL*A;N)3s0ioc{gm=d>AAKj&ws`Z*6CH#>Oyd9#BnESevj)~)%$={N$qH$Qk?+x%do ze>6Wh-nsd~hdA=zZGNz5O!I>ebj=TLoo8|I!NU=U9lq?6n?7S=ZhH%_+zvQ)w(!b5 z+{!DrTRX4Zt5#mQ>Ak#i+xGU#?f;fnZtMuJ+%@mw_px5N<0g6K8qzoAIz0G1_dwO> zxmHb24`L_9APSO+k=Tv;&r78Gh zkEUSuAx(kfVNJnE9Eqnj1(tiZ7C6MUIQxAUfG3XY#zoONy3^6d0REzdS=+wyGkADRC;ysPBg(?ccaUf{TRxa8d9ypnUypO>7Q zbh_l+$3-RQ-oJ*wzbiTC^{C`r_>+=zz3Y{pTVLMp{5uux&bN5f?)*2uw>$rKo%ZK@ zHE4f+m3jN~DLA$@Ykz*2Q~UF8k8Xe7d_w#4?IyNAf2Ci~^V3~=p8vzO=lK`^?0NoY z-=62k2lPBY3CBN!d!A2Q(DVFPt9qXQes$0D^|s=iT|LkDvtD`Ox4KmqK5bETVSmf2 z3#WSFaIU)W^~kCVAG}v}q349E3vc51r$^O={5e$@GUruY=)An@LdXlVi@9zKE>4@Y z;9`5<1s99_7F--ZXTiluIC_LFxR|kE!NsE=<2bzF;^LDFE}px*;Ntbd1s9#ZO1${a zwZx0Mvc!wC%M&kdeVKUCtyR)Rr*=sfZ`vhYyxTkJ;!ik!d@Jdq-^iqk!`@4}IPtBf zmzE7_dTE_w(@TD%n_i0cXnJXuZ_`VSXEnVvFA(2_HNA8(vgxHKOPXGi;+kG+faB)< zcP{DQU3qEM^xrNWYu@sUJRGB2wEW`C8!f+xuxk0m&|WRSIIL~?#TEOOUrg?YV{FSW zy7{#HqMyFy7e7U`{GzbxwXcRoM17Ul%cF2@PIjSd!smtS^DBzpIa^WG;;V|HZ~k3T z^!9fZMZJEgC|XroQIvvX+ue$yVI3=r-tJaeWd3GlQ9EsA(fGk}Uk@K1_w~`yabF+D zacNB4*S}1T`+A~Z+}Cf1$9+BPlen+<#m9Z!B{}ZvKC9xs9-k5Sb=`UI{CoMl75^R? z`QqP|&xT!j@p9Oe<_#RLBsF%ta;TZ(l`@UvmEf+9S6bRSUh(Ydc*P4xb2rB;nz52`VbFXfEIrnNBj{ASiy&7UQ@9K(f^R8Oj&b#XJ&b+I?44-$k z?wEO3TaBA{^<($?*Y4ifb#3q)b-t;v?Qwn6@nzSCo?3SO^$W|cue`PF`dS=@+sm#O z|FZ1*;K$3Z_kMx@YT~aqw~4=QWgCCp`R(}YUWH#@@Aq5(Z;P6|_gycajPHJaxc9rb z>}|!rAKq5{5=W!lZN&>uZ!7M1W?ON}g>A)ESGN_XUc=vywiWxn*j5}-XM6GPrrV3Z z&Ce;mfn!%ePI2n>oZ^0Wa*D6s%PH>qQ%>=KUvi2+th2v(z-#-9ovrs5yLa1P+`P~J z;)|ZAi$elV7iUDCEEf*mPZ#%!K3(jx>~!(egHcJq?`s9RMJcHb)dZTGFIf9$^XA&yyfvTogYE$h~rCRw-Kt+H-C>6CTrUtO|p zUG1KAt4Y7CTU8EOw;se6-ddVecx%_H!dpjE3UAr$F1%&fTX;)xy0Ztt)opxN^RAh0m9*D|&y|x?;!ott-yl zX z@q@2EUi{#r)r%kO&0PH8#LmSJ?i|JUrx!nP^S}9E)6=%UwtV;LuT6fP@~~sol!v`v zOnG><{?vyJ+fIF$+-~Z_EbFNc`#4U0xEMzaj^3lDK0NL+_2Ia2Qy;E~8&x$rWmJ{N zx=~g7eWR)zkK(@|$VGN%0HlWBKxuJp%~)yc_EuWw9# z`aO=lX~|DD+mfI5*^&Iz{#f$UqSMJwr(MJ6Tggw&%aflD`#t&T_~*$_SLQr?KIZd> z&!^=-d~Q?x@cFPC51$|X34d2Te7^IyhtJPFefaz}^GDD7TRwW8|Hh-|1vt!iTz*ke zT>N6yo#Gegel31+y{h=dm}kW=^mT8%Xx!w+i~P1XUOeu6o=U&DJ(X5Zd@8NMQ8MYNMlRrt8RxQz3OJ?^{RUqhjbF3Z}+OJt>{&EXl1Xu z-p_m0y*|X(?4P52%@&XKHCr{#*UT}(*X-B%_%6!VY~f;GGxH_BW=(MXxZT&RPNuKf z(%rsh;T?<28gwo)TVq{h_79sPv!OVa;pnF=GRx~zWY*?g{N`F@_SySIW``#incep; zGW#uLaJ@fpEDs%AZ*}RKLT(8Bd!SxpG8C>sS&ft1I4-c-VJ2$xA?29<( ztHJfI#@(*JWW}8Z8>i_Srs3%Mfv(}e4|NTp)^OLC*BYL`a;@R@@2)lU zyLGMMq^fHT4LEk-81w8}!w+sIHr}~=UE?DITbX-~YGoesZY%R)Ivn#`nRi~%%DiYz zEA#1_TA5cETA5ekIDN8}`K|m`=Cl6Q%DnyB5#}9m>`WbDet7)|^KOO_=2y3kFi+n% z!o2Ol5$635k1&tDIKq6*m-zkb5$5BHN0_f4?`{5IUXuCJMM>tnVw22|E=@AGNl!92 ze41pg&rULLcL2W~O)_tMBFQ}EMv{4#n@Q$fam@RzU(+rwGn;<*NBw3}^9IctW#9D2Ej!E% zYjwgWtkr;z!dm4-g|#|}V{uGaE34$NR-bGLYt>{=SS!0zVXX$A#&_q#T9p-swK|;G zq0P`09okG^)1gh@{TDcz!8OOGv7aZH}{L-=Q4>uj#{_-QvDRXT5 zRlQSfcegv$_ISrrZFg9mYTMiTRNJvVPqmHEo@!h4?y0t2Tu!y^ievitQ*D28Kh^fD z;8SfazN~9`;k&w)-+f=#^6rzmmZ$2NS>ClUvwYUl%+jTenPo)>Gs~sj%`C$l%q*P- zn^}&)vHyg*W#fWf`QQJoCY=$*zXBgh7oql|ycJ18pn@6K|jb}7! zm-2C=c3l=VYS$IVApMH=c|!`?_r1Np!{<--cPM(gzeA+Ofes(zc(28Q4k2w0ba>kH zK!?k{4|FK%d!WOvcMf#8<$9pQgZB<}c=G1#jy<)jJGL6Ix}%Hp>W))Jt?pP=VZYTKPx-IzcoxS$LRNR|8M(USv5zKO<>46p@nowrOD0=IBuus% zx^A-7;f<56u56lYHF*n;qm!+=eLdN#-_^-hKi!^e)ploqReDx{)$x4+R;w=uSgpZf z{qF#)p;rT}vWf$&KKvoTs%fJ@s}8RPS`D=bv^r}UXcd!`X_c}((`v`sOsfymGp&w( zn`vc!Bh#u0jvco$t@_{2v^rasX_fpa)2iWbnO5CiW?F4Ma;Ve5XM;7He@yCp_R6Eq z*RDS5+@b7I=aQds*i?0X(yOZTyuMYPT?SNj9*5%#hpNsoL#sNE7+=-7Y*JO{;&1y{ zw;Q?Ax+9K?Q7f&z+*VpgOjv1s?Sqxpn{_L#!+cj-w+UQn9kY0)b;=U_mblXTy)`SX z>$KY1Wp_l+u2su-bS+ES(RFItj;;r$y!*4c=H1uDns=YD&b+%j4zG>o-RoqT zcfWYty!*E&aPC#}?r#lo?s3DXsqLH#nx2h{G(CG=)AStlt)^$jFPfg3N1C2@rOrJ! z)bHH0QIpO+k!KgtKF{u2w>-PeJ@f3s z?DOo}49K&Kam}+!c@Mw2=h?kCJ3uBgxMG&~iKb7wLBP-q-Bx z+k9tdA9Ksje*KSj_BN$<_Pud*Ew{7ZU1?|EpuwB=3vAu`euSfOPq)4)E^d9>j&5tK0g=U@A=80{X0JSxc~dJa{8~k+i1YN z)R2MJpM?zUyCY!g1^8kb%P+gbuWA8ai-`b?Cs2 z|LQX+4M)%OeFhD@*k{o8!ajp0UF|cd)9pTkQcL>`I#}LkQ2iHu2DPr!chJE4eFuHn zwC|w%g;9fk^X&0X@15T7{L;+KVP|_YhhtV|4(A7&Ib3|l%wfwg{OxAu@ZAJ62ir+z z4)1%IIaq&S=FkntEMGH+?4SV-)9N@oOvlj+^#I$padtSMadw#0!`We5PiKcicFqpp zJK(<~oE-{2bauF+b9P7$a(0+!FbuWcYZ%%$+c4DeJdPg?Lnq!b3@xrS4Bh#&VQ3bP zTfZ5G4sX6?sBN1qL$~zWGIZdnH-@bnIm9U)$K6pwoUVK@#K}iD#Ocw@Ax;s#L!8zG z4sq%bg3pOVoEoed;?y>Eh|}2ZL!73K+TgTk(gvrMlQ%eRoq=Q72B)_YHaMM5-r#g% z+XkmbJ2p5q$B}S+gOl0m4Njf^wZZ8~+UHK5yFYgd$@<)B*jXGmKX>X}^|@2g@1HwO zuY1a=qW&qTN*t%Vo^rZnbINI!-6^Mq|L*NP=QStiMlGD2d$n&VdFTStcBIP=!lk)2()j=cB5){)PqZyotf_|}nQ|9yPaoaxCfjl7dxdif>03<^wk z$yl81qKQj(xw|IWWy3oBmY(bq|7o(z>!*`lw&o|hY{L=oW1&lOX`##JyM-=So))_N z^s>;U$sdI-F=j0>u~O5sJ<&et!x*1dJn zwbc(7T|aMh$+dHg(oX7v5_=w%Je3$EN+1;9fq_xG^l0Ft=pIKy# zdyXT%MaH<--^v)5+Am{V{dY3P9U7W3?vt?@jPv%*7#HS;--0s6Y3F8)TN1hJ zz0=KZzgN-h-g|HL(YsX))w?})(z~7Wz!9c*8$D0&Hfz4#?bbrQTV}l8ZCbM4?MkZN z&2zop%?roRDb^D%?XsS5V~_QO_x4*)2s~yzVeJX)3IDomJ>l2u))R(*YdygkN3&Av z2|w5AGGU=vmkD40zG*`A$CD@KkDcs3f6`=kgZE_jeLj=j$45?f|8e1D_wQFtcDG5H z?A{wk{<_KTyLL@>?~*;)-SOaL_pS#nxgT*4np}C-$gr8ArDT{ik02r2o{j%lc2flh}W1z{>tpn?%l-`oXXIsT=$bPVE`r!s9o? z3XcnI(>?ZgO82dHF3e2#m>iz&p^HfO z@WIjQ%}z5mzSU{Qk$#!-PKPA=(>J3KsO*`fbPRw{P)xVo%6Q? zbnZ6>=$hXfpsP2awdd;tTYL5x+S>E`QLQ}(EpF|Z5Zl@_5l4?@tvwGWwDugiqP1u9 zO|3oS46Qx4f7aTw{bE1Q4mftk_<0^);^)~d$M3KlStMzs=7xHrLN{ z%`yD`xu55_^M0P2&rJ0?-Sx58F}ufJKR7)0`epEAuP*OD_PRXlvDfL~$6hm{AA9Lz zAA1Gim>Kuj>)yu4UY~D!>}74c-208W0Y3Rl0(^?%0(=aH0G~el@!ul>K2cu<_#7_| z@ELO}z{eHGxsm`M>puc~hM5KW#0+fc+u)nozM9p8^k*&a=}YYI=^wpyPk++!p1z0v zp57z;p5AZXJ$)b!pZWLn|6X`ce|GUb{ZGmF^abng>GNWa`hU3WsQ=Z?NBt*kJL>O_ zqsxw?{+;rU`tQDc)c-^gj*6rHm+Bq!zfu2~e_ZQh{!=2a`FCIS$p66VNB&dRKk}b} zBlG_dcO8IH6m5GJY!p!tX%>!N4OO~8=ruvQSl~#GP!S>rB zO%C1;>HF^OkPlAY4*BZT?T{zGMA`3ehy1v=a@ei?mBaeHS2=9S$CblY0*+5AhgG;* zIqa9r%3*VVs~mRb&&pw}NR_aX#jAwfDp@6Lt~Dy`w~(l?=Fw4Mt%0XTMukln9~Cxx zVN_Vdl~G|KYofvq?T!jd+!GbHYhP5@+`~~}6Io)|hQ|}b_C1jp_D}i5Fx#t%VP*Oy zhD{ul7`Eh%#ITw}62s~NRfi^qZ5@^v_Sl5Pu#egm4{vvVilgPlDGuw+DUOJrra0C- zlHll7KEct>lHmBEVS?lO%L$I3fXhu29K(AgINJ70aNG`VJEFqswayoSj5TYWfm_!) zqqeVgesy52bLG*s_*>g+on=n1bx!Ho`G=aQ1MqgMhOOU;h{qT=l6 zm#fW=u2FY(^n&`cqnkFI9sR;9_|AWJ^y33&N0%8iJGw{k?CAYNTa9cptkuXjhqoHp zXlkpGje+-Pv>MrSL93DXmb4oA!w%q7tC8C-wHkTka;uTUZnYZ8=8qfMZRP!#M(gj# z*!SL#8M*&{%#9=WW44^VA2ama{g^J_-H#b~`F>2d>-S^6`s;qorN8gTqywd1zB6iI z*E^#I13z`UGwQb_|B*i_B*4#orv!y-5E7)=ABXRt-LcTCF#znt~Z_? z^FoE5W1<`P9CN>E&oRF>?>Xi+5Z9{bm{o&&j(Kue&oRe__Z$;8x96BK^YGo0o@1)7 z?m1>|MW3->H}V<#0}%eQ&sbMepRt!(`HcO%x6jyL`}mA~-GT2Se8v_X=`+@Ms?XS7 zGknH&R;G;|oV;;t;pdKyb&csdZbZ$DanqY;jJx|v#<*o|GR8IY%NVz$Z^pP=Lo&ww z4m=W&F)nc-zFV0wE@VT-xKSH3#yQ&hkMGgnfBc)T`H%nfP5<#XZT{oS*!{Hg zInjT7GH?QTd$#|0#}@zb|7`Ogf3=M|{=@)v{P}_E_?5xx`2WSLs}+@_BI^q4ySVj^Q(8Yt{1pg}{0sles>3af_C!N8aG6xIuv2fTsr z%LA)`S%nog8h}nT@*GBcn@%PR@ghhpTMS03d;oQVSN~uRe&EmD(sLCWADDm zSfQGX`2xp*3BZTd8A|}V0%xi*Rus#QH5hvgNCqYX0Sy>CQJ=B9fDhIu*JJD#;8oyx z;3~KeV>t`k2LaW9bw4QVw@!>50DL+#W(U>+KLE44FjfKR2aM^8x_~oS_r-GOS%no| zuP{Gg32+7YV~xUw09ydxwF;X7d<9fqr?4>K05E(JV?P4xfZGchI|$UndV4IN0LB3y zFJ`Pi@EkA)s8x)yRz(?`4qO0+K7nTniZHed%V&TSk24kt+{AY)0e_&+3Wc3tuCUTr z&IO)drm&Su6?PF=iuG@Ra@alqSO~n1&#hl&?4Mp}KYzwXV|fIK#(ERr0MG)T+x5oz zLAmjOg7vpjalGvmcKQZm`@1q$K8mp!Jrvd*INe=gMX~IRWdiURaI%}i+5;1TLcn{- zkqN*PUnwm4OXLc$<2-WV3*;G=!-3}b?lzWlQKl#G{MTq#EH5rW+v0foV7VALy;xx- zuxx|nL?B|3!s-C&3z2tN{tt*Qt}q|8)!AT$)wC-t9w>|TBSFYdEWgKR7K%EDD{MZ{ zEkt2IW4R8?XOZU(#wqORbcOk+Ggc0`l*X6~Xa#HrZsWV7z+@ohDq~B4p+Fs=QdOK+ zU^j}gG`+-d0cWiqXs9lY*!N7aHuzx@(Q07(Sk-x&;2g(5_dMc~}-~f{S zpg$lS=(!#GX{4}zz((LNAgM9V`^z{NfFIUx18tiq>?S_X!!jMX3(Xz@n*5Ao0(Js7 zfwjM2e}Euh@vn@%2wcMY>sUHQGWIRbc@)q#8oI>iby%*9VXQgu1wNl0#aP49j0Iy^ z7A8-V%vt_QUHEbu)($7A{JTa0Z# zp|D7xFwhF9n2h6iUt!w-6=(?*0_NcRfxt(=KD2w+Nzl$9w9jh_yEjl_Td^#R&x3%$ zucHj`1aSBbg_U|!VS!ja2@EKU+(>5Z+pIZu>*_W~Rfx7*`kL6E&7^{NiK)@gC z6M+iA&fbir0SB?Z13BRQTwyN*e}1O0Y3CI7A@B&+4+G->U!c|(=(A8}2QUol^Q$rT z!YlBlmI}KB)Nh6I`1}c$QPTfMvkD(a;d!16XlxM}z0=2F5->d)^0{wZ(n_slfBT&^GWk zFan>Ow?lpaAL6qU%kvB1(a52lx8M(GyBF4gdm{D)=nFIeGUCzBSgyk|4mbr=pM;zT ze#G}jf%fZg9Kd^Pp&y_S)=vTn>oHaUW?=t1V_6i-(^z)LGRTE~xHrixe4$Bfn*xNj!*~Zc zgYQNG&usxuUF2Ou^ur~gizM`Ys}=SEmWhc9I|mfP`U0R45C|N`_G!RtKqhcB4E@$h z?0-D>KZX7Jgu+^EMgD+$DsTeWiO-S1eXM6-S$!L0UjfSjD^O}X#y1<#{y^Q0=u0;# ztjT7D{ea~Ke2xbau-d+ zCPUL$-mZ^&{xy zG{%is{snY7gS-GP1NBg5JHD$EtFRb&!ATs)z}6TWKZ`!6H2NhhCzgT+fp*Ve%mFZb zKNx6HMq%G#`=3}Y0UFIkTObdvqTil1g|Ux-N1^Gq!1t3ej>PA_z^zH(2lfLm;rn1< z4X`yzVb>!SRvXI@pgPtARE4btg0Q|3NW*$%;F1e@4wU&2*BKvRYybpd{Qz(h>#--{ z-N0wS->1;00!4r=!0O`g#$u=wD1vnd&NkwB0Y`3&5}_LW$+`w4yQO^kVf1Hj-+=n*)dfomn8JMb9bgKgLG{aWDXL)Zp1 z07@T3Ebtky?J&FpD1HRGJc{#=WeAXf^(DVUx4_px*+0&L!{>knlIg9Kcne(?1FuSq#TN8`s#t z%fN8p%q(~kUEr^&;N>X27D$E*bYnuQij4qfeKi+VmTRT1>8dWm&I}mFli3*0SE?G z13o}W;46$54*|DE{EYa_H{IF=s*mTwq~0VZC+xx}&#me;;T{|sEj`YbF1Za^>Bp&={- zv0Mav1T4ho#n*661lWLkKm&Y#2FnhG;bVp1v%p;-PJxdycnNS8>*erWQ!M-Ab1~pu zoYSB%T*Cmr0vAK!??7d&HwlNw1MdP~IB+csJPOPLMl8p;W;(`fz`1G2J1k$v(gLJS z#aIe(01NTCF_w3cKgWR1YoSGqx5}kqf39HvfXr0vBhVD<+wpw>a2L3cj%#>mWfqnl zfHhyB?|1^*1Z=aN4SoIiu@KIdDVY%}WT)zT0&<;`f?ipYma{qsT|2^;me)quz z{0a1Ah1gnr*F!19?qGd2kc9P(+n{Hl_I9+zR_qh77DxlCPe+{&(1-}Lqlnn%?|A_00J-ALA$k^YlVD+!ynyEGB0DeaSTfpw&ykE9Z(wR3XB7K4}zxyQ-C8t_17`B!E!d1uLBXAFh<&laR{&t z>oxFdkl1U>|G0#blyn?jSoAfQ$=#;#x)j^!Sp&fDngg3#WmH>M|i6IhP* z-@9Y%3p4^g>xS_kFap@$1N{V04&@e>g${tKtpChV{KWF?X;7=Mu{&@wpe)?`%gO4!n-Cg@E70?j)YZ~?!D1q%Afib}2j}~IR9x23T0v`dF3l(B-6fVRn0_(6I4AcRh z!L~2)-4dWXynR?0#t?ut6yq9TPYC)U;89@vaLjuEMe#kua+4kU0!|NuZiAp7Km}R? zg@6J0{zYIm@KG?11E>hh0q)hrzAtC&w`I@^uot+q6m)gfMuWNm>c^ZV_1OwfUy>qCoW=tfdrh(uYl@Uzlm)t0e_$sG=6I| z=8MMQSb(DA;R|EoMZhJX!#K?20V()C3}`d~8U^BePzJ=f|=xVP5rLn#h z)z8HG5TFr&u7Q1n?-0%q#;O6EF2fsvU|`)P%>4m3upWrzkRLIZ`vdxKEUVyi5|)>N zGZ+t#y^rg6E5>?5;ip&*0U8cQKLw0@OJUCe$KJ%)56et^*L8@()&q}U!x;Hnl#9UF z6sR-;UWVmxEcZI#k-+S5oM+@xXDq(~D&xBlz*P)w4m2r_bAaW|BItjB+CT{4D2hG^ zXozjUV!0D&@Ekk>XVQdAoRpod1=V$O+pd{c2EIJ3T2L=IO06(T+d2mlg@IBCH1?mNofV#k= zz$oCAmB>TjXRHTe8I6;+7B~mo16s{MyH8ixQ&`%8g+Ma!3-AKAcLN+HaK3@-K=r5K zIY1(C5vcGqGzFXk<^v^4Vq6JSJb-@L2i!ncU<@$!CGY`dfh%=!4F-$@Qt^2wPzGiG zY=#`HjN`2YPXel9y_yB%$BJkREazd_4N!pn_}mp`7%=`g_HzfuEy?H$kOPM=fD3S9 z{ljk*_B5~%zWh1hi|?o5b5GzAtd9d~q`)6P#kDW+^I7D#zpvLBY^ zfVseAd}d?O{#f3|auo0iupc<{9mXIRG48dkjQtM?#rnm!aUBJO01bh6;oElse{3%UBmj30W1QF##|WJ6fc?SpW_!$G z0ONr!KuO?2JB&xMy(%ylxH|!U3cXefz%?AuBoOlnSkA#R6IgA-*c2Ez44#MO9xO{> z{4ft*@-A=_PzxiUfX=`tz+XTT5C?tr18M?0fSs$58^B|U@PFj=H^3wKJP4Qs)B~nq z+Z#XwU^h@F6?0@0(5`?#Fd4Ws9v%y92Fl@cD=Z%cJ{gDWb6^NCCk`5q#X4{SD2va1 zfnxD!3*Zy1-^a4mM2rowoCK`-3tEnY4uRKz7h|CbU?i{(_zbuY{D^IPfG6Y8cL3u@ zV7wZEu?KJpcod)80e68WPV^C2-o&!D3wpuQ1yn@8Qe+N1>Q!vRwbQW<@O$7s(6%Ge z3Cq9QqyGY$0f&GNop4PCypQd%!07jI-hf`fr|)9S1FXmTS}gxOj&lc`1!C~I3g8F4 zQxVrYyTTDTry=aJw}n+Ul$9#v)d4!u z$}ZN{yTU5=IE2NaFtRNSmDrJ4kvI^}(H_e%{AWcQ1hQ?!n13NH+b()vSY;&&yAP~j zW}a=WyemxEHp08Ylx=IgD@@t;nRkUL+a7zws{^ELd)d3flx;5W3RAZ2^tLe1wx7K# zOxafHQLhe=vTZC1zxeNL^M}?{KWH{&!~ zW?qy@r?O{@NZUJ};az0WkqY8u(UDS)%AzAR6)q~N?MQL2$f6@vM#!Qg<(`*CN9y(| zCaLYXy$@whS#;zQn`P0FlawkB)5$sR>EZSZ8yn09xIF~pLAQVSqo|K}MKwowQ*VgM zhU*LA%8u3(Lh7BSCxjHbSWgJ4aI2mWQri1^LP$-Y>Iosme5WUbRB>Hj2+#b#^@LD% z^V{-X=NM&h9fZ{9X&KIjAyb2LjpDnbzl&jh*sh`nQxpL-<1(Sw$ ziv^RK&Lg-s3Bv_>@OhXlnK&_e?1^t&_>X2}ZlQP+?~D6lXc_?{MGD1NzEp3E?#9Gz}1xdDo5egFS3q~kNz7}H?T*Ot3 zP>_`Q9Tj1Xkf3XWqM`RP)1>v1NSMl+>ks1+>Z&)4WYb@77)fQQ-Y}BK2)$t>iLrXa zNCs2&hEeJ-&>KeS&TrZ2$`ZP^*{U~;()ys@FiPbQ5mvK3W-e?P!Pzj5<1-sp^ieKc ztMEGoykDZZjX?G}3S(I4U12Tjf6yPKvg>+-c*g&xFNpH}Z+$_O?N8v&tB!`CTrZ<9 zh%&vVz97o;M*4y%%RA`{q8#s~H;9|~Yx;sHzpeU$D7!}>2$v*zB`lxY8;Qhuc!|*n zZu{^}TpIZYYuUfjIEl*k871NB_|zB)spSu2B&423cwWIk!;+d_H%3D0T4s!d)W#oG zH*h$lzF&-ycm`Od4DAW^KOK#ckXnO{k&t?KfuvPl0_=_6Y6NB;cuJ{xDEJ}>Iv63) zR9xs8DMTu(A%lpkxrcEgQgozoB2x7}<3yzF?~N0Yx~r8oeAYhsF#3!;p#tuKhO zyC;Ggc0lG1Lk36UI*a#R*`JW%+gzB?bK*LT?;JhLFa%*O^oMB~Kg&qC%65o{a}|6h z8cupBTvqVdNf~WL!$~6(M8iofheg9lH-CzTlY*L*6Kp!tk}4j~`J}p389StO-~4Ad7%nA&OY@QrSybQHrLS;LK)l*A+>p0@;gQ< zkh7;~j~*o4ed~vB4`hQ-+}*W!S6s{F5RovIjnN;*GkT`pFv{xHdc!EQPw5S#?EX=2 z7-je$dc!EoOFXCZyiuk%&>KeC-bsHL_wqq{!zk;+^oCL9&qi3y{4!sA$bjEHU5mo? zy)CR|{Z;)zDl7NA&YF0}chVO``94}-5M_I^z97o=-}MDirZ;#&S3^*q2kHx=EMKcH zh;saf-XNagJ}>GzCdzNCz97o(g9vJpcfw}h5}504jlvai^D=vZoLxfpExALr%&%^g zLS^lYQE*kfWrQLS{aqxRWQ2kgv&9Gn>F0AJ6r`rVjZl!bYFZ5B1u3n+5em}XIAauC zh3kw^kS0$Xp&*6c0Y$UyV#-Tq+Ee1$->7=F5@z8=gEf`DY@7sdU>POhN*!g4gtWTW z7zwHMLt`YQ*HmL9q}alh4V_}rti>1!skWCf64LEtqa<9p8;y~Wc9V^fkb19zq<=@u zpz&L@PCUmzS4!ddq_WQLYbpBI7gtuHx!&>6-jvj4m|gCt^HjY4=)g5r=_+`|J%8$f zEbHx$KJKZ8$OqGS{DBImX9`;dRkysTdLYn>`uakE5!mCL@1bTl}$^YH?+Fsy(JesESr{mv2qQ` zdrM9^S3WKG&f+yC?=87$lx$k^*PEd2;H}l<_NnX^_-ZZuFr*A3&8;@dB~)4A+7kDW zJJcKU2+5m{$|EFKs#QnYA(J0Xmq$oW^s78V@}O>Yr5!T4&vto)N3+6eA62pZ=_Km)2U_bpcb3X_xZK~)5%74YzB zel_P8sM*kS_43*Ef(caC%?JUvnJ94rkFAIkkc||sFWd%X8$HDd$R_5A6Ob)@B2GXy zP^p1%8<6a6MhN)8Vv{%l$@+#k0m-;aL)=}{4PW~Ma}xmBWh6_BU> zZe|7C`Pw%(t%=AV=a^Rkx#(Z0AjjyR#@+l)Y5qzZ-MI0?ScLyZ9RGPA%_V!Z$Tev5 zHc)fHF~$f~mSlv0yVytK1msfp#0kiS>b4YaL~@ya;soR(OT-DtCB79WAQ!0IO1KS3 z<#rdDX-M2achT373lO4{fPB4xa;FNoAIQC|=#V1wQu zp7o#W3!+T_LthYOccV7B`WQY+kHmPK_cGLNM4_kQnZW0lvNF9V0&~o5c^jx@y~7xR z%H|j$;A+?_PC&}|QJjEOQoOBjBa&hoiW87}g2V|(Nz=s%NL9zg2}ogAjSz6PmG>1s z3#7a*;sm6^=^&`t36sI}6UY3fxg*;>c+#Qv6ONlvxW2cAHMPC3KS*WY=?&tl`b}RD zDXCODUB^V~X{;}Z6yv8ah*T1xFNl;eOJ9&j4f=ve0U3IOc-BAFURRS-rsFrD^yD68 zcOL}R)vitWV`zL_j}`Ze1%w`%03kdklMFROzJq z7)cc)5LjDFa{3`c`VD*jBFw*kD)=m{pEVYDDzo&{Bf(S zAKlD3x@m!pmb^A7@(&YUl(5YLd}e)(2M`Or20LUi`$p-_dqPOyu8jLntv-mGOnVSg`* zsm!iC=*KbDq!Cz7y1=cjItrApkkB{Ulml0GQ#)O0``q^Vot%u zcvgH}P%+Aja=iv=Q+;&rD&<@Y{ehH)k@^EEFO&2KQl@^SKag_ww*ElM=9l{FZdJJ9eX$>!I*IZHPNL5G}QIU)i@D(!1J#gJ~UFjS`bmr5ek2Sk&jOJtAa7CBj$hY?vaj*Z30n`{M7F)k>&!Q z$R$+SPcjL)D?IUrv_mF$@R3JIE-_FZA-Tmkd4%K|Tjddwd!)!CBo`_7rnIw6Zqh|2 zA$OH1d4%LH%j6M~%X|dFf!>?LzpsOMY}Qp%fA=*Kf2NM#ow^G0A!d0 zp(a*AK3g^1)GD~k20Kiwf;@Ikgo#yY>an>NqY;tD@-cd>s?{e z!+CED^UVI;yTX*Cr6RpLDU@v;QMgRbMA9BL3d5!sAHGS;tdxR^sO;sad|7xFy`5hX z%AMz;^I@Tk*_>Yy%9Hvd^I@Uv*pXim%8A-B`LIwXq~}+J&i~|5`LNKOf0@zw6rpoH zJ-;G!hP#ZxJM1}6qd)GlYd_B4-+lj(|0$ZRmsidpq=~)|Eh92TLseE{tWd2ygI*E| zrOfIl5=t2tBN9rPm?RQP8G2SElrr}>kxROWvC`CG_y77~bY8xERmyqXiYCZ2rBVVwAZ9Clqi9l%;D6Dn=Q3 zr=Vh#eXU{(sFX5oRYAoltL_zCjAu}8e56^vyf8UVId0WE8 z-azq&-WS(0Cqg7lWmEKr@yuGLH;gjvUA{}#gPK0K8NGyXk)i-SL@WMlnswGzroOu_?NdIGfU zw}=I*tg%QSS3pPofuw=A^aqj(9Qp%E2MPKENeQd<2a*={>JKC}e4;;)^pGYJ$Q5x< ze;{e1^km(+N~&mqz^8eV^FPJY3E*#V_~FZPxxdu1+nV=Am4)WDk!R`Hhi{}zobm9D zlx?dXzL7F&_ro_**1Y%djg%Q*JbWXa{fviiq%-_?UK{yYEH>q#%|~ad;=?!6*=UZ9 zP56%wJ6H(L34ceQo*w3dwXA0-`8mqnQ+GvZYq3!6%({$HsBE1v3Z5mO8lj+EDxP4V zEhyvK7@?rN3^GDN*}KvR1?Bh$Mkq)L<)<2GPSQ(nV-#FfQ;blM_O=?KAcbav!lx&+ zne{6+1SEua`>@cDv@QKrVsD}aqMvh@4{ic#iWD5w}^Si*t=*yzql zqlE<&ql`%@s2F9&_C>f4!%uxq-_7o~{)_SVTJtZWWynZxi>hqBcSU)oeBn(|%9heg zyx3RD7(Z`{Qr5(KQyGfo*{?Bk z-nbh8?a`md;(t3q`+Z0L)@(2WTZsf}YFliKKxLm9A>fKDzEb!QNONt)2}pHO;sm6- zcf|=vdB2MjkoGFA5^h9N-&^7Yq`%ch2;2%3Cm;=0OcZVdQsFWX)Xy%a!HnL!;D64Z z-!tj}M7ZAu)BZ3@Ao~VE&GZIo>MNfl9E@jpMT5EOCW!@;;=UCNCbd;xEp*JJv~aOt zQrQ8qU{cs^v0zeH$Qq$zCS{!z4d$vUw^pcTQdFQ=FsUgO!9F=hi#eY~$oe%gZEmRF zI!x5*3(=GmEE=k^Nn)W~MXN+YNka!kLP z-Aob-<*HdB5=xpmAQDQ7xq#3f`KLesU2hwnu%;&?`Mty3zr>pT&m`XhVPBbqn%bV* zUG@8o|`jmqwrUk!J| zs+&!2H1fu`%&vxoa>HoF>f*ch{`A-_Fnel^^Ee=@ro z^5UvnO-=!F<_V~#cYfBXU6*yjoQDGr+`&En!JnYz`sTma7vuh$8~iuYt@s>-Zv)vO z(B^CSVzinQ|7<3`${KAmZST4BjxdX!9QUAE^yIW9wwu&A5~ebItqa zSE91EyG%Y3?x3;eRzg1dp1GBfn^xX!YIBjNMw?p+IqP|IDHjn-Dq_6y2_>{=UNZiR{1*@f5(LXu0;pHg)3?&Tr2B$FYbV#gAoGFor)Zk zLZq@XGKjbXRWVLPo>R{_5xGn|<3yh8SmQ+GBzEIOwfk&6ut8q1U1teq^a^N@nDr*5e?=_{8cQNG{_DM9W$w~gjg`??gg=6Qd~W; zVA5J^v0zeJcd=m7*Fe!=uB>5V!KA53v0zfu3@x@am$t(& za&khzuYFR$EfVi&GzW{7NvE>i^60o*{V0o$JgV%clG=`(sgo=^@}n`b=*Wfk$)Y3g zxgv{>9H&l-q*FmY^Oih1?k-DY(UGT|l0`>OQuG{SopQ=Nr<3%-eFdBQJ^o=35x>b9 z;huTauFSH2S-UZ){Nzw+j$)Hdsj}sADY>(JB$JZd<>}8Pwk0`CADNWoGIM28lGB`% zNl9)~=yQo}NsiN3CMCJfV!4#uc`nGLB=;%xg~YZb2U0;s@V9t}DRC?kw>+i;^2v4)IwQ(NKXi5bi}thYN2dazS9-r z9(C#op)9U;QAaJ5!M`9R$IUJN3?2VvVmt0n^HbN!^A3o=d(229N1pNS-=+DUv9{UM zUr(@><wN(7UP{WdbdKe=iMXAO}NMrMik&x<686zPb z{?8~0S7wDD4V@j*Y7=85q~5_G!Tsz==rkBlzVdgq_{nd*?f*^T) zL7H0kiU+IgE74%C&cc_3cAHe$NGzDtI9e>2RCrJXSTL!s#$}-fBeg|~1(V9& z5e??*N)rnvRh3N@I%ZPSC{CmWGQt^DQ1>EwlnTyEN_FhlOd1T52R*WrxJWxvIVr4JTdQ6AdS2 z)kzm@I?`4L(Qr~%fM_`BYn*5}DQuf)IBD#>csTE~Zi|MK&PrYtJaMG71_-a)+kLgh ze;btlwV&L7Lq_|JUSC|*g|a~iXsjndQ`cazK>VV(NFY~~Lw_J?DN=u+XSAn3kn|I$ zKaiA@pg)i_GfRIUsb#7DK+?%-kwC7HZTbUA8+-Hzk}3`(umvT#1HF*bJ^xIcsucMO)*bjlR%@g*NoF}H6TRB3@SsS`(~Z)QvVQ|j>)g`R zbenThy$fbTcz>8%_w6%r)7%(^rtk%_s8sfy94fB#7jH^9KvKIy1{Ep(TNzZO`i4JA zI6zYVTp3iP{!|%M$KDpc0ymZ?X< z9cR0lRge?a{msOqAcy+E%qqyadfYbgD9F*SnOOxnUF7d39tAmI)jv$Ef}HV`xm9q- z9B{|PqaY{!&de&vVHf}TV9Ma(4y)ZUrsp2A{xVsOyZ9-D{QNVoK+PSy7$d;kgAoGm zZ1Lg*F=hXQB)NYmpaVubPK;^zEC-+Qpx^N9iV;v0#y|X5lLA?5R1j!p1)8eg z{%+?zi^O>fOdfp$>N z!nyC4X1}b>Su$^0&7o^6Ceo|y7xU=38;>qz(%zE?k11>tJvndeBPP+4-v&Nv51xJmRLM}5L1dh*dhMNFgT?%BDhN%Z8I&lfX^o}6-KaYegl_wcLCoPXmu z_qArslZtlFP9$7&zU4A#@GOaO8t!E0jna@~-8D)>&Qzm>!6qgL8f27)oF>jF4LQnY zqcr3k8AfTyA*wxPu!-r(FMp#nr1&XDX-L^8K+`wp5G3bdCHuCAJ>aGNeo*eK+B6X7 zdr2%xthqtSr%kOuWu45dfIEoOyb8!$wwPA|xy}vqDj+|4p`>XinVhPxc@>a{O){?n za<{L{tAKp3Y$?;u0XbrK^C}>(9A#bwJoY$N#_x;jv zHVm_XS#N)_XB6hlPHAp+n&Y`+KKDXrLf&X>2uSmM@BY`-QeV*G+5h<5QV+5Czpj@0jg0@kT7IQnugrhk$kfL~{MXe|&+|R1&G!kL z?A^q}d!8QXTU@yJ#Gg**cb&8wzctG$`9EPJPObGQ{mrgIWpA5X1@9@6&8&iYg~H`b zJQvgh^fI#wa{H-fRzc2w*32r%#S4`;@m!E&_cOB!a_8mdR>7V43p1-A*L|#liRXeG zHW*b@?}NMZ{Qd24cqRP~1HV(BbJo+vR->qocSSV^Jfb&5WuNN{;SO~}PY5|q@#l0L z3pq%2Jt5=>ZS;hY!mWBjNUSdYo|ByRRCeUR()wwe{8t;IsM44-0@LS?&*QE&x(YlMOn!Co-X7Nn47 zMkq)zHX{_IphZR~NKyNZP>{lI8lfP?ResUH8KJqSH;hn_A}1T6AcY-M z(?br?S0<&W50*(u4zgb+B{|A%xs<$*Z(K>@=^@7nlu1brv<8&u=(FC)rJ3if-z}my z)V253`R`uS<$uA!F&*2!4 z*cpFTom^oA#whemdmwwPn!>snr_r3@Wtns;drKZ2cZeyn=*T7Z$)Y2t$dE-xZqcB+ zq%%v7F;W&CxyA}vbmSaqvgpV?s@9Oyc61dJD2t9tNw3572a+;f`U6RWvHAl^ zZBz9JlCGAB1oA$3z5YPb%5MFEq>2v^SUbNY?}&aiOB3f&yrK8SHAQ?c5~i~2`op*y zZtD#rE!@)^MoK7BOJ|di4xZH;Mk;twZy0HymfkQ@Km)yDl>g23hw<$1s5gvq-%oEC zW&RL^VLm$V3q^hxI_urX>?MN`n(t+5-l3WzHX5Z+*%!trxKbXgZQwviH7$%#kbFl`=) z&jDq>+UtbCR@nucdm1UPKuw(q#t2lFY=nR-^Dl7%k1lML&XV5jqi&SkQSe-FMJ53#D3xgq{C(61f;?u4HVWTe=#{&PmB>&Je?4XYbgF? zLeBD8zxUA@WW923%=w}Re`(8GGEJ?q@<~;8KsG5?^%c3Kr1ZKCrM4-lf3jRsk0Zz> zC6}n)Nb2E}a}1D6N^Y`FE-5+8PqIn*NW4j7sclM5lpvRs-03fn;$f$}NAcWHa&CX+ zS}=vCsJ~=U zsqD$7Quc}~uA&SoQeYDqRHVqBGN?$QHW^f;*wHemNWpK*pdv-@kU>QXe_sw2SNug8 zROA4^$)F-fc&S+dw}G;&-xqGcGn;!t_P^UX%>6`ID8$nqbiK2O3#(~?^$EhV=tIoCkY<~yB`-D+~*-_6Uv5jVGcDP_iZtQnmq5a8Pw!> zP21^kJU%HE26v=)3iEQj2vB4u{m61d{ss39uiYp0T?%Yh&6&YF81IujC&V zq^WgKH{oEFZ59pY>inNrFsU-2yU=cv8n255lM1apglZ=BT@VW<)wSy>R5PjV2eDvM zS%{xd%`{(fUM!eY)yZF|W>V8#1n2vLPIh5rS6y(g0-nmtD^OF_55@>oR_j#*d%)Ed zBTnF%Z4f6QmDzd=9|9@uqBsGmZAc&CJs`z>EKWeGbMzJ715(~!MhJM{7Sd064@iOE zixZFvH})@Z4PKEP+(x8sUl~MP{d7EH=rD;7*@ z{ZK5J6q+g)Osc#u7EDTf{&k`AOzP_*7VH_xiUxC)%@7MFWo-}(CN-TwFy6v_m@u^8 z!R^RKdY;L2u?!T)Te#j9*A!IY4S_I~b<-clRTHH*jP$ZWZx|^hQ*Rh)r2LyYn}pQi zr#FmrF-~t7DdK|OFw#QJw{#vGUC)K<4Ws;DrZmIVep=VZ{~mQk^f>N z{~M9~PALDKlNh{g$A6L4>bbzOvJyk_6umJDO=(SJQK@XO94fBD$ug)&lN)7FkwU+h zK}C8k79imal6qUqpdu{?$)F--uaZGUI{#J<6(2no50udC~}{1eH&2IrA}P zI27N`{R?TKSo6o4ie8rgyB8r%^o3}O|5Y?pWmRlKwQ@!Ki-eMbXNZJ)6e|)+3av9t z;8;nKsz@j)@Tf>ADXwOazz&nbrig`dMSU$2N(!nTEU?3*m?VT+@=Aa0YV0Bk)hwux zri#1X7gkv*yO+#7^Xqz7m~y?7cZDgt!@Vm^`8?gb!j!>VyemvO`+;|bDNAp8SD5nh z>ET|rGmYqKdsmopuZwquDcfB670&%v*k+;dzb=+EGtXUE%f7Gl2dT^wqO&HRjc@7; zqO9DYFNm_UM5wOaqbwb!FNm`BxV|9D+NZ;G?H*X#c6tjxUcpN*R^|;)%)}X zQFg!Nz+WxN%O-RF-I6F2&2ylVv#6HY3-pGlY`4A;p4A`g389?6rYD3lx@3fo!>4?H zSx*RM^P74?D3>Sc3873rtS5x>I743u&*Cy8bhH@da0fjhl))nrl4nQHZ(4@B$IN#3 zjMY38&eNZB7uK?PkNzN)eW^EyXYw!lf+&}tbm}@L%H}5ef+(M@`hqBIz97o$vMyb%N2867`hqC8N9qfr?B0%`e@&$JKrZvYGsW-M=N?q>UvlEti+pDG zEP~tSHg|7>wd}8<8YaR0dZQ#<9q$?=A+=P9G;}zmo&;kgq^94Ek&wEEMHxCAQrj70 zB&5EM(T4Vf)VS9u3Ga8Rjx@9VAGNIMu>-V z)ol_DC%vVMhLh6jj!{@uE}0O(!J>0L@sf3CvL6c8^`@YvrZ9a0Dx0JyfGcK&_W`7l z1KtOaDn9oFOjaOVe{R=T%V4(tq>?sO;HsxlUMqHSjEb$@>7x(k|WyP?iq$K7g`x zoc95grHj1}pe)_%eE?q{j_1s7R^jWKfZYACHxA25DxYi3}>z{a_hX$+(GKdrX@ES zB%79;Ws+=Ka+yuCX~}Uul}$_Tb4@laInk5xk`s(vsgrzKKK@W;(~?^)l}$^|^#N!b z=WR7v&cw^`CtEu(npeGzz*ZuGnj4jvXovuBAQ~ay4%0!LfV|`_aRPFUk>Uj84-3T! z$O*QK6OiUli4%~z)5Qr$$HgZJClZY*>WCANHoJ)vkP72LfEy8cT@K`)>U{VE6N^C7 z#VCoU&VBO8aIa4m8CUJEa>z)*PfeE6W~AmOa>z*8gXNHs$`j;}k>a<@AtUwwD2I%k z;K?ac&L+7+Q#oYh5H>kv+lX-MVQjM9*@OTC@% zd_b0H>96EvjVJ0NFrWE=yaP2gcQHnwvT!2=T)}h12}rx|ixZG)Z-^6+UW?2UJ`1E& zA8`WGXb*7$Qs-!K0@CF^BLrNL--{EF7VnD_kP2(eR@jTVC6n_J@}4My2h+8llH{1P zh^DL|-WFBaDDR58)#6Q2(!~yMijo4(cvF;e{jxVjDXahXrYPlasX1P>D`o1--V~*r ze9OC{Jp0CZQ|*_al))e952Sp)tUr*lyWBk8r-O3+4Us^e z`BU`=k`6ZL499`k<#u#Kn0)AJS9Lk5wi zB0sr=DqA9xkgM)nd4#0KmlsHDLsID?d4#0iba{lN>gEfj9WtqXoIFBufj{ICk~_S) zNZKLOjLBYkgybfF%OfP0nY9?N&&SHl2%|sRXsR{DQMB!kSwHFQM;#0+fNpgq8f=OtPBncfe z$*Y@KFo|lOSTITH8?j&#kk4wNVpHUL7wz zEKestEa{yHJ3gsw27><87Zwzxsr3`_V3j?!Nw8+F&aPs?q{>BN!KB7N#DYnMO*ac2 zGpR31ESOZcT`ZW?R&$HcZj;JpiU#w(@QPS4sjBi;q1`4mO+;|3+=Ilfm~D!3&nMY2 zTb*-myAQtSzfRo~3gdb9J_tG(A<$HIO%4$hw@tzxarHGdPDDx^YMh8vIo&uBDfFmu zB2w!O<3yy~vfB-xHBxbB<3yg(tqdafXiAeP??!X=ju;mxB_h$Q&vil0Zc^@sk`X-i*_1pjL?%iCbh1^zZpqOvl_4DJb6ej{Tfr1{s4k&xOa86zQ`Z!<*Iz_G3nXgC<#~cXk#R#;f2OXNWH&{@q2rtQy%in z7!TB5aKpl3oX`y<>jxh~H?0olOZMkGp>`@&s0r?4Srk!#-9#{5d~u8`T4 zaG%^^ZYAWN>E>2Ko?8DiQ%^iOt;^g>$ba{nTM4=HKjv0KUOo78Q_l%G_$IR};l6&w z+)BvptA1f>b5S2K50&(mWsq5uB;SK}kbGLraoKqj=~ecEdGy?6o0>&W{@T|pdUDor zX3>+UZZ(UZ-1NLz^yH)LOOq0l9JIAr^yHmE=FxN4oNN|7`QM~Xqr6=cgwn zc*s#8ZD;JYH7l+hUY8cT^lIE8PYpGDom^rMw5k@DBXX$2X+b5HK!;_hH8gqKiJ2jh zmf_YAhXr*hbvjs8o83|wB`RA2ZDE#P))1R&v4*VTT;j5% z#%@fFU6dNTJ~e($YJ7ZZ+^*F4BdKu*Qe(GVJAM>x>)v`SHFk;Ob92VVn{jJ1H%!P} zG@HU}W-dQcDBoV=a08;gf+x& zQ$20Vx42Iw4lB?eV0VPOn{7m7wq|puh9hUIwN06yax7(6%FdMCDThdwn){E#Idxq4YLJ@QX?TEd0$W{QqN`?U~{TAPsB^@9sE!o7AqaScFg^)p$@0b zWr=XO4_!OxhA7U2wEB#QTW_R6)&Z54H1lbvW&w;;m9+<5FWM zrp6*6@$cr;xIL-yD^ug~@6-oPmOf)j+V-W?%qWvKVXY^+Wya=>Hy0ncxj@U;jLnlW z5)WpsoRl#=A!GVlA4_K3*36}wEXdxBY3q2o8$0KEf?x5oKWOxTQQ_pzkRbr(E>fA^&vJvSYCco63n$hldbgQ?rj7HW68 z?4v9pj<85o+ZX>(t^~N-iB&ddsLkmq{Zdx;J0eslR4boH@?W1Y^?K4lk>tM~d*FKF zqz7k`&SCz^pJWN+>F#i;`6Yk7Y{_r&r-N`?d7ioN9}!?r-V^T0Wvh0`dE|b552udI z_m3Ebrqsmppn^Oyzsw|K@sDsnKQ}=$Fz-{p-h}99e~9JSK|8^;?@FRw*(|_3w1;}B55R| zVpF6uk~*p8y*y=UOcQN)VnCLB#Ayk{0Bn%m7MgWfG{UhtU8*fKRP|9hz|A7;q3+HB zwZx!Rv@uQudQFQh)DniFj{{a>QJvPI*5U5eK$}aohkM37K1#Fp&g4y2x1j`Lq{H_{ zYupG&qJ?jHS@$c!o}% z9%+d(AY;aa%=lQ#&G{2E5)(34PRv|3<7VtuV&$VIY|rX`c&Vi2nTacIPCJmX?uaGh z;EarU%Wp1Nd}B|dC39s$=CpS_9h1_oySu?1(N^B!xSI-RGE0D#z)-8jr8*<|`HjJ# zCcqZzh_(%PmvHAClCr(ijC>@iN58)A!3(d%w}1bxef%F%NwFKdkK!ECmXiK``t}~w@#Nu-eqB4#S4DlZ z0xPAgN=ZyvfYIz4&)Pz)FMtOwPFdjNKANKaUi0nUl|oB*x5dz2SunPP2}Onn`Y7en zryjdtD;hh$a$-xgP5PMW9WhoN+?Q>H+jlsoW;R7;UaO~x} z@KK)Q$7`{MTZ3Uf7)cF9dWKuWtbr$X`6xwsGuMo%=U2<3C^7v1h0_=BP!wN98LkvY zAE+n}&j!9(qQXa*+cqTbr+{9Vl4G6ezlb1ZrYT+^-_*LV71lj!q`$(iZFMVh^?d02llr)`J4i-_FE{`Q8dEg z;5(jS&-)39-QHJ-jlj4*|<@-1`Zqti}{A^9q`B) zw7F8DlY>WLlETeD3jB6Dt`WAL z2Mchm)&jE!ytiACIej_0w%Ip!Z?a^rUx5oSw^$RG-`MjGI=;;5+fX2L@me3HN-vr{ zz(rKF-5MAGSF)pvvpJ(}a2R(hw@@naJ;gB2%IAz8w1z!mva9ZdLhuFjT#Kus2 zXb`4goc3_LWhlR>!T_fQc0Ol@Wy>u-ZX_$Ok}b#9T{$rQ%ArGPa~AU{07^Eu_g8t4 zaG1J6AzU6|Xoy0d#B&#_@gVsgns$7TsmeD0PCws*GtWILB!c`?kdOVSTt|TvblO6Z zi|93y6SXe(`G;nr`@$3*aKrSiNjg%{-BbUZvs-4^2gkxae5(Ov0?@Xrd+yIzBXYJ zK40Cq3tyG#ck-AW-K{Ou6@m_Wcv$kLaJN5a<0$3l2pbA7fW1VzBCSq4EcRJ^2MfVe zQmEa9@JOfIs*B@mlu}_Vu)6F!Qi>*4(At*F6?-$*PekRV{j8&+t(M4>M`F|vM|8M7 zgnM&IHwRUgoaC;p5=KHtFp#7n=(!n~(>MlMVhqU#mV8b_DcLCr|TRMb{7V` zo!nPjcAE>X;+c?(c8{<^&=c73l`A0FX|v)UR!j0G6{$cS2YNGp9TtG=D|e)O zb`SgQhk;u1M)YDBsqwZ9w_|Ay1)o;0=i1DDF@{}sPe@a}n8KUKFR!xA-0`{Q&0Bdx z$L`QN5KLLm;Iq=6ODk=x*9z%l@l4xgT~1|ATxu~*>ImiiVwBCPJau*J`m|a5)0fXo zo0gQeWTX3wYfDyLIdX`9$G@PGp16}cl&1svK;O;cA$gWB?SYU8WQbuYufX9_o^fwR zI`D7ZCvKhT|&O z7N|UxwtsiptcB_O64DY9uP<4fgTBHq`0BbU7}5%5)61huOoeB(wl`ow!_3O81G3h`=nli_e!@!^0zS z31m~sP=Qd-ls0u5b_4>zj^Y9b`^pDSTmzWu#{HR9$}^d9+jx5KL}KsX zm^pby#_WwsseZ0VniH_120lGtQ%ZIm84wzYQJ=Om4qUpd;S7jc$nh4FepJp@Ehd zOfq3agb5$6c8o22>w^15trH4i$Egf_gPb`DJN725u z4~Bbj(T6y3uY_M)woy81LGCFpD{ksk#_mX^dY^&}`VO-;SM(7wRTl#x40B)zc zY$`^hZIt>N55M@OEN~_#aecU(<^AIw;Ow{*rpB;3+bAuvB48jco*{O0x(`D7I3k15 z%iBi1geg1TJ5e*Ddu^jMco1uKCMSAME=Ek~=B!j<|45kBgWAyet+f#3mAfI`SI<+m zTzimTOVG4e_bt7agmgo$@wOZwoj#OBdOq)WtDANwk3f;vMdZp=wX-2Rej04z$G6yJ2Njhv!`bu0SO88 zuq7nW3y_fovUo@!iI-4}6QbwCd_ePcVj^Y&l7~o5Pt3Q8n3$R0f33aGu$-D)9 zwSJDpIP2*c&U*7>v0De~r?!ubX0tVh9mb3eD=;xIGCI&?NuPSC`0+IjaZM|HVjssH z|7U}(p~fSfef`bJeP@1&NX0N$-0zJR57tTSfR-H*#|r84wzqj^hnbIuP12bC@E)J@ zqj=-QTZBoZ`rFXb-)27!k?KG?j9N56Y^b-A&abVm_NTRxe%#D;oS*or(tdq*bL{5i zuVYKnP2$!XZg%ti!^=@0JY-o-6K=O7gXMn+rB<%|;)QYy3Nf&5(Q}-o-jc-v~>ck?(E{4dMDl*+k>xvY@HGev*u z*OxTES>tzB^$A7-)P@>qpnkE#f*=@2lKGeSID#>5Pi_5B~ zcgROHD#xuBjE?lL*wLj}L%gjdX_KxHSC~PdCSUb?%Ca-Ir!l%3J$2c{zLo4F7nr9v zHHaV#;HHRgb>Uf9po$H25|qi8!ttjtaeO)mz~n2%BTzqv!Hn1c@>V-7J&cfDbYYfM z*w&yE@b*2NJ&1!L&O8&IDTfkY$=`(r@F!f2AFy3kb@Z#J+4&jfp4dp}Wf+6M@YA&P zeJ*p7uIXn!%8=x5VskS~3kilw@{bINyt;}h>2$rlHZr%bkLgm_={DR9vg?L_QX3(% zAI{*>!CFedf1pwMsl}-(L}sL>FGK=Wmf|H>cx6d-2+gXJe7D%Xp1jfre;OX_YgBvE z5qq-UJ=7R(EFY=F*A6Iz(5=^1LXYnlh#m0G{-ouZp1!{1AI?9s^ZcQ$QV}{_HkIL# zofqGE=fdm9Ctuv$cF(PinO>Xx{e=y?XkB}uQ|lK`JUqGaRMTyDd`R2N7S;O3 zaE7B+Ozb22%&>gKCepFr?d%<>uWYOy7;Sr$!KozuQbF&b;jY&1&)m7Lw3n3>9QC$u zTc4+SQ+V)N4y0(|{HKz1yG=RR#1o1TM&}(;C~u@b*l6qD%pKr38>Wp8Z%tOT#cv-BNtz50fcV%2)P)@-fA zk5N>$UMe-;zAspK2(^o^?mN5g-HRsyUaiYQ8_7UxTf94HU#(X@<#?+FYosHu*vbEDTTYyI&<8XCbdMBuaaDVj_bTmWR3UU>1`vEAp6{c7^W;~<3jUqXO{DK6|jcJbts=&rJSt_@-|-2SV{ z&0CB262tZWO#8RvRn}Z9wr0 z?8DW=BhA5PC7FLme^4&-wm6V}(&)i!WvZt~Fcx6SR+rVP&E08fC~hutE zQJ3E>DoK^(x_cD%!_CFanzyi89~kC#rg>x~4+>h*vVrFi5%esS$=yfEFqwSfx0Ab{ zJ$K~Qi?958^1(yrpFRp8({IqceUsaTig{XqS-a?RlA9U;M`R@aXE-U2+YQmqGT`mfx}3{BcUvCAM6Y_FKG;#*H6 zVOx=uhFi5&FxJAj@gTdZkNr*{cpyt$iDO6K7TTK>t}MK2us(4#Wm0$~-Wud&PY4Vu zTV=1oE7xixt@|yGKeo`t-4C7Lde9<=@0?u^`1xgB1%u-F=VFozU@_)Mo2e8-G&j+m z3Lt*)_~FZsKE-BAryhLj@~`)RgHAnn#lnlfd;;7*F76i&8Z?+VqTV3;xlw0F!woeGp`VIKBN%pEv%26uO!7^6+~ z-X;tC9C`0?`l5g%aH5H~hA`CR2}mbiY4q|x6K{=8oFLL4Qe*_g#P%>()x3`T2pLeC z)#2GlN+746*fm@+@xH<)9C^F6xUtSZ^O6vZw!PXn)_*#&<=oL7*$r(i=G*V~;`YlP zC~i4#>--zPI{*B`?G0tSa&c)_4%D)J8N&E`!+so1vkgg{ub%zd^3j!K(KMUYkIM2^ zWqoPLoquo zo8!W7ZSDN9xc_xtX5Gskx8yYrUvi1}80wl?x568%c?YLu+zR%jpfX zZ9&BgZ6my}AVdaq>S}}~NrLz=nV!iHwqmVtsjan@?b1c3a|aPa1UZu@02zw%ngv5ch?z!rcN3>kPr8Q4 zyudz+&?uOaCrJXUBA4^3$=w?Ob+)9Io`2@8$%l?zdTP7hH0$E_t-7h0M|yty_=W8! zNZ+`$<-o;PJ+CbJmU2kbu|bsh7)SAw+G7bljnlh`!Sxc@99?$mId6RSZDikZk}j*& z`-SnW4)K@b%~DGWnWmvD{(VXxwU2VQY&@zB{_Ho-yO$kXV+??4)bL3UH-Uu^hfm3& zV>RGn9a4DNe6Rc~?q_SkFn0{jElZhlfGdHyjA0wa*GE72b(&+*a))=2--$r0n6(Z!rp2hoRQ2|`3bfHrYhbE4~yBRs>I?(HdVlqaXA_0v1>^wo!YSt2Wi zxLO6BoC30WIs#m}CQ(Z0<8%o7QGMJtPp<_FJ?|Jpy1m1Z~K8p75xW$>@F?*qCpwc&M%e6^7$#U{I#!#)*Fi}KCW$9A8A*)2VJtuLifMM0?C&{`M!L0T#u1|%PqG! zG?Dax07bKow5}8hKZb?~el{7yE$!y(<+_&j(mDh}&uE+rD`!?qW7DpmZ?>+VmT_xe zMRUR`YI#Gp6|`l*NXHd5x8=T%v97+Ht*e%{d~tpLwkifTabI{5lnNQMdcNpxDoYrFftnGWgv5u)!fyNbW29WF$5s z&g5_G_6%!~W$p=fdimAkrBf*BDtKXrNw}ux?uW@(5y*tf&KV@7BI!88Lf2-Uc`-j7 zJk<{UDo?+AGY-$^?aT-#1DSC#@#vcm{OsrH&k(2S19SZkrV?NQ|6cU-bWwx5nQa8=qonFn;$g!{QfiPQIr{7B}nui2g8G^a!jS@u~PS)@pW5puKPT+g^)rdrgft zm9AQPoxhCV>kQ>>dPYu69Cj#_>)T)YAOm0!L&S*V9ehCJv`+g~_4lvE0|O6fTp zy$yvGK>1lK;qTV_07I&^1*`D|`}g44m^gvD9DbrJKdO$NdVR>7xhGx@Ki-_&E&z`2 zNjp}-=~R1^Ot7rh4a7qJLl$VHu<0W~xd$*K8jE=tg9#$~mU+q}Fy5LL0|dv!I9W>r4G!4FRdu|OnFnMzJS(b`pXr=0SS&cez>0Z_s8QmOCrSd_$ZXurfW?(t z?#wwoh3CYd;|UJKnG+NL5Zl5bhgGA~VUuEQl*}G>8>+bAw+y{A>s00W(72mxv-E&I z{C>2ElHHRk>p|fG*D*;|Zz=-$=Y=-56HNn zu}JZgA%W%nNdB>%=%?TwMM=JV01!W((C1HFKCrfx{zpo*GXPDOhOu~wblB|hW3edg zgld3=)ddxa5*M(H+=A%wt__<4@VR$Ese#l+<32fl*CY> zRZ|dTZ+@ z+`TR(?9`SwA9Prs+=Z?0$vv3ta5-99St_ox*a%tn#nh@W(BJ<1JiR`wPq;|NY`@mq zr#cSM_ZQ?dL-fOz2`Z9c)`eh5OHbp&KO*t!b?jtvy?x7_d4~@#GE$%{XS((h|FNcFaY}(=YaewE#mh29nR_$9$ubFn=ss}4uU+{7E zt=>4}&efY1wZ7>?cP`CNvv+L@7S_ARmI3{YTX-~=*osT!ySQ}#a6;y5;--8r%fTv6 zpT@9_;fHT$b>pt~?M(QxxSiR;Z+zI6E_La9ts_-FY1!9diV3~uBY5zA*HBjb56*ZFk^P{GOUzqtS1OV8|A%SqSdZ^kb@ zzxLwZLl+Jo1R8L0&m+3$x0^4m-Fb1(t9H*d^Q(QQcVUf&_bBo&rjh|=uPOJt2Y;xg zP$b_YFSx1XL!*x{*T`(iN%U(VF}3uL77_`Z_baP?99bmZERprqSSD{R-=(mBe4oh9 z!iV{5W-Y;06(jtGuP3I<;WgZ(#WCyp)k|ng|0G+RWTuMfpl9kid-Zkm ztHc_XSI4lkjaVX3A8B}O0f=Q{td8>+H3U_V8FfdL=_F=nf^Y(&M2I68qC@aJ4k1;+ zH>Jt7nU~I0K+cEU)I70OYyo6rYz472o?OLCXQkmjvM9E0G97OnB&w5-uh%qjlO=0r zX5m0>kbFeNa~nO7QI2ZeMkdu!UVn|pKQfdGfr5z>ed42ZlaP=TYj_x?)sh`&*;knr z7r&y8nI=bQF!gX3k+{U?R*4AQ;CiyOGsC8kfso<{Tt2bazuzBphFOK9ZJcaaKGGP5 zsFdSU@N-T+f@cOe3bBTqJvA=mHjB)2D;c?Y?m!d-<;87y3@<*VKEU!U5LB0&ft%5N2N{knye}%Qe9HXvqSy0iJku04|v@ z!ZMPy_I-dG4Db@4KGtjt^FSX54Bqds%=Z$DGhAQx9F#-a&r{dl|eg3Pj1<2Q!vl z;fcr35Zq-6pMQ8QB0FF9FFYhC%l?Ni{(7TUwB>7AqH^i};7!SIenB_Sy)QZ{vt>Rz z?L`o>1)2<85AvM%&rVli?6?nQK)aOAk%`PksU$z(lXSQQOB1A|`Ab&9!ivS?e6$uy zi|&!|ZD5Vy;k)r?qQoI0C^Vv)6$6C4nSdSn|6!j9--d+an!GE~t$fQpN--8QP&T6U zTY)R2_{3h8K^0`2hR?!o!S4}X&vgjp?lF%g?$8L|zz>70fyE`v3Qsmb;o;W^&vB`e zRMjXOC<9D6sFf977T{0V9zqc*?htzxJ$x7 z&J!m@#1xJa-~Jqk9k$t5vQ3ojtn94OPWfWI{(|o$U+%i{wRz69jgwIWiV9}F#(baP zl7tOzK61&nL)a(^9(LRkUyEC!up|F^5*RdTs&SViXt%AVG142VqcI zj@D&B_&5ded@uOdgCc{3EIGtYOUag&dqJeeJiO`r^RG?rdj%NBS-iJD*)tzZaApMAscPmpD?u7$fe3%2Y{ys%BSKC-&2g*e;6E@ZiK=%@Lq@4`!*=T#+j3QK8vc z3b)IJ?mAT^xofG-cGdf8nCjS7uq&J~WfwSjV8BM$4+17dW>V>B!XZhW1sn3ymi^@_ zR+v!^lQxH#C}%|9YWQG$n=FpWP58bJM5QgFK<>?UThP0_Q9C}s-*3g8{mR;@om=e` zd1J%WPSPk}J0gqZnZxGn4hme$@v-Dq@+F~+VYXHQOz$Pl;<>&CK}`k8Iw@ua>=9y+ z5iKcmY~FvGYpcL#miPooBk1;~uW3!_6lq@)VMVxZ=USlcf1f%9BZ7*@7_U#x?7~;Q zmQzFEVi_qo)x@`R1aJ3xb95_L623g#ZA9S$!|M!so-cTweq09;|C=RUcF-Dm@6k$f z3&MC7%SpPRt!DI)Q?Cb8C~)kdWeQV-tn?vK+X<0%^tQ=jXP{yE9uWr__{e-VUx_>mP>Lk*m_nsjCB z;j9JL(?w0V0K%K^_HVM18M8nN)CfNP--#l9P~$wcCp&;IzPoMmF|lG?-25`EIg?Ku z;OgZ5_3ocI|L~KOC)OfK%by*&+h3SGaP;DV7a*=yMDfwBlMfx|-mL8l`*(rmgHVbZ z92XxMzc7Au^5|RVf4%nnj+Zmx%$v`>vs1lz`~bNRm)1Ue@!f}@1G@0ofytNFPd>iw z(nF7@UDJMFSDItiy7ZY{>G>C4_OHnLk$q$GKWPLe*FCRqv{E0J#y6Bcf9nn|@iCX4 z-aNT=N6TzG@%-e$N6Fyfp7UECwou9D^Z{=yeD9OHex6=9u=djN7m2@4Za&FlKH+>T z+(bas{aXX5?pG8dS$}c=Bj>kr+q54#D;37dXEJ4==w16qBnLjRc-cV1!86g?z^2gY z)H|_j3@xbvc-q*G=8y=(g+X|~$yQ>8kJMI!QCH%VEoM-vnCQKn239Eo+W^$MqSd9P zfF(ni4$2sJKsAA$rAr54-&u`sD)xX-jwOrl>Kz*KTOm?Y9*bdLD`}@b_n$sMaG6$# zitr49<tW) zhL8*ZMLS`8YE>n{nJJY{JiUB5fpGJwjp0q38!qvaV%GC*KBHl zX&cN~0o9p>;i-%wg(Th$S`l~-nT3kfcRN2jVzZ}?`=$Z?o~kXNCfHflGN3D~D^OFA zzOr3;8hK0p8h^~~fjkXONrtVl(q-nZa2Nf9ECMLYf;=wwrc>8BaST9!iw_>6WvD+y z=$r{7Vpl-~pwdLqO#h{>46%=9+T0X82txWleqy0cV;?c6z0a)>eipW1*fK#;y%<3G zU;AcxYUmK#xVd@=mn&UZ)|l9@-5fVG00>``!>wDGapR-L1OyjkVc_(W%^qlF*zsgN zR>kp4Bk^3~yzLgN-B=v7`{W@2&$ODKCibqtA8m(I=uvSH+rEYsi^{0^8CImUG;XuP zNk={c132^GtwsyChgCr)8_CB%ad#b5uvZhH`*d2EDm&tBb&enm<(3YMb*Qhw^5Y3U znAYeQL%?E0@eCrdJaBP1XXA|J5nFw*MTP@7Egk))kQ-EkOkClFUa_{8E_3aw3 zLKgB`;yc-XR<0NRkQctR@o>=WPailtj%zb7GXl`;4}N;gyfzODXQ`qJw2s#lRNFH&;3_GXF1&OMdgGW8VC-~uluqt` zm>j?vhlHo&QOb8k7KLWR2#}bpavO#{}Z{jaQV=iQ_nqN znF&@msACRp?r}=n@85R$agk!fh`AdBLzxXbcVFInIDMcz6<(?Z44B?PwyXIOO6DWxqIUgQigTg2kYS6B}Et)^K%<~jpmV)4Ld&|ZDMOi3tYkp36y=w)HAPrq)!j&%ccCv|MnA0hQ>zMVCdk&mYfQ8 z!z3%$j4GF-dyG?~k}A@J#7{}yz93Zds^m9WTfNSaii9>ylX$1t&R+8U* zVs3WkIF&7;29YTr5aWmgl0#j*+c*)zp{;*S?3$Au*>XPAe_{TF!h!AaFjpJ!^ea0=|8neZ_kZ&G4o4_zFjnj)!7p3h@VIZ}4&}<=-3)qE4ngF##M_ zv6yL>J0BwF1k3`4x8oo3Qb6|4f0abzYHcM!s+ey9Er?txT1iRoaUU_BP3}7aPFm!jtD+eBW@<9?r-}UiafV; zgB(CLI&o@;1B_vL8ZmljpvS}{qXX2~0*U+7jR%#w6&~&lZIya%cU4D82I@xDoOX0( zr7+jRMXKU%bQBXstphqdwPUEsMXz=Uc>=0B|0A?F%6efGYSR1hJPy>y$ST7}bE8~G z7Ho#kttnNG(8a&83pnREsBzx-1N-uBuA zslNul6p)(p&mE?cYyJu?m-}aWi9_6Yi6{GWf4S9751oIUaxO1-YWR_4k<~wH4gp#% zF!cwdJLhG2?Bef^coTQdvO?8Zv)v%ov3f8#E(Ry|i6%099$+4-bg^;9nz|u0RT#=G z$%2S`?G@0qbwKAJQ~bqqa3mvqL{csbUvolge+({AcvFZ9JZvX^&JbS8FBR39?_*C(tVi!vA8%bf1*ECX6cmf&ft%ki~sE6-OI$224@W;aPM z@R_$D2Ii%W4hMhMuS)VCtX@IuP=r-Hf^nQd{5buy^_<5tc&nAGkOeJPgMR8>GF%vy z;nlLXVG1Jelma9vSpcQik1isY+;D_E-L@RCBc)QB5h^p(I^Z=BvKWm_QtQIlm~6_v zS&dKtGQ6KTQPu$d1YItC`O&21eP+o(A5^MNXWT(#Q6IPxW`q{_a27icMqkkF1AoCT+a@=s_B#&|VoM_Gq#Q!y*gLuvRSzyk)>yW&U&A?vbyd z%!|qVQp|*k#{>=C5>ZhT@2b3C(45De@A-djQ{;WEB*R{Nf-|F?04A<=f{HxOeJ2pLhjmhHoKK%ZTkwdgAH8@`UvcW77m*jKAWnl;4qABL;$5 zN$Q#qnvaK9RQcJ%B)X27mV7zWjY1?swW0F{zOg7k>tYsuRR*P1y`{i#eqYzU8Ke$U zDC^BI0Z8hsNt@Tpx{`0)M?lN`rP#t2N6xjUVq4d8U&yhRtTGQ@u1HD2%HR7nYg5*> z3%%?w>Nc0$N^-=gN^^eeR|`s()v5JL%=ry-*ta>-t(Zk^7XFRFB#-0! z^}Cdj@LD%FKFM<k8!Q_k-)n5Dn*#J(0!VGW9#FGlr83j05*bG~bv# zbnfJX{1#s83y~YL!ejmvi`Vbdv)`fMm_Z`OD73mN0(YVs|4t?4x>v(9K=pJmVk>BX;IKKRt-m&p&br|k6YP~C6I;)sW+rKgV#O&k(nHFu+Rz&}_( zgd&QZt~tIndlcHLxjubV+sITdELb1e63oE{)`|wI$dk=^e-4Uip7DBt0p|dIm72A9 zP~`BynIKofv9MtIN1L#~VN$L)*eXprHdsmSw6fG_sH|J>b8mGNqXBa+x9=)w`d)+3 z!4AZEA#8(ilRGIzNMR0@AFL$QY|GAVmJVUYO7c03>Td6kepObt8 zlMcm5hWadbCHd^d{YL;3Ts*KYz4z{g{@b6Oe{9qF*B;4#_`+PJg;ezeaA;cSM(;qD zB!9LLP|U#Cob1sr=oTy^qF;Vvor6zzezEm}m0i^p?-Tg9tWRGQXOp5o#~p;5cjR-I zUpRqmnA$+$pWXS--oHZkR&o5P?SFIckhy8CFVye!U%y6Q=E#L-N+evND3n${`W>`V zKuCVqXBUFx0h&y=W(|k-tQMMSg(+c59CD+{pZGr~k3E_Eng4ld`ymiulgECWd|I*0 zAEtsEl-)G3uNv-ZeF#bJ?*fqJ?R?JPb9NjEGx09?tx$7L z#Mi%naQEdMuMwSzuaA#?HoVG#UEb6e@%M_zQP|yMi~&rI;U$0SFYzxJt|wP7Y&7sy z!g7XI7hu-FE29a+PcBekBKFEKnSub!9lVax2qOYsNXp1PAW3jF9K=B!2@`*CbK&L} zn)Jh<3Y?kVIif?{ZnK5bDhMUc#*9@Tkt;$4KS3&}1wi zuaW$&HR(*h7ETj1y^tIdq*!~6`0F7$8*BE~y&hZm znxG_bEU`YLWi>15x=#(|Ru%WLO0km2m)r#O|%L(c@|8&%2q*pKmE|T<8NuZeqADQ6QBen8Av|6Ez7s|G;G~gnWsW* zL8DT%Lfp>S25_(-2dyVKh4qJv#Bb;7RcA71$Co?wKI1z`z5`AxDqCPl4XlJb4CDp| zui_Q%n9Wxu5_kzr&UMUv#=(fsO-y>fC~f4bnPGvlLAv@L)5`FCiqhG5ONwq#fxK-@ z&~0sK1jooYoSg=}Kvw`Xggl;ICPjgxwCi=qNvrU7vo)H5vk{D{EB^e{&KGd*fjHPQ zWts@YtFLj|I+1e;RPg=3w^;xH)gUR9ge7zCDMlY=L4OLPomB+O);G0b7i2mgg9&B- z(69}!dF$fvD&-m3252A*z|zh+%_xcZ#_XKOH^zT6uN%@Ya-G1;0sRw40 zIgDxqMHg6`?IVb~506#}17-Ql$-hS%pS$Fa^v)V-OaL0W&ubwIy3A0-YaEea90AUm z-C(;yXS6`m!(cP9-m_b^V=@CdG9b+Y(mZ#t&TEvv*^kV^`1nOuI@uncJn-P9U%d=s z_}r1FGW$d8Ui+tT2OGz+%je#GVRGL#oD}8jj${r|p04c0U8xsv*1p@2lwW<}@JodE zE7aWH6)9JD?vPg|qtwjX2#GC}BT{>vFEkoFW z0^YT>jKonHY;*1epN=rIp&oW&Rx8Pic_fPO>rnK%yH870qe*O=X!l^FJBF^nlTgz* z7mkFML(+^b?f~0j#z^6G6ThS@sP`IG7i6c`2rGf9FT9&}Dh}}}#b~SfIRg!HCVphIe zRku{W8~lvQcUzaW$N=XAZxl+eq*;b$G+P{{B+3iPc^^Uk7_Rn#*TG`UThjI1gVh|? zWe|`Zhi@WLt7pge$(z;6aYK;m5sE>cUH2>jD9H)|6ifGis4pOQew7~`|CI}v7CdBe zkW2N-x&NJa!RSZrJ!dMA$oZ{(#RucNEyc7$LGp|?i?0ZJzP^JeX7W%3k6Pj6D?YSy z?bM4e=K#WD*F(8Zq4{_CNatkiCCAf=lGlIqnK{0R1Pkrm`nH*t; z=5X@8jTV70>It>b;jvcN<(Y%C&7FyA-4>E7)~8>e<{#H3$*Yl`J5XIo(HE%|?rWrC zxT0vnEz!TAjjRlAu9hd75JLw>QhfWF*V2JirF$*BRvqmLWs?ngRb0?esDiNqVO5C1 z%|gB5Oprb=RbH7D#=sQq{0h7fR##wlcH~b(gU0cm5uhc7yF1gLad$?ol-x9}(3#8ZLuGq1ucc*^ zzMUz|rlxZO-(+Zjp4>ply}&*bvJG+~q?R|rHW3}fHl(?>f#Sh|*FGAzh7#kZC!GfpHy_$cum=}&=bg{(jmpSIx!-2;y$yFi#rsSRwCP*8h zg9v1AB9L>dm_ZnR?$J#=6z;gq;%i(D;RC9V*;j@p(&TUKMkx^Th@Rm1#!uUlyb$AL zNf@g40sC)$p#W+302dkcG`_0Zhdvk2GM0Xf8H6r z8lBALaqftURi)YU=ik^xC^+1xB*|@%k%}&eyQKHrz!5MDt`&dEJ#DZ>KevhK*+&E07n{Umz6^r^p)N}Kc|8TgCr4NLh5_9x*PGG-dF)qHUUc-NWZZ46`6Vf3-OIlh!Uxl28+A1Os zSzp-%VXTy{j*%3C?R>V8J;tved!W_%2NiU1C|_AJIf2#_Du?yvnYRTq2mX%6V`b+8 zN*onUQI)dSQ3bJnU<)}bw2Q+!>wdsTgSAEpQn5K;5P5u-3Wk;BCSRB7EWnq`8pE|& zoXW$D4GWH9)kBa)!&>AJt#C_y1Ye09?@&fpl5aakDD4W*cclu?oLDb=!?3dFpE}Cl z@j4H8Y4=^9y)O8;-+trLy2mJ3qla?ugx{q%>Y;mn5>P}W8-nCmCYB4-2xnlanZJN) z%oDqcfYyFhI~U8)N)`Nb5K??xsl=7!4xv24DCA$Ey}6o98qlXBt;kdvnInW{*a5}w zBC1c2jrC_RrQ(1xl&c6*Xj{sp6ld7Ibi6pAG==@r*QAI!r6w0f2vgol)MQ9hB&~Tb zFR`r4NOhrO9wz9+Bq-*C2IbP?fN{zjlBHZ7M8zXWO%BbyX5$tfVs1%e1pLu#anr|^ zStIP>f=#KNZkcB#%ktduU!NU+{OtH6xKiVzwswdm6&@~8=V_LIns?QAz1Z{I7A#OS zKk_Dg@Rv95t|%HXwRvap4Q7{y9Jg&#x$$_HiwML*Sbf-OE*gY^WL7Dy1OJ%tK>aX_ z8KuRlVCYG#)MeylqiaUqkHK+-*g@BPsucFEB=KJ+-f(jnt81h=u}_Ign(c7CDja{K zOB$Bk7@T4QmNx`uxHVZOE-sqPtS{0-u!s;6?>%ZFf>XD^djX!~S0+6S0AdQ5(sHJ= zp~7<4rmQ4`bCs1bKz>Kxsn^NY^mGx@FEYagjldI5(q&>}>gCRU{ZL8gVc$^Un;&Onm@H=3Cs%2Amk8gR)C@$~92jdFci9TUvmCvMJpj$B;m$rqIg13#p;OkC_+{!1?jkQyhoo${e4c-#~z)uayYN0eZ|0 z_}c=3c<7%B5|ntrGpBuwc7EQDkIVya#N9oJ#aVOFuqXkn;3fjpYpFyX*6Oc^S6u6H z`pJVYTs*M#-0{aUjm%4}Wcs+~>FeR%ZO>fX`K=lFUVqT=yKU|Ya_wviy1vX1w1H~^r?6$q3H$`hD<2W}(E9VjTzUiA7b z1yvsT&N}gLOq0L8NB5H7XMUVQ#g<&J=eTU5tA$l?q4Ak;I zAceZsd1V+C$psr-MJ_4bjsGb24k2TwB|1FXzpT-hK_=uej&l+aC1Ir=Io7&TU;EAE z<8PPN>)#aDtMQ)LZt?XBdDq45JMvX~!xgSt@1e(0eF5Ltc>yoAPjM5e5WFa_;~PG7 z9YZ~|Ph&#puMp--lkZ*0nohm9*c|4X56kze4y&6puddgSrl&yyJnB=5@_$x^Zz&@E zJcV)zotA9Z4_WOuStA8E^7}^)ey~lU)4)Ee!+wT^fBlRL-#?BNM|``5E%0|AvIRO0 zX!7^J&yK)Bs)}mLpH(g`@zhfI)U8$bJ~GUZuGz8x*x;4*;2-V0OOoqv6G4LsY$TMW zS9OmHu8mik)GKs*PGJLN>Ao;^^Uj@o;o_T5lh=0P&~dO-*vwvUK_@VoZciOWn)}|E z_O$-Nh3AgY3`P4w16I(521GIR+~K@|jwkwUOTB7~2<-r8)u}g)J|%U3Ohh~NK7&IC z3|Bb$1BA;xmQLxDEX~h9oV4v0eE|WIplN`up_Rfkv>9*9bfB%< z+RwAOR+MFokPYFT69ZH`1mqK`B#nIl!~`tP(wH1!<~dP?O+J4y^O5B(`HThLIVtd_ znlriDANb=MFpn~WnnQmk?3BFtifQGRkMxFt&$OKtE3ps>S@xv2>lUf!3b&jg@1RvV z<9e|GXFMV2&htcB4nF>mRObkn2-(pjeN*U;YQ!#Zv&*K-c>(MnPA3K6ddek%-QcgZ#d%dViq5RVoiiHI&Mn>+*i*`sQ`G*-LM7b1HE-QGbBqci6 zjH0|}2iYO-Kv8}amV`MD%-lh+Bt0`qkOOTJBqVEg5+c)7cG9#jyAmNn6~n2mkCZFC z_$rRFq(hCT%z=>H)=8EqhTK7(DB@y8pWF&7kYGY1S`aAl2nYhjDm;~?N`A9f@~1E@GU3I`E#{CIy_@+4KqD+!;cz0 zbbz$f*rjHBZ-e}>gs@#QEXVMO-8r9w_kOel4tRh;kw+r!YVCQK%3E-LdqRAc1eOg& zb2-jJ1VF^zN)$LLg*gdZo0DkZg#xD7Xo*fYgaG6$48O8-UHhQbEEO`g+u;ja-2ati z!5yJTGIA-M&9lZj0rO((WJ(s`G0A#_rR&p4zSokryCX$~@+NT|GKsMcL5cR(e253m zt{XR}AzDX}8ObeeEjpKl(k#t8nB{R@HpW{E6R0-i_TVH;ZoR(>Q@qtgjw8}((%Id{ zWt(Sho!aP}xTiK&l6i~cgh>sE^VQb3Sk6uO3{96p2ia^r&@pGz=xH-QUI#aS+RCSa z#ohX1iu$$wMD{`_iVdds)DH2QvV;W0yT`^C?JVtv;XF@gV4to-Lcn>+D}G1&BnES^s< z#e`D2uwVa7k@LfU`jj(t_&@#>x^2oRpj`OB{}o36s#-5O%Nm0}Rbz!O{Il42{5-p% z@Nr-2$bS$Tn|ko6%fH?;wQ-yO`!jbp#;6OkQT5+L;*>R4_~?IqW+$PeO{QUHpZw{G zhdiy^oB8A&P(y_0KSj}>OD~1?KQ+1a@k_h5_+6itGXq#)zyDL8NpLaz%P-dA5dOK} zyQe=(#`<)Sbn!LCmOmvr>YdMbYV-r#l1D>`At=C#n4HinvEx*Dz3 zo`Z)HN+gQ@0NxAQ>_B&4vj-7cC1}&XtZ&p1;1rp;4e!v(THm0;&F12iCG1{-%!L$+ z=qNe6l8h>de2C&RJI-%GKj^IMRS5df))r?e?1Pj0pAFv`3gEoC0l0sL=J$F|?LI4B zcJb9$*;pmzzYCG&S)07f{9icA4B--MORa$)*5~4xq;m~! zO|&_Gnc$_O&?UGgMwwwu=NiP*Pl{qqq=SvLHyd#Mhu~ zG29-40LG$N)zKycKipWxQpfKQ9N8X$L>Y+bt5OUyr&UUwH7#w(5X2m^bVb~$>DhN9Ci8|dFs-aWKkFK(`^q2r9|Ca7D zix5%_FsQ#a%y|u~G4Jo@q*%E&5R&rs*~X`@^=?Rh(JDo9UL09kt=rDWTM|WqvDna6 zrmk9V66v5;Sl_JBuJc3mKvNWan_=t(3%1ew-;x>KnHd{rYo4!*B}>G6DI}<_f0_i$ z*2I!>(HUp?EE0~kMo(w-w93u%Bre#q>mSyUc4({KzYdLTL#WmbQu#WN%F~xNlNkDs zNoU_{r(*Ix`10OPJ$-m;xiceuXN^(?JAu*H*)7qfjJdKd}KQqpq z5(fAqgnW!?K=2m$B5{eS%UJT;&Z|Xb6K>IZ|SBiRB2E{urVjOLUAsd+GW{B z2gO^d&=LXeVpp6(5ptg_{$!kyq-r}O=0RO4d)xUz#qNk-p=rmA?)TQChq#Qi#~J^jEU+Q*PP>oJYJv$LY#7J9MlvvD=sQ3db3S!)=3_W05C(%6!X0`;!tVVz zB>PEev=G2-bql!G=yGLE=>_F7d%yUIS`lZpm|XC1hK2aC)uZv`qv}V%G(%|NS6hgv z5rO-?6{V3YRg@eHI%zK^l=u5-Os^!rr^e(cRGsF^vqWdzlsZ`%3$>rHnF13L%% zzbAEd=nt$w9r^<+P?=mysc3_L$|!gK;JVb2Kd>$p_9Y0}f9O!xi4d~?A@h!$I|Nja7>c9VZ`jyijT|W4Dt~JGQ+ka6d>-9gRl9gfB;YJ5M zZi-h>o&Jz&*CSV~b`{(UWvwd^HRWbk5H>TYUzv<58)(1&MC3dCFQj|Pi6*RJ3=CzO`OO?MDWO6J$y}vR50{< zI2kL+-#W=d=5dmj$;cRq;*0i1&73>(s{8X~rYRQ;rU(5lf(Oe7bF7j#Y2**)@YCJj zUebU{n6h}m>xa{23|#dtDhoOd&gz0+h8LTIj@pa2DH&Q7hkW$AxRfV)Q3J@mhVBL= z3u0+o(wF*YFU2}CCY;<&5j2fnp&80uFVxYbuI4?IubOOXDt>ANPJ-xba zPXc?!F@VDXJIYgT!VoWLG3CQJ3BDXg5uk*7GH0TEpjI(6mIIW`IyIN4@*8OB{ey6q zlt>=Fx3F_DdwE<#4wt3qkaQpbgUx%KfoyQX0PPnS)+!}c&v0uaMM?+|SO>n+VPfR|tV{+4T3J^p#wJe6A9iwW=;Vy3Z?dI$hp$%QXdY*gi zLvdNLf=nD69waibi-LLS>d}$a4QS5%OgiP2hP3oJOfdH#YOo3H~jTw$LK3tE>^(+y=Pc|P&#FQC8ZT=WLmh@n(w}9u`$E$jNT*>jp~_r*_lb+Uwg^@B z^ucpSH%~tNs0YJ5x_18IwZzY|g*r)%dsBeU9eXVMG5C{q9|d@-1^Phty8L_ionym; zXTG2>9(w5f)7vL^!#ldq+w^z&Rb`Mq{OHB??@aD^JL{erUklAcA1GDl5yZo*1DWqZ z-&YYxh;y4kjoOEB38fUOc-s^R%VTG&ID69G-|eI*9Cexi9Hy%YGk33~0&fh6jdHl8 z;9yUmrv5gM?JoOTnEOoHfGOmgAsd<4qq06Z>SKD@>^dV_IBesAqt=TiqA} zAuMtm2=01jGnf40bsEvs?)8!yVCg`n6X|DnKb$O5o<=A`*j%QwB_HrkK`&c7ynv)x z0$vVKw>@u~prS+7p+T~$lv4Uv&))f~p9ddYp=}3WJ5dD#?~CO2qf^v}ufd zlr~gY&uuX+5J7Z;wzE|n`iJ{PnXzg~#4{eDke1g}43G}>bIiud z3`C8KF)M8?%4t|#g>rvNaVx3>levg^7-^D;qqj2}7R}VFO_HgOL!C2?XcnQw6-_G` zI4jm=utc-uP?EhvvaB;R0YPcoZwmeF-@*EJZW;ETLPVX4<>5tvFqBZzTFdvvi?aRf zI;!*$kRr-dM) z`4x)6CND8J^-NxH!Ey&T@BZNFhs%n_)}kB3h9}e)zhTVnnj-=!gQgEtnoV(Ytp6@zDU?Y zu^y5CiYggaSb6*13U1BgMbrndAg*saOAFCy!=~fxT}9O;<84I)r8rbk?kyC>SK68P9`5rj?TylsxC+ z2P9!zQo=XwVFW}mApz53>(;5aRME0M)8H!%frV??cK93J95191d}%>Hqo;^!A?6yo zI13wr!V4@#DfY^ei{^oT@H}1{XI1i=RfrkJ#WUdO{l5cQD>KbXhfzeZ3MZ)MilE(W zdR42kp>r$SFF^9`6d0F|Z@ajEH(@qcque6npIQRfI@86aolj7E6kFQ^VvYy$8~qpE zE4CPxNUVAdq!D`L#uz_&xrD5WLa3f_+X6~LWEj|`%Csbs$?-0gBU3KO9j@zmYq6y7 z^4qGY6+b|$SXqVNo&7LzEGv7!3W9Q}HpHgEE|`t%a)Xu^h{Ga8c#<3EE+b8!J<0m> zfnjVpz$y1zZBlJ~b*zFQJ!epP^~-g@LqP`*gmVSOdIp@bVeDt;e|DeD-JFShhL|(gX-lJeZw9?CtF>W5|1O--I{&O z3P*D5dK0?i$lUdqD&r=~MLK2ye@WpMUo)PT`#|zksW&L=mu;Y|wNfF0(MIp=6yC>? z0 zrn>Tld6~s4YnLn&F^IEPueoF8P;iC1BHpjXO*ZNhUYJgtfR+;HoE0hmtEHB64v_}G z$+Ct1Aqq+cpj|j`vnh(>aQ!di^@rkhCNy6BdVU0kNenrJdIxh7+V~s& z)A9gI^T~qbhSIdsp)cIwQ>t_^JqIpQ*6nYSFW(bpFC2V!6{lRd{15$WqD5j8U%`_o>*k_Xy&I*|A{-y4tLu;r-G&%_5x|J+0fAwE- zsB8XWe=K>J=29DWv0`WQrlU;{bEt(b>jZSNYhZYKC}L#fZPb+7$wnvs+1$Rq7L%hPpcWru9jwD6Qj#|{6hLenw>+upOtL_r7^W6`KpM2uc zbIx#K(*dvW7%+ZrE&o{TSddw92~&ODMh8;-OFFAGm$m?yPX3QyT(@I zCSlYo$*TG2;0$R=DyX%K9gQZm~#{9K6PC7Ia8C&=r_MA zHmW4|EY2iTestNKN_2toHd^K|@Fwz#%Y_x~9y-ogIgH|DKAfESwF=fxBek^h!QXVO zLPEG+(hqy;T?Kw5+GLTbKZ5}_1zGP1sREzR!RPwTiJ04x@fqwLplVF(CrbEBMKZj zRcNq+$mVH56u{qsMVLa%>S3q(IKaDka8wY3IWhMx`?Zp+KJVFEyY^$HLrX~1=;FKE zCLh}o-jvyYEaB|$ES!X>%Mc2kUJC4>!&@aF`!9Iv)~4O(pMD!?!lm&I`Agldl!UhW zR9PHKugj1P9lmk;*Zo%>lJbfQ9x_4(904ZXPcSfAwO3^dAwT3oOJFBHTiV*pOm zjzesSiB5z3Od{!RCR_+5DU0fUl7EJnEer1=e5Ehi+niKa;2Cn_3XG zn6Pg56Lqx@NtYKG?`rQ=7vT21Huzu@-%+*-J%ZFJ)MOTq)?!upMCAZg7^$LXq{A|V z|JSoi3yW}jOKb{Xl>4z5HA&?x#uUx}z`YbYVs{n7PoX27;2^^d2yWVk3HOy<=+L)E zvcA~?5glx7MS7OSpOg^Ari?KtdPI&esxoA8Rc9p06 zL6GX?-pP&YLp-S*ZQ|~7Y-;>h7al%pPPuY6*4pv*k>PA3?Ob)qJ8WbnM$Lq0*%i8* zc~Oj-3L@s`o(Og~Wxdl6wr-eu_9>lb*?l2wrQ2f`)XsEj6DM5pKi&x2R-aA3m~LR0KlR3$=EEqjt>XrZgnqvCdKd!H(iw=KO& zauzTnIK{kK&;SRqIe*yJdlBX7Sl&5hw==5%VGD zWO1zsHR|wogBrEJn@B&U4$yhAf3i}p)D|Bks^yWIMlfqEiXPB|U%q(qNjX%lAejoh z8hFDl>YV)2#S=$#`PHY|uch~y1wp(4tAGR<3=C@-10ytkt@|(rR#cZ^ZH^$MUqIY}Cc;q(LJ+%BO@{pe@PagqY3;ZBG5IpRkGL{ zYZM|;kZ6SeWMlG;oM(nDlzpa@)zXK47jM}{*8E{xwMVl+Rj`)jxqsMf>CAT@dfIh6 z%$r0;A}3V{Y`Wna?pYM*l{>7W;A(@igbLwnR-y*D?k7htdpn|uymXfzrFh>kZHqh& zuEXUD+cR$w!V1LH9o{J=aO)Th{?e|6==W&vSMJvk^7>ObY}$C z?VZ=|VOz@ZwOLoTyVviNzbzs=m2dgt0I#-4^KY9!dlnLoZVp*I?&*EXM`WQZqTMfK%6)A?7ENg)4$ zV_S3Aq4|N}+w%nlgHFDf@`A3um@v?_m;$WinZ&>%6dIcLa4qNe7S3aH00uskZ~|MX z$r$$ETia4|p|MjAqgG&e|_2um2obKds@A(&5p@*be0Lp5x;k#N;nJ{xNW`WjF{7bR#<6CBD4 zr#P9-o~Ju9z3;0JH>szxbfno+Zv=sZKZ+VFjTN)0_=cZOF%sPx6enPy76Rd<?e8fuk1>yl{TkgWNv(=+?=Hj&pC;#)bX6EdGV3) z3*$$r=!k3c{9Bu70c=bbv(dq@DSSoAP>;fBhL3QT6%G)6!HI|cJfT15xA2O!{Fpuv z%zf|9le>PNUO2G!((xA|KlI*AZazs(1RqXy<^M{M-T|jGgUzl5aqG%%MDiTcF^0jC zLEh60!#h@8Hbj7$^(hGHNOfh0BWT&F=T_prawe8=oux8TTg}0QUGA%b3wzT>FK0tM z`FL5CnkS`6A#(!N1oXG;c~S^)l~Y%dF02NLQ8@O1;|~cQtGL4HUA;pi1Jp2pwh^-8 z(KtJR~+^f z*kEj!b!sFvfR1o%AlDfzer);BmDLqMbrBZmmwFn}plADFPL+KChd3AjfoAg_C<}WS z(5{GJTyUhx|B`)TIQ?6l9DFBVDJ%!bf{9lVJ40c8(=^K+x+?stLzq8motFjOp`c%6 zbTh2%=4vQyGzQBW6Z;tr`&vmphRP>)xb;i?KOP0k?R}bJU@EOgfxeoet2HT4x-Xmr z`+eWe;J#@^E`0iv89@yKnV3WPJcF*3F~x%Utvx&bo3r8-w+fu=DYi+2uPQb%zDeN; zvD+wH$^JNiT2@-OLJp=6v}kHSr?)h_TcHh_9Y2WA=hD-(@MuRA^owWLKN`?A>wnE3 zJpGvdO<(9Bu@U0H0^ArBaqT)A7`c(`txmt=rl0tglmGog9Y|9K-1zh-o$x#bpW`E> z=tt&GGL>=j?x|-FQ)&A0lg|st>=dc6<6PcFRWMK>J9l5+dpLcdJR+gWk`-9r{!}`( ze%;iIPnav|nZrUxK9Id5egE)UMQ>hw;^*n*jYoKkZu?*zv6+45+j=h4e#&qtQyX@E zu>OT~>g_$mtUuVgH!?H6pBM_IpkPW&L_d@*Z#!>QptFf~D&JLK9UeSbdK$|_z$2@- z<%vvhEz<4iFk3?!*qEPRVd0ZWE(aA`hvYZB-Och&2qzg7Kxdtcn=|LZ3JcTMB<5~Qz#*UKb%p*(2jY7-dj4Qc2WzA~4aK;ehp zTT6iJ?M{w27l1MigyfdP?t~QmHa;MJBm$O^2vJw`jq*@fgpD_>MbkgZ>*+aMMZ8>m z{1nl#=ovqf9V;J6+SOXF=e^Aw?JN2&E24qkk3eq zgmOO-AK$?Hg^yHHZm6YxsnzLMHa6tY(^^^_N(0E6QoSBg1>xed0h<+3=>n*ey@krq zkIM>Y9td_37-dXDWeKs=s{-}Ba-Mq}L4UB~q9 znK*H3has^Bz(s(V#~BH#bI?~&!J9~n93mj+N9%-a{@Pp@-NC2GK4NuS3^CSx$MKe> zcNH^(r8`Qf{zY~*O9PIVx|X^lk)1E^h*b>0t5_&&4scPP80$mti2D>nRe(-;R>^gX z3K>vrB+JF-z%Kb3d7VSzO}qU@!NX`BaR3+RGt**eORDMzsIQnpWJOCkY0Fu=*51t_ z^bX*2{1rQbHgPKDq$dUs<3OD*5h%H#L;9^A$|d>iuUznxJPwt~ERk-6mTfh8{M+rxn!y>&LYR`&e2F za=*1FhkT1fF9Kxd7i(8STD@aFT65xPe&w-J2w+zH`HTT7PLb&2#PT>vi5Cw=KmjBnk*eg zSY}0r(=c!C8+_Ugd{=AhAz%iC?DF`wsb>yfej=DiE%s3i%eU|8c{A?mHKUf#NpTs- z3f5RkScf%obDV|NJ+3u@a&3uR#Y3)jk@bD#U61Oi?0@a~hfL4(e93nx(6&*!Zaf;% zYg@*Q5&}!g@6XOh>5I0E6)Fx3na8J>6SJ>CPMFf-(x@mKTojnpW*5&W55nPFln0?A zq{Q1aGT^3=$$M3EUirrc~>4Go8XQw zLM@{}J&ul*QLTjkfh%8F@C*F!UMY)5%*q*7>Qj0*R2H=%IV>}k-j1?vG()@N^LN+b z@!}KY-F6zHx|+Pc-JCK1+0nD(c>VCX9csz))k9YmO&IfEJ+gc17@(_S_S0!+wk~`8nYcV-#4O;s!HBWnNMhzE#HB>y57yvb6 zIaN}Z^vvXe2QU5Vim_aJ^i4OrrCeQgr@=z)vNoj5 z-=^X(OffUzfc)>wr^hrfXKD9JaxG!a?{jZNARI9>!_zB>#|+kRMx$hq7`{-E0Y1W- zcmF4-cPOn_G|2|$DC9W7sIag5rrk_@Nj>#@W8!0NfYQ8*BnWONaBCqlzmz&?;2@*uC*-5-qa_FM6-UJVHS8ntN9^qp&`UVOPN zFY<;vsr*5{B5{?W<+W9~$<<1pK$8`ds#3J13ayK*1MB3~b5((jA!2Lj8UJz_$M!vi z0s79FUs}#2UZj)nZDesNyB1c2lm&SFy~o~r$ewo#59bx>E&hgUGz_lld*gb|+=1#! z)*$$8a$I@{8d;X*>?>G@7WV!3-@+dMW_dlrq@41*s0FPj9oD6u$kt`JedhIf&4;f~ zS2Xxv>w4u1*LD2VAAYRWEuX2q?wcAZc990^ z%GP=WJL1nU{&OV`QN5h<56S>FFj8$u@>ds-WtmM%!lqK7ULm85ySq|SJP{@rwr=-d zB~>eKdBs0-bs}y zRa?!`sMT(cR^0M^eb#&L4}vlk>}!ay+>ipw$fR_UeZ)-*6rb%^y%5BWvn24BlF1Fv zmCjPS?W~Z78SeCU5Z3H?+?D*Qhr_7pKzH*Ye4t#H$Z5S=^lP&hkz=G=I1B3{J;@2r;`&5$aJTE%X!o`H2DzygE>L|xTADlO*cUT?|RX_q+}TlY<0~%ot)5b{4;Mkavp!& z`Au&#ZyXhz4i|Q=)8e-qcWw%dBk#P+T^HAG^vw}&eCdR4obzJ>{rEenWIhI5cLWy4 zlXVsj6brqK>nalD2xQfPaKMws4(W=0?J|S~unX^P7nGh-V#u|m2C{RHggldEKEw!4Nny(bfi7yX+GtE^Hnm9z@-ZbD<&crhEH=biED^ZUOt`mRoM&Zl(7b?Izt6 zbis&f`z;HKr9mfHBfua4aZS7`&WFfC!VW@{U7B^l_MK=0xfLX@J*$zYl3@Hw4v0NS zgfJtV7Uz!a0^W+k9P<_w4rf+KK`N)@*$(=l6LJ#Njax^D#MIWOD7bBoP|LK>csLJF zZ6)NJM{;jkWW{(%zUm#SL2{@NEgJ*{TERtanC;HG37^hDy|+3$U8RB+D1*bN$2D7v z(?wzPFtii9G`0*!Z2)bt2bi0%NsM1MDY#K8$s)9Gc{&agX9HNTaGTKd+6)#w6jslW z_a%x=h4)N73cx|zD#zd3LMQH2-Zy%A_o>(CYA<{j0vCv z%7RbLXVMK7GqE@MSJtRqg7M&w!5mXGG5J%UBW7h2W4|Z))cvEU$4_nShDfrOe3IsH z#;sOcpEZJppchI>Jaba71M`ydS*LfLdOi6U3ykj3dp-HYfvneCcj(W5xdf;(jvwe2 z<_2KWNb<=a<=y=ka|Pg`tA8@rKSBNU;*QB5dq0y;-k*2xU(5}C`*ZB*a?k$x0;tah zlAEP{hI1v%vwX`C2@M#QH_HITFj=i?QdC$pT%6%IcHR;1)7 zavNn*rk392%tkt*A=H{U(Kk|G(VpC9bi!HP(C~~>*yeM7hf}+_6A()}vIDkO1aytT z#*iVlcujmFUQX^rXU&Egj1rb@$ZMcm2&I6`R*fOBLm+!FL7PJZ;L4P#9GIn_l}TYV33d~7LloNM@d8K^YT0fwMR@RM#c7)DMzB* z3VDp4%3x^>A0`kAE#i+Thye(FmNhUDP>%Ym(+L6Hl;sEBkP}R+a3*VclqKeHSwB@& z(}6m9U|I2GYtM%ivTtWFo^wYY^Sy7OqRA~sFKv93jehR<@pJENEt=@syw_%rDgVy$ z>%(FH+!0k9Gy6Nd;%}JR7rEA52R_{Tx6#+VE^lXzGRU8k4EbeF1hnDZ{3@?hz zh+$u2$774sf$@rY3ZfNT=M-y6ID;mr%H*v3Sko?|IgqAg7NZ4 z_(qG$SrGvDukW@X3uiQC{d~f1O#U2`_WYqelMiljtIB`5xaaufgOB)J-I#pNAH*ob zJcZZqkaH8Cp zeD%Wr!`^#<$$6D|zE!X@uw)DvM_fq8l8w{~&a!NyXbF_8mTbe=i0ba@R!QAmO;xv~ z77QcHk}cx|#sTMmb2iR-2HDJBnBB2=nVd&%ImyH9usgFm?9ATZ|2^k?vAVk?n|XGh zdj+zp>O1K>=RI!@*F5va#Xk(D8*+~0@_$iuB=mXfdk^F3ZmK@x`)3so{0h1{g*euw ztc!W^GV?CCa)6C!4hkC0DxUE2)n23ah`UuN39>phU6(X_{qmEH_a%9@4k|ScpZd#q zy4GhEXRPpq&b=j(N$AqV(%CKHieZBxy25gQS=A0yr`q}*s?V|eiDyQ>eKXh+R@dkr zXOX3KNBbgAM$^RlCdi$eBOdOmhemF_Q6PH4R|xc37sX_eo-2`x%Uc7J)jSl%>~fvX zlVF=5Z4oVVY--X?Hd){U!md90*maw8xYq!?tDH6qQYXp?t9eP?_5b6vGqLM0U5lP|UFkerV05CsaTp;B1-S^#+ z;7hY%^T5)(BaMqZ6Fy|!rs=2#Xf10{E1934rNDXHLaj*mJO#KI+N zIFn805|LHS^l+ozBd42tNB2ku^>8hneorTP;$GgP%|RoGhvdFpq^?+x*j7BXg?rSM z)HQ}H5iZk7<6kC%*@AL1^!69(#IY}wor+UriXlx^-C4Dlm+HajQaOdF#de_&`nF}z z2={7v3-@a4eS5CGTlXpz8?|TKZRC~9UAGex*KK2@aebHW90%jHw%VO{{mzNiODJ2( zsNLF~L**Xpic1}eAX$%92(}I_`PkR6;`GX+-a@Bqen+G@8LZmop zxaB#pPsA+JAoF1K8-UoMCP)!zbgbzRUXU}NBlM{N^4iGBIAc-tHEmB zx#5g+!RmxT$}wRyc5!6K&2LR`}X2fO+Cn1k*YmoNx`SX3(lY+Uy*%({2 zHdtd+=elGGV}mywwr*~StPR0V8LSO86Oog~DoG8#*4voY2H!KXHk>|+?Xh)>L!^)l z7RUvoS|GD#hp{b@&hQZF5n8c5#En}wKbS^Bbr@PSKe+b9*Ecgi%zo)9Tr9fqps@6B z850C18<{KD;mKRxSXC?H-4i8W<~KJ(TCOHyg1r4rD)lxTjF@Qq@N(#aKc6=`=N9&_S%3^3g9* zA%Y@+)n3V-k&4`5k2q;@Y|sP)OGhv3MjIuAg2w{n8jGg5TBq#U@eujfal}aKc?gSn z{g%6+#F3?QTOrtt1kh9Eq%u!-{_cS#RIH`c}mO?zkIRSCNAAo|9ye z&FUN5fbeSD^7u*+a6z_d{LvwrbTAx)HC7dbme93CE)R)## zw3<7Hwh!3A=Mdl{j=-m~mpv-78N@2UKma&}^?2CIYc<%PYs*dXmnQ1#DF3C{0}{wN zMIZ$^fVddWtwxL>&uI{WJcDxOwWlzZIr^jw)s;=1y+e)a27vsb44p_=mgTqyS@2b% zl*~gJ_Y%57%kLrM)eBO&^NbqxsoeK2&@royw_Jd!3?w-R7-m++4bd-L08nEW`F_MMWSN+3Uu9e&rTk7t~>{+kk?1no)k*`hto z`0QUb?b%Xn&XPALKvvfH9a5l(gAY__9V_TIt4!=Tgg0C$Yv;)jHOhLD-BveSZd0tlS%u_i7Hi*6h2K#_&W@3E!jsAeZF&(Q6r<1Etmnm;Het5@ z(Y-&qnHB%&<{v$SN4sP9lLh31KGW;`r)72iZsnowdtTW;e^hSTZ>cmL!C37p!RHl_ z`2Z`M7p&ysV^d|+v8!FZV^376wm2tNQ^eNkTy(5FANz)h9g#OBCQ^T3a@fH38zF{> z`Ao#hhFauV!}Y@PbdhfJvX<&h-SM{KG}kV{>gBA|csMINCM$Vn$Ah%9sNX==%-MgqtqxDalaF24oCSmB+=dS09Hc?`K*4jp? z#NN8|;UU~slBa*0Q*tqu+mGeU?+3}KEvv#KHHMiYtq z-`;&4C64hIPi1^%{#{!ycRL&sMWjPSmUxH^NX*OpHztr*>8(p1*mKpRd%pLOPn$Zm z+g$ogFOUrmQAv&%uNim5ae&Wsc<@TY>&R}`GbA6TA>LIB;;EUS_2$~Bg zAdaW$5e#+M*6h50s!m{CfO(Uj@kYy4>OkVX!bQ9J3_%Ky^KvxKf)b}yPE*3~6L2>^ z`6WU#9&Zn+KpbwOq{;`r=GwtIj=~lCn1Vy|&O4nnEBAmth4fJvhIIGcXw3LzC1Qfa z40s(C`+$@7fbh$brLDJGiSC@aW46T>1I~0K=nENnN2Vr5vy^)oiLOk0ymIE-jR@?tAe%a#wEt(akKvXMv>TE9CJ8$zj$6dv;;^&tovO#v1`#P%V)5SFSYO-2b zR08nm;5+S{ush0)&LO1<2~}&|)|7prTfaY(Lnk?0_tm;ALy5dLG;C`4nXbIBe%o~} zu96qJ?cmG3Bez}1cKHI1p$?7q=~Q<$uKv!RZNB>A0H_W-yf&#OqV&S>}do}i5~ zSv~da>r{xo$^vw8i0<8tN=b8)!k_r(Xxh#43KfqO0p@y=uPSMmi$PfG*2j0(Tg+@I zt(O8+-vG$-W0U3v2PIf8|57so%n48_gv44btvI0tn9mi*+K&&g1sa4dlR_7Psa$Z& z_<)7+Ne%0vmAs(t?Hz>fI`5WUTVL39-UXIn`vvg$?r%RwlE5aAI?MW@fEA>^b@vr- zpSOitYjCD-Z+pm4_2XCUd7MPp%?hGSO91J{;aXSy#cMh+kze4~=N$fHluqD$ljNm- z8&t8(Z5vuywk>SHyisA)JRx7XK()mT(SmSg^>#)y^TvO|0WDNhXDH(byIA*dLe0dX zK0NAGH5i{`w2#kRS6!2J&`eld%ko^iIs6(dn86JG=$``iOA$&}ijez&A@|#+z|#`+9{LdTwMzBCo3G{b$l)=9jZl!pqe&HnHU3Z zMHd>Dg~(Tbz^6MTtD?9hrlj6pE`YN5wiUmWMAX5#-fwYRX-=ckv-4@?z|e`xat8`$ zX)cMc6Z!Cq7{LCg3>l+D)A-$=LpT9BBc^Vkn!Sz1jj-1oj3htDfhPt^M$^Q@R4JTZ zMvH7^00^>Slpag45oyiD=PRd1)jEOdM@{GOmIKyy6Okg|* zD&!VeS8CwnGKE+QzqypY3i+2BaZ31M0x9$KCO&^JVi@BLW!1@Z+8!(cPnk(@9S5Hc z5evpf4>!H$ae86{T(durA)I?|8N%V%E+#7H0w@$#4`Hhel`hX5S76Y4^MUM%3ec#}gH&@y|#g*s@VAgn7@udg(j0gogy zCS+*!rTXk=UEu><#zj0^oFL;m*b}_1()IbmU`I&hG9RJ1X=T3BWZNGmJ(&CGB?_)u z$aU&h^qD1bNxozuzA(vw7&I+yp~6m`co7~QmCRv?yy{jSmKg*gt13+|a0g?|s<@5q;L1G_&(I(Qu`N1Er>!$1ekS4lm{$%ak>aa~oY z@N8iVsC}4?17KYoE@}vl+Z%`*KKm{xW?U@aEY+@f7;{=cq{JLyiaQxDZTQj6dzw6zP~$?M1zc&4lHJ6b|qlO6Z## zczGB&Pf8$2DN53)DwZT#Y17ocm_^4WDVa4$&`3J~gs81n$Vyj>WF<4HS>&w>t|*JM z$d5r?`V!P75%$?cU)hY%cmhNE8?3%Q1&%(Bnk4vZ&G^WuWVV=DJcm~ z{MeU2_Mvx9kDdolQIsOesw3wvvvF*5cnl<#*>tcJlx!e4UC-Vqv=}{Z1~{Vn z41Tvh8v#JFRO~1);)=k=FmiB;BxNcrv^%Ee~&o z%cDXG;cgsnre@rzZa3-qiCyR2x$C?~S^$^NJ8(%9MZisIG+3FEfR&gZPgVp?f=g4u zIYM~KaiiBr)fPHt9GHO#j%pl$iX4MoJk+2jQRX1U7Z=ZpA*Z)SvLY`Ddkh6z-`j zzx?7Q**7Qsh_s_Cs|UET_MA|MGTv5Vtw*P4Rx4@GS>-5NaMLn(QnC_U6Yt8n5%nuAAP{62n}-%5#@jJvw345~q_WbL zq0B{7p!qBpVERodn;a$-ArNR{tyYRj%Pv?Zi>JI_n9ya?G?&s1dd$TMoMqzW(xXQ( zx+$#KRbse%*LnA<<{ee*M%NEyhT652oPxLQqO#q2lsZnCR|1dMOgYdOJ5IiR*`se= zLb5G<+DE}HnbRokA%*w#+eqkZAI!K-yqpB#K)-g8tTJX&4AuO=AEZsIWH5W zjq2eh3_VVD@Ni`6iMczM9OyBVOx5>!B@cS#!dEVSYKZcPydAv0+rhq^z+zV3Tt=+n9n-7)&|u1AUB8 zW<=`=ut}B%6Jk9qYUOF7B}~{s5`6R;YH)Q;{%_ z(-|sby6-X1F#xzuH(@u^3Y1axiLvR(YDwbroK*OQnh$b-9_98^Aq?G0s3n%VyaVZa zV#-VtD*(|fOEem-vRJa@itQE3Jf)xauRedAixq09WVCOg;F9M)-2OQus6mg&#jwin zrSC&=|JD5zQr_5aGy!@v%dMlh|L*JWdF`g#N51>qtIymvD}v(U{>U@CzkjJ9)jjg< z4_rq0#m-y$NK6T--g*D}^5!tqfvfvFDzaND1N7WVcV%twEC-v#gXY!xj0i%0A%Zp1 zMb$z#J<~%KI^tRXP(>Lv?8V}K)~f+x9E|!M;#O#p!PI4?JSzXaF!VfR&y^R4L~9oQ zbkFv;Zhk5}J(LrNheTqg{BxyKB{XNnCtEx?yuc=J!s#cR7C{37dE4Qd>g@TqVhH7jBte)i|#3XL~C4+?g_ar5@jK01$E z1i{buDgd%26-ZwAo=SAF%pprT8$SUWX+P13}>G zU0b~r5Qfd3hb#XYo*wAWLxiYv9iv9Kta$KpPnz6X?JS~c+wy1OoE0HF9FD8)`TQ$e zq8Cq`h8zaK`0K+OJ~L0Hu=RQadDVbPd@k#C^ueIy%+WmtqyKlOE_YJqde!)G4 zqq!p9jSWm58r^d%jqplu+USTCQP~r0Y9t8)tpxuwWl=171MB1kBzL2xxK;FCZT^g8 zOGVQxWrAgGgOPZ)8ztcp04&$P!|oU+V)H#LM@6({QMGqCB2^njZhxd)ytzc?8u6C{ zLu9|Tb`$2gbz$Oza}CaZQd*h?)!InQj56Q>|MFQ|%{B`tHdFF3wDk7Ul;0j4S!8(0 z*T$L(n^fM|wb^R!g4G6Jyk?-k_R@8o#EW=my|PN)`pudI*B={B*5c0+y!4RGtQ@W1 zgBdU1WabRr-Yts6P?^<-xwR^%;tE+kX(5F#N7m5KpBB9B`mCIa-aKIq72`lLyM9-` zt~755i*x@V+5M>PZdG!uC|g^Fs=Bsw6Gxu=9^pt1o?d_9JFnkzv8zTFnusTu_}X<3 zM2$vndZKBgsdLx~k~8+dc%2gWHwIMT9tRVITQiGfCO&dP^uq)LB7?5YL-ox>vH@ic z%aycNuBQ{)&3Z1kAZ)e>5dws$w|kGnQ{Sa!wh}vLsYf6o-mQkk=hNwKL@#crnyGMN zy1Q&758T9SqOXrL?H~X4_qXl2@xJsJvCZ$I+upwFLfoSA;4{UnCg~}ZlB-nfsC0tw zOZeb*uC4X4Mr?D3&(WQ+WKURwMtk9-UcLp?{}e^f9#RS-^K5cP6|&-{rcx<_q0=gZ z;Y3R>-9%Was@97fQl_ocHt)VUOCj@l4YCa)AOOk-c&M)YE`97%%=EV{kwfG6g5b zlfR~|JU-YMcr5eOW7qGw=;qQ}KiszGX3jgRm2hb06zv4v_1Cc-N*ecQoBg6a^3$4+ zfBDAT&3-J=gG-NWgToo8DiOuyD7>t#xL-V~ceG`$SSpBJ93&wt4iXh%#NYb+)4q$Hys+m)p%_7S3<@r)c~a;&|7 z7q>p{M3S8k88&I-GywM)atC;$$r2+#WH=uJ zG3r2HUKWzj2kjm}n_DVg4EQ(J`aohq2RM`l>l}eXpSe1usD~-Z4e3gnww03D%SpVH z1b3={jC)S;^c5cPbAu_HM92|A9SC_^xK!3&%jY_XUPII38-y`-g*%`8`WUi5WTq~K zi<7=#@u+4v>OqxIWHE=y}EHt;R$UWkiQK;e=8= zOV5@+ptceXgOs1dBqriVbK$4Bk@7l&_04`Hy!6lJbWoz@$3#eq!DuX9Ks-FWBB)d@ zrW`(lwiO3VhEW9a3knP|=h0`K6cfb~=ch)!c6n0chWnlovr!dw(GqY$J1ugfcE2W1 zhW!JaDmYPHeP*y|(K+KD9^TH!p`WLhk*I@Ze_TkwpDXU4zUWB0{)F?Ev2u<=Z|s(4ZLkZ`O`hnO07!FXWom7GIdFrW7-Oh~FHM$LtD zSa9Too`Qg8^2QMX5PXGCr#g(r(~Y;iIV^xgdcCF^beq={EaHKvuWapN2HIW=C4v;% zaa@>_Mn4__h-3fSfF`fVuuX`VRTUl$)mt4bL>&%ke*RhooeuI`sU5Ss)$VvTVCsNv zKouZ8AbYUUlTRj3Gg^R)CrccX*$m|0Eddjz#Ybv!9}ijW zOqFiA29kcgDt|S_MOu+Q6jq1Z=vzs5Bu}A~L%uLmYAjU3ekf(u&AQveRJhSH7Ct50 z4p~-;b2HBN^0<2I;w{Q@Vx8l|55ZD-t-$A36BR{>C#*np>R6>g+)#WlaA`o$6)F~Bz(VgjtCdD^F(QG`eQ4K)j=j6u1JpP);2C9Rc2wzA>)8p4*LS?AVOdqB@ZQKDJi~JUQZO;&RaHAsl7*lT3hjJc~43w9>nhY zmai^VK;3v8yyh&1ihK`^(wvU9L{{}wdx&JoT#lt^#L}7lb1U~CA6x?9HWhyYw8-ir zhCG~0xgqJqGzZ~4mG6H@MJ#G%BUjLAR04;7RD%5x3JByNgFbqoAUTR_b~^HTk{)N-TLYaTjj5B z1hsj-{B(+?^fgypWwDZ(H!%Iw!-^uiOK)u@)caAT+K&VXYj^ARJ?CGtXWPSsj45Kw zWl?IX7F_)m13yq0+obr(%(QK#GlwmDOzxEM!pIWC&YvHtz?BrlYsJ9NDiAu2baCMV z*{@k;^jZ`wq+DS-_%h%twNW}EQXEO2sal@rg2QBsH(8{3)`|?E8vjr9vJ(*$ zAJkD+e0m=Ui~lPt5qeI`N(7WaVW{J=o>a4yIG2%OwxX5jBCjH`4h~_8K&rAXDo{A+ zGBLaqaI)usZ>h1iM$|MWdwc;bB`qNpnE4gw>k1qwFp12nHIl1B*?dp)wNZ1aU*%Y( zf9e&59(9$5MZEmpwiOpB(cN_~?7rkyFy9-fSDyPGRCZ*`v%7a(<+I#Aa2^5IuWfsL z;JOnr zPZs86Vj>6#MM=G%9CzgWURkm5YrKg-Fa4?s$RTnLWu1+M)7J<`B71I~y4D~dMKdvF zq^TrP;YuICy;)u=Fd|k9kWp0C6yN;2zdpnXvsT{NCDSF$iIxcd^@?JeaE(gpSH9RT zDq#`csU@*_5y0S!gHYHvq>%ew+_m_kpe9_NXFIav0_@HsTW&*BLa-UJ z;CR68I(LUEgAnn$<=l{2*IbjfoSV_3fc{3#zhva;J9eG-J#&d7@@dR!sjK(gd@j{b z#p=BI(B&S>Q9$`RewMu_=bG3vk0RDN2NU45UO#ogv$|Jr0AvxJO3HxFylMY6yq% z*_Hm58H$FT!KD%$Ac7m#bnLFJkC{Km)m)D-nXVfb$LFkW=JfZuJ?DM@?JF)09duQE z;=_j@N)!)1e6>Ne{O#7GtLajn+6d^QH5_h70(>@m>#}3*h$4RZ2Xgm-0DI`dmJCpl zp7xdVMOxEP`BXIe5j(E}U9JyH4RBNkL=upLs3dym=tv|NVh41f00=;iMxMR`9nZ-2 z#~qXiD5k}|JZTjtZXiORSD(CM_XQWee)kXKTyUn)}?iPQ`+|zw=1{E+~v&=5iAsu2oBL-CP=PIZ*!7zj|Ls$bk$Zo zPXr-_+5O&m=~~On7pPC%LQs8{I=ph(6u99)SQuT=k;X$ON_*R_e&sT`sZz5;(>NjB z?Jz#4y1p?D=zO8N}X7Ots!(~6#zXotKGdqPqt%5UI0 zL7aM}UaykAxJUb_47yh2;f*UEi1Pou`t+qE58sG*{l<0Yk9=o`d?$ir50^P|{xzOV zoMZ~B*B-rf6Vxa6^@|L&`v9eMb^On;XZ2n69fFZ_}@@|&J9l_E$1lI!uHTaQyt z{0>M4${JK{kVHuL(or1nL%;9?-uJybpXF-w`h%1c0(;QKvjvv$75DsZ_|fuEw}c2P zSHttWT+4|BLlLU+9#T7@!=*3P=?P0Keiu&ki`@UtUJ7-2?NXFyNOnCyo4u}dI!`#Q z9tNn4msuOm!=8baj@IryMj#st$aUxFLv4N5fD7*gvc>R9T&FIW4GRH8 zTCw0yp0WZ42RpSD7uxZx2ROhemaJ$%NdV-sJvw-tA+B1JR5T22@#=T0+qLz6bFzOh zaVn@!age-k*H%!Dcg~Y4e3T?dj@Z8XWR&U%dh7Ea6~F2xd3)<^a@~l}y|m|<``)_w z!ads&(!XPwhn3MdmE9Ui~!=> z#h0%|zjx^?7kD^E&{kGq#O7R9BtexL(D+S? zSuVA&uMP}=wWLo!HDX(Yb1vRK@|}C}GjJ~QU@Q2SSD!my!ILh}RzO~_Kk>-xFI=Dj zPOw*tdb$WpdRn(T@;zU>66yO=^<3iJ>KVY&dTd6%?q{_iEiUG|3dRajV!^!kS8U;+ zD>kIIj;o1Y2o#2(Ll~0WrWoxUhxnm_CAl290mQjjFi}5oM&L1$&DRjEohw2TTd{#5 z?As8Qxoc5(dDkr<6 z3OOC^xX`0mFae(WQz&(eUL~2u0EdE}elSQ~-H)^R!9K1d^SGIv&pz>Taf$;DnD{9q zWA)qQG0wRc=Uw>0FC4cJhw95Da;DQ5_9m5CO&Jj&T^8;G@*>;^jN_PmZv$YLQjq0v zU6c|s>Gd!C^Y&%;y?y0faWFP{KHj<$pYU5A#h*}eu&c$E&@R6D@S|^S+ws3Ds%oMG6qGQo2t%+`cXqd)7!&X`ISLEf4{T!kIZV10)Ne<-gyYzN;i zMNV*BjH95)mD#6o8xgYD9ekXMN4ysKE;Z@k29wNlWDsGpt#tiiMEbqw(k+Opv8$2d z2~7_l);KkmU+@x(#1IyrY3LaNT}LED+$)V;{k)^-xyG@3P!!!j?mbVz-4$YQvr|O9 zV1(P)2lLIjC)Z2@gKAQ``=`Tg$ldovG)zZEJ;6_auA3S2jr5UU)tp8PLM?9 z2z!y-x+3=&Hyf(KX1*>!9!j3Hp!`^NEbqaO-7NbuPIRvN_m2n}LYt3dz%z0MUi<@t zlj6)%m|{>ALL|Fn9DzSvR_k9ZzbSlPs~aklxRzYIGUI5j zazRM!?$nGpA*(~2C=Zg#^NF^9*VfCdPAGQm%)-XY5&G4{a$S(jo*R(lGZjP=R0|Wh zT&J`1m|ZLw!V@~*Gl~6iJ`Vw~{&vd^PQW)-8M?l?3|p-NI(?X=236{5ozEufPyshT z{Zfq3R$MVBiXu(C0_Mv8+R#+Mq1J)22b;Hpg$WirXReB>M_QWz?~RGn;Y# z|4o_1pb3zbUCR`2GL$JEtMtkxQt2vo!(uz&u3@s}C?|EY@z`j3p@ODPPFuqm%%vjD z-r~06+OSWPxPcZjMD~!#l+}mmmlXQt+TT{1I|P@Fq8D{u9>5aANZIG>d-UT`z`bk0 zPk4DEN|Hqgac^xS-TKM|;GhCS8A6pzfEiM2@rt5%mNToC*U(z%rBw2qfESOm3y9)hJ2*Hn zYuYrXV2y3HdQV%acy7`AmiZOABwz6TSj0R69CK05wkGw4={a1`cYdalWYr6|y#9is zBVK>-{#T#6@U;tf?7rSoMqTFDuzTxWJPk@$msPCQWjwa+)n^`ex)Kb0ZW*uNbK~x9 zm$`Xfzw3Sy5K13=$JQo0!TRvd*7`v2rQ+R1u=<9IS^HvVq^kFe#>(d`0K@{o6$;%O ziZ8ok;UHw@lVmc^fhxN~MH!nLJMVVTvmQeXhl3qrpw=ZD8F~+pFv9OfjH3B{GM(pCRTjSLEkkP8Q8!~bt?XwJo;cNp)l_rE=nt|X1?2LV8nGC>2BCLuC zl3fgJROnk3UpgTNY7@{B(HNnf>hM9)DnZAC!ysJBm-+G7pr{!uB|_ipu6?K(ysHqW2tkVbg6R~eXQIT zKM5}u(gqUSes#*<(Hn)wd@p4g-@fCmC$D(xj{8b)fBPx9&&XO9@eHK9$h{PTwbI^F zNcpXMwj)GKQhxJ?=e_yJvlynYK^8~t_kAm5_QOZtd}JG6LQI#2w)AKz6LhI~Tk$Ia z|H+^%mmGg=W_xx?%m?ekP$Tm43GZzj8uLyffMc3Un!dnCj3;rMe>L6P&-P?R=2NMFmGz)Q3P*f#&O~pGr2b& zvWU1RrF(>*dT7ke4%yls_vTM{E-de`WzW&|Nm=9h1q-@zdPGdwE0>~@@IFTh)RxDQ zJSXAKSW1b{k4nQSMe3|3FIg%+UtH!83|>^j(mQM>-$CU0Ob!qB4_$w5e=x5Pcgrjz z4JbbOK8bO{LrM0Wa({2RuTk!WWnLf*Ss_HxyxMBLqKSe@S?S3H8R%NAI4$KA ztdv7|!R>t1JR(AmXfaTg;sH+TLYd@~2?VTOY-*PfXY0bBv`}$oE=NVYC)4g$l`J*&s?1X+LSc^UKuYU_tlc_KxR4Qy?$ zLNtRUWEzy+R8y5qsZ`qXf?26gNHFNJT{atkEk*%R@yNl)v7bpu z@~zeKeoJDr9p1$i`zSK$4%Z`a%M?S5#3$MEewqd_@X)P=0Y}|e%uu3Zm&5( z9E~v6C=$zOgD@bX@~O(|`mA|RYj+N~P5U!_{Y4eP!;Xz=Q@PiyU|g-6~Q(b`A0 zsmznj7CqGR@4d)p1ZlqGO_z=SA*?(a>l~}}^?Su+SVrVI>`(or2j%zS%vy%@4G7|g zE?t4km|Y_J=$0;3+FtKGMmUeQ(!8~u_!m#@t(JzA84`>dKb-=!4K-f69Ti2$CzS~E zOF&WFMa6THx{5RywZQ``aJ1dP-P|ty1y09#*|r{a|h3wJ(CF)ODNQGu)Wfr?dvz;G+ATxL8nPBsO%#cS)CmX_zI**Ts=j>;j3 z)?XCcv+*c0*oJE0+}WM?>%!x{=x;i*wHX;<>=efmiPcLW#wMbR62#CkgX!*)7L znGcomChhW2=Mbd2vlr(aExmtIuo?s%2Sx*@^nm2r5u(dTjA}*{`MlRFJs3hTi1SME zfV|NrOJ{?+rTIH;AbKorFV+a%9J*RfGTQq+S4>3f_}KqORVU_6S$aD&x?_`<2?qHz zB;Mq6xtEtN+xrS_OK*aSHiM^ENSjjbwY;67wIORpf)?hmS2egJIxH}Si1FZdxv1IXj|S~oMNaRT185SM7 zZ;D#R(0!&fTk~UUl`zR<<5Cj^kA#%Dr}&4(H=lgy?e9Ms2PL`Jj{_3PJZKIFazOBH zQKN{(ienh{7T%@GDN5wJ;C&R5yp-(>4b8xF^Oij~-Yz}N&9}XE_oJhLXR{-eX2#EW z&LkTuq*6-QivF>`6f(BX5cI z=H6wtpYF;!5CRLqg#fM1<}%6{Wle3?1l6!h*~DG;N&B>jrdVdx#+{F>YdVS;O;hv? zHwSIAW&KHUx%ea^60&I2(mjSut6jlUqtd@tDW#ob8JAu=E9Z^`lC>}v7ws@QMs=4J z6?l5ADJA(RG9;jYSHAAZF? z9=PdYH(PH~vxV`am#p;)$&+dUIr-j* zq9ZO19T`5`7X#xA<9XXo8SV9WKwaXns~#NEc98u`)VWhkfz1GSbBVvJ!u z6qUD)v~Z;3NSG#+GN9G)xf6Z0dZ&TF z!mt@<(wCm~Nh-$-m^&qTW^Ti!?$?If{c!QX+9}S<2g)VG&0aK%oeJ;?w@q!)@% zyx?F$)X?E^fW%Xp7*-BH!&KWzTrHg^@z8iw3<;n2<9{?uzH+oQ{dFESJJp<1Avk?@ ztQRQGS|*iZPSDN)l)vv}V*ouH^(Cfo$*3sU(Px~v@yn80Iw9fWt?Jj>1m*gE(mlsQ zBw;6GgPPwrq&2tmp*~|)04pBv#?ONu_TZ={Rufr<^2ltlx78{ogVW`v5ozRT3Z%^$ z4GjHKP34Oopc1{NGHsm_l08O@sIvYhOfaLNOgmx)VJf>%D$9ZAt|EAiaUfY_jm0ZY z6iA7cO0ze(9m;bnR8@|kRa!RSH<^`103H>bp5Qcy?j(F;?>`k)o1PMUpdB=K3K5RYq3xXu*DnDTv^N$P4v%q`mCEL39sfwY95=c3=Pp@HOavyK}x0;kXNRu8n$e; zAtl7S?FvG?4=WJPSjkXPAi(&=b_Q6{spK9oY3}`b2K9!P&B^IkXJ8B#sQ|W^) zePL?LtZk5-ks;j1{4P=#0Yal@{cxuXme&AT*~INva7^@Rh`#k} zg~EUO$bwv}-Gq?3Sfhfgc48+&uTaG*ACqI!TLGBxB2p4{@oATZk>M1Whb6` z_&rm&p$Yv|rr7~uJ^4LChN$Wl5q5(8+M z!NNxVt7(p++X>@$T0m2+Y)xS)-TDe0{I||pTIzfnO)I?0pmkbRSZl9!E}g$P{>;rg z>)jU=zRsG;?8Pc}=vIcCIsR}D7r_?EHs{?svk8rV2W z``zH}v~i2=gll}(26ovA=oQL9OQq05^WtQ}Bjiv=mJ}x{_v7@7K~e@ZB2CFKghmRY zK-zWJvXA0e5~({XgBvTzh`ObccnxjONVkDku+h!3Cdu4k^$Nwh(Z#tO?1!bh^r%=f zq9&nhHnxjIVTEDdR0GyoA>T@GlvwhR^k|sI8;PO(%Z8!ir@3*KxpCrc$!FFpPJshzbLL+i#7rEy9FlJDM5m12Gry%O48^lSNG(`kyH*?{~zt$QQ(|Mk;OUBZ7C&p-L-nKNfhKNjmMXBd`c z-jpm$#N*FC@_zvaKA3x^KOO_W|BwE7%g3L6_`LmAFTQZ&KmFwL!6W|Y+JP&7|HenJ z{? z|9zp!%geur38~k<-ZI9yfZk}zY-97yF zr}V4v+XS9{k>9T1-?^c@6Udm=RIKEz%)ycBDK>Gc*!R?8oZGP0xF~D)H4Uk>xrvbX@ zt#yqTypT?F^PN`No{DO9+n)BZ%T#INvSF4;`? z3c5`F5Yl$;Kp+1r^BTjWfWxtLn=)3n>=J*E4o8gDAzso4zzegIhn>Fb;&60P8>1|3 z3f+%srF(XXx6`q$m5%XJe^*P@B^7hh$+CJ3RyP~mCWmT%@PDVJw$eSj)UID=sjYO3 zm-;+irk`m?yb_jgIry?+`zRK;GRj{51)Yu>y;F8s6T#GnkKR09OgjyWJ76L3YYK5t zPOafB6uAGXt|sKZGzqb05*9ucZ#Bk@Vxt6?;MuaVraTBRD&;QaNBsynRFT(V|+a+I>Fm#rqSF+d;o{RetC*?Lw`D=J}Kak(AGBvdD zvD;Xyub&#UGSONFQ!hDI)6(K@=3?^baQIBHBKZkeg&fuT5Gt!m85zyjm{O`*h6C3V zVOx$a=n|v-5kH<^XSBTaf(C!g4i=J|!UEJ^yU{~tH+S5|j#pVxcxfUF@CIjn*u_aH zRU^AvSEgq;6SxK9$7-upaR|58ja8%! z&cp2inKOqyH%kR1e25%yxp&AO`d|%L!2scH#i=VS%cg+PIMQ*Nmloss-Zs#fL8-WU zb%0xuUP)W=x3pA5|Ne%;5Z$2BWeY&7=_bj^mOTnS1h5KumQ1%61v zSmbKM_q7}Fa|^J|NMIp1Tv#5!*b5FR&BN|QZ}OwsFxt}4L~qbm+^Dzq2W(K$`~Xv+ zG|KUL?}ir7u9QE!)FetRK~BGV`&ulI4z|M4S-a@q21k8^%)O3oj)ZWB>yyy8cgsI3 zbb5AyYT~PK1~w3PkukJIN*DowTenI1f>nA3(M?YZ@Ddu0Vd6xNX9$ z{|m(1jYvfLr1z5clWee(79Oqz1b4=?-n4`vy&r84a4`ZjsNh?kACXDKm6o-Ui`~X{>wGL_Nj+nI_|*VJoeaw-uKskbMTpW zz4y>#e`odKy=VXL!@v2l-#X%(N6nf%^wTd-{{F>(JLUdo|I5_R-Fwiq;~)Ll^pD)| ze!PU;6XeHy*Y28lk>j6{@4ZY`LoLxUS3_c=*HhabJ40>zq0t>?fIQ0Kfe3N%ifs%T>Dj54z2oM zjek6=)b|HpUV6gtIgh^CRbKIXf8Q~@^XEO|=OO;S!sm-VP$>NVL50GbjDI(L?H{A!SqLVKhMr* z%=`KK8e@KpHtmdY3*#R1p+ez2{vJcyE&MjXc$d@n4c4_AaJ`zbp8as4a2wxOG3R$b zUMQ?)-KQa>ZQBnu34cF8pZjV1ADQb_=9tQRETPZoeEw_J_F?*bn6=J4q)>R2G5&-; zZM^4yX6!rZdo9mC$Qq?)`8dxvv%Uq4wVZ!PnCBMu^DxGHoOw^8%~_0h8Q&lNXrVBP z_o=eBKc~$#jPdLA`vqhCFWT(K+<(ZseVX_DP4=yq_J6^=uke2FXRJQv{a1eb5o1VI zb1Llz_&dnD4x@b=&!_SHP4;GxbI_khN~RO(LXwx7L^@t6s^{4!Y2&4UkmMf6&PH@IMJyJv5Z8aykO4)G1Z|+bFE$ztVnJt&OzNUH0?@I!{lU4dz(`H$-eI zSizv3s6Cttxk{&F=Gj=Cj435V-4->#m`;t;Pje?0Yb~9&qOQVSuA|3E#mjJ@mFC`@C=Kcf*lSZWWN^Ete_US4V)r*QbL4rS~n zlBXB)x7Y96ZP_kMcvxeHoOVD>>xI?4MVCL0M&WnqG9fvO&ZLiq8&ZprGVi$Yn@J^A zqkD2u|4P>*#_Afce1{pJ}?vi#%YMNPeJ7w>?tzbY0vYQHDV~Df% zeh8ggTJ5Lp>*lY97Ptw_Z({I8`!e`kyBj*)02GbFqYSoWUj}o4D!|u$jR*W420VLT z29(jK*|D_SHR%NCgbq3jFEhaz`!c}_CXj`%&K|DiUl|OOnwdu7ErvF*OkPag@kIbm z!_<$lzVr+~lv5E;1P3nEs6mK(DCtUCK~X=G$V`wZ`0C1wogT zN&*CdNIQEa1ST`)opd;1J|Cqk)fPF-mS2qG9^!LEEV}7{+kAiB6(Ijlyp;)68T?r)gYD(|(%jwY5X)EUfP!pi%fM zx^-zB-HJL*McvYKHd1hqJR!k}JLGkKSXpI-HuGbA45E8P*zPRFkS#j4gtpW4Goh(~ zN)O#tb8kFYxPd0eoyLQ~3^Lc!1Cn&FE?MPDYl=T@Pe{(NXAb z>7J?iCt(ea2?$nJdc9bk>*!9elGuI+d11-V(s6%vj4$3oy6OP_JdBxq6xJJT7|WPk zHVsFt+{!P5M>!!qX0SlF({#|@LHh3o3)Owrpy8PRn9iki=kyUTW@&Y}p2_J(X>_6M zt5?@7Kc{h` zNnNyMKEs|6hLuZ1j}146HxdR)vPT*`O_=)V9KA-|S-YU+>n>y=`8>LulAljw2cBWP zS+8Pa8@e~HHdiH^d0^g-KT`^x^`)HX^w%l7_@-D zeoluk>`ey;*y&k61`~f=7$m)eX{48ab6u*rbdBaucg;7JeW=qQM+m2zZ#v`Q!r2V6 zFrHtOD{COy9WZ>)@4E4n@)M)cL0M+u-E6xvpE5j`tZ@47zi%+yEJshp*NG^}w=2v32JEd9i6y(P!UI}OzI&3MI)2%4JdS;HT$lEJ z0V@)UalC7z@RR1N$qzRSgVL3eGrJ0(MNVFsUybN%Co>JvB`_w!gW9t!GdT}R>4>cp zBgLbh%&;fqht&%7>n5BC^a4HX+*+fbLg9Kn{q3jo7n*fWQc~?mKOBj;MzCx%Z%Tz`iUNDtY9K`5qH+9dw$k+(@@3kTPDkz(4#8-A@^B z)+DDAKu(?q&hOwDR3k^_hj2_(9)qu@`GnD$3tJEI#$AQSX|^!mEWZH_FMW&-zJJM3 z$Fw%oVvc8l=M;tuGf~iVH}4qVHhI(tvIMjhZm2`}Vl{M>dZIPQ?HlIY68BgQZ1fhc zVyY(RknWnjnU38I-UvTsh$bMN9YXxC*5;HFW$udD`Y_0^p^I9Pf8`_>6eQc$Z|bL; z54ws~$a_`m+516HW6&n|McY|TNfF|;;xpYyx5?>u&Z{~CkY7UMS90_SF8K?nk6&cZfcVeFh8rqtTq) zT2rJW8fV^0``Mo%VeW0(Ib$*1Vt<50y9(RsJ~us=KvV=yn-VC2E|WM$VSb5`>kD&v zz6pnnib;@v$1sPdhmmkn_ma~BJ~@TrfoOzRNVidEw7^E@kKECIEmCM&J1coXT1vN$ zG-jB~pl7^0gJws$lA+n&y<2Y#-~UASQ}(udfD2KlwT0u+VSi!YhLAGKQH6^c{m^aNFl^Ppr+8K$vdqv@GI)#ns$R*Ols;l*gUsUM!C8 zPL&;y`_~%8s(|2iA+=jwz>uqKS zPC@`@*a~EXIG!fQtU~ya!XShf2sO_^5u*1OwbYV3GjH8N$K%Aohr6qJ1PkI4!aYsu zRTeXh25c0rqwDl+SMA^ED!}2w{aAS>jMeE(nB0YhpVMq+wwb`d`HE?avv^jT(j~IX zjor;ru&>kb5H(!SUnyp#{z_^XE~0^X)Q#341|?y&?6%%R_sQw;lWblZ^8S>@wvmk- zV7Rkh%Q5x>lmZxMzv(OdPdY6U;0Y^d0aYNGK2p7%*!n;KwCJ9QHm<<7HuFqt;xrdh z1_g|ero>Kfrz^`}x=kLf+ftS*J*8$6j9*RTCE3R5O>Z~rj;@9}2z&)fQZg&#VvxU1 za&fQy+YESic0j#%8mH)6R-%-MC}qK1yQF2N%|er2@8?YLv9nmoARJC0coQ*=7LRkD z6l=>3oLl7fM@=5G%3116!aB;h?^zAKu1Yrd8#9T%n?~Mb5)R?fL%l+xU2(*y z6KA2UJmo=>11j~t!snOQal)B9OnUfvE`CH+Y3ZbRI*Vt(PlFDcp8(1@+Mn?#?qlf> z@tZ2$H1L-Dk_G4A{5>%G46!P-*ac_kqnu`YOF5_3GVi_(fg5P-? z{2P;D+>>~(Np3Ybb|}Ih=PSifY$SZcjkEVN5PPWj+lF#SBqmVo#wasiju5m@m&oOpCNA=bqmk^zGAiBzgAPGd^( z6OpfDyp zM~+bb6$qOpWT`lLehhJL>HfA*AjX^={UPJRIFCw0T``2}RwNSykl48&*Hpg`0GYcI zebr55$RTV;_s(3Ud<;-%*jC(szK_?@J}RC{u~eMu47Z?AlhGCy&p*sZcWnh0w#!r& znM2rCz0MD{&d#BFo$IraWs~)SaG=C70YB88p%>dQOY;>!xxl({fvufA!9yy(RC*HL z&h_RF>FzKDC4l0F7J1~i%{|sW#qeP;JrUO{p%eZpwwg5V7Se?OVs{ll*XiRlwj!Aj zlH4&`!?3F{#^Y(LYLt^HleVcmG+0x`h)(d7^FXtEiKCq`7O|iMAd01{ok&_Bm|Kn0bX{*+!BOti z;z5a#qIw#xN?q~Ga!i{P7Kz6j_-vBjQaL|5+>Jw}a^dxXDC_uXDjLno2g*5)8Ucz( z>!UoScE6&$&;3fta(hu@D}`-e216>49OOvVuMBz&X9<{SESLGGv_f=%XyJjLIzhGG z)swf>?7c!!R<_3y6d0_v6l&IW^2`}WpLop7w&G`&N4sU4BH$HR1b};}I4{cZ1Jv}Z z-k+bny8_sIe{Mc;j1EY{k{MW8KHLGb+5nm^TDUY>IkXbatN{p}Pivi6@l&8u2|#${?7E!ac?1Wo*n2*ds6 zH~i-R=G1QhT-pHW^VLNcNEwGGb~;CW;J3_4^!A1|1|Qxs6_FL?iEDM2pq!fVHNup2 zoVaxkbyLVeDQK0f&*J+;)!0qD^B@xkwfr#Qb8`j;sH#!^+}Q>O?!SZ#>e`Z+>Kg*C zu%CU+khNq9<*|1B~QGKR8@!jX$wEIN3%|+ln9dBe;Q!2Qp1OM?g5% z{V)=H1ooi30j|hUs-q5uq^FPFAdkU{2e`Khn3pqjAtB)Ab~0HLUfT_{4GV~Ttcc%@ z&uK$LH%GZ5BCmb)^a{W2#3-;5fMK6W0)PJ<-;fzdgUOC-2)rB$IiU&pPkc+j5m~JT7 z>e&%z4Bv4|z~>I>3)=k^D|OVM$+NB0?J%29F@3tS|Aot5>=%BR)K~4F*DHK;hU1{Q(ot0> zA3+ds0OY;eEm|TN3g`ngLB;7P1RKmY+GW3{I7b;u#p&01Z4oTcOe~-HASJEXNT(rC z>hFcd(XJ1Js9dCXU#-hgk2rUnXpP2=Tl+|NP^Y_)Mim19_%McZ0y!v$n6CJSxLZV^ zx6u@%R74<&{0TM!{Kt%k5nTeQ9P?AFLv4P&5J8%M=!Q_HzMkmzYraIIe$L_9XlHatBlZ9K0E5zINNTR1NU(CSXnF@$7{<> z(pEg$!uR#sF(dltcpJ{uDdolRmxj*CT=h*0>Q4AEx}y}RoSW6}@9 z&V|~$exX~BPR`M^fYhQxcJWc2h6|(jbd!TR${k1yE)Bkp)Z{sMG}kF06SmQ45D9aR zm}ABG+;a#{lm;R|ho8SuoV3t%CL+YpJdtsOE%0oc-^K*c z(2%N^k0O3GCIDgQT^Jj3=V#oY3eZ`f9Bf5$7- zaZQ>!DJFew#ij3fP`6aW#jszxW0BYxu8jcA#Osq|Smo3}Fp{){5XHH+-sPhcR@U#^#drwT|{GJAS z6K>#_Q5{G>LWMD-k+r~OrzvhaIGbmY+CrECXoxXLz&{b(giC5VlpGfV=8KfHP9@_; z*P^mI0-HA4%eGNc&e(hs)P+^NZU^1X+8Qy&UxQ&bkL!ZCs~KaPmHMvZ|@%`)MxfM+yPGulIk;)@z{OU9;!LHm~yw7?r1yey3=jNFOH@#r5Ll*5=8Oa4npYeR+SRqxMs5! zVse>%>ZZO3gPlBKM%(lWD26%_8Txxpo-j1nJ@teMr=IfR56s@wnB^JlYS+mVTv@>h zAO1jzKaQpfZyByX*>pYbk>^YoqA;3Y=`ItgfAR$7zfzcfm+#I3`Wwkk_QR~c#$dCb zH_$IAUEW81@^pux%BaDHUQCA-W0wK^>>Ja(1LmGfRiOQZ~;{IpfO@P$=df2PS@M~I(lP3hhI<=SD+2lTF zCecXxlR-tJvh(2|yR4la82%Ed9^$)11so`(ak zc>cGTit+~i{}%Ij^u5HXxaLF}t`_&QgDgKJp4pepY%1#hTTJVm4zN6rG1_V%LA=uhmO7{2}ra?d3$61|7>?>7Tl~eP2Lxu-x;j z-^YlB#8Nco8;(N}g|1qsvH`h;USlu<$)?HAHrd)57$eC6jK(mxu=?!aY2q90({P|l z$zR4Jf61}+IgixMM35iaaib|QCp%X5yJLK9XIgyg(WHXx$l-ev);c_D_OCVfyE)E^ z^7|5LiM)a8|3)xPLsIsQ0H_2D`z`|T=)8BcQ_B9PqsU5N!S*P1Lx5GW@#D+p+0jS) z-fY%=7DPvzB>Veb!gI3wUch$fW?l9I;qB$>_F_f(+ZIWp|357V$JGQ^N4pg6K;nmY znG!@J|9^-O7zB?kgEvDHAkmbch}RXy`l*(M$rf(hIJYc+v!uMzrHYjmX-p5R-%oN; z*-1~HK#I&tQj-$FA|Et#YnWXvTf{TX*E_~?V^!oh6kSVieaYDu%@UwSy)&l*ZuwY~ zgNJ2hCvdc8-$AF9IWSMkOYz;Aqy<<_u4x>&4Na}tIIz>g%wxobUbY>B32W{=`!U|K;L|wg39} zcl_7?Uby1oUtad!Lh( zeE-Djpa1of?)t5{Up(uBzgRx?W5=w#^vdb0Zg_6Xm+zRk|8FeVwy5(De)#^bp~r9Q zI&9)a-5)*l2i@=6uk9PZxNOTe8^po?B^im9vh&OG}cYOXc z{+;|@@=NmU`?UK9(#ickd7D1S_jx1~`V-pT!@nP<-SgyP(Vxpn3w9T&5PreuZ_xLT z`2I5guIBj-^xr_Asf=+U<4pzJJGhHq+)B{(YLjN{7)$E+vJD4H3$vD6BZZP!WO3ucSW^0tw>~#_0@oMk_-tWGKZNtH_XYBPoEi+hXJTc#~ni>*=OM%+|vUy|R^| zlXZRaYjpwPadyO)7GS`sX)PA51I~M%x}i&4BFDk zFXlix?RXRr+sWYw78KGK!0B5I^@Y}6+NFU|thocIvn*^=DM+qmkR|VQ5Eo&pL_&jw zpE6*JryH*-d%EfJVY==u9z;B(i%WLt2Zp&p=^-11{kZREwesQ`!rj{ZX{Lk6;~3AW z4Bis5h{KUp&Bfgm1&P+V=Q?wRu6~ulT6%pQLmVcqoFSW7yN1cX{__mFs+HHz4(hTh zR6%koGm;3FO-h+Y*7v?|8Fe+u>C-$zb7b2Hr$DbBCf= zvFecTNzaw0q)~W)N%rwzHHb1FjEzH;c6J}PCnP7H10cHO!CJ9?RsODj*!S&8=F^I| zNvR?lDP-4G_!2@=OUzULaNF4&!HEQ;@CL*1wDBpZDp&8m!Tk7x>vWVHO&OF#86*My zPbO@M2Oj%vVy5~iWYM{;yg^!82#kGz|CtPP%H9v-Qg3KSdkSYWM6qp^oWh-#s2o}? zNCjtNSA!Ez>3(8Ea1r+o*Hj#eiuovzc!|P~75))o;(TnkIZrc+r5PxjBaQh{O>q^R zq%?B{oyZ>1OOY9?s-!R?W$Pg5Egnv;t2lA@;|)!bmA2x33yE&(Ej5M-(nM=^Rqby^G_}X2&ODLG$t2PI5KHXk6THzOI=|{1G~T%%!>Tg}Aho z{TCC0XO+*~S%)({G8KN>fUNEgxBYuik&J@5IM(2g>Z~nHEAp z@QW{V=XCDjdQj^w6dH5y5#u|D%W$<ulRehR?B}#Gdi2)< zsg( zIsL?$$LW_}LrFx@oEP6)nAmPM{f2&B@zQ$>o7TLiFmZKz;XfSy>xBouSu9MP+gt14 z*`{0GS3vh#n0VUi#q;@h2j9O+;o*tj;)}%=7G}#^;FNW1)~s4lt`}w}>cT}ot}3iL zqm1fvflEA6nC-7s)L^#EXLG_OkADu7L{l_*T` zGvij)%?FOfzgAltr#tT9osHuk7?`{yppE#a(N}bI+(?u;{x;7g_o&q4pBCf?j<8zX z%HAt|2y+R|;v&awa)%wjV`Wr9J#L)b+T4Bx->$(GF77Pu7M^q_4mGg^@?zz>EN}AL zvdYF>yV19EMq)qVs48`~k#IqR`Qn8wEw8I3c6Di$q-pU}b1zG6RuudnH?pA@rY;Ei zhx+0Mk?T?;{%vup#VQN_j)F$wX1(S3^I-gW6@}=b1BK-sDs?YrrO;lnV_m7(fW~0K zZxeLaGyJIMHr+oul?}Zr<}NE&Ac+qWY?VjTK{k#)MaT!DGug?3zb~& zgw~pM6xKl2tXe6)#f3FP4O~}!y-WWsfq@HaVA&L*RhT`mMp4cFE<u^lL zgGjqFKy8Y;M|8I^TezcrxFL*?1CqeQH#uZsGaUHyeHK-E2mGY@hkoOmJs@*Uf8|Vy zzCx}n`&Pb@hR**AA?Ogz9?m~nR_k9(dO;JgvcOQKj?RKqJxB|EJ z{H*Zwo`+b-#9}3~gD)B!99RLNBZXMkmny@;GlMWw;LgJAISr$j*)D5(D#XMPp(^CG zp<1u?OPGyNKy;wFxn?i7vcUNxT+=Dk6)%=m*o(1-1>N12)U4uBa(j70Whtz8e(+^A zLxA}Q^Q#T{MZrGe6;|nGx0Hl})#sazH_4g?sFXA^gB27f>kG4&4-F0s4bHbx;mvk; zc3=&%dpO%CJR0@+iYPBoewCtaOewf$&!Q?3P$4nIMH`bROCMzFnVATgu7XR@`f=Zu~1h| zsT!&BEQ05wqY7)N9oW@&Peu(;ob zZ6EIIU<2}}>7Pe$*X3WgNmEN_njgV;;2*!@e3QiuZGZ0f)=suX$}$5I&=RDyNP=dR zTO6-d2yKy-nmN6-d1oHmV!IYE+RU!PnmRl!c^2TTYV~l@!_qM>#InDn{TO12Ic(kE~TTtBSb3}+z&pO zF#ch2rv)?m$E%30@*h{iVw3F60@wUc34YQ40=r@P$mT~WfJI*$mA;N%l1(Dj=kNl{ zCGtP_^RgOCCC6<3$((^I6g>BP;L}iS7S>P-en77z^wovVR=dqYh*-`|w6t7budGhF zCoNs#7GZzJ%z|hnTvL-*Wm%p+=5d`zmkG@6Bj!9|1w;2%J3QdVzcmcY=6?35oldX_ zGG|2N@>A9`+}NYS?75tI^OGADl{dw)vzOI;evPoq+>X=4FL@4`orfFM{C)(6VgaBr z9PTN^DL)p%Qp(Xi-(uwu6fXC3OUHhEYbdX~wPrJ>Lt9Ca17gZ8XSQeS<|0mzW48%6 zx2D>~=|zc$ryA%x*8mme4BS#iD&5b{MaU%@u zxX4_KWg-B*h0}+sl|d+y1=ubmbEpvF88erlC+QqsZt023kWkaDx?@N-h&s%OKDCof z8Nv)}S@-##P6*HZET@51Ds|MK^$nG}M)SWrOcDRAJ@2asnhY@;{~BH@NP(V37Ok9v zBsS;(VScTX>L^2}RR;Ab0WcBY7;(S>I}F<#kOvc=3yyp=+9ht)`W<0d+!#-8^~ast zYjG+j!vzHg162x9p6$Y7yNHEEtLXDQG}6>t?&<#T|3+=Qx<^nl)afQZ31`DdP&%|Gy}g{dEIgTL6U3EKV9OpCgh)p z@y`DmowrO(O`b2XKP`%oGg@Vjk86>Xx5#(RXweKXqXYGfDVc%^ zw9(o^W}@Y3MaG{wQ?hJO07373Cg|if5*f^&baY7hQ+65TyvT6I43MBQ%7yPUGvANS zd_N}h{n*U+;|hMHRWQquN;aU~&m;m-XI^q3|x5P9DX%T(WnNP~;EXj2fV61Evf8h8k-fUWL9#8CPKg zHlUS9Dd(xbYZCtqD+4(ztm$S^&XC|fJ4UC!vWb$$)y7g7hKz#0FuR%LJliC_JP0mM z4v~MnqFz-h%sgFOqoh?FXceD3w$Px3J$C!*`L4ojZ>o%lbbf-!EIE3c#I8A5WRo@x z&x^LQVfPzlQt~G$NK|iQ71@7PRi&WAjlQhu_K)43QU$lx7!-RYPE`K}C5%3=VF&Q= zD)+i$u-QGUh9Wg#HCT>H(spB!i5qrO;RXcr`52EpI44?IEd=~Ooxzval){VxfK_=K z^0n$8Xoq;mU?)c4s374aziY$9TT04t4m-)l=E1}0q?&FdnzRp=2Z!LlyZ9}J+{VfZ z?ie`Uz^-Fc1#7ggY8O?f5tqE$KU7)V?eb)sO!fbg_O8us-N?Eyf6V!MToXHU*;THo zvK_~xvwb_3GM@L7ik2vuJ2I(FQckpg{rfz9!(}xVnK@O7%!Ph{g@tZ38jS|fu&U1c zWw~EpG2_`D&3=8hfi-3|(e>`jYhCS#X|ZT0xc7GA%EN14Z>~RXvG%WRwm=GI9e3Z# z;{{JEwZ|~nIb~L(Smx1JxBzJT4Qyps!^)i>^Jx5-%B<{3&MO6X)w)V-{mhZZ<#=C)`p2Q*1#bYx+HXo6`p%49ar8l11u83 zbr{Q30q=D>NFk^IQmUpwd0nm0WvoY6E|X~~tbxgm7Ap^L;TJ?e$v<#v82&}6Sf95C ziI(w)#l`l1@p5|r?_!{WyXNw{9>*PNT*+i-Uw>Qf9*Z-IgDLoW-6lTgSCFxHC}!H4 zH+OVLBPT2o3@_FTKdo+8SZK$T3vQ|IRB)q(nwEGyOu2Bo2*&79H>RfoM`l5V+*5^^ z&%ksAf-OGDjc6R)OxHnQt$x|9Q3(-ADnEByn6T&=-nbpiv*)2u38EUcYJUazi>vRx ztwe*xc%)CUFyOAgM+|*VFMlRcrGt=n3|Y2H?-yZyU}Bp4I-zFi z0~E&#?;>1?!`19RHGHSv2OlJ~;57ZMEW}RIFL!j|i1+Fsw~OMX!pJ=weDJkNM?Z(L zxwyp0xO?PafGz^%vE%AL*LpF-fjN4>xC}oA*4y!=4_}yBEMIjMb5Mpe3Tr(g80!1K zyg@1k@#wW?Dw{)Do=kPYGwK)oL<34Z6gqOS2N}R1+J(5^YVxJ#jABi?cTS)uN#N6G zr_Irxof=ql;`-{d6DP+%o><BW#nsAtF7w6hX8TF+YLK~(c->=K#4$%wKI8TSPKknITJX&S)}d^O z#5=J`2jdE+BA558)vuTe!|Jsu(BvbYe_X+8*&L{#?r`(w1R%2qnbwMqL%39^w<-3E ztdyxr#AF}yJFMc!Wm9Tj%Ep(KuI!v)%K&B$M&}5Ak}&;n!(IiKqxp(iKg@y-D_iP; z$`<7;MI$;DQ75+O$=>VNi_*0!*FBf7;e-+F(NwL+B4%Yf!893JoDNed?_}!(3W1>>>+Gu^WR-t zPL6xeZS`i-G?&~kqxQnyJ!5Ep9=!Z;FQJi$6~Q6^mV0;ZX`_CZq4@%PdqZ2FUAuIx zug#1#S3Ij}&iMz3?DOcGliEC2F`uSGZPVvQd7TpVWNG$BvmquBHGp}ss;;q1Q%U!7 zOIxhrZ@600Oq9jt7YtMoY)mcDdgzw8NM*`7OpU~Qh1t^Y3&yuBe1^Mtj_JoxBH*i9 zBk~j>X8Sbm6wNx9clrBFUUP>`8o^yyWXFUkn&qd}mxo(l;0Y^g`qj_5KlN@7@EKPS zon>9+>+0%P-gPZE22W~X)_%op2$)9b=pyrWqbh5y(RqDXg$64pE{vF{hCZ5t8ozWn zVWrzqE=46zYJ7>ypuUP<3Fd>ZQcqBz$&G*6?EDPjyc#AIbD>Fod5c@p=BhG(sr7p3 z72YE#`ZScUMpNt#FTP-Ym##S33b@9B?Nf*?I{BU=jS48AC zi*fU>%cgW-%{koXFCY1reG^s>w7glVh3RHShX9xEr=H(fP~c+c&)7xfR`g43I6MD- zrK;7#4Z>w$`@clY&k>||^|fFTwxA5#m$XY>6{K1;}VYnFgu<&ZdjCL&~UOg?nuK5f(rlgI6EWx8o<9R!B` z%FF@3*L!}qD8*=1SYiLUTEQ|l&hKA7M#+z2h{2VhZspI*yI-lnCpE&Fe=kWB3}z_a zLhiz#3xW5MuCCQBFjbb}#5nAA8aFLge0wkjz9H5NT(T>?Tq?PSMM`{I`=XK)hGHAP(VB5Lwy!PRK=95nDjLZrm6V+nWjU-aqhmNd#lv{~DVM-Kf}qUEj;liLSPC6)DC=IGs&yU&o#V zH%v;3sDE6p>HAI=Z`WVSgRB*oN7)omJ+8~X5G+aS2B-ezq5EYrrE6y}QQPE~E3EM8 znvi-Cu?lkCJpZz}_r(I&f$((OtYIJDV?kS{u)>2vU-YE2zh51~#5$8p6}g3Zf#g@5 z@cBAaMe-G+jP_!GKPXTsqcY!ODtLjjRX*%q(}#%Z$s1Ws+_j0ySg+IKJ(Muw9Mw4* z)qy{Cy38S?qdhLRQM@m$)I`hTB$;F#)?e0l zher!)z)0`aW>sF?t(SXF!J;yZ4#9*C191O6`^EqIU)qEI^!s1_^rt`l$JG!2`-d*= zPgj3MTI|y=0dxG4u!yTbI-hz;eH78l_C1SK<<3ZSk;zPy6& z#UorL>>{GLO1$SJXi`*_QWuHllwJD;KeMa*XS_*od6uwM2EKSq##kcTRrRfn6RrxMCMekHZ7=0!0wE4nb zF`Zn2odhRKDdD#I4KLF)7YzDoVzrsS6g9*5x`Dgx{3XBc=dYe+Lw|XW9sMQg+J}?+ zD|lQ02Z1UL*6z3JF%-qVO5_&2>ah!UswmjlUuuAz{qX>9a90STf=kS|MD6Y{>~PM) zRmS~dv-*u^oJ8E%0?U$I1ccK-*2qbG|M2Y##t8oW$nCIP{JHXY%}9Yy=L*Z0o7>Oy z@2Rg)>;mkmcX`||;IAU){2t@?ei4ES9==8K=emqag^JnTH75uyb__5hDKq$UV*UJ(Dz*67@*`|WEtOFP3 z=m6JhV&xx3x2DOlS8OKzDulf+;ELz+G)5$i6N_4DOXz#+-`X%)_0d3bGFsGfL0u+N z;X!Upzy$GypR#4XNVex|yM~u@58)<5h*&$3+Nd7_)o}dKTX-Y5Tge3#&YeAcfs*3V zX=#3P*uuSu?3x^wAod*mD~UX_1mmneI!oiLEEz(1vHY^I=sW(jY{GTaU1E&n;nmkQ zj&xvHI1Z#hv;mA*a%k}a_tetqzJOJMc_{nH8i67_U&A|=Cj_F9V2R${RHc5{NrD$@ zkIHxs0{C8wS7f`1PmzB(+h*^UvynoWCtU(A0j+h@B_K*4ENX2LdD*zC3(g<%x{~g!jR8^EdeeX2Ub$z7;?Myg%Vr2mVPMe2E`KiTChG5)64fUu5Uj|f?a``? zm7gNPnt!pA!-54j133Xw)DYEHYPGmwd;!;n+GGv$kiRw#h~nUC9fkjQ%CFWYI>kb2 zK-V+nIpCgfV0;`tLt+8Une?wJns%{A5Z+F;!Jf-n=pI_Or%4IgR+^-mNse1s$dQQP z{k0BM&;NdZd3lb3;LnTr`IiMeQ1HkgN>exS;L#38S->cNfa)&Cv6O7ExV}F&McKW@JiLX_L@z1UE=I>80-TjA}{BViYJ~&eG^X?K$hZ1GX4J^xRIPHp2 zu1KK&IxpIy4UZt=V_M*nha?UN+*mDIwswcxI1_^Fpf+%9aUK|PNBf89Gqtv8$9N2^7o!E7!B;nKM zpewZ#GRz_ioQ4r0(a7kh2tvkR1TtXthO4)(p#D~VBD}^^5f6bifuD>eg&! zFGxcrr;(vLO;|if2nG6cU*hwSqn=X5yrFe5Iy}Ey{?DV7vZpkmL+n?p4S$06ZM~^P z4_yKY`bN;XBl|#+A{a-y%tW~r#p_m%gph>(MQ%tM#IRalDvFF<7~ji7>Oz@PZEzOF zxa5yIQ0-x}b}^2)K5qZ`i#HM)`LGr zR3B8Q=qSv(pfVZxg=5$C2l{}0!XC2b%T(-&R(lLj)stmkJb#PzOPq1EhG+Adi$W7t z-${I@6~g|<;=NFG28<1p1p5NK(tdH+LkF)05=(J){sT0!VTIcqbxcuW9~o3AC#RC% zf185E0V!onzyRU}V$)ZYfFQ*3o2HGGj4-pRehZ7$qFR7=B3g)*?$g{zmbyYe@?X3V zE-?JS+l(pbyHJEogA$!Es!&lU?V}+Dg@%Udg6#?otz=>3`VQtg_1P8e8MI8`u5mOXe3Gm|9Md(}^!lNhIBnz-*2>yNJH zRI)qj-xg@h525rR5q9Vrw!3LC5oK(2Te*l`U&~KH(l9ag2o;>KYpHwq>ZqP=^;X(o1V|PnJJlbDqnxc-oG4 zlOLcGDTE?kaEIy>|1EIQ^5xtBjB3T7$Jv{O`JWUT`qWPUit!&KfG!hZHpnAfv;uoE zt*FrGH)EyI%i+*Chk+ssj&!0j8G4qkAO8HCPG8{RfS9$$pH5_~XqnW6x|If5`bR|b zicT`KCd`bBjs7f@ZRfK}y&C0_tSR ziQ~M5;-@*4l>R-1B5y2@J&=1s;X~NmKG{h4!IR(j*Vy!WwcGBuH-{&mS6`mMM-|S( zC+DB4YU3zlz4hXpeiUg2ji%{S0ep?s>fK5S9<{SjbV_<;YxHPJyGe|a*0i`9UrM#1EoNh2$ z$&@@yVcOR*5p$X9r6$2$%HL2~sV!b*Z{PE^_PhG&-vM)rQ(BzLE@HlTUeJ8S)4C57 zeRd1p$I=sG26=1AY9Z0u*id_h4cvG4l9Ohj*^+Gi?y(fX0y$&FQOd#q|IPI3BxRx1 z&!oGnlrQk#n$E3O%~{HlKRbfK-xDv8+W*s!RF>rOM09&Ll>oXVqVOGdva%qmM^exI zcvn_E9)uzaA0xc3B?`SwBC0qq+{;1WG}Rd7Jbn`b(Q*_ukk8hmm!-n?lP`N?2QP0o z)ebcQuC}ZV&rA*cMVZ!U*q0VtbuNX>}*=p;Y;8qNsusQkaYwWriwOLJMQvN{D6+w>jK-Ciao^Sicf^ zRl(_tQ4U=`fn|0;nrGC~IwpIDy8GF8VPCJmJ$yUIXlTX6=%a<(v>}#?k*7o8M01bF zU>3f*QfYii_h>y5i)FTr|B)b16@j<9Sb#UE{V$@-+LLXBA9ix=CEl&owjUQ;mlqkK zbyF12f#TM~*NiBZLMqnl(WN}ifIV6Yv;Lnnv-Dutsv{D-Kpi6_2RtfSGJLWr4=$Wc)F&Bn+z?m%i z%pdr&I47MsymYX@Y{ejhIam>b5o;=(mrvBv!?G|KEg2U;&qO^M`eg&%px2FE{UB&B z@;cQYfrhi>=$<+Wtqm%!2T?)E43}TWMaY=$#)E#hZV0r-vZ)DX9v}e-p|l6x3T7W_X;6@i09ZlY#^N*%TH73a_OwFxe!y<| zy`C_i)8ZKV=V_fNpP2a7H4~8)iH!O&3FC5IWhOz6lZumqrKABx0(mYmgN$?FrQMnY zYg23&Ayd($_+>Se^9{*ajZh=fkB1jyClNIB$Q$ofldFC9-{ zbU?E>OYHSBLQ!>dN+agLn-#CmK%}=kNddHn%qWm&YfSrM-`XifK_;}q8=M4L^#F*eHUcOkRXObiqMT1PJxRTq@lNnvEs?EFq5QgiRyQb=0UzI;YX`NrV)Er(v97QWv{hZ%f{@(FH!vOI zo2EZoFecx=*31griDtn%}@$*#afnUdC+^>LgwHZeG0= zPF%z4u*5LCrG>eX7D-w)=>{_qRHT5Kj_OA**tC zy#xsLJDPfGh4?{ra>5P!2vx(0i%4ZxtWO-vqK2wAS-pa_9X(xahlNE)eXX{ z(Gb-m+ePn@z#W9pJS5l8r1uvySgUmIZ}b0RIYfTx)d2F-Zwrtvi-0$Bk;l~VUs*rV z84ZF7e9>42i2s8#QY9=+ScAb&3)T?kJ_}m~67-f^htAi zdf%FsK50x#_qG(%3cf2-S?YJB|ID?jwC+(QPbLlwJ`F0%XoIH+n z-X2Ffw~r&8|HqLo5hox`Mvfz0YK|jaj!enXfVgxEg9HeBmryI#CDe*<3AJKcLajKKP%CyN)QVRLwPI98EnG^d6^jz;EPwPk zvoV*HKF_Pb%y7&xb9^(*EcXmE%R|G=a?&uf{4~rgR}C}ETf@w9*fDc_Hq0!y4HG;s z%THJ|tk%n_xoi*$my=vYqozdXYeo!)X2jrRMhq5a#Nb^<45nqo;8sQqHl;-8Peu&J zWWDCJ}iN;%qtQqK0El*2tJ<@5xU zal8kmobN%I1Tf+aL0?sGR8?GYZVE}IAcalkAcG~6kiaroNMM;XB(O{#5?Ce?2`rO| z1eQrf0?XtggC&xYz%toLV3~9@u@9AuM1({t37#v}*oByIJ%|~vgP3tRh#6mlm~k_R z84rV)aW04%zX~zoQV=uV1kpM2lim$zvygrxz>pfenL}s?ok19v#t3KC7~%XHBb;et zgmZ3;aQ2N6&ciXn899S6ZjKSo(lMgrt1gAYiJ$k z8d}G*hSqVbp>=#}XdQ8MSaHqvrf!v>_JXrtDF| zfIC6VnB&BhH%81^W5k>@M$8#w#GEfi%-LeZoGV7mnc~EhCq~R!V#JgqKcTyn!$qcy zY3VtCCYXdxJtpH;he?^%VN%|8n3RPbCgo&@Ng3K64a4IL#g<#c4o z+DYXCtwMJq+NAF^w2R?sXqU&+&@Py#pc@yYZ?0V>o%yrN?f$N}i($+!eM6H9)$yo=T6S59ECt)3QPP}^PjBIt# zIl=0nxm5A!KnF(bR4t|mBf!N4h)$9K-ifwr$*G`CAX5SBA&FphkW8RDNG3=fBom+x zk_k=+$pof@WP;K`G6Cr!iC}b)OdvW)D+s>-*M&f1pi}}Tw2r?Sy=HDgZ#X-pw``r# zTb@qoEkmdDmYY+0%gQOe<>QpzGBKey9Gucy_D$(6@7`~nYwWt%R_HO!h*0Btbl0Iz zfp;7}F-y(#)3P*ja8j05G)~IWO3O)ES^+vKODj_+WogCiq%5uEos^{&#?!Ji@_AC0 zR%B1g(n>MZlN~2lb}(1tOh)dcbO5QPGKVx$m_k~qOChb4rI1#tQb;RBDWsK}6w*pb z3TdSxhcr@$Bx)?up_p->xeDa zI%3POj@WXlBep#1i4Au;V#}9~*m0ye0B6dVgxqr{hcrA&AuXpeNXM@X(s3<=biB(T z9S1W=$HxrPaWjK-JWU}jXER90-wfhh{&@+b;M$HSKo6CJDCqyl2*KF_QF)pZjhiXa z`Ir)&gDKH@mlB<8Dbe|r5}i{i(Rq{;jXNpP`H~WYBVY(EGw#xR^>vHeZQ}SPnrc&0 z;mU|?d>N3PGdVeUlaqrxIXU=~lY>J!Ie3(lgG)I%_%t9pr*d-eDks<6LclF{marGq z9%Z$*J4(Z=Nj^{us}gF(sVTK))RbEDX-cixG^N&Dno?^fO{q1Hrqr563AN(Tlv*=r zN)7%n61vfARYKuLn`lgE620x~#Axz5FNH8%gu>s>wTx8UCvHJ=kf>5$CHgOwpNIAp0>aqGLX0g zmWkRFmde-!mJ8PemP^$HmW$H_mdnuumJ84XmP^hAmW#|3mdeTmmJ7)QmP?1Jr5ATQ z?DBrJjm=Zll)(m4GJs~1l0#EzNujyKq|jVyQfMwYDKwX!6q-v=3eBY`h31l!LsMx= zp}9n*&{?U9!Oq4&KYL;*UxN(uk~YpXBXh${bJ91;G%JUrOtTU>$}}sRqfE0>I?6OF zucJ(}k~_*YE5pM~bJ9G@G%MGmOh+VKPMpP*o2ThQ=Zc+}46)XE>>R%ie@wo!cOcKK zc;|WMr8~`YM7YyDM`SzAb40Y$JVzuu&2vPs(>zDyI?Z!Ltn)ncQk~{GBGhS~BGWQA zX%BoiWgUr?bhhY91bafrSw|?s)Deorbc7-n9ii~PBNPUAgu=y+P*~Lyf*&2BGl#qfy0(+FuGu85YcWad8cWi;c9OKNi8QTk9ZBmN zM$)=A(Ss)b$t$cRKESl4e0NXl+bW?q?KY*iZ8)KK?Kz=$Z9AcN?L47(Z9buQ?LVP+ zyqM5CzD(&Yk0$hvUlV%AJ3GNj4LISLru2M@)P_5r+VaFwJ5E??*Z!8;wYsHtZEdMt z3tMW}u9n)hrl+=TXsKPxS*o`eZ=^H_(B)fr=wToHveWg-oD?lMBx_TS$kwIGax&Y2IPcWI3HJozSYxNWh|78R}FQ-s+v0G zR7Fh~RZ$Z@Rn&w{6*b{fMNODgQ4=0j)PzMfb;_ZNnlPxM7XEx+{ftW%WT<$HqvH5b zaUl}C;XNVR+7e3hT0&`2ODK(K38mdEp){2xl-99?(jcA?ZD0xIvn?UM6xX=ITPJ~% za?3^BH)rxVrkD1Z#aI)~VZ7C5Fwu}Rm}uJ>Of>flCO9yI38u_of=@G;VBH+XxjBOg z2G3xe=kIVCJ9yrYy&~)bTJU;IS3VEv#^Vv)`8%RJZ%1_J>xk|=9nqbiBf9f)M0Y+8 z>Bhqm-T6172k-3O8(R#cy^9n6{cNT8TRlvDkP=*Ma7b0&b*aX&Hr4smraE`pROd;X z>YQj(z5Ux%Z}m3S+qz4&7H(6$UE5S|O{ji55w>ozCar3-KHXcbOOJMI(@V>>>80)3 z^wN55dTGBly|iGPUfQrtFRj?6M?1FZr6t>RZ_5uiH|s08p13-RSWXe8^**FV`_3V> zrDqV<=3|7n{utqW7$ck`V}$c&jBqZE5zenM!Z|mCFdmK(&do6*_L7P0opA}y(@_b*(h(pyIsybkM}Xkx2oUTX0fL(&KrnL%a9)l8 z!O9V!<)nFrKCf|E>%)PemUc#hav~#l+#EoxlS(aeNW;+-((*Kgv|LRgEnibe%h?pt z@-~IE+)W`Ze{)E~;S|#HIE6TuZ8~Yr00{%5l_1vLfEN56)0MeHx^Z?yceak`&eIXy z89JgnH%D}5<%sTl9MPSLL%MNrM0fU$=mXxNYhn|d1XIXnU(24casVB1b_`D$J%s0c zAHfG=FoF-HVgw%u$p}7>nGt*-LL>M3MJU@G zr^@Ce*GVGU=Q>SNyIdyi+nuWzxuk|+?wgz>hc-%K+flLWHLURBbD#D9Jy@I<;dlFE=MlY zb2)N(p39NT@?4Hwj^}dZGCZ3jmEXA>x$Ms6I9+ZLDDuZdw`BMO(%WP@Re2xu!GMz5TRi@L0xyp39JXe`c7wIa~=~7)~I$f~qOef2BmFaZxt}^8kZquLiO1)Mq z50C?y9-}j19->nz9-(vb9iel%9iel99iekc9iej(9iemC9HDce9HLWc9HDbD9H9sD z$GZ#rGebMU>mVbEo5E62o4|4z>tO@o>R|(^>R|(M>R|&p>R|%`>R|)P>0twrnZR;c z>0tvQ>0vX{;RjqKvm&`R%jt7+QDKHcQ7~hP2+WK)1ZGAS0y85Bftitlz|4q1U}pFq zm>K35%$W0mnPGciW_W(FKBR6oVYm?Ixb1);s|_&bvjS$AtbiE~D`1Ab3Yg)o0%jPi zfElhTV1}gz81qvBGt5*#&dK*%THIJttuMyq0#>WvGuVK;18BzQ9GY@Gh333Zp}7>K z&|Dr;Xf7EkG?$SSnoCOx&E+PCrV^Awb6HBE$E2!tJtV_JDYbJmOH&7 zvsG~kY9NC-G84ZPnMz)U%!MvP=JJ*yb5YBXxs+wdT);A9E?XHg7poMRN>ql-g(*Yk za&&&bx?0~n#ue!$y57pLHOLe9`yT%!^b7=YA>^u9MExP z2&&mL0yTUZfLewPKrQD6pq7OLP|M2!sAcK^)N*$KYS}yjHT)icTE-7RApsBjuQt5$ z?8=ATRkgY89r2 zDJNS%!pRnpaT0t(4>p{t0NB;0Hf89Q4<#?Ka!F|IaZ4nt;TZHo!hbZtSF+Wm^G3~giLztSAf}=ySvU5Z>UJl64$N|~8I3PO< z2W02pfb7g0kezb_va@YOHl7X0&aeTw;+7vTdcWEoFg?B6?hvzn;5{hLv@xRQ-7G3t zIEN~noIzC#ok3N6ok3OXok3Mxo2v^C<0oM_jVXap){e;`tcHh>UxTN6c?cKF)nSd%f6<^EHAIKB(bb6j5TPNqK zqPtH@x=*f91c|AQ(k=>K?RC0WBo?NEsFo4^pY+Cox!Y#_P#HIQ7|8b~f&4J4PP29k?T1IZ<&f#ibGLQ*+sAUWF`NX}*5 z%H=@|Kdov0YHz^eSOGmSrFfCuGYcXtrEhk7mmy_;|Kds*h&NCH-i&T>5R+ZJG^*z&ZL3iwMixu>UK;pFs~~em+Mg z?z1^kNuSG+3;A4*T)yXWD{e$arTLhijNo(dfutPCl8McHmQ;pjv*ZFbnk-sj)bm!&yRmhDmQ)5Ls``$UP) zbDt{wdG1sDL7w~6E|KRxwRhyXPwgmq?o<0rp8M49Gst~nPs($j+Ntu~C-yHMA-K9e z+i;$E$}mymPwh)xo|Ae|o9DEC)8sj^w={W9>?2K{6MIIJ=fwWdjgN5xwwa7Gu1b z!#Iy-Fu|)COz><56TF+j1P^C0!OIy;@N@lw_5=Rd7(mN;Ti`Me+V zKEVxTAjjsikYWcikzq%&kzq$Nl3_=(l3_(CoqNcP2HISDanMq8F zOl2lR=2DX(bGgZox#VQXTy`>KE2VsX$1Ty^@0kxtKf_i1&@r3OsO443e5f*I(D440{g1efx5ipzOC#pS%7 z;&Pr(aXIg&xLgLNxLg*dxLhU@Tq+w=TrMM1TrMjrA#xhIZQQz4GUlg4 zW;kk)Ii4D1j;jWl+t~8)U%c`)gbzSj>ow(;Z~Q z?I|qf_ym@7y@w4r-@^vn?_mQ8=wSmX=wSm%=wSnC=wSnin80$W=wSoN=wTrpI8V}? zKv>%69h2`Lcvhgw#ei5OVhk`@7y?`hMu6b|2oRhf0fOfvKyZ5m2tJPh!Ql}gcsm3* zS4V*0=LnE+l2Pb#NT8nKNPZfR3nk-WL!B_NrcSw6Q4`ix)P!#pHDOvsO*mFj6LwY9 zgjW?cVN^|>a;c&wEUKu^pI;=x;$r)1yZMeYa(FC8IlOg53@*(8lu=`V@oETgb`1f} zu_3^jHUv1|h5%>X5a8S!0-S+kfbnn$a5fGBJttrPe%LLq4lnlm)weHqkMCB8B?i}} zs7^0rfSK@hjBD9D#C2RA;d*9|a6QjQxSsVRTrUA5TrUVCTrUqJTrUuQ%6npQs)h+Q>*po5;qL-pa&;-pN8w?`5E;_q^}vJGf#tI>1JP<J1To`A5L1rmPWcH#ntkv-z_If~v>oN}rV6HWy&<5UncP6aXJR1h;x1u^4P z5Hn5%G2>JrCY%al#;G9ooO-*x!gS3USsjq(oXG$a4h=vpr*crou@uyEE(P@*OhG*- zQ&7**6x4Gz1@#Qf-IoB)1faMip z!0!q%V0MKVaJoVa*jyn7JgyJ}2G@w3yA@)<+6qzl`s-%FT%A-$P8{$9!498FP-|oKr_|!L}o~;NcNmF!Ts6xO)T_tUiJZz8}Gb5FEp~ zL>$3|cpSmyvhsj~r8kGW$4?J(c)|OJH!IwTVq;%Z;s802n=v|*o*_Dwp%FTlq!Buo zrx7}rsu4Putr0qxun{_!vk^L%wjnx|xe+>-yb(H;Kasaz?k~1K!TMH%s}{P3oJ(2{ zNyMvzWHQx4QbB4Vsr0mvRAgF6Dkm)@6^<5?N<<4u#h`;^yl){X!&^woWuHTFc573t zDX$Yy&hZ?W@I3`)+|Pii3}nDm5;9;a4;e6(iVT>_Mg~kJBm<^$k^(bn$$+WMWWZU; z(dn0nS~@y;fh}%cekRx%5$dsXQq*B*1*yZ%%2J1&6{ik6D^VSGR;W7ctXy^2S<&jT zbJEpeX9cXo<}xNDP8Y<{Sj+lQ5CggD$^j~qs0k{Sr5=?_QIE>yr$^hU6I3c2Jt~)q9yOK+%%4QtB;yndaq^5LRFIJv6v$8p0x}kWfQ-2xkTL56 zGUj_g#!L^$nBxH%v%5fsybj2i(E(|=j8r?Gt+2MFwft<-D<;)kGJ05s>X(y*KFz%nry9b@ekBA{bGXRyFF~Fo{2yhu00$ehN0GEd$ zz@=aaaNZ9A&ha6@`8);~cZUGy=@4*?lfGOkL*=Lx2OZ^X1)S$_0UqIW0FQAzfX6r= zz+(~+z+;jSz+(~-z+;jTz+)0pfJY=HfX5^zfX5`~Vtud`)REW-;;76xYF=OrbwpYe zbxc$gbxckabxcSUbxcAObxb@IbxbxCbxbe}bwnx@bxb4_HIavh`@6M-RM;(rIES~@ zA_-tB4LPV2h7{DxLI#?MLI#>hLI#=$LI#@1K?a(LK?a&gK?a%#K?>?+AOlTAAOrOh zzz~Z1>i35GJ#oVM1kiGQ3g|dK0rcFS0D4YO06mu{fS$t>K+oL?py%ua&~tSP=r}q7 z^xT{PdQRHuYGxGjMCEte>(ifJhRBHs4UxSZ{j%W-!TM&sx(;viV`e_g0bNfu zDrhc)$n*&P^LlmHTBlV}LYzo%0%(PH3g~2Y0_a6_0_Y`k0_X*D0_f#%0_eqW0_de~ z0_cTp3g~2P0_a6+0-!|Uo{jC@_tlHdHT=tKIdVa+PY@QLKiuDMcbIY5vL5yjt~NAJ z_lt|&Bf=Riozt8}`?z1YXpva&@zyf;A9pw#^lr7>$IcLzz@UP=_5a}fsKXBaZ&Z(` zNxjEukUaJMXH0~+9eMHU>vD6ulArv8TSKkTftLbZyvtI4=ylzFV$03*4cE;Q$O->E_GVo2%_Lo8fG;<~U*Wmm$Qcm33kbqdz=z%I|9)8BU2JpYTipIUF2|`y3h@tqzui9E@a%uORZMwj@onVs+Ctl4EW2 z7c6q_-mbqaj^H5A`}Or|h*7t|JxL-DKfNXC!q8|je&w)`IdK0>x z7pGs&rbBPNUv3_~v&wHxzkl4VZxLLi-PoVNj#!6B2jXV>cxJv(l+m7QuYX(~zKYPj zzPX_;`VM7>6ch(Oua@70rpMq2fp6^|+nmpzT;Xza1jvp=aC zhb?Ps-EX0sA0X6%yjb39ouX-q^a#Mj(hn#2w7S8-fEsA7@?I!MD+mKhN8~Cy+b+<) z?CzzCEE6$FI8%e=Vz~n&rI709>))^L9?T#2ZoPSRw?zmd7pbvg@j!^7DW(PuU^%60WBJ`v#lHR;~-zIS=V@b64 z_q-l=@w_RRs+8453pVLYgH%t_Ai0w?v<1_W>PbUDpytSsnK?4Vg5>rarCh5@Q1Gde ze%mYcsrv+xi1Lu?+GWPc{KgBY;`iixfCdozMTu5ke zn9u)-9nc$1SUg8@6O4Qb3v`N)N0Rh=^?DzY7XMuD*M|>Z{tKp}{*Ec8P=xu&?WtZM zCaET)QIggudkL!rehw?4)xJeDo^AGO3KpaDPt{B`$i!Nln&}obVckZPJ7YgS?7wQV z@T$dQE(EJ~#knu7b@eTvbNcm;!Y&DaL$#&12?yUJ+lqdR2Xql|Mf;)ukfa9k6AabE zN>TdzTpa{bCAK1OH(GthLf`kxr-Z?_a?`Ko|1?HAlw%0NB11~Sc`rIQ}ov@WwU0MpxYE;KJM1r9sL>18V~rq*PHKa)R*o5&n2u}c$O9~ z_C?-aK@L_6oXQUoxH`ODLgk(>Z&qjU6>%%&*7smIqO(P7wv_4;btUOmx1yktK*0k0)3hRvXf;nx|`g z@5=|wNq4Jjx>`_EHUet>)9PyXc<PPKXb;{?jtIZ2cwXqKs z-S3tSc#dY-!*4B}Sf=9gdQ)dQuiDd(Whob19rj8Q)?QH%pZ~1Am`dyJF1%bS8-3y} zBm(CQ_sJBLBpEef(|p=)55Mfz@|yF*^6-F5i*0TQ1$Iy6+6pkI;#9P2xRg`|1gdT0oJz``}OVMON)a6y)zETTv|D$m?}KY;@+b{vQ!&J$vG8 z&qN{+lmapLHlWP>=W2ET;tuW(|Mv5*+lRaBNPf9O^{>y6M?dpy|Ne0I?ZfUrS37w8 zw!(b!oSt0tDf)opkxlwacS2*>Tf%p?05^6A1<+f1e%RiZrw^NRgyJLJUk1-Jns{}7 zaqcM?a6YZzaE7}BsTbQ{;g)1u`fq%ZkGV{jm5icWFV0_M@9lPtkqbmo9_;esytM`p znW58qv%32k&ZBL(PUQQciyx?|^v(RhD3K3VO$w9_NGM((@V?JyKkM)R3m515MhMi| z^2#vi+*H`3xHNI(!P3)4mUod!O{pZAHGKu&?v~Pb1DQ4l$rnLvNf^LU7ZTUpyz9ykygL!EVm8nb{{mn@z979Ftt2IgMbAPk*g&(JR>>-|^z2KVr_*8a4WRZgrJ zY49jUo|ad?E^k+PX*7)XFP9Kw$lw(63dSt>Jt2tMozz5GtPKjivc87$;KG5M_4a7i zwY;1!e*HTf*ViiyCU+QM#5Vo-05_)EFEsHdHGrV++}fC>!vFz&P*<28VeSbf(!^hG6cz7p0Uv2SF#Ff7c16x@44$R#g+K51?*TNl~`j7fZ@BQ4Y@7vc&Y$YA>rpFiHl9 zSQ+D&O$#XNX-@NXu2IoWPYZTpDb~V%gH3(0J-sQmCzfJk-HYqRx{}tHoIskw6M9nq z1;&j14QyEQZ}^G^mA|EdG%sSOutkjI1r@a~xFVGTwJ}UYF1FjdeP8Gl(G)x-m%^v( zl6~P*L{s>bTnhiZ-h8_Fz@ZVoi@w~3*c91NN@?PsSl-p1hAXelfdM`D3^!1m%X1SK~b-NunDpL2YZD8LqHw zzrDYd6?{(S@%wCpX#?>3|{35r_#M;stIlZiS1i7Y3wLE8(Zic*;RM*#0psMNqSWRc%(~_98rnK5--M&d(J_) zu&d*5IaDL2W~P#9xfC4w;QEfe?G6zKqA(>2r_=P|q>UIeT1nGXYShnPmxr?((fjt3 zsp|f#ie>9sGl(-0@jBNFYu`!w6wFaGMUzm$V@5wIy{2Rtrf-NGBN_b6jZ(YlVEU?uOi-{HN*_Iq{DN`%cGwZTMfx!)tu#ofiSBvIZY)|##^ zw29(-t7}9gjLfDII+uxbhgxd#;3)dPw-Nt4q4GR*T=b1J>Kz1@l9t?GI@ zD|V-7@F{FN=oP#8v3;elnI&cN+hm~#Llyie()3J21xKZkthZ3uTg+6HnU`pys>)P1 z)gv{EYoEiuu{D!qqdVowB3G8{{Yt!9A2zQZui%CgJ@xDQz5p#}Id4@%_X0pSCUZ#) zVhL)rkxXgtmb+ig4z{1np#^Xk=(@fl1gJLKZ=!v?o?KWBL+1wE0; z;?-&MF!@J(EL@uND(B7$B>T&D<+{gn3Aph>GP#mB-ZErl4EBQR`C z?Bpj?>FKp$`lmHw&SbiS2Ji=SUH^54@VI5ZD?h;ofX8g^6F9BnEEbb{F{d;af(z;(=h{D0v_!HYy-59L}QdLty8f=Cma9FTG?6x+H0Z4&XeLU+dnxg zOASw2@GTnMSHu|KE6=}Uza!?xHi?7qi)uHe_J1E%@S&oV&_tzh(f1!HJ*#WrxH`xY zKz=gI`NJ1xCl}vtZ*9j#MZjXk4vdn;E#&--%?O{Pzlk-nRP_wFel0anxt1}esfRXH zp8gk8c+v%3z7`3HD4Cu)s^rx;u9 zm3(}rzA7D&)iHpzFO*-Wz{#UshBc@5dwW)j4AojMVA2Ep+zLk_JatLnLx%9f5N9- ztiIjzeT!F!3fxiD^#@4eIFx!I6`HhOFnabkDb~idsL9=;6vG3656@A}Ezqw^tNJcK=}9l(HoOc3+EtHPt4d zi`A77sWIErDuq74pJnBa72m1~|WVT1l+lc4E+kx(f0Gnzx!cU~Oe4EO@i zKG)P?V!~ls79yX46|%r=o&;NfwkN&=EBo8c7N=+**0!u=Cu(!H3@5=iIKkf{ZbRe~ zD~YSakAE)niE}L6>H*w6{Flvd)Nd+1Tuyy)yV@`?o0A@7izJ;)?zS+a_xl&$p;u+d z-l-=5GS?B%c|NXIzrq&yZ;n1vudrXFnaUGC@Z$>m0Jw+a+dZ8;g2>X~@2^KhoyF^V zw5ohK@96zPjkr&xRrphyIrkZxNe2RAhrVvF7oKW`Jj%y&_(Z{wOFCWb!`nm~FJDr} zu_us;xnA|>ix93>Ks)lTtt`0Bn$;1t<}D+Gh}?ai`Cs^|73a#lP@wT;t)K3!BJQ zCHj7>%C@2#ST|`^iMQ0U^iX0A9jl5O2)O+A<$7r$f=k9_!=QlUQXy)}i`|JnyEg?V z8j%}2@U^mFdc`OLhB}zW6N`*XpdEp1{*&l1GjZZ0#EK~9NTkc~WlK$#M`msDRn$zV zMm)oJ%y_aU$)*F6ihzz?8D_&vG#t~RqfNTy2%{^Dn8RT853`vo z?(b4?N@1S&tKZD56JYg7DmEb2)asE`UpA{U(dbK(iDnoT2eW_dR_Ziv5^6;KN(Xrw zaUKb<`?!T1TjZZD>wDV8_FpDj6gZ8V!5(Tc#~8}@nrjRD#q)yG9sogq3A+zn>eK34 zyaJq7GBjzBT)o9V+>m36hNV zdL+`nb1{c=920yb$@ans?fTcZJ5DYgCmmk;Q!)A|u;q!>c00`J%b+Ohsdyi_6UCDqO+oqMQped34*iywY1w8MuRFvSy)>PZz9MwF+2Ko_`M-b=8&X}`MKWC^*6 zee3!_2)^hu)@~3%xzq4L>B!>v{t4Usm%H=z?Zzmp-_@U6btRV9btNWMUC9u2S7MF4 zpOWmqKjOpytXKc*`Vd-2d!j}Yi_}YT-LrO>V)MvX_A9M6;jrl|<~EG*q9%)I`&eeQ z{;b3g&=PwFK4W-VeAv|af&}mIhfQQxpsEHhBZz?Nw78w9%IXAl7S`kt`OL5El3-j# zja8`T8_N+<`5emx1lb>h-e}fYMTwi~C_@*0kT$Pe_P{+rI;Qt7a#w8;Y-3j_2Hf?D{b3UyWeCNh3CI=y!`7-aUN_c~XKl@khiio9J?@ zH2W?b*Ro@`ygT08Mr&9&xEPx{ykbzB{Z^u6P4g%U&Y`4Tm>(9ZH$$l*TZcqU_r^rb za-EnoAhW}oK7xR4eFOnZ+gZUPf57*aR+khQ{#^H9&)jcNVh^e-jwTr9k0Mk06I$K30T$9yrm|l4B zFLayUONt6bFb+%{Ct3~d5#zR)Q5Y3%1*B;Qwm*7$-x^FtH_zIHB=qBsgq9&8mRXYa zA^oOw-nKPJ`~mipe}p`vV$HK!QwXHL!4^AgOpy_&dZdJ$?JaP@i2kHBj{=b;j|(>y zpmUcR9nz+Aup&hI5eR{6x&HktnyZr zDv#F9D`4=FNTkm$2$GywDAqNxYkf!e?D%4!nV!Hn@Ktiu&n&TeesKrW4cj}u^}v>J zwr@lE*?M!K|8OkC+x<7+>@v;#{=5EW3%Br3!Xv3H3Z`+HGc*aU!664k*!F;y;=NK3 zFgVnfcVk2Cdutgiw8I-&HIt6?rT;?9^hq{%^Ag%2PsaFa-xM0?f@d4cik&%J4Ism3Az}%- zGu*`J(-GGcFoIX&;25=P`415DJQ`itm>9_sF)VOM?!nzTE1| zDJHOEPQqdAJ_!e3wCuJd|>;7%9z`sCnO zQ7WA$;1(*z+P+}GXWA#=k1d};9qZn}4>)dbiSB3qdu(&v{)SDRFvHAn__VrO!SC~V zwfyzkB4Is#YPRu(F;#rBlrg9^gmV z{!0G{`zF{VqHcBb<&@Z^6af!{`BSP$5GbIC3i)%{?kYMT8WkHFi_g(a1oi6<%sjY-jvEUy;@0z%$?qewV&6chD5jswK1DHBD-_Rm6{cyl9_ zc9K}9%&hSvS{?;0R_4O=6N?ZFgjp3>IhEK+?MPDDt?S5C8i-xhCHbk@9N1iH42nAF zxWTnQ8Lg8_iUc7rffz7I-;Jrz&w69II4s~LwjPT_G2TS~h;9P4reZwS?A$-!gpk%Gq zFEN|@6){FSn}g$~`6lR;LMqJQreiDwUr{X}n)PU*YzegM2}Mhrakxra94atPhf4y& zUh2s*GziPSF-#EllL$^?dp8vJcFQ5#Kj8$>Rpzss_ZVQX!mW{%eW_HL+>9E9Nqy>) z?E4vz+tP=uDUE0|6j&Wq}gMl+fRO&ro0gC0jJ{7Wj%@J{ISpe5m|sXI%0{ zYDd6X3$Lhz*h*@7hLw2%hbo7Z_z! z2!y8}j3pfcl%_+7(sX=qnvT&iO=o0CX9UQ5F)pX+;@6_YXH*MgM8bQf&tX zf;JH-d{{g$?^HpVq`>RYVPL#;Dd2aPqA{iaI2`WPed5QW&@z~&-XaxM$HsH)ORad0 zHDAtU#8&Y5{1wB!@X59kB7sl>fS}=cY8|bmHLSNNalPY7JLTq9`uNlo;LGt&wcK{Q z?T)SI{Xmbn7?X7e&`+x`54ZdR-HRb_?9wi_HbC)pN~5s^lLpGz^hcFXClkWVJe8CF zNr|3o%M&B2e4Q>58@SXXp++oESO@ioW*MrHbiQPhYUv5{RlLNJwqfb}(Q8MteB>M( zhlSkT>6Y#z)y8;wv;N)6^rZoeeYOeT%@6$|r*lPHh3(z?cNdhn3b>yQGn)Frs^Pc4 z)tPD;m#taY$Ml_sSuMUwc)#Vc7H2XP$n~w%TS*3faR0H!;YDM+G6zIGP6G$jpNn8OBG7N6i*sH2G$G$smZ|xXVXE1KH#JgZ!+8mFXCN zuFt|n_rz=LCBhh*gy(jCPE2TOfVBf0Dq080WwKDl*kvR+!hB5?HEJ{-D`PhZZ9fr0ZC8-i8L z15qnCs8Z-i#G~nXnA-2BNSNVyS+<&`px1rGGItcR~aoPsG)TF7pc0ERl|E8Qro-(Ltxu4A~y#|Tno~D zVZ3fX-}D7%)93cJ&JE+6?JmhNzFsy5vIR_a9`OPKdjY2P&GiNGb*kJ>+;@`uMZ;tO`#+bvcI$s0E@Fr)FnRzxgAQ*-Y@ zb%=&a#MJeTy>c*Aq$r7}MC-k`^N3kI!*F@s&db${?!H0E^2>&HCT1;-XGGR08#+TA z>$sYPh%+pk)QLACqk8o9FPmR^6t@YgA?qZc7}QN`ncCi5g~Il7M>5iJ$4J>}BVrxtfjjXnJ=-wEF7a1h1)>iQ*a{e9Tu_+cKCf{66y z4Kd`G0UAMAcjEpIRiOfa5J&C6aDKrwk>LhE-_-s`W#2R@d|=EKaj%#{Hc@l%1X#i1 zDww(f;M?ik09dE!Qn*slC&7u-Crv#HKfr6;qy7fxF9M%+Qp3<rC+jb9j>)mw+>D8$ zVz?{kX6o^%JY8FTHuO$;_oEZkk23fD%La=_2)A_3NMH+@7NAm)Rm1vOU*p{d%XoN+ z@Jt0W6^h~DRH!a@SM71f$tiy{#+&V8k7SW~8P+QWy~ zC+{A~m6o!0Z$;#k9|yTYF1nbc-B%RRd$$-Gz+xUPE$WWXdpRpjkh}JH-Ft-aYdMvt zM-4zU*pBB2vw-J+|9W{TRvBAwb2yXf0B%^=VebP00(N!9g!R#k8Z7qR1{5m#}66^%ZqLksHPn7c|W?X9BUVEsA8PAf> z6Yab=EevZPH3ojH`VsT_`f;Xd^*gt?1A!MYV4&dG$r`>}{+CUYG5xk*x4Z2FV>Go| z;$!zSl%)i*#+eztNUw#waUZjOAJw5`zS9%)aCd|*EWXF?X*tNsJ>(dJ)z0@H9C%BR z3zi#p2fzIV_MHE8<44RE4H408#-cndN(&1Fj2*H4$LPCcSCb}9vBSYPRCr>S%2_CC zK7Qoc(O6sX2wQ*ThrPhwZ~&rn<3IFa^GZVeF!+T01ZBXPAU{ z(h+a4D1eJ`4$I$*(_~t%-V7T>qwQ3E=cOy!ylBo4k%>{shAjijT){Rcj@pb$#zM-D zhvIR+P)t0M8o;5n1L*o+(WQ@pz&n`C!}qd0|9ZRkcou7GT7J2(_M>SXWc-Y;F6&A- zIwWr{agwa<#4t#@b@6G*?!$cjJDkzSs^MVh4DMh&20!uRLKwX~aS6ZQ#`lg|B{IsY zknT%G&vPY{w@hS;R>U;x=$A(>Xrn*+BI#X!G?RSa z(e4lD(wCiBx2|UYUaq5pVBcdNi4$K^fBV0Y!rgC5$jS`q0&;s9O3MAha8)gH(*G=1o#Ev011@>;o z^ktM8P+u|zxuE+RE!LApq8Ko$17%H6@FiH)OS_XzD z8R|xkWWalPa16;TRPil zYX=Phf)8J&_1nXfOZk!cr7W$~63R1^%VZn82q;!K;PsKaLA64pTyPOls>&yn>{1J{ z;DHc8?q!IqNw(0T8}`WI(oEoPsSTw|pY(+|PewLLoNfFFD_LT3$2@Bg30%zK_^1_@ z4U4dY2t|M5>W{P;rHW}E=mm*}quAipVw;D(Ko5t!fmGXI*hxv1+KAxK>)jsv-f#{O z^VTCa)Uob4CV^Gs4cepJnxa!^1S`f23GcK(;easyEAE0c*i>@SYt1 zxxKev?-87ZD2}ZUgr$1PU44{;bc6pLTY=-l8Tkari6llx6=x{ z*lLnEvaO(=4J;8Ye2yh3NRE{$%ocSMUJuLeZBN8|Nb31vbX4%%AnJoxudZx?Iuu6*DHQ=*SgwtQ%C(pDrXk3kHn0^1m}(U+nd9a&#Ny_kQ|Ej$r+ydtTvE8 zsePQR4mDamh^wEloU9i_G3c!ntPI9EZ(g?`*}y(YN2A=Uu@_}JhI?`&v3}7)Cu2D+ zQXx2g?6e65Cj}4*@1=3bEZW$@eVY1XN*G|oUF8vqc)0Iv{yP%)Qd8D}?o`!V86WYFGQ zV}{DKj#htMxx|&b|5W{9b_0_f>%Ef1ib`;kDQha{`k8b+B1Td%1Hw>=aZqAM2_6!e z6V~v*?&X*z79Tm?Qch`7BNt(wYzw16_4pcx&U4HsLI>&1}tnN?R=AMnR5vu?Y!oGP@{~8D;rb zJiANo41j2JWv!YA;QcPxwx3w`%dF%Y4K99)XCeQ7-M;$l*-d)@B4vr@dWv3JdLGKH@g`At($x86+VvzF*RtC^& zR+%%sMu3*{6a9%h<>lV<^tu`5uV2-~s6EAkF~m07>?s7!o-ejlbK{5%zw=9%Vx=9i zdk_t7A|NR$lb9Ex<+-WFm7X>`_S15qkk;vtNiYKh_k`Bp z3vQMa<7WTF8Xn^+1Px1c|8F1ihhJ8I7sd-lE<(j|d?$0Cf}e7ji(0NsLQ07!L?mV0 z>44tO!fcLYGOPY{mmiB6Dlye2K%9SCK4@_6l#v|hK$S2dPUXagzc6+GZ#HGg-ZV~C z6+IuG(~{rC9?@UVKYv|qUThxiqx8rt`W2-mMfWL*#j=?_{*$&){jBuWow!2e1wXI- zNvfnlJ$6DVbBmF?!Lj_CTST=mHp};{onM@bIboHm!}u8e#AL+!WXU`OKt@Inx{CgW zBP-nyv`-@gp)YPIJ5;OzB^|NDLbnJ#!;u0@Vn9KN(i^xOJzhM@b6wwD0L%G?j^-Fi zG)&pVPn3L6Q4zI;%Kk9)xj0Z-gFj$*Vml?gHEh>-c-~%?$;!*N_x#FCl4aNWn2b-? z)*W6H$wKfwHvDKuVj9?GqbQ$LD}(?0gPat{v>(OHMO-sN%G4i$Q%R7g=yme+AHvD{g2D@4)I_FE5-W4bR(~*ilK#?|QXLI-s%p=d;!C@f z(=~5U}GnMBbPwg6w5mrpa&hkHDkegU^RbivnGPb^EY$L19D*$0j^QWHw z>(d1m;9JblNh~H&4q}RQBLp{Wq8@ZgArqg76S|bRs8^k6+;JSBn_L zQOqa(T3%*U+S@(??_sN!Czi#_=;4)wAtc1=5kqw_+b7sXWLt{~@;kXI2VwxWp~3oN z0a|_c^bbx^+&n+?*tvhREwzgzr8O-a9y?l1+M>|sbTNAZ8lp}3lf@JK!b_(_tHM== zSW!4ER7F>609s>`W+tB;O(kkaTqQc?#~AFRw45OczcSBcV8x(fj-J1bDjdZ~@8$74 z-o7COZYjf)@18Gj1a@&qG#-&LJB^h!`~jm`^R-d3P$5HODsKy&j%Yui@#7$lG8qT9 z8&}ZIRZ5JUTI=Zet?dUD-W-)EmHacdREbLx>O2Nl^Kq7EG=(u)$H2wtfI!sL9rBjJ zuV02e2Gy?(!Z%~QXr94JlZ1G!j!OWR*GCHr4eg(w1cquT_f*PY_~5Y|^}_e-$R5Xt zk?~R{>S6Or=^i}XQ8M;tld%jkaZJP)S3Hu|))dY(F@?oM*V;%w6tyINKn3?^s9^jI z72ir~tEYazm!egaBiN&1XJDZgCO?y-z~qya9ojLI>9VwZMU1ji_+S9vz3ez1jaJp9W<$467&L{9u#o;l?;(*m0{y_(S{SB)@7DZ^S>I zAHDhH_fNOkNfjVN=B&Z`y_kvj@GX)A`4~_zyl4~%VWv^ss1(WbtO83ASd}I|Xr|a^3@?n@+rh$%Hf0}hslx_d&L&|(bs153Ca<(KH zZlhGJPgjw6LbDGi#ecHGJe&pLN0ra0O;HVs60qp4>(2UH{(AiU%ZD2r1aI*l_8{?w z_8V;hAFu>yl{GPW6cvH9RYaFG-Cz@Ik{=HH}Hv9|d7G#Wt%pn(c1Qh3!WR9gy7yHnw(*-i< zVt&3Rt3<^iktiG?+zGKCM>Yv0W4mSf9=-S<0#eU+IhF7&i5%yj6>;Xg2G7e8c@` zj9c&oiD6jof7SZqD5BjO7uUXmbDvj(p39mmvl#&q z^LlZPlQ9sojHoav-e$F>@|2Oq|JklLLjG`WqNsJbTOMw(TF-D%s~6|-c=^bKK=}e&!GLsbv>&})e_J1*M>aNz zm+gN&8DRmt-STnqdAXJwli{ENZ!o1lEW|o`iNi9(8Ng3I+yBhd4{Te)i~Zx~N>w$9 zB|MT(=wH6+E2=6%PiP&N8>#wh>loLVHQ24{*aUUAcjj_H((qj6oWy~$1CqcH2578E zIqYD5%Gyygel4KE$cbn6R_Ul@KZj&A## zVE;e(IiSc^zuK6MSnU;J6?8hVD2GYWaKLcIBRu5OS68^JdvhDs6B4Cco228C2?@(J zc&LAOdIu9eJ_DKo4H-amQiJ|7A1i2Q#?yV{5w=fW0q*<@F+}ln&qAMN@-{6dsVR>#cg{gVbH9& zHJWzdou$2mJEl%sUSIPD^sCh!gpGFA9=AN$xmCKkmL0q=>ErVH?drzH2WUp@VwNCE z(UR}**|LI!Z7JSXdmMR=xIBz}XgEAH5y*J6WaOK`4GfWfV;IL#=|UFc#NLDkMN{d! zqtDTo$@9up?fP4(p>dx9`wF{}@$8o~AIq4?VC4rmlke(8W`dXLcP&taKXHx-_wCl)k;rE!yzmNASW(Q`S@au<0pLJ&f7eD zdSAbsaDZWtpJ1Gv2alieV+6&{EesaA2B3b{3b_vrUt+cFyCx9SZrCs}Ti^>!Xem7P z^aKtCu3xt9(0N!a;l7-6HnPx!FFb_~N_5HQP92a)l-lock2_xs)b4wCA@a}WGj53& z-Mr?`7&n3tsqvOq9lHY`ZLK`ijf7EtlO?xv`3nwx`rQHSg~Cuf8EHTTS8dK5y~<4_ zYLBit5{)d@9HZQqdHAFVo~a^iTf+M^STL|J0W))oGX)k4Mh^_7Sg|YW_GclWNh2S0 zT;CQons1|ELB@W48a_4}^H%%53yH#x4^g@mKboxHES9gJdCrrt32`L>8sA*&4zWsP zBhM50B00bgzj~o(w(u?to2525Pke6(!3f-i2Cn!aPY17zpDcg8)VT;BSO7Q29nI5E zj}d;O+y*OCEDH3rF5U|@h;!s`?g{X-PJ0sUyABWS&MM=mhcme@&qtU`379lv=a~hQ zJXK7hV7;%-q#5m_0YNiRjK08U!x|d`3(MXGQ369ioOBH#F;Lor_nS9i6SU?0g-5Gc zjFdl5Cg=Mzekk1rAFK(R?0Rv$@Iw;Y6?jpzp7TzF@bwrTRpJZPtNU_3uXbV={2dl0 z2r$rw#KMR20x=3vpg6%4l& z9GKfMl6{Gj-4JqO)D^vJ>}dfGeHfY5m*EGXJH~DpH+wat6B&o~`AZyy7RJ2yR5MBC_#x+11Jwnlr`v8jo<0;MzM73^I{B zu|7z39%7JT^bn0xYy>*~O5O%t!1Ky#syvishdT*iF@x)#V)qr|W1zr&JRv$KlBU$EK0dk9g7_KsYL;&uqej}!~w4;VDmmik3=S7=IN=Ni^&jDRDkwj$xmO(0mM3PiW0)@#aR6> zH&~zPhI=)M?2*NAhgaU7&EH@{3tRc!1zxi`#$(D(F}{0s50MNh{NsXk3Dnah5$8wJ z8nF2QC;IB6p-=K2;xcitCl$c;bvkT4uP|Ku9o`}c^PTXq;KkKdNGY00bx0x!1BB!? zn73F#dw2Lz2kz3fQ;}GPvw<De&fP;{2!qEo}4>@e5o4ewD6G9cK$dIvdMBp>>Z>tXgDXYCZh}cK`dD~k%TkFn z5F!(28d+U&1R>1b^#zmg(3osJnV!DHd~(6SQUlJ&nerjkiQ?u_IoyqAERo~T1IMT^ zi9cQNCNG%@c^}Q?S=H=W@OV(p3SWX2AbND*6Oec(f$th2H|TFYxxkN1Qd6YH3(kNn9~?r_llj0h$g4@GGxzro7$3T~X{?9Oa5)$**cF^-cmo zxslk(kSPUlc|L_- z>lnN5+i_Gz8fv~eL5M+?RX3h4yQu^2THf2VZ%&a1VF;Dp8G&)q2{L!meoA$t4tDJT zYGhm&b5@DL(g)y|{_`LszN`+IB%!>}a1z9L*xWxH~d5t#h5vQo3wje(4a_lENXz*A#u@ys~)e!nY%!AIvX(&QB9ej}~Wa1G87qTPR za+2tFooME6kvN40k_gP3=7d=?_<{oN#iT9`Z1^zcO)sRUZE>!qBAVF zV;tcS(J_6Gqi;Je;QMg*)(xLd8qujbvWcCu1ZU-fzo0>va114`o;5!a>^^VkZ|JD8 z^KxbE|46_hMPqktAM%Qw2+gx~xvQ>u?(U0jP933pLgR4zXCu*BblqXqW*r_UsMG6; z+6*uQw)LnJ)WHEqor5n;dWghzq$%3imR=XA60JqZDTuW7NBzT!((t3;& z?~uYAhr{-@a$;JT&MGXZXo}cMUW_B7xLG*k0AutlI9q#Z=6pF_$b`%q;!JT4$1?Y? zLK{(T3S3F&6}V0pVM3lXxsx0T<7RXt)z^|!9jJO`Qyz@$Hn(5FJz#F$y+EYFO+$w~ z0Bwv2=n@~kL>H5P*Rt~gjg@^nmRW=|dh0g0pb~frfYyz!-Nn2p;&3QuYMekfE-fVe zT^??xFU1j=43OBxUvO27UsssSUr(2dxddf^17`b{1jZvysIPDd$E0sQ1lZ{HSif%` z61VUvnwLc#5`@DyYr*JZ{Zbl0d`dEOq(>7XY+LiCfAK(SlJeCz=e>L$008A9vz-5G zI8JBd&q?>JNFP-;R69wzqha5RxZ4%4|0%cLQ{X7Du|@<;dVe82!C!@6By?|MBQXAz zuz|5gI%tPZY;!CPcci~c114|J5sKv@k)yd2Ab%5M(d+Pth~+XnVlCP~9%EfC;8E)A=fk{nWJsOjFDgyp$_VJbfk+1rsw&su0=7 zg9YIkoV&(cnqF6CcZXN@==b%EM{i#$#YGJQL^g%r8$jS=H-d=~V8C?16}HddG;Em^ zbW4Pk0JFwWZu2zfbdc3qBRM2e!xUpW=bT5}H^!0k4i0>I$!v%-3R@;Q5DaBZSO;@4 zOOBB!Y4b$Kc>xZ9t{|CM zIXICsYvJzuo|F6Sn-9uO^=%?HcQ%RGojGz#$j;_&>}7YAO@+tk%*@4LLlkqTNJ4in znXEy-VX*zp#S|H77XmY~pTMVLHzRWC+~^MyVqa!C1htqt+{xhXeOw;I!#m)8rQziU5ipO!xFFePe>;^km83jkZ7z72>FA!1z z=}+uK!mDnHFLdyI;EYgK=0ZT{VOX8_H|&F$wYl9hNG)LTNB+jmh-uB-FcIQWWHSSR zTfc)jhu7biV0eWYDE0>FjLeL9j4mq3!cZ}_wV%-onJsZMh%3UhpD`DLp$m*I1~krM z-%MV(K@j+jTAWTT23GFS3qpbG0iVyan>FlRo3x=y<{$jBYz zlW8me3vs|`dHm8fhKpRhM&mJAo+BRD#TftNP-a7LQZ_fHDqPQU5O8xtjwTrB0V2ok zVRI~BBAFtiM_)OHGkkn@R=s6W5Q*QdRr*4SEOL$dF`IyvLW4H|df0#?Lz@~WI`D0UKr!9_MfBGrX_ zBGUIujI40YrneQ>AIAPYba4*62d&7Mzy%|43GA*#Xx$-?XtKT4H85*MN>1284 z$LjCg0y3x-5$TJP0K(Jg z0_7O%nU}hJ#e+<+w#cjpNBN6Rk~`LCscfg)17tlh(72f**0nHvVZg%P0lqmv6*0qw zAuD_prrpLDOvlPG++3y7)7@aimwR~3^~;-Rsjf9jrQ@(*D0N%kHuoI-3$m6j@(rJK z(quXMAx0e;I4_pdJ$S(B%`j7va*W6)qApBhR?riY*Vv3krI_ZcGdwklhKhb}=j!7K z?_8AjQ_tdxt?uG8XCVnAmC-#);)l@GU-Z1_lQ{cGf5{iCB;6;gR~+xqcmxuh{9xEX z#(E^V+bHi_{ZLf+NE8I+OHJ~xYcVb~BH;!}Yt%ho<|87;B$ljo-$U}9*qJQ)o>BBI zBP>pPhV&Cf`gtO3qyg^y{1=O=Ovq;@=;o`lQ}`!M)QbOMCe+PCM^sr$!mzQyU};st@~pR_wc&%{@Ft-iU-26BBC*%(gnrBiT7 zkX9gW!cX-l_pn^0*HCD5Oe`S0TdCQn{|Y4Q_ZHa7 zm}9>YVe8ipQLD~>!ni9tKT!4|s-53SKMRNAZFA~2k|Lo5A=0k7mYA5F3E z7ha3`zTlx5zqze3$bumBOP z3}x8B;J44nkvWK!E_`xO!cYv8E5uKLN)&=l^?-+Cpj`I3bZ2eniYzm_0a6!CeBg(N zA?7ePzM*nioyC%89(jrk_09*ZJb7x2JNJh%Pd2ODa2dVG?Kf4!NrE?JM#bS_5)bk`$ zw88De&IN@(G_BLU@gcUiDW)66&@C+l>^S#9(t&!UlRUIlrXae{n9&{aYZa`pe^6dH z^IFmYNCFny9C2t1rgrJ$cegQM#m%~Oz4*uVRGc${?dsTlARlY6hs23+ZmTD-gux2H zzmM~~-Fle&L9_V`iIWW$^(5qMmxA^0TqoygzRHG0g8}5lkp+^g}_YCU{ z`{z^UGainXuNe7bhr3IQt~v(Pq6;^*cgm2*d$cg^85QrvEUdlz*vN#D5$`H!PN0}C zM#@50)v1QF>Jtgi>5;hQ!@v`k4gx%+X*^^?rk_+1!+o7%JFBdKj3^4l)-AVqEBht7 zJMyxPysD$`vFPeLb~&eA3P{AhFAIWPcEJae?rvfa=P6wOb+64}^ABx{rE-loxf>`c zI66kcp;8DV6`U*Vx6BL2lZ&zt>ndd-;;yoAiqNvC^FPn+oq?#~N3RfR6`bd@SCzu` zi?R@V(96R065{TIh|UG(1*)Np3x&{7QAiEtg`%OnP&AYmiiYw+(NJC}8p;brLwTWS zs3@d{@r_9!EK-jvi?w5AiE^wg(T$ZQsx`CSXrVL zR~Bo<$`Ykm8FeD>zrvZCA66Gm=jyf#Y~~QXZLCLtHh>Gd-ONR8Z|0)@H*?VEGm=RxoH?3xLv0d1IhA){|3gk|u zP_0!bz)V#F?NcL&VQK`iNR1$-s1d{lHG(L=Mi8yn2%_#PfqJeHM8P#ep;>N3jjBki z%ZR0>ONLD21w*d+oT0El&QMq)XDBR^GZdD|843&K427j~hQeY6LvFd8p|D`ikXzCg zj$5`Y$%QqmEU7(fESW_sEV)f9EV)%HEV*4PEV*SXEV*qfEV*?nEV+GaESZHXEV+#< zEV-3^!YHz{ys6KuMPX6L4MJ+iK*+4;3AxQYA-9w#TKfTIj@n<6@H%88%)O(vs<+rJ{K$}W4w5ha3n@WT9 zP+^rem8NM^Bl~E^@=)JhF^RY=wnofKkd#NdPs$^$C*_g8 zlk!N@NqMB>q&(7YS|0S8lt&s(%2k)##%560*SE3S$5pVR)bv@WxYMt)noOrP8P#fy zqI#`SRI@dT>b6Ev?bax&-x@_VT&0MPYZTRTjZ*5Fi#XUtlB!t?D_Zu1T*;0gxQ*{@ zgi^&uC^c+^Qo%+j^=pJuy+$atYlKp{j!@{<2&HO`P-s>)>bCGJ0W(j9yqIqZfwB=!JbUdSRxVo?9!U7e>qIb=xu7YilUV??++aI}jVz zOBibAiy5l+ix}z#j2P+`j2P-Bj2P-Rj2P-hj2P-xj2P->j2WtSj2P;Mj2LK1Ir)Nz z>CG2B*$w~4YNMs?xe~@el&3IaKXK0cz2trmZSf9E$yohNc=!@Gw+2{s2Xn_ej7zW* zx35_rIB}Tda=BRAlk{)`J;OK&qhXxH(=bi~YZxaHHjI-{8^%e@4dW#E266(0!#D}Y zVVpTGoi!bNdZ3oFG!ay=)!~z~HL#PhwSbecwNR6>wIGwRweXU$wZM|GwUCmrwP2F7 zH87H~wE&W_mFVC$S;s`5mB0+n2YV{8!{T7heXKD8Q&Itzg4{2ifg_X7L6MPjFl6K$ z1Q|Iuen!rXo{@87XXM<-IXN?KM$V0zk%KYaJ|JgE=PZiy$WAqyGg5`-EmYEic}iNa zO-T!eDQUqfB`uhwqy>AFv|x-1&0C_R1v8YiV1q{sr+pdvW$Ti>-iX&Dx&9S(t|6?E zr3SJ);|*i?_8i0>OgxA^SbY$C0Ky>lfQ>=y0WO2s1A+#z2T%=T_c$BG9-udfJ)jY; z!Zsy7y5QTv5&CMR2yG3rLsy;b(NrY|^i;_KEmd+rN0l7VP$dWJSINQJRdTRyo$Rey zB?s$O$-!D~zos5WYU6BL7+I(wINRg|Z;986jvqBLrRN z1h3_c5cHc7BF!e-gj4-iNU@gdWUuQgIcU5_j`Ut5N7}EEBOBDnktJ&6$R0IvWR)5@ zvQ3p7EL0;$cB+wSExs=@8RNv`_>8hzp;)LvSXZl&bZk|E_NWqQi7J6Ms1m6CDuH^h z5~%Sifx4~}pyeuo`mGW~vrsG}5bh)2d|UByJT%kHtRqSWiWMp2blD5mj} z;uKy|oW4tnQ+G*m+Ab+h*(Js4x}-Q&S16|GlHwFyQbf-@)(4|2-&nvY#^FM>nik|7 z)M3mas*4LmV{w5fDlQQH#08?3xInZK7l;y)0zP|OAdegu zyE@7rcXgCM@9HRj;MGz7#H*wHkyo;zK5b$C(5s{K?5S5r`D3q+_}uau9nT_^SEuYe z66;X@?Ui9%qNhRiiT(z1La)O(sqbN&w8Jn?+GiLi?KX^)_8i7ZY!Bn4{ReUa7sEJ- zmtmYGj^Isrf($?KgSRyC`J1$&oLoUpg+;zz!P*S0B};*of~5pW!BV25U@2iyu#|Wx zSV|xiEF}U8meTYkOJVbZr8IWIQd-&E#J--+s^3h{EL~%%Si8ZNTfELzSiQU>TEE#5z`iE zV5ce2LQzYgiKmu86Im^RCdOI}_h7Wj-viMqe-A>d{5=4z^7r7g z%HIReI)59TRsJ4uR{0g1atc*=@!uS$FEEs)Tn44QQ1Fr!Dk!o-wR~2n=FJM#hFPH+ zDl1fLWQD5oyioL-6{_;G!bn4$?xS}K1AoVQv1|6Oh#s^d^hgnM9qB->qsN!)=*i_e zdT6#7;N7d`_lyV(Cpj=1KrZ24ig-2}2vaoxOygYv93e7)%Nej9tXpvqDTBM_b z7U`>?MY=0!ksb?Lq|<^H>9?c>T^F=S?*%Qgft@O22-2%q-@e3a*_8y{T1li)D~a@I zC6V^5B+{9cM4GaaNIzB*X~mTUU06w^0W0Bi^TTmAw>o-&OeV%b9!gs;7ELrs>!uWR z(~zQ)8YF6~L896kBCF@{zQyKAe81hPbi7XV@j&= zh>|HhqGbAxD4Dt=N~Z0Ik|{f)WV((hnW|$-s_BT5DLSI4o?lPyPp}jH0S>+457VZ< zpaR{6=*%uuO?IK0u?tm(UFh@Mg+8)f=u_H-KA5|ZXR!-yzFkO*`SCS5_QfVk{!l{T zqfOK2>ZRjR`{;c79=a%`hb}tmp^Msj=%UFUx+u4YF8c1FiwXMZv`P54W(^t8_|5xjeUf)>(P*a@jW%l0XrC61wrSC5mllmSY0+qpCJnY|(P)Pj zO>BU}LZ>*4%N8&gW8uR`Sj525Lwb-=JNXt|sX&H6mPU5u`l*U7HG*gozdg`OD+qH`B+Oot^n@WAQsnl$n zN*%YU)OMRny|<~fKo1qRXj5sGHdX8+2gw~FZ;0dT_{{^Hi-ujbV>@B4B&c;7Qqq~# zE7v5^ubV_MO{-e5Op7RnX%WRPEuxsEMHH*Fh+>o`kv3@&#Uw3au!zlzo=@=t#>JXj zgz%1qR$3!PR%wvYY2>bTvbRi?9Bfl12kTVH!9G=Tuuzp8Y*Zx&D^$(_&Xeefq5Vb6M-Q>j0FZ^F%lRC$4FopA|ruepo|2D;W82!2F*xd7&;?? zVE~N<24OT37zWcwphT*L_{9;}sh4bURj}FS%t6`YEYR5EEMeHbJED_h@ zETPuoEHT#NEWy>}EKt?rEaBARtl)^>@s7`+$^ev+Yse{C3dj^JC025l3P^I63OaI@ z3M_J#3LbKn3J`LZ3IcMLis=iM(&jl!#n?GZ!^-ZZ*({Peptz&8>>JUm#trnkWj(!N zR!?u()YBUV_4I}{J-uN{PjA@K(;G$%^tuH-yIY}iPN4Ff5$VI3tlOk>2lU6j}`iV{nUJX$cS#Iniq!SsdMBW>sq>X6Y48;|sq>Xcsq^4mE*El z$60xUKj=5%kF=YVN4ibQBh4n|kzSMXNUKSCq|>B4(r8*9^qG`L+DyuWF1g21f@`b- z=r{c2TW#vt>cmL9O$Mjt7K7J!gCQus!4NFaUebAUyromsXuMYSN65ORy;#Ka;QF||la%q)@;GmE6e%pxf&Np%gaBsi{#iYG##`nwur1=5|S`xnWXj zZkd#tno|GT5Uxd#*{4mXO$GIa@%)=eqcq#;Fv zG)S~YgG5s_NVG$PL?bjvv_OMI^|wgSeS<{BH%O*+cWf2Eh>PjX@nXqu-^!<(A=sum z8}xFBDf&3nB0U^tm>v$ZPY;KgsfWX?)x%*%>)|ll^>CO8`#98+Jsf7x9*)?qG@}K6 z-{Oe&#S`1O#S>e(#S`1P#S>e)#S`1Q#S>e* z$rIVT#S>e+#bY+tSLh`W)T7();=^=9_o7gK>9Uzihdb3Dj{f#0eA~uSYDA_1dB4?vWiKdMrCBim}l&IS% zQX+A)NP*6cA|+xsij*j}0noe+DnPQ!eUdkIK|P!m?Dli#uqV*_)A$TEkf?4~_y`IY)`7 zjH7~~jH7~^jH7~;jH7~&jH7~yjH7~sjH6=voTIdN#!<0x#!=c;eMd+qahHl4E3=WY zkF{d)4eYu3``HUD^s<-0=w&Za(#u{#rkB0MPcM53re5|ES-tEf%=+02%=NOD0PJNS zf<`Rsn4^k6)}htn2sN22mN%(N4Z>(kax+{Tk^_)immC7(y5tZ%*CmHQyDm8d<#ov+ zfUipq!G2wGNH=Im4(Jhe$swJiE;*o|@Qk3z_+ZYt&?dJX^A70~8UMzfkn(TFf5JZi z^$Gs~tS9^f5T5W4z;(hu0L=;i01PMm1CX2YZ^moFKLDi(Ke2gujPERReRr{RM`YB> z1Zx#ul&|ajz)p>y7^#&L3$=3EyjD)z*2-zaS~+c1E2mAW<d$~A7@_XFjoP-ulb=ymaj{E4;bq`&1-a{AN_t3=#J#?`}4_$21Ll@ih(8Wf5 zblR$iE;j3-*KB7W^;Vm$>Nc$KRBc)3D{WfktJt>2SF>@AuV(8SU(MzsN?rC4$9LF;|QaRtRNf8M)JKFipB*q`waJtDAU; zjSW14iw!)2i48o0hYdV}g$+D{gAF`_fek!j|4lr!{RSSf`v#uU=69BpG0uvfj%Kj> z-!71shKEmTsc=;6Ugv^m*j!g(`&wP4^=n+E{cBt$0%}|(25MX-3Tj*>4r*K_5~^GU z7HV828fsi69_~(8qZjznR%SR?T(Nva%FP`}g^fL_G^{6;*7T&(gq~E|&67%Fc~WU1 zPb$q4NQEstsWgBmm0G_yuPNP1D>@I9T;-lpXxvdsg*!^AZ$~NB?I@+T9i>#Zqm;UK zlv356QfS&yN<}+Lsb^bkQ>T@Pw1iyREFqUB zOUR|g5^`y+#zZafbLUB!+}RtrkVv?jD;;5jHOlUj1~LV z87r2qGgfR~XRKJi&RD@gow0(5I%5SJ4aO2Bb;b%_>Wnqy7)Ubz%nuxMQ@(!9E(cPe zm5}hX3JM&pW_U-d+1k-+=5@51MIEhXL`SRH&CzP6^0bO|9Ia*$M=NdcWOB+>E+zxU zli8x6SNd#?C$~+7r?64UQ`)NJDQ#Brl(s8*N*k6ur7cUI(xxR(Y1;}Nz=FCyhC+)p%PltMWEbR^x4ftj62I*ptcG0&8F!F;?MgVyw_*BO)3qB z7K9u_6F0`t28%JY5n>E&SQtYa3&zj}fHAbKZVYWxJBFrxjiGH+V<;{8Y(D+l#e^x6 zY~mCM6$>WB+h+@GOQEb;Diq%>~ zu~&;ImTD2jMol8E(;|vpTExsE+_fo(0byTQ5DBHZEfPv%h?tUEA);g^h$xx~eOqGW21D4EV9N~Z9bl4?7mWU7uR)blf0;eEQmx4B>Ai~zoZtf$Mg2%_yC;<{O- ze4nc+NwqhL)O?djNoAlSo6fiLgbJNOLrau|@8^T`xz+>;2Vga`s|& z`EarxVeko0)1W8zY4Zd#HF?lOTtiztvC&#QvF%zsu?br|u_aqPu|Zorv0Ym{v3Z+3 zk(FCKv9Vh`vCZ#K=VSS9ksj9N^`Dg|HZEzwwgoM+X--=v)vbk@o~1BXuoUK+mBL)5Qkd&f3Ueh& zVXi$X%+;oanZBejSCkay8gj24C0cTYg{C4xs;xlCH0BAp);uBCoG0Yk^MqW3o{($N z6LL*@Laxm~$TaE+xmG=)(5$|dm?+l2bLbp00qhD3ZAXMu+kud2+Y@qadqS>lPsp|H z3AwgCA=kDikN_Rs|>LP zstmCustmD3stmDZstmD(stmEEstmEk>I{+PstmCOs|*!OYR{p+^1a7I1EpregjyId zrk2)=s1?&i)Qa6AYQ<;~wPLY|S}|8dt=KA}RtyzWODjdxiisks+UM1L@x~_5ZeEcW znMEqaG(}k~HYkc!`9-m6y(m_77saaQqF5DN6su;7VpVBbEV?X;Rf$D$(B53n!-%yP z7e(63ik#Qi+(H zoWUd`=h#Te1t3y#fr6A=m_8*J_D;!#kyCPE*_2$EGb86VOv#1eQgUG}7r%3cWTAa( zvkF6LxjIv3zbaF1#Trv#%NkQ*(Hc`>*BVn{-5OJ2;~G<8=^9gE?Aw9WnMNh7q(39)- z^W?hmJh^T;Pp+FSkgGQHy#Asxue7+#X(}r7>WRukCo2>@x>#Y68;8Y&1;1*sYN$nmq8cRXt43y6Ysbb(aLjGWf}qNA=vDoKNOBf#7&L zL8^e#y)3~Ro7l`U8`zwc`q{k2`q{ko`q{iC``Nrz``NsO``Ns;``NtZH?TPr^s{+H z^s^Bi&qqrh-gIOqm2d;fUq-WwNs}N*Xi0&%?vaGmo07D3lSm6UiL`5zNNYBUv|*D- z%QcC#SDOecHHoxMlh|hwx5U$wb_~51ikRA#3QRo~^Gtn~^Gtmf^h|x0^h|vg^-O)1 z^-O&h_Dp@24op23_e_13_e^~VkS`23&9C1dK7QGTm*Z#~-Z*-!t{i<4qa`YM4IC`wB9DSx$j>N7P=d-CS&Ds}DaNt`oYlWEh=sH7W)G9-4 z(;7o!(i%fz(HcWy&>BNx&l*Ew&Kg5v%^E{u%ql}{%Nj#s${Is#NiN04pCL^F>Tn2?>FuMz&@OAy=$?M+sa5~Eeg!E7~hWVi}BHeH1r8?Qo+%~v7E2CR@{ z6IRHv5i8`_j5Ts($O<_&WrZ9Y^Kf!D!Wn7Df0-;-(?y8Yvhu`EB`p}KphXtSX|Z{7 zT5Ow~78@p~#a78_u}N}TY>%848>65_mdI(b8FE^W4P2n&yGaiV_{|JF|RW8*j5>O466)1R#k=`lNv+Yp32Z;Ol3$cd3p9?F+1d` zqDy?AL(@MOJZ#dit+Gl&g|>-WI7r94B%xxy!J6Uf8p`d$1HfPao(R!W&wgx6+%dm zS0M(#mBa{V1u=$KL5xvV5MwYE#27;bF@{b-j1f~1W1y782prV#-^ja*%cn0# z70HPBczN${7o*wJMOnpz$!b-Xca$Ds_fU@(b9=CR^A)`5!I`SleUj_J#t;L6Yai}lOJ za@9S4Ge=0=9UL#_&z7_9!SopRoA9Sult1mv0weApA5YHLqxtcqyUXS+&S|mW1kf5B zM!n=WhQ;g{EwD21U^G9y7@bbKr=!zV=i6y7ZE@|uW?Jqwt|yZd00(zj;WoC!aEYVU zqf1ITc6n9*;dFjyHbra95go3VNJ(e%7{hQ_^7Oy>feX6@t8El_vM_q%z*@)K+;DC# zCAnp&8%~)Ga-`!1N2K}&hcjP;Y*uYhGj%q&+U95o)NINbc2F~LHeWg9oQy7JYd5>r za{*x}EyZ`S_04i$GCF<*@c8fb0HgDDcd&Xm;t7?L=hN{7HsoUU3f)3hrf_FLI#Xi0 z5RtG0-_yl%3YQ{nn$5fy!dB`jn;91!u1D(&oT82d0NQX4|Asy??PvzR2-YF9zz6jL(uYB;RSPrUZGg3%vLU)-Inj+fK(HK+9@ z)jQ3Sq+ZpPKf@a4vQeF*6_q)HR%IC@ZDOF;*4Bs;R~8j(CHkOd+NX=fY!z2j zQhjA5Nh^FdM{aI$I)}3#ms%{TzQU5E6+WNNpFDlcj({&p$Bjz}zM7=eo{>@WXy$q| zw_VMm4iJ2^<9PmZv_3eou{+{4*D1Z$dXE!c^ZO6lU!IJnD}7J>@%+x^F}(5NtLeF8 zJVxIWMtx^O!!b{k;dM4!ya^$kHEPv894%j=JF3L<(d^a1`~*P~Vhwadf)>yCbhJF3 zz^^S9>+arc!YDzU<-NB^nXV)c2QT#+-B`ZF%?8K!#=YQ`;D+I%B~H9Qn$CbRuLHI( zxvP*VdXxcf;LowtvQ9xmn!?2lhhaU_uK zlfRD)u*I?uFbaX`(PaJk7dD{4_dUjE=C$hYCOG(Zj%oARIib_tMm)?BM4t>DPc}~I zDh{8E6K%v|E~e-Z4@=m$oTzI(<^Un%2ULg{X|-7HFEJLb5$++5>>f?tH~>AKjonY@ z>>?`1G&bhj<>UlIr(QUI=f#)bEwIkS0n7rXvDL(;@-ZGG4xP?NGZ_>mK4fd1l@XS? z8llJO{9t}OyI3JArCLXC!zCW~8%zdX zPG&O=I(k}wq4i|?7InE9E_$VfgwACK8r_?MG}EW61T?lnVZGH!SeNbf~*)QPZv)n z$aNsrvFh0SYzj@z+c*w>j_z($-+`nWAzIL4453_mZClPymSf3EWogmDe9iT9no(}` zXkngZ|22k50bnk`2NW3yHflqUSES z%*}1&^NK_AG`gWL?kl8qV>O8K00WazbmWB`9TpesC>%tOot|JW?;R_lYoYNNA-+*# z=K=Qw-!mCoJkNj>Q8QF;`H( zetTjMsIwqH(o(~m$V`?t6y`nEM~j*cMF%*d!Mwt)JJ=9lO>$4TOdlFe0{u{6ZVtz^`7Jg?d5ngMA@%QKifHP6e>OU;Ga2U6 znRuJs1x45zfS(eJ`D6-WkOvE-``u6*=8UT&c`-pgof?%^)=e;l$6(#b)EZCzXrZZN z<}JA5bbGR&ZA$Rxr|>menW9Swae$FI9ABp|*EyJBBAvcf`TXF%F9D8{L`dK@3a8k5 z*vH&}m0h|3)v3yu)oX>qm0L5yW4;U%#%C+2)l%t4u9*mzRu&41M|4a6yhx+Ln1cOs zvOetBZCyt|2I_n=zKv~jNHre->j!h(*0cE`FpOx=r?c5@M4hjk4lzmle!SMsgs+}* z`Zl|~KRsXhCs}p&tkeerY*|^zCJIg)bt>}H6Qp3#iswwMtfUuw|M0=nC;N{M?>~O> zaR2GS<3~sPnAVwK#1Ia5Jb)d|_~FRcNU4EvVIqkQxU|9Iu&QlFVPFGPuhn7{=dAdS(*? z`T$mqs`p|&?kQ~*xyfg<6}K!5EEQIjkDvLc3$ut-?e5|xrs)2R&o8*xQ>oF7uJ_*Z zuu5xU?bePF1J3m9;;fH#z=L=|xq2iRd9?7)R%?P6lDPq13O~N!G{a4-N_BYwuPOAp z-ly~uSODAR2GS$85wJ#M9sqKKHs|B4e zvm{BV1RxUlBqa)yq$u1s>a=?Is6ZOR$9Qxh`-u4SJFZo8wMeyw&M3OQu z>r4I3kVn8<1X#s!QGz|R*6&nscX2HHqdMASe?E440G>`KZ<*2;_b6s(a3*H6O$kw> zq7vVv`_tuW&1@aR9=OWavT%i=TqC!6|WcKCLExSpKzW=}B#^9B2Fr`%`24H&1Gq(L!W z<ULQ{ z_c#2UmA2cpUQW(rZq4_2W;bv~)o6b{U%)lNo~s+OB}IN~kh%43o*G1gAaVXyGShBt zNOq-ar__}PO?2VCdFH#3?!dYbUYQ0kb&^oAQB8AC!5wb<01Jd}?)8+*)dOouSNlSJ zTZtP2d>;zcINZs)EgwFCrfOVkgQI?+ zD=yJQ);kwu*f=!UP|D;ht_+vn<72(smiRG@A1@cH#mV}{^T~@F-=Bpk>x~0kse_X2 zmcPa7hW}~4|Mvjs#ccA&?%|;<)LC-r*&16suqRJ9xp^V1WnxR|e9V@?HKMb02Udi1 zo^G{M4tMFuPSRHOO$e@D*0t;s+xKSYzP{zFG4=;M#mdSIQ48m;eRRL{X7a*sYYwh@ zdpPF=;9uaDkx!9lV!CSoe2V=XzCd?Fa(uCoMPo%PG%>1$0Y&_S@FU4LvAFL z7)p%qF0E$2NbWcHT+%==x->{;+1SxYBYIR+3%P!v8v`TGP^q&=M0EHAeC8``@*+<7 zZ%7(?NU|m){(toZeZ)mVrdk#yk6zmY zYl>31n4lV06z){%+T(NFDF9$ax0~NgXKm$jEvG)NNnr!0tI-%ab+DhP0ef!F#Au7T zxmg-gkjwNZik6U9>v~^OrWuqbeKm4@qPLy(YE4LKncgVVe&`}S3kfqZgS%h^O zlp*fBmv~TY$TLk!nXD~%Nqt%&I7v3Da~JNezQCdJ0O;U2f9boI>jy63d(6H6emTnD z(LJE21b$y6?Q!+_Cb$A(8T}z#+7`F6ByNCa)1A9;!j?}C<5os+{9I!wp4ikH9w36G z52sk>wCSGR6T8epB?H_UwsTzQrbP2Nm?u@-8*U`q=_{lIM?((`8ynoRiw3TkNxYse z7jtg<{L2V0Twt-wG+m)QUyiUMfE`c>s33q{bq)uS3$hsvDvAmpzu+E%sKB)7w^>9a z?zy^TL!vVGC{fYV(dt!nX*P{Yh$r7Gqw+|N8smn?;RFwZiGMuFh97Y67-A?mEP0~< zhFvb+Vuug5pL7qA=boOAmTO^<_Xb4Wp$?}a4y#To3~pWl8c;K*jIhIe@u9}TR(S6< zbPl9S9Ci5gkgKW2h?OZe)!*u%HrP-f1Qz$s(fQigeo&OZ=D0IiyOl`PdIHf(QJB$I zh)51V$n1>!X)Gnnt_V`(1yv*ZzV3RABw3vQk|6Vp!}9+-JvRv~rTv#IPPuDw@&XIuNLUAwY(ti^GTF!K==poIQ0I0Xm7vOWSj2o%8R-i>1ggF zJikF!x~B!G^yBQUcIk$ZsK~X9@$g#q)_ZmMaJ`Qvz88S^%#r179IA**;9P>poRB(7 zw!I?irQC2H94C}Y2BA_sGvKF#WC~M$^W{9{w^lapJNvivcHI*@e0+N$t>%@ZOk@vbKzJ~}n}r=5Vl$ZE z3m52kw*c`OGB=3sF&<8qMA7;AUA(k5okzwoZ((m}xX_0_D`INh!?E97vZxWRFR^?1ID)=liL-=kZ0O>=3Oz&EXC7W!kcZ>@$w-+v z?Q{n=TFJHty#I_hA+)1dkMNAXhd4j)CTi*m55_hLL;8%V_iz<8#VH88`u#q5H_bPt zMGx+#jmZ7IMBoKOx@7rMfB9E8LeLX{&P68o<{C93se9%GW;6R~Pvl)-khRq&ozxkd zGpb#>sR)W?0`B~$4heBDF#+$bsT~gmb7sbqkkEJIF(>Lb#vOs4D&pHXuohpMAOKL} zorDR<%jvQ4t=!(uisG8^lc?*$rGsi1!z_7a1C-`&?7|Ia>y@LrS7q(mKZtqV=!smD zqnv^QE$~E82}4g*=xcX~b-u$itHW0!uzu-0t@tMTZdY)Ju6vQF6}gDX=q##0lcG_9 zHNmYf39mkfmrI;3xf^Ubb=6qsb(i1UnnKiq-y^uf+8mDx z(0A%&&BER<`s06p_4%z^`=8yu{|R#WWsn8GSmxmVZlUR}j|bsg{3mAqHi@?Krddv!hU)fK&0*YsXp)q8ba?;T@IsOY4w z?Y+9X_v-q-S=aYX>YM8pZhgZov0FIRv)s|ysG^hkc)*q}>Bc8SAY-z4XJs1*Fqp}g zwz#rxsgbZeQz0HcVNl>!{Jn=jk=rs5FWhsTTKH>3KaQ}%i~*6B?AgGO>_*$Lj1-C> z$>nhuMB>t1Ra8PJ&aDVerK0ER`Vd=c1dMgEn?hFbZBvBH*SbsG1Mk?$0z$O>Bw7y> zb8BHJ@Qvs8V(bEnMKLb;Rwjh{UCg!^osn?1-2=M8?BK+D)9 zmlAM9s9JYDO$vd}DgrmLRaa)OPbQerVsEDd3QZ{SMbXkdG!?I3>6!hEidKIdA06GS zJkw34`?Ke~RcXboTi7n)_F@3gY5KUR;_FJSWf47ccJD9CA}A5Nl#X;}?Xv-e>L6)B z^Mnuk8T^HQnH-}zUwc7dE%o>na9@=u2hSXf@4>Qp+iq3>@Xx)K;sId>m-VErBtOXZ zq5h2I?X-8R`#qqDgfL#a!|nok9H2%vIQTi6kGm@k6~CTo_7)1iTVRnBtza-L@6ty3 zP&W;gW<5QW9#km#R%bcN)#smupB24~%2s_<7ICwYy#1wNf5;J(tS22a#gY=n-oR@!p2(qnE_diX79CfwOigsc;0 zc@*?oSXL$zk~YFcG?8;XylnSP?Rj`tff)G8XEh+NNeOOw$g|PEKg2Nt$QUamr9m|W zpv&iKy7@w$$!2%hKEu1B37BkgpKrY{eTi4lvG%}yIY_SRon>mvbeXR8NPgId57sbR z!KW39rtDQrv-f%Tmnrs9;#d~h8hQ$M+IEAQ+Nmy3dY@w{_VVtP;iOV@lg{Y((Wmf4 zVgJm+T(x^pZC-Y>;!1A9RDc!s^dZrV;}{I6s8QVJ;}gTe9%Sh#bn?k9E^_Z0_P=6# z2-O9Kc!VrATNM79Z@7s*RW!AN7WAo{ZLi`lFKZJ&LLf5K$AckJGf^HTdE`rg^djM zA_+s>oOtgwKC2?JuFJIXgdJBEcCNT%MCe+YJ4BtVu3I_jkSunhz~kQ>uj8_Gi(UHI z(y_T6#;fbT}r zC6w%{kA>mT-MK2!{#Gc5o2Bs2K?oyJ17At*Zex?j1V=+27>lNXx;ymL=WYfj_Xx&O zBDR6Z5kbrp12@Da<8QmP$8oy~=`#Uoyl*FNb=753Xq+xI{Kx0MCRcI`CsLvd;`q3W zv(7yiTmm1+tjGYf1OS3I0BAb)o-<64^t4QQE~DZ^FF_D)_-*K9?glAuL{FTL@0lq)dksp?GRt zQDb)?OBUJEi(mN)%~y}lt+193_Y(kmO;@i&i#dDlVSS2oJ(&E!gUi}82W&$MS6$q$ zH*VCx6C0Q78}2$dicU0cy8$E;53p8;Wbeel8_6{`>Y;(Mx1FvEoy8`^xg3k7!Q3^4 zv5?2$$i&?(lmqh@c&o$d*%~72bJ1^R!l84%8)EVH0WBbbN`6bPL)}Y_=hqR!yG5j8 z`#q%k7uhA=9f%XdBS706k<4V@`(%M30|OQgO@ga*8QboAvElc@J5n552;}vQj0rs%yNDTnPGp<6F7{XplQ#s|g>ImPhX{p5IUL5q zogd#~V=bHz?iN;Ex&xmsN4Yn5kaghXGv5fsILARw+zlnsA|PsVJtMB&|L*t{*uac~ z*leKpS@Wnq*+L(6ak|)u%uO2Z&E}xxhj{JwEwUCJ+aD456-XwOb66pV)jJ76c$SIHAg7QFUMoN z!EtK-68;5;JMz8A3|}|wJi!CF=IK~5U{PJYsf2a(vJPE3<)CdoJn;y}Z{2k8L1vtwRZcZ{9TtWI)m#pLoKE{e~8H>`3Mg293?DR4o_f80_jHT7Et23{0~+k!D3PpEUM820whKi2!EgetSJY2 z=(4(zK>BUG=Kv7k`xb6@hW?zc7IPAGVvgaHBQ{BRUjcU0{2CZAg1nmG%?^ph!v$>w zDz_DI#%YHyb4<;LbB`D|s;_R?xggxz$3acK5vz(uI7X6m7r2R(+8#{&*2A15bD%6} zBfij<<#|iFwrCA?k6wubbs>&57dy7`nhEyadIqY1Z|`-_4{v_<*;i~X=L2*GCS(0s zBNN7ZSO{3TC72}2(oL9cQ82Hyu;m(6nZsoL`AeK$N9r4Nk@n`EK4=?rN9$|5d4G8G94F# z+WaP>Z5nbqx7E-wIt14*uWif9Z7A1?mcGy(f`FiFD?_J|&EY_m?{pEn$Bf() zp(B7K2`0GU)DU=GGO_~jA32af5Lr&_{dq*);kWxYzxYyzfw5g_pE9)jc7gMYy_*py zuIijx3jFerPR41@!O;@fR^;|6!9(&*jP83zDvSrbHr=qqGMJr%5b04wgDY>g=vm`p zsgZMZ)&e`pDGd)t=iN7VSQL`Ftk$k{1`|m7%Lhu_l7VPIQoyXkB@FEj`VxRS?6g&P zC&!rPisy0d5^Jy_;T$K^IlKOd3**lHa4ZS)gc}$Iu$JEOq4bteYVKT7hs?6VWXXJ! zB##qBXR}WF0Y68O^WnXweX$H~^UIRp1}J!klIXt;6G+*;sS|&G1p4Xfl|?f+kLR4Z zFC2GC=cDNp(V)DOa*Xv~KN9Y*F6YPIpB%^~93t0?@DFq6!5hn{iOKYegBSv@r{M@9 zdEc8ivmG^|(H6%5IQmWAUk+67okP*1$psckFb%MouKO}h;PA7?_-(C{ z41Y1kVN-#iO`R;|hyeKSu-7Ea44x21A`aa|?GlH*E)bEyTLa!NP z{|t|!(2RDMoln-?y__RmaU5>s=Jogj5)VH}YeiQj-~zW+2VWKSgL^z?dwCm2ck7mX z-zH^7Jiv}==W&IqSTP8i9SFs;Cn#JZ&K%+7Eo9h3*_{W6e7@Nf+c-lh`!Kx-cXWhO z4Ml7qyo4(}M|%RU zD6;L!GN%fqVAKUZB$h)dpH$O(QJJM@zMYOS(+}k?YSj3{!Nir`_-?X1<7uBTke_gd zvQUk6F4|1fl(s~02rI1JSXVB~=msw>*EJO_VRFU$0ZmzB!t{b0@-AiZ1x8ZN!^z~8 z?g-?^9@Yq(bZx;}fl=Vkn43A8HXadw2tH~^U=)S#V6aUrN{KPn@xWYm$* zDd+&hX~vhId-ZVE*Clbh1y7IUY@);Ur5tq*8{zg0dK-Rq5m7%UzgVNK-Vjlb>w8v6-ko0b_Ezhlb8}6GbwJ zl!ilgJy)wlS~~679G93v0>*Vh!MwlgT5&m}3o00qj##JYFN^9?iKz%#r18JPrF%%Q$@P7?z~%ooAq@6_?^g|gOBp}g%Y^C zpWj3IQc9nK{uWZ*PeNJu^AEeCUB|yh`~JN<>@KCX6Ws9zuGXk$ingA1f9?JS|LMKX zaObr=52ZrQ?rtCBN&oNe1N{Bp-DlnZ;NOq=)IWCg?!8aD?!8aCAN=G5HO+9xN!R_D z_qyP(PalfTR(9vdc=k1ZFVK#^2tT>PUAy=G&~-oBhn~ieJpk8h;hN%lqwH^X1x{*qO7S{N00=;$b-RQcXWi{Qp3mP?i0UhHNx8&(aTLEpfZ_sb|ad@61 ze+NYX(iD&&T4(sL`{gLiFDKMC~V7c>gKNV;pur{Au@ppl9)K{`nRD`vCuSzx*Jp-Enlc+%NKy`x)A& zeT)=J^S?Rn&wjVY@9qcPo~xY_u}xNT-`|681Z&SA@kc4?K5;dXwzN`L=KdvZ3|jY- z=c0Ssulw;M)OE&RAN{`j)Hvd{71Dp#u-|X$@>=TJ)cr!Q^Y+J2Ob@{}oy!yIGsi9J ze(c&re-!(zLtMSavpj_5#(btfJV(hA`c)hq0Y`Bk-5`yf?nLSuwb9$92ufvBPPN%% zSQYE^Qirp8_ul`7R0f@@>B5{6L7ON6Vw22^3#230ev96-VYC0_Ori$x`dpL5#3Lo;@4W*-~F7LI>)`~ z=T6aPjD>f?SgB9T@pRGsr`#2%_@Q-6^8`*CxdywB?}JNqG={_?8m-gXx=(W*@o6}| zgjU45?S2v8-)s6q`rdEwObh6eh}FALQ;Z5O;ZH(+-7j;^d;jvIJ^1L)Fb<`yB)U$d zE+Xm}zqO3MQFIso0%2e02)&DEq)(heOY8+Wd;)yLn6reAYTf(i zpoeG5H&i9nI#C8JT-aF{kv?jeBFm!kwHD`5xsL`cs7yEm9m+rTYraDx9wx`1T;|I_J zqt5Qb=VYXv2URc^VY=rUE*m6)rxtRn+YAkM-U}*)V=N=;BVJ|TX*Vy@=wt7 z3R<6IUJ=*yP&^|&g+@nt4e0C7WBug1i(5j=YYcxTs3@O|*5TMk|I%M?e9yW~h&#A4pT}O!c7+*S_eEdruKh)u zlCg^Sc>;hL~Df^Rm_Lf74XHl7aI9$Q$4X{ zBqyGM_UV4gr#?fwS!4Ipd(erCv$~&AZ;WMi_R;fN|J~ifufKNPC#h80JM$jd{kD%g?QPvp_1=2t?q_Kq=zgN}4w-xW_}6%n zarbM?+Yuc3X$>^dJr$uIas>3k2_96X24 zDbwe5qeqB%VF#eS#;aD>h<^;EUqJuLr}uOjE>~yw#s9K74(A#F()|fm`2V^;zy|+> zzyH1aKk7QzV-U*Wz%7wl;%^s^j)8e{#W6xO zzJ5#zJ_|iTif3La4wq7*3!1mpQm!q0(ET2@#s2kVJchb&3xJ4>Oe3YW)bMC#Y$Oe~BCAqsBkL-9Es*)E{Wx>KG+yyKZzp_-G&g zP4h-Bdds=!E4oj-a) z4!!&fJPG3_qUKzlDe42CxVcKI1Nt=PUf9nl`wVi3s(xGN*4`HPZg}{IDQeUD_kk1k zEbX1zJ2;0A8IV-E`*V7o)s*1O;K)s;pn-Orf&j4bxcw62&E z=S&kT|B$xqKBWH3@inU9!&K*7L7~=Tj&zP#;V{3R-2Yc<`PgS_mM*IKH+?*`ujIBq z?bBR{7yan`Xdw&pKJ!Ac{koqip8hv7M*P+N$o_rb z;ml{Y-(R7IW5J#w@R!G^HDqjwN~8P(^CF6nP|BKC7&E({F;~b@M^`U+{ThghuTNq+ za_xMk!2JSG=dXXxJpEg&f4HGc9Xw3u6v&kY#rNk^0E_C~aA_Uxwc)SNHcV3hpDAa8cl? ztf*K9^B=>k*Xc=-pl$5FVor2{taSVw2YA}BMoR6Z+Ql8e&UF;$R&#E)+~+cH^-IRB z^ZLG(=U!9YI6n2&rpfovV?WOC1LW;@MZQTlWtuu}eLJ*!_wq&R1`^J{aROuK*ym4fwU+nIMF{NCe z;9S>L3h1vgJr*IJ&X3?_zt-qg1un9Pn2Kt z!fDH$uX9;fSnqlNUAL)A*Oi%9!(9D$F3ZE{oq4G@@*m>~w`Yodk2CI1p=Yk1d;HPSBzm=v-I*6xlYl(<{Vrcad+qiTvkh-RIp` z_^*2hwSU%qDgS-lb^o2WeYmcU#Y?L%9b>l59T^;RS9l`bB{9a+zQOfBgL)24hZzWY ze(>Aj_wN4P@v22*tDNEr>!1-~I9_?h@yeJ~3=fp!bsd2Y*k0 z7Uucv(Hw7?rRd%#5;>o{k3Sk;GCR_}$IJ+J1pk6}auKoX)7_t0V~D)c(LB8y{YAh5 z{a&_{=imD3+9aLcE6p)ePrA>+b|a*AK6@jyKw6q-N*HE=IS*baU)5alW3uB{O;bLSMc`!w`eWfOl}!-F^>&<>WY13 z9|uoxBl04wnFw(`3HbUE_l99s^Al=C_sev@OR>eZkaRAoS=qm!1YhPHgN`o zY~lvWYVwWD}<$1S?cIXS#A@9%K+7&@q5du(6TPkg<`^aIulkP_dEEFtL%( z5V4WZ@UW53&@g~cu&|NOkg$=@aPa4l-QZY`Lt5ZLw%a4`eaW@e1w36-U4pr5smpMA zEp-`ouca=-_qEhzec)Q^vVL+cby;7!mb$FJT~l4sC$FV0>$lfZm-X#W(=op_OY{AA zD}MCRE%>D$4(7MMHkjY~(_nt<{KQd=KU~3~#|NI33JySRBl6c*Be} zWH?gn=^ksYgvIZ#dF1UV2@LKeDQMh5(r~$hq#<+%NyF+6l7`wHBn`hiNE(uNkTgv1 zBq`|LLDF!(gQOw;mLh&ge#>KhYZ;(?OBuoURx*a{tz-<-Tge!jw~{d&ZzW?0-b%)> zyOoTgc1sz->sB&`)U9L;qkkHX(KxT_@p>hBVD}30g5#a#4bwZz8@_jzH>~e0Z@AxC z-ul4K^41@Ambbof1$pTwJIh<2*;(HD56-oy<>#4)PjlqH$80HqKC+dB^olJctRHM4 zVaVS?!mz%DgrR#23B&Ui5{BR{Bn-1#NeD`}kT6_sAz?_|ZzGX8oA!#3N9Gl408+11 zgCO?`H5ihwP=g`+3N;wguTX;_{|Yr&Pq;!2)-$e9gY}dv)gV3R3N=_yxl$i754&#kUf^}H+AV!iK* zwO9|laxKyeuUL!q#4FaaDdNiaFZiC<982pGeGykpqCes)B{oG|rNpL)tCZLjag`FA zBCb+mQ^ZwDY>K!_iA@n#O`<>IDkU~WT&08|@ppIO`NR7PX-<>0@@*=Q$(?0^&K+e1 zr#s0SVt0}??CvCMDBeld@Vt|(A$uoT!}w0JhV~t01@}A2S`XMs*80MyX>kC1V)eO2%-wm5gC=D;dM%Rx*aktz-7&5nzaFOcp=Wvng za0wTw4wrC|>Tn4cscs{ok?L>>7pV@HFeLt|EfG`l_c}c;uOttYUO`?kyR-ZlJrH}1 zca}Fi?<{ZV-dWzTzO%d`e`k5?2RqAKuegG|^pTzAt;g&vZ~bRK?j_|sPkp@GAsN4i z<9pB*Yd{~mQVr6Ju26&Zqbt;4J?RQHSYNtA4c42kP=ocSE7V{;>IyYjpSn^F(yOje zgY~N`)L=d9u=TDg-|^qPIn<~pT6UTdA!L$9??>$BHd zr}g4%t<(DTwbp4p{aWj^zJINCnm4%SI>kR+Yn|pXuC-3{ArFW8klbT%kA^#i!F!Ku ztPOtS+G-PzaZRY7-A|O|@Cy zzoy!(*Edgf?|ZY`_wp-9qgU@NExmY0Y3sE+N?R}8QQCUtj?&f(ca*kXx1+T6vK^(Z zSM4koq${TQB)49vkyhv-0t96*7@K$)RWLASZodYdPx;TgzEL*jmndz}9kx z{jKE;^;^ps-nW)Br0*an7~fjX(7m;s;rg#JuKDIrj%9cL0cqT!o1TH7XTj&^t@zR5 z9oCLM^3H0Pp7f4txBm8yYPVkbj%v5Q`;Kb29{!GMH$U)>YBz83j%qib^3G}(&-0FI zH~;jGYOlqx8n^P~BA%_I*<_xH!!uBLGNkip-b?)<_JP|;=P~Ry(q#-gT)Gy+4wtUQ zu*0QmG3;>ZS`0f}x)#F@m#)RI!=-C6>^9P63_D!97Q+sgwqCLqusb1>){g4Uk{gX{(88C z^Vh>AoWC9};r#V*3Foi3kx+j#5Z@KrM%<0j%gA@?CwUnT8ODRL< zmQse!Eu{>hTS^&9x0EuZZYgDG-BQXByS0>{c1tNk?v_%9-dhg6dNPil*2%MAd(QOS zS_a77QbusQm5iZwD;dM=Rx*aztz-a2U3FI1EWW9EO`74ns)~hhd_J!w}HJVfOCj5bO4Em@Rua%yPeK zSuR=y+QrmY3z+Ll~ z_@4J$qo zduR2VpL=KZo0ofs^^1>tXZ4$hduR2VfBP_v&h{~5JH;*{ej$~9i{EkE#NmC}Ku&n2 zL7d`W25_3E7{F)4bhBy?U2NH;n+2P6v)fmx zec~_bh3}HadtL1-dppU&o;%2iowt@V`)@60xY%0G@Upd>;b?0)!`IewhP$oh439g= z2~M|`GyHBXXSn`VitBj4p!Yo+d4SgqJc7+$9>ZZTk72Ht$MDt5V_53tG2Ha>7)E+| z3=bQ4#P+>BX6IfWv*{Q79-N+P9)AJ-NWRI=cT~8}Ji#}km&iJMEAJo$yKgNeNZ3-! zu(73-p=C=c!_SsdhNvy23}ahL8S1u_G8}F#CCJ=T%CNenl%e-l9lgv7q-!S8+Q`;2 zK<<_@g4?ZR47FRy7-qMUF~n{qV|d+4#?ZQzjA3;v8AIxpGJ?~sWDKQS$rwg|NsPXR z5512++|}v5XoCX0RVjk13dOKfp%_9c6vIJt8YM+s9k<(@wuMTv->|$OF7? z;1O*0@)!<#c?@&CJch4c9>Y>EkKv}5$1u{%V|duWBew75F+2D2m`y*@Se{?D`%gqq z$C5j**mV#eY&(EY?7NZAY`l@r?7WfBY`u}s?7flCY`&4t?7orDY(Icc?7xxEFtCx& zaPYgw*!^;Xo_&lzBjh5|=q}DhJp!HUEI!WOSr&NNQC2XtldR!tCt1VVPO^r-on#G@ zJINYOcak;i?j&n?-ceRCzLTusekWP$3tz_(+fk=GzxWLvX6o+Z&$;XmJw`uR<0_hQ z?kI_Vu#=?pf*mBS59}amJzxh(!~YJFhW;HS4f{Jt{{QT~U5s4El_nTjWRWb9EUJns zQ4~or^+Qv%#3GADilk&pERmF4BT}MNq$Rg0)5*##vPSji%q)?#Elg!qNp{c5yY#MS zrhB&=iv?`J1B|dT;Kp`i-M}m`4{o3v@Iw#a0c^kn^n)LIvE9HtOk-z({k{_??qA-_ z%KMX~T20HWyfrQcjCl}xN%kUQ&6hjr=k?CPeG|VpMp{~{!~!o!>bT)V?3yv z8_@nm{JK+PDnNi||1(fS^HWoc_NSy)1f--^6r`k9B&4KPG^C_fM5LrvRHUR=#B_>-#B_>)#B{3riRo19 z6Vs`_C#F+PPeCU-o|sOxJ29Q=_2u?@%~QfE@c0+-Ya_yZU+ra#2Rxf58VB~Yz6)G| ziQKsgVshrHc*>cpqAh2xip89{DpGUisyNP>tD-z-u4)5v=PDK=XRc~La^|YmWip#} znFxFVwHAp=c($hNWx(=uxeT#GT`fbcQdi4R8`ae^)M9nD47FcfEkmtYSIbb_*3~l9 z(sj8Ev3p%DL#<#}%TSy6wRSeq%^zOD9GiQ_&Pv3IJH?3LIYel0V>O;9uirofiYxS? z7*yAI!v=PVx7fa}@K&4G72ay=y24v+TvvFjZR-kewP{`9t+uQyyw!$viMQCUuJBfy z)fL`qt4ev;D*GN`3yam|5@5HwT7p=wE|#D+tcxY6CF^1dYR|e@f?BmMmY}w+izTRq z>tYFN=ek;gSi3Hkpf<0IC8*__XlMDtt0dgHH3yGu2EX-;piOgKYGGyeUwjd)HMSn&viua6M6yX`UD5f)VQ50w7 zqPWe>MUa}2i()e)7e(VsouQGt-DYIJ!!)h}ILt&31ZJQY?4_nx)TO3ZyrrgBq@|`; zjHRYmbfu5j@P0^PVH$`Ml+!UobaZ}{x#!b+i6E{V8PTUmr zZFfS%p?;#1T?$>`3M-I1SFr{;b5*O5Ggq|^IdfGjkuz7d7CCcOtC2HTwH`ThRV$J^ zSFt8Jb5*O7Ggq}PueI|j!q2$QZzDt#n2*llzY=0ZxBbXaZoFVga^fZSBNtw3HFDvl zwjvi^Y9VsrrFJ0~UTO_;;iWbp7ha0|oOlV|bK#{Z&xMy_`;9Pu!|w=8AhyeQY^U+2 zCHL^m;ht~RxH&}3nRU7H1IBaYCpgcIpJF{Xev0?p_$lUds6VixoC!$gPO+=$Qx<9O=XA#w6&)Gd&{`YUpq0aJodYmXfbT~nN z(cg~qtL}E3U-h=*{Hn7Z=U090IKS#@$N5!HJI=2aDdj<}w{~0(a zA~JALjAY^3BB5UL#P*Vd^!t)ba{NPD1{(OY1aLFh<45p+6_EJ#_=2B29 z)>2R^#!^r!wo*_krczKUmQqkEhEh-}c2ZFaW>QcpR#H$ZMp{q?$D|6@`7A0zZ{Vb) zY_0L_%WS9d?I_!8d^^ha8sCnxy~ekrY_IX{DBEj%JIeMN-;T1q#<2Y(Niwm8F>6wZ1CLvWaJ?j$;3kulZl7oCle1vQzjmYtxP-=X_4BTm%7Mb>{ZwJs~zhaf3W42fJ*jk;WlTCJ{^BG#?TrKlC`aw%#pzt+xLhG#|LoYs=s{XrLG z&Q#OclQ&=L8gE#}F7Xz7*cIMt1-rsqZC_V-tHtXIZ?$t>;jPxKE4mHfrm>vet4Il$i%?PevYMV>0njTat-~+Kfy*)HY<|p*A2B z55;;W9*XHqJQTYbc?d=`@lY&g;-Q#(H$3Wx_pSt=#qD2lr#3`qTAqi{Mee|9*SHIU zyTo0w-6igd`Yv%-d(b8BYAL$JU2RC0xT|&P5_h#bUE?kmsY~3|R&|NHTCrNtQdJ`R zvV+fBSuy)IM0h^LGvaxE*%Tr}{Ej=n*AqCC>8_Uvi+7jH6gzlV%T()lSIbnJc~{F+ zOL|w!RQq~Y%T%j-SIbmed{@g<3w@W%6uW&_%T#NASIblz-*)wFZ5z*%#cGKB^IZeR z{G04~SNX$scagu?>8|lto7^@2YJa=NUu|vI_^Vy*8h^E+UE{CzvTOX+Hg=J}*uk#x zSDV*0{%YS2O<-k`UnH7?rQ&yqxW-@A-HI;m6*%W0QGVE;1o_3Dbev!9L&y0Q?;Yn? z{C1pQ@z`;G#aGAq6)zpe_&M}C6$-1sT}bK|G>;ML&S4>oq_?@qUGI=c5Bc`l`$vHEXFdTS=R z@B#jF;3K%t%t!H_nUCT;GatowW1*w9 zdPa7LQe)>ZW=~_3o>8oR6XUquKN|3XbK?cP=EO@dn+q?+Z7#eNySeaE{N}<-F`Nr8 z#c?jY6wA5rQatCxOE8@aFU55(ycFBV+F_gfzqyC;nyljmP@IAacuh5ws7Oeq`k$CeG(91e>U2UX)!xx|+ItQoQjG2ib5shkWA{y1#L^P_ciD*<;6Va%qCZrKPO+=$wnutbq6j2lZT#V=}hwEzo zyEMU^-s~n5J#;q%z36djde!OF^s3*f=~dTL)2rU6rdJH4rdK?qrdMoapckB^rdQ0Q zrdRx&3Zf#49~U2BY#?^Nk%1c6Nlh)tNlC4^NlC4!NlC4kNlC4UNlC4ENlC3}NlC3( zNlh(CNlC3ZNlC3Jd9yuA=FJ88LUpW{-hfZfwbF3@n75;I<_I+9%25!OBS%GDjvN(< zIdW8V=EzYInagH1n**S7lwCBoEEI^JN)hgu3Q7y&sF!ICiFGNpd4(y&tMF-TU zpc8B-rc(qbrc<0Irc*Q~rc=x%rc-1krc?YRrc;!ppc5=4rc;eiOs9I?Hrh@9@VZ2^ z>?udwml4K+H{Z&I5A;3oYnqxg|k?pE^t;`)CJCJ zk=_nkopu%}yehe-J?aWySfeiR6PJ72Xzx${*UlsjLwK)Lf(`;$9g zwLV?oE4C+hzG``L=bO~-Fak1*m1z4;N`7`H3Ew1kr*po^>`teAliHn5`6jhHo$^g; zcRJ;p)b4c3H>us}ly6eI(<$GicBgZ`$?Q(2e3ROpPWh_c8SB*FA>Y`Q`HUQ3OEPf~ ztC4|&+J_7r)DmRipqS6VLD8LogW@;?2SscK4vNi890Y|KI4IsSa8P8OY=^ANSi7GV zJXP?$>=H)w!1+2UDS@O^l!BlXl!}}bl!}-Xl!}xTl!}lPl!}ZLl!}NHl!}B@l!Aa1 zl&bkDC{^Rz?tBSqd=MF4>^}oFG(R=9Xn#s-MLvD zimHrU6kQp)D9SQ&QM6^`qNvNvMbMX#i=r?i7vD?c*DEI@CxF(;MhrFufX zayjX*Rb!k?LutLVHC>TV($)!KwKqOXZ)R8tetsE&?sw&33RyM$=BJ&%?#Z+?f^edY8V?s$G}6*J$Y z6rLxQl9F~jO_|n?C#Q6FJUOMama9URtGs%mpaNR z+S5T!)t3%(s)jUq@y_x^zc%4%Nk~_&klPxi)$i&AGO2$5XV^?Kq0| zy4_5%mToswtfkw{6l>{rGsRlE-Au8TZZ}h`rQ6LEYw30zMSIuT3VhDK#!Ag6i+iVa)730;vkxvfrDyt1`ev(891n>XW*ckpMirSAp-|R zMg|UwluR51IT<)8k}_~mWWCTHS&TfouM;%+ys2q{rj)dTqg1qtpj5Pqom8}nnpCuk zmsGThlvK2ekyNybj+C^5i&V6Vh*Y$Sg@eSxd-ykRZbsHh{V2RQq9n)-eeXE8Xn2RY zRi``5tyo~V)T!*<;$2!a%r&V)^gDt>ou^#qX zTGc*xRIA$MZq};Sx#P5|b?!K=YMnbyt6Jxd)2i0FPT=+l}bKoO7nwgJkZ)QHK*O~dK z#%JcESjfyrQIeUD;wLj7MOF@c1aq1BC>k^KQJhW$aTmquB4(&nV3nuQbN%uyvxsZ= zg)r{OdKFLWhO3@|z4N*A1b%boDOk>xr{X$So{I5Yc`Dv><*C@um8aT)TzRTZ$d#ws zhn#tet;m(9+KpU!stq~S+J^Wxf_b=$=&VM?ym?9nYS@L;)M5crQY+R|QY)%cQY(H_ zQY%tZQY$7?QY-pWQY+39UDZX>zBv{XZlj1%HPHF?*=p4bp6(oNY zjd2H9>9G@9)k5uJQUxVcqpbb@lYIR;-T2h#6$6#iHBk|6A#5@ zMjnF2Ogt2SnRu){TlLXszw6+SyB^xw%mLP-Za-UXPrcS`wH@`^v(zdb?&m)zCHbsV=UWPql8%e5zk-=2K1TAfM>Wn)y^a*374RaO^VPVDhbD z4KWO^-Ej|Fwe|Zl!DB&1SpN=T);m5@p`Eg_ZaT|z3=!h}?+lZmNB zLlaV|z9ytn?Y-b@Z*&)?eVcU(y6sk=&qA}Q-Fz>Ie<$&b{qEVi9JxW;bK@o`$cdYx zBPVW(nw+>PnsVZ%D9eePqAw?Iipre0DOz*mCMeE{o1!}>Zi@PoZ)1(A>a9~>#hO#a zOd)L%tJrn#6p^nWh3^vtR<2W00?nx?1-U6G6{RUC6_F_@6@4iv6=^9b6;&xH6+tN| z6)mYK1sN$Q6$L3MRpZB#YJBsX4E&m54eL4iZ~`K7;Uoyjfs-O82TqEh95^YWa^R#0 z%Yl<3E(cDEz#KR!B6Hy+2+e_$A~pw3ir_Ozad!#cDR*TBMz72~fZB{a1h1KRC{i== zP>g2cq3F!SLvfjjhaxf)55;089*V+@JOqE4cqsBR@lec-HDhiOGr$=j%YS!|Zy0e_ z$TN>_o7Zp^_*Pm*4!~O`4uZA}929FAI4IIGa8R6O;GihWz(FyVfrBC}0|&)dCJutG z3>*|&8920>fw_0{&H_!;8Gakr?i-%QUEsu(CLg#n%D5R=CzSCsu(c^$&A`^CRK%oc zXR8_5+GQ$2Qk2Os9xROs7~$Os6PGOsDurOsB|5K_{3Dl;F&S!O!`P$2&i! z*BlqU( z1`dkM3>*}buQiXIx8;Opp2|gxROP##RgF;u_lfX*Xnu>r!&q*-fVP}?3GQ;?r3lQ0 zmtrv&UW&?Gcqu+};iX8;g_mMB7ha0qoOlV2bK#|k&V`p^yJZrbZZ6~e zlXE!L;41#UgWq!~W5SH%3@!d%Mf`URsb4l`^B2u9P&b4B%-%1`c_};@l(xx6x@!LO z{ZYQ>!0C1HB)_-tdjY?fu%rGfpsk^G+{HG5_}L;#Ah&1??G@0xh4bPppKzU-HV*Er z`&+2(8C;En!#ULXAUM`xGf$YpX8%nXGY~!{2GSfGgJiBFH@~3DNZwWOVN8!F)$#6S zq_Yj&e>a1xH;|t)vyt)JILWt}!v*BFD?^r=f9vKZ{;~{na&3xp$j3Tem3`{`28Z-u z2b8a(eigKUeN=@+Zb)m)YipMoPs#S4lP}G(uXqZE8C{##d6drYhj`_mOHLD0atBSNiG)HgId;ra5EZOdjF*`VN)PVnRe)IUGw8^5dZmvMH z_?81(aMT>bz1hiq0N3K4?5Ts6vxQph)7G59b*Pyu%ZlfSCj-P(@S8E_AW=?<@e4DQ z`EA$(>go;j!<=06Y&Dhr#;cI;RmgNdvq+537w!utDPGCz-c_?*UY%{#7| z(TeceJi~tEx2I;nXTh^H)Q0e$eOQxi)XQ1?R}$%1YtOcqq^5T5StPNYwOW7#sU6qR zQus9kGg2z;naU70pI4B;PSzq^o_$Ba?6=+md+Vmi>%GX=e>C3^TRw$<{IzWZ-_6%? z&yi{Yn%|tmJh>Kkw0fwyZbEiF2wHwm4*g=^?ojWVwz^?7p${}HJ? z9guxT%Ro&fcIR<754qKm#y*=x-UZ}wnr_QDWR94_W(5BZ;ooumy?bSjNSP<@dYQJh zcdwmCq@B;*)pm|bnUwI|Yo{&qSgSG**i}o;DQQXSGOmSj%~%o;cHnjBy*($7*2;em zf%HuFr7R&7V4Vukh6Fbua>Rk$w!Mn$c~DZS7}Gbuj4OKnu*QA#z0bo# zHQOnL>&VBJSl`oZ4e7@Yj(;3`ZA&@YPa(hAP8-g7k^Sl%M{6_mZCJn$pkd#JJ^TQF zkI*yP6Het*rhnUv^69?#r_9CY--c&|T#wV%Qm$;x4E$P_ZM^{Pfqjd}v1YF02Thv) z3hQoR$IC2wg75Os2c(Saz$s->MPDC;{3*YZc>(_7HKejWOrx)K$`lq^u5Ig9^MLlf z^8^pV!z2n9Ku1YCy?P$J;!7ZsIPRY^Pk<|>{e&T3{~ZMOTp}5F27c;|IOxclur}K0m5!{_MV|#)|g3;?x4wK&#=z6 zKN%J=+sAOnaFTU)S!1*7404f{8porbZD4?A-3+{e(wJUF_+(a6{8nnpfmZT3C!oHF zGT29TQ`pxkr_b{{rqK634C65VnLdXQv!AVI&1pbbpE6!>kGj6-n|03^bI5AG|IvwP zJ}VW6Li6}q`OQw7!%+l=xor_%JHk1=q3oQmFO&z{;G78db=WN#lV0{6csd!J+TIZa}HWFq<`YfIOR2%JOhZ0*^D zbU>v3&jg(ZI*z{0rIigjqGpt6ayk>#CVz@m(a=pDB@)qQyA2O;GG;l$o!Ju?%<)5? zQ;;N`2I`!fy4epTi3UQyW7GH1!QnIeYCh3vgBc+gZ{CeV-I+>89e%TU4z1B?^Z`nQ z_QSdil+f+?T1MLFVR$1$#L)ILVcATe)pYxcke44)rCY-^s~1IR#ysw9m<~PjB<%$C z*19~@ifG+v^XSG>dz@<&o9~RT4a2d)yD`xm9uLm8*JARR7os-D+I{Ayt}^^{twmtZ z9P5OZlb_J1;SO~|bqVEanDBTsKNrv}j@s4^f$MNMBudv*vuD<#2-a#GwXo9*N~+m% z5yvsX&5^a}nrpQdMW9xvXcy_lFi^(HXjuY}l;s5OU7)o^-Hvxg-BiPo&gcU5Jkv}; zpNJ@`d9J*(BrR$i)7I8Jn(7?W*4EU4nvdF=6Cbqq9RK`qW{a90Y)8{nv;FOI2Ws*{ zut&qFrB`Kx;4Nx&tP|R%njPwdDp*4zj@Fm9>^iQe&9_7+gM3=muoK$m8g@bzsNskm z*WZF4^aku)O{8W+8ZGMeOlyj!+U#qcH&BOl@ONdlpOt9A0p-9!#V_V|tA5{@T{n@BWmZtyJPd2# znXVMp@tiW@FP$T^-#ST_Pf4D-sjnX!g+zy}lgfHfib}ot47e;x?ak)5p#zjTC)jV` zm%UQO>Nuv(PloTEVPLx%73MHtwq6qYIZN?)@Gdrg--LW)32RDylr4h?X5Z78(p|@Y zti^R{r*$NKjX9wEBf`C>>sOep+Ueq3ptq*Xnh&#`Pkg7IQ!3Lp3^{U@-|X?-NbU`= zt32#Car}2Wjh{j7h&TI{h!SF@es0s}$~N1=WtqJ;7RjlYjUiL6t`E>R&#<-rdL!#j znX*qUWuU*ePGHBz7;t&aJZH@H2^j5Hpe=7B?Jb}>e)4Olxy2Y4tq3inm0*D33S9KF z(AF!!{ZagV1*x140WUkqA^&z4wxgC<)R}#L-yKBli7glYcolV}Yg#jfZ5Aia-kDv_ z;xII1W#CPAHrwzMCrM_9wL`=bTlp5a3vv%Q9Uv|DRA~7>Rff_Bhw4xLQIzoz!y^Xmf1{(?DBn z1fSy+v>ayu`vE4VL zseAoe*3>#*MCrEH(9X?n{KxeG`-H8uL0;4v=wj`sum_5Do5L!0P3r7cunz*m(NNhz zA!~( zt$+ADa$GTm&7`&enQe}b4f2?0qP7goeDyEF(=@#jWa@J%rk7V{ z``66Z@s$4e)A*$i&L{RSMyIVVKYPLKU~XR7{JgRh^vKP|VVM<}T|p#7u-IEfUo$7i zdXDD7F~w|UI#U+dQ+t9M%wF;BGoB*U6|~|!UxIs-D0MFox`S(aVT(42QKE2&3m#+M zl+0S}^CErP>84##@C>XsOsizpL8GuD?5EI2=K7;~lKsFD+U}|0h{-3xd60RWHk>w$ zDDu6gn)r6&I}O=A1D7EMJJzvI++P*U&z#ie@DvUy`@(e?Lv3avwhy@;=ZNTh?&k0k z`Gh&2fn2TMMEk@t>MXx^zu9jJ8~0)4Lil^g_ZR5F)0V>m?6IM8zi)!*ulie8ZwJNf@ zKY@Oyt{d!h4$217-b@?MnHl@XA%*pzhrm=jxA8U8`v0iYX6)% z!85+;M~Rd|`Kc}R}X%bb&gL2J>B# zp0d$}W=l6NJ%i=KuO_FVe8YYe+Z$eVMURZ9;rudskV{>@)WGRN8ctS^sF6g!(*tmO zHw`zthqhUc&T83rtzf<0PQ#rZxV1*sC+FA|4CkWOFD2(|$6?wG>>v`|3(j$9<-QCZ zsmWr7zvvRNe6OL5-j$*q(F|lJnp~yVuOa?05BYl64~kV;q4v z!Cse-I>RRxeO9s@BjP+owun9&V-B_@cFBkHxjOnC9_&8q!z6gxc@Vt^yYrM-bGxw3 zF=re3-VR$*1vXt2)$d)~C$6Su2dR7%y#hKr3P#gi9o-Id7bBv@r#H>onv-EFdJxto zgySO3o6F4OVajkni=TZUoFj~P>xoXIEr`D!qW;K7Q=0@!A$5?zvSTTI$UzIo^UHuXk2*B z@o80()j!rcOxeuays;LqpoVq8@7!Aanl-Mup5}n#(wsZoA|DfL<#gDMXhWbRH>;n9 zo?eeg4r7@obzev0bGo?$XHj)n=$Odo3wDh4-&JV#Se@0S{QX%H=2fpX(+0;odXvtl zvEGyQdnn~|s0;0d^*GU{n0hqxx(zR-ES51;kBR1baP0A;EY=3omQhYoFYXM+Gdmt~ z=EX4#^*PzhlVvmY7G}h(_jTmHAwCQv(`*(0nXx#^;CM$V%)v|W{$_gX&&1f@ zCtKyEtzASb(HE~a=V@+nO?~?|+Ig%qK8zWj(~%rjHbxpOJ*c~399S>RZS9HEq$oYT zLF%qKuhmlO0%Qs6I!<0Z7y z#UoR2jKKnHKOOSd>Hh?J@IY9e&2w6(!v?P&a0Q8|45N9iF%??+LU<$zzd}GebJd%Z zTQmeTJVG0u1%#y(?IW{4T&}gzQ9A+;`#@*;XvM=hDZROfJZuZNb(P*Oy%Ku3kV0g0 zEA@`Ppp6jQmQY9OxzS=X{`npvuzble2(&?WwMrUgfLHgVIaEx-HJd@Qlpt~<6rFH$!I0t5cW9jG_Npx#wn<+kY@t`*M7nV^lp%Pez(bivjB_#IqVF?(^Nh}$Gyh89;rO)rHQ zvhQ&Mvm>3nLP33ZGi$S#wX$pVR%eLmuR!AVi&^HF+h^g<>7Z5RstMane}ttspXW=z z+0Cw40{fh!xI>@q0UL!F2M=;%{iAgB*zG!LTqF$R@{{c--iD9)3ZhZbJnU<(=-L&V zsFh@d(ClkZV@Dgahq>OvKf4+kwpiwcc29jxp=D~4;DIo0(ANrUm6cfyUJ_#!*1}LT zY;WE`U(EnLOt)(fSZNz~djg{xNL>r}8(Tyl+S*O`=Bz`o8I7KC<(`B`N)gDcSJ;;8T_4dMdbEJV z8Lgv7q5Id4VNVJ2?YbcxWBVH~Nw5XfQ5V5xKXKK)Sv$}}SeKs@eR_{}7w0YUj-I`& zQwH{b5&dQ}SD)By4(JU>}wkJSAMw% z^kw*XucDt{$2C`QdHxYkWjcp@o_56Zn#`dv4Rf4rmw)?Hut#BU-0X5MxixzjX4e$z zY-6y|_@>!tcQ12SM~j!_BCq?ghw0In?cCqTx4XG-kNZ%tZ*N`YiNdz0u@{$Ve1^2m zem=ADX}e2z-P72$R5V|}f7~bQ_?UI&Cknk^V%mVS7S2wa9j)reF(IbaTj?=7y)-+B zr)E23Hd?OSBdS->4!$fH!*QE^!cl_b8`s1mDcD;}t>7kmv(G)HZ#oHz*jnQCE%xLr z?274~fR32no6u#m*K#TgE`l%i*`29oBXgCY|KWU0XAR6&B6tB;_D|#2YzPqBiCx8yS zUn1J#R(1ReBuD9XRSK4;Dsv{xcE zaCwP8$gz($&PrvH_VHNQ*7+|xb1!7r?>q%>L|?GKN-c1^pBM?XFFME5i~`+<4WEVQ z<)^R?P3$+@nL-|H)K zPAlSdO7-xd^9S6x8lRiJb`16?tZ(soRrVd{g+xY@)Iwy~TLc=+PM(BlR{`8wik@~T zZ%A=g#Q4=1Y=B)$rr*t(v*tR+7SM0tydrvPTT7}vq@vdlj$SKIIi1m!tZ4^GK|AE) zx6QUwb8cme7Vu9_b=(`Jb2AxzE~h?*X9x5zDGSZ9gZwye=dO81x`Ui)Q|Zyt_SNyr zU84G2T0zdVkearWwF#G|Po&l!Z^>t)Eab>_Bh7b)wfPdf?Hh>ca+eEd0Q7*#m9^pQ zi1}TN#?EW2GG9|3PqG$1_ppW4JlmLuJ(RuNDaS)v#o46)_LOoLeQ-7AxITegd?Qc# za4c<`M;(XlYz@8GK)>R&D!YG}60_@itXZ%|=R}(9*+`9~zUDo3BMnq^$?lGqT~?U6h+=!kXtwwT5~OuawDkP#&QGNC8fpc!!yiDtqSf=iyoh( zbr#G;^D@4r&qdd-yF1Bk_f=9-{+=<%jV0Ta=lJK8!|SNoO_6jHeFnsM`m>*9@Vznn zh9xb%i|o>S@H7?Yq7`L@_n(U`&=$yBq;@5P7D7|;UR@DB3+IoFqLGg$6VagHezlNg zk*9ubUI8s_wDJ|ZL!G5Pb~0rBr%+ZnTHxyj>oOju5sP~X6l+>s(IU<-a^#y9mw-M>>SUJW`xC_H+=zjJWHBq zVROF$M;>~Biy}?#QMd2uac-#l(O&I@6W_fm$BcG?*!0_-@Zoy{lmO3Tb{5*Tj4?TD zb8cb$b5SQV(Fxz+JA-2=A#8uFUiy2$%^OL1vZpwfTWuVZYkyofZ)W4?Mpipo$$EWa zj^PM(vkXi}{XC8ym=sOpn8~@Ad+W|=jv4E$t+WFiH!O!T^x4^ab0I4ST64y>cot!& zed*UH+Ly#JkNU9VY^Qzc^J84c-%&e;HV64(Hu}|0+Up~-x$na3dw%@BGb`Y)A<=K~ zZ8YC!VGq-;bk=Ih-CxP$jttIy$s?xKmpf^7l%pLD_zpIEB!=s`PPnewBh#I6iCU;= zeQZ>Svt-J+x!=(DrF13vyG1EuB9XHTYke+=_IvL$Po^uOS*PrZLTq2Yo30G{iJZ%< z(SnpR^2dlZOX$d&HjhYQNz*nun`TmKP?#gNh6tF&Fa7^m4NBR+%_Z5nLu?tj$S2qm zlS{3G9GmNHD(SQ`ESdIY0KpykA8Y9jOW~U&w3dkUrNmRX=B@hET+Kr+t+=O`i1Tu2 zHQhS~+)*I=FFG16nx!1aa{{vwmH^)%rDY*5{qfjZ-Jr+M=bXnu7Abp@9JoHchGk4C z6-M6qW|Un!v%6tyvf3=#mr^RheTTkuxQ=h%x^-3zYhE3BJk7o7R;0MT?ds06MM~mR z$y(fOI^ABIGkb(PFd2!brRS`qGf%aBI|6?1nNI2+_T!z`?eyHdl$?j%3qdJj|3L2{ z#K_FL2k`GS?!II2ce6ddm7|;4Ifs0+qT7tcP+y+Me>kn+$*ZFC{sE@0PJvlV=?ePI zo>FE{GIQKsL@Ljr;Avs*6tk!GtTNlXz6V}I4S8O|0pvV@|6cMaeh;gu&AH9GKCC{k zT|1m^C0`P$Nvr6Y*d0@BAJ^Z>+0FqZ1#8Bf3;BB`B^6&BVNAyE->{Z}Cz08Axa>Xi z74~z#3H>(ihG~;;M>9VuM|L5E=_MT}YcVQ4CB0(yMeBGFwX5PBk6^CC&}rSBa0=%I z*u4ulH()cPWRwD;-5U=<;y9;bfRTOrn$!T!#ra(p);@rVJ^W&b>sP<;SvzRSMm-y2 z8*|xfN#7crp_>mbNyLcJRvs~HXPT6hAC2@myS)`fXb^1;^U!+Qli?R}M>^J$k#I($ z_#5@z{Uh4VKr}jNwl}xOTu*$cJSoWQ=oWT zfajoUY3$UGp_>Z$G`-QLOiE%&4PSLyAP#W zZr{-ogIwGpgsxqV+Lro~w1x-Xxchy$`oI)UTKPPMJ78Nw_O^+0W{mf<x?Ma7*3W2+tDubZc0{@V$LWibr9PL*!tU54ur4Wy>&*9op#V)8yMNsdTrT zSuwSwrMKqAC;oAA_J|U{4$jSixqGjBWxj?w_(#>DtffCmRv1?c!mB{1aOx`8uiZIr z%y9<7WMq;e<@ZV0v5ejjS~I%d-ujF-avVX_ehR-Q?33yEP}FxAKRcJDIkZUG^~i6gGDLu6R9=wLO3LO(~>NG($n9+$+o2*v;#|j zf`kohTEA`h?ta@Ssq*0A-4AH(ap{`%pV+M3Y?i%(K!ytan?4D7`}NDA^x6Ak_mo&pu20m**JinMV9+3<9*ae?h_yoqE~cMr3nihKH@D!9YnJ=6pC zNn-%N=!M3BD`5a7;K5b`X)F61+B3L%2iY2zkuqRY0xd&INk9X;$pWUSfk!lh9s~wH2kX zS5BQK54OJ{C0&9pupu?-{ zE&b|xGF@G7%UE4c1VGA?Ko%}vXPQSJ!#lbCj0!7(kQQ_>+rtP9;YQ{ig zpbW_AewU#BG!v}M7B&ft*ppk(tEj*hsO*qu)NliNC@)2T-r4L$K&yY{xu3Ef&qF<2Xr8WLsAcRWc5sLFEdi;z%;K|B2HdzQXK32)9HEC>tK6^wW~h^;Bbv%O_$2TV!X?Z;@=2K8g5CCVDU2 z(V(TDv7T~aTicH$tTS>$na*NTD+|uEH%jW!BX%6|VQdLlorB%mn zRrv5$p7*m?msWvdRO|0S421WuM~SW?%VjBlxojCNmzmSuQB$P?qO!|lDCkd-4}*(X z0F=-euz(mq4{@Ng_Fl#pvHWV98d^Qbk%Hd@5rJU&O;WJ?K6TGOW%6Y&Sz*1p9vHwi zDlY{69C85nvBBkYs1NEfMnRC(81;9|Hovixtz4c)0Xr!N+>3O6!oMruke2=$bm+R} zS(*gXv)ybjAZ7Xch3y=1`CD*2s0^q9>MkO)egBbY5Zlnle3x-S0#;@DQ_H*IvMm1? z9e^k6FubqBGb+$G^w(MSD?O^oS1gkTu`V*=1tosV<+!B}>Mlp!njz?Qqt6X6B~bKH z)f*pkfN{4V5##i+lH3|>+|kQB&D2Pug>}?yk5J!1Z?Q^uq$cbY-hBft*UK^xCT`Mh z3cGPddjHB`L$s`MOKwrLt2mxmWb)!cVd|IaFx%*}BZg}VvZ@R}c7nL$7~hF9u}|2< z5EC{~ncR_@%HQQrg%8^!iR@uffIh1Lee#%XRNA^|*&Qc{S#*~4!(d}}XmIsMkj?Di z@+Xj_ca8ed5Ag1LwBNi02q(beCjl1j+Ocr=9t)R%9?|YVg{}m+{ks4+ckQ^jdykt- zK<7Cz&x%xL(RdD-o~^Xq%EmkCd#!=c!?M^)Zz=dVaEGQ#0dZcN z9xNf{WX5k#5e8F1^mzv!SJ*E%T0qH{mNGIqhE2_B0hJiXE%~LJApyt=IE3jkJ`u(x(@Y%}2jd*?jaXR>$UA{w0`0 z_((u6F8>m_P??pjQW=^M)DD&vpmOZDKd{cLtxE$BblmbU+1VfcO5H8@5Gmv>rP`L* zjLW}-Y#`8q<-de>-RXP4T_F?v3YliEoVCB0#K=8Nz5IUkYkfX|U$8+3p_0sG&C4=o zF?Z0AUd%;t1&$y677wlbm+y!^E#E;;tza(5pPk~z2q1n7C4j~cHvV{U`6LHEP1(eM1cEBnA5Zp7d+i{a?&T{Qj>DWXHT7+5)27+BduAY3l=1MT!xR=&zv z`QXaCyU`f`2BrH?`#HO{&-!;ex>E1p2ZLj~`-RTC3DNEL>2~jC)qwitr&yys&}uu7 z;8*>^MxXO1`2^_1BN*{q@$~`M) zomB3EG-5mZm|>+sO*fd_a7WtW7XVG=-ly2=Y7#~&qf%=LB)YUw@OEJLqgr8Rl%{|!o11HNrK=FPmY792piFayB5Ic%!TXr;F$)5;H^y!4NyCUU{i@Q-E6 z>P^{Yn()WaB!nkqZiB7`Fjv>xK~N1HM?sl=VeSKPx~Cum%rW-p91ag;XvYsnQbs`# zPUH9myoyi2p@#7@=(Hc*f;nZG%fC|_wER1@3M)Udmil*M5q_u6?DFphSAHZg;)ST` zkH8YMbeSkY4BKVc71Z+v4>3}2H>eZ*2is@r+)Dr9@d?eQzsrAzdzk18_`xi0yMYpa z_sf`)J!V8cD2U6gc%hhg)29ZAzX?SCKPgIE7pr_$o}1n`g>M?FZtwB zme$+pj&Km)FZdKK)QK#rOl6jAVgUX>)``r?;Wv2nU02e1n+Q$NM4^<&=$MuNh^{Jg zxO%hT`ln0`icbEVCb29Sx0ep%_72a~8X_o(irP-+gl+kR{{WRlMxT<32&QPv>d!?h zm&+U`Qp=$-==_*+o^o20%T~ z0Be3xHyde_3+M?oxh!7Fck8}Gf?Q?MSzd^jhhOapX}?5r*)R4E4VD^6S7{Q5#W2k_ z>0_S?{l&0epl(ZlhBVUSLg;Y;O#luhlm{F_*bef(ywMp_yeDwZLI8U|hnbJPC-mui-d*Q&jGOhsVtko*e{ zNA?cJ9xg>B&SOGpJ#mTs zu7*KwY_wBA+c=EwL5YCKs%N-eyQMQ)2xq$t?qPz5g&>?3EL>aX3b(gthAli;X2wqIvjNP_ch-7cwYZf|Y_Y!lUQ(0OqSPYeOx#Kpp>@3HU$FvAZs+F1UL<`W3ao z-S<-y!zr$x!>K)vQ#cvazYY}Zu3h!)nw72IZzZ1%Ww~oNT6p&o2cfI;-CuAp{O&Ix zt<^qjo#DvRgTW%F{GgfaUAd-A-?8&Xqz$ePka%@KEC+riCoDZpeD3SuQu$-faXB7qj#-pC`QAmMlNl#lF{sEv2$xlsDh@wB(#V3mDk z69xuxy^$i_=E}E)*iymTnk2n-iswBQ%Wdy;`0{dkRR=?lk5Y9khke#0itYl$!ReN1{vd*jQeJL$W-#8|PeGjTS z1O&iyWyr`beo0J47Qd)%ekiHO4+$aACv(5XH=zfoD0`K;yH+HfMzZK1FQ_JriZL(K zA>jfLCJ_a;|0hs-czU}BzxNRP7~LifeTH$yFU%^6 z1ZjGE2EJFcZeTyI@Z@_>SyRS%Q{xAyh1jnjh;P~Wft7*SI;4X_dI-O$GJ42fATKUB z87EK35`VQPvA|>j2j*_I3u|&$>|^ZzXq4^cGNM%9u)-81`GMr>9@dmJk*cBuNHUSQc$QvewYFO~*)vA&_I zcwTj|F>rvJbMD{@M2OZl!(q{@G_1C5NuVKW0=zux(YSV5kH+1hl~!pRzqL4M{MIf- z{gzQ>x+_7ei7f117Yl2gx1fef^8(KK;3@+{pc6+MUOEzJ=2kaRI0!tfa?gz0U9!4K z!!5WFJI6tDQ%@*|maw|X+p&U6!cKos2i~CuvSuX_$p4BV2-WwCfpnN=M=Ojw_Euu# zQluupVxx}kM#|6|8>%D^w&TyWK{6zqS3BQ0kMO9r2XRv33}UcCTs|@Iy&LNP$u&C? z!ljDXXa<%U2wt0r%((o%+IU*e8$IA>w{m$|q$9mM@S>^zDw!g`*h3b%`oC$4+C~g+ zeg35e*lEsBOAm-0J}s8d&&kKK{4N)0FUC7NMf;%$hLg1)29r;F$sQ{S%1b%Oh>4hB z86dRnUj1*ZC5@~!!@C8TKEZivPe z>cPrOaG>-OBGpt(d5>j!F1v~M$P>7QsNU|`aRcbHPhc$I zyVzPub-`s@(w4AFIJE=c%Ai${H>AJ3zmw5a;K!j&I*l@eygG>?`5DJ7=B6=S(n;>P)q ztVH*yDb|<$sq;NTJ;!O8*XPW^>24^8MC?K-n5ii-zICPHyOd;|h;w@}T{z3Oi$ZBtyLto&29jnht2 zA);NnB_ys;C%6!S2riVesX567Trhq5BQ+MkWwbcps(!eeazwR=R(Gv~rhhNno;mCE zr)%EBfQ#s*sX_AY?QrJZif7^7fhqKj^BcN|qHF3aF8vk#9i}Y(6;*fXudrg(3k!f* zW-t22ns5HyL%+#h=zu6wAU8}l`qc1CsvYFz5(WSMCfWfiI?cg{Mfs6h^s&2x_g>cp z6SwnijYvkFRfLGOX?>65g0qOwL-;>Pc?z7Y`{XWB15tf<{}}b8HKp}gJVxN$*l^k76qq*<#?w$>UQ&VGwNqg4ep$nr$l2ij3=Sxth|bWo z$wrOFqUzeR1T~jGfo{M|E`Q=CVz>}pT>hy(`6+dB8PQRhu}Lztm;~~^#OhyIwPp_4 zyR`ZjBv}0m+;Uf$M15G1pXwf9HiwG0{E0W*xCRKwA`Nz3>+&eb+^q$fy8}}T2DBfW zG%%~9BqX4wv1J4pALBv?_HT0EG_QW*UHv=n>QnFPXWrGHdRIU9u4K7HxX2QUT>Zj( z`j_4nW8Aj1zxJ+v|~ zwep8Z;PikcGFH!DynH4r2zU<;!vH)j*~QOrW23-3&co~-GQ%bK@T8P^O8$qhF~o@ZkmfKw?(d+-Y}g4A@_oak<9|z5HTNqKH^l65O6cyA<+e z5hgd16ttzX;Y58rxkka>rVJVkx?1wUPGq^wNE2lCD34$;C7(|@L=)7+PeN3FCV+#$ zElaJjM`63&(}ML~!S4xgsw=OJVG9gjL%`Hu6y2^VQkB1JL4pD8O?_n102xx~%fF#$ zmwzLrZATAsxyG*O6d0?0v%p25-@jvphmfascz`lqy0zb}<2h3`)6ZiSXYrpsQsoQo?2xN?Ka?lg z-olHtJdnViJYv5}Q#OS{zdiMUXB+VS-3>f#<(heb=Lgu+12)t^Kk4}MWmBGtAjOfu zk`==zE%1Hb@L|V~@S8!`k<8-_?Rj@xc zl?hMy;<6tY@+IMpR2mFs;c*icWg=XAz$8!zZ@=h&BwPr1n*`mli)3oTmnEXu699(o?{>>ff7^1CcNiH|)rjNj&yc`3J9 zMN+(Xt(CUOV=aob_*+M$P~q42^3MkRdD@58Sty);Ko=D`2zJ3q!E6bp6@X^`=VPdT z5bwC#FH~(c59oQZ&eHHqBc6oppY3nI9>i~d@gysr>vs^k&Ld+=<^`m>qjXN0C-m&8 zCk*-W-?xFA59Dg;)9#nb+PHJx!?j{9Y^~0q4ZdCFc}@1bY_{*)W>jm-x*z5#K0MNq zhJwwtf;a>QJP?nDP{xS@V#yIKeUDvAD!xF&;eAqr-9J>FBWpJa1+K<#R~m z2i@IS5>K?s$RUHLlgyfyS^RThd|g>DhFf$JiU>72>+2#*<>2U!t7;yV#vq>>3U;5z z;84qxhK{h>jx5M9I8>1x#mhn}W_QpiSLC=X|6xBzNyo2j2KiY+64KbS-zLrOVc`<| zngzjAhD2rp(1>lTE8@)**ddKzQ)b^OOpyAEzdO87G(2UOs~;urY4+gE`z(q<0StR8hGY`Y${ zcI%K@nj((TvNq;`d8*YT9xY;_jZA(~2T?@ui;=HFX?T8i%~tQ}vr>T)U=}`nhrsv0 z8vY&R-ek-d*j(RcQfK*9&8UGcnA3DW=WyOw*daLxU5z><2gSC}A}>3N9|m|IE=ZvO zTV96LzDR%sKb63{FQamGR3!LGE>;o$^WD2hzZJ|AYn!?Ut>??ng<4n&&qsr;2p^nO z;I0e|n!EAM$(8e!+xNFPIZSPxANR zKVDSd@eB@b8H1aD4)<0&_{&c_TBUi2-v{KkD=(l{{F>!C+&%CLF3!tk!ECyO8eK=x zR)iu)v4;N|m#FZU!QXxoRBw*R(M7@GR&2`G-7wG{Z*;>|XvbCasQVUD^EZMXbdNXy zQf!a7gs2(s4}UcA;qa{^?^nVBFug~HPYjP5)3{MEjcczjzFMgdU#isV!{c-H%JpigK0QD8 zZ1LjawVCP3*DAM99X@>I=wbfv=n45bY3~jne(vxQNzmuVjvO03Ix;%KoRZEf{ySbC znJOJSK6$iMDW9A=TpB%6IdQa9Iy_aXOqCCpD<>-_rpnKiDw8LsP8^@Qc6jvobI(nl z7(F_9^5n@QBb94MMz2+lpDdM6mXDPuCr8Rl$a{Tzn;S*Dn&mBH~{P@wSBj7)JH(EY^6sc23%9UeNBgZSp zDy6BzDC;Di9C>bJ@ACB}XU0a3A0B=Ftx9cvu{v3qs8^9%o47H*I8%PPGOb|%vY>ok--Sx8R=QppFZb0d_0q!hdzIUl`gh(%}x9wW_ zduD!iwlr5BcA09u1**)_zs$R0J(a1HHCp;%-gVhBxmc^u&sy=Otd*71rh*}2jll)M z3Ezi9@TH!I->#MxUZ0;kcWbh;K;T@uQJue8E68Z_Y5~UQ^>deAsg`CdH|MMGJ^PhP zwT4Wm-ak5g807sAd5_Lc*A`|JjD8z~iY>EFau=Y-glk%&!!sw~Ve&XwPu zua-Uga;8*kXpfa%>X2n8 zR&+PMV4c$*jGJ4>W@hGZURa!|PcO_=kkB_iccW6Bu2;$hWLrY+wX*qq<%*TDL*KX# zjTcID@Re|!b0lZn?b82RFoTv{{wj+Nlk;;^)7J;i$d4wTt<)yrjcwX^p>%7kdcDS` zUFv`2GCi}2Mfh3O+s>!saQ;w;STnf%=Wdl|A%I~1p-_I!P>ZYdu7c@*tMsk;>Z_Hx z3M4;Y-8eElGJJR#U%HOV-+;>!)5wj}bG173bmzGJ4K%YlEly3Zw03%qtwm!yZ~LQR zmO&9+>fbH5l%^IFl)_^%l!2^mA5=om993fQ4E*$lK&Nlj_vJ>s%Z=P}uvrHn_7|IUi z8bcY9YXjqPjCygZGPW>{ndGhschz1L0;mYQ+A2#i$~dZ7PF-T)xjUA2ZJ6$fr3(op zfFIP z$=C`-x~FCJb_6(t>ZUDHfkC&V^)#4|L`j;3y7d3X)_Fs@GE=En9yu#N&Gio2G0x|F z8_pof8Ww3!h-(53+u2OGwt79Z7+EW03viqls?()9zQnZ`GgCX-sgP>JU#`q7=ndTI|0hu(^_!-1RuC53GJ zH;&EJ*@K0be95jcI_?N?3+qQ)#P5od1#05J5~o^GvXD#znD~$J?Vp*h)gL~OKh3(; z7Nk1@G)Y<+CF`)2QL;Z|ZhVV@YW>1u?Z%;t_|x3z7tU6uO7K>#Zx*OY(4>UNEio|dot#~P!lbSN#!ic4 zhO8Y?vbD|0BTX=I&cWnhZ`_J44nCv=IK}+e(()(T(2VFc5S`}mrnh)gR!}X zG&cfyjDc56lXdue_y|#K?g7nhb>Wq1%wJ;NZEU!S8SW#~fl8fy9%OJRmTX68Rxnn( zSgPNMQu_AGzk*LDkxS^NpXgs93K>(!N3@%*YY|Y&AQMc}Vo3Xy^!52OrO6vkGACm5 zx)>&6pCtr=E|xG@%~WPCR_DdzrjtD!OZc!BqDEq(^nT^SeAyNDd@Riq;b{RPO$&?l zv(wd3>sr(^OsYZL5TQu)wb+99hp920m{}}Wnwf;u`c|~xT69#H_r)p(`k4wMd&cbW zDl@KPdTWuIdeln~PIPyN8NF5c=HfJGEOWKW9KMGp%{v}Txi?HXp;(7x5g?kcI^YQh z-|ZN`0PR5J`t9j*1&(x`D<47S@-;Mab-G|iCvMm3mDyouHfqBRf7gbwfHFUKedhM1 z+Y6Pld-(F=^h~)@-H3pDU}C)VaVVyOuAII;_Y^!QEM?Jif(kBFs`cqglkdb{+WOD1 zt}s0{Jt^VIu?v;@jrnr%Cf99><@sW@GBty?!{8OM{76P>uS}>W3hbpgr`dMRGUG3E|zp}m~_nwh)_<(vU)ts3ffkUuSrMb7ZH=d(o(s( zA>*;7kAzE~pwc)V<8xGbb=p=i z50ood-zrz;CNWyJkX4xJ%NXM0ks|f&gQq+terXGbCd^HR-=czBvE)6A-3FKil#J<= zA(y#ZFzkqhx=Afq^cPGYhFN$!ho2 zg27I^>H&KZT^wmKqK2#X`n=9iRdCLmYGFB}#BVLmjm?#2Zr7$AAp%S>L-D>ap%!)y zHt)7`7T`1~ZeNOJu|LdWd`|q~w_IF-Qc3N zb@j^gm0A%Va=ilY?%H&{I6p-vPiHN|9q9rhvu9?e;a#iE3fejW@yoFd-yN99H`(0a zU_&=3!9deW|NUslrRmwqJf=+kdW}~3#n_5|Ay84B;Jv&!1<1nHZ|2cLWT9X{Q**Wa zBeCU&0}10X4pNJHhsu1bf-MTuoX0eo+ZJPDh83?Vl0!(qp@U~GhO3v48;kI9UvNy~-TUHjwh)0HsN^`f1m1=dqTD(~)!h^z2D@eKswUEf?+;xs~;_szZlh1OK zdKsoSOaD3~bj-jpE**f-v32P4^?gR$0>08Zse1&RTgpW{~ zD~`XpSgGD-Oj9C2*xUx#y-%Kq3DsnSFXWDp(RBTWAQ$4FEeet7?v0%5hIlRlt2QJx zU8BvIwA+fUEP{Wfe47xx>@ut%)22PrSvutDvEq$V?MAVN&9W-X|XD5-^ ztu^i2x}{(?F4#qmrU4DB6dQFthMV{Gf7@*szGu9=(ew8nMA~?Hv&&+P6+0P>Eh3~? zt=T=tk^$3$&F4$Cx*eX&d`&?A=T|kSJuMg!?f<+BzliQ#vu)63Np4 z5i%8UlM%g_0rME5_tj}GBw+;etT#qbyiZ|Yui$c}{~ATl=#x>u>T8}~wOFf&alTxg zDPng8hs`?HEUp)?p};a`fY?m|lRkI-S!_)%PS?3l`8^K1)yhJ(!i5tg7E2fr=zk$! z+R-#0IsYRASShBDQ?AfFEOYlsEG}|1Ax9$G%%w5Nxj~%f16d;AE3&oL_%-oCe05;$h0SSfepIgPC2uB8#wyg^taY zFVEe?J4sk#gu$A}OBFC&3k#Sk&SHP-XGB7otMR26Atqoz4Lnj+i3!BcVH|X zO)>d=(i=SO6f_Vs#kg4KyK(n+5cJ!9|0<{j_oZ=P8lGKP`tR|UssD1st26W0N;A{n zlHD%cUOamE#PhFnr5}mTuf8;Yc>$YTCb2Nt92^?2d!I8HX18dm5XV1NS`PKN2!dk5VE>dtm^j;ZrN8Ff8FR+3& zrs?-FYM${ojeQOPu=IaoA;j;ti3(n%#LmaVXU6cJ{81Z8JX@KZDOK^lQK|gK+|2C@ zi*-h+yu~-huwr70*TjKFuT9U+T8iPKdv^A-EmC zI5&TAeu0582~$@kN;Wr-NfEjLf3X9l{P`ahFwSR@ z%BA~#(CL-8nsfgG0g(Gb$WaKLs^C4yhkmxJsNTiW@28zA#!PH=^YB0<%@JYMFZj_+JJj}4eOb%skt)nSlr^H(V3bcbiKJaOd6 z@#p;wnf}W>?krrlT+!@+Ke2Y9H#q~ibn%gM-0`dj=;Cc;Ic|vS9go>f=q{-?Ecf0m z&Z>x`WHe$APMDjX!i(2e-8+Fy)pxJU1Ni2IbN-obOl)7=jq08C-Y*=-i}CVme|aOs zGF`@b-Pmr5i3(#&#(b-5vMPGAauv9sd(1CP<8-yT>+lcw%91gg3uc4WSX;S@JupG0 z?7gxv6+2`f#CGkt*QZ>o?qNgjVbP(!BA?*$>M zS-}QNe3~f62H8EG&&i%n??eV`6oPv?o6W+$cV=gb*rF+x=Udod#mMPoiK}BbundWS zDwalT_n=5xUc4#D7aXP4?6GOAjNdB}>?xQ>)Lx&roiQe_4Dlo}V;)7Mq>M1e%=`l1 zQ1Z=)+pB&L73)E4h(`~*vm>posM+H;#a7s`53R7Ci2{|WK=JtQFb+(&=Mx8V@;zs_ z@4y(oLR~NMP?E*G$1KFQ83?M$i(Bx7x#8g!^wo`%qOCY8WsCP(I9{oRj`ut;UVD9U zW`-;FJWI9rdKHIJ?y|#|Gp~Lh;m{0xB`NfF`6I3J8gsk3m3z2z;xG@Bet)`h^YR?y zTK?AR$U?^ZrJ2P_bT73;@uBWK{eo1z~s#GZ)oNWpB69+V*OgMVJT05J3qBCsd(<1kt4 z6?(gB?j^dsunR2x1KyDdP3B~uGbeg1WK$c&nsmPxyI;b+!HYoEKbGSjj+?QQkk9o5 zFmhOkzX;`F!@IzV+TK|w(Gj=%s!5NXtNr)P>F`C9=!)Tv{PvK4DYA!rzIbhBeiAQM zVJ|}E7WPBnh$XDCad+snyvtmdm$IfxkX+fXx7VkV6u$Fw-K(o>F%nDpc9Y%zH!EM; z!0O}u*P8+_H$=ME(*o}zlA(LIIBcZ#dGp+(dC0 zh6z6WWut|nw_Oo0Gn z%!+st0bYYDVw-ZU_8F)`oMcj_VbIUdN*;XqHUb<|d;t}q4V>fP&U}FEv7x%immLu~ zvqwdwc*Q+s8}B{YU9ha5k0S1US9Pxj@5MkZ%&y*)RJs(qkN5VDcp1rquJ#C$vCAM1> zLf$4%<#a)O|J2`9>K^br;JkDvMi)}!=ZqVNr=tTgBb;!7@ZM8sDCVSv@S2(_U1u;9 zVbRZCjXDwIhk1(eM8FLBg5x<=&WC}}N_9Fq+%n-ES~a|MzZ#!CX>_1I_dd&A@Z#=$ zBRi-;v)QR&91g#W$?386A>G}gxoaO?#M@N7RG>! zSa>eK-!b)=Vp6*j`yq9EqNj7n<-m+$8G9Ab2W7lUEl2p>lR_h(Fb`OX#Aj|DZaQ=8 z*xQx;P`7&ZHXDkg0kE1p&94sGL7(SmO;_mO({!g^e z&*d=W88gT9S^ZdXv5te#u|&7@zvOBBUk^-xsucv!=ZbUK z>!V9tdR*`PB#!jJX&F4-8GG1#dx$#MQvvf9V_=qe&mlc@I z&tPz`7U$<;Cm%bx$aF-H?7{a_T5US(8ENt)&W_{QyK>nEW8g@5aogIndQLiz=M;D; zRBA8Z_V>)aQN_2y{_*x?7A@!sEsrE>b&?c&c@&2x@FPysQ@78+ykg04d~Rw!ytQyh zKlf%2aK*RY&J%gjt%h%1lI;N!apQ^@T*Yc-7CS!OIf-xZ`!#wjzI?wUunSQFa+9_K zI9WkYkL`*?y2q4@rFTcxkGL5Tmu8VX)^?q1`BGh5g+t4lrIIvN;pXXY`- z#T(mRuu)F!y;-W_j7~l&!;rm?N%UN#g`d3|MJxDG4&GApocJ)&rt^P68iEWso`eV9 zQzs@hyudiO{q)3VUQk(AWisyXSo$}4s`fu~s$G^3p>sx!CClpkHN0btc-xIiK-F<* zHN8qk;fMlsnf+{~D*WB^*dm%Mx7a`uSS!mbytT)P2XawofSPI@{fw0}|%sr@R>Co}YHLTQqB9HGFRj-`r~U?cS7E zsW5fNLfsAQ#Kc?NcyB~UM|-#tZKn7A9IWL@IkNxZbA9T3qTaZl`M%-zPSn0v%CPjK z;lk3Y3!vzM(8I6{vwZ0uQ?_AuO0=IgR>#L`t}WIp&lYE~K3tudzV>WUKU#I_=S+{CMfuQPL3X}96oV!SK(66 z!xMTEgzThzMZOh|Fkqo)L;t1?Lxa6NgMB^gHVlsSY}kaKM|w8&^h@&4z`35m2M5Rd zH_Gj9`7IR&i%M1ONA@l2Hw-TSdINHl3jMfV`Uw)>Vd5j}Hw-L&jH|{^c>5##S@{M@ zmplO{d%>s$(69OX>j?LrZ!G%%lms= z%0T~vu5@8P*f_mz!@!qOiAGuT6bHAce70-~%W=~~wk|4G)X6{)>o)A6Fd9E5J%lmX z*ra7NKDG&~uE~upQlZ8NBBiCj)Do9|=IXcfGwIPc3xlic-NRM)uyL9K8%J47f5jRi z*U~EPmd~-8%WoF6_)n4Rn5)e435p&)HaLci(tD_4QJFV(qWaqn{uR(|jfa@LJk2JM zuT*>aGHwf6UZafGu5&k6+=XZbs?+c!f2GIC;ELADiMMgfdvr@5)m1wh9}hPAy#6lr z;BJsBa)=Ren;3(&bMEsz zko&$b%-r{NuKPM)*SWsV$9+#)-`z_yB+(I(WQ~oOw6>u==dR;(W{6hK43pL_Z>#^( z_EK&=zLX5nDkZ}p#@V23=+t1|o>bHNoqo6l6-oo?QiHC6S1p~4q{oWy^Zw}7g_eSwk*pRJFzU|du@s4YGX${Tj{XCG!W5x+3Jz4Z%VOS8*A#OOC~Gw;&dxKRwgLH)wE zF|dMaH|EmD;jFLCf|_!1g0*ZJ%v!Xj%|dX5Jd`i2X6$Sh3^I0gXiygDvR{%RODQk) zpc-|$Dq;}p3(PFTeo^+;@6+W0<#{A)O8sa?do&yH`q8Yu#*V~vmLZz^6fFVzAPh zL1$F3Ok{mQ+aq6o9KJh@y_A==teIgbF$zs%M>IF1*98#iinew$>Q`OAI9fM3>6kCn0X)k~}k*>yMNtX;+L%~(;QT%RE zLw>%Oo(7teHZG6|nLvr-qxU3;kNDFp24SqN3!~HykW(ru_oz`~Co-}8m7Nd_?ztsC; zQk30mbY)_r-bArEV&fArzUHHbjK;|)Te{-Y)mVm4lQs0CQOT6Jz8jO3*4Fzj_`)Ag z(gvZ(y4bA@0s41wwRNG{jV8~faabD$5c`Y8oUg%@tT&YtqgGoF0~QS(;SQL$jWX^l+C+&VUJ1Z$qX`?ya{Wf9A;^ z7r!%SRJ#lV?87iYON_Mh2(*)y<>v`ssgGV zYXcu@tFu8SnA_;{!>0CN@zOxSq@cDkz;z;$%PFU69#O`7r$nYXQ{7uHXfTBFY5l2u z6*FSCsH5K+NSDnx!RNMz*U4SVuC$QJixkq_MDg{bx~x@eG=`6)4Pl$jto-Ja@+ZNU z2Y0;rB&dm?81MF{W+6ArN?3HY0LDY{v^l8Sk|8;kQY z(QfRhCZ$d_b5n-BAYvS`gmz@(3!KFilA|RwWUQ%1`cl;-0w!ZweL?K@^Tr384|?Zi(_M2LNf<9p z)C|G36=r57WsOsAhDw9tD{MownvW?W?$V`_ zwHJx2FYpG*Djn`gDwWJbkQ-WIcoycLo@#O~n}141Ap#+qOc1SJNeFE=M`!{Su=II& zDw$$3Yg#)pHsTmk)1FMz2(KF^Q|j|0<|f)?bGYC7*pkhNU{s6DJcbUKRk zd`7T0B2UXY@xp=4jXS_lN%07qCvSK+kX~TyPvUb-eK`_(d&! zj|(q68=1<*8}nLCiVf)!bq@Vh6B|!+sjD{)^M~Q8S-PCA%NbniI|Z%QMLex<7j&I2vbeQC*n{aTJl) zHJfEKC;l(6sc2i${M((a*`v;bo4an*6Jp3dhoEcF)x*SW&`9W7L4CXLegDzcK1WK2Js|Z&q4ko#3wLD19rPv`- z+>K^xrxJr+_4uSrKkLu^or8vgDb>O+!w$#JwwqqAxytY-rUNDEqaD-ZSV1rnOTFWS0zq zGRnOO-0PDM-Lg59W9Cisz+=c{N4kZ-u_MTDlo2>8g2)-W}E*3^mB zKNw3j8G$%!(pKw2${90>TV$Okqt|(Uc3~#zi7|rtdIw!23Dh-+V8WZ5l%Nb>1l5>k zs5mM8(0!^!H+s+3km)9Olx*r?nN~>Gn2#86lr_|}*^I(wn%jZ+FsVmeD2e51uQEgX zta(HGB4b!cjf}zZ0y83AgHmAHqQN6ak`6UHXrY;!1(*aLm>rM-fN76fKT-9d@Uv9BxIIbBkPt4)n_0JdMR{J5+}r6LJn#a|-p4A8Rh`AMyDpgz=CUj00u8Nz_N@qz7OdKq* z2FJn%y9VqfVEkR$9U7I(xRcg;C9B9zZx!*xeHw+lsibe{HawTOm+Kj5vjkm;5WOG} zMr)&Z?hz;0rK6boE;=ag%%{*?I^Vin-+}vtl`?CflO76Al4?)2HF`nXiNz$O;?q*q z&hFn*QvLmSa*gED3YCKk$#ea<$dnDz-b9Q|Agl~n)0xotF*qz&Y0Non#SD6RxDxb4 zC&zJRkeH!eRKS4mN@qgPU`-QOSf0$mL?S^hK)D3D%u6StYD{K0*of3XaqCoX)vB6V ztXh{;Q-%+nG@f@^=is7^qC66fkdaGk^CTu_JX;Lz5Lqo}WVL0UYkuM*&ewAGcw3~e z?=i`h-fxP?ns@m-4PgsoG8&Nh_ypOm;gDr73~VXnPNpcHa;Ld(LRKGzD0U4RQoFR3 z_0ovEyoL0Egi7AnLT;Q8?REze!b5gVFNF&T*rm}h)mBF2#ujn|C0Q2jwaRN~*P3Ml zJhHcAK-U?8mfyr5Ba#AsIX?ubmww|meVfAQ*miDqVwu8=SCzop#?zCa7KO2rkklOV- z88VDzxV(+Ts#0rwRk!|5&J1HY*@9_i2!sOS4RN?Pv_^IVnplyYV;U=N-YI#nut+L= zKN?*jzBW|-ko46X)7lu08MbwJ<>CB+#0xQQZo+KE2T~yt8_AD`ZZH+tYKmUF86(_< zYO)0kls1kKw9&?GJ!yS0DvQP@6XYINK(X%_xT>o0i7Rq)$kpmZQ}9+A+pEA&>UOI| zH#irDHjlZJpGoUet5^~*1ABy-DEn?UK*<5J2A@is*C`y;U4SoNY)} ztR~T6(|L}Wl2St^Gj8ovVOTS6D$g2Z5UG|-*AE+swM#KE+uv%w=i{UBk^Fm15LkuY zQ?@1~m5I*#k(*g$mmeVzl3y{`@{*pMk5}8KLTm~u#ahL5*V|+9>Gg(z3)9^MFrFhk zl~gkZ^@E;8^@ApS0RtyS9;8wIJXgDI(_C_--FOs;{dUw>i8N^4$b%8pCzRL@4hCg= z^OU1aetxXCBM+*#^@Bq7#AZ>*EwnTQm%BP#f$`ae%mBf5plXIoTDZh4J5MK5(sbt+ zKEw!CXQn%|Sbio=7~W@<9ufrADHgMwi( z(LNsQUE%;Leo#P+vg>j1rPGm z*kQ6z-$+nsRiP#d>8J!IX1NIpk!?+KRvN_orhZt;(Ci{R&a4X&(73Zwp|0!Ph`6Ax z3s%?b%R)l7xWd11uPPkn%$}cMbFb2M`Ig*(w1SMAkees{V0mE^nLj+oJMJrJkgH|m;z^%VtOU30{PBw~dZf`&zA!svV zJg*+J&0D6!1N4#x(iz=or%yF@oXAKTy{WXFKGN|dqQ7L8c__Gf>>~^7q~uW5_KMI} zvvf#BEF8K)*V6WC`h(Qb$#Y$F73 zu7gpASm7IOO`yjrb*V27j!etKNA*M15k4ta@bDRJu)GFc5KViMJuOIdJIVw^U3R-LHO(?&o<1Nx3T-3Q+ydkkcwlU`#jk}KgROd$L373sveP$_>~q@NC{Jh z6Q^OKd4OjCJQIv9q|E*sPt792^5zs=poRm8l}lq%$dARY?d>uTVrn|w$thyRy*V*j zNFK_7`&AcN82;Pgd4}H2cvp0IGKf|EM&Ft3T6rnj(3`O%9Jl$`h_uLetA zSMrdfFNz3In~f}+u>1=SzrWiHsM_1u2uf|Sg@XpWMarvA9Mh$BR`wc;PI=1dMU$V$ zBf0W}rlA=w4ts`Bq-Q*8Y^<_R7GbveC$*A(;J`(1r&6t{O<2Leb<6TMnyn(5Sj)B| z5E96RlY-El2HOr;cwdGYk+HP4J#CE99Tfy6%Pb14rXUkWXbUe%CBxW4gZVsm+}LGd zdl{ZBVO&%wn`KSyjG%r?(%N#{8L5F@o?volD**8V#UFpDQ*9Bil^2`5r|TPHYoX_T>a-?$G|O!Dy77!SjYC#*|?P)FH^mgW<} z_-LOOdNf}&b)|bjV^?bDP4x2J%zTuC%=efE)erh27Wf6d2Gb_w=W!^s&ck49tALXn z!E-aLe%m4ivYf=+Hwnp?F6k2jv$jmf0#D-h|B0sSUK=}36X@ewLTvtz%1?} z`yyY`2`Bsl<`Ua))>Pf(@H+I!vmUB|^Lrw-wPCVgbgQkkjjMv;xXLe7GGm6je#LZ| z77=7~uc9YfK4C)Um-xdJSjT%YMzq7?EG40Wc-iz9S<$pp9oC@g2|T6}uW4;~ax=xo zFcd<=X@U+OcsT)P71f^T#4u|Ytp60iD#*efXeTVE6U5v^U~;=QuBfl}5~!tN-*--6 zg%PvCq>o8O!pxRDpk|&1X=Hu3Y{x>l3(N?qBL43I>K4BN&0yB^zTWLeZ%Dar0 z5SQ~*`Xy&g;kwIEnbEa$(|Pm-KE?VaN^YgpcY3B9%WOu$ zRC$kA<>n1gqTxIJh8yy#BS0BlBaE*MiwARTQQpV z0@K}OhwnJv#B-DcpLoovZq6c$Kc5zy1cw#vVAB|^r96GgWUIq8E9Fd@ znH{2^t}Mek?jt_`&iO`SLE9$tk5Nhyv1;ejajVE9VU@L`jW6ojQDaRzUkn*o<_B3E z>{YCG%z1nk1!&zdD4Qm06!v?DXM?goW3{NVaD?$(QXHL)-g4tGD+ii;>aHXOdj2cGRpgt9uttQn8?BNXE zj`-uxP5EY4LDK|BVd-9j*J zOryBKi54SO#hfg#@f8`3U%`uD+xcE1TSI59()xNei@;?_F^f)zicQfepJrIfMw;-{ zf)I@|K~)FGBy5mEX;{8Od)jAAII!ml@R3Q@h?yRtW>cnw~%BaAn$0kN zgd7Tb9!VRUtW(Wq?9`xm(@yAxqLqPdeAp5wraSN4h@{M-K&&M!U(a-hR8eqNm2cJ~ zSSwf5jQmBO^w6Rc(TE1|;_qbh69$@Xir;!q2^p&`$wH}!*5Zi{t#HYU4MPWzO*2om zy^4<{sumO9Qf3EH(T}a3_~1jakYXpharL6DV0@L?x|$*sUr5d*k&cU66W$`?Md267 zKGZs7@*vd2L9M8*fy(qmfE_C_RB!WcQPuA?8zR(tiC&;IskEKV+8Vlm%KL?CwiSJI z?9cV$ie>=3X7L@*SF#`2h_kqBWq_T0*UkxTs`j zFmQq@AqY~3uU=$8o!la%4Od$%0SH(kLqUj&ba#+FovfJY?nFaM*i`s24>@lmMVYBR zpx=22riZNlK_Hn3LS<)Js-!8w?=G0AdQ~*Xx;XN!k{_f%kT+J`{j9XU*JQGJfE%*9*9x*S6-?0SnpUB>X)5ovXqF0&KNe)}hI1L( zqySOe?MvKE%$GI~p}qHau!_T*cskY&gi`l+XkWZiF^pO_t!i7>+_i@<%=~KU3=Iu) zR*tv&!x{CvS-$7pNjB%X%1ExDv~DpUz%J5Xk0-3Jfa5JW>aqB9_@cD=utCj-)8Q)! z_NxAt<3_E#apP+@-L+PrW~(eVtj|%?2Z~mCE^i46#wzp-DjMl|Oh+*^fXc`ZMe~*=C34h$cmpNsxT3_zr|CGSzqLun z(yZsJ=w$1LuQnwN{WL+a9*gIy$kzcYFY%b+d6dGGIUSzoiD~0JYpPm|kS^5;5tJyYLfBP7^0D^X0f?1)a9ePa)2MfShG3!Ga z;mQo-DxX^-b|x$NqQz% z?3lrWem=yzKFogn?DJto`FJgVzKCTNIo1w;resO5c}r~8lR3$VH8X)tTSD2qw8ND% zId)?D{9Kinusl!oR)M;x*N8M4ELc2l{TOQcO zN2~+lv`&ocY^9lrba-LR){#N=ohG#`p+UU5b(+79SK2^1oWAu{?lX!#)-D#`4_dZ^ zehrd0q9!@XL3p!Zq0Ll16X;wg!puvQ>Um>nFCD}iZ*UY32to=3C`$M zq0t zvzByalvP+)zt6H6#A+pMw-qnsi${s)c($Gct3w|%Out5aSe`JnB+)xTDvTj zaXFl<3*4_ObO?cu6;j{ZVw3`qP+t~!nBLFV$~T@~mGjoD@^SM!I5-0N7ObrH;4!He z$w~B2m2Vwh#nTnu^bX6{!yLjB7&gB@9bOZcg21XwOlkp{?KP=m9eJ9fB@LE8!fiW= z*ot&omBY7kOrkyoOfH=vaE4?cosrI%M7|Kz>Sh_ULV1rRm-0O^=x5_-(mD1jWqPI} z@oGABFRmgZJ(KeEeXntx2Bs2#aL?ZT!P4mMCtfDZmmR*UcZBz=*dtc(l?;IG9qA== zMS0{i@R=@2*Yu7|sZM1fV_omaG=)#-{?c@*2i9~tr0hyri*-8Hq8EN9C6n^;UY zL&NFP_EQS=hZQZUV%DTutZL_2SBSz5cg#t1!&IjAWIU$+wv5wrosM1`&Q9m&dO~8~JM@%lbjV?k zy^z%(I{BQ1-N$rN0zUM@1h`071+~+HAwW0nVvd)x!(ky(6{32E*8 zl6YLZaQgo+Hh!SsH&trGrB?>pKq5kfC+iB$GhtT0%u?4bF}7-|rr0EM@8q6DD>b{5 z=`EaBoaqqitqKhV66sFzO?pwvpg5CqFZ0FOna36Xmol|1U?(fm+A8=#PjCb4Urdy3 z%;b<^k^kfb)<1x#UXCLuvO9SM1m2IsZr1Ln+uRQIj@W6jn)SAJNIf?eVB=V%;y{q+ z0{z}A+e6#ssS}#Q(T=qC0s@`Zo3O2jsZX()N&OTJMtzPF;P57F z2vjhb>Q{Z5z8An~qDl2LHHu#8^Yk{~r!$Z%OzgdR@~5Ebx|tE#RC9i&88+f5DH@;` zLa9-vi=e$WpNf!)hrN*^>M)aG95eL@d@}iaFaLU| zly<4rvC4<0`X21G7ZYcDPl-Z#KEKS3gT?DQX%;uYt2&Sv;A^xVV!%LuTuQY8&H7i@ z1|@d2zr&#zb)6xhchYf9FHi~}xUB$h?YdE-U*BR&{x>oe-;&mE&;ZzppFD4u!~cSF z1AgUnXc%GW?BN^lUQ3Wm?_&_67gcL3eXp*y^-^cOF1TKusbHvCZ@{dZ2J2fflX5IT z2y2@0IJQ-%q18A;tqCP94a-wT-<{^?!}^6Sr*h z2kOR`!@t`40luT*dx44Zd|yAgCyyMl9Zu77(+L=KwH|mmF#}#-OdvxE+o*SOuziFh zs8O&REEh3K({uhAr@&ssg1*#h^42x$aLc+XJekhhcrwX+v(=_Jp<8Cus`>n6x<2#- zT%V$fH4W+=d4qL1%!!nB8t6Fuxk-b0@mdoYnkYbly=()FJ0BrIABOHlNCt}D@@8w7 zb)r4RV%v=0i`DfaiZW0XYZ|hhy^Y0EA*PMzmr;Ypx%N)9Vwr~NmTiF3o!*g)ac$@e zZsZQKqMNZ(n7y{h<59}=3u zVshZMf)vYrj0!dhD}cS0X|{p5pW|>|kfe54zUl7R0LgnE6HJ_g+Ac-%@)V~F=BpXA z+(17a(m{oRV(-WbT8~J9M`Vm4Kkgk_)jP7r`1fPU5{^~O_b;~hj;!+$*2sE)-nZF7 zbBI<@6hOY?MAb6Bz8h(?dmwE*(=xAniBi+%DRN3nTiD%Ys*wwqlJqv`P!1#3@84A_ zdR;@AP#2B=dOO%`Dy=?xb@i(JKdI{!^Xb?C>`AZ%l%vt9ZP(~v*Xfpmxb3{NZ+@w_ zxibd1usa`(hyj=p`aRumO02_a->uhh)8-pA7q)3IL!}nO*$N*L4^xsGVJlSU>ef4Q zB_6q=g;YFJuAh)u4cUT}NnV;4Y8e%0!3rzpfL(gOs1{rxs25eTs+Flc57rkaWI}yIHCoQ2Gk3z4Zh5yL(N17_9?h6mGbNN55CF zhVTF)+Qs=Dj3)mS{<$KSQLXYmhET#9ECWfN2Com4%ytviF(uA3xC41OjClzSn3aue z7GE4Yz|s@%Ea|4kxn29Bf3NRufqM%mi{6t9)U1A3ki8yVl~t*|Agf}i7f47wSW34k zZSx5mXmdMsew9RdxTs95l-C(PRODexK1v?b!eLv}AD*IO8auVuqNQiE`f&4ZS%vzl zVC;s?dz;U;S|77ecX%L-wtTtp!!u+FRi*uSQmiL(&<|6YD-PXD6N#KXx5Z_v81 z(sK%she6_5TR6j32_CkYN-^o@Otc=G6JR6C2AnCF{3w;r@6wvY3ZRA9Gj+N4g0*`E zyD;Z>suea0GUvU+7h>8gJyvV)$hco=FvVarHTO{u`HG!!E(dLAPp4=7dJYL5y5RTkv?*R3VZ0Q}@M)<~qz+yZ$4NRBjXYn<* zS$r`Epdz5FJPTXOu2zu*@5OTE`(<30>oV1_Qz1@(dihKlNWM?TwZ21FzDTD0gmWw% zk*2+t)>p@rJjAuPIcAmUJ7dbYQdbV6b8Q@%mqy4lOSp{@w?Td3&nSdinwu=WH8!AG zFvP8PJ*tlI6#-h`m|>Wkb>)Kqx_0^Y3{ZZDA!Z4Yy_K#Ctn-@-WS_4xnq4Z7_f!RY zRxc3D+pWT!Zcp#C>jPHcBf@h=fotRL*hEIO0>T>6DA>at%EWtD5Y`v!I%-8Wd=u4- zM$H^Dx8nfK3f{!^4Xk`9`pOs|&;xLN{v*Rmj9U-r-5@GGi>bhAO*fi8tLa%*SUAqnQ zs9i77wRVHyFVGcBs=Ouw!&i5!n${kfav!vMZb1J`o!EL(uAO#0Xssn+#Y92SX&tk@ z8a?J5p)Zb&3jxxO3BDh;KVdXB1Z?O+&(yojrs821l*`Nn>FqR=(zUl;3`n(la&6ji4Q_2&NDVo11+|rQ>gHfBtWoQ?^%7&~|)&lN~>E%v6V_w2) z*qnj1cBcKzNo&l8d4j;T4+J?hZs+>k=c^ebUDlXWPAYpI)q1tgYYW71n3B8GMf1wt zUd*|o6y1h4j%L&}7ZO|7_Ia8NjXqM$2N87;md004F0-54LD5nee^p4vma)cCf45}W zDR6v*GiKRtw8G0Fd#(JfAT&ywu6le{*&p?! zRvl`!S8wpE2Aogq5wQW_0fU2>mdduI+sRE~DMxfBB7R~yGjd)wX*%QK{Ap{ zd>$~qdTuxMc~Z=zBO=Vh$aJE&T)=3vQA!O+srS6!(NzWySB-qKq}Pb5oI7EOSEZg@ zmbRF1%_CMYg4#;0b7%msw~r1=5m%<=PBSMr`K~DMiDhcjf8UEP&%>^?O9Mm`Dsz!$ z{t%fb!PQ|PHR(Z`bn2c|cK5|tx8h}^8*lr_b~^0{0(XqZ6?JR?2hwDJUGTVr5f zZAojp(d*qWdYd60sZ{Kh6=5n87;bO{>7lor{Vi)T2+^5A!|O|Yt6EpfZTsDI@CE8eFR| z@qON}ufs%NJ@%n%a=A+bqf%Q}6MnU)qCv+wR;uL!J9hl{N+Frl;4wdxAPPiCSZ9{l z-sJnDI@k`Q{q-e0H?y~`8;f&wdbJ;Tn%AKc5u~AgsvS9B>0QLoK2X>()6RW8avn$I zl^PrPuC(lgl(Zj?s8QrRt=yB2oSrr(>!)#~H8@gDW~m%5$9-`bx^a#fd_UGBXM}O~ zO@?bW`nLUp@l5M?(EvUZ3LnyudFjv)mOLHdj~|s;reu5{(}UW4ik?-5tu#NZn~>Qv zzOjRq|BW^xDt(Dqn%15ohXIUCP7)HgRwx6Z(FlnvO@WnuIZA3-S~iC--_Di{Y46^< zVKED7UUjN{5^}ya!CvMfD?|tBpT@Zo3;B=jq$5jcsFk@*CJ1`!!l-M5l{-JG)`V;s zF0ul*gYs&Zr(EF7Kn2#5V8{|Eux5Ft{uohh@$f6IHJ+R8gn?b*nSogOG%h~7^4Q%daW7f+4TauE|*K4Z{Vfs`pD(zE8@+JOKP!l5D_yY z<*x(W7z$!4kF4(!S^W)0yjoIQ89;dIZ?ISkJ)rAM{(M{{A(IByFXJb$WbGLuUxm{^ zCd~_R1~VNZLcu!9rcCBb+$!S*nwog|!^O~6sZClR zPV3_qH-x5Mp#~4#J;D5vvsH`|t|{yG2JCk_veB!jHN~up#dpd{h%fG3ZPy{YHtjlQ z*Xu0syMcWp_KeaILWp_7`|HSNgO?fQO$P6jTg;yl{Jg!Zh`pfp6-X+_ka>3znd0qB zep^(;6~|DK7S$%GHd*0YyCvyCb()*Rdd~jhhTI7RA+@!>dz`%$8mpus!4g#Pg5GQC z!a?$HcSuVZr}clUh2|}aKMW!r`bFYSv6Ugj@-!dOVx3LgtfMop}S|zRDYd?3Q&t{j*yxdB_ zk6j&LqdKi0QUt(u#SO^O?N)~$vpv9Qz16qRuD5w1IdUrDh_~{hn8QDl`X#lmN4Ww-pCTBD^M}eAy9&Rg|V| zwRI|ruTIL+h!Xp)B1FIsoANfA%Io#pU#DgS$3HLw#o(KrR0$_VX=803U9)1tl!q`; z+mZ|uOx7NZmD)yT-Mr7P2d$lZU_PXe`O)0hB^QGgdO>1k;xh686Sj%IGulu&pJXz_ z+ORu*{n2WuA57~9F>G{t{ zyKpYE@x+VCdZrG1a&52*`DFItw(LGhGb#UPBze!$1J?T|)5cPb)lzLT5LwxDsxgnj zOh=xqCND3iX?-#lqj4dM#56aX@MbgLHrxbSxs*A+d?!xK9A15LG!7@^2rayVqer7dEM@^ZI#U7h_oi{|sB5fd# zLmgAHs+9AX$#=d{h*_DcR?$7L-?VJBE10L5(#TQff4(Ex-S~`pb@bv=^s>(?tvEVNvZ%HSQFS$>3x6x+6qTH3f0`0R2o5fJ;F)^PSUS*90(&)MM1voH)w zKA)hFii+dUvb47BbTjS8{b^3+iN!^rn#g^OPR`mS=bnU!T+@Hby(tbSFAm9+4&5$O zF*-GE1%70!$oxv8_1{e9CLahi*soC5_}sSM1|U(%H8l(if5|CBYtSk0B&(xKXbi%jlJfZfRbk>P!v0{jvlwz92yAX><4syC z<)xKd>EO}VU{aYW2KQDcS)(~D6u<#l2iexWt$^+qnhHp z)oN0T@X@gTo{pYTsbF&{l9pKBljO^?ycsgYl#|8gc|ImwA$p9_O7VGgLG`6&&e@87 zlYYX74~g8<`h(V9&8nmu^#}Pe33xD@6x-a-Q081l7nxAha`G`gmgcOC3=F&aA{$*| zIhk=zLXY+;qFrjrGg0z(jhegG1S{%(}EKkJrA5QWqHDkE*OrL>FA~D=;g^YHOkMG zZTPe<y|6_X~3hsI=GFtFeBeIHN|<|Z~5WY!pciHdxF_()n{u#M@) zv+z(>5hjaQb}|w(vib^HoSqrRS~t^L-A3O|jBPfe0h7Yk(85y$CHtz{>->fe+j3B} z{+z6mw#wS{%xE-?f-(b?eMU`fA0q~5Q$?{AjIGUQkhe4c+|DP`#I(j3KP=m+aoW@| zgRR}AiAfAn3q5RI4=?=#t;2mRikhy%?xUb40NC;75q-w(gBiD$pV?EYqMPWn< zdNekfAFE(8p^J82n0jby2egzde77vy0I@~yR&cV2PAL+JcN!U5$f;30DSuw=OuEVp zp=jb)+hP`{vDL=(4%wpyJ7|O!)=t*Q*7Ag7NRZK|yrLzRPx||rAF#hQFCD&_Kj%n+ zsNeP6a5Gv0$%3WFLwn-gB%75wvsqfQQ@woNRAbI3%0Sy4fXHDHKYe0?)Jos{XE<#( zW^04ib;WJ05;7xrTzh#~g!*!CGqoB=Sri^bX^E}2>P2ikz4lXh*f=7q*pW8RVF&AY zavO!Ib|vCs0!cnuL~h!CYh@Qn%q6Ryu`6M!B2<^8&`^_O?MhGW^Yp`lb6^J8et)_)?)|)iLz~JMj$(2UdflKnPng5 zYHO3_X9bp!P^K9@{w7|C31}35azNrY%ecjxTU}Zf%n~!1vyPoUkca1-Ob@G|5M|E? zfU#Od>S6V;Tx3UqZA6Bq!luEMY4a*}F=^@=KV$KFPPa_WoeV2C9GhW zyqcVHIj79TjE;oQa@F_v)RJ(cEIJ?T`evwLoEiE0o)lVd?IHkjk0_W$V)VH(yoHgK>_UYu}0Y5ZNc6O@CV(k+De`n8n8Ok*CY12ssl4C2i zd*J;-6^mo#(pNqj76sN)Ju)5C>7Vh~da~BUN^$%G2J?!RN5PS9Xsz01!b4EN1N=V* z5ql%c7=+q{;cAOIn`czaXHkDepH9Bz=6Cj*^L}S z?R{&Z^N*O#d7d=W$65 zB)$26(AW^EX4~zt%5M@boUmdNVmR~pV>cVT!>%^N7`s&iRpaZOrjis>yH0Z?uTB;W zjqSl*Fte4w5=!}K9Rap8-CyQ)BOdHkiS;Y&I%LK=6CpkV$C(d?W+>}rA7U_eTUx(7 zR1$99N!>wR*1EgwdJk=}d{rMiZ17Q$jS2)hC8mmRGbVfXhtZ?XSI7kH_wsn2q4g&& z311ah2O)sszN}$ZqfPqjm*No9u?Ir0EjnVKV()Bg&2vZy9vCqTf6AN!TDb6CCD}Cj9%e|Vg zs5rZ4GE4Wy9{;~~?U=eeM^pmL^M`{t%8`ye896~8yb97b-pbGfxv`1h*)!-7e62Mx z^wTez*B&w!JZsnIne-x}6Q8ActWczwOVp~V2 zusEBK-Sk4|uPS%E&_TKU=OnefumU~pM9b}K3+Mpxj!H8)1MUq2WluD8YakD+%^bDX z;vdU@IzeG!1zp|3pzmapR!?lPQqFUdRtFzsU6N~_9C2};4GNpPS&*(UUE2fLG=f!J zt?Y(Di2F>YDrGHz60-@*N~%tN)jIw_hR7DRZMV<9aMzwR4rtasUST{e#n8ns3@LE( z`-J2)K6Padga|(!?=iO^w~y-Ma<1kK~8lMfSG-|e-^cCiAuS38}}dtv@Mev$<`xj;vZL>cUOSo2kh64Su8zKGFssZG#_ggP&}JpKgPnZG)e0!NY#%-grWk5!=S+BU*4* z8{A{C`8mGF$6JKrET5@sQTMmOgKhB4Hh7l7+9YMeK*srn0bUZ&p2JcvYU9gNet5_X z)E~*QqY0d*XCX*}!?aD3!DfW!a23AYB$>K^wM6Y2Hy`5f3o^~N!Z5bYM`vciEkOlq zPvC}b0L^|2b4HKhE0-Lj$r4yOCa_PSZ~pY#(A|8{V)#6*u~wL8B4#SU-k2D=y_Y=% zKU>mjf^~@)P>($k9}gP3UA}L)Ll2Q)M0a?lofpDO4R6NU`tBgVnw&Kc`Blf^4*I1P z9GT&c<6}H?{$8fE(ncAv>q9&`&hiL9&^h_f#%n1{2&1cf) zbEeD7t%%j>@D+3)b$qOMm`@J#uE$z6+xQ0thUl60^D2356&l+x^fb*J8LzOQ4zCpf#c|RbyqN(TZEXjawjs+6r6$LhZjQp(E4P94f@G)!#wb!WD$O)vZ#C_%3|0o z?=cyN@r#$I<9zXIiaN(%6xI!1qHMOXZh{n=Vqo-~4nkT-<*@&d~?UhUA10D1kXt(Iuv4cYACV{bkz<0J)duavn+WLeWw`x>*; z+Oq>Hh>?hYHwuQg5^A^Ka~W?k#(n~BImb8yE` zg{-Ran_zKX-V>3%NJfb_%n&y5g11VsHdrz`$BVdZfebwX3;c`p1X(nj^C1LfGWVqQ zA(^fz=~V@#^&J^LsKS-7fRFZh9QKHKLom@Cft?^d4{wVnl%nqjlued(*GiXaVN+PN zLR<`ja#;P&FqZ|)iily$+A`nh@5rU4>uNG(*h~AA!Nz?k!(M{t2`5@ap{XWsnv%XG z_GTif6-CjjZbva)= zR76OEfx`DwU>FW4Yp!0_6P158wDQ#tUi$WF^Bx(m?0MZgukAh5`I^d^|FGcgyI=L@ z`R=cFeSFcu+3#8U;qq-OKXc7jPIdfC_%g#g{{1U2?|2D`U*%ru`kc!F?&9*E#JTU6 zojWqkxt~pR?k~W7w$r(9@cgO?o=&u<^ICTo_v!o=K-|K!8@fIG>q3jJhvyZ3pXB$m z{Jx#?`Sw2l-g0hYwL0Ea|K(@%?l=5)y5i^Z?nHh&-PdP3_iBUQ8gM1I3g`od`$)i5 z-DiOQ)^J}7xK4L(gdUC13lW;U%hyqIF9Uidza@7LSGAzzO0P`<^p=Rb2uSso+`ECU z;J4)J5w|HqpNP;-AYb0+BlLIi>DK}-b$0_jz;D(4oNIDR-lgtjt{yiVNVt-l$5rK3 z-QtLQW5AW%TOxFEg#Hl7w|OBb`TNuV$A+v&a$p+i7F;3&VxOMBm~S&o-R=oJAfxu0L++uZ417jP%K^MKyYZ=bt@tJ?BKw-LyX%oxy({8rsQ zuAT>X0g02--M4@a@jKl;7NMU+=;;8J+;4%DqvWRS&iFS2=#~7I+=UVMjtE^2YWmr2SUt)aI(Y9q0hRRadz@gC2{xUqmSX zw^@!80#tG{fc%)B2ITv3;sUSdRrgA8D!Js|2qc-Tx(gz16_C&SAMebd4}p_hm0T@C z*8zEX-Wefjvd4WZ;vSCB6F_Rq$?jJ`xi6}`0j{qw=)3^+xyyijo8J%Q+j4D$J_7V9 zeoOAapJbNn3-ReUfPCKjBXl@C?Q{PLm8NCD*ejt8L4B6E{GrRYuuLhlSvpIZ&|=lu4$>jG4AUqbG^W%v|0N$06<570OHJ=Oi3YksewIb1z- zc7)E4(7S*>$ZyGQH2%1FpU;j|FJ1`x%gO%yoq?Wl$B! z=a>rQE%o#Wofe?E?o~i<;&-llcZ9Br(1(G1TW$jK+_*VHp9PYX^ts!ByzSf%ioy~@24XstaRk54}q zp}z%^{FU5Y5qCI1XSx3d`X#?-xi_Qx9-8o#44NIG#Q`e0w*mPUtc=hV0qS#q2_#F{ z=Z>D81n9#NcSD4>N9YzHX%FoK@+IF3D)Kq~n&?n6M{lZ*qY-q*P8Kwspy2>y!LF5be{$CQRG(wG}nC-NZQ!vz5}EYJj?wD zkYuRjo(58sQO;$pD!J~jd5+I@Cj!YH%ynl1d5u3eK-1kjfLg5=-D&P!;Jgj{^8l6H z5RiH_*WCc*%i9^Cs{1mK=g>C;^hNgokp5NOe*|2~Jq6@P`nLf>uD+h(W(TP1ULA37 z3($0TIgnamEPf7HgwpM&!~ z|3##8cVyPF3M9#%>z=tPtM{bvbgr8dq1Qy{4FT$NZv~S6&UGu}(+@=G+DLaj&|mY5 zH;A}z0C@>}FhW0#Q2x$LdrpYZ+z9=CgjPoA+6Zk0lK#HReGbT5!mk4P+U}0fcO&#@ zgnkmC-var$#z}vlck}oyxr?~2HfWTq*Y3Xo@_zp-Kt9L!0#tQRM_l2av4eYurj8Z%Z};d9N`X zpg#98AX(P4-3}mmjXt+0;7aZrKx+M2?%P1#6F(H7x$eJ#ytVxWkV?6>*l|zRf~i2h z&9i|t9%s9=f&8exJwPS59!TS%e_eHd*Gfjt)p9*<3K*??EHq3G%;$)=g<0$heAP{u zmj{}~uGhVkXFcw=udm4>f(VcEI7e`FZ z%{R;;xT>DcbWQg;xNycVHnjZ$t?|~7`Lia=WytU`TNOQ!!kjoAB zxc3B_X}P2BH=Y}T=F-AZK6$5-Z;Ivomn$XPDe((|#<~A?6B3P==8v8~|LtZOW?#(p zggei2eLLp*iMzmPjs(n+zlA=K3k?iNd7ASudV+Yb@9FHq4*Gh90+|4YMwx z8?K94r}{q5>f!q=*LjiVX)qeE_eRV!ZoSpDF=BqV>*o>kYj?(t!44fCru%qqi781wCM?$q3080Mrl%vDy`oPb#f%?Azh`iPm8yVfw5 zMa*frreUs#nA3CPhPftU&dhzxM)@NVb8+sl(QJ#D6}hJj^TmjHckWrsbx*{s%KgUr z{)2#7?EWlQm|&6_Y2KGBPmr{9j%S?zi(Ic^UJ@}Ib5jg6FJeBJn>ImGcTT`KcTMgk zM)R(KS(dvdH)xnf%yms}rsdiab2V}=H=5m%rja|{X!b{%k=z+Z^PNaDl3QSyQ-g(E z=tgs|H_X3BnvdlE#L9U*mh-XP``Lx)apk|tdit^42aFphw_&cFAWeB)#C$xr(P%De z!+gkSR<&WSHq21OY|V`uW^)_nuMG3Kh}o9gWSFnCVQ!d^qxSs~^Xc48){931293>q z+IsO+q}iU^PA(#m5Bq+NyU*l4XSsR;hP{=5nbC&%f^lq4z$}Djw_%n>%;$4ovbvT> z%olQBwKl&yV)o_kGMcL+=8oLmR?c4q%(?l0$^DDv+8JrSmHW-5z~{b?b7JX zd}lT~CnQ&Q$=2-)G}ChZ!a(!AfEjoF$+cGd;{oH`O9Cc$y)Q?Yn$b*Z!!!&ts}0jM z%#t?DuwmZThPlo#?`^|eZ;EY}ZXu9K7P zmg~O)=IZ3+WQSo2AMyRVI+>aLt?_U=m^1iY=w6omeOGQqk6RLG7P{HV!4S)_eGk^lbem^FC)z=FyhZg0>-)3$u9P2 zd)y}@&FbV%%XLe{{AqH?a@`SgtxcXXn(s%NwaL$|oS#R`70Gi(Q`nS^^2Q{siqBI6 zW}*9V@@m7J6=|+ZUTc_h1LpbU#^jF-^Y)l)OY&aBtc#dWCRZ5d%Ef-4d7-;GsTt-k zBhBvQ(^c{3qX9GSzL@N+YLq`6Y5oq3tnuA#n9mu_cOvG?$!^0OiI}e>dkyo;h`BBK zieV}@WaEBY^3|$1KRIBW`+D*xZ#QbCOpq1PR820Lu!$$LoNb`;4zl>&Yr1@UbGf^%4r#8%#i5jo($6Vh_PT&MU zk9#U$4&;B3oNBoWAM-tR?r<{OFw@&Gr&+F72h4%|;biVaasEwU{+Qo|?vZ4^Vb-=` zUS+j6LayhNN0SAH`BbENJh|L3Ux=7zlWVPfZdH5f3Wxz);<(}wvw%e5%xD(AmyH0MN`a{ex(c~7LN z-!NZ`m^A+bD`$ViOv?YeVZIwM zcjTw%U$9(9Bh8F_iNCkO-~VLI$a7;xzHFG^e^aJkC+DXdW>Tb?nSUu?#OCjI1kAWQ zH9yN}u6Tb|&g_7Bd7ybJKPUf64ulYW#d7B6U)3Wy)QXjAcW!R}HCE2s+h|^Em=ytY zS>YA=C06@~0%qKumH&OC8H>4I9WbAWm}U8TkL2*xmu4+|Z~j5c_1Q@CXZfRs`C`Oe zmGABqf9{N!dj8a2wfSJcFqh7c7|mmm<_q}`8^+z3)wMgn*)S6W=IZ3G{P%msjngB| zxAVU?nkU|!<@(qBZ>_GyfyTK*`L0y*^9M24q5MR{ygOnZ&YzH~&FceZQSxa16{+gF zI@0_!|3<@nIAG=#exAS3a(ymf?nr)~zsN9O&dMp23s)QFA0tgip>CM(M$Cl5^@jOT zz$|pr3SZ>VUyu7u#LOuCt6_TnSJu;)7oIlE=@IkV!pkP97fT~%N#Sh6Xh!cNhh>E~ zOj5~z7-`;ExXUo_kC?X@H3;i zrw#LS!+fU=^Xw$a#=`;Q+{K05WNFId0W%J!d$QX6Ld4^Dw;V%r+c~e%`jfHCrGZ-CP8|GTWygy(&`17>4!u=odt(P;v4eq(V)@y&*LQ>2+uyudL3`axg2Xx`fA27BB^ zfo586M)Bf4H>1bBFJP{N<{ji>P(!Y9_tN4!4KoyI#@#GoXyHc#W_9uOfY}u=3*DUJ zdyM955woCJ>(h9BCt?;AZ#0@eKg;)Gd2vzk6Nb6vuAKEtm@S5>{e#Es0khRG*UxX| z`fJ1ddnoxfXl^zh{xo1z&fgf#kAC0hx~sUjxWh14wP8MMn16enr@0iGU55EpZ1dvc z=M8hhHqX^di%X09472qYzJ-?-&oBN{U!I=Mg+~7ty7Pkk9wtC8l9ix2eW8NqJ_%pJ+) z#qSyBJCWvH#UCD5&U=cF9oOd7#UC5yzhkaHE&jJ*ej73GEk0qGDSw@D<9)?v3^O}o zt|CMc_>$u=A1}^04zs!VQp2=LzOi`9ahR>eImhMNS$x%T zG@mOjI1aP7xb!&8*NbmD4s%cOZO38$rT9mN$$dJ=M&VF#m0>1j7!E$%hUoe@(k z-ENrgM9k#UKN{x0BBrnO&xZMJ#7ryQYnWCGPbz)OFq3c2^t8Wp-~?HT84)vB`rdK5 zW|R(_JTHheGfV$wH17zQ$;q66c~=|eF{5ee&YaTYhG}tQZs{k6Y3a@@N>3VQU99Vj z($5UjvTSFTo;FOY3O3xeS;}Nr{^c&;GXCr2D z$xW4QX|;Jtsce|Pk2Fh59foPypWiEW8Kxy;uP;>%(;ACqr5?ky+Vh4|I#pwFe=PZp zrO8HfIAYE%on)96fBv8}V3?;O&H1I57^Y);Hp-WjUS^n93;(Dz+b{!>=Cabu4b$TD zJ4>}%2$M~dWN0-B zlcz~C36n77HVKP#4`C7}$q*LB-2U&+_wznK=fdy){eQpX|92dY<9)o&@8fxW&&TVk zGOf(mSU)evniR(RdpX@?TrA7W1e1$n1HF`)Oo$Eka)-&p*x_CtHTg#@+sjKP#jzn? z{$+AiY^awnO-f?J{675EWJ>H*Z!Eb3#1bOLP?E#`pCS^>t!=e=ql2t&`#h zcv)gn5Fg+rn%BAU!Cv05GUvq)_p;VxZ2Sl>(OL9@_>o?sCxvnGqr61t@C)Nddx@^X z7sZe9@`JTp6hGEW(nH}{>mTuxyhQ8pn)qlh(R#i%e!7=v%-6@y=_%J``L0C%Te)3% z{5)SKI+NTGKi^BV52wX1_(Pc+;}?4AW#gG1|GSrdCNtxgdl_z08K2}OItK5EU+d*O zD>FBKotH@_cgD-SMCZf1;#0ju&uDkYZ}1Y$qB?$~muMFE#BcHv?Xh|Bo4rJLAos>^ z@p6;(eqa1HFV!aZ$1A-oHhC~!)e|pjT?6IxPBrm4em}3XGNF7!$>}^lewQy3o3EdR z$lbo>XcqJ1)xOMbEOSu8g7`cy2U2$Iv^f5hmm!o-6YJtH^prpCI*M{urzP>1eaq1? zxFr6nmuPQ16MxN1bk=${{<@b!wsV+!KK`bc$tEwxS9+;{=(**k_}gBhmS2g#??<@M zYP}l&z?X?W<>m2@ygYAZUXOq5z?EB?WvRx}W0u}PIu@-2Zi87@`~+G02;2ck z$NO}Qr2^YJ%~GAEXarF!QOjX(X>KiQWN<+sU3zSt#oy=ci;D^0M_EuRHRM1m6lh!nl3AeUMSIu--n^BK7yF2`SsBcBJ7xrQ;U|r}&g>sTiri zTQyRlPYp=APg{{LfdwuF-)!rs~ z_j6z48En;KW-Zb;zU62ggr9!kV)!;zuU2~^$&8|XpXQE+=~;9*w+1fX627!vwMwgy$NzM^>q8$E=FY69;msq;Z(!G`*vGl5?k1e%X+HN-; z%K*10(=Kj*O9L$BF%7hGY3@RpT+au(iQL-piaiykD?8k$fqMuhSCUNkBGQrAR~={m zHcUR%(WOn6)-nZ7t~kT|-u;4gySgZS56kmggC5}#x(Cv!u&!=jq;rt+N(UjyzRz?g zFa<8U8yO3`807*N-j{TBB`}$X9zD@o(`KAz*Q3MO+k$`_ZZUk zzV3gj5x7@SZkiv}he*0|zPHo`e}9e6Xg{QzQLf`YJp?9oN0$y_$}2sRHS$VNv{Y#6 zS|mA=ySlkZawhC}J(4rm?f>Oi?q@53dyXm1y^D00uOWZaj*fm4lHLmhy+YfAN%ynX zkb4HLk&Uf2zLhhqMznRZS$Jm6bQQ42P$Sdb$JFs^`mEJ>$I?biKUk78tDe#S+(+P| z&tv=D!;$V~=_pHQS}I1;qhg+==(Bv)%B{5&?#aNln{`hOd&@w2!ta+{r00CP4Cxgl zJ?l04Bv;t?e3BLWrBAYAzxPSb_X%G}4Y`_j@o6Sn>FbkR16KPp>oc$nZ=WCy_9?!H zb2&a8f;7e_xg#6v(`m5Fe3HAbYmqYDbud|1dX2xyZ0?))W}=V%eOiii1X8A3ha@Wkk9PRWoOE49Yp6TyWRx3K z+MioTl@7CXilwoZF1K_&Q>L4Rq;p${G{%oWo~_PD(&qrZ>-#6RYI?`gCzg8Q@2k?S z$1-KQvymoPd-@5=rz)RsrkjARmt$+13y-ZdcO~q~KiCbhGT(|^-=-twl~%%ZZ#;^m zclFZVENl&2J(8~2kCEhR8n`wjsgdU5eVn@k<J_%b-gN-gdhbgc0a@Nf&oywHvW+KV+QJT98sleL1 z04cBZeyj0_r6-s&-3v$;pS(1)p9>!`ZAzaee( zThBuJ&Zlyutv<;y-ED)^mGzwJldN5BPu8xaywb|mOir7#+2rdUtJM9?X4>! z&Gki+GechKVMu!XjzZc8y#;RDtK=W=6yz#-h%a{`YG{&c;~;Nx<*Zfv zC8MkOb?8^u#vMr6zTfByD_7Iu-sEaJ(x)d;?hGXTWX?y*bRWZtefkke-*d^)c%!ee z-9FeuNNMhHtX)0V^?+&0K+?T-7*m=%5lPQ_a;>-|u$WJOzC+F{JrLzm zur<^5N9yU*2&Obwh@|^j?g{lCu9%irT4kxm(i2E}25CU*jrP*qN^Zq{EK}eHCF1V@ z@S~FBO%mR|acf@bFP2jG(-GvA4zP5zrIVS`TzJ(@bK_wBeZNzXvVFRprh8+A?^ixc zO|mvLErXrnx5|B(Cb5}@x6}T3(+f*;1CVa=DZCpA+%TA|tH51=GzVMth`QS+`K0A(Tki?wZZmM# zqMWvT6VgK{*KyU!b!HK4bm^VgDhc=FHmV0{9c#$*h)6#pSF4&q^dKfGNFY+dl77;)NP4%_gfz!*ZAV&wl;+YioU8XK6X`XdjzU`P($vGlAZ|KVT z80ES;*}Gr)BX{0QV&{hLp6tX}Ms9II1( z8ipG8_;eA{<37zoTIthsNT2)k8PZmyz-@mp+HaMVhSVLY<54k$TSu3k$TZN6LfYNe z7=zRYNzXo#GTphby}ez6B=g0cK2u)l?MO1qywZm)z08#6nvnMQEw4k8_aON8i(3P? z-67$zkdBmza%t`$q=CLh7Lpzna#lXVZyjkh3XpPOd8Owg>8E-rlDwtExH#q{-jtOSBSgNzM3Q4csZD!pvbsm8mz@)Rh z5B+AkeAq&t#xr$12B%mJITt^Ua=Nl#@yXvK`6O4wuYD_(sPQurzWYQ<`d0RHH2Ub? zBAUk%_JMmJq^_tj&^06NjihIl1AJPVE6V*yuZBv~7QJIm4}OS6!4 zU4?J=_1%)Rtaqw<7xXl1q`9|{q~$dC5z<7IOLH5LCi$^kj}d5ZZLlfcx(o;{1xaSq z@hr7FOv;Te?ZXtf{gL#}@>nDt-vwr6X7?jaMP1$Z|Ms(d4yGfB?is$vijZTWW54gC z+;rcsyh+v6Y$oqi#X4S1Ps1L+#oKU8C6@kfDeNt9pP@ay&q>S*Q*?cc?wj_;R-NUM zO!_IDg1e=>(z9SPqfw<3nbO=1R^z{aF1y2zS)R)@{r`O~(>}I+^19cLU>?T&h)+)= zJ>^sQ3>LWXNluRsJ?CtD@2Gc?a^2S?PfA)&?r1g1{flmu^S$gpoCR>c*LRk3mXF?Y zzJL*Qb#hL8-6y$Df8>)~oxku&&ivo`ByTT&_DR|czLUDkx@yWR{g9tb;5H-efUSX( z`R?q?$vk@cB%|NkCmH=AKFQnABYcuEALo;_Jjy4z3hDb@c{4i(mgaUkRF9*M=d0+u zyPnvpt0{VecBq;Bjre+eMBg@#w5^kn#-fkFO*NZ?BsI2kHAwnwuzQ^n#`E*(Gp=sQadGZ!*#!V(K+m)-JJW>m;3XX zOOLYk*s4dxcAF&0@wTT=a#uCTr(p--9vCUFbUyl(J(O3v%u=(ZR;Iw;;pu%7{(3(* z26fY1GLq~+%%~q)Xpyuh>?~{zTo>#=xo+uM^g>uyC&$)hKFRT`X&H{PYkZAN)V%>I zuk>gn{oMY1g~c5Yx9aDnS4sKYq_=_YdTgDJRtCD;n9|%Vq+4KktBZ6ulHQky>649` zJlUv8dtzzsVbplYr^QH1eGR#PdBZ2U&-olFue2Upbx*!&X)RNlJ0(NcOQvgu$$d$t z+lnOb(W7ti^hxs@-?FSac?Z?i$yrL@OKp2*-RjH9=d0!9^X>F~$M&M{EAShF!@_+U zeixYLQenD^`!V4;8L2B;3EZh>W0AUhyBw*HPq%Pu;KJ)=;KFD2z(wyZqP4pSHDulE zzCReP=!(@R*JWM9XIPohFuzr1G|DHrLg;g?Tn)x~lRIRoi#t5DEbm4Fw;oB~DMTZP zAMRW!YT%a$NR>Vfwsbs_?t{M?eKgCnQ0@WL?f9FWtE_JLc1Ew9IwM&dnq*yR^6Scv zIeJRC8?ESSmwi?1=O!y`6OK#0|I;M*&rkTR{uh$|x+<$gEU)w-wvy?dMAGkP|{J{e7N zrqPyHVpQ77W+XlK_c$Und2g}Vw{k43(+@K936^e0()M0J+TCyc0_gyscFYd9%6``S z4cX5}`K_|A@_mwhRqT@-Q8SP--Jz&^pHD-O9zw$Tg-Jh4vA_B)gq}B^@LQ!{O>+OD z{mOpPer2C(zp_uI->y#fsn(FEz~{ZmQ{calGTp@(fqa6Q?kc1&VR@z3BkB2N4pW+2 z$fQTqHz?Qf?pof+$s6I(rO%?Are#c{N|&?lsM61w2D&YlIv=UM4RpI(%CK~pr4g1+ zvs7qF{$?c^^FTMl(*2g6vh)g)+&v6*ZzIY5#z5DK^vB-@Y=y~NA$(^uB-|SZSsG!f z(9#r3)s_}94Ro&{$-AY2ZndSam@-|$Q96Q5w=4fAa#T3r}bCAby1VGt4Xesd-xj5aJ*@fH-QIwleKn?PqLa$^GV)_oR73i z@Em%ZAj!RjbU9L*`wU5c9n*#+zmDl>$wU8a8D>L~wC>4fe>a1>EX!b9&4Q5flQQhCHT>3HLn2$u#F`xJs zc9w0OXj`L}%gyFmxrfc_&E7Qo!t4iH$1|7wO79x1*wLlGW2+?GTjb~wAGjV!rLerx zeUS87V-V6!zT9xLJWFR=nusL7O#iDhI{s_{w(2h_=OD=uAGjJMy*fXSG}7DqmcB;1 z)%UvvX|_+lBhB@x^RalF>{ANT13vAKq^*pwbT-n%e(MEDwLVQidcvpcEma}uSROHqg>>w)Ip?MV9_yX$q3gZ3a`vm3Rm2YqUJN^d3tKEIn>%sioH~eQ2r0 z(zlj=w3IkP`v}}$eZIS3tIjf78{vBC>Uv}ACO`9Nzw8U!4C^=#xfhW2H@dVRwrY}h zZkpsxo1{P1_{G=AMh#78G6n83Bv~Z`+_gwDz5#9~Q{e8iw20*bx6INTNWY^leg}%x zsq6kDSYn%4D^h2lb~rvv=}5XZ4!3jyQWszDbR_)*FR`uHA?@O~-ip-2r$tD*C*QPk zn=SP|0dIwTxqe8}+dwCObDh)}=uWhBx~1`!uCX+oDR6fp$x+eOJ%=PmKv(ywr4N{5 zt`$l8=$N+9y1V$_aF>fwb$8v64nxAf?S(YLrwmJjWUFuGNT%-Yc%(dEHy`OdpUyG6 zh!%5`kYv6wcQevN-|{_3SNZfXl7DaOorHOI>}Mz1F9dtkLo5 zGyFa;=9;m!!g{Opsm1#E%2Glu-b7<-;Cdm+2m-e+lAam*GvQz2Kn-o>IGDWijk!~h z7WkIWN7B(>g7lcTZD;Q&jgO-f zOh+q;o28Z6HY(Y>oxRP2b@54TH=n{8{crYK)Y~K0Tb(88Bh~lOz<&Q<_3`#LeWd+= z=|e`pk8e5LYyX?kODhMWmH$oMXzzZw&3yZ9Gpd;T5?kdtIp%&q(({3=9R0n(e|3(K zV_!#gs2|^dRqk*b{eLbe-!#PBZ#L4zlftuWw77m5jN*Nb;v) zbfo`zM#tMo|8u#Md>;p*wHpO|~d((_2;eYplp ztC93NxYp9oNYWerUA2?L)Z3D*KmEN$229tV{NCa+-|{iEm^&5e3U8+)UFXyJNb)Wt z<}O8=?(Hfh`2|VfZa~sMr4fBk`~Xb%+6zeXraa9xBhB`8*K=#keP?Mil6=11om}nb zqMY7QEc8k4D3~yh=e)U*;hn+`NR8gQBgu6Jf3=@&&0vbTK}fRdI;Ih3XCUc17>m@5 z_M&x#I}h7B)smbyzVPL4F{?t7y%BSdAj!;Q?r9|1~>0+_C@;HpQR2%8smFA0qLwijIRLpcW;GAx(_Zx()Cw@B){$H*he&~Td=j**PVl; zBUoTb>dG%OWA1sRDZboWNZRrmOY1E~`$a}2d#&T|C%3}neW&i{>At;aEYV)ueH6Zr z@>}K8(sBpEX2Epj=r_Ut)fH^p8Zmbm>&9Hv+X*n8$LUDAhsIf&Y+K8b^p0^hlAOmo zj(*!w>GAfTM|H2YJm1ojmR_**x}|rKwBJuHePb!S2i8$l`@I(4;lrey#DmspMJtxZE-PZnZ z&WT!M16uydxAzm$w?1_q9Ul8BNV*Twk>uO!m^%dN$Dbt0-q_-kyczoqDR75lYr@Zx zPDPTs_^*B(!G{sq|s5PHKH{xZ+mn#J&tm^ z$K`Ibn;-pOt^l6%0-`BST4;)zZ)LH-th0YBaQf3W_gmNHc6wAbie%FQuI!t7^c4t zlt0@iZ%qgIUnQRMv(zocR=xL@QJv;3IyXh1>R*-n?`Ns)MdkiH%V>4R+$@Zxz>nYo zq(83g#jx}Kuyq+se$%06)^UDoboG-N$*({a%Fv3YN+f;S zxx>-}NGpABPgr`@wyr_?(r^6^=_j9lMbay~oO9e3*?-X;Su~H%r-xUU@aXBdu4Jp; zBlSe;>|2q#U3@wKrblShUbJ7LC&=hnI21MX?EUAhy07H(`0J;@sFkD8ihgd#BkA+P zXrx^*Ds4rd+WzW{8QuMlwN@q|_3|^CXlXK%wtNGU{EAEegrI!CkGF%cJ$#J^ko4Rc zm0JkY-vP+qnxi#V!}|If;W<`+r!T)zkGapWRc91kLpQ?Y&kX!``z3mc`5ra&dFD4H z`35WIwm&1><7r6RM+TA}gK~9|`;EWehI-3DlDdKWt8W>HV5_dC-2Yha3~be_+6743@1NU?xht?$ zzJrXpX-LQVk;>5{cc55TuoJvJjHKrSna9cA!j*$3W^B#(w(TnU&u3O`MfyE^n|{+= z9ZdJ%GNkkU*66wH9hi>gUr6Kq)@Xb!unB*#&9KY8MMrI?0(|Rh<@~v+!{6!D9b4ry z8eQ7k(hy50TRPp+c%~6}I>(=&a2KD2Z;;VKAeRp>Qy?=SiLS&;rK_UETuxspQ|W3c zNp2J6HAp3DZSNBI(lTFAc5pqsOiPNLjo*BuOtS0ar7WpCfKFE$Pi)98u z9`Z7jWk#@_F80vb%V(KVmf6V_P%2nvCpVrl7xE}t-q}s0)Ic8hQbJkG zGWhiQcN+OPa?r-Q7sa<1EwN6;PJ3%x-QjC*R>@l5mb9AtM_sFDCyzZuySepNI1qWQYP}Pso`arPDzWX~)WRIuvrgmx;cw za{nnynWJ2}_MVi3GUL5eP)0#6@iN!T`p#irL*0E|%96&S%p{Z<>S|eQB4i5WXjexm zfs}h$Mwt$|88XbRpv;BL^df&Y5vwI>0b~y37}rdxgUG5o#{XGGDYFc6H_GI=O)Rqp za=({$$|lGH$gwW*5N&xYq}I#Ml!URkr-BT3Jt#Xvp7A1oW>Na;4Uu+^a~YIOh_rK@ z8%)W8$hmHW8%h}kk)vsZ%k^?Y(pbogsCB%{r(6nY@NzDt1o9^21UH^?8{{1?#gqpi zYaoAfC6uL*wO*!sS>HKaExB%im*qjY5^`OwmI=buf}1A)-||jxp&eNpC%U=)gq8;% zLuA#R==x-eOiTJ6@`acEC|e=2e@=A$D2eCm{yEWQds*(nmQQk}{e3&O4o`A(y_6^I zgj(OBos-;sl>H%_z0`VHZeu>#E!KA2F(@PZ`DE8f83}1et&y%^z_xwmxrr)C;r_{U zlPRN7EBFP!hj8VTiy?8yDXxNYC1hvFC^y$jS<+NUFG#+t@v_{7do|x}8t8j>Gf_s? zM!rk*r=aL~&v(gQ%9G}yOdsD)3gvmoK3@8$_&s(I-* zmrMB$B5QTD8$|h>ZC> zS4+7DB4a+!E%j2CRLOS6x&|#{YksV2@^XWHreoa(FVT^5zPs;m**^`zqiE+u?6LFR zVlPwOE9||{-FJkPDRb{Z@=>PH6=kcmLC%JZb9G0m#Kz%03}n2^@)w9QwWV4x3-LA>QtnD9`#`>e z+~DL_rZVPZS!SB6P_cV}8=d_AwBr@zMt2`2yn@{5YE}HFvk5Ivck&BW)bjExWQOaP zBT|-h9r}{9`ps@UWi~|C#?5XbWMa~$MOmw_i`sWk+0667v7-AkD}4ss6U9#=ydMVaSXI><#R zbFbUtFQApK5M?Gq?sustNy}w!669LQ18(NYDz`&sKpu3(c*8Aao`A@BYFuNU$}-AB zZty6TCde$5neWQ+O^%GY1@bWDVYkLhncE6^4)Ta=WtsRzc*71^=vLsH45_sTxayCR(_~WjIG65oI{>RDtbl zD3c1QbFF8pG(q-;JmVG=sC)rA1oEs~=ViA09x@Q}yvxBexyb>C_Dccw6=l-T^qV$2tne9!tjuKuO8{GyPUG)pMmAthL;Ys3mLlEmw7}v{UA)A@T{n<#s+# zB)T@e?e_Ch=4x4M75-SY)_Rt`uX0shDxF-L?nFDQ+!~e%uU}1W@cCLxuBQ*9%)73t zP$j$yf9N*hdMx9StMKzE)9l70iM)xv-hixi^^}hwYar`fADrK%%vQ)c$Y(AC=RsLf zDHELA0BLcBUZOjP^)3rXh?EJhp6gw%mudcvQI6a%T|Q-hv?HJ3m+o9IQ9B!4366*G zswaDOgDY1_%0ew^XM?Mt9LaVzxVaR$JC!nDx%((#UthUe%5b)`(bZ8#v7L=>8RcA- z`P!|ZOkkO>ogC+K9lRPME2`BsQ|^G&df7m!gS26fed9J!UV?1(((Wai`?szY$BgXx zH`va%t{tC&$U2DZjqltFB$01SaxPW*#bnh)l`fZrwNfrq>1)zT8Df(94=EF^`R`o1 zmoj%2%KVNof9D2MuA*#mxs(dZ_wHQEy_6qZG380hk8V1pk<#X>y-Z7550OuBv#X)} z43X8c*)66dVaBrOe{xGH-5|2(e{v08qP_aFOS)3#wcLeIqd&WJFO@D8wd8uS#SQin z&3&sIs%7k~wbd=wGPa(#x}}tGjcs)el>QvyFRqbt6i4`rTSLi($V`8A>nNu|WTwBm zRxi;k+Fj{YGK;BhEXv5arQNl7iN^e!YxPp*iclu@8-73Fa;}zIW%j=JcUS6VT2dLx z$UgktO{d%hk$w2PtD;o0j0>tM^H|0O3sn5wsk|o%f|d@_1-&N($=68l(JW%YL@(3a zqi9Fgb1cZZ){oHDL3Tr}SWw}m(!C7n4M_@Wyi9YeAhLft2Sue?>vOiVeUO72FF7uL zG3j%?NTu5WM}vII+Xv@TxS4xmW z*$9!(DZW@6j8#b(eA-y71tfLPR4k43$kyJ5tg~W z5V_adEhzR9eWtqwsW(cQXdUhr^z%~Y`k~etzMTU zr5GanFg2*7l(NhoK|SRrmf0g%K?$FLdIzg0;S*5rpqVm@?d%z}P!_PAJ%de@7g?rH zu!YjZGJS%?>3+{AZDg6WAej=oT*sUi^q_=KZGD42l<=voZ;(Ohi(2wTwpWlv$%M#f zu~(2y84i)pVy|EX|Lwk(3J{MP3Rh;k(McgF;I9u5#~SqL*kz?Guc-N#<4N zE=4<&(at`>xieHs*!z9~9!T-uRCg2PdX(8e81JRb&4JtkIUs1KJOHVJ92k_}s_jV2 z)sTaNddgzT!NG{zw9GPy%=D08nU~qF3GyIn^$RvpzNKUaeP(K{9dP}TC(ZsrHRS+^ z^fe$zpCx6cx}zYAAX&lS*;?x)$kUKR@yG2{6FPM3BFFNupwUaT>W&HK-r+~+#-YqdnCUS=>RgdB zR}A?Sa%?cyOQkE7GMM{u!3vhS4e}Mrj0ke?)LM5#euU%(O!SJ518E-|0a;r5i+6-RVICWp9Wa1*ZqAD1#xgSI-F6P>zPkUOgjdp`5@n zX9lg5Q7m(2u!Rzy!wc|NX3M;$C57kkf*_d^UaigwQYhio>a3s-r2sAOiamCAupi|g zklnpxd5J#dbAqAwNMB`c2Fl31&IyX{6{&R9kiMvOZg8KMGJnL$-Z(GFxKGPGj56}M zpBIeqQs!P@J7a^Q4zfSmIX~!eztk#ojVL4cY~zB7l(m!#gE~q(Wqh#T15yk3Em!Dx z{vIswGS&B$g_bW0rq^i8a-}{Ia#@f%Uu1o-2Wp)Oxje|Hq(Lr#ObXU`iB`)M!4@x- z?ogDG6?H|B{IIlB=}v?cq1Kf_56T6Qe?TS&eY|Kp@;%X2L9Une!L=xJHOgEa6 z1)IFgaq@1t7IJ-%yFgm5blc(1dI_XFD52~Ic^PtJfEQW#Pg|Dnwr&dgs3e8oGtLP1 zqlDix&Iqzp+}>y>7qxBo3#H}R?g+>`kXwSmkBT$|e}jAoxizS! zjE2bf_%nlMFO}|0h`b-06|{MomQ)1!47Fwl?Ow`~u7b3BNvf5Wbr$mcUKO->sdUp& z=4X_d6ZCsb%3#ey+P#dSEP({SOPNy2N-MKK#m?$;f+jE1l0HQlIa1~X&0flqet^jF zH7D4>GTTko^Tr*)CYI>|k@LnKLA#dmPZRPz+T0+2k&f^nl#wyt8Qe!14v{h68PrkE zHd&!!BfK-%z%r9iM&@;AQ2e;Gygmq5)SbZsFVTv+J6P;xj+=#Aan!my$Xe{jV=d1M z?)#H8dWqVZ7c_Mc*&FkMbsa?B4c;5Hb&#Iu>wf%9UHa1R&C?(c2FYHuW%+jdp&-@E za{q1DfhaRSDDhHhBYZfR?q!a@g7in3hl2%_XE3kBA&&$tl;sdPvn>oJJ}JG=@o#&M zLYYT{bDt84T7E1%PIvqWwd5)s$wr7=g(KO@GK;KM=c}|#B;j}Bk6W4WJMl<*qLy5D z7F(HvA+mlpO>QpLE>{F(HZiWAjM0WUAun` zQYqoJ``2JUFYAL((AS;lz1>#$x9r^|q&y$BET4``=;vj9=O0l<&PYK*mY1@mZdc=+ zigtp8Y)Ws4e0v)tjL=%a0T5{?At9GC6tW1lwo9m^oCuNcB0D8CP|k+Pu^dlWq0;#h zh#a{|3C%1sg=LZwS}C_eq_55iJ?do^Wo{m13HsVTA>RwW8-=_ANlsYcWqs%Ho8lc4 zQeTi->jQbFmOI!T6Z%lX{nI6(AEl1HcS$JnqIcA9pyizsDp+O(%B+IyoUn$n8X_a? zmXQ9U_Vo#54a)42P~c^bTMvV;anC)88UgzV|1fpQ^4_F-y5Go{3&jdHt5@;{~b*-48`(kZW*WK-6eb{hN%E^Yi`7)DK%55fDl*deRDQ}t-Qob@Np~RMUL`NvC{jl1=G!V>q6C%Kj!rloL(LDVLd4Q*Jk@qdaTUNcq^Lg%V5; zdvB-gZIaTcvpC))gEGM+hf--$Kv`%~OnJ?ug3@AALkVsQ$5T(~ZPG+J!lacV-{I)- zmAF#JJjo=LGRq{3^0-MZrO~92@|8&mB{n1MtBSIhNi8MEq=E8xlV-}TCT*0bOp@Qy zS*$Thr~GD;P3e1cIG%jUaVABSB9n4TrAakqiAf#hBa=o7e$L7L+(Oygq@6OtB;{?L z#ib@0lsP6jlow12C@m(%lpSsfd#|AMH>shVZcn!du$)LPsl0*5_ zq=2$rW!QT$Wp9%T%5f$&l)szQQ*JhCqC95ON_o{J@n1UTPfb!OznNrFdd><*m`gd- zq>z$lQbL(vQboDRq?YoKNdx6&lV-}tCT*0hCdo}Yi(O}jBTT1cnq*V{W|B|&yGaq{ zMw4>NLnhUfS4`?CpPDpMem7~M>`@htu$^**Ny@uAi_=XqC|8=~P;NIVpgd(#OnJwo zg0j)1h7!L$98Wz(z6sZBQxoNAlUB;vCW-Iqn6EKOrOY+SqC9PqOIdAFNcqmBgpxcb z98VQxf0J6uaFYhg`6kVjsU~fdhfR{-*IB%2l1}-~B%89!9pQNLDT7RkC}*0KQ?4_q zraWj;M|sVpk@B@k3uULdVejpf0VXMHbQY(XWKbrXTws#?q0VBqNjl|alWfX&Ci#>-s>8mDD1S34r(9)HO?lX)j`FTa zBW1gL!j@YonI`R&F(xS==`5z2WKb5F<8S(n4u9 zX{RJV5RNBht zENV^CDW8~RQ@YI$Th6CsniNsw*D-wmL%GtVnsS#(9pwd+M#?&q7Rq)Hhb^~L_AyEM z)ZxF`NynRHP~>+mI-VTLZ6*bjCryee@0wIlwwTmVx-JM?uBRMq(nR^2Nh@W%N#bWZ z<{L~>DG!)rQC>31r8JuqQns3uP`W=7j&vTw_vAnP*Z*dD*0q(qhs=i9Z_l-cC8dB&9`XaiU2E zWui$A!O%lJ*F<)kqO1ae}i}I*R zE@g#DA>|8`5=z2j;drVjdzjQxvP~K&XPPuqt~P0-+-Z`$UT3k?B%SiUNjBvtlYGjq zi^35WQL;?RDWgoPDVLknQ7TOuDT_^7C~ugwQ{*>LdTvSiQfCo+Jd_Mdx=9XYm`MTU zT$5tT6q5={wMh--Ig@(I8j~ibQZl$(kTN>vMD1>@+oJT6j3fW zDW{a1R8#IUsiQ15X{0PSX`wWmv{QaCN!h5gNP04yR|ciGNe(5;q=0gQNipSIlM2ce zCN-1_lX}VnCQXz^lUB+vCW&9`nD=`s9CIpVw2AyJfO1_fH_4?uX;Mg8XHr7xQWv&d zMaed)rCemvK$&aOOj&NyMrkuiZq*U?eLCzbopP*6Hl@f!{`Gm8*KCs_%5x^=luu2n zDM?GhzUnB4m^4yGo3v1_HEE~Rn52B8<9X90gYtt(4yETaVebW$p(gUL(aXFpG^wD> zGO3}wU{X)nVA4d{Wog(~D`kjD;C_$)|LBK5R$+R#lnl;U?vj zOHHaNH70eG4@?>QQUZBjwG$fSlc z#iX7x%cO}i-=vlDoQeDkWHRRWOj0R7nPgGA|1+FdE@hxeA!W2l3FQitD#{#_TFMfW z2FiOT@-IZm2!AqZqjY~I98dCQEi=$0oif@an{tIoK4rQ|5oMl9Ii=2|n(~H89c7(~ z{3{}U%qA_A1voNh_uIt6}enKkImoG)bitm}F6|G0COeWl~6a)}(~8#-xh!vq>$b`|@x+ z4U|Jonko4vZInw)lDFtAXZ`N%F5c!Y57ADeswNQ+_eYr}Ta!?7fIG%%q$$&ZL?$!=#R~ z$VC2@B-vxDO2A_M$uMcA3^Qq?j4?_6H)mmzPANCZrrc$cPg!hIL|JZ9PLW?B>Ab2bn@s8` zNpFQ^8Yz8DS}21}L~v-X{3B^ z(n3jG9ge4+vbRY}g3e;NNe1OYlN`!)lLE>kCdHIDOe!c}nbc5{{}qm>o^psu6Xg_> zR>~xk#O-vU`pwU{{lw6a1N|8wsrNX3~QfpF8X*8*$w3;+hlHUt^Z=qzE zv{Q0TQab4@icB&ncbMc*mYWn%Hk%YvBtqEIB zjOmz%n50rBm}F6^O>!x3niNugF)5*Bd=U0kMHy>SOPOoZKzYlgnUeTn*iIYeaFgV? z&SIiTI%U2|Hl^7lpVIB4u;n7k2$OQk6q9O7ok<;Kqe&yBM|0S63*`ipc8dH?K|ke` zB%Q?~lMKo_lN?I7kHcC8lpK>{$`vLRlm#X=ln+hnDamWYmYXO;Oj;=uO%glnm>)1n zrTohzi<0z7*m5rANRvX!13!nY2*kJ7k@EJLR7yDLVvC z?j<&vWKep49@fgC1~CRLPaCbg8uOd2RnCe4&zP1-1Z*N43)@2ImFW|B@B zXOc~sVUkZ-WKu*~ZBkD8$)uXn>&vkBI?51}M#@-|7Roe}cFIDNlrB1pw@fl9KbYiD zx^D=3FQ5!CDW>F`R8S_F)KKP_)Ki`@X`-w#X{Bs6N!&@t-0Q1wJgJl;OtL5iCb^Vr zO$sUVOiCy(nN(5MnbcC^8^hikDE&>EDQB6qQKp+D@2szlD-XFZlw$|N$jR$9%qtDx!oj-BHvBve$J(AG%2K{ zd>57}p$svpqKr4GrOYyEpwyc*Q`VcbQFh)G_LaPg&f;*BbV{K~Hsw~6e9BUjBFbkb z<&@;_!@jC1@~w{UjXKJ?CXJK|lNQQSlXgmrNlJ>&V&@;imNO{w-HY~>Lm6*UK&dh* zrYtk5ptPFQPq};M#9ejF3rtce%T2N|siVmE7`o>hDXUCcD6J;#l%$`+S}D8fEYeLfC__zhDDtg^_EkWcY*I|AGO3_E zZBj$|(4?O7yGauz{pYZ+R?6`vi9K}8mzbndW}9SDmYU>JJ~k<&By0)$Dxt`~)u%J9 zqU4&?QZ6%Tpv*C8raWcRMrkrh?y0lbY?4mtzBTMUn=;5GpEAazh;o%lIpr>sYRdB_ zb(Ch4M#^s{EtKBBgyU(a9A%QyOJ{MeNe1OwlN`!ClLE?1CdCx_S2c9cS5V|%s8Fe) z^fjrc9AnZ%DKu%N++dQpyN-FjNh;+vlPt;xlUz#Y_HaCflmkpkC?ia&C>NR3Qsn)* z&Z~j4$fTLF%A}3*gGq9#&LZWvu$^?uK$C3B7?XU;)h0!hYLjxxizd~SbtZL`gnx&< zH&XU6X`#qlaGhy8MgBfQm6Sbn7V>uls$@{^G|8bXH7TIHZ&FP8$)tj^>+fO9HIyuq zddeu1CW`#+Ryv+mN~K9+ZyobulT^xTlPrq-T{7BEE~Pi#1gjKMa!g7n7nxL1W}4Je zmY6h9OICdHJU@t#ck zs-PTaQbUq>0jM(n{G2@2<4v#B?3=878TeN|P+g3X@#Q?sj=Da%djC|gV#DF@(viMHHAk+(G}?UX4dDf{RwH~<>dWz76VMuDdSACDR-LWQyNW*D8HMO zQ~Kj+Tl=b}$g{pm9c7_OBV~h03uPZX0c)*xN});0{yK{XOfo3zOmZl_@eHc%6i^CG ziYa%QR8ZEK)KI$PDNx&~r^r*ON)u(4Nh{@_CW!~=n13=!r5u1~HEk!0a<)k>WtK@H zWw}WSL}ae=|p>Pr0iqTLK$Y#PC45oB|~R%wMhnLj!6#XNs|J~TPDSn zjV2Y8?eXNHBdnnuY*J4-#iWUHg-I*rE|bKAbj&ZBq*6XJ$)d#Y#Gt+BQVuXFq?}|@ zLb=?eiZaKfmhzlQ17)pAGbM(5f9<`E(%&TcV4cO;Ch3$JCfSrFCi#@lOo}L7aEGpa zl~WEksiurGsiVv?X{5Yp(n8s2(oX4)`)KVenU1H4vX4nCV$w#DyFzUzIZJ1;&?KF*!X%rr-XxzQ_kh|?5hdNEoHD|s zno?v^N118TNU1Yvp{y}!r?i`-9ICVEgF7P~PX;B&B!@EIq<~UkQcPKFQbB1lsiAB! zsi*9QdmQb(iE_9}E9ESc#DO~IQj=85gC<#&nwgU$)N0oE4KEPLm6pOKq)dQrc{_zP~?6=+o_>6n$%NT zO`0fjcc8UeDH$e-hv}GeO;RaECRvmUlUz!zNg<`tq=eFHQbkF|RaD1QOX+9QKpA7w zOer;Kqbx8q?)qKq>i$|q>-{Su2I@o z3uUlLJ7ug%$`Lw?nI;*O7fo^~ttJJOUbwbsU&WLWCKZ$`Oll}{9n)I%l+`9p6#V00 zEYnIk*d#Gq$6R2NN||AjMOkW+OZmd2kkTDzWgSlmz*l25@udd8NED48bZl(8n&lv_>eC@-2cQoc25q4dUC zM`zJaImskth|Xe)Nd{$+Ne*SLNdct`&JEgfF=dEJ1?3Wx8p=GAddgcSO_YC|v{DYh z5v+YB9;IVG!z7h5!z7FHoJlTaqe&s97mg}zxrB0pNfqVl|3lXO2h=(Ce*nMFeeNIE zeKdv;Vxh4Sh7dx?gvQW!hR`Mx8e?I$(6J<2C?y-QMhGEQTNGHoDM*3MISoc-Uqb!FSNyt*8xX4H<%bi9t zS)MnN$FjjlF^h{eRJB#kvZs*-mXnRNv6L9;W?5-ufaL`v<1AkqNjyr8LSW5Ot*5c< zZ6urJR3rH;R~ac`p*3CAQ^oR}ktUW;jI^`-&qyyzGS(VZ&oIm3Mx3KnpU*Lp%yONP zbe2XVxhyXkDP;M=NGVGgOIx*G$C7HKh2>ZyohTy=vrILTn5{;UYb1^3 zQX|!U0*yGjzKt5M7_lFD+4 zkxZ6bjpVVk8YyOZ*GM_be~mP-Y>i&1*4tQUo|JU6oMdExrP#lYv;1TvonjjI!*2|5g1-I9`q7Fe9lf`9?BX zt}&9wa*vT>mY0o`vwUl$fn_WFuWG%G7_Zn$pdBaFM%b<~77Vkve`Y_AxMjZSXz4JfaNHWVjBk3#^ zMsismFjC0UWu%m4&`2GNm#h2J!m_)OPL}CL`dQ`~8D*(3l7RoBQQT!Dm8H!{Cd*q! z@>m9q6tnzoq?{#bhVF9%%YH_v4Ql-aBi$_L85v+HH8Rd}yOBiv7uEBaku;XqjAXNH zFp|&myO9!>ZBNpDu3|aBNE6FUBke5njr6iCGcwHbpb-cEMSboxlFYKvNIJ`3Msitp zJX!a!kR`)NDa-#Esbjg^NDIp{Bh(M-&wWPvSvri2vV3MF0slp%{xFitvfWJG!%UWR zBY7;R7%67C#7H^Iaw82a%|_Z-UNzFq@`aHBmcNaRvm~FQdzgs-AC2O0BWWyijbyXj zXe6Jd#YhRuyGE*5el^m>vg4_`^>&slBfTu=8yRLI(0E9Cjij^uZX}mw z=hJlSg)EsyN?Fb^QpZwegvLy5-D{+irPD}1%b<}_7Lli0Pe9M8o;{7Evg8`cWVysh z9!s^6VwNY3l(Y02X<+%)NE^$9|I_{HW|?ASfaMq?<1FVGNyG?g6jvKbW2rNe&GNXB ze3mzil(2kdq>2T9l$(!(O)NVaX=gdeNH5DNMuu66jX0PE>hn?~$t?F6NoRS^2rU&V z)oY}XERoZ7pX*pA8);$5GSbO%mXUszD~*h@)EG&?Oi_PYjHI%>W+ap43nO_f z-fZ2&VwQc3l(Wn-(!g@9kv5hGjC8ZSZDfGuS0m#rJD;IjPsH5OD5e`pW4XjgHcP#c ze3loCl&}mMsbbmuOx;!!ONNnlmU%{K=~92{j1059Y{bEAQf99*2Ya*gD& zEH+Ze(rl!Zx?w8+-szbHST;LHw?59YkC8+yZyLoZM$%YHjAXN{ zG?LHqvXK&&?~GKjB%Z5VZ(=#zNIT0sBfTuQ8X0C;W5mIFLw(+8B$*{rpxa7kIlxFR zOTLjpmK%(evOH#l)?;exLnAFL_8i?-C(AxY`dQ92GRjh6BmwIv)%>K9RF?HdGFc*X zbz6BX2N@}5nQNq+bXx-~=|;v`&NPyUb)800W+aW}0VCNg z-A3|R{%fR!C8IRx{)@PTqE5qg+>NgN{x)O)ES|@1C64^NE%D0k!+TJBl#?& zMoL%`iu5R|SW=BNv1A%)XUQ|t%W|oaVU}BrIM^3ZpC2`n%+hTno#i_txh#}@2UWwwzL zmMe@@vD6!BVtLj`J4>IDUY6gD472PoU-!qsUXS{Gn2}_bd?U2arOY)(a#`*%Qpob6 zky4foM(S8Xm+018Savni$#R^LewGDBMp;%GNxeZkEUb-PQoh0Y+$#Pkqiel8C)9$+bq(SelGvv%Fy>pJl{I z3Cp&Z>DH@QGL1B`%r(-^QfZ`@GO3rnsMI!aN0E;Z86a+{G+mZyy*;OIi7`i!Kq{Le@x%g&2*f9Oa? zrH(RE%yPDoa+Yh1G_c%lq>W{*k#3gHj0~{+X=I#br>k{;5^>z3{v2*3jpb}3*(}!> z$!EFSND0eYBULP)8EInq(?~nZE{k=4dRdM)GR$(B5eLUM>hnq?$t*7!NoV=SNG{8k zOLSX>EQc5=Wtn57j^!pJEi6wM>16rXNIy%cRJS$CGSx@|j*&Eq6O5#?TxukfrOrqm z%X3DGS=JjVX9-`UTW?^QVx)~_hLLWT%Z&`M++k##rQJv(j=v5GmT`jEHRSD z(qN>RWv!8Nmi0y&SpGKB#*%Wq?oT(%QAP$><{BAiS!yH^=MXfCCL?K5>aABHldX`I zZ09Nw>mhqXcDD04^C#pWDTQ&837KR!vTU&wE#O!2yV!IFV_EgqITu;hr}(bHu66>> zAuOxLnuN?E$Zj^B!9bFpE3mA|_!2~_T_vR=DFJc}YEHEqq;!eC^DOHQe7$0_-N>>( z>S;!)$##tvLgY`00Nw@yLkOeq~Wic^0IA#)&m*gY&uAVrWVb`y@Ql&OGR z3fa@{WVxM7?Pd3|JjRk{4@;@BIv`h|)ZTV3j*(Q)yO3)k``QI811!_*GL}DB_Oq*{ zREWwsmbDz?+TUKmvI0^qrHLgd4_Q+o2iPqv+d|6G)&ce!mS&WyM<(5-qqt>Nh&IUU zn8S3tTS~py4W({HsRQj^mJCR(lzt`F49E(|L3TBcCA0)jh1>%<*zS-LwJwI-56Q4g zaSXsZW?cz+9CE1LC#Bvhhdcv0%ud0+o9ej((hkYA6R}n~s zV)rGMwHZnsXD4E>N88THkR7BXvz!H)Bqf!_y+LiKId&$?o<{Omjx$osQe>o@xhZXSw!V%lcVLp_DG`G_*xqZm!+U^7&%R8k3onGHSic%17oA zWM_GykU* zvS!+&apHtR)@gPD_QJF+7h%RWhn#K~Nr{TfA&F8-I8(|qKie+lOa2 z_VZ`i8QAN^uXShH1yX9Phf(wHD3xzFN{Ow9XWPyHD0Q}7hi#bl8*5RD*5`BVUY7SD z`=Xw6?J+4)S@Xe=Id(d>koY~xTsu=rY%7>+7fPwIzCt}mpwxMGBg=#vl@!|j|ETAD zd*C1SoNp(fJoWHkE;Y~2VaaE?z%G{(8)=bUB_-CMBD;-C-N2LPpWAEhp` zQ;?1y>BaUGDY2R_wr5JIu^z$y(##jz)&D3p-(I0IW)C^vwkl=Kat}c3&wN|39FBbg ztv~baM3yfLE$dj+e2JaR@+0I#DN|VVx_YTSjYY4km)e;udR<*$XS3*ab%8yTtbjg!k|rbWt9+2^Yv zSJ~|*U)b3-^tuM6)l$e^!?P1R7HLBbm zWqFxvF1M{JwTGm2j9S0J7A&;)qt=y>sc%jcY_v}a3E$Aa~k!<+0(N{8Ep`rP^bzD1JCB97Wtxp!?R=J9xX-uRMJ%*+Q>NN3W7(TC)pivNZQYcq zvFli9pGRY_u^U-vd#B88b~DRyoVm?zW1;;9WoqqqmRX#swbw}*4D?>2&hFvNTx43X z->9?uIivR*x7!1pxsEfp+asLOdy;y4j5GS^S#LYHsO6>iEqB-nQYyqs?#~@|5@+<@ zW`&)?nfp1j!cOCi-nXo@(>bG$uq*8>&ggyAopugqp5c1#wDUNl_gZ(^`J8!$Gk4iV zEZvZwu|_r63Ad^(ZV2)}DTyq!*K+KTb+?_&;$VB-T*?#{+WTz}xyPQyvK=zJNy%i{ z7eZ~VvNy1t0HL;4*+VSnL-s+bMtg+iDoBQuaVZ;vn;>*XwAxOprao^BXn#gWkkxiN z%bP}KvixMEm}NU`d30=AZMU!-YGjxtA3}TV)plo%YO4Z5d-c_J>upM&htQsXwLKyw zwoh1XC)QG?#`+eSV=#()?R=KMASXlavpb~3&S0DDb)4BA>)LE&n(RK7!yxBE?zcCv zoDG=|dB7fFxfW6adC(qXSp~Ts(rjCGsz2{QmO~!0(^>w8G(aA}r;+vD~^KkK2tbCqv$XwA#%q%OM{_p0GPu zUV?l9dD6CSr#{zPUqZfvJY`o(sj+As8HKdjdG%^se?tC*JZ*QZP_hM<(3wGw+O0I_F zLteB?S!y8XLptm_mir-BLSC}husjF30rIjv#PSxT9`cH9-J@!z_3=SSr=7~O8P>;V zAg|gBSoVax2I;a}Sh68Kkk{-^mVC%Y$U1w7r35kpdEHK5CEGIBDR0>2QYu6hGPI|C z!>(ev9YTBTH|!NsV(a{yb~_85d4`;j)ou4liQTDq%N|wrm}BKz_PCVTvGOfjG*W*m z%(=~5_CywacJY>-%yJL!P5r62zJ}}pdC$&Z8Hdmn^!s+Mlo~67JppBU?UZ{roz)NQ6;jmp{y66F z1G`y?pd&Ti$@$Q3l@b+nq^8V=cDs~%Ycguy2Q`0W_eiO+GFbZT>ic9{)~S#Kk@?sj zVmSwLDC85ns7YmtAxA;_?QWJUAvuuG>`|7RAaoCQy*=fARjL6(dz;Vg43>5X?V~o> zIZDj>v(avl5;gO@(QcAbFZxmHB($~BZjmw=(&w}r?bd(P^QGM`Ma>xfU8FDV5hZfn zpsT;H>@gNP%RLX7uWanNvAhNYeckkxohn7uGhb$A$H`*I*LJ@WL2DHKZNwpafJI-C z4cWs|Vl@xh6CYIb^9{xoMLk32x{2=c(YEuAog_1{{rop}N}SNu=r?v+oY2+iw|0h< z!9dsZot-bG#`*^dpx8g>J|JN%tJBt^|LWxlsZq*MrjV*-7{_j`M&nfhEI zc7;$6zqhA6q$I;go|M>D`Mn;O`#G*kR^p!L_jZ9QC71FoXzM3?L7c3B{A{;M84T%C zzuJipQ(IB%477DWGJlwwFNU;2#*LIfo`w8v;5t@y`knZ6Q;Vc&2!=1v5Sac63g)~J;58Y!{ml@jiZ%UptS zrG$Iqgw~c_!W-h`4~%rTaB+)T6W+vZ&{g*C;S!dwAhfpZ9&VOWZ_!ys*v0*?aLc1q zs>VvdF?=FqYB=vP6701glOSo~q{r1r^&IXKULmE%s>jvMzR2tswpu9@wGKzA10e^5 zXS2|m$l;I!!&OoSLwe>j!b6-n8<`wr4h<(iq1w{R;)rmil&DpV%q(QG!abbPOD8)# z^GQ`oFP-DUsZS}F zb!xco8C4Hma}udyu&XnOWhYXDL%-(Yewx$n0<#%Tp|8hC5igAa%&(hYQ!J zQXfH9L(UCXvJA5ngj-lziY@C=Wafm|#L2Ue^TNYYqE-m!TrWfBh0C5(HE#oX9dbdq zie+C&52PsEEG23k520~g819y$*42-ZxiD<~hw7;idC1TjH9st*L`6P?&MxMM6FF1J znM=aSoVkQEmxQNqW)Xze)l0+EII|2w*Ls(RGda@$p(E^qa5iTig3u9mL3pN=E=%7r zyDZ$oqVJepZuSz-pwzM0b}kS1%gkU%%fj%)=jFJ}QLiMNB&F7(z28QRy(FB)@)qj( z39=~MCMCAzE)I|BQpo&?%;NBrcB;A78iwrTVf_dz!!0=X1Y5pH2Q z2XZCk#&8?U#gL_t%5Xc&B1k>trtmtJ8z7HDmWBIRY9Y@;s=`Aot66RikFh)sc>$SQ z!d3^3qTX5yc@1)FIDw@bcP~GI)P!?c-a}?1;Z0HN*c ziEue*p6ASy;VRC&$C)R?D^y0V&-B-{p9+sk!Or^C};r5?u4@SibT$~UOz z8f0QZTfvQxHQ@qTN*!UVA1QZ%a_;S!c(A>`iYvG1BNYosD z2w4|yk`g=XUKehW5JXartyd56qdh|%&4$pp* zT3>1{L4Sskc_&=jO)?nL^Yc!)RZ5L@8#4bzrYGFXvKsO~$h+a3x2ROCKktRhlvqz9 zL#@9TZuwVc;@eb8_1PAb$x&jxh*Bw#zHs}yl!;nzLUxyu`M#2VmXE`tSGF!o?TySQ z;VhQFkvRzRX}FLjX(i4;ApPNLmNdw5koDnimJ=a#J-8w4d_XnVTltVvkl7ecWGQ0# zGMp-jddTIF;c(`sO1_0Gf&3U=#PTcTddPpn zjV#Wc*qR`t;WaEg-3p#3jLw|Psl0ZOdEvu*deEk zGjvt^25PpQO3u8F%m-5HII|H#nXuEqnK90Uon|R&AHES8#~GIr+lRYO)_T?FE$&kN zah(<=)*cWlLEGbU16t!k{Vx+q`X>mgP`CXmNIJpcpr#cNv#7n4$&Zzfr`dHpE zlKu^~ULn4Q&^h)VP9DqOoSEV@vTSj;%1m)Oq~JY92<;)KI;aHyx+YA9&^`P;jbuRR zTsqOom9@yT|Rn#v6qv^8M@0yZ*=VCWN=15k4tm1IHRA(r8&8r z(a+ZRcJeqw&*N5M6ni@boT2A&^o(O4r-(E3JdQH^I3=9X&)@fTN;yN%(H=r(U#F5Y z`icBBrxlH=6i^f zD<$^K_Yh|`XY^KZsFTm4w}L~RLKeLh9O@KHsSqz=eWva7P-hWm^mcKWQ^uKYWLmMj z4s)tFqn|})I(3}U&muFOM$YKB8V+}wIYZ9}+fmQqP8((xnWB^tPO(_atV^)zayXp&r`a zW8&VU_6b=|rmUx2K5wG;F0!1`@2RcZtx3qxUh61lL`v*^o1>jEDK*vs$b62Pk9Mpd zsFd3G(y{v(r!G$D&Re!K7AN1M)O4rlM=GW6<&Hs)cS@yHh?%IF_I@**YANL+dyZul zNm(JK%hGQI&2U8Z`DEQcsj$W`h_wz)+&e-!CeNYf#pTWW{{JdCYH}3iI9_>7M5`c z9noewYgo2lg<~Kxr#Kx-%sK6;&X5!}&vYeqsxuZRG|zcX!LMqhdVXd(YgqKyXE}Ml z$#Iz(o8>HE(K9y78IcmT_D6qczdFmw8lzHCYdVC^A!j+WS@ay9?j-!KX5(yRQZb6t zon)59ko_RDom44N>kdd3mfb!*E@YIdghnvQf?RK=YJSQxskUZ z;mvSI(a6V;36M%>gRDob&)Y+8a+3Z>bGTG|iwvz}%bgS!YqgdsQewY_S?**gF~59S z?&NSrk7BtqTgqTa_i(vW#iDz7v(w6=dw8oe%%azgYA1c1dKfzv)H*qFvKvNG=M+eZ z?bYj@A}LXE+8oQGXE*guiIk-x3FF!urS5P_S*Ah`kWwinHqsT&3Kl)C6;6W^Gxilu z(w}lY4C!&LaB^AnxbAesUn-+#<1QyxN{w|WT0a!6-|ZBz%zzvXS>-H{61C2S%z!jH zyLGsL2Q%W$tV!g3b&B=jqidz~?srKp*{X?dSB z@o(yLt+?OF6qZgSSu7ij%x1Cf)%7f3*~v&H%fUt(Sxz^yhNZ+v56dk^hFG33Vp;T{ zvsS!oB#Gq*Bhy$C@q}NEG>7G2BLys{8(GA1g^_9&dUrt8)6CLrq=V%RBYiC27#U%4 zn{+)Qq(-rakrbBcMlx9D8kxy*oslAzl}5^>#I~mUoE33$KGvuvrzKA2LzIjg<;njm+~-FJ}&fP(3d=Bb+$_@+jm*C(Wgr2SWvrXCNKUqBwaT z@{(hDvUPD0GA~13c2ZcbWa)I$Sjt(toS7`uEbE*imit)Va4K1zWa)MqrNqv$-*%d% zsC!o3X#H(xgfse?-8)W-Pd$ts2YZ|XDY1LYJx)1i)}gKUQBRMvg5@JfKjb}Uos`&F z(+5tE67eN6gUEc~BnWD&Lj20|p_8M;JS+LgS;rZDg#Fm*QDWU%Vp;!1Js&%%0o4<= z5+1;lKFE4!os=3Y1u`K4Z@f535!sfNK^g4pzH;)V;LMX{$my3-W1R!p4yC?x5;jvc zFNUN*esGGV)LP|`sgR$Xb)2~qascFKr*ncT^%R8e$^7C>*_@=tdKp5`Tz+v1S?HeL z(J1w+K2lH6S0!Uc^zkbuAEF*QQvYr;HAR-SJ3=4(3#4Tep|@LHY$J3Y`h0N)wIVQ7M{xqd7Vfo8QIm_Qh?q;!EOd$Ss zO=x2Y8F`zZ2rj%Y5G0{soCVDByL@(vtOg(xj$0P-# zpe2}UGKaEEHgY=4?nW+W*~3UR%M>FmEK`lV$+D*rdb^5R-%E=t9#YF9Z4;qc-PcGm zGS!d+HW6Ba2N^j487g(iCPGW_@J)nD9cAPMlsXevK-osF!Sf4x6YN+c)sP!Dk^3Mu zkm)9~2J$zqbB}Y2iEKbF;Tmax-UUxdT!LLwa1NyLH=9 z2EUX<<}Z|*?G8&B4C#@c;U*_hW-z2jdZt?-1;3O;smKIece^Dl@3G{&l`LPgob5KS zj6=3WsdLkPfZ%d#|#@9pwx?+nR-XN?dD4 z>a%KVF60U~O^L`se=dbAa?_znQI|epq|BU zK4)%+ERlky@vqBmfm{z+;ugtF?3%jFEsql)$FDNCRf&09HR`UB(iyyfdgy2sb=z6! z?K~wNEI%9RWQnxs%sMF*Lhm!9ZV!vzXGYzADdl2!l&VFauXhKeM8$rPl~P8ebOxDR z^HO(I)nlCup*dXY4(&uUUt{G#8d0j;&DohGYF);e8{ApQ|XSf zJjHU8JI2xlc><-D@!qCW&hr|`a(7&2Zm>Q_<|Rm#n>k7K=Lg6ekel6nmcJq2V42?H z_OMKR6lZ$K-0JqR>ryHjp&b^N5jMAoNc1 zqejwL9y3C34LycC9*=J#hoGJ(jJ%2O$xzKt8OcOFbT#_4kr@!Wi~fv}S&-={wZ_OC zmgkII1et-%e~c_7To$o6l-9cAQq)^ad8lWt>+D8r%V40F&RREtMK7JT zZW4=LIxn~>N#Ao^H}uE_qh2idgkAC3t9BczvmXS=$U`tEn(3! z-|LpL=;i)_TgIZ7`-g71l-TuhpIa>@c6{%1+f}JR&-}-3CyQQQAG>`ldU<{94oK+= z>3ROdby8_5cZKx)eCp=L3C+)EZV`)~pU>SADN*ZotWlKN=ytQLhS2)_rJFHX)vTA` zfIGmVm*7|K2n($(RO)LtVRuzZuRlX>H4CjjXQHid-G(@!?S0tI+(VVpYt)Z!JBwaA zBkr)2!H`}$Ke@?MsMKIcFP;Coxl+19dX4(oEnv}W)Tmp+qSu69+)5U`w*2a@lTvQA zV>aY_O|CVS+Ug9w#q%@fCQ2C$={4bZcea$SkX{r1&|9u~Ixw!q{S5V-kFk%tJ+hQq zH;N&Dy3_WQ;|la0kH6d^B_UnvFSk>QdLyU=rT%uiS-wNfOCXll%Q6nR9uo34NKtLw z1hKvJy{Ijt(UMb zm8!9*hnGN-yc8)>>r-SFLw4{Mab^%w0ZI1yS;kp*_L8Tm)}2H%|3hSr$U*-cG95 z&T>0vCVQPMPjP0l*TeEI8ClFUfV&cX8m;X za8G0?(a)`p_GU_{x3)q(nW*PjFXs@-s4aH}B*&Y{vOO|+kmJ1qmdTJgkQ2Njmi-`9 z>O^k=%i)j(kX&yO%S;HJz0dH zDZM@yc+;fBo+=i28B$_T6$`va8Jg=BV zuVd$XjVyXAnCG?X3}*F3jI_vGaU|8V)S@Sk-H?mCf-EIe&w9vwZygIgasDslQV$=s zlch)!w!{;8FH1_)`Vloxf-Lm2V>uskGNjC#lCA2w5>fz( zdbun&LN0_{@8wBR*Pisucd55XO1-!fnMKHyd*xDUtp_02O6ix1A=F*DGZ(q~CnF$xEM3r7Elrl%n;y%FB@wwYnhm-e8rtPD-bxXY6LL zFHUGSZt;es)LQSL6g^d}_QY{ivyx@VRC^1gM6G^g8X&iM**UV$=G~q;Z-El)OJr#5 zbzZp?b*1p2lqxB)tBE?VPD*TBuJam{m@|+%uTe_uEcbS=MT**%A4Sc#dm~DOeyj5? zZ%j(md}ZY>&pBSsn0y*YZ*?|!2~yP4Kzd@>;3aY9CyZh(YQEb`Ve#?H!dIlEvFKkJ z-{YloJ@jjLy613@m&KW#P>PO&tGpb}90H-^;3_YVGylh#MlYW;mvW}jE0PlXrQ2$+ znnho&-|O|U=(k7i^$Jg*k*cR}Z=lcjdW}-ljM2XCKCeAa=*_DpuP;vCMX3k8vJ!nGFT2%7{ zWY&5!<0KjKvR8Eym8!A+Kqd{c&MQBeBx*TrxN?EK={3$&LZirnbbFbnDA^L36Cv+- z&Z(-NoglO=_joBR*^pC_dDly4`9H`U$a`KM%RI=nkoUb3ma8DOkY2Bxr4sTm7I(G8OVI$b;rpmkT1P{DTAQ{P&4fh2fSe^vHiw?mz1ZL&SA*V+m8cYvXojY z2SUqhz?;c3i{&e?OiH~KzDj-LchKul^_aci*Iu8>h(eUQ6?g2v_BOCwX=I3{#z^J= zQGe>K2O!j+uf1w1u|9w8HFKr|nR7AHue~9b*C5pAAuo59s)vMrm;H^`$?_Sb2&KOB z&=CH0P52ga8RQ2e)-y_eG@_rIj~LO<&42Q8PnYY4T&CBcp8tCJO04Zs&yA3uy-t>0 zAhaw-yjj6$hPWG^Y30=oHRiG=MAY+GV=iBPfwgdt;a@T z`NdLVqpeJ^*RUbKHBRUn*7g(5 zRQ;hVM7o9z`}uJ~*RYP?A|-0+YfsltJc~-j?yvfOniO?(_z(Ky`vp>BBMtmgDY20T ze)&J@34B_^XdNrXxae(!m{daOI2e;A2ptEFxE&C>cN)?4xF51?rey-HnO*2%f)QFX zNhTQi8l~QYY;NRtmMyfnG#l%YnYf933)#}h=Fj4;31n*{J3=CfxVvqHW`0Y^Hhx{) z9MUf-w)2P790qzelKe3iJ&Gh>J?BGvf(WBVW&tTEBn(XJW=vhtn zXG@8SH1v5F^l&G?Qf1^CMI+tGACD87jYos^P3)#na z=Bk?agIodG*Uw=&8BzzC<`=Qdhdcz?&#z-y4xyU&_gh#Vf~;^f? z@0T(d(ml-f6AEeUgCX6+WBoKKU7k_cJ-8dw85*5+~Hd9KVG{kL!4UltuUO z1m8JdwWUXKf}bcQYQ0ROKs_h;87!|uGSK=7evXvbe)U9uHfN|mMPIR1 zPJ6DOJCE8@vvCqKGyFm+oz`E-oCP`A?_}BhKUgy%GtI2tUWoM_S8b;lp|5VxIrgbW zrn2N2IRZk*%2`HEVL9C&jq48`n`ZkHFOWSH^N^u^_!<5bDX|`&;jdx20-5>f;hBDF z5tZt+qL3w!v;0L;@Qw$h9Fp&6UPzg!wE{xt3Fr8QQtGW{NEI^Y_)RR<-ng>-S5+ zol-~xq`)7M61%#Y44M?EhNEB z)g}JK1*#NXTfGM<^&426Hg#=vt>1K+%3Od_bR~72Uv;^XMG$&2Qs(!tR6ywcsHmT` zP-W^_uJ_lm=({~j{c)COk)gAT8~of7Dpg~3Li*943V)`QsFjWWd7oozh&?%S$=_R1zF~=UAxIrBSbZueVR z#9GzX?S4DUb`W}JyWa2QdiH_PJKObs4`+^rP@nJc`#3X;>$$@p;LHV_S>X?JW(j9j z_+y;8l`|`S^>Om3xQ{a{{RH{B@(Sx&$nlu@JN=lfH(vBpuUGSv2e|~J z=)LIuq7DHb4JEW+!m(Bq?{a#hdEagsrgOqw}5lWS#RHvW5R4w-?q#n|x=hf+ z6!NLRu7XO%MzP-SiIV|j*83ZzbeYya_ov-RrMg18^)LJpDK*w}sQC|+8uTYtQYLD3 zK)fWZ<$7Eb-i6TK?^`3ELulq>@;!vE#J@9{U;ZU-hdQr|Wp;$nwYd>jf2C(w*W>Po z41MV+CdWYNSJ>bCDXP!%&Iv8kAN(m&)ZT9^jN%8sl!f+e$&eAhTuSWT*H5~w2{TbM z-La3!xez)RjmhQz%0x}3TGkvpp8n*olcK(uv>RIg$?sJes}7~6N||^Q&HPg9ZpdMf zU;LRY4?*Z_Kfn14q(rS&mN9=+O6)%5?|#BEswXzC-~C)A@{38-`tN?86!l~zTS}o6 zb+()b`NOaKM?K?yM_fJUAT#dwO2J<`Lw|}OfBE=$G+I}3H6$#CSbCA6d-#D6RVwop zwF&EE7Zl%Vd_#MYWVz&0B~DDT9H& zGr5K6W6^geCyK;dsOG^y-yX4+lhRZb2zh| zD3UT5=xs1bEMU>wV3H{1Qki%goYwj6MLEkU5ZX?+7u8bg#U-ejzAd(cXkw{?+>GVE zgJ_Y`71G=Cj$%y8U`TJvJBo?bG-LQHLMTOBce2Q1>15eS3`iLa>Fs@I(N?2MZ9slyQ8fZ)H6kNNQqkX6)ak&Q$(MXS3?Jzj8m&d znhkkUmRcaC-Z~BPJY=fqkfQqhDr7G)6em59G?7vl8`l?*eMFWN^)X{~Hvn*lRPZY7-1o;`6{ly}dJ6R48l`N04q>B|S^hHA|b)aZwd6nfLv4-VsmV?DQ zmOcpeCqwkH46_^}hNNr^{R;UDnM1`0XY_OG!^9X%>MJ-_Aaj@yx6^D4hW3Uy+v8r7 zNKz7{L+Bp<5n>9<5fIu6ju2^FiuT4-&k-V>Mc=nMQe;bs-L*av-!PRkFZ6dxkHj}j z7wVzR zbm6R^ng;`Y^gK=^NU0DHBSU+V<3ti?^zk%Dq;O^}*OMdCIHQlS$BT5%baOq&i!3Rz ztE3Y|(MoE)LVS)4ZQUn`1yZ8oI|%JzFyBg){os zF>}Nk&Sc|`Vi9IzuIS**X%Kn~Wv=L!G8pLJJf0^+gIezTH;;uPn??V2@_bRoqJKL% zPqee>-#lI*##r=k9*ac!-Ku8&+sO;X0v7$-$%{mb6g6Wxn2n1?XPnH0%olxeayDdv zD7Z(}tVdcRidghWOGF8a{$=77qEt#$?1fpS-wItJDmkNn1$m{Y=1dl6t`rTN(Z4Od zN;Gjszpr(bXyuH4`*D%zWYKRwE)qQ~`ZqpTi#`_p8=tGi0E_-z+z^L)(n zO(La{YL3dcOcz6L5o@HBODTs`i|W;sDYx=4WAq!18j*CbDs>*qZDL4Dg+;$er9Rh+ zg!@#VNvO|tVp^Qg8y)o`PfAyaGBk>nqBAZ-ZQU&f;$%7cbB}PEsAg4)M%pOyrF4ca z<^J3!2Bbu-GL|MW^L{GT8KUpz)}o&KMKQ}tmIp*B%Y!TriWMwRu{4V|mX}!`5}hnP zEDwtfQli#+$en2G5s~nKnhpA$Fa2(`MNE@27$TwHgg!3H!LbNX#dBSpUpS--V!NNV&B|*OQdmz z?p@He-rFLbGy0o*Z;LF>l;WC^&d}ZwIh@ho+mp41IHtGVh8~DT9Ij=H7dvoJ;9%?!70fIitV1_r6%c8U4+@_eB$D^f&i< zMGI&2H}`tQ8qVnN>3twNIHSL(_krlWX|aC>3t$nIitU)_lcOsLf_M) zdHz&nNQnyio*vEfry`r{dFMv8O#4MHm(t&_>ld?G==*iwq10!hfa{^}*ZnG`nCmG) zhW;wwdQrle-LcNwJCKxdJ@gIi%^{zQO3vg!wwF@J_2@6!Z4eEd(OHat!emEYW>J;1jjqm`t>+Oi8z=aBhcG(4eX2gp3g4}$(U6qTZ{ zeJ_NJh(#<1L6$)NE7m-vGW6@A>mfgjqBfEm>n8lYiDi)A#5l`QD0K(q50U@0%H%@m zZ?gX>nptK+?nUM=k@Aen%!fP#`CH6nSq7n}Fjg>|<#q`D+B6jOu{_UW2kFnMdN#6z zgGQGBaXn7Z%#!q)s@V-%Sq_9y&0f&Qayo?e#(prway5kh-j)bPSnh(*U-AlqF_uRl ztr$flz@N&J^RpK69AvW~f#n?tea$Z+$YA*#vJROEK^Du;5PDx?^B_k`z2&UKn+C{i z5md8m0r>&4RnW<@D}>%L+a?(1da@zjj(FoGXn&6SQ)}fxCP0z`@gI_U>q1B}Bss{H zQe#~Q*#ok3fKTwqQg=hB=3Rp#mNv-2$fO3TYgOho$dQmKK_$z_5W0)CXRw0hTb8|o zVJUd4lO-(}XW8s^To0k1y#w(A)m&rk0hs~WCrD*Ek!9Z?OG@mVe_D{IL@Yoi52dCB z`7GB!Xe*c&6tU1A=}cty3l^}@9;r}DsS^2wh3=j1A2h_trN|r*w8zO+kOPChIJpjT zaFF?;n&)L`y%KV0kQ*nrK{A7h9crZXcd}MNjtugps5dqrfMf*&FR4r|>Y*=z9~C6O ztmI+H@;W3t7-JcLdFj3Sd046$TE#vvyJm7S`d(;)PI%ZWh`O93RDj5i5_iLX+o)}p^To&cE<3`xOL zGtB(fkduPmE>+J$l-dDuaD}E$0SPSZ?Eb3W8}Yja*MbkjXNg|K8S|Ae-eh2wgkO31+esaAs~W zn`J&{<_3i*E0@Yj?)Wr$?diwmJHBK(UtX>fG#K{$qi-Np2 zX@1o63;L$YCBX_Qv1hfH2J2Y#d-w~2Ar}4gc0n*ArNW|Lgk6K07X&lAWzFW?ctJ2* zO6=TtK~Ts-=f-p{aamBzLg&UbuFHZ&T#r6CzC0-7j6OHMJg8!!b7QJ!VNl0H=f+ge z!l03b&W$Nk5;U{uzYkjyv`LAb8($H$b4H&VUlFY1j6OHMGU(xqJ~zHH=;w?+H@+$u z;EX;uzA6~ujQ$?=qF{_O>#>yS{`{iAc}uOmKSAhe*VRD+i{Gs>R|iQfJ3(k%i-Q!F zeIPXU#X&L4kr4Wea!Y~|DN%6-WGR-{lAw$;3nA22X;8^h38DU!26ZfVbLN_$iA8_y z`I?}GMSt!2nqZBT!9aiQ`P!g^Gx}@K*9P4zkD}%(^yj*umt`&FHYppp9{s)UvS5fa zZy`hX=gWdo&gd(SXfVzheZ>(C#M^4k)K?tW2NR`K2z|wIeUQuV=lyF9W z!TZLbjD`OGAgu|NK_v_Qm2FxRDuX(f&oN_^xhZI1`4K{yn}TMRzd5rkXk|%yOJ$Y? z?NSB<{oU{7K__SSLx%RN%Yz=y=&yxW1$~@3mNQks0B7`7(#^pzXY^Il&A}K8T{}>p zZwah-)OJCCjhgy=OOPNXD(0YO%G??xvdrf`-x{Q_==+e>LAsQs;%a18VjZgvGFfhb ztdf$$QUiGaQWMN%X@s;$$!B>S@)YE@U=d3P=ujKDqmX3C-9bA`7bFdGPk_$gUzhtignpH|%E%8ajhhJl>TtCY z`Wh{jx^EMqOjFP!+frxUwDvv_Y={%O*Zp8H9w+pR-R2ujY?w$^+ zS@bvVo(Z~H^tb7r4a7&P9{p{)H9I&NK^cqw#@&B{b}4Gcreo~S2d#Z_Hq0-P zUkuhri3)vZ`^BJB%3z@Hkaq;#oY8m4JAyvW=sWE%1sgb{@3g-Z3`?mH_B%MIMe8pI z__MZ{jS8_nBwtE`l)*rMUGSA4QA$+oycl0iMCO$sg){n2duNc&qVKeK23ainPW!7t z4vW6i{%VlNqVKeK1^F!cPJ35SB&91vU;LwW?6sgOPUx=nx}Y^q=&tM=!BCvgJ=nK{ zl22%)vAgK+2Fah2#O|WMALRBcnTFY*JKMd%fRxVA43-aq(haJfiy#+aHa-kezEBc{ zR6srnDy0mD^mhXLgB4OLEczxPjpDOlO`Onnu|DXN(iPI*E!+@fe@Qhfp;3GpR7#0j zcXEFQgAIeK^&@dlhTfk2I!OLnNjowt(4V28QOZ(_zII5he-m`Z3C-cR!H^WZ6@v`j z>mCkrhg9qO8;K)9tCXm$=YG^Y8WevU%TN!04?3ij%M8uoc(6fA)cOSdq0FDb2+J=J zy3cP#624P?-n>UiC^Cg*cNRO6&633ujuf!yFUdQRW|p&%c^W-*BLgh@KEEHCI81HT zSW#rwAtNHQr9`b-$Y)ZDIP*N@zmOnO%JL;7VQ1XiiIj1v&ECatJ#c>`AyUVg10bo$ zOo%jc<}64CWQ)ie&Mbu-2bmb@U}=HOhHM$>R;9$N5c)dq))Dc&oFDNqOJXEjirSON zcVHrUaWWV6Y#S+zlM5l+MMhLT)(GmMzjU^JWWx_sbL`!o9U~(w`rV%7i1nk&=y!W| zilnmWcYAh@(X=TyhSeO*)loB;(GLs@1 zBh>m*q0eL{MY5&fUL|T?h;gMvN?GVHxGjO~8fj&r_gj`iQX}0`VqY4W9N8dcFr>c+ zGC4BzlN_n}M*WmX$InXix7_!P{q>K%Bi5J_{aum$BZ(~fdm#Hq z3RouNN|XL7#DS5NKU66a`l}EJM~44Lg1<|SkL8t8RwdSn5MUdN7$w6N%PEGIG~r9ym! zw&>jW_{fNqs2GLNp6&R^xRkEYmhWR9j!~Qt$@z=g!rgDk^N<;l)W1oh)-I5BkeQKm z7FvI3yErA1EhTnG@w7;@lp5U7X6968u`}Z z7RVGtx><_R*7lIOkx`bbA(S~UB0{PZ{pOl7=SPyH#J>43FOtHd|8B*+$P^a+w=FJ+ zOk>g8@&%D}Dfs>&+M-{_To5TzV$qvLbpCunWCcqtGIYEsigdEv%W`4l|G9hr_`HVn z|9^7dce#7t=ged&f~X)%R8RyhL6(*xNGphHD~Jk$ASl8n8dU_TW~=BDwk0U4T4I$A zqQcfH8#YUgR1uo;xn|~dopawOy|>->zW4k6e80&b^O)y(&2`PMGiT16Ip^*J zd4h<(#$6CA2%sJoCfxwcf8IksXjflSZH2wXxix)=rm4 zDh-(}))Lu?M#$P&JBWCfMmB1l8)DIzPBlyD-$I&3zNW|0LG*7S=~Q*G43Idb(!Y*$ zV=NmahlqX#G$WP=auktTvU5``A7ly_5`Jx70YsTxvZU5B6r0~63Dwm?u!i=GCkT(In*l*55*#`PBlH+ zMMSU3JQPa@;XM@($1*^8PsPKrY(v_j{1w^eSiT{3-WSrEV`B`djq;sKbF2s&{=V@e zu@Y$b`^JyN$_#0X^7n)vja7i~_kF$TfbFtEG zv}T6T#+($Y=7<~%^R@>Lu`NklE~Y!c1xnaP%8bE-M`0Hf^7Xas(C}Y z3~3YGg73$2w%2*~YyN{+wISA*-YM4=n`?;mHDnjMSBWjNsk|>Ie-vv4;XeOStj&j?G6-(d2=6MG71pS5Tk7JVz zS)ImTPVR~I8e&V4M|M7q4H#leaTJlyV@*41JN{YO7i%`eKP&rUOQ7LWd=py+4VU7Z zSi2$qS-CdW0m5I6UK{H)#2;JM#=1dxY*`!YMJletZ)1^6ox|FwJ_k>sJbxQ=K*m%3 zlp2y@$c$+868jdwx>y!8TM^Nt*1A|OG^LbEk8Iz?@<6U2qBY;e3Jj@>^0(8!kBx&J z{&xEJv0_8oyz~8sSThK>MSpA!$kWt1dT#SmYygB?aD6OmC)0vyw~+S=D#bu-Cdd;+ zt|jtwEN{4NWj^0Uyvz{)d{2va?riEuaJz@%{UGm<9X&$}$H!%94L#u{qR*yCya*)P zX~~dy3CR8+Tf|F2jsw{;UT#ROIGc#J9F12QVtZ6Q<#}klGlASi#EFmEMSJ(>v@%`* z!ZTPI9|yv-Tp2F{;W@2}7aKxf?Rc10QpvK4PXf`S`T|4BL9V16>X}S>ywZ>v(YuJ~ zH`&wU(+pXi#_y7+$7kA9;t|s5*>ZZk3mSfxJUt%W)s!NQ-zEQ5d>jbRDz}PPfbgtx zSbQPK@+SM`fo$Zz`L&M)=+%6v7&HCc|(~Nj&0@3%UJH*$3=p5?r zeeD#_%(kg`O=g$)7(;5so8)UD)z7Z+Vi5gBvZsmc7M}~Ezc=_NBH8hkhRhPHDAh&u zJone}K9IFU^fxVcj}L%E|8B`1@#wF$clu3aBJE@+C!P*+I1z1U&v-V-WN1dja}Ajz zs)@W!srHH&fZRqz`^t^a1ersmlQetB^LN+2=7>j#=o#Za@kt=h68VTU`^Kw5^cRo6 zBeGw-735XY+)DX5AifghU68zZH^|2z2gVEbu)g|04vH6n3=q*dJUBiRWXJcd2HBs8PE`_L12U0_9>-3P_kmnYqUgJ-av~o&z$6 zh_2O1@sS{lh-{!#ljHf8c=rxd;x&fM5ew-23hzc=+lVg$(dV)va(28efoxBtEZ&_! zb|Eq~p1Y5&`DNrQm&p0?0uViJ97^P(_@o3fmdGXXnglX|$Yt@y1acOUtK$9pTFctb zqpUKyaM5f0}K=g=oH<26T^@hwAoutv^rg-~)w#Ia-W+FGo`wf{RG=RjKuO|B#Ie7wPsIYO@u&LZ++ zybEMc(rD5aA9k=!HHye{MBa!O8#2f2LTM+mB3=#6c+$L04@iz)~RL-eP1w&NN0TEA=b{Lq|v?i{rD1)Wgs8K zmw~J#axA6limw3q3Yrh&9U$=!t(||wyFhY?=yT_zcn`?oMD%&l9q$D>oroS=R>jwW zR6+A`d;sJwB4?BLPvUup+B`o=NA)Jsd>(HG(K};} zi)S2e$q{7vEh0a}%Rwf9{1~5>K(w#*@ph0iNz+B-=Xm-N*3RWbS}BKVPO%|##0^9~ zB~92_0-`nF5!u3t6zEiQ#2rXA)TuCJwpa>H-04UlLw2V<*-pl9tS|k2e!ZLbS5Cem zv&3>rwGC;ua>_v7CXz*Dn6nV%A4EnF+1BX?SxrRmu-)E?9;q$Q68%IDAk7X=21x89 zx>gX`(a8qMBytv!ot!+7eTY;M8Sdm8V!!cw6Oo;rF^0^D7LevPLrOqSC324;Wri&G zzQB{^v{;R|j=hW1YKZ?H*)GmHOU#OEGuhe288F1&@93G;u1@4AQ_s=e=_(u-H2-l5 zLG-hd9SkW3(O+-SzrMPwGYO=Ea;SHV@8(p2)O(~J@amLzGw0k5Qq}L<)AkN3^s~9BDBNZSgd87`c(j)UgZuZC$kog{20rHARdO%ir zWB}wRj|@B7mO_2PIm`vw!y{urrg&r$$dw+c0=dN_^&pRWWIo7C9$5y`?U4?UK9BT* zs2z$Bp;;JBgG(}dZYrxUCsHa13B0u^FYq=$P$oSJhB4h4<6|O zdCemOAl)7rcC0N$?4RsC7bMpsV?a*z$RvC#OjJ znk}9q%^O5^cSepk-i4m8zE5ONXQ0@UHqxvovbQtx6r1XOB3tc2@6zX9>tRlxh-@L!3#cYCE&Ukgw^UlE~ps8OSh@Bb*A5-H7PL`3@<>okMZ5V@Yn(asW(*+lLnGR|3M z2z?2h$RCIt>!hD%YvbjI?B0yyoh*>0q|v)I3Y}a-W{H(V9-&kxI2F*WA)>$JbCNR` z6JIfNtfkduzdJ@RtM6P$7v;0)!h}1ZFhRhNdlAU5AH#lX6%of)ZIfKY_X9+ZS zg3NIGpwYj=P)?egosnm2%d^D;q`8d9tqw)$Z@J6Qsc-Yh#n;%~@*WvdKt%sOqfbT< zG0(d>)sX9n=)EEh9=RQwSss}J&7B_6qn7@b>1-zMw^!SzIgLz)v{R~;`9f{d>l_PWd^y?DlNZd7kUkB@q4VNOwCemiWJx;7;3LRv0o% z{0Q=ZlX0%?8|iClM+0ddbf#Gn-GN9Wk@-#|NIsE!3~5HH(}?JuY4e@>^K5=DAfm^^ z1x`n~wR{JW$K$jX=MoKR~ zBCGj3k)NaVG{R{D=>>V*5tmp^=v%9K!pQ|0PUJbV{G^j!X*Gut(WQ9G$xR?HlV-8g z1fs8%D~K$0Ixp3!W{XoPl`dC{Gwd>J=R6{xkmflj-w-;Ri1ZTqqf-jaokV^j^1Raw zjV43)q_%gKT(0fR68D4r$;qp-*xwg649g9Uz}c$ zH$mQV20;2jRyxkr*79!O*;H>knII<;IgGr&<4gm&fXDhLtv!q7naE{pKd&&!G-giW; zCHoTjfXIhVA;>XAdWdv8bqVBKA|E@;K+Yh|mLo{#5@aQjn~7|28bDTq&^D8st>q0w?j%jvZ82n)aDJe3 zhe*V22icxT6Okcq$t~8`K|~%VvW2?{ohFQ6g)JY~^McVn^)NG`HE>9cfAQ1=4&)WE*!B$m>MD zHKf20f9&1HZ3f|AcHYJv0O9vfw{^$W>m2&NwslJ_@qA^t=I?F3<8 z8E)=v+VXP2zP5L#8RGle-fghN^R=Tp7vwwm+R<%-9ez)0M>l%AwamVDbSHtZuS~bu z5Z_m(+iFQP+;8h}xZ4im5Ycrw++7Jf>}$9?>W*Yz!`*ri_O-LyX^8J@XSdIiXcm0! z;`W2=4qv;tqQP2bU%R;FAY86p++`r_YgacqOQ-UE?doP(;*~4g%>g+OzOvnshWPE3 z?al>ZU)gRi2>bfAJL=A4U%z&XEQyXqx%O~NK#JgN54Y41-`5^)I|%#Q!_Ay+%g(-X z+;T(wa^<)+mPF5luMzG{kaGAM;Woey*Wn1aAB264a0?rgeeLDWHN^L|m%GT4Xcc_z z?Y4ki17CZ)t+2zs_I7iAXDzd@z1?aMF4sP8yCJ@>ecWzKqBG!Yq`L;>VfY&9_Q4MO z8tInIN%l3;T?oRy_H+9U@qO**rr)Jg(OdlRb%2`z(gI%xxLJny?R9`V6NJlkfZGMa zzVh6hxyG{HLr&*To?Bo^^iS}0kUI{f8@>*5iwyC79ptuvu&;w$=Wbh4_I0pZVu)X^ zgWXC?ymF0pr-Ags*J!r}cDN2lyK6w$*JwBYo@8H#xHAp$eI4R9SrYvT<;r)PL3a4j zjvM*zBG_SH`EJI&)-wCbcPl`+T!*POS9rksE zJ8oXGuOr-fAndEaU1NyvtH2e%*QxxzaikjsIRd_pbkhy-+v`Yo8VHx`NOuJY`#Q?a zxX)Pj`o>Z2NK3r-8taY%IRU=Lx&?;#zQ(%EAna?bI{?DIj&{f0pX}>sx6~5P*EqKv zM2{qToFC^_8shsJ=XQdyuW@c}ldV+o`Lo{-kl3_9?EsR+XOq@ zUdOx92driGb-X(Xgnbpd&4&2C3f)#qylcvdZac{3@O7fQ5_Z_viSDQelYO1&)`PIG zliW^2d|xNIeU^B>#=HF>`s%6caJ(!2U@fz+@oqT?mutMc41|46aHI2eD&N-xH_H-p z@2UHIv6};;ziy#@6}ux1@!PA|oeRRgirroi_H~LoYC*EEQ`{m;qPL*EPIF5@?nfP- z=9U`b`#Q~S2Vq~QxtR~yva_!ex7-lFTqSOeCDA9~Yoa?7WC?stbQ@rY>u{pm55m4C zx`huX`#Qs&Yl!db40n+w(O2MWvfBdk4tz~^TVaQNO?Gpdt!4H#*{ufQa!qmD4e@E_EqYZJd*6I)LjU|zRq&{4e@=Q<)%NXQ_YC3Lb=XyGeAN= z*?xGAn`MaKUgx+oLAYG!xLqLZtIW-L%vkous4};}lIS*MS@*-K?l_R0;cKc}WQgx; zs@np>zNWg)LR(Vyb)H*dh%J}?2J(4sxh0;ja<>wxxW>xe77(^v?)HJO<@4R_$CFEO zzMF4JbWfDxLU#j@XTRUl*F z>teSWcDP&@yFDQ6>tc7*lgYj=aqA56eO=4UFt4`9rkspJ8Y4) z%)Tyl%R#sfFLRd|;`_SHU13S|0{E(OJ3#bmp&pT{+%DK*Usdjyr;>eDxpP6-*A;G$ zA-=CG+yP6Xm&4apZe+2xGb4Hp%5|0N7~;3rRc;jsm+LCG9fW;NbIAz(Egw=#c7CM$ zN}t?DWM5i!yvEaL65fl}X+6^Rq}BX3iMXVxO(NS6sq@HOSiU)l=v22R5uNIGk32xB zPNSy-vy;fqq-jhddk~r9k!RuSo+P46G0!7y(A=LywB`YK*web6?amp!E;`@MGo)6$ zMXB_2hXrn#)rilD>_@2&f*m>H`0jUOA;*J8D zLqyluQg@so{(hvTZjm9gz5Pf_-MJvVA8D!E1j74~mb%R#3(2zHkMxY&3i7T;Izcvg zqz`280ZtWd(WRIjJ;NheAUAnr6vz^f6oIVrNIA&R4Q#mvWFL>r1v%X#i$HGlNIS@4 zk9324;*ow3=V$hn{+unvXpiK8lzXHA3;SvUnd*@ikb6C{ z5~STDYe48rF=#LGN1Nw^J(2-Z=8=&g_jqI+$lD$%1^LAz(?Ir5V_yv*Q$5lQ(&&*^ zkXJm?3DWP8K9IdaY&rV8EyYBSWP#k_kx?M69w`D@?~!tl-NS6T2Bgprr+VG3G{pb?o!8xkhO~({Jx!NQWk$#+ z$AZ2?Ii5;9NvR=OF>ZP`Z< z(KYtAJIRpcUY_4^D?kn2W}t8ZAA2Uce~ukYdX)%#XUrHe!AUEkcA$}1$n_E`5;Xw=_8`Ao~zsnkoXoh)hBLs0?~QyaqB^bk!AzU$NuRygX9odNICr6 zZAlwNVSrH*SU@ z_I+yYYpt7|Kt7{XYu%9vq>sqAZiyw)S!Bn&o#&QWe{g4lWD^m&no#r6U)zwwi0B;lyN%EsMPy6T^t($yPVq?Y>$;?M(J~^g z(G-DHd88ENW+Gc#JC@LINruGshO|L5kBC0s``r~FJD){wtC;)@7*ZR3($iGDp}kL! zzUq-CLu#X49$92H=C=uUHoi(%=v1}zi(J z$PAEIi0E165IF|q9gkF6Vn)ByjhzNdyfz&wJCRDSbe&I{p>hq#CuB#r@=zIl$L8lt zkK|ZFU*)Fjmq}FyP5MxqpO|a}$?-@V$dN>@qEs=t0_1d$bQn@gUt+c1V{+7AZ5>|c zk$E6@61mRWF=Tp_&w{vI2f}?mE;Bo{o$1jgN~K#cE(;B*jV>l~qanqXh*yc+YRE#6 zRYY{_I5P7+o1ZOXA<;<0mE%CRC2}tjDN7Q_d?HGgfn<|r5s_cXvcFk7qli3X$b3tp z#}ip*NINv=5_#E>nogVQS|YC#*;;1)-IPoGj>x-2wvjVI{s6L_jJ|JEJxxSkt+tni zhWI;7c92DeEcbrraR*tNKt3YNJIXfLd6n$waWzvGd|)lVM?~KN?IhblJ|nW4QVo~$ zyKD}BA);$_XIcHBA<-S;wm)addXT+{=w6p48x5(Ajv}Iac$RDdIng6sAX7a%Js?#? zz9wH;vKQnwBHtM@?jPFwa`Sa=U5YH(nn3iIPj-=cA6dC2HBR#Aw)*YI*@&c{D#OOQhaJt9Zp1-;!v3fGLDG8 zQs>J`kdr;q1u~h)(PSrIrmwbk^gajOqYjtZhRhJ>lSY#xVjkV?|%7W|E@ z0htbRq?`wGHxb=}N6D2SPk5yMpVs?J9vQaAlDCNH9d1X-GLVm<86)R`2x;qQj4b#} zr)nFz4H4~YtQ==ZT{N4B?sa2j2{Z?Lnk68GM2e`SV`YmaX_Gz88f!;fNaO>0uXe2L zPavmKs$*pIbM3v(XwD>ZoE(-wrV=?pjxnSy?Mhf4FAJ?E%JZM`vI&G|JL6?D2+wxL z%T7zg&16Tn@_5;kK=iuJcscG1n;)&IB<~aCIzy%lU9KyM6w8`kt(h+F202YSUuxpd zeom@FHSid%WIjPwO9I|)Re zUz24<0@zVXL#B&9BD!_TL(tJVW zTG;_|G7-H-eVyz{AU}}idbtin&*IYdp?%9T@`LrIuQD!?8{|rmbI6Wvujz6P$W@}oTWK${jBFpn-_Rrdv zC3+NHAg38JTZ|^n{-k+GE(AHA$e~0Yk==&03BEUfTvq&I^6ZVhkIO1U5;e_+)QPjm zj&AqIWx9yy+DJ&1F^J?Cl4xh-AWi-tP17KfnHKbwZHT|}|F|4!NL{poa;WbRAD5*e zohs3LyCI2rUO7lJDjf7xU`V2!ibzmXWl8k6b9XTg(lKFG&J^jYwvTmc>l$c|fC%_7-miPt($$(0GjJgJdI zLv$(Xq6br|GRom%S!ak}(#3M!7FshqT1c7;jAq!DmYnX9Jdn#gQfx?_xQU4FnTusB z$lXLL$2EFVNFEfK%kn(qF{ z`^$1>K=X=hfMx_VugLiUO`BW@&C#TJitMz>)_~?!*$&Nlq|v`P{Ho0SmC2!)4zgU9 zfZR{yS+et*EVsn#3GK2Hn&(OL0%_W18^{Wf*QK*na@pUI!z>XWlI9gk^@c18X#Olq zp!tq8Z;|HDa$bVwLn13=t08Ss{%YA@WSb@4x7Yq6*MRW%zTT2zYg^KNhK0=cqu!Qf zmY8yVL6+Z^y@vQ-mHew*XNdn*$-hc5OxsyKl)oz3A)}Uf--qmwnTE^~`Q+;-vfLq; zfSf=ieP7yjFW12iztQtIIbcX_^i_Ilza_mF@HZLVM*H%=McFBH4e@(sr_8fNl#u1^ z$#SP015yF59Hi{rb{+K^9R`Jl8X#kmG&Z$UsI|N zWprDU=d|HZ>QZWw4iehNjz}NM4oil{iRf|cL)it*2}JbO;X~O2a+ycgAk}R|_BOuy z46(JLM!tMQ9#ozOQ7k5rdw78 zG^=DaG~B~i$(97oD6;&qY&T?e+6nXyR35eRC$cX=qs#uO?1!B`@^vJo`jkE_l3cET z%3+p>A=}!Ou7Ap$fM$&x3C(t-IgaeCkz)dy&txGqd8ENN` zzLmuR%{n;=8ePxyy&JJkRt?frLsLU`wxTv&CmREr@8mpa=8)!k^7WluW{6)u-^;9> zlk4YunFGT0^S$h_#H*k0WfwGDKi|t95U!u^Zm%EYOiR4>`aw2;aDIM}9Z1Fb`9XFW;^*fFIbcYwc${iXw^zT+ z+*SLs8cVV*F)gTjc)uKDh+nHe%4QHQ#gB3k2$$kVIVL-~6hF#BOT1G2D2qY36hF#& zNX4c2QO-BSFU60t&5&C0JWBDC?13Gw=bz-bU)!4J9R4JWKsbj#$==<8f-u9um6Cg*Uy z%(leK;d+@1!Z}u^A3gK!Q9?YzE={Y>?@D+f_0?O6{36#FVlC_X5vtFSRcSq>xr;PHjoUZ5q(T*ea7l$)X35YaGiaTJ zuGB)cLBlmK)Cv$TsZcACic2a~ry+hxh3ZGD`^fSGl%F(JG%`6qX{rQ-^OL4}_Oo>u z-=6LiOsZZ>yqZr_>p(a^X{sNoI6rAh?62+k`AJh*hOA2C{Djmr5I%Q8ss@Df6H;w? z$@vMX6_$AU38@Yc&QC~nA{FN+q`D39^Al3yKtG4N1;Z+8iMW!={uGrxtcncr^Bhsl zAe`rjS_Hy*j;Mx%lJgucg2@%wXBRf2HYqiPxmmp!U_ zEb;ntRP{o00!EUkS_i^qkE(v8;<87T80EK)o^?l6mLYzAhN@{GoS&hp288o7RP|co zw;C9MCvw5j1*cr7b(Edyr-gG+&XOzmhLU^&8^fCAq5T5K{{8F3D9TAY4DL zYCF`Hl<$&UwZamwuez!OgzLvu8TrZe+*hf)=Ra1Ny^Mk;PWsU{iX=TNF@q}q;h_$J*&D%A_Z`B7>e z2okKsrewBAEnkH73W8(enb5HD3wv5Ez@scp~lkHC_}8~ zeQM=&Rb)x@0n+H__*{!tCvy*Bu#ILcPRI4T8aLS=B*G{U-kmb?bd+2?9L&QQdkSjeh&5*We)6HD28bcEEGZU$N?~Ne7_ZAS}dl!iBU5v4L z_Pu8tlIT6xkVNl!gGjz5qJ-*a4Sn5WCsi2G3|Ga_TmsE-RTj|ftSX?HL7E+?m3LNk z0Zo>whvrGrf{@~*Fp`>6VWW?$6^%|p=as}_O03Nliy2&CFibwJYt&3>vcpxIybLlfJ{ zdf#8=9Fttn2dI&jh~cEs`-4kh6#^G^7Tk?05EEyo1$Dkc)}vcjyjQjh1*L+bA_}5Sed@xR@+YA@8G9 zOF%PPwL()(nhQuXT6Gw*Dy@--zE+M_qsHm_S>??xMymo4zE+M_dB+*e(6=a+zE+M_ z`IdOs%F${J2wy8lt8qxh*UHhV#1K15=xf|)H7$YY{&R?`u|!0NheQ?S`4H6*&>X7f zvgT#`+aHIjMT0ag(CkEZ_M*K9hpN7SCSUbKvnOe8BwzW;Io_UM2NKaOc$mty#A}Pg zR30?HAM%7fpgCL>K~qecdr5P+Dhp_iP!-Ue4Le7u#(<_k&4cDz(&%R#1*#{Y z`Hkv@rjaxclCR&W;=<%Uf25jZ34H~ec1MurNHr~>IZD+)^8{&HNOP2G3uwlu70@h) zW{gTdA=%ehm0^kKB+Z}6&RCTf&>XGuq4|n5x^<3LRUiXIIw;jKsv#lOr$ol7W=o>{ zYu(4GMTXQyiyxym#%NqUPA!8bwsXk*YW(r44Vs;a=wFRLUahnmuT2Y8rzIkfGIC80X_4v% z;WMsCJ@y>M>1jF#X}Y1g-_r~jvMTLKB3o16I9ZjRnp~?Vt8x&o z)st23L|dx^lxioFD$f$HR!>&>AY7{_s~V)@T0L3KG{mpflU1`Jwc?mvY~LtW?Fky) zH;UCtOX#H44EFS zCZgM-M6I+MZ`NI+x(!)1luJ>f`k>+ZDN!>g>HMrtn+fkFs?iYtE#ne3-w?m-C92hu z=#0G60dd6R$U3Co94Y|t96F7iROuILGF3H# zaO+G}ndjJg=4;ASm2HXFI#X3H2)E8uRfSaCI#X4(A%5#jRr3s~6^Bp`b-B({%Mvt} zv{@ofSwiaoRL|$B&VZ&|bwe|b>@<_6T&)Xe&Q}A_Od`#bq&Z&=D>LOvt0VFZkqgu` z5U!02R1FB%#s#Y2Tw5FOP^wmwYMdorZCs#=K)5z8P>o2%wQ+%(XNX@L7pP^1_|Kaz zR9y)gT^kpw9!tdV?2xFSdyos&z#vUzs_h&5dzws3yuNXv%7*4B(k!P^T&VIAG;a~9 zP}P=1^&N*kyDw4oNL5U!J|xX0stuY_BD&94s&xr=be=2K08&+uM$btqRo;0fhhBav zRlX&n#?w?;66IfCtyI+@jh?T%1Uvc{bt_e40@2{9RsGsdhu`nwTGy(EfTmi_h2~gTu2wAp&2_33 znu|%Jd%|@pazS!EU#A>PM7^ge8l)*1L`nyd@5SeC)c$9qU`ngWk57IP3(@L5+ z)!21vNkH>kwG5i~NTYub@waMaKvSbSp;-e>jp_|(YSlVu^vb_JyK7bC!sPm?RgNX% z7qYW8<)>C<2Q)XRTxc?Pw|(^nm2Zgu^klkf1mUr6x|#>VW8HKWtC)%54$sM zwki*38dW7U+d|W*ngW{Nsb*+KK=V7*7tqX6{m>jq8ojeNi7R#9#CzA z$O@#&rBu3}A5bH!l3VaWHOdlkBx&@D{e!A7p!tI;hURS2bWjfepvnW9`Kl6{>qw(( zW4@{%q-liaE>F{Jh`;7AUx_PB3wrk;^HmgtTY0|fx5V2oN~x~2sZPL?Yf1&etvp{9 zTH@VT&R4~T_^mu&RT)CRr+^$TP>rzjJdqEm>OsVv< zKe7W2=Wv0V2f{gApqh}1`{4q$$Phn=3)IR4qU-q~)oF=1V$YDcjavC3hN(TuGTrU`B_Wk36*Y0n~;B`JEU#s+fS+>L8D8$NR6{Zyh(O` zq*RO4q=4orRR+zcq|tM~r&LuyvshI_Ge8=B?Ov?v4e|ToV$}n}t-M(Ef^aJ@Rs+|V zcK7<>Vimd8_5^O_#mWKUR$i=%Eb;o`VpU>@-^z>CG(&2|RwL}$^t76rpwYGQv}&?M zWRXT3KyN^(c96q}NFqyAmm#atCKAacvQ*_&+Z=Mama2RZF4t1kew{7XLzHSP)z4D3 z(h_eJU8*`kxLiwB7gBNWU8;Hv@yoSTMXuMD{eJk2ax4*jJDgLRBNEgB~2b_UR3=7O{)^WHMJ@ZC(Yrc zX;t}_c<10tY78_dk!CDuUQ)#%=MXuW$jhoSkm?mR4VtS+qwm&VQ8N=X`kQiXs>zaQ z5v_sf`^x2N5y%~sNbh8=DS9!(vs+_q|wht|DvivdXS&DR2|4K(7dG@ED<~GZO^!sY97d*9+?kv zkVh7Q9OaQ^mPAh^qDTIfstu%+h<+lrQmr&(xhOh^VpNB3tGrrWidu0QX*799b%R__ z}xrxa8s>6_4@jb`~D*Z-V zis(MH7Dt*cm2JrCwB3m4{_~;Av1BN}cl@Cm3BvCkf2c+oG9$XLm#V;!RcVJ4(O-)A zP!(G;w2p}GhyPGzhWL5@hpIQk_x`bJu*CEJv6>6Q-al4UT_TKe>~~ZOCd<>_Bwu<3 zuLqjtX&prFCC#TE=_c|Jku|ExSoX)m&s4J|;&a&fOsxoLK35&k{7f2MW1p+s8M;<| zJD;mOOT@POhQ#CK>vL5S(0rjvp*e{(dbiIPs%ns?8k$L@(bvi^RJ|eo{plB~2ZXOF zU#MOXzNUPkN^a6Q^zTo}yCvQ=2Nk6?OE z*M#)Vu)8P_-}OXY&an8C;Ss@pmP5z-TBD=qaVEcEk!!*T(hc#_y@zxhetSeF7E56_ z6~CrX`=M2mB!BdMz(9T`ApbTWv}*$CZ2rSYA3=WUZto0jcW`)CpkA#Xx=RAL|KWzy z`?5Gb0`UV8KO7?XevWgYC$1KCg}q+?08zTq+)<=}8n#QA(=-9g}d-UY+Mah${8aUM6u z1ay49a>(@&JWo^cO_le@lWjSVgI^uj*O{f}IzwO6H*pR_!uGSUxDD4k*71DD@~}AH z*o%m(OgJQkJ>Tw)@X!62!%g+)O_i?^c7yfZ1g^(_-Of)_T+b(72LCHUPX7_&b{wHQ z8xyCm9-1&Lwmm_g4-upvB7PaK>v0RQkKtQl-p$uz`UOG5>C1Q~4143bjni)+8$S5D z{0y!G$KyEH6NfxcIhoU&>m#k%a2(@4&nLLwaXfk4rSI#R`F%+6^|aNL%fIfI8oP9d zf&TIZu4~M>pO!;ciF`~$$nD4Zz9zu=_^sfq0*JIk}Vd^C;xL&v(gV#&T=^h*Q z7ML(3&K;}Ui+4L|fM}Dl%?>C!D$H!kv zO3&@+&nJ?P`}X;MsRQ-13U+!o8V?J5{b2i>9|p_I`R4F{dOW#YoS#jXJ5_yugYw(s zVcf?=yz_W->nB*RT)&6H{%=g^w}XxQ{gmrv6z(_sq5s*05#gR_uhZKiWW7DF>5Hv~ z)ALw!Uh8u}|M~N89yeLfp`EAFx7%?4&mrqMWWOWfhu0fyoW9hFe2qoO^I9I4CLkT> zo4GwNxgF_AfI0u|^@-lN7VP&N%3p~76XUvkCGf}jC!CMtS0S|i=?PhYZ#*6n z7olG4dV#KITdy2)c~XU357qF;;~tMsHtw$r+-P*Pm*2EI?Vv#XHWP+DZjaMSAm)ej zV4p*tH*(1N7Lm3ERHAGh1gaL0gd7sUBKn(d_0?FK#1=Z76*$AyEyk1!#9+t-9)Q376u z@JbxN3GqgR3lKhsknLGJ^u7rA8-bAdYH$vja~LYJ^~52!AJ-?3stT$*SDcq59jqDj*o@CB7|%w zcwX|j%;UnQuEXaVd-OXru$vqYiz%3gZpw~6uCHsi!_LF7`=klO;^_bn#{U>N&gr<` z+1{%ubQ|Y%zmh^9^dF4>*YksQLvF7XoKM^z*?3sI8{l7yPV|%bSJtT+jCWqWe$wYeIVV3%?_cp6hk6?m%;#+dCo-3+RLOYU5#H z(+$?MU(O$wW8>`|7R--Exl*OKoXe3WSZ~u0&VR68PJw;yUmQONbKzcK6%q z60Rq-n~nSJUyb8@pUU$YYd?hfK0R+DU3$7-k6dr5LLR^7z&?lfn>fwOOc)k={;uuv z{Qc^a^tcrgc3hy{!R9(lzpY`yuy`KVWxhXM1pDmoIm9{q3)ZnYwCi*s(FPr_+wt{{ zL*{lJn4UgD&-KG0>o}h)pyPQv-&foA44eM0%gg7%#*XXt6|Pr4uG>Se+x(5}Z!A6c z2c1qI|0pRRe4f#-Dj7~+1T^)WCeBF;>3JjK`va>1}SbZ*oXq*+9OS+xtIXzc=hA_uDja5d0o#!muc$P}k4N z(A$2PCeB2h^ONfSlG~#^Nl(ADg!>WBABR;&NAFyjFf3-7Fe2ueaELh9jDK5*g(gn- zIjBz#UjXOOa(d?_Nl(vx5dRxO?vLI7GS27M;PrI&|DCCK-#*7z8y*&)!T#`Z_Po8x z%wKq&fZp>k^G|x-2p#9s+MzEqg8zt+@B8h24PE(9w*CHWtW(*3$oDy6b6lS)qfy9&*UK!||M;oYQ06tuk?Xn;x9=`CuUabRcf+gmGPs zh&CMmD?+~Beu%ie@1b>BKAz&artPP+i_`Q=dLse)}7aNAeYtZTE?Rwt)UP4@xIdnY?(Q|yhpYJip{dI3{Z$8eUT`%YBLD*X-5Ax5A zy@;^)7xa4vIKS-mHzYdE^*Ah6q*<=TQXD7l(NB1=cm11@9Wi#q~-gPhX`xgEdg4Fq9x#2kZvZ+2ejXK5pmRVG%se`I4vVaVRWyFd>gqeEx@pJs$Cn|36)STn>9a z4z3?Q9xTV^mM>T@Y%f*4?QQCbo?8dn!}_Cd;b6Yb_mLbPiu$!-NF2Y>cvzhMFL^{v z!|_Ih_a}w4yB6^k2vg;U^YuP-!I0+xn+o}UmG8gp^W?DT;(X!0j=8=5(R+8`+@EaR ze{RqDWB=d7o(*ZNPBFg>pQigW{fY~oXK!obgRh6|cO976U=JK&Hxhs=W^m!IcxiSuzkwEKDJjU4#naXmTY=c}A9Bo2f=ReIZhLgJ`@ zX_xK!;s5RBO0E|_-$5QM|4D)NOxFLq_3PJvvK{&^8}{{VygeyhQhyj6-`su^^qVTq z`#OWigY{#_efs(m>WSm!h;ujs`M=O`fBn!tcZi59p${Iv(eQ{k&Wtxh@SI``(Ez>W z^gEK~_~4M&Cr=3+e-!!PbkD%va{>OJ#$SP54mlruz68V9p||^;=zh_}{p$i>XL+6- zjN5UH_D`DfrXH8|bw$tr$}mpzK3T4ZRQG|OB7dAuZtu;#KTLH$X0LN}eK+#}zr6hX z>r43MkU57u-t)N5<2Z8;hnCp;M-I9E?7W5NarCuf%rk9BZ$~0s@P08l9`cIA9= z$bOUCk^S>La&UZa?31+VdH+9sPbY~7FG4Dz;Qm`IiL1DfmHbk zd-oH;^6+(+*YCOh`8lt>f1&T0qF&kGZx9dKOOE^N9lWoV(+Bf=oUs=YXTh%BPw3wV zS-Z3@XYL!goMCZ^(et{=5K&9<^z<#n-LMmE-%Z7Ne!%yqT+e(TxAAyb@V@XT;g_Eu z^1i?1d=Ks~!Q;vCu(0bxgUcUWHwvb+^QeedA9g=M2=&PQG6MTUy!EHe)A{whaXp=% z*uSu`$Kx`` zgXwwQx;QMZVPR;hVHRb2^=CHUI>4JQT;SujRr{{1} z<=E75zFu+|)U%&dVQ~Cqo#u(-FLU<8VJd&z|2X7vn&WM#uVBdYi&Z$z?*nY=y#05N zuZ4Zy=gHUe;JyUD-t&8Fe4US+VbAAn10j#cdw?g`Z<=7v`&0IX?g)fjeh#_bZAjw= z`eljH`|+tr&-E45+kFMTp3AeTdSm~*?v%>@rO3~$5_`UKyc+qk&r#_8PpnJ*h;jt^ z4Mraksg9@OsnRvTzkR+N5<0H0vk!pV>yPDn9hmu3I3CR3zw4j%FBp6Dn>;C=+wk?| zW%vz_NBms+E!g4p#7>IqeHWh~oj$Jrxm~TEevJmtv%iOKg@e$1kWdj;JuzLJiWEU3J3$S5m$Il8W>5etTW>uj><0ADb%2|MWPY zKiuy*Zr3At{|bFC3G0f_J2boRb8*=~CJ z5bwDl>+F1rpZ^bD&)?kq+vj70%N2aSXZ^At`m!0GyKsIs{+x~V^vl(FFJ^N?_Gim6 z__-SMRL|r1I6ogu^*nE?>BobggQj}E%KasI9^zjoufTa>`)x?r?-GQ~^;Pd1Dll9?IG_MyAAcg>sPZ8x9ve+ki&Zi{9K?3$5YjVeSd@M!=$4xteNwk z`&UT(5$U<#u+H9Rht2aIJs*f*zvtUV@3#+g?$5UU!s0{N{Tv}bN8|CFpS$sTAcuC` z_Mcnw`zy>jveyoFJp^SvHvZok51O}m5|6VAwBmq`xzsc?*{D+UZ>UX8S3>Rt{1L1=E3uU z`ycls?iU>2-Q*(;?`6`uH}VzK?Hkb9`Epn==ld6ae}?B}96ucCInMo^=fH8U5B6UHekDRq$2|BQ0PfG+AKBjjG_?Jc=d-+zHW&|! zI^>W0wY~pj9uc?WIM4I={_}sDum8WNNRnHCZ9jt5BT}UBdC84 z*)E64*IOQ!gYydiximi?;p?RS?vy^yc^!e{Y(E$V{RQK}^8UN=;Jng)&*(p&S8_gV zeh1f2aygRcfx3QdJ*3`GwyAj`r+>n)`tyUhn7oZGPS|3U-3m0q&=p3WMo_aju_K&)d0u zgYBIhr{4}R&+#|CKbxQP1owUG=X-jj+tmK_V0n*2z1nhxz55tmALaJrkl#l-74^!` zFM@Vc#d&?i_7D1Q4%VegkdGke`sI2*4}OE|FTpsE^H)M|!;rWZ@f!HM3Gv{%7+-&b zoY&)0aV|Ia|6s`HOOXHTxWAuHr`P*FW+UI5+rPN|cs-Ec=iqv>aetqL{eFNS=l3r- zpZtCX^Wc2}pT~SW81iwxKBoF!0qgj?N%p$M?>k4ZZ#d`p1@7-;8?cx&1dc*X!Zyk#3hW?RY%Gg#JEx>n~04`wu*C;{N1f9h2j!LQWTK z7hBE<_B;9YYp+9Gzy5c#xc!6aZ2N41{bgHXpK{b&f9Ce!df<@v8S#A4+8KO(Nv=o# zd`!0Ew;S(^v-``3c>`T0IG z80z~Hf4=M2=f&7B&G(7CuZ`P<*9C(0vU%~4I0WZ8+bamfPfUt)zUUWcvChwalJ9%{ z`O6td&-0lJ1M!Q^co7z<_?1ceG-37hEqd@9Oc)ZsH(}WO-fM9EgzG!lFZq5i*iM}O zLD;wRtT5iMi+JzX@%hX3z;V8=C&zgo6@R~vpZ{2{$L;?OJ9^#KK9?IJ9!5QFJRklz zYxP^C)Z2cyY5mfT`U{rVkK6kL+82lGMlhf3cT?rrl+N2fYx2kS%=_5*y6^XI+kRoM zpL4$Kyq@>r`sH9=ZRYJ<|Ne11ZgM()uVV0Vo?mcz?fVy-tMlhOoL~04al5{r&rjC# z`yOn^9uJ%MTXp}^?~Cf^G`hTcA0@vp`mgjt`1Q?tJC5-Dg3HMv{kD*)Pxi~>3jNv$ z`q@O&Z~W)nww!)C+m3#Fa5=er!SlWic7vSH%ODTNc|74T)p3twpDM4v^S)iaKH7DE zdXE(Qr8my$Jvz=e?4?S_>kw>bhp^zsHiy{dFuZ2hZP=bzu?YeBZGt9ot=t za&o@y{D8ibhwBmV2j=e|>A1c>>Zf#iek)3o*BO3=xD9Dv3fA9we#;@xLwMiW4oJt` zu9xw1ny|PL`C-oa=kG5w=XT08_99+7yI&+^_66wm$`SCF$n|`|`l0K%IUW{QnDydd@+|>8&R*2l?ArdVL)#gujV_Fjal=d-EJ}I@`}fVhZe(Bjo2M z+>RR$=~o<~=YBHB#OYV;0(KrmobB*FKO6GOe?DBUQc^<^~!>QH{ z_;@hne!=~dzrSd8^!yI_O-{FI&hr-k;i zY)Ib$55#*ZuJ;8pUuTZ{`z1I(sd)1Fo+k8pq5t{$PxALf`TN0xL-xaAs_%pHal7x` z-yhHOX|7KWd4D_~=lwivhvOSi?$B9wT-g%gHVC<1*{%+CIr(=8_&EFF^c?a$Tepi& z$LBZa!-gTT3;EOI1RuBGQyDydus{C(3BTump4pEU68j?G96xR|2;Z zgRe&%UxxB0$HQX9Cb+dv;{kNMuR55I9@HQ2_YUr}4aV0&{{!su{$bv?$|0BUC+PTm zV;()*o}WQJJizVxqJMwOdcKa?{Q~}a5bOE-Gpy%w+i}*{^ZCm2N!HtS?U2X` z5)vZ=ob^1Ou^s!q3-8zR{j&bBfF1i@f&ZME?HprtAu&FnXFF%4;OC~`JYKQ=OJING zfywQ{^=zNZ`T1wg>%VO0j#2tN42gmL?D@$2D%j`sD&}?^p!W&_`J4^?eF5(I#rvjg z$M(aJcqE{2ft?pO%Kh>L=bLPo{qcN>xgF=|eV2gUx8ScMz=P)v+u{D#13iD&=ySyR zdt#Q;x>pJv_xqrZ`O#x^dxXV!6Grg+1VjF7_MXB0kq!OlhRN};u=$E0Uuno!$jjH} zmOqtlOH*E2pElzxHCI=pM#L&!I0l?43@)k`Ysi|pL{;- za(pQAeIe{~ScvOB$FIck&&~Y|e|Iu0t_Bb8OWu@@zl--W?=%8>*^Z7YvZLcWo6wdg7}x&TKEI#L=Ml&G{NwvGp5O3s+h5Z}u-*1X zIZ|=XKZl1v&*u%_|8U6d#^D&~IsLyEhQvvM^n4zjnZ!fl7QBzUx%N5z^+?C}_fuh) zVS^>&5Ph~n2>(K8DT5zzmND9$hQss{WQVz&ekXWJ_g=*uXTggK}D)dcLx`y3O^QO1BpE%GbT0k-yNndVKTyV~|fU?GO>^Mwfd0=6c%; zy1XPkeFqltV@((mB_<4ub4?f#m*aR{Qb_9rh~I~B5&XY6l%U9R5?(Nq2Zh3LQab&xae)%W-JUVrKFyS|^L z|JLKo^|A4QUN1k5JAdz8pL#w&o%Y4rt>Xv9p1bPjxcWL%r|Wiq%5pN+m4AExKLGuH z(#}}D`Z`UQ*UuMqKK=i+=)axoEKBM6o)>#BL$6o;w{EAdSHF*>|DQ&kuBUpw`tSeO zRA0C0@!x;S?_BTVd#$ncrk}U!|36_*Y(407J=N{jQ~lnMPXFoD`CW0m56b64d$+6J z{_1jP9HHw=E0e!54)_oMkD)H7r~3Dko!=X_l%6KW>YebD`t^KtKl|zS`|00$K0lpK zXT2_!#VMZm{D=AI`RVPb*SmgiPX9jBPwUrzJHLOlcRT6dd(idhsosA7X?p5UwzF=R zp6YVCK0Vd@aWJ-gx?Osz``b^a`gdk@f6(drZ#~ubhxG4wIP=B*tJv>;>#3gKf12vw zOVIT?zrXqa?Qi-%w*I|iz1;e}NL^mHOYhhEIHuEezdt9|4=#=M1O0v8D`M-*S#JD3 zN$hWZJka-(bwAY4vGn=)r|&14_Zef?r@EZpen0KEIiOuYJ}&F$AwO-u{#$Pk=lu{X_MXVz|L;ZDr`xap)>FOR z^mAdo-D+d&Ums`n@m<%Wr@9^g_wz8zyw|Mfqto?NKPS`sryft<7Mri`k9t0OJ?W{A zuZrc@{p&wXb-(=S)H(m`{r?$sJDmGVmeT7-m;3*F-{WDnPv76b?|{e76M>(c*ZwQ3)w>T>eXc`M?l9G^4#VGv;qSxv_sy>P%3~dlPmY@AIuX-T z@HrKq)9@+6=X89|z^53W1uD;4sPgg2vqGxaI#(^gr`Wnst#w@lUj$zSU(C;2_9d7u z#(c_=zXG4jQ1&uRFINrt~b{dc1NJ5`SBE_Dn(3#_~F_g(n=epTW6BR*^Kc>u8o)SUPS;U;{VRV_X;UFv!W zeh6+=ci64!^!QeFR{Wo^4%^U<=dn&U;q#)}?&?rcyF>jUzC%5LPZ~b2;_n`OK5)H+ zI$l!G<5P`K1wN(tEOq@EQKM(@X} z-u^`0ZGVc-XDSDudG>boz4vp>?+Z00affQOcObq4@vqeZiC?QZ_@u>ujrzY;t@bXI z+l_wmJ^IV{nEs#|T|cUj^&{5pk64eE^-7}4I>zO)7FaH;7@r)M+w!^NtR`2ym1DDZ#F`OnMywgJX2c#s>>L9zv`Iu@=Ny5NkoK1+jJ3$F6mVt+Q^7TZh;>#2&W3c0G*P!`93R4|jUy1zyGx2XGDq<@R}80sEFYz%deAvT7Xa{bAxTp!>D;cJ+##^0Ni&vl18+4aJN z4A;l545VeaZj8%7S_abQy57R)1J_*Fh1Oie<|1~C>nwaea2?~CgXwB~HmO|Kr*gSOH?^xx8`bxvJvM$Mh1k;S!`@ zf;L=&^h=PQs{UuLOKm;EQUR4P58vm3(v`1Fse7PYwRyHuA2Idy>y`Q%>QR%BZx<9) zY0w{cDX8;tOZ^XGMe1P0+&Q8gD3L0F#zU#!Cv+pP6q!-GW#S zVkzowNam5E9)gZVY@yl+oygRep;Mqr^%p1&SI9le@?jo`g;gJ7?JPSCorhSCnMbzD z>9W*gOwGAkse7Rw^%0ixQsi@V1r$-gTxY4JjD3Szezixb6|6;;W0_H@5n3?c#ac?$ z@V%D$ahlZUL+n?qWlAiyN6mtMjaZ&()xqjqto!fKmO@o=5B`50EYWIp5b`x47E#AQ zb!^p@u~tn>_|?mns-G(PuKK;DZs$BsN50=t3!p|yTF?q@RTDShf98b-)CGubgsK_a zOkIZ9jYCS+s5cICsed9?Z&K@(PF=;+L8PWw)tK*>P_C*%*?&XDW{H-ml3OgU-PP(& z-PxRn>?tGcb)RB6Mos@4 zGc|{;AF;2#k6s+M#!%K#^5KqpR4OF>#cjzps>9rTmTZ~3Ait&iq~FrZ>oZ$$nRUc7 z*z#!A3hQU6<$jdFIU7sf1Vyal5PJ+-X<8alCn5GUVrwkv&mB;+CB061afJD#zDiYB zu2kCh3g?w#lsI`zbU{oFrZ=UtrP7D8Ea~IYpR+98V+T|jO1y@7%*Y z4>VkwE_AH`KnCWt=4A5zDBIa+5+u@s?9p6=k}0&rp7GgDyBL|>Mu~@2b9=m zeFG&Og)L!HtIa&BS<57(Hkw)*%@!QC_Mq$xq*hz{$Te(PN1&e~R%+_YRq=?;My$fb zI*hhi(~v5?bDMP_bm&}Z)savEN_1NVkZhw~QzBQ%9xCUVkV#Etk1axpRCOkf%CgN< zT+*AQkEg1$P>b~YLTh~|jy^aF6f#xTMzKq8kDzr5j_tAq^IWn-lFw~*KJD~VHbdF|4oZDqAaZ@bcLdL2PTSWaD3RBXH z!>$R7m8xgz-6(N86j5ulU1~AfTqJP_ zZrCb@Iw9j^1g4BMLD0InQD`z@qy~y&Wst2&${s*ZwE`7YOcIl&hz_lD@ zWvdps>Y>D=Ep@BC>mG|oYG>>F5er*?K&tFLeVnfxOB&2NZE!t|RGDvsEAXsy1n%Qj zl`Y+8W#Jqzsy96%RXu?cnW%Y{s|%8?-(XVbyEY@{Ecv^TEP0Acj!m-l8_j$hU7sK| z8?}t=vo_`*i2wN#u`IK$#`a157fKW%wbks&?PhJPa{Yi*X=$TNjv!UYH)?tD_L7VM zI$XNPt}^rOV1Je+cbmESjB=ICZ8>V`Wr@3>Ml+95wn~<~k+q0=O+WDOQ_Cc@;69Y? zGh1-Xw0X#-&jF*R?1)S6jU#*8XX|sQa}@1FeQrxHuiMgFA!_NqyV^A!^L-8Zwz}TI z*{Wy?#wxBX#4dn_UGi4f8%W*elJk*lty~_T-$5+hwBE=0N>B9f(*oI+EeA=zlB2SW zW7AEl&u)6pQaeyfz;z7fAvFhFXG7(vrAJ)^$%wH>T@J~}ZPZ!{@r+~Zo|t9kn`P_c zP?oLtfNYamXy%(^Qj2Z9f8^P^?}kjiJUfE1-X65i8K25>u+QdYJ{GL&c9S87^Sv5$1krDh%}b^~gegt94iBP44h)s(PY>kvyt z>T1`YpaAt8bS%%Ie}-h_7O>?UdJs~p%(Bh5kBK_la=zV-e79S2O#BBlANeX>+o7YO zpsmMTL3^-5sSU`7-i}l`A61+BYK(GmCUvRfQMT2r=QgvRoj$n$sqJQM)SI&1rbLG+ zvDu6_Hk;TgQ=);AJt}OEVZP_1=4LDYD3>aO!gd-|No7LULJ?b9|658%uMb1HY7SCo z$e7iZbI)xDi;hN2`ec#qL#!M5ikSKawAv*j9@%FwE{Wyyx%HlxZT8E=R<0{rO^Jvt*9!+@Zq9jY%QV^gJ;-+|V(aa-9Kt3OrHeCSTpvYRzGLl#>gB|0dXTZda8H!59n z6_bC6)FLDOM^K{Aw0X#AK9AW_){$&;N2xe&I0~3i_|emG+%?-{%xsSU>yxr$X3GTK z`DpV7%s1!`#Z+YG8+0#0>^Y{s|0~7uc+mYhBxCQOd$);sSLoOS(4SFak^2Ue&Bxpp zxfkH-WDKo$BrR|>7o(d+?gEsMWpm`@lbDlFB>95wTGS^s2i>Lz*K!M3%XxNs$7W# z-S0r3A=R13m&~WnAM@SYk@_`LrOiLvjWMCpXPjJH&Z|61aqY_7R_vn^jKXDkSGY$} z%T;Kz-;6<0Rl+eY^(|^nRpX&WxL)wb=_A*GN<%F9SR6~@^fg_2++|oVa*rY_PWrf9 zwWX@B&?=d4s*-VRDoPZ&4nWyV$Qe&c3s$;6#i(>PVyoRJAax$K0J=RUv1_j~=x)rj=Z_qfmB@`w7JE!ts8M`=yw&P3wbJH)1CsU!jTRP$wgn6DQ*r z89fv;Cao`ylf6gIAIsERD0?|ZJRwuoIfgAj*@*q+KuraGwkLqHbTVOI;6ba=(sRZp8N3_c(Mj zdZ%;G!#)n@g}D21&6!BaJw`v&W@2q|I@TIjdJB3wQ?=T;4h}%9!>o+}CH-Y~++k=z zHd1HD@aCnMIQ?nbN_+RSw-eR7$S z7RWhyvzf2ulB3>^f6JBOZ&CJKlvo(|Aao&=t1kN1QZfn;TTjMPOUvTOfB0{6^_U6{c@wnOL?_o>8Guu^A~8C+M;0 zssufv-R6ov4tFoH4i@pq>KwCWG%VL@4g1t0_sn0#{yV}p%h7fuzUMpVe!GwPB<4$a z-^5;mWUt6EsW}P%j-}=#=+rhd53eQX?saI(R!gow$u-F{r|Ac)_t6}b_y@|aj?07IgVyjYIRLHTcKZ@)=X`k= zR>|Gu?-1)YZE>!T@E}@kF#F)9I9a>z@D9DKUAc$X%Mx-N8ZjC&wTx0S%FQz)Ri8(1 zBezFyBfm!<1y;D_-kfYBXQUwa$FrC!eLTxkj3r7$tHRd#ko40+lUi(Q$uTPQ=&hQ! zPfYLUg&y6Xb8+-=&OL=5JyLb<2KX^I=MK1BLxinL!7QOM_v;LpbxX{$@dG%~ubGD|ef z?Rz}(&9`4b*>j-Xaox}b)cepSl&oi2le^guDiCX6EwW!Wcy7fnH(!e5c!Oshv};(@ z4J|>ek@=)-tGU)~Gt1j(mZROYDp$$X&NY}v$3C&$Xv^)0b(>hPk#mRY5v2OPx|ig5 zbuY>G>ia|4UfF_jj_>p6BXzb{-(?yy^^FhwOe*YAy^X*?CUlsO>9(NrSL_W(U*I=*XYG;VONv_mKOmA|oYp%G$Eb;!2>v0QsotVE@|2Re^S*&9oBC;G70%K37Pt&*z*U!rUy zIrl7#{}lP!(68p>r+zR_L<^R=-N(Ds#i(V4yDWA*?QzR!Q%2Hj?1@Odg7e5itFDHE z@p8vij<$Z*EaTYu_RC1k@ZhKyp8&~{FN+VLti+bZ9|kQ)*>tu~N~9;|A||a$PYlg) zMh^?)&p=GZ5%cXz=rQ!pO8hvE)6z<}UTgF1y;_cZvP28x?}%w-{2!sNL5Zz{M$ndp z@tYv2d3F3>pcQD#>i9mRQ7FP#{0T0#l9~qnj*@jMd*cF*eU3xxgB-8P(}ygx#uxCM zJR7Oa+|u`>?0Sxso1p>qGt?(f53>{Xb!c{?z7E|KKNqR%kuQhq^hs0p1t`yy$TKA} z+~2Oj6&2QQp(#;p@)etWA*ObsmQs^iYEmmqS?4Lwo5;6X9gnv3LJ@U3^d7X{mUjc> z`mNq9Z$qL!Bleh{-N;lK8Pz1d`3lBE&?-}+m6CBpYofk8-I}P6&uxkNc(B=&7@}m| zk0$D){79lc%6D^=A{7I%&iTa*?k^MBTdgbn_KS>`goGbbPtkbBq7%g?K z7DaNkm}T11Yt{ko)}sF{Jyh=2%2m14m%{PE7pSF2U5BzVy4hsjl-Q)?{_ZGJy}Z-A z2P!l*S0w3YPt_(?V^nX{O38R_XZ&rLhvx**T~G>SB|Hqte!erl9h!#NaQv~i;#`YZ z14YXo@#A2^>v&2LP>gj6*9{aw*DEbWz4jt z*J#Md=hMB(Z!}_3vrH_TG3i%*NxogqmBT1w((6W(4m7b0h(~YU@sjgLsi~#fsK&^- zOMDX6S_NZLUxn|;Sba`!x(s(59G!uDUTYA`b`BJ4_%H}FoF9j zRFt5foE6!VF}`xn_Op!2uzi;!-wI3a3tTTHY`I5K3kBGpe+zBmeHG~?ixT8%+Fgi+ z6XH(9y$5Jff{fB6)@GKim6Eesu00v~WItbKjy}$tHIJf1t52VabM4GnKIb05lSoZZ z)Txp9T{s>*i1SP&J|~th9IwxG&QpNrP{O&-Aa|3)@p=T1Wk!No9=Xdd%aLWqgjwcl zVuf`&YL;grAztIi@h_C1k4+(GouVg(67(LB#j6B4s&xCzQN^c^AniWAN9Ea4LY~Rx z@i~)}*lbGV*?N?_*_6$*_4qVbU4d40Vc9y&+U+wMqGXBM%#)uHQ}YP7po}t#Y}u1} zM6#|%eSb%NL#BO0K7F(uqolVx&&%FHzJd5_QR0dn3RfpGLXp&{B~SbMkhZ&md!%GVJK?p*lO2(vDB!gwc7QViS1Ps)Q6Bd5u>GK zv;P`p8KoOlB^S+6>Jwb|R3(QYxdN(6J{S6ix(Ql<(Rx+#jXtG*q@Kk!@p0pEgiKzJ zRO@p|twU@)V;4bvIG0r=e+u1#*y@BUp^xQ?KH)lOJCx(o$G@uNy^_>olnC1+SPr>s zx{588Ex0OKeq~HrwJLcn-hKN4qxx0JKc9i?JdB1L%o0UxdHc?Tbr7-tjasHaRmtz8 z=Ic>&Rq~->yp4etw6erTj2&B(FUQ_3$B3$A_fz<`Ir6oc`l^!eN7+=wIvA7d#tzeh zj%0nc*v(XFX?ODJXjKVXRgs+fEWWLXd2}a-5o_Z-x>4WxsINQu=l3}8MQ%1_H=Cv0 zZ04~!*)tQr`Gop_Ifj~LjgK&&T#ZE%^zu4O{&S?RNk~PB zVOzHF0kd)&uZ&pXhT8WYQ+;rKm zOu><`gE3i-j&bt;H&%Wts(|E)Ne7pxkg*1=-G{IoZKjs=ar$bk+r%2FmFPL8CbrqA zcbvW=t+wnW*KEuvNkHrJaUZkjGS{%1zMVI>Z>-X^+qA1ER$~_z9lvusu-`Y zr?X7^vd7E4zN=7kfagm&3l)yncM^(CeWl}<#Flc^c-@DxIKG$BL!-&pX4L1_b89p6 ztr@4U78^{yRh0CwaGab|WNi%d{Tz83xk(*|ZIp{6oO7?L8ZGTGWj7o384VeY7>ybE zCg^SHH%d3!=92dSqy<|!2EG|{>to`v-KObJqf2^9o&kxtg+968lawpOJlJfJ~O;u zVyY4`8MSOpxE^XkY-_?TP#YyzSveTXN0dDI-iX+I*J`A0qFQ5G82{1B&Rva4UW>ns z)L^`985tc`B=;VMCpROaen|G(>SR5Rs4!=Z0ggswO;%0Fnu}|0l(;)y&Lkh=XjV1h zP{fWxtZKs5I2OzCzG{L#N8KGi?m1j_A-4B?)WotDj*c}G%$aE)<)A*E8z<MvTUc+@=;^O6G27Pw}P9hoq%`lbUYSXq0U< z#QJ3a7@9Dm_ahSVx6{{F>7n`eaI*4r8RlnJf)^0)jd(4 zYg?1|poENQi#!Kl9x_t(P4vY^JRA7-o!m3>O-#yk#%t*l4?{jV7W*cCh4peL_FDJE zqY#sk{zA`jv0Cs>8{Q#s=S!>*vGGtLYnFAtCgEDN>T0%LpQoKSQr(keo6D9NaQy}~ zpM;v1@q16ZP)qhCeN4=oq}O=PBz>C)=amv@c}ZyvoF? zC+WUcGf76Yd)>v*DroDz(O}9}y4E1|UZgfm(*3m3^%P+88m)Ro|hOO_;|v*D-&z)IXpUmvxde_Hq05 zabu-<>K36sK)$g_di3h{>yfY9ueXQWpNv}Mn-*^WROkzoh2weyrbUwb@1sYPK6So z`F26n8IgtK)9~y`o=k<~7b8{fkXo)Q@Z3HH`AR29`%@!vVvml!|0>OPzNSqn?jVY)6HTJ$sw8 z&7+>3CiNCPjdaTX7>iZ9%$0*DrNG%M8qFIi?S6gi+!;RysTZ>?!PhKxCDg%|%J%5+ z%hC2a#JWw1Am102_dC1&`gm1u*2XHM%`7X=PI^t*2EQJ?I->iPR3rLE=9h3P+}{OW``Xt$F<|}v{c@=7;sf0wF#-aJ@T&o!_;+% z$#q_iXyg4rkK= zm!5A0%gWqJO)XwN6PE3kW!}WEnymX;iuVtwZGv8H4jYe(p`fg9dAW4 zGl_qSzSsP^=e@7+KT6|>UTmKCq)*YuCZBm{te8)JWGs+1MX!UbDSDqtpCbE~q-LAc zY?JCU_uF$!YK}=QPSPWDpZO+DpLsVbk8PH1S)8QzD7^cDp2P1I*;}wSKEc{3H2H?i zIv6#|Rr1}reR``BTP*d=4BTa&tk0ZPCf_`7H`dx_^z3TJ zr04XR^Le#dyCa;NjJZZ8FOQA1+$Cu~Qmygjxv;Uk=98gXQ3;M^J};rq=V`My=f0f?J6H=33@_cA_5 zKBq(@)H+4pdLECfgs}ZhY(0nV-O!0xL-njhw#+J{UP_j&cS_Q=c*mcT=fpC`4BJ+r zOPztT5nFzdpp24lx61c%a_u;z%2BG>r{9HjPt~IlA60cU*4k8kob(y_r|PG)*(R1g zRX;UKpQ^``=_X&!KKb-$$v;&;$;{eEIyGym-e-n*-&T&w{;4`Oo%PAxQ2#!2lb&-m zmeOxhvy4Kf1yx4XMm0vIMioX8OU42#P;_#)+Hlv}bQdYj%UdXcY&Gy!*`pDHf zRp0k1oTiT@^=6&coAwQj(_5yPt&-Nan|Xw$>C}d)x5ZlJo7Ns{m2cV;rlqoM(mOlM zd`qY4Em%5D_mXPPL$+m=nOlWPt(vB<%B#$LSD7_bW7bgpG`*Dd)AUkCZ29hW1lz4) z@7hh$Z}em*>9>4ZP3o=oNB3CjhdA6>GcjjRzU~0MPv1>F^*-`Z&tX6Bgt|@ZyQls9 z0(|pjJH|QFuQl@s+f%SN%6{HwV$OBS9f*bP z8Ax^Ki{v|Y>ll;oQ9Vh?QCZ6NDEZw0IkV@b>VEa8=Wl4~qnSGOMdT}HD$nPs5wx!h zcTS2^?}g;8$>P)v&>LuTsi~#PsK#i>w6s1|pTQc8oU7v~=2oAo&p53{jg(w%x0(9d zjhy43ED=UaSaSI;#h96!&*V!t$~GFAHW2H7qto=;32vrJPxPniUgA!bcLQXPa;M6> z0dmC5Vm`U*%`y41Oujxdk0GN`qcJ0Qn%?)_Y5JbR2vem6BdJHl=9Zo&M_W0@rl;wx z<+J7e&LJ#$mWgE>g^coyD$?}PqBPAMElkaQlcfbRj~tee7UZNA##)eTAC9#vZ}U0t zAMB7d&gbXfL1FtOq*?`d0%)BL$$4)cPlst6oAZ1*wb7n~GthN~2{)skZzYgxu3gWlOVAw#Kx;Yst4D<{-Ah z`WtGNHyfOJ9EX_Y`X^$t9L}5M=OVV%CD(M9LQZUniB&_Z;|5Wp4J8IRcC1A#*WQJg zoHR9Y}TFerkf8_f;N+iV~Xfo~pcu;=J?p6k12FAkMct= zQB62ry#h6GtIBb_m0MMg<85jBxYm}YdvR-;K8`!1Tscw~+496r-g{{0dg0MKE&D>H zlJMT)Jnvcmw$$iL*xWXANjc{L;RR46ncI%l>&P=QJksg=S zr)f2q)CLo4WK7n5qlr2933p+=Y;$E`ntlV zx5>B7H6N+cpSw+oM&^@i=glVHqB#BinO?3__6U=&&*bZ!uD|CWjw?XTGD03QTj3b* zDc|94$Q+!v%w5Dj-f81;&EzYdpx;p+HL(gKuPyhp16ZOl6RR@vrRzRaV`BA2tBm~V zx<^!cWXyFS>I=smj9A~X7GjBmOgyWXR zG{9fPI2`q*r|Vzc$WE6dwVW5y)AgCInr)FKub#d<)~^cFKR(fUrdgOi3d#56{+n;d zm74lW(|!5>eSOuYzUuVs{ZQYk^nBDJ&u~|zpS>UIYfZ0=)z_MS?S81Q!?dp>{kHv3 zUvGL-tbM)d8}>teBk8Zj>KjRadq33Yp7B|%KKG1o_CtN?Gvb2I_D!FW4gI(MImfgw zXGXz(sIPd&f>`^CXI!@*>Z>sAtC+EJKh#$o4Is9v@bOC=2(59nQQk$eN{7`j@4H+^UwRCzWSN}h}Bm= zbNha%uhDFe#{IIq?K6LfwXc06PYpEu7u9<@k+-aPZn{ZL<@sqcRqwe0=={^-n$ zWAh!IxpY6w*Eh@TN3-hpLw#AZ?u)fAYt|$Cp}xFX9kKfIX8mnH)EAodNvyumtgrV& zeWkNx)GVX=h?3_$GV(2*C08dhW{s%pF}9WWZX*1Zo4i6i4dSFbZGS^D~; zYL>qKi0~N;YmU33%lR%vB<|onTJkF`GPe!Nhiij_Qc<7M&rYh8+~JdN093Pma_@Qr z$K$8qyNnyWzeTIs(B?I)*|{qJ17e~L-Y$=G#k|4$R!nPn6(e)2o+aaP`4!7GagW7H zl=9oW@=T=0EXOAA%Sb&NOX;@seUKjSSBK&bINDNUmU63CkLpX6zPDL#)@l7LeW!9O zzkey;9Pjb!U+Su#rSE&!&(ia7Tlzk*+x$AE+tT;US8hgLPPrE=$~!L-HsRZ&^v z1Kz))_41rE5~sh{5sA}p6D(72xSj8H?B+UMEL+uF?KpZ4>)DZf&w7}tQu9tKiqv86 zneq4^Ua0%yoce~nix(?(73%XQzK<5j8XxxR77TljGx>s$e9v&$dln?$P#yMO2=RK$ zdl@9Z(KGD57J8eiHK}(%H>XKz8zik7_P%8FhRN4wV*i5V+2^n~32k=fA#!T=LqA|m z4to!Vq+dC5YCg`SmO@V1CD0&S6^X^xo9+90;_3vv_mC}Hy0pns2~bL+?vp8rx=$8b z`VE5~ZvMa}J|hxK@mL5XCxMcTX@ zSM)vJQxW?OVjDb*pl6|G_L7xspWJzpwlsUMK&otoX0Ohdn^29|-N-lFeCu*Hzrpwi z#I|}PNL>ppO`L+`MlRO)nz%a=llq)DDn63=dIzvy{+lhhAE|N_u=q}S8umBKy#X

f@-- z**~5{OLuxbr($_cO!g?e50Zr6B4kV3AUD4K*yEjwd{W;Uv#)M2&+R+B2O{-F)Y5If zg}t7~rdOr)-ji@YaWgeHR!fifcq94M@!L^zk9Q$rqSVBiKz-*=zQ>KP0WQi;bJoV_tQmL5tCRzZX6_OaF%W z7?)Dcwa)i;=3{Qv=6vMK(D&EeRH3vbL-uaD^SRCZCdW436<>hV)$Va!F~5Ww zbRU42{EAz^byTcO7k|_?6?aMTZH=UmEWRX!|zMVQ}`l|RoMTiUn5_Sl3%2k<79Eh z;l<8-`o$UYKgJY&EHS-(i!=0Zh7@N+&Wb$;%UB1=cjt;T8jSQe!-_LrL`;5Bq&QcLJM^%RY7E?urjGN?1 zdUeLRNR?mb2xYvEb?U_QuXuzL-bU2<3z^e=H#@|Ol0kgvgf zXCB`e#y%tWMe$7#)FM4+zCEwn*^d^+k0K^}R4`t~r*bCgPEL(|Bdt4m9wh6pJNf1e zXL%bl^taR+GxQtrRmp95$~|X1j&;d;t9B&ot=f^Sw@gQJ1KyXHEz^;FPJ&X0;2rv^ zV5i{{~ZK!zSWmilE4 z$(HFz)?2?KS?)>8HgC<)N2IFczhiA&f))&!?@Z{0H(oj5a{i8t8&0lkO1 z0}sW{R{nr)vo9cfw~Ty!0evou*z!LZ{1Lq=V#{xiirj%Gk*^KQ>ksJb3V%R)nDilc z;69|Dhq>*|=MG$Wu2S#Hx(_Uaq&H=yoQP+_e8L;h-=)Y3=-%Xfo6;leFGcsJp%mSl zhEmQt!0AJ?J%@&sdIGJ=W(%ah45e(xJccp1p%njl&fJDlvLN}!(oo7==o_YbA?Z0o zDSBHDrJRb^%eEX!>A*Zh0go5|+hRNBF_iKxYW@mk@8&m=Qt&;Ip%lHJ52c)j)Llp| z3>=G^U3gkw7&sS_<*lE1W^8$j*;0AaIK@sy&GPHUE6s1w6bE!q3@6Cd-3yp+DN9IO zN(2ACNU2|7Jy!(uTa$awJ*5Hp|McXWbrk{q{$y$3ZTycbf5m#KGHWP(s=g}9^XhA> znt)b)K%WQe1Lis^ApKtYZk|^pqnrAGe%@0b(ECT8SF2{9ROuVC*VgP)i^OD!YRs}V z@NE{EN5j+yW9w?nyeHFV+U(<72h11H?{qW-^gA7G0o`BP0=mDnaURk;+XDLCinhS1 zIC^(sU9|=DTNZ7B)fYJDnYO?NNZz+-3p`}BdKUgCSM=<*z^k!*tpWYMKwIEl#6Dw* z9niOw^fk3hYT1L>O*n701*ESzIyTekVQm53hepk^jTnti(`#d7nhR%G*=IJJcYiy$ z*UCu2`Sn(LPqjOspO*Fp^fTtLM~`S%nJ48Dm9fOk8Q};<2@Ah$@RW- zH((I4C`Zkbx;0T>$5*=YV)Jd~w@j;!#v7|aEK1aVh?;?}BkdoFf zGO;kV3neN|Y#GJ&nOKDS5&2e{m{V4kZ8c*uw>2iU*=Ri_>uQ6EZKC2%!SX;Q%`f=m znX%~JQQBS=?tPMO_K$H2KMj?OaR&bPp|3f8N?FIs-#%*&rxUDPPW@Isr_-!6I8C?C z;WWbvbDCu><@8`{HK*CupE#Xoy@)Bs#n!=?dQ_*ih0|{9ZKh|~|Kc>uo^+g~m)Qq$ zT45i;=@NS$r``6EoW5=!%jstOL`=u25xbCo7rU?Fbb-5?@sN83rx&VEpd*jhr^b zKhOAm@h@@O7=JjP(2v7A;gj=a`c(WJPTS*)IL%I2%IUm>HJs)obYSXHc?lnJnx8Pl zX<@=Jr>7-+h3Po8B;k8bZ%sH67jNU#s)S!~dUwKIoHispz-ep3228PCJa2MZ;rR!r zOFZv#TIKnG(`wJhoc_x58K|%Lqn3KV<`PI}@soOUNxGv1%Lg43bI22QspZsOFRG>EB3U6AA}l=|wEQaF7o z=~zrXD&Kb#|NfP)o_{y`p5(O2*Usr?-zHAC`2Ngkudj>KcYUv8I!;YU{yV4sKT77 zr@rxvIh{Oyi2W#U{B``he*B%B-a7s-oUR(*hv`K1<@j&VFK{1a!Y=lY`U%5KFHDKY z{}Vq>ot82QQ>>4a12`>BIf&D;lx(I~q|E1ZNy^=vK9zDWr|l_EbNYNr7pEO5eVlft ze8FjVitALV|MiqOOwoTQP2@Cx(o{~1C#7RLQEi=cIJbYkKfrjQ|3FSp^B>~m^UvdS zf&WNOL;hnqy}*AWr=|WvPRsm7oL2bH;&h3>gwwnI7jW9(znIhe{EInl^k2bgtG|lV zNBzq9v^R{?d$N zxZO{i^;^b^XRYFN!K~GshGzYN(+g((k<-#yO_-woSvQ~@#u&4H&FPT`yn*RNwJp=e z_QI9G>{XoZobwsCt9S0#oF>ox4pWc1c-|PN;d%1^{bCGvcm}2u)v_ZzsDGmR^%2RO z&!H#$2GhfoQp-!$#dJf-Q!&+*q!l@-x0gH~Q(ehMqZ3beQh#6ac1(XPc|WF>lA)L$ zEBP{}jU_&e)@5!lmZZegRkA3izm-(PbVErbrn-`kFm{v@btON?Jqpo3N)pa;^nS^* znCePujFQgQCGLx9uw?cE9h+^^ARC$#YJoBsdlU*#k3;#?(+~!^ zDDfOr08O+ypfbiTEGLoH0b3hHER$s#NpR9A8})CDDl z9*b$B^*5*ovCL4%g;-veh(ZI@YtSHse9$oSJ#-P4+?FMO1hS|$$c83b8z3)~8G5i3 z+mflRP$pB?L$jIs%*9v-Onm_=VCu_I5pRL0oxP?$Oi!taP8^;D>qvFo4+RRc9q5vZBE1?r@3 zhoaQoP!DuS=pJZ*u_nlhlUA*RGN?a60qRLe?u0G2+M#^LK7oSN=TH$-cR*prrj((j zj7^7DQXweH)Hr3GV<7g}stVYC1`By?s>drB^VW;69-r~pa|-3b*j z_Apci?J0R5s${Gms%322Vyr7_22=+ng${t47&{4SrcQ;{Lz$sssFSgUP?S0k8lWzM z2B9NDWid?;T^7@xl7}IcApQ9<$c8qSSt;c8fNS@2>&fQ=Igr*fEHURPzGaXK(iTJ0Od1w8&t&DT~HZg+n`FuK7?u+ zn{v6lH81r|hng5W4_eRIMNlVWcR@XjH9-B0ZG(mx`w&uz(!Q)K&^~CfH5})8X z3PC|?5mX2r61oH`f)-nqP#IJ2hQic6PzCed4^=YXI;fVZTcHT`4z!Y~??X+DO<02M zOHGE>Gc^tBWGndeOnnGaNwSn%Ad7k%vZ3Qb??GN@v9%M* zU}|(S|~{U1}b2_+n^%mYk#&}td5-6X#0BWKxhB~Rm zP(O7AG(c5BYJ!wq23gcEp$zHQr4bW_+J`3eD_A(Ttx}h?v2MSYfLbcRCpa}IY)I@y%HB%o$oz!Pglp2Bh zsh!XO^(}-;6SR2`WKsCx9F;-EK>;ce%BRLdLCO!6LvJLLbGnr(`Im>7ijL!hCanffhmK zp}9~KW9LK7R4LR+g`p^QIn+;G1r1P3AvIZAbuDC3*Fyp7CMch(hl12fsEoQ33RAy_ z%At)V_dyZH)x~@DHRW?snXIU$f71d8PsGb zpGt!Yq2-}jP#I%cP%SkZYNif@qSO&kKU7z83^c%4FJw)Vxov|2)Q3=z8ic~sb|^x9 z2{lt+LtW6ulJB4a#>SvVHNsyhU)D59WV!|~T6hI6|`D=0Pgz8F8 zhYFZ)BUDapg6gQ3p=M}L$*WKcV}FIBjBSCs7<(HkpDyitAF8AJC6+F+PoPd{Ps!&{ zKeYo=GbA<&Wl+1JeCkK2jIw`;rKCJiEtL#KsEJS$H5F>6(xFZ&07a<-p?>NRNX?Y_ z&Vw?cjU`7y`HUS46+m?*CqiY6JrC7VFF{SzD^Mr(7pR}w43*82W64&i8QN3w4iu%{ zhX$yA$jXq^PaqpQF7!DRVC*|66RHc1K|#i>>(DCb_E3CGzYnFG*u0o-2n9{-RHKWb zFl)IEicmFB3shGUfd&}sf~)|3btm*X6ri?3wa}iD_n-*%5!6HtLCw@K)Jc5>MX7I~ z0csay9U!&*0A*0FUtx_?2~Zj3gKDW1s1Dj#G6ia4Y&z7!SO(O|*w3IY#^yl%jLn5q zrpzq|vY{hFxiL)-oefgo4!Xp)%?|C`_$|YN>~y2=xfm zOtnER(8iJtP?WJ}p)ST=ga#PvgbK3cc=ak2hW3>F6^c-QhnlHgsFT_TMX3*=E@)%P zAk@#;c4&b55*lRc*O2;|)chS}QDcxzSu3y(sCXzqB|(|g1Sp@H3 z31w2(Km}Aa6sA@{<JF%x`W@6l-3xV6jZlNyXn=YO z8l;|q)IrkH7a)sz8M3KvD1+*O0@RyOCiM>}pL!PxQXfDC)W=Ym`V1ZEEXy$yvZzeRrVfTOs6(LubvTqs9S!AE zc~FoFLIu<*P#JXw6s8tH<L_TCIu5dC%d+J|Hgz%-piYM}sk5OV6@m(=MNpW!1d34QP&2gz>ZGoQqSP;- ze(E}CfU1Gi9BDxWvZ!024C;0$K-~>xLPvz|iD^&C{V`1sJqYD9Uq2M2K7q=p&!I52 z1FEG)p$N4bYNCFGIw`vbZKgcX0F?}>L!_k>A&Z&{Wl-r*fC@nQ)PYctIs_`C=0RcV zNT`-N7K%_OLQPa5)JzpYolsrLSx}U*r=S7q8OX|(R=oh_Q!hh7sv9b!dY~}%CR9uP z1By`ZLQT{MP$%^<)CFA?`V8u4Yy?t=%G`EB8PvB>KD7s`rPQxc9~B2RQ;AR~H6Dsm zeyE>Hh16kE^Gqm%%7pT%gP}6&P^gwV9IA(whmMAt7|Vm2sUXxzodQLvGoXHI0W?UR z3#qv>-wPp|x)jQwDxgfN5-Om64wX^Mp>pb1P%ZUqC_>!~)lt8Nny6J!3$+^Rr2YVP zQGbN`sU~QUS_i3l(!M`I1=N#JIn@r;QJbI^>d#OY)ddYwuS52HspSnQlX?p(r~U~= zs6MEU8i1OqPoWm-3n)tc3+kf&4GmD=Lxa@+K-S?hH}{R0FXe>-)Ho=UngkV4)1WeH z1{9_afXbi8>Byrt+Z{>SU;sIvt8qXG2|72&R~ zsUu`LmOvJDHDpu2fHJ7-pa4|^Wl|9+pSlGKQny0|)ZI{+x(6z!?uTlr2cbHu6>6f^ zL(S9^Pz$vY>ZG2BqSQ-J7xfC%PyGcNpf*E;)K*C4$g;fyS=9TGP4z==XnE)pD1))j zp-gHAluwO91=Mb+jQSBOr|g@s98g_}2Wnxg1?r+6g$Aj|A^S*a%hOOM^&C_{bwK4* z6sn_MgIcJ+L0!}qXpnjvQb$RB??E>85tK;{K?T$>R7QOTg{g0#a%vY;OZ@=VQLb7X zbD-s+1gMEIAJjsnK%LYSsGph+4N@79I$B!wGsvdqKyGMxXfBk&SPqm)izzJdCw zUCUJ)J**hYN6&pQED#KMdd)<(DG0& zG{D#i&>&R+S;t9RPJ?XfOejE|17%X@L-|xGR6vCxd3Ouj11e|iDyWWH3bjzzLQ(2^ zsEfJ@>V}qw>Y;wdRzd^RozNildr0L;`|g7*YAs|_4?zLy5h#;tg9@k(P#IKL@+?#V zEe{=ZGyZ03Hq=7RhdQaFpf2h-s2f@y%7^+HI~f|JPKVU-(yFr|n+idh)FLPgS{}Lt zDqyS}3R6p$i z)FDt8H4hq~j)VrOV<9VFT6H31Q-x4~DuObpv!DX11S+R4fa<7=p;qXk&|;{Iu`8fK zstU>s$~=}q1=KI0Fm(e|PSrvY>Nij`bsH3=?t;3g255j<0}WCSK=w&8j}|DCdK4<4 z9*4rz(@;6}92B8CpgJlFHB+xaE!5wjD76LZqTYrEsrMkeK<4`qlt~Ri<Fr50B`+Ds)t<&+PKP$^IyH3e#>rb8`M1{9@!26a($ph0RbWS=ba z$bmAcT&RFL0V<~opgQU_sEIlgYNpPCTB!4(PO21&Qemizx*Y1Ku7U=srI0#BT5v67 zQ`bY8)J;$URS%U@E1^2-PN;?YJrt$xgSx1-&>-~?WS=VYcm&F%+Mohz15{2u3)N9C zLS0lRG)TP)*{4Y@e}w|n-=R#Z7Yb6_paSYcC`=7P<o;f%6%P$iNzfoQ0kVpuRg)o`N`nH_EGU!8f`Zg+sDL^Q3R6cw<ip(u4CG(Z(f zzL3;XB>AYbBp+2G`KSvZ>s-lqF%+N{LqX~aC`?sB5o#IKO#KpyQa3=s^JL4^N-$-}$1GpeS`J6u3ZQ#iB)`g;0<>4+>KkK@qA9YNjrOqSTep0Cf#yT`09wLjh_9 z6r^s1!c-j;p>BnmsXHL6RLcGi%7QKm-7EPRYm|IcGZdvBh6bp|AnRf&`xF$Qo`HhY z3s9JP85*FvA?p$;(E|mjH=!W)4=7B%3q`08pl0f0C`x?>4NxPJ?^3C6r{tr)m3-75 z$w#S`m`9o9i-W>cA{3#a_7&7jeFH_QUC;pa17t0ce6HJ2mP&wvln)A1DbN5l1+pro#B?Y?Wk5md zXHb}$14XF0P&1VSMX6k9fI0!Pu9W%;pa69m6r|3C!qhoXggPH;rb?kG6@~_=%OUG3 zsqZQ%KrMxW)U{BUx*m#9H$lx*Jrt!@LIc#DkX0r1{T>QX_d!8wEfl66f+Ex-P&3sA zMX3$Y0QD?nT`l#!2nDE4C`i2ug{i+n5$f+yGt~OnnJO zsIQ@B>N_Y(jX?vHbvx?2M(T@)f>aU|rY1lUYBJPJr9n|@7BoO*LDo{KWi}L`4wHP; z5t5HOM)EC_e8)>Z>Le&goeG7iVkkl_gqr^kbN3w|Mftvqe>SufK$;W<+)$K`AVm;i zBfXamNM}Otz1ImK0@6ef0fi0HK}3R7!KJ8(fCy5gZV(X=5$O`ZbKUpsT*Jr1@qE9p zbI$Ac`{Uwy=PCEh&d#=kOjIqPU8*)vipqpM1EpRUC|1=Iic|H25>$hsMAZl=Ni`PQ zrJ4Yxs3t?6K~irz6swvI#i`~&391jFL{$=$q*@N`QmuhfRO=znV5#>x6sy_>#i_o8 z5>$JkMAf%YlIjq&OLY`VQJsW5L!{nWC{}d=ic|dxB@LC)YtSy$pHPbGF60>|qp47= z%9w_=s&r6-DjG^uJp(1FazMLOd7u>$R861+ zRdXm&)f!4tb%1uMIzzEzrAiMdPSscX#!25G=~E4d5>#WLMAdjGN%c0gOEnEjQN0U! zEUEH76suYY#i;^Nf@&F*s9FsrsXm2vsWw8MM5(eBidF4|;#7NNo@yVIqB;n9#>Ax`JhBq0VqlJ60}QI0!mSpf#lOos=opht9lKJQ~96-RZS>SRS!y1 zHHLPn;-M5(E69U)u=szLP^_vG6sPJAC8+v9iK>B6l4=;VOEnrwQ6)m2x24`BNZ!?^ z`rm=#RI{K2)m$i1wE#*|ErxcfmO?42RZzlYsj^N+Rmn1{+5&l|$mk9zR<#?7Q+)#^ zs187hsvn>v)p2N->a>jF6(Q9hg5(upg8vs0?#~JS5i?N_O7&lX5>(ekm@&bB3zFMB z!9QRYDy#m0QqZ63@AEFI;Hy;sKqyu<42n~Yh7weXP@-xQl%#qG+NGKWrKsjY9t1xL z#=i;3!uZAbSJp3#Ux|N{RG;DBU8-&PH$}Av|9WQ2yaQ0I>KGKKItL}FeuWZMH=rcd zU(ha91pZA?#o%Ahdr~h46syVy#i?F|5>%z3MAfTMk}3|`rK$&|sQi#;j?`-l#j3hM zajL#hf@&C)sIs6W)f8x#YBrRjS_paOO1-5}tm+dePPGY2P<;s{s`f!isvn?Ts#8#k z>SxIFzSO%0#j5T=ajM5qf+_=UO`<9bl%&c7?NSwhQdGqu&pfGD4vJO14#lZzK?$nH zP@<|Ol%z_4cBy(oDXKw`=L4xX8j4lD1;wdmKnbdOP@*aTC8<_HyHp#X6xDXfGhgcM zg<@3)p*YoXC_!}|N>p8jl2m_+R>*n!0NSPc8%j~7#r@-1DWg$P;#yH=C`pwaTKc8D z)6XrVyJa+=jH(Jq-yZ3EN%~YJpk1mmP>QO8jD92YUXxLkPexTWrEi~%){{O}W61NZ z^u^2Q_o7xZs%kHNC#A2G^r^Z-o}Z+z4-~5!2qm1BzF|OZ70D>;5VbVcXQgrd^R=z_44L^ zz5rBLRVu6r{;E(9^iA-87FKe6p=H$~{hp)2Uy9)ByW zbAj$FLprTNLmeayYj-LzF)zKAU6==B$>Y;sKLQ_-+p;gcX|B0}U#y7;{ zS^7@L_o3#5HNn4ws)$GXx4L#dl^WKb_yMc1&pJ9Zte@krg>^JO&1z>fFBBCidoVVv z)A6_2_xu`rv_Q+bVI7TM6xQkZFT$GOzW^20DmCzYV{3U>d*btcf?FOX=SFN;+vAHv zQ?ze6B!4k|f`2YYyAhH;A`A( zWsv-WGXD1tWJ0&%KZSy-jZjFnRp!OX=uRl`hOF2Fd8>=|K|a+%$W$GHEY%4ppgIEu zRVh$NbqO+RNR{7YUTx70nWwrV^HdLHUL6_z8}h2seunL;q99Y18M0K_p@1qk6jbGd zLaG9gQCI4{B&sJzvIJy7ss5QzKs5*Q)|bBdkWaNp)Ij<^7Bv>FgiO^ZkfquH1yq}% zkm?J_Xd)|iL0;9@kWcj;WT_5A0o5@ms5%9OROcaYQ>k(h@~N&srs_Imscu05)qNLX|sR6l+R^sQ-H?37M)rP*AlG z3aJi4fmX8ih>WUE$f)WJgkL4cXbR+2U4ne7-yl zCPG2g6ey&c0eRa?z4suW>I2B=AblUnsA`Ffs#ZXzYOVB{GVe1epxOilRog`gG8%+J zs;{K4ll1L}tgdo^}qfR4I_Dx&&FO-=Ki% z1{74?fkLVWkTFoI{0(_kX*Z#sDhe`HnITJ+9STDDo(vRH<%5ht(pLcTs$PP8suGZ? zDg%Xv$-D}p(W2KNV~ofLc~vzbpQ;{Ysv1LozxAuc{y9Qw@d!Gh}oG6jY6c zLaGUnF;hk-LtfQ%$fueOnW}j*`mXHNhmi5UtVp{ZeX1zPr^*bOs_de9GA}n|sq#Sq zRRJgn;Zv%Nekk)g%BZTFjH-G=A=Lm;lFS<_^Hz&S$vl-MqpG(cV~vbXg}kbnkWV!S zGF9^-OSK3Js6K{*P_lm|6jFTx8Ed6)1LRe0hI~+}{|m@e?Sd@T*HB3H9b|kW^A1B^ z)iKDYIt5v(^H4x_5ellVKq1w2$XF*;Zb4qveaQEz^gV)1Rm2zg>;mC`{6d!YWrPB% ztk7N^%?YJlFY}&p(tLL&#J$gE~Wl;@^ZU?Q09|)xHEM?M4~x3V9*CkAQrt{*b8}0(I7TBOy!s#z6tq zL@2150);k9l^Kw+Mf9GG?i76>qpFW&^h@bmBBMb$Cs#n$E>Wc&m0LUQRPeE%7;pb7r#kaw?W$WF{tJ%o&}q;JBPxRp?(Znsx#zM^$;DD(Y`WzNHj?LRKp=tH3qU&gDll*D0o8H@dALP8swhRBuTa^ncbVT~{LK9W7P!PiZMo z)vu88yUe==c~yTxKGj{wRHZ^em9YnFRp}t(x~z?cysBp)pDG7rs`5aVDnArZ6@-F+ z$l4;1_fJtt$fqg`nW~DAbyG&G$mlImoQ$e!$*8J66i_vRf~w|FNYxrLZp+#ZkXO}N z`tHbm)I<7IeIfI%oL7S&OZ5l}LaF|@_hK9OWON#2JdnOCkfpi~nW-{*OZrsZzrqUD zS;%@QEBb$pKGhJ&`$+mmLO#_v$W%>)EY%b!pqc>%RqsI|)d!IASgL#k`TiCyflSp3 z$WpC^f<^>x?`NVk_!}?&O;AX+9Wv5NUl8)DzJh$|q;EfDs=kNhuNoQV-et(E`W-S=Hz7-PPevnU-b2Wz@_d7Ns`QYhih%;EEKpGOEab!A9ZK~- z2broDAWKyU3aE-gK~*UzqFXkWs-BQf)ekaNgCR>b0t%?cLP6C8D5RPU8QG-XbjYik4f#~_AXD`r zWT}#%fND7uRIPzRs`ZeOUFv-fc~#pWpXy7`V}%&*C0#vClpZKg@US7D5Ns>q29AnFCFAnMMFN-Gmxpu0a>a%P(YO*3aSc1 zAyp9>%_;Rt%BZR=Isro_z)gUOS8V-e2V`Mb1)Eh6Os<&lSH4XCSlhJn}`Fk384+5F0g^;BRKmo6e zE`x%q)lf+FDP+XT=tjt^+6wtpJ0VlG2eMTApn&Qi6jU97LaGyxQ9$aQfxIgDS*TBS z2{Ki`L6+(U6j0rPf~p5lNcA@)7xlzvx^GcW6$SZJnITh^9kNupp3a$Cs!BjURT;=sRe&tjYfwPtgMzA>P)Jn|GG3H=jUlfp9`dPLL8hubWT`qq0abS> zsOkfSR0AR7C8;+I@~TEdmMRens3t){)jLo~H4E|ihG$`;70+5!1gyCGBc4P>bfKmpYcP*8Oo3aL&*Mlq=ug1o9%S6u2vLS9uS$fwE%nW|inrOFEhRIyM{^&%8f6^D!xQtxHR zt11uqRFxr9RSmLKHK2g1E)-NXf(-mUHC%}e@~T=wK2RrfGy$@Ncg-}2h5WONRmO(+)YAB@o6vCgc!stfGtJ(_rR6C)7Y7Z1t z?UQ-sWyL|6r#b?esuPfbFsj?I@ zRjVLNwGIlXl4Z1^9Q!Sh(MVM85c(kez5!&Z4nYCcQ7EW7358T=A)~RZxBz)ozd}CM zHON%`2?bSmp^z$7`kKgE<9qa}(m_U3>5GQEs%K~p`fY>6jC*Ze9fg^Ysge}fGkyKD4^;A1yy~e zuZ64_Bz>ykkgt{Wje$(nc*s({4Fy!wprGnqC_4OyyB zp@3>56jW`6LaLpR(MIa+fxN1HkWY0GGF3+)OLYPYsLnt^mHaQKkm?d-w3T|lL0;7j z$fvpknW_hnrTQBRsL~!rJyjGGQe}p`?d0st4*4K_?+Y?j#~@2}3JR#sLqXL=D5Saq z8SQ25b;zr_1^HC>Aqz_OKY{|Ph#yczl>rK=GD1cNnU@vvs&Ya;)$@?4@Is+^FidLFV=UMQd{ z30{hJ31}kf~Y)S*mqVK$R?g-DSlV>4WgUpCDtn?7@A=2TkxlqN*Ho`i_N_?9X)^ z+t3wxAxjkt86%|fi;!1U9P+7NhD=p?$Wm2?0;+0IP*np8sp>+;NU7II=8cxU^2;cc z?Egka$A}In-8mm#0(cgR%Tge=uPD4==>1y%AtQ$x_L`1Fu( znkWV`RaqcQ^(+)nJqH=nW!?*rS5*k|sft3TsuUDdy#j?)uR_KQS^GNVRlNaOs@hOM z)c^{rnnEE}3&=N9*0zC6l?hp@E>J+#6EbGWync{ZH5l@#MnI-&EEH5tfI_Otknygp zoep_bvmr}04+^M0go3IhD5P2r`DV-7HIS)V4_T_up@3=|WV|QyzJ$E0y^v4!Eo7<= zK|$3~D5N?G8FOUqS;(up09mSEp@8Zd6jc2Qg;aMT-&|Rn3YjY76xOQJK><}XWV|o) zo`Jlo9FR|y2QpRpp`fZD6jBv|jCr!QB;-|Kwed6$fxQ7nX0}}P&Eh&sfI(w0$Do-@~Xx|mg;RNpqd5+RqsL} z)%%cdp{!j9nW_L}sg^+j)oRH2Q09FKc~u)BpK2>)s&+y_)gCCM+6NgQ$=ZXES9Juk zR41T->I@WAr9dIoCCIl(*8T>WsvD4{x&sAN4dMOSu)F5Av!CK$hwyD4;3<1yyCBkg5XYO_H^*K|YlaGF3GpOH~gFsTxDZ5?K)s zc~z|-pQ=3+P<4WWs_sxo)d!M4=5i~3AY`hBL6&MX6i~_ky$`A;LEfdZ;vLARngyAv zxsat=0EJYGA!C`WSPFSnt03Poc`jZDnW|*SQf+|(svS^JwHpelzJZM6vi1PvRsA43 zDSgKw>$K=J6i|gA?W+1ffwOuHprC3PWSo<}(U4b_2$`x$kfnMD z3S5wRv!I}AE+l`@=2rXy$g5fmnX09brCJ3AF3P-hP*9Z&8NW#17RamG0hy}Zkfr(t z3aAc1LDdhCaY@!5hrFuOkf{nmmg*Oo_p8jiD)Ur-$UN0;nWy>-3aTDMAyt~QSaDg_ zMnXPSCdg7{g955tP)L;*l0WKmD?S$Tsa}LkRdFbwdKn6;%0tFgSy37Cs;WVzss?1K z>Ow(PBPgWuL*CzHZA-|fY6qFBj*zA51_f2Up^$0-?qheAHpC@28ostl0vyNu>J zhk2^}kf|yNS*jvXP*oBNsmemeby-^x@~WyprYa7yRJEX>sy-A_HGzyjWNmZEt7;9I zst%B)>I?-{J)n@PFXX!+YX?E5YB&^7je&xy@sRPS%zGR1s-{WbP3e1A`c&^jrfMN% zsRB??wG0ZWRzt=uS^FvERc(Y!)mF$-?Sz7=Jy1xs4>In^+Jlf+bp$e1Cm>681`4WD zppfbkNI4j zLQqik3lvgam3c9;_79n-x-Iimf5|-6W5}CP=A{W??^TgdFq8CUfYB?sz#!0vRyx9sairo zRXZr8>IiwW%e-!oPt{vSAza@KGIEH9LSEG<$fvSI&&t}jAX7CJ3aDm6LDd|{$SL#Y zLtfP)Q7-BG81kuBLYC?iD4^N^g;bj%Be$&hLi%11?UFvP=xfNQ`VKNxhoOM#7!*{U zl6i$?#d(>hx+wEhS7e^*I%K>k^KL<2)qN;XMEV{1 zP*BxYMoY_R0_0V7g-lg1$Wrx(LX~9R5Xh)38VUJS;~-Nt5elfLKta_ED8d-g>OHg` zBR!TkiSX6)7!(ST>Jw|z?7SaQm ziF}UiLT)1u5KkkIkrBy@6hJy5eUL$j)Et9$A~GGBjm$%&U-n7%Z8^T)iu{0FM5OO3 z+BA(lMrI@rQV1!5G(ZxNA;>IbF0usKjGRK`cafyFZ0i=Bxe1OZQXOfDNPl0navbNQ z-Hz-<4kCXb50Lat?HU=;=0akTmymi$ZzK_!ioAy`L1cTMp#2=#h8#d-JJNqF?9*tk zATiDCdD3P@oNX0=iz2Te?U8B7Y-AxKb(WxAgKR|3BA1XqkgR?jA4J+%q%`t65|7Bb zE@=B8laOiQ%t9-X<2(=TN61oS9kK!0g6u~2A-^IIkPPu2BNLJxk-B-ptyuB`zAl8w z?JkD)RirA?0g?N>3))_YoCp2U&OqiPi;=^K)R6Q3Jp2HW;}X@}V`M>cB6$(nf9WfN zltyG*<yI~$R0e1LWlvNW7kXun79guhNj z8`T2)i9pY^FFnkv=TH(Aw3LwRiGDr{PePkK3 z9g)Y&F0_Y`lSrD@_TxMkS}#&E9C@t00>6fILk1zkkTJ;H$aLfbWD&9s*@$dI(ze0# zBH~A+Mh~>(kjaSbuk5#E0lr=w?vuyKD)Tus?w0b--cL%%&*&mMF z?jx`~A4;UYJZ4U!PkavTMdVN9E)vzjV-!Y8BQ=r6NH=5vBIl4~6#N!44OxV2LG~fX zkt@hUB*OF3*Fk}|80FnKX$nVFkL_S4MBNve0kz2@LNcsf4ULks%v7fnMdA!Qn zd~hM898wwaA#z)qqHT@HF>}sq6JN`ko@o1rBVWrg9s0!AW8kUCEMx()1lfSd+%0H# zB3~m1kzcB5AiWSdkL3A7_H{76wvegF zEMx()4EYS%jQogPKrSQEo$&mF6hw+6avaN`t%TG>+9TbO0mv|9A|l(EgI12y613}) z-H4nw`_QH$@|+^^cDCE9XvZQG5n2Bc+U3Y+$ZljGavV8_TtogwBD>ggpF>+PoT6we zBQ+7ZZ{&DMj$vP#p-HPP;eV4y7*E#~ z&zndNBrj4JDTUNT8Y8U{ndf9AzLv+BAV`M{eBZZN2NC%`BG6IpuqC76%f@dOfoadumfow#6LQ)Xtn8|aVM9!V7=)Z%+ zbjLOjS^GTNSfnIU0jYx2M!F(>5a)5^yynU4pHo}fk(eWKURR~H@bw4CK}6=rb|nA1 z<9-_Ro#QFT{tCX9V}1v%9P@NNJVqm=HR2rW?yx*I`=i~D97dcPzrfd!TZo)TvL>o0 z&IQCdf8;$$34C1^X@-nIK0@{)DafzLi@ogQ_s?^$Gy3}=W1h$~ctJSw93_#yHOOa( z9FH&29z?Dn^7yzAX zx8T1JdES>}Bez@jBchKzN9IYNY*Xe-n=3pf^W`~F(h+^q_D4GunTSk9W+Cq*A0kVT zRfxP+Y(%>g`5uwy{mW?OweWhlm2KXJW&6^Xy)WKZBJy=Pw3QJbQV(f@v_TS(Ziw`a zM%E%55LvSY?N^9A9uA>(&dd97`hIx+45u2}nn-;_ZmYb$HxIiV+D=Fhq#q)$=flx% zM7}}F^~ZZ&MDCB8XyyDCzX{9pk;H_%BE6A;h}?GB&Nx`|HZl!)4_Sn)Ky+Wxeva%# z_9H(caz4xNEnj;gcVKxfdW<&x0DG-$LvFY1%QNVc{dyj40i--q1*w6wMA{=VuQ%E$ zh`e8zhcIys+kfhSz*30Ho? zwVrUZC*1xC_k6-bpYZr6JmU#3c*4t{@P;S6^9k>N!pEQRg(rOd2|swk>4&(tJ=+t` z`-F=;;c`#d_k`aXQ=IrXtN{o zIFJ;8%OSOqrbsKK6C&qvZ?wb08H0A>6JJkhIhifCU)Y9UP!xlJv?t+->@@;SCAEVrW{+Odf9^>kQngFIKtxMVTwm+wpHn!5n!nD4$2J!jU8K+KA*p@*)Kg>66b5PHp+zATEM_`5aUV zt-SZIfL8XSI@(5vyvKeMt=wLD5ACd%_q&ceU|iA#abAo1z{8&ST3*wou7$oC$VZ4+ zvJR2w@r`JAA~H{2pPc&gIwgJbdg1iRbGP*E$Na;Hv=Zq%hV~31=YxEG30`jC^#|X@ zT#4^r9x$SfgGP*T#K>qIH8LB27+H)*Mm8gzC#R9#^PCas$%pT8R!nOcoABaL3vIIq zTsS7&_6WD5!|kkayFA?P2)D<=?RB)JBHemYw*|g7(&4|XhWnrV`k(Wi?K=D6Y}eTj zXS>dRINNphBQv%mx8t96|MTlKI6i4{EFy6nqKq6kLIrSy3gHM9HZmDSFjC5R#wd>? zRn5q1)HJdiwJ@uJ@vPCv$Z0e+avAYPZlk4<$LMZ6XY@8+Fa{f5V~7!J3^j@w!;Rv` zXrqLYh$`caa>gW7dfTXIOg1VRKV_(FoXYUJaVA4mBb32s{FWik_&q~)<9>!3#={H^ zjYk<88)+h&7-=J$8W|$}MyAMk2gCvuW8C~}H1IPx81MC3GMWaM;XOymq>T;xn+V&p7iapVU^ zQsjJNN#r79S>$44Wn_}ECUS}KY2?R7a^zCuaO861NaPCRXyi)cRODLxjO!EQLgc5$ zFOlnw-y=5~HzGF~HzPM2cOthK_ae6%4+)>{f&*Q%Y=ZiXGc%zOQ1)@$Fg`>_IFGig+ibSOt zWuq<_6{CJNszhBed{I}88c}~3HKT4A4Wn)vO`>iY&7$ra@lp4TmQfFkR#B-&>!?RY zyQs%Tmng$CF3RIcjEe9~iAv|$9F^YlWmE>wUR);btEd>y*HM`~`=c^@4n;lV`93O( z=clNwo>NiTJm;dadoD-i@?4L~?YR+^*V7|9*3&b(kf%@di=KYbMLh$ei+Kh`7xxT_ zF5wv#UD7imx|C;BbZO6+=$Ad?qRV&^qsw|GM8D#BE4rNL?dbBJDbW=???hMhOpkum zGc&r9=iTVap7)|(^URH|;+Yryx@UfLRnNlcYMzgxeV)b9ah{~;H#{FlSNAN7uHjh` zUDLBFx|U~kbZyVt=sKQt(RDrRqw9G#L^ttljBe`rJlgL`iEi$>5Z%J_bM%{@8__L2 zH=|p5?nbxvM8vf5M8&lAM8~xEWR29Jz&ddA1x z_e_j==$RB_L`;rJ7cn&^GGbax#)ug)-iVnor6b;rc{Sp_m`V|IW8xy_#Z-@&AJa5q zVNCOgMKLWRmc+anu{@@I#L5`_K7Q8-qi`)eaTqtCqN+#id$yZ1nhz?aqlKxG9IZul zqLwvs){bW16lwwGZo}QKyVl*S6n3e;Tn z2x_B>=pJEoP-TSdzULwPxV#8;(b3XSFI81&fXdygCg>ZceesZ8r45w$k9h;o_qO(p zfM%)2L+`5=LLaJ@Kz5bY)LzKms~@4II`1O;9+ttc|7qWCu1M3vsgjw>2ifyVv(G&) zHPL6^-sYV5W5)<%wN^H{q8}Ai&e_Hk^x4N?AxD=$>vgTW2dmI$?}587+qvQ}M}MK@ zxY#S?H*##beMV2GFN(?r*~ib5F~ZoSdr+LCuX5C_Qk8wRInRB394Lxk7SSq=IM2Pe z{;uwfzR6Md*t_>pCsuY>xc6IcjN149U{)DTxyLz?eeY4rDEFS(fj;~G*az81?I&ov z-j3&aOxwo!trOLe6tPyuQOb&9%2Wg6^Meu;{s+E7ENnbaz3 z4|SHhM`a!2Y@;|;lj=Z?q83uysnb*{_54t$@@rHZY817Q+D@IOQmN;MIrUzq>QWu4 zQPdpj6Y6W~JawPSI^5YtF{(P%mKs9Mrjn^2sXJ885l-cbR7+|EwSd}AouM94`A0hS zs!{E!L~1GZ4Rw{uG|H*+GUca+QuC=VsPk0BXs1eHsxsxLdQ)#vi>S@iVd@GMF~-@( z^He#iA=Q-{N6n`;Q^%-VRMxT1HcC?ssb16+Y6bNb^)uxe=hS0 zJ=AIH78PSTTPsMtM)|2;R3f#I`keZnxa;DQR*_~8SiZ4 zIqDUv0o8>XOUkh(|3PI1=Or}|QJsV}IXspzTB+A>s2Y8)2PTn5sc_qQ+B6RFFDPJ*Kk1?^G^9)uh@}!>C!*Drz@%nz~J8n&)ie zMXDOrlIllIrIu0hTdnr_a+MY@;mIfa*$F z)BNRz!PGoz8+De#FJFbXTac`w8)E`vNB~F#ssm|29)Nbk)Rp?`9ZF6c8wVAq1is7H;W)>3Dw%*&h=m8qW8 z0%||?fGW1!S=*YLN^PUAQqQe$R@9?LQmd)$)M4ru>H+o4N~c~SsuIO>8vrc+C( z&C~(vBz2j(PerbB_8>P^gnE^#L$##3Q^Tpr)Cbf`DoCB7?o&BdJNxo7)qv_wO{4pt7uY_Vi_{3DuvPMXjd}Q#YtApRpd*gz8UC zrj}Ac>I`+C%CUi!sRmScY9bY&zMxK0x2ep@PUXB*DauDRqdHSVsmas=YAv;sI!s-l zZd2(uI{T87Dn?bMno(V;k<<)o3AKsZPo1M~Qt3Z;_TV|H9Mz2KPtBm#Qv0dPRQgR$ z}VpMIaGnGg!qP9~fsXJ7*El%aqR0FCTHHlhA?WKOE zBDOm9VksZhff`GFM14V>rXEn)wmFqcQT3><)OczUwT(JX-J~*acPba7YEd1jvD7l^ zFqKNZ_=QuiDK(l}K^>tUQAKt*YyH$%Y8CY(6|vJ^GY-%rc zmnsr;RC{H*G!^l^v$iPJgc?RIp$<~_se*@{DvhWS)Kcnu>H$^w2WM>)Y817c+DBcdavX8i zR;Em961A2(LfxnGA9bqKpn6iXs7=%ed! z3)Foo+i|B#391%lQX{E3)H-S(b%AasSpGu~Fr0!5Te{!}_k!neepcYWusWa3gD*q{`UNy?3Mo{ll$<$%$cPjd{ zQ|~3J29-dKr4~|KsAJSkD&~w+xd>I0>PU^FKBTr%$EjOXma|Ufl2kpa8#R$yM14V> zqV7{U&pDOLQO&6S)C_7ZwV%37r9bb~D@46UwV(!2)2J2HZt5&`pUNI`wo!_zOLeBk zQ46Ro)G6vAl|RL)?4vqT6RG9YKI$r!@q$yO6xDm=@IBT0xG%HY*(BrHL2m$8tNRC^{TU?1~rUYO`WB({N}8v zP7S33)FJ92RqUFxwk0)*+DKiXvi$C>s7m#w7Et@ByHt+r&e{r8OKK$bA+?i|>-pI~ zUx@g_Q6Z`p)t#D3t)&i8e^EtlIQ8D7CQ+YL7pd%jIxF6w22cU&d+IS&@}{%4Ej61u zK&8Ls%=1wrsg2Zis>p3;MMvsG>Nu6U{2Q=_Rl)LLpUb%wf4W&F$8Mj@&S6;Ji1CQ~b^ebi+t z@_|#i2vvvbPEDp(Qv0aORAegaQ4OfU)Q8kw>Kc{fp;M&_)tQ<`ZKTdnX&*UjOHlFD zVCn-)EUb7+o@8BszddpW>K4{GgO3;CVa*gp&C(xsRdM! zxH)OPA3l_|68O{PAhPEj|h%+XGjVpJ`vBQ=&ME5P z*B=d^&r}O)B(;dzOo!a|Sj~YxZqV`cYsl3lPYim*c zsRh&?>IRh$*GCO+ml{SbrGB6uQ^j%3RC{ep>Md#mb)L$ItEbv4%29H?Rl9E*wS~G! zWx=&o?RizG?$jJANL{0HH}&ob%V-_tFhW^>rjI!xoWFDdVso56~a|p z?a?OGXlf<(BlRbh4cBkA*SM!a!Tm{yy_Zrof8bd9jHd8-Re^QzA zIQ5ECb*L`Xcxo~A1@#kkm&*Q}Q~718KGl<&Lan5}p)OJBo_FdMqN-CJsYEJ3eMz08 z9#P)BPUY%US858iminH$Nj;m-sq!k-h8jyHQG2Pg)B`G4ey3hJswvf%dWTv`eMMcM z{-*Nc`oZ=&REcUq^`<6K0csodBXymM^g5LbP*tcFRDWtJwUpXPouF=0S!11Tl%ncW z-Kn>zCDfPH8R{=8X91`3D^wGzKQ)KiLY<+Ef=-n}R2`}}HH+FvounR8-a<~j8dO_q zI5me_PaU8xQ)vr3^}JLSswFj;nnkUp_EA4k*QqowI@`!im89OF-lTd{iPZbl8Y)N~ zqpnhaQ@LMq_Mi+^kLpB?rsh)XsBfqc^%s?`h_j92R1K;n)rXoyEvB|mKTubxG)0|l zb9eQG0h zgt|dxF5%QGLDi$WQQX8mmsStIK%3RvnT4AaR<)?a5iPQ(w7U~RTyzEpiMAe~sQ?saz)Jf_g zJKV&Ij2fVssYuD zno6yr_EEo5>B~Fy3R5+xPSkknV`?AuJC&`1Q?DY`mP(|SQ{PcHsAnrWRVq^*sEO1{ z>M(Vm^1kX+sYCUrKA?isWhzT0XYH$$Nlm8KQ-`P4Zu z)VtIs>NI84aHQViuIn-9_9F?Y~Q>6&ikQzwMqBc-JQg^A`wVZmDskYQuDnRX~ zE>h`hJ5`EM^{8&tWNH<)pZbl;SjVYXl4?Ztqh?W`QAem-RF1l=N7bXcQ4^^oYA1D? zx=%e@FNYE1Q}rcleNuc#39HQ~7nOEj5Z-Ky9T?QunBw4V-%Asd#D-^$xXy z+C!bEQmI@GoyxCJjj3ML+tkO@m(*$M9+j(+Q~5Qj12utKL48O4L1l03RH;a{p~g`k zQ{Pa(QISoYDn+S!R4-~8^$B%|`h&{S)Tvj7YDNvB=1`wgC#XkM!Dddqx>P@EF14LX zp)&ZLwIwM(HG*19eMMcRa>YAU;;7ctFlr9<8Fh&Ijf!mU)GJKAL3N-;Q}d}!)KMr^ zuZrYebLu+!?CVRpS8R)Fk;X_{TdqszUb`(D`Z8;ud*vnfdP@Z{YJY;PLA9esQuC-C z)CuYVmG4cbo?LM(n{HRGOPA~agFSc!`;u3;TM4qSNabE5uNwR0+Fk{8p6s(dPp&Rj z@_)Iu4%arJT0^hsioWa{M7dYON@SmVrMQ1z;Z~OGhyAbjZyG9B)jgO)Ere=o-^bKC zYAa-)Lq{n02wg^>y|o+EBPxAMXVg6gG3aZkTYCX&sj38ZRLRxU`lzP2jW7nOYwQ~Gy$x*Wp>YnOn^gUADgzT$@dfGV3NxeXopxnoTdj&oB2$jb? z`)J6KOjAdm`D>xiRx6J7fimmpP{=+)7Bz6ta)Y4vrq9exa^GIkfU!=mph( zKFaK~^FJSD|Gj6ar|s2$|B?5!$D8}i@L#=^?&H^eUUQ%I++*NA|3$TRj<|c>ReAj_ zrpL;C4%^$^xtgi_%Ko$?>E8SQzBVhiYhUZMAmy%bx8d$pMa(PxkG*o|J?-5?4Xm)= zJ#?hTP;;rzsQpxkdPqIn&Z$?Nsz-ID#!*RBkUB%%r82d5D#udQsMgd#Y8JJI+C!b8 zZc_4`XurOt>)<>t-6QTE=NuTd?-}C=hbMfBDZb)Klb2h&k+gO?qr?k z-h+d7reRiKM+((Kl_nFXr#&jQL?xWUyzH*;;Cvxwf_SpK*cQW7WUdiVLTmNrA zZ`s@U&-WlFbr0N6RPSN$f7U+tJ;>AE{jcDWbUz2U@BZC)Mtd;tI=69K<(}j2bJzhM zHTT{13HCkhTH`4k{mYS{D0>isC-hgZ<`<6u+@Qb-*LO|Pu+W8?jw7}V6Jc<6{FFYwyu04 zmz+gIC(R95Yd?3o?|0qz!0w~WedM{1##30EQ}@7q z*YfX=M)$q)?^tW!4)?5cUk%*b`#0vJIB;r@wE5Zg)uL$?$y74C#DqU+27-F z9|7&qXCF!TdBc5o%Ql5 z_uk!S*=}{M;$H86KKkri>ApMs_hTxzsF7A#jkWf9`VC~iipzVNcpW{4zSgRzeZu$^ zqxL6^G+i8dsY+Bl)sK3IT1X{R-%)3&zo<-Iovpn@#Zx1w0QEhUNGOb=&kUaCCRkQzqKq1I9ds7use zRIZ-RHcC;Asb17HYCUzFdQ8Rka_Uu}T2Q^Hcd1XQW7I7wUvH;gWvUHjQ46Ro)Jf_( zW%P0C6{4z99Vm<1NS&q9_jRgxsq&Pc8bHmYKBl%%$Eo{NOh0EE{#apuLK_Wr)91ehP#;y3#?H@99wazFJ8}CW@jY|<=P7Pqe)ipN zVT6C~64lBG&s)*P>Fd2v4xjd$uLHge|G=DdsboxXMKJ6+e=zarf3v6fEX%j|RK zZ9o-!pPy;v%xm4m>2v43&b4`3JM%I%bmrB@sQojgrkv-lZS$1AE>G!`TV(G`Kh}G> zymJiPW3{f`|J)}>&0Z__rM)lm$~pv4uD!`)x{C9* zaP7{g^kp@jBh-mU&8^%XqxPOwt?$g+%PQZqvYf^CyxTbc>|1${XX0l#Pwm(9r@dl- zg;Dzz`zn=rkkeP0>P$_eKBvx65rdtz1*v+}Kx!eikNTa;ImD?FM|Gy=Q~RlhRPmwC z+P2hN)HdoG_3SWbMQv&%wUjzar5Wz5C`q-Xrchg`pQ#)poV9V(aB4BNi%Nm)cXM~3 zLHe9irH=F0huToa$TY?X9eoxWrz%L5g5J`;%2d9ghVhQ})kmLwmbia;tTp?3Q{$-V zkXaB3-R}43v$uPUO5waaRLrO}#ynk-pK@=dd)rE5 z)ZVMtp^tP$1L$K_N60>s{UCe0!y$YBrc#SIZzZ%!*Y1EmRqcoDwWlEa_FkfHaWrDI zGcODE5><+-MAd-odVcnGhCbKr{`1)yW9;*^haKkfWiKixIOT6?=Qq0?G< z6Lenn|Ifa^uz&Vnn{)TC=~(>K@*n&BU+sHZy{Gj(?N~jnZwvOtekF1D?@#pEj}-ZC z@^8AQ_t5vJ%KffnnsLtkkqLUJqwY^%pZ2p^_b0OdCqHxjKUw=c_u2jHdhU0;3Src! zC*PAP2H9t}e6Q3#La(6jpU2Am&Q5oX+TV>D0ogwZagXU!k3Ht$PkhREfOtOtjXwJ| z{{CD~cn{ps3>Lmqq$@H(_WtFB@@QXisv7jqJsp6)0y=LLWUqbNo=(DOsekO190U9D zc8F`;N9}R;$#YXVUHi0i@@ZRp+HpB6d!YZX%Dv@JoA+Fz^PP8hp8IoM6QlOCMNg=* z?$sdn&7hV+_STXi`QoqS zixSr`RvEXUA+&R9jNgG0*EH4{cc3A(cN&a$K#6M{bJ<(2a%?gl=Cm<>CrVuN*v@zW z8p8cf2dxK6Tm#w3cpw@=Pp2Q_y-?!1$o`DG(GYq&0~n7%iR&c07!N{2c)%IRcr0=T z%2$Y7IoV?Tb3ZrZjVN&)We?+R&=5BI1u?z}B_Wir;Hute2*>?`89#xN(2lPVg*(s? zPWrWB{5O=i4s$5u9ncVd_iM}ed6a~Xd<9oUM??6-?+(T} zOURj>5Xbl^l!R2Sqb&-fkt@s!@$_d<64JPzB;h%(uq_D_xQDpnIdXMb?iG>iYA4aN zxhJ^#Jn}Eh`0YimwVh0#!fz}I)A{{GVFpTEb9)Hp@f(Q3t0)Qi{M1}^9l8E2KdC6Z zj*>8opN^}~@^gt?l{=L_ho4In-a<*3%g@EN)KL=V31jK+pd`!}#?#+L&P@4vMd3Y^ zgatx6e!@>J3M)_&RtlN8O2}g5Gn9nY{2rpP1|^}6--GMFqa@Vxdx*k1l!W!dH2MaV zga&>yQTQCWN~w@b--wd1iQkXw#v|9)gh^q6ZvJFftq^VT4#qABmDMN?b)xMM)SduBMMeuG=ft(Vs*1E3uybJW4{kxSsw3vX6-k zI8kh5Boig!MR5~73nk$taWg#|C1H}dm7ar=Fj?G2pMsJwRoqUWhLZ5IxPv|&C1HlR z6LZB~jJ$&Elj3grtH?en?xDYiT;o>UOD{u7_)y$WFGopODjuL$pd>6457H}^JQdxkUTg;@?x$Og0D!Sm?yQvSEcrtFNNW2Qb(LAb;8%B&R8II!C6u` z7D^HLh7^gjr6_z;ipDuo4}44Ng>$7Cd|Qgec~Ts{BlW@gQarvZC18=%7vGZ-ae!l*xAT7WKX(4_t6=S1Rf*Yk$+$5FZ7g9NHmMU>G(rWxhs>7X9J$@^#$6ZnbekV2JZfO&KFKxy>(pLOI z+J<|j?YK|cf%~PM_@lH74@kT5Cut8Jl=kA!(tbQ79l*oVK|CTI!lTj=JSH8(U!*2% zl8)oA(n&lnox&5+89XU9<8M+6o|4YvY3Tx*?5nfgI(k)c&|JS!{zCC zpPY*kavpY-^D$DMiQVJ^jFJnnI~O{XglKsV-Y?I^9`ZcwDbL4VauN2H7hsIM5Fe0> zF;*_Y2jxBq6ln>!3`3R=U$8faVgk$96_>6oK)8te5tb7K?%FQ@VZo%>Ld3;X3fD`14 z_`G}x)8)(ff_w!t^I7N2iRM{V= z$u4|Zws5-a!5Oj_bL9|xMGnP0xgEYLx5s=r3}2Hw;!L>{zAksh0{(mh>%Sb1g>nSG zAxGkDISSvDqj8Sh1K*N+;aoWe-o*ako@ISe-{>$u}v zQclN@@xLPj6 zHS%n%ljq=Cc`nw=^KhL!AJ@x8xItck4e~<#TrS2&xdb=LrMO8h!!P7=+$>k%7P%6) z%2oKKT#ei08vIILf!pO;{90axJLJ{)ja-L2<$C;9UXQ!v2K-KL#NF~H{9fLSd*rS7 zgS-v*%G+_DyaV^kJMl+(7aovz<4^J)JSgwQpXL2{NIrmv<%4)cK7>c*BX~?chQG*7 z*d!mvU*(f{Tt0;-OZjU$RFcg%IC@P&$QaYoobU{T4M^%YHO^HNZi9$n(MpNm5Hl-KZl^Ap=vFKFd z&`;@u{z^OsC<*9N`eLAxh?bIsZlynZlw=H22BB9OjKRtf3{i$*8zlupmEqV{8Hw$b zRJ=nOgYA_x?4XRrFl9X6sZ79*N;=-9WMC&H6Yo~Cu(Ohl_b55oMVW&4D$_7rnU41< zxfr43VOJ#|BbAxhO)0=Ar4YL-voTtkgZC?Qv4=7bdn)s>mr{hil?521EW`(tVvJQv z@Ij>%^B0jHN z!gS>_zMx#e4CN|LR9Z1pxsER?H!({QRn~t+#%x8!Ns5j+iiwjIJ5EuYI92h-X^IP9 zRxF&ZcyNZ|#atx>Ur|CaPicp*D(x{}3B%WvjyO~4gs&@|u|Vm9vy^ZwR3h*VB@$;V zQTV13jdPS9_?FTO=PEJywi1i;lsJ4x>4Wo?czjn$z#^qDzNaMO0woFGSNh{ZB^ehf zgRoc`j2|dNutXV#if+msso zN?C#1m0J8-S%o{4)%cB4hdY&e{8m|yyOaj}PHDv5$|n3?*^GOXt@wko4fiVBai6jS z_bWT`M`aftP_^)E&RmFqJSW3hhZBv1w+;0*j62h?bKAfLmh+d)ims& zj>Rx_Jl?5Jz>aD<-lb+>Cp8oARP^FkGFE_o=xUq2^&%H6J6@ znb=J&z$mp4yQ{M?TAhRUt8=l3IuCoQ^Rbs&guT@T7^5!42h?JWRZH+ewG`vjGJHra z$3AKWKCD(^yjq2isMVOD*5IS+3hb-a;$!M6OjK86KeY~%)OvheU61|M1{|O^VzRmk z2dbNKkh&G0P`BY=bvr((?!Y1HP8_Q4!eQ!ed`jJeDe7K)THTMs)dM&}J%}UKLpVx3 zf~o2;9IZCt81*CgK7$3Ex-y<3cqV7pa4=SRITXs6((s z9fph56f9MT;}UfwmZ_=up*jZ3)ihkHj>QUfJT6lwV5ORlAE_BwrDo!CH4CfNZ2VZw z!5Vc6exgpp73y^SRL#X&H4j&+`M64*iJz$jxLPg5HR^1vQ|I7XbuQMc^KhLyAJ?lz zxItZj4eCPtTrI{%wFEb+rMO8g!!Oiw+^kmM7PS($s#W-ni`7N)pmG8ZI3tAFch?oC~BQh(mJE8 zbwNc7M^%eJO^ZZbi$X(-MpNs7Hmw)hwHS10vFOy|&`;}w{#rZ+XbI@j`eLA#h?bUw zZmmChv}6p@2BB9QjKSIv4AF*R8!ZJxwc*%S8;R|-RJ=nQgYC66?4XUsFl{{EsZGF+ zS~}jPWnd>Q6Ytitu(Otp_h>oTMVo^6YSS=Wn~wKsxfr44VOK35Bej{>O)J1Ctq{9w zvoTtmgZFE5v4=JfdusEsmsW(mwFMZXEyM@3VvN;F@IkE<~N;`t7+A$ohHQ^ZT zI6kAD#5C;`KC7Lx1*Pczjn&z#^?LzNaPP0xb#O*ZSi^Eg2VSgRoc|j2~!2utXb%i?tLi)rR8| zZ6uaysraEb2FtZHT&j)53T-?t((WSj)j0Z3=#(O~V!1 zbo^Ay#ab;7S8DmVN}GwFX$81iE5tS0Y^>Af;96}i)@$={oi-oWYel$0TYwGPLi}7S z#zw6KH)^H0Nh`xIv~t|6Rp1t_61Qqq_@!2j+q4?|N?U>3wOagITZKEc)%cB8hdZ@; z{8n3!yR-)UPHV*7+9v#7+l+g(t@wks4fkr>ai6vW_iH=xM{O4#(01cb+8#Wp?Zuz9 z{dh<_fQPk%ctksdN3|n(Ogo0ZXieCp9mij_lXzS^g(tK#cv5S|-?SDyrJcvq+66qL zUBt86C2ZC%<2mgLwrE%JcdZrAYuE7)?IvE(L>ueBCgVj-#lJKiFKH(Jt=aLi=EQ$A zf4ri(@L$cstC|P@)4bTKh2S+U6t8RT@P^hNZ)#yE=p9kiJE5d^Mp^HIiXM)t9)X%3 ziMk$zh8~Tk-UDrVFSP41=+I-)smGz8-Ut2lcnr`J(53gqKs^yHJqg`uK0QAB$o7c)U}efF1R8yi3o(PI@NZ zt!H6pJsa=QbFhm(1@G0TVYogW@6&TJLeImldOk+#GqIaqfKhrOcGqWPv_1##*XLpn zeIEAI=VLFu2z%=bFh*a959q}htC!$|dMU=~W%!U@j(zkBd|0oGk-yz8?GQ4LCq=#AJOF4%9c}Abl%7p>M;%`gVL$-+@E) zoj6qAg~RmS_>{f}Q}n&~w7wsQ>j!Xzeh^3Mhj5gB1XJ~6I9hMQG5T?QMn8#Z`YC)? zKZ9fSW*n!t;CTH!KBr&63Hn8RUcZFt`el4Uzk(V1Rh+1|Vy1o_U(|16mM+>^|8*I& zbrmP+I_BsmPS))>MR(#<-5;mvE__+HaJufn8M+s9^$>hT55+va9lol!$9z2uU(-9{ zOuZAnu6M=)y$jCL!?940z&G?roUKRUn|d_P(R<)qdM})-$Kcy~EY8#8@EyGm&e!Ad zT|EJd^uG9>o`?(dBz#}*j|=r=T%-@eVtp`vpbxjX&vo@SwgIf7bWoA^iXz)(_$l z{SY42kKi%=82+L+VUvCwf7MUoas3pY(9hsWy%~SgTkw>A9#88R@Qi*D&+3=3S-*_u z^efn+U&Y_`Ry?m?$3OI&ctIB(tpB=<7j+f?(sjI~oA|eG$IH4C|Iz*NitfUHbqlZR z9{f-DVyhm4*Yr@luD8P*dV9R7hoN9}MA7JklF=DuqYElVII2bjYDOgLMid%GG@3>a zv>Cn7Zp5I&h()InhkiyM^f%%$z(_!s(H8@aM6`?~bQ}HAVT$i)aF54#%q7-`JJZbkt{8HL#0n2piK9K7F{i#?2a*wdJgy^JF4 zZ7jeTVLq<9FF)Hw3qY~qdDtyGK#ss4VA2n8BU!xWuGge`u zu^Rgsb(mz-~A#S0HYC;jZHYv*o=dWt@wno4F?z7db_8VOir^u_m# zL|kAb;rm8^TxcZYB4ZF18-wu!V+fWQ!*H>Yf~CfATw;vGG9wi~G{#`Lk%mi+u~=b@ z$7RL@tTfW`BO?Q=j7(f^WMQ?DjUO91SYu4VPmF1}!kCVq8o5|&v5OSfZrL7xZBu--y558 zkFgbhFt*`dV>|9McHn+vC;n*c!UM)`{K?pZ2aUb>v#}o!83*vNaS)Ffhw!Lz1dkcV z@E4;An~dZ5t8o&K8>jGuaRyHs&G?(qf~So0c-pvtXN-$@*0_Yt#$`NbT)`IOD*kS? z;(6mb{$bq23x?>&`ftd1(NOU(L&r;oiGLe*ylgn}AHyH77%u$Ru<)wk!T$^|wi+RL z%?QQoMmxM=w8xu97z$=b6wOX3nVnHKyP#r*qiRN=W=5iJMxkLwqiOa)o7oHPW(+#a zSah0k=x6ppe={Bf%mj3qeKF8XM9WM!jhTX>=5TCl zj>L9mD&Ap^!S-evb}+|cm^mKrG$&w3Gac_TGq97HiFcb>*xAg+d(0f{Vot$(&1o2J zPRIMqT#PXDu&bGmk>*V7W)@(SS%}@u*%)ok!TZg**u$KMJ2@z!bi+%OfYNkQF8_MHEZ!Pa}_38V zK5nkZ{$>LXFdH%1+=K(o%{a*1icgr^aIm=@pEP&i5OXIEHFx1Kb2mO^?!gpuFFtMV z$KmDy9AO^Bk>(*BWgfv)^B9gcn{bSI9G@{yVw!mhpEb|mShE?&nJqZpJde+r7jS}k z5uZ0NVY+!4Uofv=hIthynyr{=UdI>Bo0w&a{;dC|jM=7&lT00ROcN)YcAR25ajNN$ z(@Yn>Y+5+o^xzEBi@9b9zG8-Ap4kpxHQQsp8HTT!9dV}F312rmV}aQPXPMzxXhz^0 zW+cuwqwq~L8t0fj@GY|!&NXB3Z8H|l7n>G-Lci?wDRt~B#;l{phXGYfFFS%_=Q*;r@J z!L{aGtT*T3I&(g*H;ZtCxd0o?h4{HyjE!aqZZu1AlUasenB}+-I)GU(5!7QD{O|W3!Qd99fOcP4ttt6Ki)JCpkh0Ss_hV( zwj*e>9pg6b$a-jNLZ|IG`q@sRzwHzT*v_EK){KF+7PM^V(QUhcA-0R$LK|e=vt6RM zMbHUy3#HM4i&BTE=I}Wlraj4CoIm3{(!seoBJo9A6lU3?@g-Xi%(nHy zNwyfwvBlzKTO3ZY^}(sOc${WSz!|o_{8YKfv2RPH=OIVEEs6dba@5=U(_cr9cUv;O z5INp$gXpu7Bi%Nb{uXki+lJ8JMvipbF#0>lacxVXFF=lK+i?0qlDnlfDT#u5AVM&B$?WE5xm~+4!Yx4sNr}#jkAhaJy|jer+ql9kvDdjcp-b zv=ws;e<8=Ct%QCRIUa4L^j74EvzO5| zVn_QbyxYDSJKO8<9(z4@v9HJ8_6B^+-iV3zP58WhGv6y6*-PzP=^4mgYTriBMD|kq zc6t`_DBE|?OOQv|zLQ>xtk3pcSZ&{pAKUj}jeReEV&9K-_5--Veh|O2AHu`-BY4Dq z43FBI@Ra#m?SPyZb`tF&LB|AE}Y_KVnR zzr@HjWQ}rMMnA_DM*NXA%5jzMLRKe7E4m%m(c`#@K@QQyz3q@O*r8&GL&rNDrb`ss zBWGm}J3S0pqa03pN93%`;ZN^`JPHmMy)*JCI4pV>4Sl{wneqmi>RM;N^aa#rT(NbiL_vW`yl804(X(U~5LoRvAc(BqJ^GDkSQ z4{}!Kh`@M9BtGJZVmtvkD|1BC`y%^^qX#_^IV*GYq9-9|WsVqne`G~<#A31|j*)@L z+UV$msg8J@saOOO@AF$}95 zDY)D*9IG88vBr^#pE<@bb2V~|JJRTN$T991i}jB2xY;oQw>Z-I>{jGxb!6Z+M<#yd z$inT8Z2a1hgF75k@EgZ8-07H(-#T(}mm?3qbL8W0$4va*QGk0Kh4_PGHtu!I!F`Uo zxZg1ke{{^p1CAp6$*}+rIu_#3j$%CID8a*yQas`)!=sLJ{LN9pPkRb^ZRMz>pFv)^ zIIA$gS|Bj)oOKxPtjGJD>-lU1vhO(? zFw)tG-JF{kk3#l6=Vp2|^1SBUN`D@CUUP1vzksY2&h7L}WUX-Spl2b^XwIGVY~=aN zxeKQ{cjGkY9(>uk7pFV-;|%8kX67Q#WzK{2Jmfsqc?jn_kKnt`V_4*D!V>3k<}60` z7UxO&5@Z!}o}w>9&Jvwx=pP~bkF%M+99f;5E%X{>b#k7^FPs-}v-2Y3TaZ=Vc?o}V zUdDsYD~$h)tS!!~__wnauRE{f4d+d~=@bKbEc|4&{M0}m3*@oz)9FFTW8r7ggOSI= z&yH>UoY>3HAN%;Z@L@lTIq}F*>F2=&KQAMXB1e{A2xj<&Vy0g^%=T-~XD1gn@ z*)#n|(le1g(?6A-h3uLBW9Zq)xt4z#JqJ10@*hi|f~-jX<8hJy1T6MX#}E87u*5$T z7yD;1vlKbQ@z180A?uZY4!s;X!||U&uRzXl{HNi1|LKfuKwd%k=i=x7dD!Tm&-g}U zMe?7C+x-jhYyU#r;XfO9`p?1N{O2 z|Aq8Fkau1Di|KzN`?G%u{V(Kwl7Lba1IkbeD94b13TzWliFX84F|$4L+!s(y4?~{I z0&3{t$a)g6f*ygaCjqteNMt<;SVfOQ-dhM*O}`&`lmqH8KA@hFN07ZeU_Jd&e57hMcPhY^J9m=js7l>BEuN^#R-HBax#fU^_h(d0ii{gFXg1 zPY&2ge+_v)3fM)TgS_Sq*iD~{yygwqL!XBnBLREq^O3bEU_ZSGIX4bCKwp47mjxW8 zFGTj7fJ5|RWX}mWLN7t~oPcBWQe@8wXrh-PdrrV{dO5P^1e~N-AbU>0DS9PxZX9ri zUWGgx1~k*Fku@!#gyN7oJq|f{a)s0T zAgi=10^?ngj68zux2`DqqsV^iil#q?oMXFs(EB0pXt{dPA4lHNa>dXGAm>i5So%QZ z)u=0u{si)lma7l_N#q?ZS3G?v@{X1(f&LWoj+U!0{b}S?sw7$W%v|P#bXOMTaT!ZM(BClj!gX!atSF)}l^yiRgeAh61-j%{gIClC%e*diYo)Bx-zlMmBlT5h&=navgu2aXCGG% zeHrpR?wUgX2zefNO`|VI&I4W3=^rC!gRWfqC&;@!t~~mu$h+3Ad_3Wri6>nJ_?xQ` zPq}6@=QQ$4!8Hfvz`2Yl$o?5Pj~*O2pC0mmY!g^S4-H&^Nr4Ob?BmGe7+6dnfIN0^-h8v|=FEpP=s8(7Qu zSme1ca20(#^4u4=8ea{pVhj2sS5q!uxhJCCiKKn58y2Cn-iPlL*`XTqZbqXK1 z&R~D583$M`m~5TLfz|~aWL?B3tV=l9x{Oa+S8#}R6^B}_ILx|^PgysSecH{lh9%>0 zOT`hEjw3A-M_G1EwVXKG^2afj3!kwpOtU=rtmVbARtS!>LUFv+4xh8y;{+=VpSL<< zy449^usUOg)deS7;h1Sf;EPryW?51Ak`;~FRu7zH^}-x01}9swIK_&?sa7AHX2s*n zRsv49`r-^L5p%61e8uXIc~&yMY7N4CYcRfM4Z)e#Fnrxg!2)YI&ay^gp_PhnSYvRu zm4#@#iz_nH* z)?1r!owXU)TU&91wGA7r?fAL10~@WKxY62$o2=dVg|!DaTYK@iwI4kh?wcRT#t-5wj=VYtQJ5x2TK;n(iYxWnBAzjKGbdo9{7v97dE+L@HclXo^r?GId>myamVAI?gYH(?u-Ap6Y+{W39q^P z<8^m3N}fR|dj_N78G=sFF!b}Jpye5kZqG;z@uXrK&lqg)Ny84FvDnEo9`E){z$8yP z4)kQ;AWx=;=Wb;E@nqqXo@^ZA$-$wXDLBkC4WIH%#}rR4KJCfF;huaP;hBjeJq0+* zQ;4aa**MxW2gi8k;xnFknC6*}&w7e*tY-m^^DM;io??8?Q-TvbrTDz34AVX3_=2Yb zGdz_z(Nl$)o@#v2Q-fKa75I{;7PCF8aFS;==6LFGvZo%Wc-G@oPXkW#G~&yiO*q}N z8E1I5V!me^e(Kqdb)FsECu@=Y!LyUT4mox_yXYH`J;bvc8$Ek)qh~K}^6bYiJO^-# z=OAwN9KtU>M{t|x7=Gnx!tXuDagXOD{^&V{2Rvu+Cr>jT^t9k%&v`uJxqwGK7x5R* zC2aCs#$P>G@VMtHp7ONfY0q^$3EMf1H-+U*v*@T z(cWx~_2%G1-YNK~cN!*or(?1=7oYIv;Zxpx9PXWoqrC-~<}Jhw?`(Y0I|p;Tb8)J7 z9_D)IC&4%AvysOr zSoCs!i#(SF%k;U(;}xu8NwALP!6xHNku!&2JAE1Q>>upJ&w~ANW3UT12V1x^*n_)* zz4$|L2<{CI#lykvPzq^}YDgGHgmlDiA)T;aNN4OH(gg>HgyYbV2%H!aiCH00_;N@z z=7#jZ;*ehWK}Zaigv9bwm50R9mxlDAS0HCHA@THTJ8zUu5DvJj zYp7FPa@Pm$3F5c+4ZU}!_)WwI?n3c!#7v=3JQ@+J7K*<_6wrT-D8v&H@t#8Qw}>YC z>4-SBSiHCEA^s-QeO+VeUAqT4VCE8W zATyVUPcUwW#R~CmWiX7StgEVW|{a5Gt0zhnOP=|V`iE795c(s zmzY^DPGV-cIGLH{;#6jqi!U>?T%5toa`6>rE)`#A=2CGMGna~QFmtK+CNr0cZ!vSJ z_%<__itjM9LVTZ@72+ahR)`-kvqD_V%nES{Gb_XonOPw&W#%$*IWw1uA2V~A_z5$Y ziJvlanYfaf%f!!^xlCNc%t~<`Gb_an%&ZhYXJ)0ik(rg^7tE{_w=lC({F0ef;tyS$ zwJLERGpocOnOPmB<87ji^P&(v8Rh zp+>Yt_GQEo8IOLEO^gIY#;LVphsb8FR=ks$wc=gOtQGH$Y+>XcX4Z=LGP72^kC`jQ zZp>UMc4y{F@qT8m6niprrP!O9E5!$xxl(+PnRVh5ku!xl@kwUZi9?xLCq5Nfh)*-K zP8`9^I&l;;*NV?FbFDaznQO)8n7LMbo|$XK7nr$LoXE_z;)~3z7bisyv+Bjk%&Zru zGP7QMIdV87GniQ~zQW9U@l|I2C_dCJ+4xa>xLYj!k!}UTkK&`<5o$|Wg13R0XT{H% zc~;!mtq{Lp=2>wIGtY`&GV{E6v|A_fy!Z<<&x^k@^SpSXTW3aoW9E7BG&9eOXPNn@ zcmw|t?~aPM{UhEJ6-&Q2szCThye}%r^N-jyDjvHJtnGP*Z0V(T8KMoL4wpAjRarg-``g@34iO9za1;^w_63F z8{yK0{5?{xcSN`l2Y)G4;4gs+{Dn?|zqcvy7eNJ~jI1E_WDD6r&d(5qJ!}sW9u@wA zr@&w36a@ZUp>CWi4>8=ag&M`Q?@ zNOH*=q=ZzFbz~go?LC7JwWEOd! zl#z|(YqEzNBT2s zY)|?6t&jg^+e!>S?myCj+)Mh9MDh&DBvVN~d6O(4OGqvGg8WKMf9?y?mH56-Pqq); z>W{Gr255=#=u0FpwUB^hK2Swuc1jbt0yOPa_ZM34tb4k;k- zk~L&2IY@lRRTJB@`TIyW}3yjr1n{ zNjfPc|NGi`^u=TuX&_&con#+5PFlz{;^11h9^dtDNl)?!8BU%dS!6ZYO@1Qhi0}6Q zW_yhU1oL$ynxvCSB#+D|#iX39A(zN?qK9zYkhUa}#FEEI3P~d`kcH$UvW5Idk(WpTd5`$MelgoB z@+sLswvavKG%@br_$0-of%xY8e#Yio{UX~|BDd%HoCFfzoHlGbl1TCp@qJA{wkafy z_~xXuoka4;eDVqLjW@8}PJSTANej6~v<}?=KJVc%#X(XT2kp1Ka@e5<$Cml!^ z@*wF)Mv@mv9+^YR$pP{!X(m^Qb|+s?!pZ$4jtnH@i0|j~C4)Yl6p;5w8TpKCBtMWQ za+X{qt>k}Sr*vc;Bo+xL4-nsd^9b7{GMJ1b8DuiaBX5!tQbksh^<)eAhU_JWiF6nD zJ!wNalKY77d*07BjwIjuco^H!S zF(SQ5f09BbkV#|~DI=edO{9tVhYLao=|m#QL!=*>K&F#g@;y0BZ1?e4kwh|toYaxc6P%KB*vIksD+{B(EFDC^C+`NT!l^$x`wqxklP{I%hLb8&4OMWL; z$lcLAb|i@mCsWCLWEojQejt~L?{@x|8+>fPpVuR#D~Trq$S^XFy@|l+S~g2 z+xphq`mWphf!li1ZN2%ne(AP;{kE?6{C|HJF8uNT{9d8A^-j0-$lH3%Z9U<(o_t$R zxvi(&)-!JFQ*QPA+s0zTV5W zJ?Tt(l0IYz8ACFNZ%z)|w{P|L*)G1-tJv0(8_e;|^L>BcbDFQ~x&QZ=->t3hbKYD1 zj@!n<>0L?G|7Yxf=gPiwQs3?Q?%TNA<|NVw-}bTZc2aMBJdUmJ{`7s_#9L$kd(NGF z>to-~<9qJ)&B?npR(RXz=HKeYw~hH8C*S;vTVpl1w!YWMt8ev&TU+02X5ZYcw?6hA z^S)!TPSA1)TZ(sDCH~P*Q@4hw9 zw?B5h)qAq_tuMZH<)K?+zT5S!E&cfT3F6ydpT6yLzWZR@Z6EvA9^abdTWfr4iEo|p ztuMa)*|#72))C*j;#)&}>xXaM@U0iV{n@ux_|^#D+VCQ``7(L^)@{GV*7v%}_xlD1 z{omXFtG#oNuA;j4_})9`kbnUnNFs)i5C~9@gr^S_qyfbuf(a@jMKB}>i00Lt@CdaC zh!!7F;vt5pw7u405tXWFuC>;p^-&)n_0{(JI*8%}-x|bTd$FJ2@5~vnLRbH}Yu&YO zC+qvocYb^C-+s)@o;fq;%ukM=^SHrubttr)vo(>7K7Nv5wf9sk$jW%c7} zzJ)8I1Jc%$)_jOGDfb_TNRx7#4v{A1_H8(Hebetcc$$=Z`4DMR-|zlOeM|2>xX%rg zE8KkWG^wxc5NT5G<$qG&DIEv*Deb)F?t`aEeR;n=blT7Z%E=~@ZMk@ssFmU5TC1)d z_F|f*-LGr=K2JX&=kl+TYcFcroOI$`QT`iUyOLXKvNrf4@wt6wa&3aHEhcuy63_^i z+AtA;!o&kgCmK*Dv4DCI38)8gfDSh+z)ElhSOu;GSAm~`)!=Gw!CYgHG1uB-x$AqZ zX|lQIdfV6BUHap7PY)6}0>=>TajpHfY zMD7}&$uqVe@bqjF&&AH-iPvPFWu0wqw^O;BT*@=3nLJ(kq1k9>@$Bd#bBCR6?zCm* zF0jd#^X#X>+->J`*SXT%Ypcv=a3A0)4o_x&L@c5hk%$%%hiEZ(rI!+eXc-ZRRuX?` z714)Q5qs!Lvjse2uQrc@-xH7OG4MEeg1A#p+S|C3y}>*Mo(9j@JIu3SE7(RHsOP|T zA~^k#$W71N-dLli35l1bd0u^p$oRR$kO%UKp45-nNd=%k7yt^vK%yrN za!1?2;CtXW5CO-7A;eTF0z<(tFdU2kBf%(e0yxoS*^|J@U^F;|*h{B^F<>kam`(%Z z!1sy6R1C(ufp!8o9h?Csf-}Jnz$BtEodqU?v%wTF6`TV~++bVkrrK%XTyP#ylg=k* z(sVb|UI1ng-{?Xx(=D?Zp++jb}o^8%H0lIK}?;fD|GYR z`8xkc4IkfhS#Hw#FUR_F;uOllH|s4Q9(7q_Q~7QU)Gvb6+c*+s;k(y??XE!G@248} zzLdXd?i9_>JsdY#%W{SaZ0EVTzHNG7o9EFlp09WJ__j6fF+blL_q=br(e3hWH@Yu;+gg{&#f`LdtvklQ%=yx# z_V!ECl6yASS!o0O)Vj3PUVK|xD&Os@#LMkLz70X@#z5T>sNV!$c4!?^c1K{_8Q69P zw)a#?zW0JUb_b~+2I|wG?5Bb4s8FiSGXga`P^$vb~O7;AUK<&v&dFh#*QvI@1 z<5C#dHf5*A#r8>g3HM3m%MMiUKuzeA@;5m*rM%o!+4kI2*=K{)ZGqYzs22nEZlK-| z)CYn3C{SsAQ?(WLP1QE2Z>qM@ff^g6P6}-0LD@?KRhpObcdoBOrX^6VLF&EzQu*!+ zR9pX)?digldNxqo0<}F*&j)H(pwb4W%KmgvN?jeOa|fqviv!gds2>OFCxN;=P%8p; zMWF8WRmfa^-0^yRpybvhOUT?8s9lj{D$h!aB(*D1a(M>e^8nR+=rR>XK7Ewrt$-Ve z2`U!hxOo?DQEpXkQ(mv^nGZD&KhQO;ME&HBmvWnOj9OoL5JCXLU=&o)cOe204!wD;?rOFX) zNlBjgN!Yuv$BA?zy%R1~OEcFgQg#@h3|ieu$}-v>5dvtTyBB`IJ)!y)_(3P_d`P*K zx0Xwrb77B2RD2fIc5dz`ZD`eYM%*}3Uv?M32|EvtYs(Ybwm1FuZ&dr;a5zmkVTIpx z!XEEU=2LZ=^tV@eh zhHil)R=Ct&-&I;7`&heOi!Qb6;dc8=xJi9B5vxRMc>qg1^gQgDSKx$A+W)5ZtZrh@ zgA?{-xG8j+T25CzMfEJ@a=6_#z%8cyFxeKuo>>Yfw06&|MaPY&_WRWSxawEc{-Nr$ zOv!&V?C}l@(b5->cScCrscN64_G;BlaJ$_EH--Ka?iH31%T@Nw8(7ZDeH$*dGO7vt zHQM8C1!5_Nb9B%2%xZL;-}Xyt3*4C{dVW~ADZCxNGkkjw?sbG8fIGrl;a2k=+-^UH z6IOIwEgp|L#BwCu6dnK%$d>(QKz1|wCY@E*D&LhYtxZ@N$x`LcEYT5{9NR5N$P6I* zYdiEv@xKOcQ7*~*9ojS7)GoRyBplcL3ESl;Xyk6}aRX+oFW=W62y5D-H&(Tt96zrMl@Se0PcvE^4+!T^y zzcWilyMLN+Qz$v|!kt-?x+h)O7rgk1=~=`M#`G-s>w z=UI~H&WyXUkMDDSws>9!$IZ2>Tj7Wv^-Uoelkt5vsC^4OzK>Wuz6UQY6kV!0^L2ml zOgg8Zb}Q`h4R!HzAzYFtIbYT}@U0A)`8~d6F8mlAH!rE)DLVAy-eO;+d>-o!&D_F$&$u}a_Dnh4VpghN zqr4H0n`c!21#Z;yY>s*u-c9@u?}Tk&ZQlZ^_Y8PKy3F}aA+blC@Fjg`V^0t%TJrn>ed=M-&Q0n0{bf8y!JE=A zgt@L@4)KVwCt7CX96c^d@?_pF$rDRSp3K$p&@WYu>YF&TfQo9w)y8hx}U4JP%ZScGFw<(llu|HKP$KBT4 zoWZi63wO9;_(7M{<>+2%HE_g=#WM@x&McXEm?zOq;Va+{cR$=K?FBg8Jpre8mr~*G zQmWm?z9;Qj1K*VQ80?wnly|~w^S)9({5Y}qRxVI}4vw2oRqumabw1f;pI~l`hlD+| zCL;U+95=@uFZv|7-Il=*XG-t(qzk{PS5wVS=9aGNXMK0E7u&;z$k@u&OT@{zH4zh1 zyva4yw865iKh>GPtIkfIDI)(fg))YnSu5bJs^3=rqw<4JQajvPD9BXa0#;5>~j?B(;qE``w?Q?{1)2Jktj6)xN|-9q4BF zXZWoQ;SRGKj)#sICiY=)hZzsw9hRf1BQt?6wnq#XKmFm3%$wo3*`k(hs=3k25g~j* zk92sx@})hb7fn6nNa^4W!gA#x+*PmBI+Pdb{^psRDYZoPQss`!^pV`#P`xZ^=~00t zVXuUlrQu@xd$>jQa?P_+x!t~ly;aZnIm#dPm_15LRl^CJKL@`68}vKt3l?+2^_OMn zTMt-I+2wy-V4pLgw*HJI(J5t>HKsaNURxiH7FASLY36SVZ(V$0P;7sh!1TW?_%%2+ zYjoEK4{8Eo~2Kzg9$@ULMEXn)}zLO+g7?Tj+omt+Sdr-Qnv?GmO(M&hz znmOI89AHJxtc(8~ArR;Qb~1 zwxAwAm8Qx>@ma%OQLC8-7#hSc5bDWMV`8M2nFjLJvX)%#34Nco@;)H!^`zAB8-?$B z>Z5sv@wtS$Deas>nR41!i}zZ3m-1WBosL@V-Tq#UGF|YM>ci=@BBo_4 z$-6(-foKeAX@n_aA1N{?(Cg7=ggFs?BAUt2j56IQE2G|^aS)*s&q^aRpwyZe>optc3bIXa{1(; zEn~(lsjkXj6s?a{*4B(I7+y52AU|4DURzOFGka&kd_tf8!?qO7X6COWpD zF&ZlvcUt#uW6EN&X!YEx#(W&q#KsmZtgksGR=yxwT^1WsU0GgV8>^kyFr>V;`joO* zb zVo|?S>SX9aT9az}&~LSU%+O#E$S`JTY74QF+T#EI|9@%(>Ux_@8nlLMvwtuD|3={N D>RC3& diff --git a/.nuget/NuGet.targets b/.nuget/NuGet.targets deleted file mode 100644 index 3f8c37b22..000000000 --- a/.nuget/NuGet.targets +++ /dev/null @@ -1,144 +0,0 @@ - - - - $(MSBuildProjectDirectory)\..\ - - - false - - - false - - - true - - - false - - - - - - - - - - - $([System.IO.Path]::Combine($(SolutionDir), ".nuget")) - - - - - $(SolutionDir).nuget - - - - $(MSBuildProjectDirectory)\packages.$(MSBuildProjectName.Replace(' ', '_')).config - $(MSBuildProjectDirectory)\packages.$(MSBuildProjectName).config - - - - $(MSBuildProjectDirectory)\packages.config - $(PackagesProjectConfig) - - - - - $(NuGetToolsPath)\NuGet.exe - @(PackageSource) - - "$(NuGetExePath)" - mono --runtime=v4.0.30319 "$(NuGetExePath)" - - $(TargetDir.Trim('\\')) - - -RequireConsent - -NonInteractive - - "$(SolutionDir) " - "$(SolutionDir)" - - - $(NuGetCommand) install "$(PackagesConfig)" -source "$(PackageSources)" $(NonInteractiveSwitch) $(RequireConsentSwitch) -solutionDir $(PaddedSolutionDir) - $(NuGetCommand) pack "$(ProjectPath)" -Properties "Configuration=$(Configuration);Platform=$(Platform)" $(NonInteractiveSwitch) -OutputDirectory "$(PackageOutputDir)" -symbols - - - - RestorePackages; - $(BuildDependsOn); - - - - - $(BuildDependsOn); - BuildPackage; - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/build.bat b/build.bat index 1ebc5f2d0..921a6ba8d 100644 --- a/build.bat +++ b/build.bat @@ -9,12 +9,12 @@ if not exist .nuget\nuget.exe ( REM we need FAKE to process our build scripts if not exist build\tools\FAKE\tools\Fake.exe ( - ECHO FAKE not found.. Installing.. - ".nuget\nuget.exe" "install" "FAKE" "-OutputDirectory" "build\tools" "-ExcludeVersion" "-Prerelease" + ECHO FAKE not found... Installing... + ".nuget\nuget.exe" "install" "FAKE" "-Version" "4.64.12" "-OutputDirectory" "build\tools" "-ExcludeVersion" ) REM we need nunit-console to run our tests -if not exist build\tools\NUnit.Runners\tools\nunit-console.exe ( +if not exist build\tools\NUnit.ConsoleRunner\tools\nunit3-console.exe ( ECHO Nunit not found.. Installing ".nuget\nuget.exe" "install" "NUnit.Runners" "-OutputDirectory" "build\tools" "-ExcludeVersion" "-Prerelease" ) diff --git a/build/build.fsx b/build/build.fsx index 6072ff41a..cc48d7223 100644 --- a/build/build.fsx +++ b/build/build.fsx @@ -2,6 +2,7 @@ #r "tools/FAKE/tools/FakeLib.dll" open Fake +open Fake.Testing let projectRoot () = if FileUtils.pwd().EndsWith("build") then @@ -10,7 +11,7 @@ let projectRoot () = FileUtils.pwd() // Properties -let version = "0.9.9-pre" //TODO: find a way to extract this from somewhere convenient +let version = "5.1.0-alpha1" //TODO: find a way to extract this from somewhere convenient let buildDir = projectRoot() @@ "build" @@ "output" let nugetPath = projectRoot() @@ ".nuget" @@ "NuGet.exe" let nugetOutDir = buildDir @@ "_packages" @@ -63,14 +64,14 @@ Target "Test" (fun _ -> CreateDir testDir ActivateFinalTarget "CloseTestRunner" !! (buildDir @@ "**" @@ "*Test*.dll") - |> NUnit( - fun p -> { p with DisableShadowCopy = true - OutputFile = "TestResults.xml" + |> NUnit3( + fun p -> { p with ShadowCopy = false + //ResultSpecs = "TestResults.xml" StopOnError = false ErrorLevel = DontFailBuild WorkingDir = testDir TimeOut = System.TimeSpan.FromMinutes 10.0 - ExcludeCategory = "LongRunning,LocalCluster" })) + Where = "cat != LongRunning && cat != LocalCluster" })) FinalTarget "CloseTestRunner" (fun _ -> ProcessHelper.killProcess "nunit-agent.exe" @@ -98,7 +99,7 @@ Target "BuildNuget" (fun _ -> let binariesDir = buildDir @@ name // Copy XML doc to binaries dir, works by default on windows but not on Mono. - let xmlDocFile = projectRoot() @@ name @@ "bin" @@ "Release" @@ (sprintf "%s.XML") name + let xmlDocFile = projectRoot() @@ name @@ "bin" @@ "Release" @@ "netstandard2.0" @@ (sprintf "%s.XML") name FileUtils.cp xmlDocFile binariesDir NuGetPack ( diff --git a/build/download-nuget.ps1 b/build/download-nuget.ps1 index 561837d59..4724efe9c 100644 --- a/build/download-nuget.ps1 +++ b/build/download-nuget.ps1 @@ -1,4 +1,4 @@ -$source = "http://nuget.org/nuget.exe" +$source = "https://dist.nuget.org/win-x86-commandline/latest/nuget.exe" $destination = ".\.nuget\nuget.exe" $wc = New-Object System.Net.WebClient From fb86dc8ddc12e375025a4ac14b7489561fb4315a Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:28:02 +0200 Subject: [PATCH 150/153] Add slow and fast variants of Test_Can_Batch_Aggregate_xxx() tests --- FoundationDB.Tests/DatabaseBulkFacts.cs | 137 +++++++++++++++++++++++- 1 file changed, 132 insertions(+), 5 deletions(-) diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index 3fb306ff7..20497c2b2 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -461,26 +461,87 @@ await Fdb.Bulk.ForEachAsync( [Test] public async Task Test_Can_Batch_Aggregate() { - const int N = 50 * 1000; + //note: this test is expected to last more than 5 seconds to trigger a past_version! + + const int N = 100_000; using (var db = await OpenTestPartitionAsync()) { - Log("Bulk inserting {0:N0} items...", N); + Log("Preparing..."); var location = await GetCleanDirectory(db, "Bulk", "Aggregate"); + var rnd = new Random(2403); + var source = Enumerable.Range(1, N).Select((x) => new KeyValuePair(x, rnd.Next(1000))).ToList(); + + Log("Bulk inserting {0:N0} items...", N); + await Fdb.Bulk.WriteAsync( + db, + source.Select((x) => (location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), + this.Cancellation + ); + + Log("Aggregating..."); + + int chunks = 0; + var sw = Stopwatch.StartNew(); + long total = await Fdb.Bulk.AggregateAsync( + db, + source.Select(x => location.Keys.Encode(x.Key)), + () => 0L, + async (xs, ctx, sum) => + { + Interlocked.Increment(ref chunks); + Log($"> Called with batch of {xs.Length:N0} at offset {ctx.Position:N0} of gen {ctx.Generation} with step {ctx.Step} and cooldown {ctx.Cooldown} (genElapsed={ctx.ElapsedGeneration.TotalSeconds:N3} sec, totalElapsed={ctx.ElapsedTotal.TotalSeconds:N3} sec)"); + + var results = await ctx.Transaction.GetValuesAsync(xs); + + for (int i = 0; i < results.Length; i++) + { + sum += results[i].ToInt32(); + } + return sum; + }, + this.Cancellation + ); + sw.Stop(); + + Log($"Done in {sw.Elapsed.TotalSeconds:N3} sec and {chunks} chunks ({N / sw.Elapsed.TotalSeconds:N0} records/sec)"); + + long actual = source.Sum(x => (long)x.Value); + Log("> Computed sum of the {0:N0} random values is {1:N0}", N, total); + Log("> Actual sum of the {0:N0} random values is {1:N0}", N, actual); + Assert.That(total, Is.EqualTo(actual)); + + // cleanup because this test can produce a lot of data + await location.RemoveAsync(db, this.Cancellation); + } + } + + [Test] + public async Task Test_Can_Batch_Aggregate_Slow_Reader() + { + //note: this test is expected to last more than 5 seconds to trigger a past_version! + + const int N = 50 * 1000; + + using (var db = await OpenTestPartitionAsync()) + { + Log("Preparing..."); + var location = await GetCleanDirectory(db, "Bulk", "Aggregate"); var rnd = new Random(2403); var source = Enumerable.Range(1, N).Select((x) => new KeyValuePair(x, rnd.Next(1000))).ToList(); + Log("Bulk inserting {0:N0} items...", N); await Fdb.Bulk.WriteAsync( db, source.Select((x) => (location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), this.Cancellation ); - Log("Reading..."); + Log("Simulating slow reader..."); int chunks = 0; var sw = Stopwatch.StartNew(); @@ -491,7 +552,7 @@ await Fdb.Bulk.WriteAsync( async (xs, ctx, sum) => { Interlocked.Increment(ref chunks); - Log("> Called with batch of " + xs.Length.ToString("N0") + " at offset " + ctx.Position.ToString("N0") + " of gen " + ctx.Generation + " with step " + ctx.Step + " and cooldown " + ctx.Cooldown + " (genElapsed=" + ctx.ElapsedGeneration + ", totalElapsed=" + ctx.ElapsedTotal + ")"); + Log($"> Called with batch of {xs.Length:N0} at offset {ctx.Position:N0} of gen {ctx.Generation} with step {ctx.Step} and cooldown {ctx.Cooldown} (genElapsed={ctx.ElapsedGeneration.TotalSeconds:N1}, totalElapsed={ctx.ElapsedTotal.TotalSeconds:N1}s)"); var throttle = Task.Delay(TimeSpan.FromMilliseconds(10 + (xs.Length / 25) * 5)); // magic numbers to try to last longer than 5 sec var results = await ctx.Transaction.GetValuesAsync(xs); @@ -516,11 +577,75 @@ await Fdb.Bulk.WriteAsync( // cleanup because this test can produce a lot of data await location.RemoveAsync(db, this.Cancellation); + + Assume.That(sw.Elapsed.TotalSeconds, Is.GreaterThan(5), "This test has to run more than 5 seconds to trigger past_version internally!"); } } [Test] public async Task Test_Can_Batch_Aggregate_With_Transformed_Result() + { + const int N = 100_000; + + using (var db = await OpenTestPartitionAsync()) + { + + Log("Preparing..."); + var location = await GetCleanDirectory(db, "Bulk", "Aggregate"); + + var rnd = new Random(2403); + var source = Enumerable.Range(1, N).Select((x) => new KeyValuePair(x, rnd.Next(1000))).ToList(); + + Log("Bulk inserting {0:N0} items...", N); + await Fdb.Bulk.WriteAsync( + db, + source.Select((x) => (location.Keys.Encode(x.Key), Slice.FromInt32(x.Value))), + this.Cancellation + ); + + Log("Aggregating..."); + + int chunks = 0; + var sw = Stopwatch.StartNew(); + double average = await Fdb.Bulk.AggregateAsync( + db, + source.Select(x => location.Keys.Encode(x.Key)), + () => (Total: 0L, Count: 0L), + async (xs, ctx, state) => + { + Interlocked.Increment(ref chunks); + Log($"> Called with batch of {xs.Length:N0} at offset {ctx.Position:N0} of gen {ctx.Generation} with step {ctx.Step} and cooldown {ctx.Cooldown} (genElapsed={ctx.ElapsedGeneration.TotalSeconds:N3} sec, totalElapsed={ctx.ElapsedTotal.TotalSeconds:N3} sec)"); + + var results = await ctx.Transaction.GetValuesAsync(xs); + + long sum = 0L; + foreach (Slice x in results) + { + sum += x.ToInt32(); + } + state.Total += sum; + state.Count += results.Length; + return state; + }, + (state) => (double) state.Total / state.Count, + this.Cancellation + ); + sw.Stop(); + + Log($"Done in {sw.Elapsed.TotalSeconds:N3} sec and {chunks} chunks ({N / sw.Elapsed.TotalSeconds:N0} records/sec)"); + + double actual = (double)source.Sum(x => (long)x.Value) / source.Count; + Log("> Computed average of the {0:N0} random values is {1:N3}", N, average); + Log("> Actual average of the {0:N0} random values is {1:N3}", N, actual); + Assert.That(average, Is.EqualTo(actual).Within(double.Epsilon)); + + // cleanup because this test can produce a lot of data + await location.RemoveAsync(db, this.Cancellation); + } + } + + [Test] + public async Task Test_Can_Batch_Aggregate_With_Transformed_Result_Slow_Reader() { const int N = 50 * 1000; @@ -541,7 +666,7 @@ await Fdb.Bulk.WriteAsync( this.Cancellation ); - Log("Reading..."); + Log("Simulating slow reader..."); int chunks = 0; var sw = Stopwatch.StartNew(); @@ -581,6 +706,8 @@ await Fdb.Bulk.WriteAsync( // cleanup because this test can produce a lot of data await location.RemoveAsync(db, this.Cancellation); + + Assume.That(sw.Elapsed.TotalSeconds, Is.GreaterThan(5), "This test has to run more than 5 seconds to trigger past_version internally!"); } } From bad5c86342c2c5e83505ff5e220bc3bfa9d6b1d1 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:39:58 +0200 Subject: [PATCH 151/153] Update Test_Fdb_GetErrorMessage with new error strings - string format as changed from "lower_case_keyword" to english text --- FoundationDB.Tests/ErrorFacts.cs | 36 +++++++++++++++++++------------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/FoundationDB.Tests/ErrorFacts.cs b/FoundationDB.Tests/ErrorFacts.cs index aa00c1cb2..b63ad0f08 100644 --- a/FoundationDB.Tests/ErrorFacts.cs +++ b/FoundationDB.Tests/ErrorFacts.cs @@ -39,25 +39,33 @@ public class ErrorFacts [Test] public void Test_Fdb_GetErrorMessage() { - Assert.That(Fdb.GetErrorMessage(FdbError.Success), Is.EqualTo("success")); - - Assert.That(Fdb.GetErrorMessage(FdbError.OperationFailed), Is.EqualTo("operation_failed")); - - Assert.That(Fdb.GetErrorMessage(FdbError.TimedOut), Is.EqualTo("timed_out")); - - Assert.That(Fdb.GetErrorMessage(FdbError.PastVersion), Is.EqualTo("past_version")); + Assert.That(Fdb.GetErrorMessage(FdbError.Success), Is.EqualTo("Success")); + Assert.That(Fdb.GetErrorMessage(FdbError.OperationFailed), Is.EqualTo("Operation failed")); + Assert.That(Fdb.GetErrorMessage(FdbError.PastVersion), Is.EqualTo("Transaction is too old to perform reads or be committed")); + Assert.That(Fdb.GetErrorMessage(FdbError.FutureVersion), Is.EqualTo("Request for future version")); + Assert.That(Fdb.GetErrorMessage(FdbError.TimedOut), Is.EqualTo("Operation timed out")); + Assert.That(Fdb.GetErrorMessage(FdbError.NotCommitted), Is.EqualTo("Transaction not committed due to conflict with another transaction")); + Assert.That(Fdb.GetErrorMessage(FdbError.CommitUnknownResult), Is.EqualTo("Transaction may or may not have committed")); + Assert.That(Fdb.GetErrorMessage(FdbError.TransactionCancelled), Is.EqualTo("Operation aborted because the transaction was cancelled")); + Assert.That(Fdb.GetErrorMessage(FdbError.TransactionTimedOut), Is.EqualTo("Operation aborted because the transaction timed out")); + Assert.That(Fdb.GetErrorMessage(FdbError.ClientInvalidOperation), Is.EqualTo("Invalid API call")); + Assert.That(Fdb.GetErrorMessage(FdbError.LargeAllocFailed), Is.EqualTo("Large block allocation failed")); } [Test] public void Test_Fdb_MapToException() { - Assert.That(Fdb.MapToException(FdbError.Success), Is.Null); - - Assert.That(Fdb.MapToException(FdbError.OperationFailed), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.OperationFailed)); - - Assert.That(Fdb.MapToException(FdbError.TimedOut), Is.InstanceOf()); - - Assert.That(Fdb.MapToException(FdbError.LargeAllocFailed), Is.InstanceOf()); + Assert.That(Fdb.MapToException(FdbError.Success), Is.Null, "Success"); + Assert.That(Fdb.MapToException(FdbError.OperationFailed), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.OperationFailed), "OperationFailed"); + Assert.That(Fdb.MapToException(FdbError.PastVersion), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.PastVersion), "PastVersion"); + Assert.That(Fdb.MapToException(FdbError.FutureVersion), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.FutureVersion), "FutureVersion"); + Assert.That(Fdb.MapToException(FdbError.TimedOut), Is.InstanceOf(), "TimedOut"); + Assert.That(Fdb.MapToException(FdbError.NotCommitted), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.NotCommitted), "NotCommitted"); + Assert.That(Fdb.MapToException(FdbError.CommitUnknownResult), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.CommitUnknownResult), "CommitUnknownResult"); + Assert.That(Fdb.MapToException(FdbError.TransactionCancelled), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.TransactionCancelled), "TrasactionCancelled"); //REVIEW => OperationCancelledException? + Assert.That(Fdb.MapToException(FdbError.TransactionTimedOut), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.TransactionTimedOut), "TransactionTimedOut"); //REVIEW => TimeoutException ? + Assert.That(Fdb.MapToException(FdbError.ClientInvalidOperation), Is.InstanceOf().And.Property("Code").EqualTo(FdbError.ClientInvalidOperation), "ClientInvalidOperation"); //REVIEW => InvalidOperationException? + Assert.That(Fdb.MapToException(FdbError.LargeAllocFailed), Is.InstanceOf(), "LargeAllocFailed"); } } From b2dc4e90d0ba40f44402c66b8ed4bcae0968910f Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:42:36 +0200 Subject: [PATCH 152/153] Add support for "ssd-2" engine mode - GetStorageEngineModeAsync(..) now recognize "ssd-2" storage engine - note: we still return "ssd" for "ssd-1", not sure what it would break? --- FoundationDB.Client/Fdb.System.cs | 5 +++-- FoundationDB.Tests/DatabaseFacts.cs | 6 +++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/FoundationDB.Client/Fdb.System.cs b/FoundationDB.Client/Fdb.System.cs index fd66c65be..eba11dbf5 100644 --- a/FoundationDB.Client/Fdb.System.cs +++ b/FoundationDB.Client/Fdb.System.cs @@ -207,7 +207,7 @@ public static Slice WorkersKey([NotNull] string id, [NotNull] string name) [ItemNotNull] public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabase db, CancellationToken ct) { - // The '\xFF/conf/storage_engine' keys has value "0" (ASCII) for ssd engine, and "1" (ASCII) for memory engine + // The '\xFF/conf/storage_engine' keys has value "0" for ssd-1 engine, "1" for memory engine and "2" for ssd-2 engine var value = await GetConfigParameterAsync(db, "storage_engine", ct).ConfigureAwait(false); @@ -215,8 +215,9 @@ public static async Task GetStorageEngineModeAsync([NotNull] IFdbDatabas switch(value.ToUnicode()) { - case "0": return "ssd"; + case "0": return "ssd"; // "ssd-1" case "1": return "memory"; + case "2": return "ssd-2"; default: { // welcome to the future! diff --git a/FoundationDB.Tests/DatabaseFacts.cs b/FoundationDB.Tests/DatabaseFacts.cs index 2dc0b7cb2..1aba34e49 100644 --- a/FoundationDB.Tests/DatabaseFacts.cs +++ b/FoundationDB.Tests/DatabaseFacts.cs @@ -233,7 +233,7 @@ public async Task Test_Can_Get_Storage_Engine() string mode = await Fdb.System.GetStorageEngineModeAsync(db, this.Cancellation); Log("Storage engine: {0}", mode); Assert.That(mode, Is.Not.Null); - Assert.That(mode, Is.EqualTo("ssd").Or.EqualTo("memory")); + Assert.That(mode, Is.EqualTo("ssd").Or.EqualTo("memory").Or.EqualTo("ssd-2")); // in order to verify the value, we need to check ourselves by reading from the cluster config Slice actual; @@ -246,6 +246,10 @@ public async Task Test_Can_Get_Storage_Engine() { // ssd = '0' Assert.That(actual, Is.EqualTo(Slice.FromStringAscii("0"))); } + else if (mode == "ssd-2") + { // ssd-2 = '2' + Assert.That(actual, Is.EqualTo(Slice.FromStringAscii("2"))); + } else { // memory = '1' Assert.That(actual, Is.EqualTo(Slice.FromStringAscii("1"))); From 23bdb08487f1c1eaa077a36712dff2a8e3c2cf20 Mon Sep 17 00:00:00 2001 From: Christophe Chevalier Date: Sat, 28 Apr 2018 13:45:33 +0200 Subject: [PATCH 153/153] Fix a couple of tests --- FoundationDB.Tests/DatabaseBulkFacts.cs | 8 ++------ FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs | 4 ++-- FoundationDB.Tests/RangeQueryFacts.cs | 2 +- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/FoundationDB.Tests/DatabaseBulkFacts.cs b/FoundationDB.Tests/DatabaseBulkFacts.cs index 20497c2b2..e796b6f3b 100644 --- a/FoundationDB.Tests/DatabaseBulkFacts.cs +++ b/FoundationDB.Tests/DatabaseBulkFacts.cs @@ -97,12 +97,8 @@ public async Task Test_Can_Bulk_Insert_Raw_Data() // read everything back... Log("Reading everything back..."); - - var stored = await db.ReadAsync((tr) => - { - return tr.GetRangeStartsWith(location).ToArrayAsync(); - }, this.Cancellation); - + var stored = await db.ReadAsync((tr) => tr.GetRangeStartsWith(location).Select(x => (x.Key, x.Value)).ToArrayAsync(), this.Cancellation); + Log($"> found {stored.Length:N0} results"); Assert.That(stored.Length, Is.EqualTo(N)); Assert.That(stored, Is.EqualTo(data)); } diff --git a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs index bfcffead6..7106ad15f 100644 --- a/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs +++ b/FoundationDB.Tests/Linq/FdbQueryExpressionFacts.cs @@ -101,8 +101,8 @@ public void Test_FdbQueryRangeExpression() Log(expr); Assert.That(expr, Is.Not.Null); - Assert.That(expr.Range.Begin.Key.ToString(), Is.EqualTo("<02>Foo<00>")); - Assert.That(expr.Range.End.Key.ToString(), Is.EqualTo("<02>Foo<01>")); + Assert.That(expr.Range.Begin.Key.ToString(), Is.EqualTo("<02>Foo<00><00>")); + Assert.That(expr.Range.End.Key.ToString(), Is.EqualTo("<02>Foo<00>")); Assert.That(expr.Type, Is.EqualTo(typeof(IAsyncEnumerable>))); Assert.That(expr.ElementType, Is.EqualTo(typeof(KeyValuePair))); diff --git a/FoundationDB.Tests/RangeQueryFacts.cs b/FoundationDB.Tests/RangeQueryFacts.cs index 2d84d2f6a..13a4525e6 100644 --- a/FoundationDB.Tests/RangeQueryFacts.cs +++ b/FoundationDB.Tests/RangeQueryFacts.cs @@ -361,7 +361,7 @@ public async Task Test_Can_Skip() var location = await GetCleanDirectory(db, "Queries", "Range"); // import test data - var data = Enumerable.Range(0, 100).Select(x => (location.Keys.Encode(x), Slice.FromFixed32(x))); + var data = Enumerable.Range(0, 100).Select(x => new KeyValuePair(location.Keys.Encode(x), Slice.FromFixed32(x))); await Fdb.Bulk.WriteAsync(db, data, this.Cancellation); // from the start