diff --git a/cs/playground/AsyncStress/FasterWrapper.cs b/cs/playground/AsyncStress/FasterWrapper.cs index ad7f651e0..f311419cb 100644 --- a/cs/playground/AsyncStress/FasterWrapper.cs +++ b/cs/playground/AsyncStress/FasterWrapper.cs @@ -56,7 +56,7 @@ public FasterWrapper(bool isRefType, bool useLargeLog, bool useOsReadBuffering = async ValueTask CompleteAsync(FasterKV.UpsertAsyncResult result) { var numPending = 0; - for (; result.Status.Pending; ++numPending) + for (; result.Status.IsPending; ++numPending) result = await result.CompleteAsync().ConfigureAwait(false); return numPending; } @@ -64,7 +64,7 @@ async ValueTask CompleteAsync(FasterKV.UpsertAsyncResult CompleteAsync(FasterKV.RmwAsyncResult result) { var numPending = 0; - for (; result.Status.Pending; ++numPending) + for (; result.Status.IsPending; ++numPending) result = await result.CompleteAsync().ConfigureAwait(false); return numPending; } @@ -82,7 +82,7 @@ public void Upsert(Key key, Value value) if (!_sessionPool.TryGet(out var session)) session = _sessionPool.GetAsync().GetAwaiter().GetResult(); var status = session.Upsert(key, value); - Assert.False(status.Pending); + Assert.False(status.IsPending); _sessionPool.Return(session); } @@ -109,7 +109,7 @@ public void RMW(Key key, Value value) if (!_sessionPool.TryGet(out var session)) session = _sessionPool.GetAsync().GetAwaiter().GetResult(); var status = session.RMW(key, value); - Assert.False(status.Pending); + Assert.False(status.IsPending); _sessionPool.Return(session); } @@ -137,7 +137,7 @@ public async ValueTask RMWChunkAsync((Key, Value)[] chunk, int offset, int count if (!_sessionPool.TryGet(out var session)) session = _sessionPool.GetAsync().GetAwaiter().GetResult(); var result = session.Read(key); - if (result.status.Pending) + if (result.status.IsPending) { session.CompletePendingWithOutputs(out var completedOutputs, wait: true); int count = 0; diff --git a/cs/playground/AsyncStress/SerializedFasterWrapper.cs b/cs/playground/AsyncStress/SerializedFasterWrapper.cs index 93ef81d90..426f8e910 100644 --- a/cs/playground/AsyncStress/SerializedFasterWrapper.cs +++ b/cs/playground/AsyncStress/SerializedFasterWrapper.cs @@ -121,7 +121,7 @@ public void Update(TUpdater updater, Key key, Value valu } } - Assert.False(status.Pending); + Assert.False(status.IsPending); _sessionPool.Return(session); } @@ -198,7 +198,7 @@ public async ValueTask UpdateChunkAsync(TUpdater updater } } - if (result.Item1.Pending) + if (result.Item1.IsPending) { session.CompletePendingWithOutputs(out var completedOutputs, wait: true); int count = 0; diff --git a/cs/playground/AsyncStress/SerializedUpdaters.cs b/cs/playground/AsyncStress/SerializedUpdaters.cs index e2a051e45..107d775f3 100644 --- a/cs/playground/AsyncStress/SerializedUpdaters.cs +++ b/cs/playground/AsyncStress/SerializedUpdaters.cs @@ -26,7 +26,7 @@ public ValueTask.UpsertAsyncResult CompleteAsync(FasterKV.UpsertAsyncResult result) { var numPending = 0; - for (; result.Status.Pending; ++numPending) + for (; result.Status.IsPending; ++numPending) result = await result.CompleteAsync().ConfigureAwait(false); return numPending; } @@ -43,7 +43,7 @@ public ValueTask.RmwAsyncResult CompleteAsync(FasterKV.RmwAsyncResult result) { var numPending = 0; - for (; result.Status.Pending; ++numPending) + for (; result.Status.IsPending; ++numPending) result = await result.CompleteAsync().ConfigureAwait(false); return numPending; } diff --git a/cs/playground/CacheStoreConcurrent/Program.cs b/cs/playground/CacheStoreConcurrent/Program.cs index 6a3931bed..a1687d335 100644 --- a/cs/playground/CacheStoreConcurrent/Program.cs +++ b/cs/playground/CacheStoreConcurrent/Program.cs @@ -175,7 +175,7 @@ private static void RandomReadWorkload(int threadid) var key = new CacheKey(k); var status = hts.Read(ref key, ref output); - if (status.Pending) + if (status.IsPending) { statusPending++; } diff --git a/cs/playground/ClassRecoveryDurability/Program.cs b/cs/playground/ClassRecoveryDurability/Program.cs index 75b766c7b..f594b3702 100644 --- a/cs/playground/ClassRecoveryDurability/Program.cs +++ b/cs/playground/ClassRecoveryDurability/Program.cs @@ -90,7 +90,7 @@ static void Filldb(int stopAfterIteration = 5) // Console.WriteLine("add=" + i); - if (!addStatus.InPlaceUpdatedRecord) + if (!addStatus.Record.InPlaceUpdated) throw new Exception(); } @@ -109,7 +109,7 @@ static void Filldb(int stopAfterIteration = 5) var deleteStatus = session.Delete(ref deteletKey, context2, 1); // Console.WriteLine("delete=" + i); - if (!deleteStatus.InPlaceUpdatedRecord) + if (!deleteStatus.Record.InPlaceUpdated) throw new Exception(); } } @@ -195,7 +195,7 @@ static void TestData(Storedb store, int count) var deleteStatus = session.Read(ref readKey, ref input1, ref output1, context, 1); //Console.WriteLine("test delete=" + i); - if (deleteStatus.Pending) + if (deleteStatus.IsPending) { session.CompletePending(true); context.FinalizeRead(ref deleteStatus, ref output1); @@ -215,7 +215,7 @@ static void TestData(Storedb store, int count) var addStatus = session.Read(ref readKey, ref input1, ref output1, context, 1); //Console.WriteLine("test add=" + i); - if (addStatus.Pending) + if (addStatus.IsPending) { session.CompletePending(true); context.FinalizeRead(ref addStatus, ref output1); diff --git a/cs/playground/SumStore/ConcurrencyTest.cs b/cs/playground/SumStore/ConcurrencyTest.cs index d47da5e15..d6966f6d9 100644 --- a/cs/playground/SumStore/ConcurrencyTest.cs +++ b/cs/playground/SumStore/ConcurrencyTest.cs @@ -156,7 +156,7 @@ public void Test() Input input = default; Output output = default; var status = session.Read(ref inputArray[i].adId, ref input, ref output, Empty.Default, i); - if (status.Pending) + if (status.IsPending) throw new NotImplementedException(); inputArray[i].numClicks = output.value; diff --git a/cs/remote/src/FASTER.server/BinaryServerSession.cs b/cs/remote/src/FASTER.server/BinaryServerSession.cs index 14b5e9219..4b8ac93d0 100644 --- a/cs/remote/src/FASTER.server/BinaryServerSession.cs +++ b/cs/remote/src/FASTER.server/BinaryServerSession.cs @@ -167,7 +167,7 @@ private unsafe void ProcessBatch(byte* buf, int offset) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); Write(ref status, ref dcurr, (int)(dend - dcurr)); - if (status.Pending) + if (status.IsPending) Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr)); else if (status.Found) serializer.SkipOutput(ref dcurr); @@ -186,7 +186,7 @@ private unsafe void ProcessBatch(byte* buf, int offset) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); Write(ref status, ref dcurr, (int)(dend - dcurr)); - if (status.Pending) + if (status.IsPending) Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr)); else if (status.IsCompletedSuccessfully) serializer.SkipOutput(ref dcurr); @@ -273,7 +273,7 @@ private unsafe void Publish(ref byte* keyPtr, int keyLength, ref byte* valPtr, r if (valPtr == null) status = session.Read(ref key, ref serializer.ReadInputByRef(ref inputPtr), ref serializer.AsRefOutput(outputDcurr, (int)(dend - dcurr)), ctx, 0); - if (!status.Pending) + if (!status.IsPending) { // Write six bytes (message | status | sid) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); diff --git a/cs/remote/src/FASTER.server/WebsocketServerSession.cs b/cs/remote/src/FASTER.server/WebsocketServerSession.cs index e0c0ceb04..db71f8929 100644 --- a/cs/remote/src/FASTER.server/WebsocketServerSession.cs +++ b/cs/remote/src/FASTER.server/WebsocketServerSession.cs @@ -384,7 +384,7 @@ private unsafe bool ProcessBatch(byte* buf, int length, int offset) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); Write(ref status, ref dcurr, (int)(dend - dcurr)); - if (status.Pending) + if (status.IsPending) Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr)); else if (status.Found) serializer.SkipOutput(ref dcurr); @@ -403,7 +403,7 @@ private unsafe bool ProcessBatch(byte* buf, int length, int offset) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); Write(ref status, ref dcurr, (int)(dend - dcurr)); - if (status.Pending) + if (status.IsPending) Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr)); if (subscribeKVBroker != null) @@ -589,7 +589,7 @@ private unsafe void Publish(ref byte* keyPtr, int keyLength, ref byte* valPtr, r if (valPtr == null) status = session.Read(ref key, ref serializer.ReadInputByRef(ref inputPtr), ref serializer.AsRefOutput(outputDcurr, (int)(dend - dcurr)), ctx, 0); - if (!status.Pending) + if (!status.IsPending) { // Write six bytes (message | status | sid) hrw.Write(message, ref dcurr, (int)(dend - dcurr)); diff --git a/cs/samples/AzureBackedStore/Program.cs b/cs/samples/AzureBackedStore/Program.cs index 2b7370891..b25a04a7e 100644 --- a/cs/samples/AzureBackedStore/Program.cs +++ b/cs/samples/AzureBackedStore/Program.cs @@ -52,7 +52,7 @@ static void Main() var status = s.Read(ref key, ref output, context); - if (!status.Pending) + if (!status.IsPending) { if (status.Found && output == context) Console.WriteLine("Success!"); @@ -94,7 +94,7 @@ static void Main() var status = s.Read(ref key, ref output, context); - if (!status.Pending) + if (!status.IsPending) { if (status.Found && output == context) Console.WriteLine("Success!"); diff --git a/cs/samples/CacheStore/Program.cs b/cs/samples/CacheStore/Program.cs index 2e6398bf7..82a6af7a9 100644 --- a/cs/samples/CacheStore/Program.cs +++ b/cs/samples/CacheStore/Program.cs @@ -144,7 +144,7 @@ private static void RandomReadWorkload(FasterKV store, int var key = new CacheKey(k); var status = s.Read(ref key, ref output); - if (status.Pending) + if (status.IsPending) { statusPending++; if (statusPending % 100 == 0) @@ -186,7 +186,7 @@ private static void InteractiveReadWorkload(FasterKV store context.ticks = Stopwatch.GetTimestamp(); var status = s.Read(ref key, ref output, context); - if (status.Pending) + if (status.IsPending) { s.CompletePending(true); } diff --git a/cs/samples/HelloWorld/Program.cs b/cs/samples/HelloWorld/Program.cs index d82fd4e0e..80575027d 100644 --- a/cs/samples/HelloWorld/Program.cs +++ b/cs/samples/HelloWorld/Program.cs @@ -147,7 +147,7 @@ static void DiskSample() // Reads from disk will return PENDING status, result available via either asynchronous IFunctions callback // or on this thread via CompletePendingWithOutputs, shown below status = session.Read(ref key, ref output); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var iter, true); while (iter.Next()) diff --git a/cs/samples/ReadAddress/VersionedReadApp.cs b/cs/samples/ReadAddress/VersionedReadApp.cs index 9799d388b..4298c73cd 100644 --- a/cs/samples/ReadAddress/VersionedReadApp.cs +++ b/cs/samples/ReadAddress/VersionedReadApp.cs @@ -157,7 +157,7 @@ private static void ScanStore(FasterKV store, int keyValue) var status = session.Read(ref key, ref input, ref output, ref recordMetadata, ReadFlags.SkipCopyReads, serialNo: maxLap + 1); // This will wait for each retrieved record; not recommended for performance-critical code or when retrieving multiple records unless necessary. - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var completedOutputs, wait: true); using (completedOutputs) diff --git a/cs/samples/StoreAsyncApi/Program.cs b/cs/samples/StoreAsyncApi/Program.cs index 5068f45df..00c816c56 100644 --- a/cs/samples/StoreAsyncApi/Program.cs +++ b/cs/samples/StoreAsyncApi/Program.cs @@ -87,7 +87,7 @@ static async Task AsyncOperator(int id) if (asyncUpsert) { var r = await session.UpsertAsync(ref key, ref value, context, seqNo++); - while (r.Status.Pending) + while (r.Status.IsPending) r = await r.CompleteAsync(); } else @@ -130,7 +130,7 @@ static async Task AsyncOperator(int id) for (int i = 0; i < batchSize; i++) { var r = await taskBatch[i]; - while (r.Status.Pending) + while (r.Status.IsPending) r = await r.CompleteAsync(); } } diff --git a/cs/samples/StoreCustomTypes/Program.cs b/cs/samples/StoreCustomTypes/Program.cs index dcd6802d8..61a6696fa 100644 --- a/cs/samples/StoreCustomTypes/Program.cs +++ b/cs/samples/StoreCustomTypes/Program.cs @@ -78,7 +78,7 @@ static void Main() status = s.Read(ref key, ref input, ref g2, context, 0); // We will receive the result via ReadCompletionCallback in Functions - if (!status.Pending) + if (!status.IsPending) Console.WriteLine("Error!"); // End session when done diff --git a/cs/samples/StoreDiskReadBenchmark/Program.cs b/cs/samples/StoreDiskReadBenchmark/Program.cs index 6f9684c3e..c599c0b40 100644 --- a/cs/samples/StoreDiskReadBenchmark/Program.cs +++ b/cs/samples/StoreDiskReadBenchmark/Program.cs @@ -155,7 +155,7 @@ static async Task AsyncReadOperator(int id) var result = session.Read(ref key, ref input, ref output, Empty.Default, 0); if (readBatching) { - if (!result.Pending) + if (!result.IsPending) { if (output.value.vfield1 != key.key) { @@ -166,7 +166,7 @@ static async Task AsyncReadOperator(int id) } else { - if (result.Pending) + if (result.IsPending) { session.CompletePending(true); } diff --git a/cs/samples/StoreVarLenTypes/AsciiSumSample.cs b/cs/samples/StoreVarLenTypes/AsciiSumSample.cs index bfe46a8af..1ddd554c8 100644 --- a/cs/samples/StoreVarLenTypes/AsciiSumSample.cs +++ b/cs/samples/StoreVarLenTypes/AsciiSumSample.cs @@ -59,7 +59,7 @@ public static void Run() store.Log.FlushAndEvict(true); // Flush and evict all records to disk var _status = s.RMW(_key, _input); // CopyUpdater to 270 (due to immutable source value on disk) - if (_status.Pending) + if (_status.IsPending) { Console.WriteLine("Error!"); return; diff --git a/cs/samples/StoreVarLenTypes/MemoryByteSample.cs b/cs/samples/StoreVarLenTypes/MemoryByteSample.cs index 38f4b8b6a..07d4917c6 100644 --- a/cs/samples/StoreVarLenTypes/MemoryByteSample.cs +++ b/cs/samples/StoreVarLenTypes/MemoryByteSample.cs @@ -61,7 +61,7 @@ public static void Run() var expectedValue = valueMem.Slice(0, valLen); expectedValue.Span.Fill((byte)valLen); - if (status.Pending) + if (status.IsPending) s.CompletePending(true); else { diff --git a/cs/samples/StoreVarLenTypes/MemoryIntSample.cs b/cs/samples/StoreVarLenTypes/MemoryIntSample.cs index cbe40ac62..b21d5210f 100644 --- a/cs/samples/StoreVarLenTypes/MemoryIntSample.cs +++ b/cs/samples/StoreVarLenTypes/MemoryIntSample.cs @@ -61,7 +61,7 @@ public static void Run() var expectedValue = valueMem.Slice(0, valLen); expectedValue.Span.Fill(valLen); - if (status.Pending) + if (status.IsPending) s.CompletePending(true); else { diff --git a/cs/samples/StoreVarLenTypes/SpanByteSample.cs b/cs/samples/StoreVarLenTypes/SpanByteSample.cs index 623cd6b96..47b740101 100644 --- a/cs/samples/StoreVarLenTypes/SpanByteSample.cs +++ b/cs/samples/StoreVarLenTypes/SpanByteSample.cs @@ -91,7 +91,7 @@ public static void Run() var expectedValue = valueMem.Slice(0, valLen); expectedValue.Fill((byte)valLen); - if (status.Pending) + if (status.IsPending) { s.CompletePending(true); } diff --git a/cs/src/core/Async/DeleteAsync.cs b/cs/src/core/Async/DeleteAsync.cs index 84078cfac..a35ac9700 100644 --- a/cs/src/core/Async/DeleteAsync.cs +++ b/cs/src/core/Async/DeleteAsync.cs @@ -69,13 +69,13 @@ internal DeleteAsyncResult(FasterKV fasterKV, IFasterSessionComplete the Delete operation, issuing additional allocation asynchronously if needed. It is usually preferable to use Complete() instead of this. /// ValueTask for Delete result. User needs to await again if result status is Status.PENDING. public ValueTask> CompleteAsync(CancellationToken token = default) - => this.Status.Pending + => this.Status.IsPending ? updateAsyncInternal.CompleteAsync(token) : new ValueTask>(new DeleteAsyncResult(this.Status)); /// Complete the Delete operation, issuing additional I/O synchronously if needed. /// Status of Delete operation - public Status Complete() => this.Status.Pending ? updateAsyncInternal.Complete().Status : this.Status; + public Status Complete() => this.Status.IsPending ? updateAsyncInternal.Complete().Status : this.Status; } [MethodImpl(MethodImplOptions.AggressiveInlining)] diff --git a/cs/src/core/Async/RMWAsync.cs b/cs/src/core/Async/RMWAsync.cs index 5bf408429..294427886 100644 --- a/cs/src/core/Async/RMWAsync.cs +++ b/cs/src/core/Async/RMWAsync.cs @@ -32,7 +32,7 @@ public Status DoFastOperation(FasterKV fasterKV, ref PendingContext< pendingContext.serialNum, asyncOp, out flushEvent, out newDiskRequest); output = pendingContext.output; - if (status.Pending && !newDiskRequest.IsDefault()) + if (status.IsPending && !newDiskRequest.IsDefault()) { flushEvent = default; this.diskRequest = newDiskRequest; @@ -56,7 +56,7 @@ public bool CompletePendingIO(IFasterSession var status = completedOutputs.Next() ? completedOutputs.Current.Status : new(StatusCode.Error); completedOutputs.Dispose(); this.diskRequest = default; - return !status.Pending; + return !status.IsPending; } /// @@ -109,7 +109,7 @@ internal RmwAsyncResult(FasterKV fasterKV, IFasterSessionComplete the RMW operation, issuing additional (rare) I/O asynchronously if needed. It is usually preferable to use Complete() instead of this. /// ValueTask for RMW result. User needs to await again if result status is pending. public ValueTask> CompleteAsync(CancellationToken token = default) - => this.Status.Pending + => this.Status.IsPending ? updateAsyncInternal.CompleteAsync(token) : new ValueTask>(new RmwAsyncResult(this.Status, this.Output, this.RecordMetadata)); @@ -122,7 +122,7 @@ public ValueTask> CompleteAsync(Cancella /// Status of RMW operation public (Status status, TOutput output) Complete(out RecordMetadata recordMetadata) { - if (!this.Status.Pending) + if (!this.Status.IsPending) { recordMetadata = this.RecordMetadata; return (this.Status, this.Output); @@ -154,7 +154,7 @@ internal ValueTask> RmwAsync>(new RmwAsyncResult(status, output, new RecordMetadata(pcontext.recordInfo, pcontext.logicalAddress))); } finally diff --git a/cs/src/core/Async/ReadAsync.cs b/cs/src/core/Async/ReadAsync.cs index 11f4fab10..e91e6d4eb 100644 --- a/cs/src/core/Async/ReadAsync.cs +++ b/cs/src/core/Async/ReadAsync.cs @@ -53,7 +53,7 @@ internal ReadAsyncInternal(FasterKV fasterKV, IFasterSessionThe read result, or throws an exception if error encountered. public (Status status, Output output) Complete() { - if (!status.Pending) + if (!status.IsPending) return (status, output); return readAsyncInternal.Complete(); } @@ -137,7 +137,7 @@ internal ReadAsyncResult( /// The read result and the previous address in the Read key's hash chain, or throws an exception if error encountered. public (Status status, Output output) Complete(out RecordMetadata recordMetadata) { - if (!status.Pending) + if (!status.IsPending) { recordMetadata = this.recordMetadata; return (status, output); @@ -170,7 +170,7 @@ internal ValueTask> ReadAsync>(new ReadAsyncResult(new(internalStatus), output, new RecordMetadata(pcontext.recordInfo, pcontext.logicalAddress))); status = HandleOperationStatus(currentCtx, currentCtx, ref pcontext, fasterSession, internalStatus, true, out diskRequest); - if (!status.Pending) + if (!status.IsPending) return new ValueTask>(new ReadAsyncResult(status, output, new RecordMetadata(pcontext.recordInfo, pcontext.logicalAddress))); } finally diff --git a/cs/src/core/Async/UpdateAsync.cs b/cs/src/core/Async/UpdateAsync.cs index d0579fbb5..9e05fce14 100644 --- a/cs/src/core/Async/UpdateAsync.cs +++ b/cs/src/core/Async/UpdateAsync.cs @@ -157,7 +157,7 @@ private bool TryCompleteSync(bool asyncOp, out CompletionEvent flushEvent, out T { Status status = _asyncOperation.DoFastOperation(_fasterKV, ref _pendingContext, _fasterSession, _currentCtx, asyncOp, out flushEvent, out Output output); - if (!status.Pending) + if (!status.IsPending) { _pendingContext.Dispose(); asyncResult = _asyncOperation.CreateResult(status, output, new RecordMetadata(_pendingContext.recordInfo, _pendingContext.logicalAddress)); diff --git a/cs/src/core/Async/UpsertAsync.cs b/cs/src/core/Async/UpsertAsync.cs index de29cac3b..cd82cd08e 100644 --- a/cs/src/core/Async/UpsertAsync.cs +++ b/cs/src/core/Async/UpsertAsync.cs @@ -80,19 +80,19 @@ internal UpsertAsyncResult(FasterKV fasterKV, IFasterSessionComplete the Upsert operation, issuing additional allocation asynchronously if needed. It is usually preferable to use Complete() instead of this. /// ValueTask for Upsert result. User needs to await again if result status is Status.PENDING. public ValueTask> CompleteAsync(CancellationToken token = default) - => this.Status.Pending + => this.Status.IsPending ? updateAsyncInternal.CompleteAsync(token) : new ValueTask>(new UpsertAsyncResult(this.Status, this.Output, this.RecordMetadata)); /// Complete the Upsert operation, issuing additional I/O synchronously if needed. /// Status of Upsert operation - public Status Complete() => this.Status.Pending ? updateAsyncInternal.Complete().Status : this.Status; + public Status Complete() => this.Status.IsPending ? updateAsyncInternal.Complete().Status : this.Status; /// Complete the Upsert operation, issuing additional I/O synchronously if needed. /// Status and Output of Upsert operation public (Status status, TOutput output) Complete(out RecordMetadata recordMetadata) { - if (!this.Status.Pending) + if (!this.Status.IsPending) { recordMetadata = this.RecordMetadata; return (this.Status, this.Output); diff --git a/cs/src/core/ClientSession/IFasterContext.cs b/cs/src/core/ClientSession/IFasterContext.cs index 4228d84e7..6cef9f9d0 100644 --- a/cs/src/core/ClientSession/IFasterContext.cs +++ b/cs/src/core/ClientSession/IFasterContext.cs @@ -284,7 +284,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.UpsertAsyncResult> UpsertAsync(ref Key key, ref Value desiredValue, Context userContext = default, long serialNo = 0, CancellationToken token = default); @@ -303,7 +303,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.UpsertAsyncResult> UpsertAsync(ref Key key, ref Input input, ref Value desiredValue, Context userContext = default, long serialNo = 0, CancellationToken token = default); @@ -321,7 +321,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.UpsertAsyncResult> UpsertAsync(Key key, Value desiredValue, Context userContext = default, long serialNo = 0, CancellationToken token = default); @@ -340,7 +340,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.UpsertAsyncResult> UpsertAsync(Key key, Input input, Value desiredValue, Context userContext = default, long serialNo = 0, CancellationToken token = default); @@ -411,7 +411,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.RmwAsyncResult> RMWAsync(ref Key key, ref Input input, Context context = default, long serialNo = 0, CancellationToken token = default); @@ -428,7 +428,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.RmwAsyncResult> RMWAsync(Key key, Input input, Context context = default, long serialNo = 0, CancellationToken token = default); @@ -462,7 +462,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.DeleteAsyncResult> DeleteAsync(ref Key key, Context userContext = default, long serialNo = 0, CancellationToken token = default); @@ -478,7 +478,7 @@ ValueTask.ReadAsyncResult> ReadAtAd /// The caller must await the return value to obtain the result, then call one of /// /// result. - /// result = await result. while result.Status is + /// result = await result. while result.Status is /// /// to complete the Upsert operation. Failure to complete the operation will result in leaked allocations. ValueTask.DeleteAsyncResult> DeleteAsync(Key key, Context userContext = default, long serialNo = 0, CancellationToken token = default); diff --git a/cs/src/core/Compaction/FASTERCompaction.cs b/cs/src/core/Compaction/FASTERCompaction.cs index 831b47d04..79cff6a69 100644 --- a/cs/src/core/Compaction/FASTERCompaction.cs +++ b/cs/src/core/Compaction/FASTERCompaction.cs @@ -61,7 +61,7 @@ private long CompactLookup + /// Status result of operation on FASTER + /// + [StructLayout(LayoutKind.Explicit, Size = 1)] + public struct RecordStatus + { + [FieldOffset(0)] + internal readonly StatusCode statusCode; + + /// + /// Whether a new record for a previously non-existent key was appended to the log. + /// Indicates that an existing record was updated in place. + /// + public bool Created => (statusCode & StatusCode.AdvancedMask) == StatusCode.CreatedRecord; + + /// + /// Whether existing record was updated in place. + /// + public bool InPlaceUpdated => (statusCode & StatusCode.AdvancedMask) == StatusCode.InPlaceUpdatedRecord; + + /// + /// Whether an existing record key was copied, updated, and appended to the log. + /// + public bool CopyUpdated => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopyUpdatedRecord; + + /// + /// Whether an existing record key was copied and appended to the log. + /// + public bool Copied => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopiedRecord; + + /// + /// Whether an existing record key was copied, updated, and added to the readcache. + /// + public bool CopiedToReadCache => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopiedRecordToReadCache; + } + /// /// Status result of operation on FASTER /// @@ -14,16 +50,22 @@ namespace FASTER.core public struct Status { [FieldOffset(0)] - private readonly StatusCode statusCode; + internal readonly StatusCode statusCode; + + /// + /// Status specific to the record + /// + [FieldOffset(0)] + public RecordStatus Record; /// /// Create status from given status code /// /// - internal Status(StatusCode statusCode) => this.statusCode = statusCode; + internal Status(StatusCode statusCode) : this() => this.statusCode = statusCode; [MethodImpl(MethodImplOptions.AggressiveInlining)] - internal Status(OperationStatus operationStatus) + internal Status(OperationStatus operationStatus) : this() { var basicOperationStatus = OperationStatusUtils.BasicOpCode(operationStatus); Debug.Assert(basicOperationStatus == OperationStatus.SUCCESS || basicOperationStatus == OperationStatus.NOTFOUND); @@ -36,14 +78,14 @@ internal Status(OperationStatus operationStatus) public static Status CreateFound() => new(StatusCode.OK); /// - /// Create a Status value. Use the Is* properties to query. + /// Create a Status value. Use the Is* properties to query. /// public static Status CreatePending() => new(StatusCode.Pending); /// /// Whether a Read or RMW found the key /// - public bool Found => (statusCode & StatusCode.BasicMask) == StatusCode.OK; + public bool Found => (this.Record.statusCode & StatusCode.BasicMask) == StatusCode.OK; /// /// Whether a Read or RMW did not find the key @@ -53,12 +95,12 @@ internal Status(OperationStatus operationStatus) /// /// Whether the operation went pending /// - public bool Pending => statusCode == StatusCode.Pending; + public bool IsPending => statusCode == StatusCode.Pending; /// /// Whether the operation went pending /// - public bool IsCompleted => !Pending; + public bool IsCompleted => !IsPending; /// /// Whether the operation is in an error state @@ -77,34 +119,6 @@ public bool IsCompletedSuccessfully } } - #region Advanced status - /// - /// Whether a new record for a previously non-existent key was appended to the log. - /// Indicates that an existing record was updated in place. - /// - public bool CreatedRecord => (statusCode & StatusCode.AdvancedMask) == StatusCode.CreatedRecord; - - /// - /// Whether existing record was updated in place. - /// - public bool InPlaceUpdatedRecord => (statusCode & StatusCode.AdvancedMask) == StatusCode.InPlaceUpdatedRecord; - - /// - /// Whether an existing record key was copied, updated, and appended to the log. - /// - public bool CopyUpdatedRecord => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopyUpdatedRecord; - - /// - /// Whether an existing record key was copied and appended to the log. - /// - public bool CopiedRecord => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopiedRecord; - - /// - /// Whether an existing record key was copied, updated, and added to the readcache. - /// - public bool CopiedRecordToReadCache => (statusCode & StatusCode.AdvancedMask) == StatusCode.CopiedRecordToReadCache; - #endregion - /// /// Get the underlying status code value /// diff --git a/cs/test/AdvancedLockTests.cs b/cs/test/AdvancedLockTests.cs index c21a2ee01..0f9d59944 100644 --- a/cs/test/AdvancedLockTests.cs +++ b/cs/test/AdvancedLockTests.cs @@ -154,7 +154,7 @@ await DoTwoThreadRandomKeyTest(numKeys, } else { - Assert.IsTrue(status.Pending, $"Key = {key}, status = {status}"); + Assert.IsTrue(status.IsPending, $"Key = {key}, status = {status}"); session.CompletePending(wait: true); // Output is not clear here and we are testing only threading aspects, so don't verify } diff --git a/cs/test/AsyncLargeObjectTests.cs b/cs/test/AsyncLargeObjectTests.cs index 21cc766d6..b74d123a7 100644 --- a/cs/test/AsyncLargeObjectTests.cs +++ b/cs/test/AsyncLargeObjectTests.cs @@ -84,7 +84,7 @@ public async Task LargeObjectTest([Values]CheckpointType checkpointType) var key = new MyKey { key = keycnt }; var status = s2.Read(ref key, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) await s2.CompletePendingAsync(); else { diff --git a/cs/test/AsyncTests.cs b/cs/test/AsyncTests.cs index b75129aa8..f60cf9b4a 100644 --- a/cs/test/AsyncTests.cs +++ b/cs/test/AsyncTests.cs @@ -96,7 +96,7 @@ public async Task AsyncRecoveryTest1(CheckpointType checkpointType) { var status = s3.Read(ref inputArray[key], ref inputArg, ref output, Empty.Default, s3.SerialNo); - if (status.Pending) + if (status.IsPending) s3.CompletePending(true,true); else { diff --git a/cs/test/BasicDiskFASTERTests.cs b/cs/test/BasicDiskFASTERTests.cs index c6cf93cf8..43b4c006c 100644 --- a/cs/test/BasicDiskFASTERTests.cs +++ b/cs/test/BasicDiskFASTERTests.cs @@ -95,7 +95,7 @@ void TestDeviceWriteRead(IDevice log) var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = 1, ifield2 = 1 }; var status = session.RMW(ref key1, ref input, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); } @@ -106,7 +106,7 @@ void TestDeviceWriteRead(IDevice log) var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).Pending) + if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).IsPending) { session.CompletePending(true); } diff --git a/cs/test/BasicFASTERTests.cs b/cs/test/BasicFASTERTests.cs index f9925bc7c..754f4831e 100644 --- a/cs/test/BasicFASTERTests.cs +++ b/cs/test/BasicFASTERTests.cs @@ -53,7 +53,7 @@ public void TearDown() private void AssertCompleted(Status expected, Status actual) { - if (actual.Pending) + if (actual.IsPending) (actual, _) = CompletePendingResult(); Assert.AreEqual(expected, actual); } @@ -211,7 +211,7 @@ public unsafe void NativeInMemWriteRead2() var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).Pending) + if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).IsPending) { session.CompletePending(true); } @@ -266,7 +266,7 @@ public unsafe void TestShiftHeadAddress([Values] TestUtils.DeviceType deviceType var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; - if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).Pending) + if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).IsPending) { Assert.AreEqual(value.vfield1, output.value.vfield1); Assert.AreEqual(value.vfield2, output.value.vfield2); @@ -286,7 +286,7 @@ public unsafe void TestShiftHeadAddress([Values] TestUtils.DeviceType deviceType OutputStruct output = default; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; Status foundStatus = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(foundStatus.Pending); + Assert.IsTrue(foundStatus.IsPending); } session.CompletePendingWithOutputs(out var outputs, wait: true); @@ -333,7 +333,7 @@ public unsafe void NativeInMemRMWRefKeys([Values] TestUtils.DeviceType deviceTyp var i = nums[j]; var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = i, ifield2 = i + 1 }; - if (session.RMW(ref key1, ref input, ref output, Empty.Default, 0).Pending) + if (session.RMW(ref key1, ref input, ref output, Empty.Default, 0).IsPending) { session.CompletePending(true); } @@ -634,7 +634,7 @@ void VerifyOutput() void VerifyResult() { - if (status.Pending) + if (status.IsPending) { skipReadCacheSession.CompletePendingWithOutputs(out var completedOutputs, wait: true); (status, output) = TestUtils.GetSinglePendingResult(completedOutputs); diff --git a/cs/test/BasicLockTests.cs b/cs/test/BasicLockTests.cs index 20fc94c9e..41dba23f5 100644 --- a/cs/test/BasicLockTests.cs +++ b/cs/test/BasicLockTests.cs @@ -166,7 +166,7 @@ public void FunctionsLockTest() for (int key = 0; key < numRecords; key++) { // For this test we should be in-memory, so no pending - Assert.IsFalse(session.Upsert(key, key * valueMult).Pending); + Assert.IsFalse(session.Upsert(key, key * valueMult).IsPending); } // Update @@ -179,7 +179,7 @@ public void FunctionsLockTest() for (int key = 0; key < numRecords; key++) { var expectedValue = key * valueMult + numThreads * numIters; - Assert.IsFalse(session.Read(key, out int value).Pending); + Assert.IsFalse(session.Read(key, out int value).IsPending); Assert.AreEqual(expectedValue, value); } } @@ -191,7 +191,7 @@ void UpdateFunc(bool useRMW, int numRecords, int numIters) for (int iter = 0; iter < numIters; iter++) { if ((iter & 7) == 7) - Assert.IsFalse(session.Read(key).status.Pending); + Assert.IsFalse(session.Read(key).status.IsPending); // These will both just increment the stored value, ignoring the input argument. if (useRMW) diff --git a/cs/test/BlittableLogCompactionTests.cs b/cs/test/BlittableLogCompactionTests.cs index d7508c39b..49becbae6 100644 --- a/cs/test/BlittableLogCompactionTests.cs +++ b/cs/test/BlittableLogCompactionTests.cs @@ -69,7 +69,7 @@ public void BlittableLogCompactionTest1([Values] CompactionType compactionType) var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); else { @@ -128,7 +128,7 @@ public void BlittableLogCompactionTest2([Values] CompactionType compactionType) var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); else { @@ -185,7 +185,7 @@ public void BlittableLogCompactionTest3([Values] CompactionType compactionType) int ctx = ((i < 500) && (i % 2 == 0)) ? 1 : 0; var status = session.Read(ref key1, ref input, ref output, ctx, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); else { @@ -245,7 +245,7 @@ public void BlittableLogCompactionCustomFunctionsTest1([Values] CompactionType c var ctx = (i < (totalRecords / 2) && (i % 2 != 0)) ? 1 : 0; var status = session.Read(ref key1, ref input, ref output, ctx, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePending(true); } @@ -294,7 +294,7 @@ public void BlittableLogCompactionCustomFunctionsTest2([Values] CompactionType c var input = default(InputStruct); var output = default(OutputStruct); var status = session.Read(ref key, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePending(true); } diff --git a/cs/test/CompletePendingTests.cs b/cs/test/CompletePendingTests.cs index 13c4b46a9..f8354d8df 100644 --- a/cs/test/CompletePendingTests.cs +++ b/cs/test/CompletePendingTests.cs @@ -157,7 +157,7 @@ public async ValueTask ReadAndCompleteWithPendingOutput([Values]bool useRMW, [Va { var ksUnfound = keyStruct; ksUnfound.kfield1 += numRecords * 10; - if (session.Read(ref ksUnfound, ref inputStruct, ref outputStruct, contextStruct).Pending) + if (session.Read(ref ksUnfound, ref inputStruct, ref outputStruct, contextStruct).IsPending) { CompletedOutputIterator completedOutputs; if (isAsync) @@ -172,7 +172,7 @@ public async ValueTask ReadAndCompleteWithPendingOutput([Values]bool useRMW, [Va var status = useRMW ? session.RMW(ref keyStruct, ref inputStruct, ref outputStruct, contextStruct) : session.Read(ref keyStruct, ref inputStruct, ref outputStruct, contextStruct); - if (status.Pending) + if (status.IsPending) { if (processPending.IsFirst()) { @@ -210,7 +210,7 @@ public async ValueTask ReadAndCompleteWithPendingOutput([Values]bool useRMW, [Va // This should not be pending since we've not flushed. var localKey = key; var status = session.Read(ref localKey, ref inputStruct, ref outputStruct, ref recordMetadata); - Assert.IsFalse(status.Pending); + Assert.IsFalse(status.IsPending); Assert.AreEqual(address, recordMetadata.Address); } } diff --git a/cs/test/ExpirationTests.cs b/cs/test/ExpirationTests.cs index 52e7b8029..e065c2a55 100644 --- a/cs/test/ExpirationTests.cs +++ b/cs/test/ExpirationTests.cs @@ -478,7 +478,7 @@ private ExpirationOutput GetRecord(int key, Status expectedStatus, bool isOnDisk ExpirationOutput output = new(); var status = session.Read(ref key, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { Assert.IsTrue(isOnDisk); session.CompletePendingWithOutputs(out var completedOutputs, wait:true); @@ -493,7 +493,7 @@ private ExpirationOutput ExecuteRMW(int key, ref ExpirationInput input, bool isO { ExpirationOutput output = new (); var status = session.RMW(ref key, ref input, ref output); - if (status.Pending) + if (status.IsPending) { Assert.IsTrue(isOnDisk); session.CompletePendingWithOutputs(out var completedOutputs, wait: true); diff --git a/cs/test/GenericByteArrayTests.cs b/cs/test/GenericByteArrayTests.cs index d5da044b4..d4eb5e02a 100644 --- a/cs/test/GenericByteArrayTests.cs +++ b/cs/test/GenericByteArrayTests.cs @@ -73,7 +73,7 @@ public void ByteArrayBasicTest() var key = GetByteArray(i); var value = GetByteArray(i); - if (session.Read(ref key, ref input, ref output, Empty.Default, 0).Pending) + if (session.Read(ref key, ref input, ref output, Empty.Default, 0).IsPending) { session.CompletePending(true); } diff --git a/cs/test/GenericDiskDeleteTests.cs b/cs/test/GenericDiskDeleteTests.cs index 621a57722..e5e05b09a 100644 --- a/cs/test/GenericDiskDeleteTests.cs +++ b/cs/test/GenericDiskDeleteTests.cs @@ -66,7 +66,7 @@ public void DiskDeleteBasicTest1() var key1 = new MyKey { key = i }; var value = new MyValue { value = i }; - if (session.Read(ref key1, ref input, ref output, 0, 0).Pending) + if (session.Read(ref key1, ref input, ref output, 0, 0).IsPending) { session.CompletePending(true); } @@ -90,7 +90,7 @@ public void DiskDeleteBasicTest1() var status = session.Read(ref key1, ref input, ref output, 1, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait: true); (status, _) = GetSinglePendingResult(outputs); @@ -162,12 +162,12 @@ public void DiskDeleteBasicTest2() session.Upsert(ref _key, ref _value, 0, 0); } status = session.Read(ref key100, ref input, ref output, 1, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); // This RMW should create new initial value, since item is deleted status = session.RMW(ref key200, ref input, 1, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); status = session.Read(ref key200, ref input, ref output, 0, 0); diff --git a/cs/test/GenericLogCompactionTests.cs b/cs/test/GenericLogCompactionTests.cs index 1399fe0a7..dcaae6bff 100644 --- a/cs/test/GenericLogCompactionTests.cs +++ b/cs/test/GenericLogCompactionTests.cs @@ -102,7 +102,7 @@ public void LogCompactBasicTest([Values] TestUtils.DeviceType deviceType, [Value var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var completedOutputs, wait: true); Assert.IsTrue(completedOutputs.Next()); @@ -160,7 +160,7 @@ public void LogCompactTestNewEntries([Values] CompactionType compactionType) var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); else { @@ -212,7 +212,7 @@ public void LogCompactAfterDeleteTest([Values] CompactionType compactionType) int ctx = ((i < 500) && (i % 2 == 0)) ? 1 : 0; var status = session.Read(ref key1, ref input, ref output, ctx, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); else { @@ -264,7 +264,7 @@ public void LogCompactBasicCustomFctnTest([Values] CompactionType compactionType var ctx = (i < (totalRecords / 2) && (i % 2 != 0)) ? 1 : 0; var status = session.Read(ref key1, ref input, ref output, ctx, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePending(true); } @@ -313,7 +313,7 @@ public void LogCompactCopyInPlaceCustomFctnTest([Values] CompactionType compacti var input = default(MyInput); var output = default(MyOutput); var status = session.Read(ref key, ref input, ref output, 0, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait: true); (status, output) = GetSinglePendingResult(outputs); diff --git a/cs/test/GenericStringTests.cs b/cs/test/GenericStringTests.cs index 7294b33c2..ed84bf319 100644 --- a/cs/test/GenericStringTests.cs +++ b/cs/test/GenericStringTests.cs @@ -75,7 +75,7 @@ public void StringBasicTest([Values] TestUtils.DeviceType deviceType) var value = $"{i}"; var status = session.Read(ref key, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait:true); (status, output) = GetSinglePendingResult(outputs); diff --git a/cs/test/InputOutputParameterTests.cs b/cs/test/InputOutputParameterTests.cs index 0a8d75376..e8fec416f 100644 --- a/cs/test/InputOutputParameterTests.cs +++ b/cs/test/InputOutputParameterTests.cs @@ -125,7 +125,7 @@ async Task doWrites() var r = await session.RMWAsync(ref key, ref input); if ((key & 0x1) == 0) { - while (r.Status.Pending) + while (r.Status.IsPending) r = await r.CompleteAsync(); status = r.Status; output = r.Output; @@ -141,7 +141,7 @@ async Task doWrites() var r = await session.UpsertAsync(ref key, ref input, ref key); if ((key & 0x1) == 0) { - while (r.Status.Pending) + while (r.Status.IsPending) r = await r.CompleteAsync(); status = r.Status; output = r.Output; @@ -164,11 +164,11 @@ async Task doWrites() if (useRMW) Assert.IsFalse(status.Found, status.ToString()); else - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); Assert.AreEqual(tailAddress, session.functions.lastWriteAddress); } else - Assert.IsTrue(status.InPlaceUpdatedRecord, status.ToString()); + Assert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); Assert.AreEqual(key * input, output); Assert.AreEqual(session.functions.lastWriteAddress, recordMetadata.Address); diff --git a/cs/test/LargeObjectTests.cs b/cs/test/LargeObjectTests.cs index 57f65cea6..c51960a86 100644 --- a/cs/test/LargeObjectTests.cs +++ b/cs/test/LargeObjectTests.cs @@ -84,7 +84,7 @@ public void LargeObjectTest(CheckpointType checkpointType) var key = new MyKey { key = keycnt }; var status = session2.Read(ref key, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) session2.CompletePending(true); else { diff --git a/cs/test/LockableUnsafeContextTests.cs b/cs/test/LockableUnsafeContextTests.cs index f5b06fab3..26a5a867b 100644 --- a/cs/test/LockableUnsafeContextTests.cs +++ b/cs/test/LockableUnsafeContextTests.cs @@ -128,7 +128,7 @@ public void TearDown(bool forRecovery) void Populate() { for (int key = 0; key < numRecords; key++) - Assert.IsFalse(session.Upsert(key, key * valueMult).Pending); + Assert.IsFalse(session.Upsert(key, key * valueMult).IsPending); } static void AssertIsLocked(LockableUnsafeContext luContext, int key, bool xlock, bool slock) @@ -221,7 +221,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget status = luContext.Read(24, out var value24); if (flushMode == FlushMode.OnDisk) { - if (status.Pending) + if (status.IsPending) { luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); Assert.True(completedOutputs.Next()); @@ -234,13 +234,13 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget } else { - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); } status = luContext.Read(51, out var value51); if (flushMode == FlushMode.OnDisk) { - if (status.Pending) + if (status.IsPending) { luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); Assert.True(completedOutputs.Next()); @@ -253,7 +253,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget } else { - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); } // Set the phase to Phase.INTERMEDIATE to test the non-Phase.REST blocks @@ -264,7 +264,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget : luContext.Upsert(ref resultKey, ref dummyInOut, ref expectedResult, ref dummyInOut, out recordMetadata); if (flushMode == FlushMode.OnDisk) { - if (status.Pending) + if (status.IsPending) { luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); Assert.True(completedOutputs.Next()); @@ -277,12 +277,12 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget } else { - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); } // Reread the destination to verify status = luContext.Read(resultKey, out resultValue); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); Assert.AreEqual(expectedResult, resultValue); } foreach (var key in locks.Keys.OrderBy(key => -key)) @@ -301,7 +301,7 @@ public void InMemorySimpleLockTxnTest([Values] ResultLockTarget resultLockTarget // Verify reading the destination from the full session. status = session.Read(resultKey, out resultValue); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); Assert.AreEqual(expectedResult, resultValue); EnsureNoLocks(); } @@ -332,20 +332,20 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va status = luContext.Read(24, out var value24); if (flushMode == FlushMode.OnDisk) { - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); (status, value24) = GetSinglePendingResult(completedOutputs); Assert.IsTrue(status.Found, status.ToString()); Assert.AreEqual(24 * valueMult, value24); } else - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); // We just locked this above, but for FlushMode.OnDisk it will be in the LockTable and will still be PENDING. status = luContext.Read(51, out var value51); if (flushMode == FlushMode.OnDisk) { - if (status.Pending) + if (status.IsPending) { luContext.CompletePendingWithOutputs(out var completedOutputs, wait: true); Assert.True(completedOutputs.Next()); @@ -358,7 +358,7 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va } else { - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); } Assert.AreEqual(51 * valueMult, value51); @@ -367,10 +367,10 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va status = useRMW ? luContext.RMW(resultKey, value24 + value51) : luContext.Upsert(resultKey, value24 + value51); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); status = luContext.Read(resultKey, out resultValue); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); Assert.AreEqual(expectedResult, resultValue); luContext.Unlock(51, LockType.Exclusive); @@ -387,7 +387,7 @@ public void InMemoryLongLockTest([Values] ResultLockTarget resultLockTarget, [Va // Verify from the full session. status = session.Read(resultKey, out resultValue); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); Assert.AreEqual(expectedResult, resultValue); EnsureNoLocks(); } @@ -424,7 +424,7 @@ public void InMemoryDeleteTest([Values] ResultLockTarget resultLockTarget, [Valu // Set the phase to Phase.INTERMEDIATE to test the non-Phase.REST blocks session.ctx.phase = phase; status = luContext.Delete(ref resultKey); - Assert.IsFalse(status.Pending, status.ToString()); + Assert.IsFalse(status.IsPending, status.ToString()); // Reread the destination to verify status = luContext.Read(resultKey, out var _); @@ -575,7 +575,7 @@ public void TransferFromLockTableToCTTTest() AddLockTableEntry(luContext, key, immutable: false); var status = session.Read(ref key, ref input, ref output, ref recordMetadata, ReadFlags.CopyToTail); - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); session.CompletePending(wait: true); VerifyAndUnlockSplicedInKey(luContext, key); @@ -610,14 +610,14 @@ public void TransferFromLockTableToUpsertTest([Values] ChainTests.RecordRegion r key = transferToExistingKey; AddLockTableEntry(luContext, key, recordRegion == ChainTests.RecordRegion.Immutable); var status = luContext.Upsert(key, key * valueMult); - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); } else { key = transferToNewKey; AddLockTableEntry(luContext, key, immutable: false); var status = luContext.Upsert(key, key * valueMult); - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); } } catch (Exception) @@ -651,7 +651,7 @@ public void TransferFromLockTableToRMWTest([Values] ChainTests.RecordRegion reco key = transferToExistingKey; AddLockTableEntry(luContext, key, recordRegion == ChainTests.RecordRegion.Immutable); var status = luContext.RMW(key, key * valueMult); - Assert.IsTrue(recordRegion == ChainTests.RecordRegion.OnDisk ? status.Pending : status.Found); + Assert.IsTrue(recordRegion == ChainTests.RecordRegion.OnDisk ? status.IsPending : status.Found); luContext.CompletePending(wait: true); } else @@ -696,7 +696,7 @@ public void TransferFromLockTableToDeleteTest([Values] ChainTests.RecordRegion r var status = luContext.Delete(key); // Delete does not search outside mutable region so the key will not be found - Assert.IsTrue(!status.Found && status.CreatedRecord, status.ToString()); + Assert.IsTrue(!status.Found && status.Record.Created, status.ToString()); VerifyAndUnlockSplicedInKey(luContext, key); } @@ -787,9 +787,9 @@ public void TransferFromReadOnlyToUpdateRecordTest([Values] UpdateOp updateOp) }; Assert.IsFalse(status.IsFaulted, $"Unexpected UpdateOp {updateOp}, status {status}"); if (updateOp == UpdateOp.RMW) - Assert.IsTrue(status.CopyUpdatedRecord, status.ToString()); + Assert.IsTrue(status.Record.CopyUpdated, status.ToString()); else - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); var (xlock, slock) = luContext.IsLocked(key); Assert.IsTrue(xlock); @@ -827,7 +827,7 @@ public void LockNewRecordCompeteWithUpdateTest([Values(LockOperationType.Lock, L if (updateOp == UpdateOp.Delete) { for (var key = numRecords; key < numRecords + numNewRecords; ++key) - Assert.IsFalse(session.Upsert(key, key * valueMult).Pending); + Assert.IsFalse(session.Upsert(key, key * valueMult).IsPending); fht.Log.FlushAndEvict(wait: true); } @@ -928,7 +928,7 @@ void updater(int key) }; Assert.IsFalse(status.IsFaulted, $"Unexpected UpdateOp {updateOp}, status {status}"); Assert.IsFalse(status.Found, status.ToString()); - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); } catch (Exception) { @@ -1014,7 +1014,7 @@ public void EvictFromMainLogToLockTableTest() int input = 0, output = 0, localKey = key; RecordMetadata recordMetadata = default; var status = session.Read(ref localKey, ref input, ref output, ref recordMetadata, ReadFlags.CopyToTail); - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); session.CompletePending(wait: true); Assert.IsFalse(fht.LockTable.Get(key, out _)); @@ -1149,7 +1149,7 @@ async static Task PrimaryWriter(FasterKV primaryStore, SyncMode sync } var status = s1.Upsert(ref key, ref key); - Assert.IsTrue(status.CreatedRecord, status.ToString()); + Assert.IsTrue(status.Record.Created, status.ToString()); luc1.Lock(key, LockType.Shared); } diff --git a/cs/test/LowMemAsyncTests.cs b/cs/test/LowMemAsyncTests.cs index 83c8be2fe..a0a692014 100644 --- a/cs/test/LowMemAsyncTests.cs +++ b/cs/test/LowMemAsyncTests.cs @@ -54,7 +54,7 @@ private static async Task Populate(ClientSession, int, int> public override void RMWCompletionCallback(ref ReadOnlyMemory key, ref Memory input, ref (IMemoryOwner, int) output, int ctx, Status status, RecordMetadata recordMetadata) { Assert.IsTrue(status.Found); - Assert.IsTrue(status.CopyUpdatedRecord); + Assert.IsTrue(status.Record.CopyUpdated); } public override void ReadCompletionCallback(ref ReadOnlyMemory key, ref Memory input, ref (IMemoryOwner, int) output, int ctx, Status status, RecordMetadata recordMetadata) diff --git a/cs/test/MiscFASTERTests.cs b/cs/test/MiscFASTERTests.cs index 1c392f487..bc9a1a0b4 100644 --- a/cs/test/MiscFASTERTests.cs +++ b/cs/test/MiscFASTERTests.cs @@ -80,7 +80,7 @@ public void MixedTest2() MyOutput g1 = new(); var status = session.Read(ref key2, ref input, ref g1, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait:true); (status, _) = GetSinglePendingResult(outputs); @@ -92,7 +92,7 @@ public void MixedTest2() key2 = 99999; status = session.Read(ref key2, ref input, ref g1, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait: true); (status, _) = GetSinglePendingResult(outputs); @@ -124,12 +124,12 @@ public void ShouldCreateNewRecordIfConcurrentWriterReturnsFalse() value = new ValueStruct() { vfield1 = 1000, vfield2 = 2000 }; var status = session.Upsert(ref key, ref input, ref value, ref output, out RecordMetadata recordMetadata1); - Assert.IsTrue(!status.Found && status.CreatedRecord, status.ToString()); + Assert.IsTrue(!status.Found && status.Record.Created, status.ToString()); // ConcurrentWriter returns false, so we create a new record (and leave the old one sealed). value = new ValueStruct() { vfield1 = 1001, vfield2 = 2002 }; status = session.Upsert(ref key, ref input, ref value, ref output, out RecordMetadata recordMetadata2); - Assert.IsTrue(!status.Found && status.CreatedRecord, status.ToString()); + Assert.IsTrue(!status.Found && status.Record.Created, status.ToString()); Assert.Greater(recordMetadata2.Address, recordMetadata1.Address); diff --git a/cs/test/NativeReadCacheTests.cs b/cs/test/NativeReadCacheTests.cs index 0a6ad0837..8c385c707 100644 --- a/cs/test/NativeReadCacheTests.cs +++ b/cs/test/NativeReadCacheTests.cs @@ -60,7 +60,7 @@ public void NativeDiskWriteReadCache() var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -88,7 +88,7 @@ public void NativeDiskWriteReadCache() var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -120,7 +120,7 @@ public void NativeDiskWriteReadCache() var key1 = new KeyStruct { kfield1 = i, kfield2 = i + 1 }; input = new InputStruct { ifield1 = 1, ifield2 = 1 }; var status = session.RMW(ref key1, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePending(true); } @@ -172,7 +172,7 @@ public void NativeDiskWriteReadCache2() var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -200,7 +200,7 @@ public void NativeDiskWriteReadCache2() var value = new ValueStruct { vfield1 = i, vfield2 = i + 1 }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } diff --git a/cs/test/NeedCopyUpdateTests.cs b/cs/test/NeedCopyUpdateTests.cs index 829761686..a8f1be9ad 100644 --- a/cs/test/NeedCopyUpdateTests.cs +++ b/cs/test/NeedCopyUpdateTests.cs @@ -63,7 +63,7 @@ public void TryAddTest() Assert.IsTrue(value1.flag); // InitialUpdater is called status = session.RMW(ref key, ref value2); // InPlaceUpdater + OK - Assert.IsTrue(status.InPlaceUpdatedRecord, status.ToString()); + Assert.IsTrue(status.Record.InPlaceUpdated, status.ToString()); fht.Log.Flush(true); status = session.RMW(ref key, ref value2); // NeedCopyUpdate returns false, so RMW returns simply Found @@ -71,7 +71,7 @@ public void TryAddTest() fht.Log.FlushAndEvict(true); status = session.RMW(ref key, ref value2, new(StatusCode.OK), 0); // PENDING + NeedCopyUpdate + OK - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); session.CompletePendingWithOutputs(out var outputs, true); var output = new RMWValue(); @@ -80,15 +80,15 @@ public void TryAddTest() // Test stored value. Should be value1 status = session.Read(ref key, ref value1, ref output, new(StatusCode.OK), 0); - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); session.CompletePending(true); status = session.Delete(ref key); - Assert.IsTrue(!status.Found && status.CreatedRecord, status.ToString()); + Assert.IsTrue(!status.Found && status.Record.Created, status.ToString()); session.CompletePending(true); fht.Log.FlushAndEvict(true); status = session.RMW(ref key, ref value2, new(StatusCode.NotFound | StatusCode.CreatedRecord), 0); // PENDING + InitialUpdater + NOTFOUND - Assert.IsTrue(status.Pending, status.ToString()); + Assert.IsTrue(status.IsPending, status.ToString()); session.CompletePending(true); } } diff --git a/cs/test/ObjectFASTERTests.cs b/cs/test/ObjectFASTERTests.cs index a92163e09..4580fb5c6 100644 --- a/cs/test/ObjectFASTERTests.cs +++ b/cs/test/ObjectFASTERTests.cs @@ -104,7 +104,7 @@ public void ObjectDiskWriteRead() MyOutput g1 = new(); var status = session.Read(ref key2, ref input, ref g1, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePendingWithOutputs(out var outputs, wait: true); (status, g1) = GetSinglePendingResult(outputs); @@ -116,7 +116,7 @@ public void ObjectDiskWriteRead() key2 = new MyKey { key = 99999 }; status = session.Read(ref key2, ref input, ref g1, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { session.CompletePending(true); } @@ -131,7 +131,7 @@ public void ObjectDiskWriteRead() var key1 = new MyKey { key = i }; input = new MyInput { value = 1 }; status = session.RMW(ref key1, ref input, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); } @@ -141,7 +141,7 @@ public void ObjectDiskWriteRead() var key1 = new MyKey { key = i }; var value = new MyValue { value = i }; - if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).Pending) + if (session.Read(ref key1, ref input, ref output, Empty.Default, 0).IsPending) { session.CompletePending(true); } @@ -174,7 +174,7 @@ public async Task ReadAsyncObjectDiskWriteRead() var value = new MyValue { value = i }; var r = await session.UpsertAsync(ref key, ref value); - while (r.Status.Pending) + while (r.Status.IsPending) r = await r.CompleteAsync(); // test async version of Upsert completion } @@ -204,7 +204,7 @@ public async Task ReadAsyncObjectDiskWriteRead() var key = new MyKey { key = i }; input = new MyInput { value = 1 }; var r = await session.RMWAsync(ref key, ref input, Empty.Default); - while (r.Status.Pending) + while (r.Status.IsPending) { r = await r.CompleteAsync(); // test async version of RMW completion } diff --git a/cs/test/ObjectReadCacheTests.cs b/cs/test/ObjectReadCacheTests.cs index 5b09f28bf..fa1b291c1 100644 --- a/cs/test/ObjectReadCacheTests.cs +++ b/cs/test/ObjectReadCacheTests.cs @@ -67,7 +67,7 @@ public void ObjectDiskWriteReadCache() var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -94,7 +94,7 @@ public void ObjectDiskWriteReadCache() var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -125,7 +125,7 @@ public void ObjectDiskWriteReadCache() var key1 = new MyKey { key = i }; input = new MyInput { value = 1 }; var status = session.RMW(ref key1, ref input, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) session.CompletePending(true); } @@ -169,7 +169,7 @@ public void ObjectDiskWriteReadCache2() var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } @@ -196,7 +196,7 @@ public void ObjectDiskWriteReadCache2() var value = new MyValue { value = i }; var status = session.Read(ref key1, ref input, ref output, Empty.Default, 0); - Assert.IsTrue(status.Pending); + Assert.IsTrue(status.IsPending); session.CompletePending(true); } diff --git a/cs/test/ObjectRecoveryTest2.cs b/cs/test/ObjectRecoveryTest2.cs index 715a773ff..e839c0908 100644 --- a/cs/test/ObjectRecoveryTest2.cs +++ b/cs/test/ObjectRecoveryTest2.cs @@ -124,7 +124,7 @@ private void Read(ClientSession.RmwAsyncResult.UpsertAsyncResult : FunctionsBase public override void RMWCompletionCallback(ref Key key, ref Input input, ref int[] output, Empty ctx, Status status, RecordMetadata recordMetadata) { Assert.IsTrue(status.Found); - Assert.IsTrue(status.CopyUpdatedRecord); + Assert.IsTrue(status.Record.CopyUpdated); } public override void ReadCompletionCallback(ref Key key, ref Input input, ref int[] output, Empty ctx, Status status, RecordMetadata recordMetadata) @@ -142,7 +142,7 @@ public class VLFunctions2 : FunctionsBase public override void RMWCompletionCallback(ref VLValue key, ref Input input, ref int[] output, Empty ctx, Status status, RecordMetadata recordMetadata) { Assert.IsTrue(status.Found); - Assert.IsTrue(status.CopyUpdatedRecord); + Assert.IsTrue(status.Record.CopyUpdated); } public override void ReadCompletionCallback(ref VLValue key, ref Input input, ref int[] output, Empty ctx, Status status, RecordMetadata recordMetadata) diff --git a/cs/test/VariableLengthStructFASTERTests.cs b/cs/test/VariableLengthStructFASTERTests.cs index 838825cf2..0ee798bf3 100644 --- a/cs/test/VariableLengthStructFASTERTests.cs +++ b/cs/test/VariableLengthStructFASTERTests.cs @@ -59,7 +59,7 @@ public unsafe void VariableLengthTest1() int[] output = null; var status = s.Read(ref key1, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { s.CompletePendingWithOutputs(out var outputs, wait: true); (status, output) = GetSinglePendingResult(outputs); @@ -129,7 +129,7 @@ public unsafe void VariableLengthTest2() int[] output = null; var status = s.Read(ref key1, ref input, ref output, Empty.Default, 0); - if (status.Pending) + if (status.IsPending) { s.CompletePendingWithOutputs(out var outputs, wait: true); (status, output) = GetSinglePendingResult(outputs);