Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[C#] LibDPR API improvements + Subscription (#589)
* [C#] Expand async session-free API sample in playground (#452) * added sample * cleaned up async stress * Fix memory leak + add serialized wrapper sample * added testcases for concurrency within session in async. * clean up LMD * YCSB improvements to test incr-snapshot (#454) * [C#] minor cleanup * [C#] remote update to latest IFunctions * Update 01-quick-start-guide.md * Update home.md * [C#] update docs * fix(docs): corrects typo in project documentation (#456) * Initial checkin of test suite for remote * updated configs. * validate incoming payload before resuming epoch. (#459) * [C#] Test suite for remote FASTER (#458) * Initial checkin of test suite for remote * updated configs. * fix sln * Update azure-pipelines.yml * Update azure-pipelines.yml * Update 50-remote-basics.md * Update 50-remote-basics.md * [C#] Remote tests disposal (#460) * Initial checkin of test suite for remote * updated configs. * fix sln * Update azure-pipelines.yml * Update azure-pipelines.yml * Ensure correct disposals of server and client. * rename testcase * Update 50-remote-basics.md * Update 50-remote-basics.md * Enable SourceLink integration (#457) * clean sln * update docs * update docs * [C#] update remote testcases to cover varlen * Update 96-slides-videos.md * [C#] Fix double dispose in CompletePendingWithOutputs (#465) * [C#] Fix double dispose in CompletePendingWithOutputs * Update MultiReadSpanByteKey.cs * Cleaned up testcase, merged with other SpanByte test. Co-authored-by: Tim Coleman <timothy.coleman@gmail.com> * [C#] [FasterLog] Create AsyncPool objects on demand (instead of prepopulating) (#455) * don't pre-create 120 read and 120 write file handles. * fix asyncpool semaphore sizing. allow control over LocalStorageNamedDeviceFactory's Device.ThrottleLimit * dispose off the semaphore in AsyncPool.Dispose() * minor fixes to AsyncPool. * Speed up async pool. * Fix ordering of GetOrAdd and wait. Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Updated SpanByte sample with RMW for ASCII number sums (#462) * Updated SpanByte sample with RMW for ASCII number sums * Sample refactoring as class per sub-sample * Update 20-fasterkv-basics.md * Update 20-fasterkv-basics.md * Update 20-fasterkv-basics.md * Update home.md * [C#] update csproj * Update home.md * Fix typo (#468) * Update 25-fasterkv-recovery.md * Update 23-fasterkv-tuning.md * Update 23-fasterkv-tuning.md * Fix docs typo (#470) * [C#] Support reducing memory footprint of pages in log (#467) * [C#] Support reducing memory footprint of base hlog circular buffer. * Update 23-fasterkv-tuning.md * updates * cleanup * Add Log API to set page count with option to wait for address shift to complete. * [C#] Add testcase: resume iterator with async enumerable (#471) * [C#] Fix timing issue in testcase * Update 23-fasterkv-tuning.md * [C#] benchmark tuning for IOPS * [C#] WaitAsync should return false if iterator has reached end of scan range. * Fix azure blob list contents logic * [C#] Fixes and refactoring of FASTERAsync (#480) * Fix to move await of flushtask to Slow*Async * Fix setting of RecordInfo in ReadAsync * FASTERAsync: do not use CompleteAsync.GetAwaiter() in Complete() * fix asyncOp and asyncOperation usage * Add XML comment for throttleLimit param * Change flushTask to a SemaphoreSlim to avoid GetAwaiter().GetResult() in the sync case * Add AsyncPool.Get() for sync paths and change MLSD to use it instead of GetAwaiter().GetResult() * Add ConfigureAwait(false) to all awaits * Break FASTERAsync out to separate files * Prep for RMW refactor * Rename UpdelAsync -> UpdateAsync * Refactor RMWAsync to use UpdateAsync * Add GetAwaiter().GetResult in one void-returning function where it can't be avoided, * Update AsyncStress to include options for rmw-only, upsert-only, large vs. small memory log, and whether to use Value, Reference, or Span data types * Remove obsolete RecordAccessor.cs * Make IHeapContainer implement IDisposable * Fix the TakeFullCheckpoint overload without CheckpointType to call through to the overload with it * Update session.XxxAsync() doc comments for clarity and to emphasize the need to complete operations to avoid leaks * add raw stddev %s to output * Add -NoLock option to compare_runs * Add comments to IUpdateAsyncOperation; fix comment on ccompare_runs Co-authored-by: TedHartMS <15467143+TedHartMS@users.noreply.github.com> * Update home.md * [C#] Fix read cache boundary condition bug (#483) * [C++] Fix epoch scan range (#484) * [C++] Fix epoch scan range * updates * fix testcase * Update home.md * add FASTER.benchmark cmdline options: --sd --sm --noaff --chkptms --dumpdist (#485) * LogCommitFile is obsolete - commit file naming is handled automatically by default commit manager. (#494) * Update 40-fasterlog-basics.md * [C#] Update to C#9 (#486) * Add <LangVersion>latest to csprojs to pick up C#9; remove "is {}" in favor of "is not null"; use "Type x = new()", which is particularly useful for field definitions * Add net5.0 framework to core component builds; add "|| NET" to "#if NETSTANDARD" where necessary; switch samples to net5.0; add Debug builds to Linux * Update azure-pipelines.yml * Move C# Windows tasks to dotnet; add build/test tasks for Remote to C# pipelines * search for all *.test.csproj in build/test tasks * Restrict Linux to netcoreapp3.1 and net5.0 * remove duplicate langversion Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Add infrastructure to support more wire protocols (#496) * Add BackendProvider concept to support backend other than FasterKV. Add protocol byte to binary protocol to enable support for more protocols in the future. * Fix some typos and add some comments * add .idea to gitignore * Add constructor overload to avoid API change, tweaks to serialization performance. * remove unused import * Cleanup & minor perf improvements Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Remote provider enhancements (#503) * Refactor remote provider * Cleanup and speedup * cleanup and updates * Add a link to the DPR branch (#506) Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> * [C#] Recovery testcase fixing (#513) * make error more detailed * try sorting by creation time for linux compat * change test to use specific token, as workaround for linux timestamp sorting granularity issue. * Ignore socket disposal exception in RecvEventArg_Completed * Restore full Azure pipeline * [C#] Add Output to RMW (#515) * Add Output to RMW: - ** BREAKING CHANGE ** I(Advanced)Functions.CopyUpdater, InitialUpdater, InPlaceUpdater, and RMWCompletionCallback now have a ref Output parameter - ** BREAKING CHANGE ** RmwAsyncResult.Complete now returns a (Status, Output) tuple * Fix Remote IFunctions implementations to new RMW signatures * Change RMW callback signature (move Output to end) * Update Remote to new RMW Output signature; add $ReadPercentages to run_benchmark * Add Output to NeedCopyUpdate as well * Propagate RMW Output throughout Remote (ICallbackFunctions etc.); update docs for RMW Output * [C#] SpanByteFasterKVProvider (#520) * Added SpanByteFasterKVProvider, and changed SpanByteSerializer to SpanByteServerSerializer and SpanByteClientSerializer * Removed unnecessary functions / debugs * fixed nit * [C#] Support FasterLog commit dir specification in settings (#519) * Support log commit path specification to FasterLog. * updates * Convert serialized SpanByte into its non-serialized form (safe to heap-copy as long as underlying payload is fixed) * minor updates. * minor * Fix failed assertion: add missing merger (#522) * Fix failed assertion: add missing merger * End-To-End sample cleanup * Revert usage of new RMW signature to the sample * [C#] Multiple device tests (#523) * Added BasicHighLatencyDeviceTest * Added ability to run each test under multiple devices * Updated Pipeline Yaml to only run tests labeled "Smoke" * Fixed Yaml to run only smoke * Updated DotNetCoreCLI run by adding TestCaseFilter * DotNetCoreCLI was failing * Add extra quote in there breaking yaml file * Fixed DotNetCoreCLI filter * Fixed --Filter to --filter * Made Enqueue Async part of smoke * Trying MLSD only for Devices * Fixed bug in test code * Trying LSD Device Type by itself * Updated TryEnqueueBasicTest to use all the various devices * Added TestCategory to some of the new tests and made the Log Scan tests to use DeviceType enum * Adding a Condition to only run LSD on Windows * Added Conditional Compilation to Test Project file * Added Emulated Azure device option and the Smoke tag to quite a few tests. * Set Emulated Azure storage tests to be Windows only as it isn't supported on Linux * When doing a check for emulated Azure Storage, need to make sure only on Windows as not supported on Linux * Removed some debug info * Updated Ubuntu run from 16.04 to latest * Added in LocalMemory to DeviceType tests to test when have disk latency. * Adding Azure Pipeline full test run yml file * Updated Full Pipeline - Added a schedule of 4:00 am nightly and removed only running smoke so runs full * Turned off CI for Faster full pipeline - runs on schedule and not on CI * Commented out some tests for LocalMemory until have more time to investigate why they are failing * Fixed test bug that was causing Linux build to fail because was referencing WIndows only EmulatedAzure device type * Enable parallelization of Unit Tests * Fix TestUtils.AzureMethodTestContainer for Azure naming rules * Fix missing wait for task in AzureStorageDevice.WriteAsync; better validation of test Azure container names and directories * Minor change to see if scheduled CI fires off * Update azure-pipelines.yml * Updated tests to more consistent setup and teardown * Updated FASTER.test.csproj to not use NET5.0 * Put net 5.0 back in FASTER.test.csproj * Updated pipeline yml to match new version but still kept smoke only tests in it * Fixed a bug in YML to only run smoke. Updated Full test run yml * Fixed timing issue with CommitNoSpinWait. Problem is that the test is a "no wait" so can't just wait until done. Need to put a pause in there as a wait. * Support log commit path specification to FasterLog. * updates * More Assert improvements to RecoveryChecks.cs; carry errorCode through ErrorList; Fix Azure paths in CreateTestDevice * Ensure testmethod directory is cleaned before test run; flatten testmethod dir hierarchy, fix LocalMemoryDevice IndexOutOfRangeException; clean up TestUtils.CreateTestDevice to reflect the MethodTestDir. * Updated tests by removing the "datacheckrun" part of the verification and made it simpler * Fixed EnqueueBasicTest issue with SpanBatch test. Also updated verification of another test. * A couple missed directory cleanups * Added LogCommitDir to the FasterLog call to fix issue where .LocalMemory DeviceType was failing * turn off parallelized runs * Minor changes to comments * Added multi device type tests to TestDisposeReleasesFileLocksWithInprogressCommit and DeltaLogTest1 * Addeed Setup and TearDown to DeltaLog test * Added Setup and Tear down to PageBlob tests and HighLatencyDeviceTests * Added multi device tests to LogReadAsyncBasicTest. Reduced entry numbers to PageBlobFasterLog* teests to reduce memory usage. * Updated KV test StringBasicTest to use multiple devices * Updated basic KV tests to be multiple devices and to be smokes * Modified test CommitAsyncPrevTask to not fully run as look into making it more stable without sleeps * Made KV Test DiskWriteScanBasicTest a multiple device test and labeled smoke. Also, h duplicate of CommitAsyncPrevTask so removed the dupe * Updated test CommitNoSpinWait to not run until we get working without sleeps * Rolled back the KV Basic Faster tests to see if that is what is causing the issue with the CI failures * Put changes to KV Basic tests back to using multiple device types * Revert "Put changes to KV Basic tests back to using multiple device types" This reverts commit 62a6ed9. * Revert "Rolled back the KV Basic Faster tests to see if that is what is causing the issue with the CI failures" This reverts commit ac87cea. * Reverting commits that were causing tests to fail * Revert "Updated test CommitNoSpinWait to not run until we get working without sleeps" This reverts commit 9c983b5. * Revert "Made KV Test DiskWriteScanBasicTest a multiple device test and labeled smoke. Also, h duplicate of CommitAsyncPrevTask so removed the dupe" This reverts commit d337ddf. * Revert "Modified test CommitAsyncPrevTask to not fully run as look into making it more stable without sleeps" This reverts commit 7b247e3. * Updating KV DiskWriteScanBasicTest to be multiple device type tests * Seeing CommitAsyncPrevTask test duplicated. One was multiple device type test and one was hard code to one device. Remove one hard coded to one device type. * Exiting out of test CommitAsyncPrevTask and CommitNoSpinWait due to unstable test runs using Sleeps. * Rolling back the test changes in CommitAsyncPrevTask, CommitNoSpinWait and put back the dupe of CommitAsyncPrevTask as now seeing fails in CI * Updated several KV Basic tests to be multi Device Type tests * Commented out a few FasterLog tests that have sleeps in them so can work to remove sleeps to stabilize test runs * Rolling back Multi Device tests for KV Basic tests due to instability when running CIs ... will investigate more * Rolling back tests in Faster Log that were commented out as causing funky results in CIs * Marked quite a few basic KV Log tests as "Smoke" tests. Also reduced number events (memory) used in a couple tests as not needed to use that much memory. * Had bug in KV Log Basic test code that had multiple device test code still in there. * Commented out CommitAsyncPrevTask in attempt to stabilize CI test queue * Preliminary check in to get a bunch of KV Basic tests move to testing all device types * Setting deleteOnClose when creating device to see if that is causing memory corruption * Trying to get KV basic tests to not pop exception for memory corruption on CI machine * Made a couple KV basic tests to test on all four device types * Updated more KV basic tests to run all four device types as part of the test * Categorized tests as "Smokes" to give good representation of tests that can be used as CI smokes * Made MemoryLogCompactionTest1 to do four device type tests as need at least one compaction test to test all four device types. Not part of smokes * Investigated why couple tests failed on LocalMemory device Type and either filed bugs or fixed code. Also, commented out two duplicate tests. * Removed three duplicate tests ... guessing a merge issue at some point. * Updated CommitAsyncPrevTask to not use Sleep since there is wait on the current task * Removed a sleep in ManagedLocalStoreBasicTest as wasn't needed since the commit had true for parametre (wait until commit finishes before moving on) * Updated WaitForCommitBasicTest to be multi threaded instead of using Sleep(). * Updated TearDown as it was missing the DeleteDirectory() call. * Various test cleanup - Centralize "RunAzureTests" checking - Change GenericLogCompactionTests to illustrate using TestContext.CurrentContext.Test.Arguments - Resolve bugs - #142980 - Blob not exist exception in Dispose so use Try \ Catch to make sure tests run without issues - #143432 - DeltaLogTest on LocalMemory is only returning 50 items when should be returning 200 - #143433 - RecoveryTestFullCheckpoint - LocalMemory only - only reading 1 item when other devices reading all 16384 items * Throw if incompatible sector sizes when writing files. Add Recovery tests by allocator type, and include failure tests of the sector-size throw * Clean up AzureStorageDevice to use GetAwaiter().OnComplete instead of unawaited ContinueWith(); fixes the following bugs: - Completes the fix for #143131 (fht.Log.Scan failing to get proper value on EmulatedAzure only) TestUtils.CreateTestDevice now uses Path.GetFileName(fileName) rather than a constant string - Fixes #136259 (ReadFlags.SkipReadCache part of ReadAtAddress - SingleReader needs to be fixed to match SingleWriter) Co-authored-by: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Fix data races between copy to tail and concurrent upserts (#501) * [bug fix] copy from read-only region to tail now won't race with upserts * use InternalTryCopyToTail to handle copy to tail in general * add code example to preserve recordInfo in compact * fix bugs in previous commit * update comments InternalTryCopyToTail takes two sessions for copy from disk to tail * indent * fix bugs copy to tail now auto retries due to failed CAS compact now doesn't copy to read cache * optimize InternalCopyToTail retry without find entry again. * temp k->addr in Compact * take back try copy to tail * temporarily comment out assert for passing check * clean up comment move logical address check in Compact to debug * address comments suggested by Ted remove debug code in compact and CopyToTail add comments Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * Add ReadFlags.MinAddress (#527) * Add ReadFlags.MinAddress * Fix unawaited task in WaitForCommitBasicTest * Merge master; fix a couple comments related to session.CopyToTail args * [C#] Slight modification in Guid allocator within DeviceLogCommitCheckpointManager (#532) * Pass client serial numbers to remote FASTER instance (#528) Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> * [C++] Rmw bugfix (#533) * Add tests that expose the bug * Rmw bugfix * [C#] Faster Extensions for DPR Integration (#529) * StateMachine callback + commit cookies + public version information * Add custom version option in checkpoints * Handle roll-over correctly * character encoding * Basic test for version roll-over * address review comments * [C#] Checkpoint & Versioning Utilities (#535) * recover incremental checkpoint explicitly by version instead of just latest. Add Purge by guid to checkpoint manager * Tests and cleanup * API change to recover by version instead of token + version in incremental * fix test compilation * avoid scanning through entire delta log when searching for token corresponding to version * Add version accessors * codeo review comments Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> * SpanByte improvements (#537) * [C#] Fix dispose in tests, remove console writes from LMD. * [C#] More fixes to testcases * [C#] Fixing another testcase * Update 80-build-and-test.md * [C#] Log disposal (#548) * Various fixes to dispose devices * fix incremental snapshot double dispose * Fix dispose issue * Fix another test leak Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> Co-authored-by: Tianyu Li <litianyu@mit.edu> * [C#] minor update to spanbyte * MemoryPool-based override for FasterLog.ReadAsync() (#549) * [C#] Support remote pub-sub and websockets (#543) * [C#] Pub/sub functionality in remote FasterKV (#514) * Add client side support for SubscribeKV * basic working impl * Separate span byte serializer into client and server * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys (#499) * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys * Added prefix matching, but similar to redis and unoptimized * Added null checking for subscriptions on removeSubscriptions * Made changes to correct concurrency and type checking * Optimized and corrected the Start() method and Publish() method in publish call * Added prefix subscriptions. Added a new client serializer call, need to discuss * Corrected few nits * Cleanup of code, and resolving NIE * Nit fix of null checking * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case (#511) * Resolving merge conflicts * Removing the subscriptions belonging to a session on disconnection * Added unit tests and VarLenClient test * nit fix * Made broker associated with FasterServer and not FasterKVProvider * fixed a bug for large values in pub-sub * Changed the SubscribeKVBroker to be spawned by the user and passed to provider * fixed nit * Merged with recent master code * Checked fixed len client * Fixed bug in Write of key in server serializer * Added null check for broker Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * Improved byte array comparer, moved to FASTER.server * refactor tests * [C#] Remote pubsub (#531) * Add client side support for SubscribeKV * basic working impl * Separate span byte serializer into client and server * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys (#499) * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys * Added prefix matching, but similar to redis and unoptimized * Added null checking for subscriptions on removeSubscriptions * Made changes to correct concurrency and type checking * Optimized and corrected the Start() method and Publish() method in publish call * Added prefix subscriptions. Added a new client serializer call, need to discuss * Corrected few nits * Cleanup of code, and resolving NIE * Nit fix of null checking * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case (#511) * Resolving merge conflicts * Removing the subscriptions belonging to a session on disconnection * Added unit tests and VarLenClient test * nit fix * Made broker associated with FasterServer and not FasterKVProvider * fixed a bug for large values in pub-sub * Changed the SubscribeKVBroker to be spawned by the user and passed to provider * fixed nit * Merged with recent master code * Checked fixed len client * Fixed bug in Write of key in server serializer * Working with non-kv pub-sub. Based off remote-subkv PR * Fixed some nits, making tests work * Changed the subscriptions and prefixSubscriptions dictionary to use sid as key Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Remote subkv fixes (#542) * Add client side support for SubscribeKV * basic working impl * Separate span byte serializer into client and server * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys (#499) * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys * Added prefix matching, but similar to redis and unoptimized * Added null checking for subscriptions on removeSubscriptions * Made changes to correct concurrency and type checking * Optimized and corrected the Start() method and Publish() method in publish call * Added prefix subscriptions. Added a new client serializer call, need to discuss * Corrected few nits * Cleanup of code, and resolving NIE * Nit fix of null checking * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case (#511) * Resolving merge conflicts * Removing the subscriptions belonging to a session on disconnection * Added unit tests and VarLenClient test * nit fix * Made broker associated with FasterServer and not FasterKVProvider * fixed a bug for large values in pub-sub * Changed the SubscribeKVBroker to be spawned by the user and passed to provider * fixed nit * Merged with recent master code * Checked fixed len client * Fixed bug in Write of key in server serializer * Added null check for broker * Fixed a nit: subscriptions dictionary should contain key as sid and val as serversession * fixed nit * Removed old ByteArrayComparer Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Remote websocket (#530) * Add client side support for SubscribeKV * basic working impl * Separate span byte serializer into client and server * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys (#499) * Added support in Subscriptions. TODO: prefix matching for subscriptions, publish deletes to keys * Added prefix matching, but similar to redis and unoptimized * Added null checking for subscriptions on removeSubscriptions * Made changes to correct concurrency and type checking * Optimized and corrected the Start() method and Publish() method in publish call * Added prefix subscriptions. Added a new client serializer call, need to discuss * Corrected few nits * Cleanup of code, and resolving NIE * Nit fix of null checking * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case * Fixed small nit in MemoryparamSerializer to return Memory<byte> of correct length, and in publish call corner case (#511) * Resolving merge conflicts * Removing the subscriptions belonging to a session on disconnection * Added unit tests and VarLenClient test * nit fix * Made broker associated with FasterServer and not FasterKVProvider * fixed a bug for large values in pub-sub * Changed the SubscribeKVBroker to be spawned by the user and passed to provider * fixed nit * Merged with recent master code * Checked fixed len client * Working on JS client * Added batching in websocket handling. Next step: finish HandlePending() * Modified websocket server. Fixed bug in Write key of server serializer * duplicate file from faster.common * Added sample for WebClient * fixed small nit * Fixed some nits in js clientsession * fixed nit * removed old ByteArrayComparer * Fixed nits, added non-kv PubSub to WebsocketServerSession Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * Cleanup and updates. * fix break. * [C#] Modifications to pubsub (#547) * Added FASTER Log for SubscribeKV and Subscribe * Added input in subscribeKV * Added SubscribeCallback and SubscribeKV sample in html * set enablePubSub: false for non-pubsub tests Co-authored-by: rohankadekodi-msr <69916400+rohankadekodi-msr@users.noreply.github.com> * [C#] Client-server cleanup (#553) * Cleanup of server * More cleanup * Proper dispose of broker * updates * fix testcase * Fix bug in AcquireLatchX (#550) * Fix bug in AcquireLatchX Check version of traced record rather than of a bucket's tail record while deciding to update in place or COW * Separate out InVersionNew overloads Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * Fix typo in 25-fasterkv-recovery.md. (#555) `(seq % 1L << 20)` = 0 for any `seq`, should be either `seq % (1L << 20)` or `seq % fht.IndexSize`. * [C#] Stabilize tests (#551) * Seeing Out of Memory failures on Release \ Any CPU test runs. These are caused by log allocation sizes being too big. I reduced several tests to try to stabilize the tests in all configs. * Stabilized a few tests that were sporadically failing on Release Any CPU config. Getting out of memory errors so reduced the size of the KV Log. * RecoveryTestByAllocatorType was failing on Debug x64 due to the log size was not valid. So putting size back to what it was * Change Assert.IsTrue -> Assert.AreEqual (and other more-precise calls). Remove Console.WriteLine in tests and replace with Debug.WriteLine in LocalMemoryDevice. Fix RecoverTests Segment and Page SizeBits to match reduced LMD capacity argument * Restore RecoveryTests Segment and Page SizeBits per Darren's push * Updated a couple Device Faster Log Tests to use "using var" to make sure device and logs are cleaned up properly when the test is done. Co-authored-by: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * Update home.md * Pubsub-updates (#559) * [C#] Misc warning and code cleanup (#560) * Adding comments to structures that were made public (Phase, SystemState, DeltaLogEntryType * updates * misc minor cleanup Co-authored-by: TedHartMS <15467143+TedHartMS@users.noreply.github.com> * [C#] FasterLogIterator.CompleteUntilRecordAtAsync() (#558) * MemoryPool-based override for FasterLog.ReadAsync() * FasterLogIterator.CompleteUntilRecordAtAsync() added. Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C++] Linux liburing based I/O handler. (#387) * Add uring IO handler. * minor update. * Make uring an option. Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> * [C#] Misc remote cleanup (#569) * Misc remote cleanup * nit * [C#] Fixes to non-kv PubSub (#568) * Removed multiple enqueues for SubscribeBroker during Publish() and replaced with just one. Disadvantage is that there are multiple small allocations happening * Fixed nit that was causing valPtr to send wrong values when there were multiple subscribers * Fixed couple of nits. * Update 43-fasterlog-tuning.md * Update 43-fasterlog-tuning.md * Update 23-fasterkv-tuning.md * Fix pub sub bug from recent PR * Make provider extensible. (#575) * Update VarLenServer.cs * Remove ReusableObject in FASTER remote (#573) * Fix pub-sub (#580) * Create 51-remote-pubsub.md * Update navigation.yml * Update 51-remote-pubsub.md * Update SerializedFasterWrapper.cs * remove stale files * LibDPR updates for memory management and subscriptions Co-authored-by: Badrish Chandramouli <badrishc@microsoft.com> Co-authored-by: Jeffrey <61218880+plan-do-break-fix@users.noreply.github.com> Co-authored-by: hiteshmadan <hitesh.madan@microsoft.com> Co-authored-by: Tim Coleman <timothy.coleman@gmail.com> Co-authored-by: David Nepožitek <david@nepozitek.cz> Co-authored-by: TedHartMS <15467143+TedHartMS@users.noreply.github.com> Co-authored-by: Tianyu Li <t-litianyu@microsoft.com> Co-authored-by: rohankadekodi-msr <69916400+rohankadekodi-msr@users.noreply.github.com> Co-authored-by: 4nonym0us <4nonym0us@xakep.ru> Co-authored-by: darrenge <darrenge@microsoft.com> Co-authored-by: Wentao Cai <wcai6@cs.rochester.edu> Co-authored-by: Konstantinos Kanellis <kkanellis@cs.wisc.edu> Co-authored-by: Masashi Ito <59102452+mito-csod@users.noreply.github.com> Co-authored-by: Wentao Cai <chriscwt@outlook.com> Co-authored-by: Dong Xie <xiedong1993@gmail.com>
- Loading branch information