Skip to content

Commit

Permalink
Support for Diskless Replication (#997)
Browse files Browse the repository at this point in the history
* expose diskless replication parameters

* refactor/cleanup legacy ReplicaSyncSession

* add interface to support diskless replication session and aof tasks

* core diskless replication implementation

* expose diskless replication API

* adding test for diskless replication

* update gcs extension to clearly mark logging progress

* fix gcs dispose on diskless attach, call dispose of replicationSyncManager, add more logging

* complete first diskless replication test

* fix iterator check for null when empty store

* fix iterator for object store cluster sync

* add simple diskless sync test

* cleanup code

* replica fall behind test

* wip

* register cts at wait for sync completion

* add db version alignment test

* avoid using close lock for leader based syncing

* truncate AOF after streaming checkpoint is taken

* add tests for failover with diskless replication

* fix formatting and conversion to IPEndpoint

* fix RepCommandsTests

* dispose aofSyncTask if failed to add to AofSyncTaskStore

* overload dispose ReplicaSyncSession

* explicitly dispose gcs used for full sync at replicaSyncSession sync

* dispose gcs once on return

* code cleanup

* update tests to provide more context logging

* add more comprehensive logging of syncMetadata

* add timeout for streaming checkpoint

* add clusterTimeout for diskless repl tests

* some more logging

* cleanup and refactor code

* truncate AOF only when main-memory-replication is switched on

* adding logging for cancellation when streaming

* split checkpoint commit marker to allow for disk checkpoints

* update sync metadata log message

* add progress based timeout implementation

* deprecate main-memory-replication
  • Loading branch information
vazois authored Feb 24, 2025
1 parent 261e1c6 commit 14af7b8
Show file tree
Hide file tree
Showing 44 changed files with 2,697 additions and 282 deletions.
2 changes: 1 addition & 1 deletion benchmark/BDN.benchmark/Operations/OperationsBase.cs
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ public virtual void GlobalSetup()
{
opts.EnableAOF = true;
opts.UseAofNullDevice = true;
opts.MainMemoryReplication = true;
opts.FastAofTruncate = true;
opts.CommitFrequencyMs = -1;
opts.AofPageSize = "128m";
opts.AofMemorySize = "256m";
Expand Down
13 changes: 10 additions & 3 deletions libs/client/ClientSession/GarnetClientSessionIncremental.cs
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,15 @@

namespace Garnet.client
{
enum IncrementalSendType : byte
{
MIGRATE,
SYNC
}

public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageConsumer
{
IncrementalSendType ist;
bool isMainStore;
byte* curr, head;
int keyValuePairCount;
Expand Down Expand Up @@ -183,9 +190,9 @@ private void TrackIterationProgress(int keyCount, int size, bool completed = fal
var duration = TimeSpan.FromTicks(Stopwatch.GetTimestamp() - lastLog);
if (completed || lastLog == 0 || duration >= iterationProgressFreq)
{
logger?.LogTrace("[{op}]: isMainStore:({storeType}) totalKeyCount:({totalKeyCount}), totalPayloadSize:({totalPayloadSize} KB)",
completed ? "COMPLETED" : "MIGRATING",
isMainStore,
logger?.LogTrace("[{op}]: store:({storeType}) totalKeyCount:({totalKeyCount}), totalPayloadSize:({totalPayloadSize} KB)",
completed ? "COMPLETED" : ist,
isMainStore ? "MAIN STORE" : "OBJECT STORE",
totalKeyCount.ToString("N0"),
((long)((double)totalPayloadSize / 1024)).ToString("N0"));
lastLog = Stopwatch.GetTimestamp();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,7 @@ public void SetClusterMigrateHeader(string sourceNodeId, bool replace, bool isMa
tcsQueue.Enqueue(currTcsIterationTask);
curr = offset;
this.isMainStore = isMainStore;
this.ist = IncrementalSendType.MIGRATE;
var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE;
var replaceOption = replace ? T : F;

Expand Down
115 changes: 115 additions & 0 deletions libs/client/ClientSession/GarnetClientSessionReplicationExtensions.cs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ public sealed unsafe partial class GarnetClientSession : IServerHook, IMessageCo
static ReadOnlySpan<byte> send_ckpt_metadata => "SEND_CKPT_METADATA"u8;
static ReadOnlySpan<byte> send_ckpt_file_segment => "SEND_CKPT_FILE_SEGMENT"u8;
static ReadOnlySpan<byte> begin_replica_recover => "BEGIN_REPLICA_RECOVER"u8;
static ReadOnlySpan<byte> attach_sync => "ATTACH_SYNC"u8;
static ReadOnlySpan<byte> sync => "SYNC"u8;

/// <summary>
/// Initiate checkpoint retrieval from replica by sending replica checkpoint information and AOF address range
Expand Down Expand Up @@ -352,5 +354,118 @@ public Task<string> ExecuteBeginReplicaRecover(bool sendStoreCheckpoint, bool se
Interlocked.Increment(ref numCommands);
return tcs.Task;
}

/// <summary>
/// Initiate attach from replica
/// </summary>
/// <param name="syncMetadata"></param>
/// <returns></returns>
public Task<string> ExecuteAttachSync(byte[] syncMetadata)
{
var tcs = new TaskCompletionSource<string>(TaskCreationOptions.RunContinuationsAsynchronously);
tcsQueue.Enqueue(tcs);
byte* curr = offset;
int arraySize = 3;

while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

//1
while (!RespWriteUtils.TryWriteDirect(CLUSTER, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

//2
while (!RespWriteUtils.TryWriteBulkString(attach_sync, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

//3
while (!RespWriteUtils.TryWriteBulkString(syncMetadata, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

Flush();
Interlocked.Increment(ref numCommands);
return tcs.Task;
}

/// <summary>
/// Set CLUSTER SYNC header info
/// </summary>
/// <param name="sourceNodeId"></param>
/// <param name="isMainStore"></param>
public void SetClusterSyncHeader(string sourceNodeId, bool isMainStore)
{
currTcsIterationTask = new TaskCompletionSource<string>(TaskCreationOptions.RunContinuationsAsynchronously);
tcsQueue.Enqueue(currTcsIterationTask);
curr = offset;
this.isMainStore = isMainStore;
this.ist = IncrementalSendType.SYNC;
var storeType = isMainStore ? MAIN_STORE : OBJECT_STORE;

var arraySize = 5;
while (!RespWriteUtils.TryWriteArrayLength(arraySize, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

// 1
while (!RespWriteUtils.TryWriteDirect(CLUSTER, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

// 2
while (!RespWriteUtils.TryWriteBulkString(sync, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

// 3
while (!RespWriteUtils.TryWriteAsciiBulkString(sourceNodeId, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

// 4
while (!RespWriteUtils.TryWriteBulkString(storeType, ref curr, end))
{
Flush();
curr = offset;
}
offset = curr;

// 5
// Reserve space for the bulk string header + final newline
while (ExtraSpace + 2 > (int)(end - curr))
{
Flush();
curr = offset;
}
head = curr;
curr += ExtraSpace;
}
}
}
2 changes: 1 addition & 1 deletion libs/cluster/Server/ClusterProvider.cs
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ public void SafeTruncateAOF(StoreType storeType, bool full, long CheckpointCover
_ = replicationManager.SafeTruncateAof(CheckpointCoveredAofAddress);
else
{
if (serverOptions.MainMemoryReplication)
if (serverOptions.FastAofTruncate)
storeWrapper.appendOnlyFile?.UnsafeShiftBeginAddress(CheckpointCoveredAofAddress, truncateLog: true);
else
{
Expand Down
1 change: 1 addition & 0 deletions libs/cluster/Server/Replication/CheckpointEntry.cs
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ public byte[] ToByteArray()

public static CheckpointEntry FromByteArray(byte[] serialized)
{
if (serialized.Length == 0) return null;
var ms = new MemoryStream(serialized);
var reader = new BinaryReader(ms);
var cEntry = new CheckpointEntry
Expand Down
21 changes: 1 addition & 20 deletions libs/cluster/Server/Replication/PrimaryOps/AofSyncTaskInfo.cs
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@
using System.Threading;
using System.Threading.Tasks;
using Garnet.client;
using Garnet.common;
using Microsoft.Extensions.Logging;
using Tsavorite.core;

Expand All @@ -24,11 +23,6 @@ internal sealed class AofSyncTaskInfo : IBulkLogEntryConsumer, IDisposable
readonly long startAddress;
public long previousAddress;

/// <summary>
/// Used to mark if syncing is in progress
/// </summary>
SingleWriterMultiReaderLock aofSyncInProgress;

/// <summary>
/// Check if client connection is healthy
/// </summary>
Expand Down Expand Up @@ -69,11 +63,6 @@ public void Dispose()

// Finally, dispose the cts
cts?.Dispose();

// Dispose only if AOF sync has not started
// otherwise sync task will dispose the client
if (aofSyncInProgress.TryWriteLock())
garnetClient?.Dispose();
}

public unsafe void Consume(byte* payloadPtr, int payloadLength, long currentAddress, long nextAddress, bool isProtected)
Expand Down Expand Up @@ -108,16 +97,8 @@ public async Task ReplicaSyncTask()
{
logger?.LogInformation("Starting ReplicationManager.ReplicaSyncTask for remote node {remoteNodeId} starting from address {address}", remoteNodeId, startAddress);

var failedToStart = false;
try
{
if (!aofSyncInProgress.TryWriteLock())
{
logger?.LogWarning("{method} AOF sync for {remoteNodeId} failed to start", nameof(ReplicaSyncTask), remoteNodeId);
failedToStart = true;
return;
}

if (!IsConnected) garnetClient.Connect();

iter = clusterProvider.storeWrapper.appendOnlyFile.ScanSingle(startAddress, long.MaxValue, scanUncommitted: true, recover: false, logger: logger);
Expand All @@ -134,7 +115,7 @@ public async Task ReplicaSyncTask()
}
finally
{
if (!failedToStart) garnetClient.Dispose();
garnetClient.Dispose();
var (address, port) = clusterProvider.clusterManager.CurrentConfig.GetWorkerAddressFromNodeId(remoteNodeId);
logger?.LogWarning("AofSync task terminated; client disposed {remoteNodeId} {address} {port} {currentAddress}", remoteNodeId, address, port, previousAddress);

Expand Down
114 changes: 111 additions & 3 deletions libs/cluster/Server/Replication/PrimaryOps/AofTaskStore.cs
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public AofTaskStore(ClusterProvider clusterProvider, int initialSize = 1, ILogge
logPageSizeBits = clusterProvider.storeWrapper.appendOnlyFile.UnsafeGetLogPageSizeBits();
int logPageSize = 1 << logPageSizeBits;
logPageSizeMask = logPageSize - 1;
if (clusterProvider.serverOptions.MainMemoryReplication)
if (clusterProvider.serverOptions.FastAofTruncate)
clusterProvider.storeWrapper.appendOnlyFile.SafeTailShiftCallback = SafeTailShiftCallback;
TruncateLagAddress = clusterProvider.storeWrapper.appendOnlyFile.UnsafeGetReadOnlyAddressLagOffset() - 2 * logPageSize;
}
Expand Down Expand Up @@ -162,7 +162,7 @@ public bool TryAddReplicationTask(string remoteNodeId, long startAddress, out Ao

// Possible AOF data loss: { using null AOF device } OR { main memory replication AND no on-demand checkpoints }
bool possibleAofDataLoss = clusterProvider.serverOptions.UseAofNullDevice ||
(clusterProvider.serverOptions.MainMemoryReplication && !clusterProvider.serverOptions.OnDemandCheckpoint);
(clusterProvider.serverOptions.FastAofTruncate && !clusterProvider.serverOptions.OnDemandCheckpoint);

// Fail adding the task if truncation has happened, and we are not in possibleAofDataLoss mode
if (startAddress < TruncatedUntil && !possibleAofDataLoss)
Expand Down Expand Up @@ -217,6 +217,114 @@ public bool TryAddReplicationTask(string remoteNodeId, long startAddress, out Ao
return success;
}

public bool TryAddReplicationTasks(ReplicaSyncSession[] replicaSyncSessions, long startAddress)
{
var current = clusterProvider.clusterManager.CurrentConfig;
var success = true;
if (startAddress == 0) startAddress = ReplicationManager.kFirstValidAofAddress;

// First iterate through all sync sessions and add an AOF sync task
// All tasks will be
foreach (var rss in replicaSyncSessions)
{
if (rss == null) continue;
var replicaNodeId = rss.replicaSyncMetadata.originNodeId;
var (address, port) = current.GetWorkerAddressFromNodeId(replicaNodeId);

try
{
rss.AddAofSyncTask(new AofSyncTaskInfo(
clusterProvider,
this,
current.LocalNodeId,
replicaNodeId,
new GarnetClientSession(
new IPEndPoint(IPAddress.Parse(address), port),
clusterProvider.replicationManager.GetAofSyncNetworkBufferSettings,
clusterProvider.replicationManager.GetNetworkPool,
tlsOptions: clusterProvider.serverOptions.TlsOptions?.TlsClientOptions,
authUsername: clusterProvider.ClusterUsername,
authPassword: clusterProvider.ClusterPassword,
logger: logger),
startAddress,
logger));
}
catch (Exception ex)
{
logger?.LogWarning(ex, "{method} creating AOF sync task for {replicaNodeId} failed", nameof(TryAddReplicationTasks), replicaNodeId);
return false;
}
}

_lock.WriteLock();
try
{
if (_disposed) return false;

// Fail adding the task if truncation has happened
if (startAddress < TruncatedUntil)
{
logger?.LogWarning("{method} failed to add tasks for AOF sync {startAddress} {truncatedUntil}", nameof(TryAddReplicationTasks), startAddress, TruncatedUntil);
return false;
}

foreach (var rss in replicaSyncSessions)
{
if (rss == null) continue;

var added = false;
// Find if AOF sync task already exists
for (var i = 0; i < numTasks; i++)
{
var t = tasks[i];
Debug.Assert(t != null);
if (t.remoteNodeId == rss.replicaNodeId)
{
tasks[i] = rss.AofSyncTask;
t.Dispose();
added = true;
break;
}
}

if (added) continue;

// If AOF sync task did not exist and was not added we added below
// Check if array can hold a new AOF sync task
if (numTasks == tasks.Length)
{
var old_tasks = tasks;
var _tasks = new AofSyncTaskInfo[tasks.Length * 2];
Array.Copy(tasks, _tasks, tasks.Length);
tasks = _tasks;
Array.Clear(old_tasks);
}
// Add new AOF sync task
tasks[numTasks++] = rss.AofSyncTask;
}

success = true;
}
finally
{
_lock.WriteUnlock();

if (!success)
{
foreach (var rss in replicaSyncSessions)
{
if (rss == null) continue;
if (rss.AofSyncTask != null)
{
rss.AofSyncTask.Dispose();
}
}
}
}

return true;
}

public bool TryRemove(AofSyncTaskInfo aofSyncTask)
{
// Lock addition of new tasks
Expand Down Expand Up @@ -288,7 +396,7 @@ public long SafeTruncateAof(long CheckpointCoveredAofAddress = long.MaxValue)

if (TruncatedUntil > 0 && TruncatedUntil < long.MaxValue)
{
if (clusterProvider.serverOptions.MainMemoryReplication)
if (clusterProvider.serverOptions.FastAofTruncate)
{
clusterProvider.storeWrapper.appendOnlyFile?.UnsafeShiftBeginAddress(TruncatedUntil, snapToPageStart: true, truncateLog: true);
}
Expand Down
Loading

32 comments on commit 14af7b8

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Network.BasicOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Network.BasicOperations.InlinePing(Params: None) 94.60758505548749 ns (± 0.42591495054418593) 92.97828890482585 ns (± 0.5322437508046484) 1.02

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaRunnerOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Managed,Limit) 2709.983870967742 ns (± 461.35079581837084) 3060.257894736842 ns (± 469.2105800395519) 0.89
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Managed,Limit) 2679.912087912088 ns (± 412.37878079777437) 3009.0425531914893 ns (± 382.2206736049812) 0.89
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Managed,Limit) 248486.70833333334 ns (± 20914.88040551973) 271668.4747474748 ns (± 32987.11622874363) 0.91
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Managed,Limit) 255062.91666666666 ns (± 26686.78166893891) 260456.71649484537 ns (± 25013.040439866396) 0.98
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Managed,Limit) 17034.4375 ns (± 287.1702093996985) 19066.295454545456 ns (± 2500.0989826799846) 0.89
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Managed,Limit) 141102.59375 ns (± 14009.675688479787) 156047.96 ns (± 25120.67048537414) 0.90
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Managed,None) 2727.3020833333335 ns (± 450.0710147425663) 3161.282608695652 ns (± 535.5871652330279) 0.86
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Managed,None) 2686.5666666666666 ns (± 90.30291297601659) 3091.3655913978496 ns (± 359.0540344749579) 0.87
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Managed,None) 266457.4591836735 ns (± 34890.95673996738) 256091.94791666666 ns (± 26875.326577156935) 1.04
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Managed,None) 259351.42268041236 ns (± 30844.56347992944) 263201.77 ns (± 31072.63914266054) 0.99
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Managed,None) 18548.761363636364 ns (± 2273.738263196658) 19457.928571428572 ns (± 257.50510503848096) 0.95
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Managed,None) 145700.92857142858 ns (± 13044.22330670775) 146342.51020408163 ns (± 14148.99496211795) 1.00
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Native,None) 2763.021276595745 ns (± 465.33546642893253) 3084.537634408602 ns (± 377.41784476428387) 0.90
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Native,None) 2807.2659574468084 ns (± 425.5808472409452) 3144.074468085106 ns (± 439.76367650030954) 0.89
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Native,None) 221948.53333333333 ns (± 4155.898490900213) 230818.625 ns (± 5877.105471473698) 0.96
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Native,None) 224574.43333333332 ns (± 4017.94475654739) 217703.38461538462 ns (± 1152.8769476445684) 1.03
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Native,None) 18036.120879120877 ns (± 1977.6926614121974) 17714.83695652174 ns (± 2374.509329428352) 1.02
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Native,None) 139044.18 ns (± 12755.493317996545) 144107.38775510204 ns (± 16358.285088724073) 0.96
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Tracked,Limit) 2716 ns (± 70.57194910160835) 2805.653846153846 ns (± 84.72824431448691) 0.97
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Tracked,Limit) 2814.85 ns (± 71.28687411475293) 2865.574468085106 ns (± 520.1681797794732) 0.98
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Tracked,Limit) 276576.0714285714 ns (± 3165.2659498005905) 284396.63636363635 ns (± 10660.550055817965) 0.97
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Tracked,Limit) 287349 ns (± 10374.533483487341) 288784.5084745763 ns (± 12769.33447942142) 1.00
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Tracked,Limit) 18178.571428571428 ns (± 381.0678882426797) 22190.945652173912 ns (± 3393.6544708239717) 0.82
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Tracked,Limit) 150302.16666666666 ns (± 15848.114787132328) 153151.72448979592 ns (± 16214.997931328953) 0.98
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Tracked,None) 2815.9315789473685 ns (± 475.7102115523063) 3118.09375 ns (± 429.6516355846051) 0.90
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Tracked,None) 3183.5157894736844 ns (± 281.8481129803667) 3035.7849462365593 ns (± 386.7342321189839) 1.05
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Tracked,None) 279814.5714285714 ns (± 4964.293081819183) 282042.7 ns (± 6411.175950101347) 0.99
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Tracked,None) 280838.1904761905 ns (± 10224.438505038881) 285220.21153846156 ns (± 11657.227808105163) 0.98
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Tracked,None) 17785.083333333332 ns (± 219.28249042288036) 20556.516129032258 ns (± 2788.7160151804596) 0.87
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Tracked,None) 151195.94 ns (± 15746.316992063426) 151638.94949494948 ns (± 15911.841913142924) 1.00

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaScriptCacheOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Managed,Limit) 1085.75 ns (± 314.9183611284426) 886.531914893617 ns (± 504.79657250782566) 1.22
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Managed,Limit) 932.3350515463917 ns (± 311.68305484138483) 861.804347826087 ns (± 291.2074612289221) 1.08
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Managed,Limit) 1609.8804347826087 ns (± 502.5988250619525) 2092.963157894737 ns (± 535.2372870189945) 0.77
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Managed,Limit) 222938.79347826086 ns (± 20110.17626850363) 248479.1 ns (± 4376.705476562414) 0.90
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Managed,Limit) 1907.6237113402062 ns (± 417.68821250954136) 1958.28125 ns (± 470.59541687268796) 0.97
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Managed,Limit) 7711.071428571428 ns (± 94.39989756001147) 9873.545977011494 ns (± 1648.8929048955918) 0.78
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Managed,None) 1104.6011235955057 ns (± 377.5401256502359) 1318.8020833333333 ns (± 504.2767331283106) 0.84
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Managed,None) 879.8571428571429 ns (± 36.57130456328182) 991.1684210526316 ns (± 313.0876314222714) 0.89
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Managed,None) 1724.3 ns (± 65.37275083491696) 2010.6145833333333 ns (± 586.576120062173) 0.86
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Managed,None) 232878.4945054945 ns (± 21685.02873841952) 227413.83333333334 ns (± 21331.710898940317) 1.02
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Managed,None) 1877.7916666666667 ns (± 409.72691590135395) 2055.1170212765956 ns (± 423.4549869156343) 0.91
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Managed,None) 7675.846153846154 ns (± 57.73480486073969) 9081.893617021276 ns (± 1391.4979278405792) 0.85
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Native,None) 1207.5567010309278 ns (± 357.7713122312493) 1288.4791666666667 ns (± 514.7257006696135) 0.94
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Native,None) 914.2268041237113 ns (± 317.82547310967425) 877.5729166666666 ns (± 322.3553692757067) 1.04
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Native,None) 1535.603448275862 ns (± 55.16555914931124) 1679.1521739130435 ns (± 505.3012143741289) 0.91
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Native,None) 212448.6923076923 ns (± 3455.16686487101) 215885.58181818182 ns (± 8934.90765707201) 0.98
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Native,None) 1952.4166666666667 ns (± 473.8497133872647) 2018.076923076923 ns (± 425.83278488599336) 0.97
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Native,None) 8571.712765957447 ns (± 864.9123222683452) 9140.29381443299 ns (± 1416.8393958777897) 0.94
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Tracked,Limit) 954.2247191011236 ns (± 290.25920049661056) 1020.2934782608696 ns (± 309.69228038236287) 0.94
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Tracked,Limit) 823.7065217391304 ns (± 250.29360947896873) 858.6595744680851 ns (± 296.97979921857143) 0.96
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Tracked,Limit) 1733.8105263157895 ns (± 365.77371491991573) 1892.2021276595744 ns (± 428.17164631150166) 0.92
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Tracked,Limit) 256435.12162162163 ns (± 12752.855574880972) 245167.61538461538 ns (± 3513.08744882289) 1.05
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Tracked,Limit) 1775.09375 ns (± 457.4760523643201) 1905.3052631578948 ns (± 366.7056641517018) 0.93
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Tracked,Limit) 7947.714285714285 ns (± 130.31230449691503) 9221.278350515464 ns (± 1255.3560479523662) 0.86
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Tracked,None) 1134.388888888889 ns (± 289.268612559736) 1048.9888888888888 ns (± 331.2024820933849) 1.08
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Tracked,None) 781.9421052631579 ns (± 350.07360064400905) 778.1703296703297 ns (± 285.42537670194224) 1.00
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Tracked,None) 1623.680412371134 ns (± 680.0031027256382) 1674.0842696629213 ns (± 258.2243918918707) 0.97
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Tracked,None) 247511.88095238095 ns (± 11352.896344145915) 253862.94545454546 ns (± 10820.21601656445) 0.97
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Tracked,None) 2070.1736842105265 ns (± 371.6284876200294) 1860.3541666666667 ns (± 540.5385224923805) 1.11
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Tracked,None) 9346.112244897959 ns (± 1050.9561467172375) 7764.833333333333 ns (± 169.6804089353588) 1.20

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.PubSubOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.PubSubOperations.Publish(Params: ACL) 19327.177693684895 ns (± 16.120885256963007) 19017.233166034406 ns (± 21.940741778446) 1.02
BDN.benchmark.Operations.PubSubOperations.Publish(Params: AOF) 19504.460222880047 ns (± 23.183563883860206) 19360.0112101237 ns (± 79.15987125715796) 1.01
BDN.benchmark.Operations.PubSubOperations.Publish(Params: None) 19790.303245035808 ns (± 95.05426714476094) 20781.735355922156 ns (± 60.529104391297) 0.95

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.BasicOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: ACL) 1931.9371323218713 ns (± 8.782669153500978) 1900.1890950884138 ns (± 3.5337793081127713) 1.02
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: AOF) 1841.1817297617595 ns (± 9.10652795594694) 1846.6031970342 ns (± 11.45795242667152) 1.00
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: None) 1815.3778738657634 ns (± 8.985779320434924) 1873.8414499282837 ns (± 8.95602272033183) 0.97

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cluster.ClusterMigrate (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Cluster.ClusterMigrate.Get(Params: None) 38612.18441975911 ns (± 352.312593862097) 43218.46880086263 ns (± 2792.1291531026645) 0.89
BDN.benchmark.Cluster.ClusterMigrate.Set(Params: None) 39386.333697979266 ns (± 140.94379920190374) 45109.846236572266 ns (± 3001.5823305560566) 0.87
BDN.benchmark.Cluster.ClusterMigrate.MGet(Params: None) 32820.74631245931 ns (± 34.720554729936104) 37089.2762689209 ns (± 2429.6111231304926) 0.88
BDN.benchmark.Cluster.ClusterMigrate.MSet(Params: None) 33247.75865071615 ns (± 200.93129137005528) 37012.85046325684 ns (± 3069.9291271061966) 0.90

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ObjectOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: ACL) 139998.4036295573 ns (± 974.9297673996213) 140673.11611328126 ns (± 937.0184202918684) 1.00
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: ACL) 135414.5880533854 ns (± 1158.7825250195106) 132424.41988699776 ns (± 268.60469414121684) 1.02
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: AOF) 154187.52211100262 ns (± 1287.2821460509679) 152132.01675180288 ns (± 641.278457426869) 1.01
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: AOF) 156778.91137695312 ns (± 690.3306174257615) 152654.14934430804 ns (± 1126.8143695917133) 1.03
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: None) 149172.30725911458 ns (± 978.6948858451334) 142678.63246372767 ns (± 875.0810752680096) 1.05
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: None) 132184.76444185697 ns (± 751.1748555470659) 133091.3905203683 ns (± 791.092573278874) 0.99

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cluster.ClusterOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Cluster.ClusterOperations.Get(Params: DSV) 16859.57144470215 ns (± 150.78365954720005) 16912.78530883789 ns (± 176.26286326974224) 1.00
BDN.benchmark.Cluster.ClusterOperations.Set(Params: DSV) 15816.514630831205 ns (± 20.150199296615444) 16370.96342569987 ns (± 186.38508074819984) 0.97
BDN.benchmark.Cluster.ClusterOperations.MGet(Params: DSV) 15907.820659092495 ns (± 14.370168836017218) 15740.238656850961 ns (± 17.21921944192566) 1.01
BDN.benchmark.Cluster.ClusterOperations.MSet(Params: DSV) 15466.919288048377 ns (± 10.78400027549178) 15223.112363179525 ns (± 44.31569686369959) 1.02
BDN.benchmark.Cluster.ClusterOperations.CTXNSET(Params: DSV) 124364.25520207331 ns (± 606.4547048828746) 121042.57911376953 ns (± 571.8258961218262) 1.03
BDN.benchmark.Cluster.ClusterOperations.Get(Params: None) 21195.269842965263 ns (± 75.79038047098292) 21616.31068318685 ns (± 126.86620496507554) 0.98
BDN.benchmark.Cluster.ClusterOperations.Set(Params: None) 21656.661826578777 ns (± 84.11297548188288) 21496.637844379133 ns (± 64.57440089980507) 1.01
BDN.benchmark.Cluster.ClusterOperations.MGet(Params: None) 16334.547953679012 ns (± 43.53554038924663) 15968.63312886556 ns (± 124.24399728320486) 1.02
BDN.benchmark.Cluster.ClusterOperations.MSet(Params: None) 16427.235959879556 ns (± 124.79124852239063) 15052.195974731445 ns (± 53.33680040204398) 1.09
BDN.benchmark.Cluster.ClusterOperations.CTXNSET(Params: None) 135005.32489885602 ns (± 767.457357969307) 134835.08000837054 ns (± 245.6686418961634) 1.00

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.BasicOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: ACL) 1889.0437807355609 ns (± 3.0879726082281707) 1924.2553393046062 ns (± 2.7126758235337767) 0.98
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: AOF) 1907.5477306659404 ns (± 8.268217469825995) 1820.3830446515765 ns (± 1.903426328591534) 1.05
BDN.benchmark.Operations.BasicOperations.InlinePing(Params: None) 1768.8303067133977 ns (± 2.50098965397975) 1893.270411858192 ns (± 3.6077352237136937) 0.93

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Network.BasicOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Network.BasicOperations.InlinePing(Params: None) 95.07222811381023 ns (± 0.5904618549605442) 81.77427236850446 ns (± 0.21896125096549673) 1.16

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Network.RawStringOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Network.RawStringOperations.Set(Params: None) 250.8397556713649 ns (± 1.5555798218802888) 240.32972049713135 ns (± 0.36551074110394455) 1.04
BDN.benchmark.Network.RawStringOperations.SetEx(Params: None) 300.05136664708454 ns (± 2.476330670809765) 282.9578700432411 ns (± 0.5166908439103042) 1.06
BDN.benchmark.Network.RawStringOperations.SetNx(Params: None) 308.44620275497437 ns (± 0.6893429280587204) 317.42219568888345 ns (± 2.0565543299714597) 0.97
BDN.benchmark.Network.RawStringOperations.SetXx(Params: None) 356.31766120592755 ns (± 0.5339309706526538) 320.0486070192777 ns (± 0.7700413742126929) 1.11
BDN.benchmark.Network.RawStringOperations.GetFound(Params: None) 270.72165451049807 ns (± 1.742676448363964) 252.11740926901498 ns (± 0.5035635094782437) 1.07
BDN.benchmark.Network.RawStringOperations.GetNotFound(Params: None) 190.77153511047362 ns (± 1.2026881286115543) 196.69344277565295 ns (± 0.08141184137228497) 0.97
BDN.benchmark.Network.RawStringOperations.Increment(Params: None) 324.9700161388942 ns (± 1.9403296698052364) 319.1273557956402 ns (± 0.34186633419596135) 1.02
BDN.benchmark.Network.RawStringOperations.Decrement(Params: None) 315.56207510630287 ns (± 2.116921429764024) 312.8313677708308 ns (± 0.7404382106201406) 1.01
BDN.benchmark.Network.RawStringOperations.IncrementBy(Params: None) 371.495185477393 ns (± 2.3678925900767833) 386.59955978393555 ns (± 0.4922395887653747) 0.96
BDN.benchmark.Network.RawStringOperations.DecrementBy(Params: None) 396.865709177653 ns (± 6.007273685235526) 381.86983194351194 ns (± 1.7524213101239328) 1.04

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.PubSubOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.PubSubOperations.Publish(Params: ACL) 17499.244435628254 ns (± 8.629494258265852) 17041.30391438802 ns (± 19.02905724094813) 1.03
BDN.benchmark.Operations.PubSubOperations.Publish(Params: AOF) 17168.345743815105 ns (± 50.193821306362786) 16954.96782575335 ns (± 23.885557194761585) 1.01
BDN.benchmark.Operations.PubSubOperations.Publish(Params: None) 16781.03546142578 ns (± 19.427669119072323) 16965.526052621695 ns (± 27.006731980349024) 0.99

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cluster.ClusterMigrate (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Cluster.ClusterMigrate.Get(Params: None) 36661.94545200893 ns (± 42.499325214582235) 38708.81478445871 ns (± 44.94070144549113) 0.95
BDN.benchmark.Cluster.ClusterMigrate.Set(Params: None) 36172.364908854164 ns (± 56.94366275681521) 37121.252005440845 ns (± 43.00028868708288) 0.97
BDN.benchmark.Cluster.ClusterMigrate.MGet(Params: None) 31456.01588657924 ns (± 35.15967359396354) 30853.61328125 ns (± 22.956245052251468) 1.02
BDN.benchmark.Cluster.ClusterMigrate.MSet(Params: None) 30417.218017578125 ns (± 15.667790968242047) 30619.396754673548 ns (± 151.14768786598012) 0.99

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.CustomOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: ACL) 51265.80774739583 ns (± 318.5055139703423) 47657.05457560221 ns (± 270.97107334746323) 1.08
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: ACL) 204528.86842564173 ns (± 1175.935695742863) 194047.86978352864 ns (± 1343.5104093663708) 1.05
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: ACL) 124461.32297770183 ns (± 235.62281600050497) 119669.22535051618 ns (± 187.6242816764082) 1.04
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: ACL) 97934.63701520648 ns (± 394.695085038672) 101421.84675011269 ns (± 482.6548729184631) 0.97
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: AOF) 49703.05550443209 ns (± 121.22709489197787) 49242.62816772461 ns (± 249.96639905968718) 1.01
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: AOF) 209907.654296875 ns (± 1050.2464241742605) 210847.7935546875 ns (± 845.1229269643188) 1.00
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: AOF) 133028.17431640625 ns (± 341.95173522862024) 133276.5953938802 ns (± 1046.8546058255922) 1.00
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: AOF) 128994.4663273738 ns (± 855.4504893034958) 126441.87350027902 ns (± 848.6650064792126) 1.02
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: None) 49637.31779261998 ns (± 102.607917653203) 49117.02065604074 ns (± 58.53459690770762) 1.01
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: None) 202227.4731282552 ns (± 1088.7048791390014) 199440.55939592634 ns (± 1051.9900055965109) 1.01
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: None) 124438.93626185825 ns (± 549.7502092078365) 122843.49269321987 ns (± 561.5043193848696) 1.01
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: None) 100380.53513009207 ns (± 497.189953754234) 97772.80552164714 ns (± 112.19148147204382) 1.03

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ObjectOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: ACL) 103023.83524576823 ns (± 200.04990023317097) 102875.4638671875 ns (± 213.5695141329846) 1.00
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: ACL) 97123.10689290364 ns (± 275.4586269974868) 98088.0849984976 ns (± 218.7807284333337) 0.99
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: AOF) 119030.42887369792 ns (± 448.4900502019468) 124884.86409505208 ns (± 350.8213960704867) 0.95
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: AOF) 117972.08687918527 ns (± 283.19943651209195) 113757.0292154948 ns (± 461.3272576750575) 1.04
BDN.benchmark.Operations.ObjectOperations.LPushPop(Params: None) 106995.42323521206 ns (± 376.01976627630313) 105021.81193033855 ns (± 80.73969388285123) 1.02
BDN.benchmark.Operations.ObjectOperations.SAddRem(Params: None) 98767.68362862723 ns (± 246.4293829484133) 100090.69213867188 ns (± 269.68743751572305) 0.99

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaScripts (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaScripts.Script1(Params: Managed,Limit) 238.4590254465739 ns (± 2.766587291331141) 283.8064138339116 ns (± 0.5267492492068668) 0.84
BDN.benchmark.Lua.LuaScripts.Script2(Params: Managed,Limit) 316.0799381891886 ns (± 2.5863624916757755) 334.0253367424011 ns (± 1.2273328070054226) 0.95
BDN.benchmark.Lua.LuaScripts.Script3(Params: Managed,Limit) 514.6559695463914 ns (± 1.521071381545808) 509.32562189835767 ns (± 2.495755121430963) 1.01
BDN.benchmark.Lua.LuaScripts.Script4(Params: Managed,Limit) 597.791415141179 ns (± 1.5164341298625748) 640.2923631032307 ns (± 2.2168511513078166) 0.93
BDN.benchmark.Lua.LuaScripts.Script1(Params: Managed,None) 260.47359669208527 ns (± 0.5439522794569979) 264.9139686266581 ns (± 1.9088694588824768) 0.98
BDN.benchmark.Lua.LuaScripts.Script2(Params: Managed,None) 335.626297217149 ns (± 1.163020651707035) 343.511763159434 ns (± 1.7691653285683997) 0.98
BDN.benchmark.Lua.LuaScripts.Script3(Params: Managed,None) 511.5280879338582 ns (± 0.6082855061485603) 552.7177900901207 ns (± 1.044211137919808) 0.93
BDN.benchmark.Lua.LuaScripts.Script4(Params: Managed,None) 597.8243410927909 ns (± 2.210383045007075) 633.7959482192994 ns (± 3.089937292088197) 0.94
BDN.benchmark.Lua.LuaScripts.Script1(Params: Native,None) 258.493289402553 ns (± 1.4972917669491153) 257.1139198449942 ns (± 0.587190198512954) 1.01
BDN.benchmark.Lua.LuaScripts.Script2(Params: Native,None) 315.3663523013775 ns (± 1.092173532792076) 320.4873980840047 ns (± 1.2679391595716976) 0.98
BDN.benchmark.Lua.LuaScripts.Script3(Params: Native,None) 544.3070877620152 ns (± 1.8522097228492616) 542.008676147461 ns (± 2.0069058296723754) 1.00
BDN.benchmark.Lua.LuaScripts.Script4(Params: Native,None) 615.5218449320112 ns (± 2.1869854617989435) 601.8679358800252 ns (± 3.0246449587723596) 1.02
BDN.benchmark.Lua.LuaScripts.Script1(Params: Tracked,Limit) 255.8451145807902 ns (± 1.6275427875141695) 259.0289273628822 ns (± 0.5209380580680134) 0.99
BDN.benchmark.Lua.LuaScripts.Script2(Params: Tracked,Limit) 322.17745920817055 ns (± 1.2743848028401026) 315.3675382614136 ns (± 1.1080950110944754) 1.02
BDN.benchmark.Lua.LuaScripts.Script3(Params: Tracked,Limit) 508.31336103166853 ns (± 3.1862793679467276) 510.0061210223607 ns (± 2.1378611073935843) 1.00
BDN.benchmark.Lua.LuaScripts.Script4(Params: Tracked,Limit) 613.0122667040143 ns (± 4.696720708031145) 600.3428136189779 ns (± 2.3374263872571204) 1.02
BDN.benchmark.Lua.LuaScripts.Script1(Params: Tracked,None) 245.25093201228552 ns (± 1.3805355923720926) 259.8492515563965 ns (± 1.3374308385195106) 0.94
BDN.benchmark.Lua.LuaScripts.Script2(Params: Tracked,None) 319.62315007618497 ns (± 1.031120147402089) 322.38264439656183 ns (± 0.6318570440891772) 0.99
BDN.benchmark.Lua.LuaScripts.Script3(Params: Tracked,None) 524.6926504135132 ns (± 2.80305941691684) 509.52048778533936 ns (± 1.4663190842358123) 1.03
BDN.benchmark.Lua.LuaScripts.Script4(Params: Tracked,None) 599.4173570360456 ns (± 3.697814452658663) 616.8123643738883 ns (± 1.423761510892247) 0.97

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cluster.ClusterOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Cluster.ClusterOperations.Get(Params: DSV) 16237.917872837612 ns (± 29.532693719339115) 16049.291120256696 ns (± 29.949328395271706) 1.01
BDN.benchmark.Cluster.ClusterOperations.Set(Params: DSV) 15653.223571777344 ns (± 25.294504108528596) 16522.554485614484 ns (± 22.31765896712226) 0.95
BDN.benchmark.Cluster.ClusterOperations.MGet(Params: DSV) 14255.309549967447 ns (± 20.031032371043715) 14264.05283610026 ns (± 27.052698543692763) 1.00
BDN.benchmark.Cluster.ClusterOperations.MSet(Params: DSV) 13125.892421177456 ns (± 11.924355524390062) 13190.473937988281 ns (± 18.71656138628799) 1.00
BDN.benchmark.Cluster.ClusterOperations.CTXNSET(Params: DSV) 139391.48111979166 ns (± 184.75486575804047) 139862.93422154017 ns (± 178.4338537788385) 1.00
BDN.benchmark.Cluster.ClusterOperations.Get(Params: None) 20768.434651692707 ns (± 16.404587685878727) 20045.550944010418 ns (± 43.267117173427145) 1.04
BDN.benchmark.Cluster.ClusterOperations.Set(Params: None) 20954.78525797526 ns (± 44.10019804744671) 20537.909287672777 ns (± 51.24895364883696) 1.02
BDN.benchmark.Cluster.ClusterOperations.MGet(Params: None) 15272.008870442709 ns (± 34.017025653003586) 15717.379659016928 ns (± 55.24183540293814) 0.97
BDN.benchmark.Cluster.ClusterOperations.MSet(Params: None) 14381.316157749721 ns (± 27.703633245891584) 14613.43524639423 ns (± 22.080724131318494) 0.98
BDN.benchmark.Cluster.ClusterOperations.CTXNSET(Params: None) 150228.58712332588 ns (± 339.890075239921) 156488.9803059896 ns (± 260.55781228394454) 0.96

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Network.RawStringOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Network.RawStringOperations.Set(Params: None) 220.99114315850395 ns (± 0.16131316751615465) 224.91174856821695 ns (± 0.5017084130361684) 0.98
BDN.benchmark.Network.RawStringOperations.SetEx(Params: None) 283.51163546244305 ns (± 1.7615397021883892) 278.0590057373047 ns (± 2.222812440396708) 1.02
BDN.benchmark.Network.RawStringOperations.SetNx(Params: None) 300.8214167186192 ns (± 0.48052295409560625) 290.3515925774208 ns (± 0.6183131875838421) 1.04
BDN.benchmark.Network.RawStringOperations.SetXx(Params: None) 303.96596023014615 ns (± 1.2075498840123395) 299.27630106608075 ns (± 0.5253712135096735) 1.02
BDN.benchmark.Network.RawStringOperations.GetFound(Params: None) 225.2001702785492 ns (± 0.40529998634249864) 258.24513753255206 ns (± 2.173154015473756) 0.87
BDN.benchmark.Network.RawStringOperations.GetNotFound(Params: None) 178.0908989906311 ns (± 0.2535040877371491) 174.847731590271 ns (± 0.37856057058587783) 1.02
BDN.benchmark.Network.RawStringOperations.Increment(Params: None) 366.1452054977417 ns (± 0.7807219872804915) 323.1578145708357 ns (± 0.7236641605006094) 1.13
BDN.benchmark.Network.RawStringOperations.Decrement(Params: None) 290.0456530707223 ns (± 0.2770438543221674) 314.69642932598407 ns (± 0.5896129914015115) 0.92
BDN.benchmark.Network.RawStringOperations.IncrementBy(Params: None) 361.0435860497611 ns (± 0.709674865230225) 355.9737069266183 ns (± 0.6735644028941907) 1.01
BDN.benchmark.Network.RawStringOperations.DecrementBy(Params: None) 356.453013420105 ns (± 0.540573001881255) 360.4961967468262 ns (± 0.5131212359961835) 0.99

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaRunnerOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Managed,Limit) 2665.5913978494623 ns (± 1461.2007140984167) 7232.291666666667 ns (± 1454.2492938873754) 0.37
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Managed,Limit) 4224.226804123711 ns (± 2257.7310859261756) 7321.5053763440865 ns (± 2198.4109608278823) 0.58
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Managed,Limit) 238502.0618556701 ns (± 44499.864183738064) 275697.8947368421 ns (± 54410.287237091885) 0.87
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Managed,Limit) 238254.16666666666 ns (± 49260.51032209253) 289609.375 ns (± 55046.78981629126) 0.82
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Managed,Limit) 18098.947368421053 ns (± 5150.531778717944) 36496.875 ns (± 10210.721435669573) 0.50
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Managed,Limit) 116476.04166666667 ns (± 23864.683198539784) 144036 ns (± 26141.717578992357) 0.81
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Managed,None) 2608.1632653061224 ns (± 786.0475929805172) 7196.7032967032965 ns (± 2411.705277251419) 0.36
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Managed,None) 2570.2127659574467 ns (± 865.6627502986556) 7630.6122448979595 ns (± 1796.7558487603985) 0.34
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Managed,None) 208464.83516483515 ns (± 32087.731728380662) 256490.625 ns (± 39634.07645508305) 0.81
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Managed,None) 218477.31958762885 ns (± 39253.968762074015) 269178.94736842107 ns (± 41670.176321019564) 0.81
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Managed,None) 17607.291666666668 ns (± 5839.5429668257) 38529.166666666664 ns (± 8265.819495357044) 0.46
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Managed,None) 120269.19191919192 ns (± 23389.020085604956) 141068.68686868687 ns (± 24265.41860539038) 0.85
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Native,None) 2763.953488372093 ns (± 763.2612367061992) 7203.191489361702 ns (± 1620.0658507563837) 0.38
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Native,None) 2779.120879120879 ns (± 775.5308552521346) 7872.164948453608 ns (± 1621.2612289537392) 0.35
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Native,None) 250026 ns (± 51837.76288195759) 285217.032967033 ns (± 38742.351374016915) 0.88
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Native,None) 228186.36363636365 ns (± 22862.72970608224) 292583.5164835165 ns (± 37583.12938225893) 0.78
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Native,None) 18275.555555555555 ns (± 3394.155952767987) 39795.69892473118 ns (± 6005.0506636985) 0.46
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Native,None) 117357.14285714286 ns (± 21551.611511970204) 148471.13402061857 ns (± 21817.743919623495) 0.79
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Tracked,Limit) 2587.2340425531916 ns (± 737.3643914038454) 7422.222222222223 ns (± 1720.597477738329) 0.35
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Tracked,Limit) 2510.989010989011 ns (± 677.6512622368774) 7861.855670103093 ns (± 1900.1887679785632) 0.32
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Tracked,Limit) 265290.44943820225 ns (± 30092.443626798962) 341633.3333333333 ns (± 38614.292052614896) 0.78
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Tracked,Limit) 281449.4382022472 ns (± 36917.19888540344) 339144.2528735632 ns (± 38277.42778994405) 0.83
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Tracked,Limit) 27438.541666666668 ns (± 6896.930969514312) 48010.86956521739 ns (± 6815.390735966553) 0.57
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Tracked,Limit) 132064.2857142857 ns (± 24612.096807837206) 163897.91666666666 ns (± 27820.128945808345) 0.81
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersSmall(Params: Tracked,None) 3036.206896551724 ns (± 1079.9625762027113) 7527.956989247312 ns (± 1593.5580637296507) 0.40
BDN.benchmark.Lua.LuaRunnerOperations.ResetParametersLarge(Params: Tracked,None) 5669.791666666667 ns (± 1728.8645371223565) 8357.291666666666 ns (± 1939.926465869894) 0.68
BDN.benchmark.Lua.LuaRunnerOperations.ConstructSmall(Params: Tracked,None) 281385.8695652174 ns (± 43113.79672989049) 352690 ns (± 66741.28626966439) 0.80
BDN.benchmark.Lua.LuaRunnerOperations.ConstructLarge(Params: Tracked,None) 275293.6170212766 ns (± 37766.59660380514) 354843.75 ns (± 50669.163907179594) 0.78
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionSmall(Params: Tracked,None) 23282.608695652172 ns (± 4068.104526509228) 47859.375 ns (± 8340.927351257014) 0.49
BDN.benchmark.Lua.LuaRunnerOperations.CompileForSessionLarge(Params: Tracked,None) 124820.40816326531 ns (± 25779.0891011151) 159385.56701030929 ns (± 23587.699785811394) 0.78

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.CustomOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: ACL) 70460.8415876116 ns (± 58.585005593264974) 68137.54534040179 ns (± 79.74037462077636) 1.03
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: ACL) 221159.2830403646 ns (± 345.4099357979497) 225112.83569335938 ns (± 277.2809597400518) 0.98
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: ACL) 140681.3999720982 ns (± 191.09481041695025) 138838.19955679087 ns (± 64.72199882710014) 1.01
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: ACL) 126735.96516927083 ns (± 182.06595853383746) 123141.57057542067 ns (± 69.31349424510046) 1.03
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: AOF) 69191.58412388393 ns (± 52.807511417061875) 69469.8905436198 ns (± 172.89442702188728) 1.00
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: AOF) 235716.5974934896 ns (± 723.357764563156) 234646.4346078726 ns (± 569.6289880954378) 1.00
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: AOF) 147747.49755859375 ns (± 531.4773419860918) 149097.16515174278 ns (± 287.5473487153863) 0.99
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: AOF) 151927.97006460337 ns (± 409.7599973248081) 146775.244140625 ns (± 305.74225010015385) 1.04
BDN.benchmark.Operations.CustomOperations.CustomRawStringCommand(Params: None) 69205.73811848958 ns (± 58.50917078938697) 70117.3372708834 ns (± 110.88535177369957) 0.99
BDN.benchmark.Operations.CustomOperations.CustomObjectCommand(Params: None) 224128.58683268228 ns (± 360.151655853274) 227100.76206752233 ns (± 305.41348681526347) 0.99
BDN.benchmark.Operations.CustomOperations.CustomTransaction(Params: None) 140376.61865234375 ns (± 671.7634357583389) 137869.0608097957 ns (± 220.6031384127798) 1.02
BDN.benchmark.Operations.CustomOperations.CustomProcedure(Params: None) 123168.8982282366 ns (± 152.0827138433435) 123168.9923967634 ns (± 135.95852658532763) 1.00

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaScriptCacheOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Managed,Limit) 1457.471264367816 ns (± 1084.202866959317) 924.468085106383 ns (± 928.8316363302367) 1.58
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Managed,Limit) 937.3493975903615 ns (± 668.4001363601132) 1112.7659574468084 ns (± 978.9432576121781) 0.84
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Managed,Limit) 4852.525252525253 ns (± 3095.894036390762) 2142.8571428571427 ns (± 1860.5508551329851) 2.26
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Managed,Limit) 250842.42424242425 ns (± 48350.909264626476) 333848.97959183675 ns (± 95253.06634661464) 0.75
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Managed,Limit) 5680.10752688172 ns (± 2792.8785581342786) 2068.8172043010754 ns (± 1168.888220808249) 2.75
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Managed,Limit) 15098.979591836734 ns (± 2989.129614996532) 8864.21052631579 ns (± 3353.6869450531503) 1.70
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Managed,None) 2637.1134020618556 ns (± 2269.926558449857) 1176.595744680851 ns (± 910.5623374070981) 2.24
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Managed,None) 1851.6129032258063 ns (± 1558.4003018163412) 704.1666666666666 ns (± 715.9706942939217) 2.63
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Managed,None) 4925.252525252526 ns (± 3266.882163199853) 1663.265306122449 ns (± 1409.6418902628877) 2.96
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Managed,None) 265130 ns (± 53997.934865112846) 259128.7356321839 ns (± 44549.02684199545) 1.02
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Managed,None) 5687.878787878788 ns (± 3712.589522792619) 2433.3333333333335 ns (± 1612.002085804383) 2.34
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Managed,None) 16313.131313131313 ns (± 3999.2511152190623) 8029.67032967033 ns (± 2849.2297011841442) 2.03
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Native,None) 3027.3684210526317 ns (± 2565.1279303364495) 1167.7083333333333 ns (± 1238.462101637746) 2.59
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Native,None) 1628.5714285714287 ns (± 1359.5984467878516) 1089.8989898989898 ns (± 882.886609482243) 1.49
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Native,None) 3887.8947368421054 ns (± 2811.571036080976) 2169.7916666666665 ns (± 1321.4123060029901) 1.79
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Native,None) 261489.36170212767 ns (± 36531.65799888276) 314805.55555555556 ns (± 73958.64015458057) 0.83
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Native,None) 5665.979381443299 ns (± 3643.5984559091826) 2518.556701030928 ns (± 1592.9748047355731) 2.25
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Native,None) 16117.021276595744 ns (± 3097.9923536378114) 7192.391304347826 ns (± 2284.5988753000943) 2.24
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Tracked,Limit) 3173.7113402061855 ns (± 2602.2335436711974) 811.2244897959183 ns (± 985.7628427134387) 3.91
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Tracked,Limit) 2724.742268041237 ns (± 2124.511263788836) 579.5698924731183 ns (± 607.1242154590174) 4.70
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Tracked,Limit) 4964.285714285715 ns (± 3561.3981385167995) 1576.595744680851 ns (± 1302.472878128857) 3.15
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Tracked,Limit) 313696.9387755102 ns (± 54199.94760541012) 404856.1797752809 ns (± 63416.64707432215) 0.77
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Tracked,Limit) 5773.19587628866 ns (± 4003.5535117754425) 3537.373737373737 ns (± 3054.3574045637565) 1.63
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Tracked,Limit) 15245.876288659794 ns (± 4113.340023776633) 9069.58762886598 ns (± 3586.5094211752316) 1.68
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupHit(Params: Tracked,None) 2007.6923076923076 ns (± 1666.8153779808679) 1394.3298969072166 ns (± 1031.5561739596951) 1.44
BDN.benchmark.Lua.LuaScriptCacheOperations.LookupMiss(Params: Tracked,None) 2323.469387755102 ns (± 2215.074922304969) 834.7368421052631 ns (± 774.0152867054702) 2.78
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadOuterHit(Params: Tracked,None) 3995.8333333333335 ns (± 3072.798027742509) 1625.8064516129032 ns (± 1473.121738955749) 2.46
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadInnerHit(Params: Tracked,None) 320300 ns (± 45958.316501696434) 443920 ns (± 80032.54422315625) 0.72
BDN.benchmark.Lua.LuaScriptCacheOperations.LoadMiss(Params: Tracked,None) 5908.333333333333 ns (± 3763.4437383771037) 2218.6813186813188 ns (± 1482.7835751205002) 2.66
BDN.benchmark.Lua.LuaScriptCacheOperations.Digest(Params: Tracked,None) 14615.78947368421 ns (± 3467.211639192487) 7059.550561797752 ns (± 2161.0560918387896) 2.07

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ModuleOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: ACL) 42808.63254488431 ns (± 320.7989711838781) 43789.0862508138 ns (± 309.59619931865865) 0.98
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: ACL) 56435.09178059896 ns (± 267.7135333021616) 55611.02716500418 ns (± 275.21452688182194) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: ACL) 91910.34436035156 ns (± 298.88692831881997) 94063.05871582031 ns (± 187.35949530481486) 0.98
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: ACL) 72224.38058035714 ns (± 559.8446976244942) 71244.55035807291 ns (± 502.2615442538989) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: ACL) 36049.14292602539 ns (± 275.4933083308065) 34594.28259277344 ns (± 64.47148969802103) 1.04
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: ACL) 35340.973325602215 ns (± 208.08952772706078) 32649.335872105188 ns (± 80.86005929476771) 1.08
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: ACL) 183271.40044696516 ns (± 738.0097816963636) 177265.35869954427 ns (± 860.8230760591196) 1.03
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: ACL) 336756.3483072917 ns (± 3237.0053232552746) 337209.4872070312 ns (± 3187.5288293375884) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: AOF) 42356.38950523963 ns (± 30.086011964902674) 43133.10756835937 ns (± 104.45614491181347) 0.98
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: AOF) 56653.41751534598 ns (± 376.39497474811856) 57970.7598449707 ns (± 306.44710539216766) 0.98
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: AOF) 100294.50432477679 ns (± 347.5516881794862) 109584.62641194662 ns (± 332.42783852574496) 0.92
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: AOF) 69500.18830566407 ns (± 134.56570212953852) 71681.774222819 ns (± 362.9554889670846) 0.97
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: AOF) 37700.40531804012 ns (± 129.53909343521352) 37423.91083577474 ns (± 151.17113195002392) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: AOF) 41532.11282348633 ns (± 330.5118954845202) 39364.704610188805 ns (± 191.37504058553176) 1.06
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: AOF) 186394.18324497767 ns (± 724.4973000043382) 172926.79877929686 ns (± 1067.9570755914688) 1.08
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: AOF) 344277.071077474 ns (± 2849.755769692863) 354048.9836425781 ns (± 2314.075184699244) 0.97
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: None) 44231.38210623605 ns (± 166.6300543345938) 44080.34062412807 ns (± 54.961163484879165) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: None) 54418.62950486403 ns (± 107.25994188029334) 54670.99203898112 ns (± 217.76992048184218) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: None) 90237.47351946149 ns (± 226.0365407958864) 90203.13199288504 ns (± 437.2697156448405) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: None) 69916.55436604818 ns (± 282.2655900630205) 71946.01606633113 ns (± 251.4215222687848) 0.97
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: None) 36483.484995524086 ns (± 65.33985654946957) 34974.52650349935 ns (± 155.18438323675008) 1.04
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: None) 34710.87368774414 ns (± 68.66521157083055) 34280.80339152018 ns (± 228.85358232877303) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: None) 183391.73447829025 ns (± 1088.2274980680418) 175695.79634602866 ns (± 1218.4521008460913) 1.04
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: None) 343549.8172688802 ns (± 1979.3890119866735) 339513.53704427084 ns (± 2492.2560104281206) 1.01

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lua.LuaScripts (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Lua.LuaScripts.Script1(Params: Managed,Limit) 156.28814697265625 ns (± 0.2874960156818887) 154.264866511027 ns (± 0.6893409893367001) 1.01
BDN.benchmark.Lua.LuaScripts.Script2(Params: Managed,Limit) 177.13922262191772 ns (± 0.517076965689394) 179.60434158643088 ns (± 0.20265232543691444) 0.99
BDN.benchmark.Lua.LuaScripts.Script3(Params: Managed,Limit) 282.361364364624 ns (± 0.4063196119774266) 285.70509323706995 ns (± 1.3694638759662918) 0.99
BDN.benchmark.Lua.LuaScripts.Script4(Params: Managed,Limit) 278.5451037543161 ns (± 0.2782170721553459) 280.40668805440265 ns (± 0.6431638060074603) 0.99
BDN.benchmark.Lua.LuaScripts.Script1(Params: Managed,None) 143.1649923324585 ns (± 0.2988777420638904) 154.0429793871366 ns (± 0.43574924394536957) 0.93
BDN.benchmark.Lua.LuaScripts.Script2(Params: Managed,None) 182.43979556219918 ns (± 0.4066846315491626) 178.80852381388345 ns (± 0.37447294456104996) 1.02
BDN.benchmark.Lua.LuaScripts.Script3(Params: Managed,None) 274.5703765324184 ns (± 0.5742135184271737) 270.2720292409261 ns (± 0.6218195822717435) 1.02
BDN.benchmark.Lua.LuaScripts.Script4(Params: Managed,None) 298.25393970196063 ns (± 0.6218471344939571) 274.79940823146273 ns (± 0.9598884979673377) 1.09
BDN.benchmark.Lua.LuaScripts.Script1(Params: Native,None) 142.71020571390787 ns (± 0.27111869167774055) 142.94448632460373 ns (± 0.4378046584896826) 1.00
BDN.benchmark.Lua.LuaScripts.Script2(Params: Native,None) 173.511323928833 ns (± 0.31052819245276586) 174.58142723355974 ns (± 0.37638417931830453) 0.99
BDN.benchmark.Lua.LuaScripts.Script3(Params: Native,None) 267.41768916447955 ns (± 0.2593756973884757) 269.00475365774975 ns (± 0.5562968304726373) 0.99
BDN.benchmark.Lua.LuaScripts.Script4(Params: Native,None) 287.282395362854 ns (± 0.4823691729377062) 304.7950967152913 ns (± 1.0182102033947997) 0.94
BDN.benchmark.Lua.LuaScripts.Script1(Params: Tracked,Limit) 140.71314151470477 ns (± 0.18542496927604213) 140.85148913519723 ns (± 0.18202942688589285) 1.00
BDN.benchmark.Lua.LuaScripts.Script2(Params: Tracked,Limit) 187.80112777437483 ns (± 0.3576106561502732) 174.80135474886214 ns (± 0.14567191788404085) 1.07
BDN.benchmark.Lua.LuaScripts.Script3(Params: Tracked,Limit) 289.5211982727051 ns (± 0.6916048267322995) 276.92921502249584 ns (± 0.6040504858259556) 1.05
BDN.benchmark.Lua.LuaScripts.Script4(Params: Tracked,Limit) 277.0623588562012 ns (± 0.6154659371655528) 277.95064789908275 ns (± 0.4768909367951783) 1.00
BDN.benchmark.Lua.LuaScripts.Script1(Params: Tracked,None) 149.66129461924234 ns (± 0.5757874623187831) 146.4790729375986 ns (± 0.5891562949420873) 1.02
BDN.benchmark.Lua.LuaScripts.Script2(Params: Tracked,None) 187.36387766324557 ns (± 0.4312336248409015) 183.95634174346924 ns (± 0.377636233605342) 1.02
BDN.benchmark.Lua.LuaScripts.Script3(Params: Tracked,None) 273.6860922404698 ns (± 0.41618935587146905) 281.46193210895245 ns (± 0.48984603463934456) 0.97
BDN.benchmark.Lua.LuaScripts.Script4(Params: Tracked,None) 276.2099852928749 ns (± 0.5699192225302779) 276.6427707672119 ns (± 0.7064051208055998) 1.00

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.RawStringOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.RawStringOperations.Set(Params: ACL) 14652.631912231445 ns (± 59.22610104016524) 15380.371981302897 ns (± 12.63586486246713) 0.95
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: ACL) 20119.291946411133 ns (± 131.17162377118336) 20363.992635091145 ns (± 137.75074480171597) 0.99
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: ACL) 22185.85925801595 ns (± 163.12550277587687) 21919.26400756836 ns (± 236.29190922484264) 1.01
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: ACL) 22503.06452824519 ns (± 86.14503105357252) 28223.77508646647 ns (± 190.56093830526393) 0.80
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: ACL) 16312.603616333008 ns (± 125.94532232717603) 16691.98299952916 ns (± 77.58509545044845) 0.98
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: ACL) 10592.02449951172 ns (± 64.88316976882571) 10547.97070639474 ns (± 21.175951794715942) 1.00
BDN.benchmark.Operations.RawStringOperations.Increment(Params: ACL) 21642.637405395508 ns (± 101.32272708589004) 21827.10838623047 ns (± 141.92657293743946) 0.99
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: ACL) 22681.721893310547 ns (± 80.1741575947098) 22561.27805655343 ns (± 144.33274223733798) 1.01
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: ACL) 27059.04851582845 ns (± 136.04630347826685) 28090.781531197685 ns (± 139.26134981451355) 0.96
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: ACL) 26900.5893351237 ns (± 135.60156181892307) 29375.118420410156 ns (± 307.73789262982666) 0.92
BDN.benchmark.Operations.RawStringOperations.Set(Params: AOF) 21710.569274902344 ns (± 167.44683341122644) 21940.91015625 ns (± 179.21743130095297) 0.99
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: AOF) 26449.711377825057 ns (± 142.3217368360831) 27157.98736354283 ns (± 183.28782015382052) 0.97
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: AOF) 29973.30664876302 ns (± 163.1087873877837) 30220.870718819755 ns (± 239.91813267463343) 0.99
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: AOF) 31412.11931559245 ns (± 198.11945102490657) 31396.38012259347 ns (± 118.72976884652029) 1.00
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: AOF) 16558.86348900428 ns (± 18.196681170337683) 16354.674220784505 ns (± 86.15184175481387) 1.01
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: AOF) 10548.862772623697 ns (± 53.96921830585272) 11123.316660954402 ns (± 19.78649997619041) 0.95
BDN.benchmark.Operations.RawStringOperations.Increment(Params: AOF) 28937.543294270832 ns (± 166.98490098948614) 28482.590861002605 ns (± 119.96712130153654) 1.02
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: AOF) 30043.286724853515 ns (± 123.92337402270068) 28367.847986857098 ns (± 128.6757117876539) 1.06
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: AOF) 36041.82740275065 ns (± 584.7868106228215) 34203.412767682756 ns (± 144.51162736594523) 1.05
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: AOF) 35264.220379638675 ns (± 487.61337456268467) 36322.79631551107 ns (± 521.1190929511577) 0.97
BDN.benchmark.Operations.RawStringOperations.Set(Params: None) 14932.59164123535 ns (± 66.14904047777036) 15038.012470538799 ns (± 54.71788955983065) 0.99
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: None) 20239.823020426433 ns (± 105.27206669790107) 19970.383345540366 ns (± 135.16423215255494) 1.01
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: None) 21531.32233581543 ns (± 98.78990652137014) 22304.816739763533 ns (± 178.4441417446729) 0.97
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: None) 22861.748365275063 ns (± 76.24757811291448) 22672.086389160155 ns (± 172.00576788879076) 1.01
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: None) 16754.502682025617 ns (± 32.65509340004564) 16736.28271077474 ns (± 72.19169126451216) 1.00
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: None) 10675.151652018229 ns (± 58.3106101504921) 10771.572028096516 ns (± 99.12107270611936) 0.99
BDN.benchmark.Operations.RawStringOperations.Increment(Params: None) 22863.497141520184 ns (± 96.57927029057458) 22787.986920166015 ns (± 156.35789803184198) 1.00
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: None) 22676.588374546594 ns (± 43.95086921986374) 24634.163790021623 ns (± 93.94804401946479) 0.92
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: None) 26489.979925973075 ns (± 67.2970751945449) 28428.721629842123 ns (± 126.67758333857721) 0.93
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: None) 27836.058865356445 ns (± 83.2149216824938) 28489.005337524413 ns (± 108.39704026938729) 0.98

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ModuleOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: ACL) 67316.52409480169 ns (± 147.9324377336776) 67381.90260667067 ns (± 58.91352320547312) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: ACL) 82783.46516927083 ns (± 106.0109353955059) 83437.65869140625 ns (± 148.0084430117496) 0.99
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: ACL) 109501.0965983073 ns (± 85.26276985915457) 110389.91455078125 ns (± 167.2979561168004) 0.99
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: ACL) 90469.5947265625 ns (± 81.32784347516863) 91684.79527064732 ns (± 182.99025919442195) 0.99
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: ACL) 59235.34458705357 ns (± 45.697713368337126) 59415.16680036272 ns (± 40.92709343210469) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: ACL) 55666.83232234074 ns (± 33.17659537243248) 55533.68581136068 ns (± 47.87292806471561) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: ACL) 195031.7537434896 ns (± 767.1668421354299) 192481.76832932694 ns (± 344.6580802109922) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: ACL) 325471.19140625 ns (± 1286.6577044398875) 317970.73974609375 ns (± 768.224958995078) 1.02
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: AOF) 66703.70060847356 ns (± 75.12894543636213) 68613.9969889323 ns (± 108.20998394536406) 0.97
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: AOF) 90483.03670247395 ns (± 168.62424883358207) 89821.60034179688 ns (± 162.46889255028137) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: AOF) 115114.67372349331 ns (± 198.10696746099146) 112769.8018391927 ns (± 151.5886984042987) 1.02
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: AOF) 89253.79720052083 ns (± 109.84890928854105) 90807.40792410714 ns (± 190.9052901069056) 0.98
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: AOF) 59676.269967215405 ns (± 72.36528228367057) 58867.379150390625 ns (± 44.50801078059474) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: AOF) 61147.389439174105 ns (± 200.86819832762004) 59465.37251790365 ns (± 181.06451713112574) 1.03
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: AOF) 188323.70930989584 ns (± 387.3494780869532) 189928.83463541666 ns (± 1241.3671136163075) 0.99
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: AOF) 338443.2535807292 ns (± 760.2932723067049) 331078.07965959824 ns (± 1045.5638412641279) 1.02
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringReadCommand(Params: None) 66443.79760742188 ns (± 56.814418458778384) 65809.35953776042 ns (± 49.74240669030386) 1.01
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpRawStringRmwCommand(Params: None) 82559.6748860677 ns (± 66.17202072697987) 83091.19611467634 ns (± 162.1459889311311) 0.99
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjRmwCommand(Params: None) 109703.94897460938 ns (± 76.71034204878906) 110180.69458007812 ns (± 162.72520365027444) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpObjReadCommand(Params: None) 89275.62906901042 ns (± 231.59433093263058) 89506.67818509616 ns (± 79.67242654053815) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpProc(Params: None) 58511.12447102865 ns (± 42.827419528653785) 60102.21822102865 ns (± 114.56752437932236) 0.97
BDN.benchmark.Operations.ModuleOperations.ModuleNoOpTxn(Params: None) 55339.53572591146 ns (± 61.567305146137286) 55421.01651705228 ns (± 30.754047511674642) 1.00
BDN.benchmark.Operations.ModuleOperations.ModuleJsonGetCommand(Params: None) 199048.642578125 ns (± 263.9241527353036) 187726.25558035713 ns (± 315.75412076949544) 1.06
BDN.benchmark.Operations.ModuleOperations.ModuleJsonSetCommand(Params: None) 316477.34375 ns (± 766.0749929169244) 328086.5787760417 ns (± 1325.1247859898817) 0.96

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ScriptOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Managed,Limit) 144922.5767124721 ns (± 408.31864579969516) 146015.51440429688 ns (± 1189.2858706226173) 0.99
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Managed,Limit) 20007.090525309246 ns (± 25.540170949084665) 19960.952944437664 ns (± 16.106241812210552) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Managed,Limit) 17602.562512715656 ns (± 23.663722732421764) 16669.523015340168 ns (± 114.47537345523149) 1.06
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Managed,Limit) 141353.5237943209 ns (± 136.3527914333526) 141616.64427083332 ns (± 262.185517381795) 1.00
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Managed,Limit) 45776.24650472005 ns (± 263.0423948486452) 43902.55612792969 ns (± 269.0203568959325) 1.04
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Managed,Limit) 103673.05931745257 ns (± 162.50419712892847) 106131.04126915566 ns (± 335.7055826656957) 0.98
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Managed,Limit) 10140772.773958333 ns (± 167847.52673878675) 10212846.854492188 ns (± 187693.54564450873) 0.99
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Managed,Limit) 276254.20037597656 ns (± 27961.293170639194) 275521.3888818359 ns (± 28586.272872592283) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Managed,None) 144825.9103910006 ns (± 578.8873507497767) 144652.980922154 ns (± 1158.3384755931725) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Managed,None) 19875.300688680014 ns (± 139.81338687391766) 19942.65992940267 ns (± 133.56753684767943) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Managed,None) 16487.242087227958 ns (± 16.400131107445223) 16739.52162068685 ns (± 99.98920108299767) 0.98
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Managed,None) 143757.6319580078 ns (± 245.08607764489642) 141287.96503557477 ns (± 664.8754256184994) 1.02
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Managed,None) 43401.52227783203 ns (± 33.95353325781813) 45326.09964861189 ns (± 144.81986653319956) 0.96
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Managed,None) 107755.60229492188 ns (± 365.24422064967774) 107583.7275390625 ns (± 290.4040246040682) 1.00
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Managed,None) 10157356.298828125 ns (± 185568.20412021983) 10218399.823660715 ns (± 176534.3207409932) 0.99
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Managed,None) 280104.0176660156 ns (± 27849.784098005577) 276089.4268359375 ns (± 28474.153885217704) 1.01
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Native,None) 148143.99365234375 ns (± 693.1786528007653) 145143.1071589543 ns (± 390.1238559077785) 1.02
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Native,None) 19601.61596797063 ns (± 55.08305206834095) 19582.85381266276 ns (± 79.03673206300314) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Native,None) 16668.71279689244 ns (± 73.70395684414224) 16587.46307067871 ns (± 132.39148890203614) 1.00
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Native,None) 140794.48470052084 ns (± 269.5593399949546) 141557.37137545072 ns (± 167.11649554622525) 0.99
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Native,None) 43359.270528157555 ns (± 192.01536418183125) 43811.49920654297 ns (± 29.20653640054826) 0.99
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Native,None) 104215.52307128906 ns (± 261.39462806700317) 103606.97215924945 ns (± 179.35033625458865) 1.01
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Native,None) 8570157.286830356 ns (± 27596.808066165744) 8517665.747395834 ns (± 13806.55340817135) 1.01
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Native,None) 225991.7994478666 ns (± 291.5101637404045) 232972.0160481771 ns (± 721.2308545218375) 0.97
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Tracked,Limit) 144113.63209635418 ns (± 1128.319062771756) 147488.3681722005 ns (± 542.7665330397305) 0.98
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Tracked,Limit) 20638.095485432943 ns (± 83.17114520237037) 20070.505733235677 ns (± 106.37997830020436) 1.03
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Tracked,Limit) 16772.29653015137 ns (± 76.11316000730449) 16462.461393083846 ns (± 6.385739196941181) 1.02
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Tracked,Limit) 142145.36134847006 ns (± 135.99449227823237) 141757.72713216147 ns (± 55.00135472303422) 1.00
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Tracked,Limit) 45779.89855143229 ns (± 167.58095064829797) 43464.95089314778 ns (± 122.3212664636179) 1.05
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Tracked,Limit) 104062.96622140067 ns (± 62.169820934730325) 105333.68277994792 ns (± 337.7398235148474) 0.99
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Tracked,Limit) 9464159.3578125 ns (± 55716.89760999757) 9392919.997596154 ns (± 29612.216381835915) 1.01
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Tracked,Limit) 250029.023890904 ns (± 748.8026050828923) 252378.41438802084 ns (± 577.0070952955814) 0.99
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Tracked,None) 145648.12201799665 ns (± 536.3055309617579) 146023.57027762276 ns (± 555.0271094562618) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Tracked,None) 20060.83088582357 ns (± 84.23486759798655) 19787.578762817382 ns (± 70.99731686199645) 1.01
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Tracked,None) 16522.125978742326 ns (± 35.84451302337585) 17511.8955078125 ns (± 20.0557520027648) 0.94
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Tracked,None) 142841.44266183037 ns (± 641.4521257261167) 141543.55357259113 ns (± 583.5274363419652) 1.01
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Tracked,None) 45072.02260480608 ns (± 32.790750158180266) 41979.796463012695 ns (± 87.67996300624218) 1.07
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Tracked,None) 103605.64501953125 ns (± 212.26517532240229) 103277.30074637277 ns (± 149.98502512948093) 1.00
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Tracked,None) 9531859.0390625 ns (± 38546.36504769795) 9442263.58705357 ns (± 24781.021646962876) 1.01
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Tracked,None) 252827.42044270833 ns (± 762.2711358232712) 253855.1924579327 ns (± 237.3832254974894) 1.00

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.RawStringOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.RawStringOperations.Set(Params: ACL) 14425.841287466195 ns (± 17.921192203053064) 15773.541768391928 ns (± 89.70508903736012) 0.91
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: ACL) 20555.648396809895 ns (± 121.69974064350829) 20412.606811523438 ns (± 29.364953482647383) 1.01
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: ACL) 21045.75453538161 ns (± 27.177680679683395) 21256.607273646765 ns (± 58.48957967615892) 0.99
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: ACL) 23119.73856608073 ns (± 75.72447509850798) 21230.772617885046 ns (± 30.049467695192302) 1.09
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: ACL) 15948.244934082031 ns (± 20.343268416917855) 16033.237515963041 ns (± 38.56796425509174) 0.99
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: ACL) 10912.322184244791 ns (± 22.580973690923408) 10655.305774395283 ns (± 40.225247313889426) 1.02
BDN.benchmark.Operations.RawStringOperations.Increment(Params: ACL) 22088.49358191857 ns (± 42.32521469460853) 22648.92098563058 ns (± 22.726719947759417) 0.98
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: ACL) 23672.83724271334 ns (± 34.35194510289503) 22813.662015474758 ns (± 22.616345830856503) 1.04
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: ACL) 28957.92714436849 ns (± 158.2598930619958) 26268.428649902344 ns (± 31.699519312538094) 1.10
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: ACL) 26783.46689860026 ns (± 51.25603818082057) 27393.201857346754 ns (± 37.62666550307109) 0.98
BDN.benchmark.Operations.RawStringOperations.Set(Params: AOF) 20223.771362304688 ns (± 43.80212779585807) 20933.899637858074 ns (± 119.28009744296767) 0.97
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: AOF) 26670.403238932293 ns (± 69.65493148311243) 26385.668006310098 ns (± 116.84773247414633) 1.01
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: AOF) 26336.918422154016 ns (± 62.59055705042978) 27622.838701520646 ns (± 42.747870372656585) 0.95
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: AOF) 28449.988664899553 ns (± 61.49834844926748) 28593.077290852863 ns (± 73.83976396833258) 0.99
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: AOF) 15520.299857003349 ns (± 53.99783719744967) 15705.874226888021 ns (± 17.893662926038818) 0.99
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: AOF) 10738.046875 ns (± 11.250390112731937) 10906.726728166852 ns (± 18.531800833386857) 0.98
BDN.benchmark.Operations.RawStringOperations.Increment(Params: AOF) 26648.473002115887 ns (± 31.95814679629151) 26381.915181477863 ns (± 120.39329018689504) 1.01
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: AOF) 26524.30396446815 ns (± 20.444539966357) 27975.572967529297 ns (± 34.613216237203204) 0.95
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: AOF) 31219.007521409254 ns (± 88.47961663962194) 33176.25467936198 ns (± 171.6039334143389) 0.94
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: AOF) 32475.963134765625 ns (± 89.71441986399746) 31012.694440569197 ns (± 108.02822009405669) 1.05
BDN.benchmark.Operations.RawStringOperations.Set(Params: None) 14646.89448038737 ns (± 14.577951573065215) 14164.086260114398 ns (± 17.953285196593338) 1.03
BDN.benchmark.Operations.RawStringOperations.SetEx(Params: None) 20244.597981770832 ns (± 48.66016461737704) 21898.985392252605 ns (± 53.47368087628987) 0.92
BDN.benchmark.Operations.RawStringOperations.SetNx(Params: None) 20192.251586914062 ns (± 44.982415007332726) 22073.27205113002 ns (± 42.90829181842247) 0.91
BDN.benchmark.Operations.RawStringOperations.SetXx(Params: None) 21436.793082101005 ns (± 29.715769182500434) 22890.131429036457 ns (± 47.450030558576124) 0.94
BDN.benchmark.Operations.RawStringOperations.GetFound(Params: None) 15686.375223795572 ns (± 25.17910830363347) 15354.956359863281 ns (± 20.30367314942815) 1.02
BDN.benchmark.Operations.RawStringOperations.GetNotFound(Params: None) 11067.357744489398 ns (± 21.682905246702752) 10649.441680908203 ns (± 22.908670915079433) 1.04
BDN.benchmark.Operations.RawStringOperations.Increment(Params: None) 22054.191371372766 ns (± 32.760983496731576) 22212.695922851562 ns (± 101.22301395447545) 0.99
BDN.benchmark.Operations.RawStringOperations.Decrement(Params: None) 21546.004130045574 ns (± 33.29100071395256) 21060.627042330227 ns (± 37.631916338619234) 1.02
BDN.benchmark.Operations.RawStringOperations.IncrementBy(Params: None) 26246.485493977863 ns (± 115.08162606950819) 27185.262239896336 ns (± 52.984747690828385) 0.97
BDN.benchmark.Operations.RawStringOperations.DecrementBy(Params: None) 26782.60772705078 ns (± 106.87848193707879) 25529.400517390324 ns (± 47.93395163249898) 1.05

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.HashObjectOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: ACL) 139398.10098702568 ns (± 452.9448539929433) 139940.02568708148 ns (± 811.5163022294571) 1.00
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: ACL) 13161.133913480318 ns (± 39.856934863071004) 12922.51993724278 ns (± 382.77417511630154) 1.02
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: ACL) 13996.712337239584 ns (± 49.03262513128916) 13938.72665640024 ns (± 49.67446612751544) 1.00
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: ACL) 9001.369826180595 ns (± 49.34816046457663) 9281.8066502889 ns (± 85.71428703940592) 0.97
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: ACL) 11660.029718526204 ns (± 68.92515165450463) 11366.58030802409 ns (± 92.14884239561867) 1.03
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: ACL) 12777.595833333333 ns (± 76.06436228276195) 13023.128581455776 ns (± 43.9404186976787) 0.98
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: ACL) 10186.45021311442 ns (± 69.57309808034282) 11722.783767113318 ns (± 8.490970655158446) 0.87
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: ACL) 8944.590300496418 ns (± 44.09081991202919) 8941.060061645509 ns (± 6.0778865160026605) 1.00
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: ACL) 15463.72578125 ns (± 336.5500197275059) 12354.70101928711 ns (± 59.828379750955555) 1.25
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: ACL) 11959.261860148112 ns (± 66.23763742879532) 11988.779790242514 ns (± 80.97162459100085) 1.00
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: ACL) 11966.819567871094 ns (± 74.12727199434399) 10495.401972452799 ns (± 7.2235543762948895) 1.14
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: ACL) 13692.704389444987 ns (± 56.883858687967866) 13648.837018694196 ns (± 50.92513599799319) 1.00
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: ACL) 12552.250838143485 ns (± 79.14447848197396) 12634.77024943034 ns (± 49.434837807779395) 0.99
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: ACL) 13776.733339945475 ns (± 12.756416802544264) 13632.78168741862 ns (± 173.86383471173914) 1.01
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: ACL) 10378.087172190348 ns (± 9.394158699640066) 11895.242338562011 ns (± 53.23961705341864) 0.87
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: AOF) 168016.3253580729 ns (± 622.5359336216527) 160076.35279134114 ns (± 605.6320540788234) 1.05
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: AOF) 57948.849300130205 ns (± 161.7885267539684) 58749.33721487863 ns (± 228.62143565419515) 0.99
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: AOF) 47977.999291992186 ns (± 295.18217171530205) 51911.431382399336 ns (± 156.88583354035225) 0.92
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: AOF) 50010.511701311385 ns (± 1170.9950943730444) 51122.859759990984 ns (± 58.055084987886836) 0.98
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: AOF) 85561.1332804362 ns (± 636.165489245472) 84576.10552509014 ns (± 272.897795759262) 1.01
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: AOF) 114806.10643717447 ns (± 452.5161834945412) 113784.50364990234 ns (± 421.32915143705435) 1.01
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: AOF) 50993.070084635416 ns (± 231.2720706474249) 51732.13482259114 ns (± 350.825880186647) 0.99
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: AOF) 54420.03836529072 ns (± 144.97785901996565) 53744.71659138997 ns (± 198.16514807657234) 1.01
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: AOF) 56051.91663411458 ns (± 222.07879765585258) 57801.333947753905 ns (± 165.09898148786115) 0.97
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: AOF) 88565.46689278739 ns (± 305.2770583199334) 89211.6450764974 ns (± 407.56268415127823) 0.99
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: AOF) 59235.3896484375 ns (± 260.9955592911177) 57433.489131673174 ns (± 167.6423453983322) 1.03
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: AOF) 13296.219748942058 ns (± 42.73861938020714) 13184.442633492607 ns (± 44.511529402765845) 1.01
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: AOF) 82704.10057779947 ns (± 395.3897438764935) 79954.41704915365 ns (± 317.6396441264264) 1.03
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: AOF) 58614.40615234375 ns (± 224.3458888665431) 57843.440032958984 ns (± 124.22507179971377) 1.01
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: AOF) 50036.27197062175 ns (± 105.02873989376292) 49590.51728468675 ns (± 62.78676533091496) 1.01
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: None) 137946.78213936943 ns (± 472.7208226490527) 139626.56040736608 ns (± 325.8767616441883) 0.99
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: None) 58129.99927164714 ns (± 235.69975876959444) 63029.80789888822 ns (± 104.83869101571435) 0.92
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: None) 50623.37952473958 ns (± 258.0457811602822) 51865.75805429312 ns (± 150.61455015330486) 0.98
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: None) 48947.60123552595 ns (± 124.03351565973752) 49587.30458068848 ns (± 80.60250097916109) 0.99
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: None) 80049.0678141276 ns (± 312.9288704209447) 84403.73039754231 ns (± 140.5456817278707) 0.95
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: None) 103717.66872558594 ns (± 350.63839572776294) 105231.21274239676 ns (± 448.42638942563246) 0.99
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: None) 55059.25359671457 ns (± 109.36019968594714) 50386.97297770182 ns (± 131.35574691894215) 1.09
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: None) 58028.481698172436 ns (± 125.38611257177243) 52935.87736816406 ns (± 114.1259055749355) 1.10
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: None) 51085.6207438151 ns (± 169.76573495149307) 52042.79920247396 ns (± 326.2717914380833) 0.98
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: None) 77503.32327473958 ns (± 478.0549706924987) 78696.4399937221 ns (± 193.39320568664715) 0.98
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: None) 58957.59306989397 ns (± 173.78856414329678) 59289.25908203125 ns (± 249.8774227432381) 0.99
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: None) 13261.052901713054 ns (± 46.962405753706754) 13830.461094665527 ns (± 52.281756803102006) 0.96
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: None) 68553.90482875278 ns (± 161.56637366134237) 69165.07933349609 ns (± 214.6154264502302) 0.99
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: None) 57643.63022054036 ns (± 149.65553039269838) 56773.11499459403 ns (± 131.12446062547124) 1.02
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: None) 50309.58940429687 ns (± 208.43504490948905) 49494.001267496744 ns (± 84.78675244932943) 1.02

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.HashObjectOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: ACL) 98868.71901292067 ns (± 233.0107286955244) 108041.1653958834 ns (± 169.19189188275038) 0.92
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: ACL) 11279.342397054037 ns (± 13.303485344134467) 11306.280081612724 ns (± 42.87155343200213) 1.00
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: ACL) 10779.876650296725 ns (± 10.789970859008998) 10791.051810128349 ns (± 20.052881778129336) 1.00
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: ACL) 9584.42387898763 ns (± 123.79960749999172) 9421.900939941406 ns (± 20.09995496385182) 1.02
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: ACL) 13699.886732835035 ns (± 10.916577760417093) 13727.139333089193 ns (± 18.490531707257144) 1.00
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: ACL) 14901.89951578776 ns (± 21.98021252348336) 14886.85048421224 ns (± 25.405073447814278) 1.00
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: ACL) 12816.999271937779 ns (± 28.918249238424817) 12942.95649210612 ns (± 39.9406557401794) 0.99
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: ACL) 8892.373504638672 ns (± 10.19679196915955) 9434.744589669364 ns (± 9.567672785738582) 0.94
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: ACL) 12728.456819974459 ns (± 20.715051842944817) 13508.070591517857 ns (± 52.681077808561746) 0.94
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: ACL) 12021.466827392578 ns (± 19.520095023883705) 12002.559552873883 ns (± 19.084376114129388) 1.00
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: ACL) 15037.039712759164 ns (± 10.720940295215241) 14715.942709786552 ns (± 27.87735734197342) 1.02
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: ACL) 9228.622495211088 ns (± 12.527064513546357) 9243.480784098307 ns (± 12.956989767394075) 1.00
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: ACL) 11918.797912597656 ns (± 36.25944961933973) 11935.287729899088 ns (± 19.093241623567774) 1.00
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: ACL) 15923.000277005709 ns (± 13.377556196362212) 14710.67842755999 ns (± 6.546555499220916) 1.08
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: ACL) 14778.819819859096 ns (± 8.225826964018108) 13794.47982788086 ns (± 40.11692037727605) 1.07
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: AOF) 117930.517578125 ns (± 556.1916646932299) 118735.419921875 ns (± 682.2283327693531) 0.99
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: AOF) 44500.069318498885 ns (± 129.26335902941656) 42201.67018345424 ns (± 91.74814975346065) 1.05
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: AOF) 42234.01019756611 ns (± 123.95150584615351) 43478.067626953125 ns (± 103.09017413884028) 0.97
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: AOF) 45387.483723958336 ns (± 61.86060644235013) 46545.806884765625 ns (± 57.15249953186529) 0.98
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: AOF) 71479.9833170573 ns (± 232.42175091774124) 73891.02376302083 ns (± 299.7643462586916) 0.97
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: AOF) 96301.60945012019 ns (± 239.83152944712137) 96897.89469401042 ns (± 365.8860860054683) 0.99
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: AOF) 48065.43790377103 ns (± 88.18466974597375) 47785.18025716146 ns (± 52.41150712612869) 1.01
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: AOF) 38627.343314034595 ns (± 44.811025941744134) 38672.81880696615 ns (± 57.647043203915125) 1.00
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: AOF) 51779.50642903646 ns (± 289.345237706533) 49165.67252022879 ns (± 87.9714917291909) 1.05
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: AOF) 69782.82906668527 ns (± 184.94514554750108) 69255.64226422991 ns (± 332.5553150777394) 1.01
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: AOF) 56694.84191894531 ns (± 99.99011844002752) 56513.896484375 ns (± 370.9294627884803) 1.00
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: AOF) 9299.225718180338 ns (± 16.84010859471309) 9216.567942301432 ns (± 24.341795912205157) 1.01
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: AOF) 61211.6796875 ns (± 240.7368073417447) 59970.20792643229 ns (± 188.71084421020936) 1.02
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: AOF) 48929.33451334635 ns (± 206.90151309601816) 47741.632486979164 ns (± 187.02965398847215) 1.02
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: AOF) 49049.448939732145 ns (± 52.653849458677904) 48205.385044642855 ns (± 100.04618408621215) 1.02
BDN.benchmark.Operations.HashObjectOperations.HSetDel(Params: None) 101506.95237379808 ns (± 111.63928963167373) 102717.42084209736 ns (± 332.73602748570306) 0.99
BDN.benchmark.Operations.HashObjectOperations.HExists(Params: None) 42093.86901855469 ns (± 109.66970886123289) 46462.93701171875 ns (± 167.56677720617446) 0.91
BDN.benchmark.Operations.HashObjectOperations.HGet(Params: None) 42189.759826660156 ns (± 57.06236069872449) 44212.103271484375 ns (± 102.22650838226097) 0.95
BDN.benchmark.Operations.HashObjectOperations.HGetAll(Params: None) 46620.82773844401 ns (± 32.141806323871535) 47321.53625488281 ns (± 42.066026942787396) 0.99
BDN.benchmark.Operations.HashObjectOperations.HIncrby(Params: None) 66964.81497628348 ns (± 162.95148634730776) 63186.2801688058 ns (± 282.3269057206992) 1.06
BDN.benchmark.Operations.HashObjectOperations.HIncrbyFloat(Params: None) 95400.09577824519 ns (± 204.85207156724258) 86196.30737304688 ns (± 275.6455245700321) 1.11
BDN.benchmark.Operations.HashObjectOperations.HKeys(Params: None) 47795.602213541664 ns (± 65.23025464875158) 48161.03658040365 ns (± 102.76700131848796) 0.99
BDN.benchmark.Operations.HashObjectOperations.HLen(Params: None) 38495.26226337139 ns (± 64.54634178205433) 38227.83752441406 ns (± 85.8289775855647) 1.01
BDN.benchmark.Operations.HashObjectOperations.HMGet(Params: None) 47816.33823939732 ns (± 99.43036681341995) 48669.22078450521 ns (± 118.49231719761197) 0.98
BDN.benchmark.Operations.HashObjectOperations.HMSet(Params: None) 62635.89303152902 ns (± 222.49727016765087) 59752.88827078683 ns (± 119.4416729241761) 1.05
BDN.benchmark.Operations.HashObjectOperations.HRandField(Params: None) 55809.2392985026 ns (± 86.16952921293016) 57350.85144042969 ns (± 154.77970965789996) 0.97
BDN.benchmark.Operations.HashObjectOperations.HScan(Params: None) 9182.670484270368 ns (± 14.000941144076128) 9195.703669956752 ns (± 24.555248035546867) 1.00
BDN.benchmark.Operations.HashObjectOperations.HSetNx(Params: None) 52710.56283804087 ns (± 102.86320441580676) 51605.4774983724 ns (± 133.49953603171693) 1.02
BDN.benchmark.Operations.HashObjectOperations.HStrLen(Params: None) 45284.11385672433 ns (± 176.10600905090172) 45611.571829659595 ns (± 96.00740626877625) 0.99
BDN.benchmark.Operations.HashObjectOperations.HVals(Params: None) 49690.18778483073 ns (± 68.28157454103126) 47960.389811197914 ns (± 71.88688401586893) 1.04

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.ScriptOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Managed,Limit) 93003.1999860491 ns (± 910.8469791116306) 92254.6464029948 ns (± 682.2883051918059) 1.01
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Managed,Limit) 25599.28029378255 ns (± 43.734224530120855) 25563.673635629508 ns (± 22.55847668932321) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Managed,Limit) 23643.482099260604 ns (± 12.825112729250595) 23553.561604817707 ns (± 33.51815116810769) 1.00
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Managed,Limit) 77490.98088191106 ns (± 117.18265564189647) 74425.42637416294 ns (± 98.36382577072865) 1.04
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Managed,Limit) 29909.42195012019 ns (± 44.067372131671505) 31054.110013521633 ns (± 32.9145073526794) 0.96
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Managed,Limit) 61626.50594075521 ns (± 76.21836726236434) 62369.64674729567 ns (± 291.5969656903416) 0.99
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Managed,Limit) 5246511.041666667 ns (± 59357.66515766671) 5376693.694196428 ns (± 62439.162512895506) 0.98
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Managed,Limit) 170303.85473632812 ns (± 29740.486935844965) 173793.45678710938 ns (± 30660.681951346647) 0.98
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Managed,None) 91808.74982561384 ns (± 269.0105527847549) 93872.68880208333 ns (± 271.0946567416061) 0.98
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Managed,None) 25411.177498953683 ns (± 16.23183221028998) 25433.319091796875 ns (± 12.667543534294154) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Managed,None) 23883.108723958332 ns (± 24.294615792592626) 23724.447195870536 ns (± 62.73665005356039) 1.01
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Managed,None) 75871.33748372395 ns (± 183.25391407983793) 73677.97444661458 ns (± 160.23643444679473) 1.03
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Managed,None) 30101.080204890324 ns (± 40.21123380747056) 29684.033907376805 ns (± 35.178458390501746) 1.01
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Managed,None) 64356.787109375 ns (± 52.76904369761013) 66881.18567833534 ns (± 105.02328010441799) 0.96
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Managed,None) 5269045.677083333 ns (± 47645.11804252735) 5343312.319711538 ns (± 40460.93311801272) 0.99
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Managed,None) 169843.17138671875 ns (± 28150.29386925498) 177898.11401367188 ns (± 30615.20398230116) 0.95
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Native,None) 92587.87057059152 ns (± 202.0078151436079) 93260.68812779018 ns (± 308.3960219107319) 0.99
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Native,None) 25734.203045184797 ns (± 23.61017014228023) 25562.977834848258 ns (± 25.71170630124526) 1.01
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Native,None) 23629.808262416296 ns (± 10.265280435117939) 23735.90044294085 ns (± 23.688675952654435) 1.00
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Native,None) 76004.15486653645 ns (± 143.23246052740333) 75050.67091721755 ns (± 217.4539186187308) 1.01
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Native,None) 30750.184513972355 ns (± 66.50566812730351) 30416.839599609375 ns (± 30.761853550812386) 1.01
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Native,None) 63427.42919921875 ns (± 128.4342921994193) 63683.02786690848 ns (± 168.8679243431647) 1.00
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Native,None) 4382920.256696428 ns (± 7815.069203235277) 4421268.861607143 ns (± 4572.12567385277) 0.99
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Native,None) 126847.31163611778 ns (± 82.77962322231623) 128986.33161272321 ns (± 241.17971110310356) 0.98
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Tracked,Limit) 95977.21435546875 ns (± 520.2120374412416) 93092.44666466347 ns (± 255.41156529080456) 1.03
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Tracked,Limit) 25451.714579264324 ns (± 26.973487847366588) 25362.401689801896 ns (± 28.242839163299774) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Tracked,Limit) 23798.336079915363 ns (± 36.24082171472068) 23849.90041097005 ns (± 65.90405088023714) 1.00
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Tracked,Limit) 74280.90022160456 ns (± 88.1664928199556) 74293.37855747768 ns (± 210.59325212038613) 1.00
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Tracked,Limit) 30612.19961983817 ns (± 55.504256565287825) 30840.638078962053 ns (± 41.180179593698924) 0.99
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Tracked,Limit) 64927.50156947545 ns (± 156.15077414764326) 64099.476318359375 ns (± 71.20967151505089) 1.01
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Tracked,Limit) 4989832.645089285 ns (± 5895.818669930665) 5043625.540865385 ns (± 6903.519742660419) 0.99
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Tracked,Limit) 146337.63239933894 ns (± 221.28612238286001) 146523.11837332588 ns (± 179.61154403147384) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptLoad(Params: Tracked,None) 92049.54833984375 ns (± 235.46686745438114) 93206.14999624398 ns (± 184.97115501966468) 0.99
BDN.benchmark.Operations.ScriptOperations.ScriptExistsTrue(Params: Tracked,None) 25287.476501464844 ns (± 27.256661034425203) 25322.948279747598 ns (± 13.018944129461797) 1.00
BDN.benchmark.Operations.ScriptOperations.ScriptExistsFalse(Params: Tracked,None) 23868.572126116072 ns (± 25.75269785257263) 23748.891507662258 ns (± 27.05178098730058) 1.01
BDN.benchmark.Operations.ScriptOperations.Eval(Params: Tracked,None) 76530.88727678571 ns (± 75.58275653642627) 74115.74358258929 ns (± 95.59507947373386) 1.03
BDN.benchmark.Operations.ScriptOperations.EvalSha(Params: Tracked,None) 30401.469203404016 ns (± 38.0295464192537) 30807.894461495536 ns (± 45.2267947702185) 0.99
BDN.benchmark.Operations.ScriptOperations.SmallScript(Params: Tracked,None) 63599.96622721354 ns (± 86.25018880295607) 63494.310584435094 ns (± 188.35597128290286) 1.00
BDN.benchmark.Operations.ScriptOperations.LargeScript(Params: Tracked,None) 4959360.911458333 ns (± 7968.104298434075) 5065440.122767857 ns (± 7389.442914537549) 0.98
BDN.benchmark.Operations.ScriptOperations.ArrayReturn(Params: Tracked,None) 146332.65005258413 ns (± 271.09117916147915) 143151.31998697916 ns (± 723.5151526090585) 1.02

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.SortedSetOperations (ubuntu-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: ACL) 154455.49079777644 ns (± 463.6288121983323) 148569.97750651042 ns (± 2532.8834625879413) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: ACL) 10359.268873948316 ns (± 14.475370652044058) 10941.763914489746 ns (± 114.83885311151228) 0.95
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: ACL) 10653.108083871695 ns (± 8.08634917279232) 10432.299397786459 ns (± 101.13775859827942) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: ACL) 12264.078938802084 ns (± 16.326447875695663) 12062.216779855582 ns (± 102.70748027727983) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: ACL) 15077.907659912109 ns (± 70.88890790666765) 14455.734845674955 ns (± 110.66939074529093) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: ACL) 12782.23357187907 ns (± 98.73914873408823) 12271.132159423829 ns (± 115.1479046954711) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: ACL) 13455.226380411785 ns (± 101.88562951254154) 12803.34750366211 ns (± 123.55599363076512) 1.05
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: ACL) 14836.915945870536 ns (± 57.0917396007003) 13843.2158203125 ns (± 145.23687631779356) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: ACL) 16264.793418884277 ns (± 21.71287947684145) 15237.481345543494 ns (± 79.7675833998809) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: ACL) 12944.675285847981 ns (± 69.16051045766959) 12629.542813982282 ns (± 76.5701820981901) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: ACL) 93398.22424316406 ns (± 396.73048914852166) 88122.9105875651 ns (± 1231.1447868240996) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: ACL) 10750.284478759766 ns (± 60.40780557493292) 10308.472859700521 ns (± 123.29863515458104) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: ACL) 85907.58652169364 ns (± 517.7059027200266) 90847.06463216146 ns (± 1125.3128765344536) 0.95
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: ACL) 86950.9934366862 ns (± 589.5569683833082) 87388.4008178711 ns (± 721.0908582759091) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: ACL) 17369.489095052082 ns (± 106.52496187892632) 17336.75942993164 ns (± 219.8225248669798) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: ACL) 10993.39264351981 ns (± 54.435230096790264) 10596.497428385417 ns (± 83.90262754335151) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: ACL) 15623.632658894856 ns (± 147.2187951793296) 14588.333101545062 ns (± 101.25286753901672) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: ACL) 10043.103388468424 ns (± 34.78971709406336) 9853.137313334148 ns (± 147.77110252062346) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: ACL) 86850.53254582331 ns (± 213.15697848627818) 88702.57740129743 ns (± 589.3084579262799) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: ACL) 91827.58767089844 ns (± 430.4985678633685) 86509.97823079427 ns (± 1118.8590466982182) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: ACL) 88434.31881479117 ns (± 226.71976027139374) 86576.40189615886 ns (± 1232.5773489923072) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: ACL) 10814.148170979817 ns (± 45.347094701502165) 11046.111756388347 ns (± 133.76291485438298) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: ACL) 13149.400136675153 ns (± 43.67088819946166) 12871.421146937779 ns (± 86.99985510998069) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: ACL) 10732.29359654018 ns (± 44.025332346755434) 11016.182479858398 ns (± 94.9940265351832) 0.97
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: ACL) 13463.055849202474 ns (± 54.238004460561655) 12696.903245544434 ns (± 151.27720152805776) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: ACL) 16411.837666829426 ns (± 13.400434180474825) 16026.036482747395 ns (± 158.72899521802745) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: AOF) 170693.5583984375 ns (± 895.8541990842049) 167406.90598842077 ns (± 1451.6835520963414) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: AOF) 54669.92410982572 ns (± 229.19777550833953) 53262.08707478841 ns (± 452.85126041217484) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: AOF) 79231.22067057292 ns (± 465.51267077645616) 79843.28771158853 ns (± 941.0048971425812) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: AOF) 128045.55794270833 ns (± 696.403617495359) 126014.94072265625 ns (± 1471.3922040450725) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: AOF) 185580.22928059896 ns (± 979.9973968730075) 171732.88272530693 ns (± 1730.0366667771284) 1.08
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: AOF) 117138.84939778646 ns (± 356.4706050747028) 107675.50649007161 ns (± 1021.576189441398) 1.09
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: AOF) 128991.12126813616 ns (± 860.0905012078556) 126656.47395833333 ns (± 926.9206995465632) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: AOF) 136072.87562779017 ns (± 633.6236049898539) 125729.67596028646 ns (± 1188.687882074928) 1.08
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: AOF) 218797.58151041667 ns (± 1457.8634140609654) 216972.16988699776 ns (± 2281.6493866140263) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: AOF) 99816.81759440104 ns (± 527.6389187354703) 99318.63843645368 ns (± 865.711860137971) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: AOF) 290944.5504673549 ns (± 3008.3433220654138) 286200.3895182292 ns (± 4504.792669897745) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: AOF) 61545.380240304126 ns (± 243.87176535840712) 58774.50289481027 ns (± 476.1623690902625) 1.05
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: AOF) 205119.75577799478 ns (± 1294.698722375718) 209539.94607979912 ns (± 3133.010071856527) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: AOF) 210726.18777901787 ns (± 1030.4413395253428) 200873.6369954427 ns (± 1734.6837384916578) 1.05
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: AOF) 18024.8125193278 ns (± 98.53729811002717) 16835.264841715496 ns (± 225.2647580378392) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: AOF) 87370.61748860677 ns (± 349.03911965418985) 76872.15138346354 ns (± 1194.4226889121119) 1.14
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: AOF) 127277.36861478366 ns (± 1057.5309607063978) 125198.41055297852 ns (± 2325.216966836363) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: AOF) 59769.667301432295 ns (± 274.0401639550292) 57512.55776105608 ns (± 567.7830288975031) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: AOF) 240575.61271784856 ns (± 2172.934953530338) 230918.81427873884 ns (± 2933.572189358014) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: AOF) 226615.0041015625 ns (± 2325.18218407786) 216266.44505208332 ns (± 3632.5966094920177) 1.05
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: AOF) 236309.01007952009 ns (± 2834.5237034051684) 227144.9149169922 ns (± 2733.1692302403503) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: AOF) 60726.01652832031 ns (± 266.95749725972126) 55929.547790527344 ns (± 319.87699143938534) 1.09
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: AOF) 13218.50124206543 ns (± 67.518593211099) 13164.69104221889 ns (± 162.93305674041648) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: AOF) 62671.117911783855 ns (± 426.44433751568545) 61199.223828125 ns (± 939.3051370454302) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: AOF) 140035.24339192707 ns (± 783.7560426464346) 125890.12923177083 ns (± 1301.8096793761374) 1.11
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: AOF) 251218.2826985677 ns (± 2908.302925378122) 227836.87368977864 ns (± 3143.1771220565665) 1.10
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: None) 155429.40930989583 ns (± 661.6343544841521) 148986.38014322918 ns (± 1327.7862724581937) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: None) 55741.60037841797 ns (± 342.4510897340684) 51624.177697753905 ns (± 484.81721131085686) 1.08
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: None) 79597.16535644532 ns (± 347.4995893932032) 77858.95426839193 ns (± 1155.832574505216) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: None) 104496.76885172525 ns (± 457.1344360094416) 104503.35923665365 ns (± 848.0658543658558) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: None) 172599.49794921876 ns (± 818.1062405741873) 167524.82549176898 ns (± 1376.7478071473677) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: None) 102103.93268694196 ns (± 245.17295089016883) 97247.59672328403 ns (± 762.855206701998) 1.05
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: None) 126800.93842424665 ns (± 410.80376304592323) 118921.2311767578 ns (± 1223.471773283519) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: None) 115844.84409877232 ns (± 664.3012409474898) 125183.898828125 ns (± 1500.78303816843) 0.93
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: None) 175500.09267578126 ns (± 1126.6205307308308) 177719.71475655693 ns (± 1885.1576061126054) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: None) 105611.55247802734 ns (± 575.7408124664198) 97561.85631452288 ns (± 688.7863421063189) 1.08
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: None) 278200.4230957031 ns (± 4887.934506584081) 275045.4229492188 ns (± 6146.257897181127) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: None) 60815.58858548678 ns (± 117.91731633541971) 58275.23000284831 ns (± 421.17724773882094) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: None) 192978.9626546224 ns (± 1482.5426886164116) 187446.8375813802 ns (± 2953.130904475406) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: None) 192607.60348074776 ns (± 1438.9831405369819) 189004.49201660155 ns (± 2386.3332010362988) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: None) 18122.343200683594 ns (± 49.67149617675373) 16608.046580723352 ns (± 187.01257368644008) 1.09
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: None) 83446.47690022786 ns (± 395.05951497002764) 86591.22485351562 ns (± 746.1052916200069) 0.96
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: None) 116754.85081129808 ns (± 220.96449013764695) 115681.03707885742 ns (± 2243.2769322530057) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: None) 59998.757983398435 ns (± 468.46425783184714) 53769.53670654297 ns (± 575.3995589011155) 1.12
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: None) 236962.58478190104 ns (± 2413.699107461553) 233815.91650390625 ns (± 2446.115764283665) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: None) 223923.41334635418 ns (± 2861.237566719962) 210232.4390625 ns (± 3872.9838279542096) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: None) 221577.4044189453 ns (± 1271.467508415104) 221095.33108956474 ns (± 2953.871555677798) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: None) 57221.0179321289 ns (± 184.9066632985367) 55086.75104980469 ns (± 487.608572033591) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: None) 13629.588837687175 ns (± 45.60756599211804) 12893.610766601563 ns (± 126.0996612037347) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: None) 60733.34296875 ns (± 318.94846170364826) 59763.05263264974 ns (± 717.9312183854926) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: None) 119042.85354178293 ns (± 342.0706319221553) 118287.88971354166 ns (± 1077.685482776879) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: None) 186088.04139927455 ns (± 792.5412626303516) 182631.39900716147 ns (± 2474.9921251355445) 1.02

This comment was automatically generated by workflow using github-action-benchmark.

@github-actions
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Operations.SortedSetOperations (windows-latest net8.0 Release)

Benchmark suite Current: 14af7b8 Previous: 261e1c6 Ratio
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: ACL) 122221.44252232143 ns (± 530.576604553617) 119241.71055385044 ns (± 467.6380404054941) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: ACL) 10176.568654378256 ns (± 26.081299948346448) 10103.671373639789 ns (± 22.128715606778066) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: ACL) 10904.222324916294 ns (± 9.950503264842247) 11387.202780587333 ns (± 7.074860341935455) 0.96
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: ACL) 13773.610560099283 ns (± 8.72386108940394) 13681.55746459961 ns (± 20.824477354679438) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: ACL) 21563.855198451452 ns (± 20.46077100378255) 21628.72859409877 ns (± 16.45166422974738) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: ACL) 13840.78118460519 ns (± 30.505936933837383) 13997.640584309896 ns (± 54.4308288869005) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: ACL) 17015.321115347055 ns (± 21.477581939278746) 16956.60661969866 ns (± 12.22241496660635) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: ACL) 23171.54541015625 ns (± 16.033242425809263) 23091.74041748047 ns (± 26.434385713146007) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: ACL) 26102.726393479566 ns (± 27.23416367298269) 26074.368722098214 ns (± 14.87730952972235) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: ACL) 15593.306187220982 ns (± 14.375793050032017) 15682.942606608072 ns (± 12.428874605400019) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: ACL) 77094.23566545759 ns (± 180.90759229267687) 73887.57425944011 ns (± 95.49145867324121) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: ACL) 13357.333265032086 ns (± 22.388316295901042) 14011.379770132211 ns (± 7.497896820286839) 0.95
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: ACL) 69047.51935686384 ns (± 158.71009219542717) 68228.16487630208 ns (± 155.0152832471607) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: ACL) 70263.28776041667 ns (± 127.07398841270474) 68741.99916294643 ns (± 334.55629307697086) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: ACL) 13030.33697945731 ns (± 19.745209721149937) 13121.525472005209 ns (± 26.488207465805527) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: ACL) 11776.141768235426 ns (± 12.790373108645083) 11670.682961600167 ns (± 12.419525257080585) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: ACL) 24046.28361293248 ns (± 19.197641858327763) 23954.29513113839 ns (± 18.246259444749086) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: ACL) 11071.029193584736 ns (± 7.093443625045219) 11095.84244948167 ns (± 14.746544901373412) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: ACL) 75279.42260742188 ns (± 146.6634099649587) 74640.99557059152 ns (± 103.08964759211236) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: ACL) 75848.39853140023 ns (± 152.82109818720602) 76874.10801478794 ns (± 165.80971248820924) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: ACL) 75159.5703125 ns (± 173.99822713256594) 77225.02065805289 ns (± 98.44470532409389) 0.97
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: ACL) 13165.185077373799 ns (± 15.171607804110094) 13160.150502522787 ns (± 20.151942141573013) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: ACL) 9250.410679408482 ns (± 13.589972122865971) 9412.817818777901 ns (± 18.90245221335416) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: ACL) 13545.631408691406 ns (± 22.323132128834562) 13417.905644008091 ns (± 11.902822024238594) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: ACL) 14888.787129720053 ns (± 7.75936399772259) 14884.761454264322 ns (± 13.963530831573218) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: ACL) 27133.35462297712 ns (± 28.342019613411537) 27134.152221679688 ns (± 20.702618466784454) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: AOF) 140980.1806640625 ns (± 381.5721994285022) 137618.7052408854 ns (± 406.1611574792283) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: AOF) 38468.39341383714 ns (± 54.501409471625834) 40629.2588297526 ns (± 527.2877098837988) 0.95
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: AOF) 64479.384068080355 ns (± 98.89657697790763) 63247.569056919645 ns (± 184.84760788936217) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: AOF) 103988.61955915179 ns (± 296.4915285894843) 113462.54319411058 ns (± 316.59580602852236) 0.92
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: AOF) 154750.61197916666 ns (± 1024.7491321230775) 157206.86558314733 ns (± 523.1329741516217) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: AOF) 98654.33611188616 ns (± 456.0953781036084) 93019.89583333333 ns (± 296.14056853931606) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: AOF) 121709.82853816106 ns (± 278.77316518031046) 113726.66102818081 ns (± 260.67740694665906) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: AOF) 131291.79931640625 ns (± 823.3094444294734) 119995.26628766741 ns (± 294.02040673890457) 1.09
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: AOF) 207421.99425330528 ns (± 665.579652401055) 196129.5200892857 ns (± 656.5304273408668) 1.06
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: AOF) 85847.27642352764 ns (± 341.2157647042511) 86211.45978655134 ns (± 157.355584804605) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: AOF) 281961.1881510417 ns (± 2629.5708462217126) 264369.71261160716 ns (± 1034.9985136471107) 1.07
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: AOF) 58639.8350016276 ns (± 152.6849788862187) 58680.277506510414 ns (± 117.53182589691151) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: AOF) 165150.09483924278 ns (± 782.8174878681982) 170813.86544363838 ns (± 988.9497527648841) 0.97
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: AOF) 165259.53020368304 ns (± 440.3023353240124) 166162.34318659856 ns (± 609.2821606773341) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: AOF) 13060.773526705229 ns (± 28.73146765390958) 13257.694789341518 ns (± 18.86417540570007) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: AOF) 72897.74693080357 ns (± 67.66511546857618) 72875.14485677083 ns (± 247.19680408073924) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: AOF) 112434.41631610577 ns (± 506.6118445793773) 111644.83072916667 ns (± 1087.9576465292137) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: AOF) 53632.98121861049 ns (± 88.70250154921735) 54968.35428873698 ns (± 122.19173466974331) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: AOF) 217150.52083333334 ns (± 1182.0859503861532) 220204.0869140625 ns (± 1575.8151094932796) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: AOF) 228175.85123697916 ns (± 1407.0450349741018) 218772.03450520834 ns (± 1265.055341865231) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: AOF) 213779.37174479166 ns (± 1459.0980715567234) 222017.84493582588 ns (± 1615.8024179089512) 0.96
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: AOF) 54373.99466378348 ns (± 104.91378995499396) 59496.98954264323 ns (± 183.36802458469265) 0.91
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: AOF) 9294.925924447867 ns (± 14.521860027795617) 9553.655649820963 ns (± 22.14410700860007) 0.97
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: AOF) 59096.67264498197 ns (± 63.975188209753725) 59151.00355881911 ns (± 148.5139994852038) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: AOF) 117366.20657784598 ns (± 333.8788702653269) 118125.35481770833 ns (± 307.84348301724935) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: AOF) 215070.6089564732 ns (± 565.2264777737732) 215611.0538736979 ns (± 650.1043307672319) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZAddRem(Params: None) 121765.65511067708 ns (± 382.4828283107508) 122771.29069010417 ns (± 286.0360286954671) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZCard(Params: None) 38648.465435321516 ns (± 93.64114350503971) 39051.00931803385 ns (± 83.03704229160637) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZCount(Params: None) 66541.01213727679 ns (± 129.75185496553) 67345.9912109375 ns (± 178.70525340943115) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZDiff(Params: None) 99154.03686523438 ns (± 326.05448566550587) 100217.69653320312 ns (± 261.24046305342875) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZDiffStore(Params: None) 148315.4484049479 ns (± 390.55803972627035) 148204.61948939733 ns (± 299.7038613625328) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZIncrby(Params: None) 91565.7157389323 ns (± 246.59613948391092) 83303.65251813616 ns (± 197.88197470011232) 1.10
BDN.benchmark.Operations.SortedSetOperations.ZInter(Params: None) 103539.83317057292 ns (± 172.11241536346859) 119818.67414202009 ns (± 195.85653455206034) 0.86
BDN.benchmark.Operations.SortedSetOperations.ZInterCard(Params: None) 110663.2080078125 ns (± 147.0240412709691) 118503.15290178571 ns (± 190.28005099399647) 0.93
BDN.benchmark.Operations.SortedSetOperations.ZInterStore(Params: None) 169674.5662434896 ns (± 1191.3384279861657) 177018.44764122597 ns (± 429.3137348369505) 0.96
BDN.benchmark.Operations.SortedSetOperations.ZLexCount(Params: None) 89937.45029994419 ns (± 239.39645266093717) 87974.59635416667 ns (± 203.0549676638671) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZMPop(Params: None) 235118.3368389423 ns (± 1062.4232484388717) 234714.32942708334 ns (± 3151.0113104164584) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZMScore(Params: None) 60530.877685546875 ns (± 202.98441309052214) 59364.543269230766 ns (± 74.84779228837644) 1.02
BDN.benchmark.Operations.SortedSetOperations.ZPopMax(Params: None) 158250.86844308037 ns (± 417.36055297026684) 153581.2882486979 ns (± 472.0823659206744) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZPopMin(Params: None) 155677.71809895834 ns (± 769.4565640454889) 154090.36207932694 ns (± 499.24822181084346) 1.01
BDN.benchmark.Operations.SortedSetOperations.ZRandMember(Params: None) 12893.602091471354 ns (± 25.544934319530306) 13149.707234700521 ns (± 25.265508905626767) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZRange(Params: None) 73090.87806114784 ns (± 176.07371683370755) 79458.48737444196 ns (± 193.63634484030695) 0.92
BDN.benchmark.Operations.SortedSetOperations.ZRangeStore(Params: None) 109113.80208333333 ns (± 213.8519705053381) 108865.2099609375 ns (± 211.26492446283504) 1.00
BDN.benchmark.Operations.SortedSetOperations.ZRank(Params: None) 52417.35055106027 ns (± 179.16739116882394) 54214.76277669271 ns (± 135.75053795002097) 0.97
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByLex(Params: None) 197314.69350961538 ns (± 1000.2875130238988) 205398.89439174108 ns (± 908.1767555329283) 0.96
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByRank(Params: None) 198524.2862955729 ns (± 1210.653819857511) 200036.16594587054 ns (± 926.0552892665444) 0.99
BDN.benchmark.Operations.SortedSetOperations.ZRemRangeByScore(Params: None) 212747.71554129463 ns (± 981.4941616680845) 206591.07491629463 ns (± 968.933087349489) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZRevRank(Params: None) 57285.87544759115 ns (± 94.71458135245531) 55493.0184500558 ns (± 118.88739011325377) 1.03
BDN.benchmark.Operations.SortedSetOperations.ZScan(Params: None) 9315.108707972935 ns (± 13.288367919452963) 9482.284596761068 ns (± 22.885045080654123) 0.98
BDN.benchmark.Operations.SortedSetOperations.ZScore(Params: None) 57793.277849469865 ns (± 163.6445213910254) 61976.43338716947 ns (± 139.3115241418094) 0.93
BDN.benchmark.Operations.SortedSetOperations.ZUnion(Params: None) 118167.86586216518 ns (± 429.7360693808489) 113852.32340494792 ns (± 291.9798455422358) 1.04
BDN.benchmark.Operations.SortedSetOperations.ZUnionStore(Params: None) 174907.70438058037 ns (± 359.85859085328246) 188395.61197916666 ns (± 371.34987107818347) 0.93

This comment was automatically generated by workflow using github-action-benchmark.

Please sign in to comment.