Skip to content

Commit

Permalink
[C#] Replace LockingMode with ConcurrencyControlMode (#867)
Browse files Browse the repository at this point in the history
* Replace LockingMode with ConcurrencyControlMode

* Fix 'Remote' solution build
  • Loading branch information
TedHartMS authored Aug 25, 2023
1 parent 4d5412b commit 32d04e6
Show file tree
Hide file tree
Showing 38 changed files with 165 additions and 165 deletions.
4 changes: 2 additions & 2 deletions cs/benchmark/FasterSpanByteYcsbBenchmark.cs
Original file line number Diff line number Diff line change
Expand Up @@ -80,11 +80,11 @@ internal FasterSpanByteYcsbBenchmark(KeySpanByte[] i_keys_, KeySpanByte[] t_keys
if (testLoader.Options.UseSmallMemoryLog)
store = new FasterKV<SpanByte, SpanByte>
(testLoader.MaxKey / testLoader.Options.HashPacking, new LogSettings { LogDevice = device, PreallocateLog = true, PageSizeBits = 22, SegmentSizeBits = 26, MemorySizeBits = 26 },
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, lockingMode: testLoader.LockingMode);
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, concurrencyControlMode: testLoader.ConcurrencyControlMode);
else
store = new FasterKV<SpanByte, SpanByte>
(testLoader.MaxKey / testLoader.Options.HashPacking, new LogSettings { LogDevice = device, PreallocateLog = true, MemorySizeBits = 35 },
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, lockingMode: testLoader.LockingMode);
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, concurrencyControlMode: testLoader.ConcurrencyControlMode);
}

internal void Dispose()
Expand Down
4 changes: 2 additions & 2 deletions cs/benchmark/FasterYcsbBenchmark.cs
Original file line number Diff line number Diff line change
Expand Up @@ -80,11 +80,11 @@ internal FASTER_YcsbBenchmark(Key[] i_keys_, Key[] t_keys_, TestLoader testLoade
if (testLoader.Options.UseSmallMemoryLog)
store = new FasterKV<Key, Value>
(testLoader.MaxKey / testLoader.Options.HashPacking, new LogSettings { LogDevice = device, PreallocateLog = true, PageSizeBits = 25, SegmentSizeBits = 30, MemorySizeBits = 28 },
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, lockingMode: testLoader.LockingMode);
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, concurrencyControlMode: testLoader.ConcurrencyControlMode);
else
store = new FasterKV<Key, Value>
(testLoader.MaxKey / testLoader.Options.HashPacking, new LogSettings { LogDevice = device, PreallocateLog = true },
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, lockingMode: testLoader.LockingMode);
new CheckpointSettings { CheckpointDir = testLoader.BackupPath }, concurrencyControlMode: testLoader.ConcurrencyControlMode);
}

internal void Dispose()
Expand Down
6 changes: 3 additions & 3 deletions cs/benchmark/Options.cs
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ class Options
[Option('z', "locking", Required = false, Default = 0,
HelpText = "Locking Implementation:" +
"\n 0 = None (default)" +
"\n 1 = Mixed-mode locking using main HashTable buckets")]
public int LockingMode { get; set; }
"\n 1 = LockTable using main HashTable buckets")]
public int ConcurrencyControlMode { get; set; }

[Option('i', "iterations", Required = false, Default = 1,
HelpText = "Number of iterations of the test to run")]
Expand Down Expand Up @@ -100,7 +100,7 @@ class Options
public string GetOptionsString()
{
static string boolStr(bool value) => value ? "y" : "n";
return $"d: {DistributionName.ToLower()}; n: {NumaStyle}; rumd: {string.Join(',', RumdPercents)}; t: {ThreadCount}; z: {LockingMode}; i: {IterationCount}; hp: {HashPacking}"
return $"d: {DistributionName.ToLower()}; n: {NumaStyle}; rumd: {string.Join(',', RumdPercents)}; t: {ThreadCount}; z: {ConcurrencyControlMode}; i: {IterationCount}; hp: {HashPacking}"
+ $" sd: {boolStr(UseSmallData)}; sm: {boolStr(UseSmallMemoryLog)}; sy: {boolStr(this.UseSyntheticData)}; safectx: {boolStr(this.UseSafeContext)};"
+ $" chkptms: {this.PeriodicCheckpointMilliseconds}; chkpttype: {(this.PeriodicCheckpointMilliseconds > 0 ? this.PeriodicCheckpointType.ToString() : "None")};"
+ $" chkptincr: {boolStr(this.PeriodicCheckpointTryIncremental)}";
Expand Down
12 changes: 6 additions & 6 deletions cs/benchmark/TestLoader.cs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ class TestLoader
internal KeySpanByte[] txn_span_keys = default;

internal readonly BenchmarkType BenchmarkType;
internal readonly LockingMode LockingMode;
internal readonly ConcurrencyControlMode ConcurrencyControlMode;
internal readonly long InitCount;
internal readonly long TxnCount;
internal readonly int MaxKey;
Expand Down Expand Up @@ -61,13 +61,13 @@ static bool verifyOption(bool isValid, string name, string info = null)
if (!verifyOption(Options.NumaStyle >= 0 && Options.NumaStyle <= 1, "NumaStyle"))
return;

this.LockingMode = Options.LockingMode switch
this.ConcurrencyControlMode = Options.ConcurrencyControlMode switch
{
0 => LockingMode.None,
1 => LockingMode.Standard,
_ => throw new InvalidOperationException($"Unknown Locking mode int: {Options.LockingMode}")
0 => ConcurrencyControlMode.None,
1 => ConcurrencyControlMode.LockTable,
_ => throw new InvalidOperationException($"Unknown Locking mode int: {Options.ConcurrencyControlMode}")
};
if (!verifyOption(Enum.IsDefined(typeof(LockingMode), this.LockingMode), "LockingMode"))
if (!verifyOption(Enum.IsDefined(typeof(ConcurrencyControlMode), this.ConcurrencyControlMode), "ConcurrencyControlMode"))
return;

if (!verifyOption(Options.IterationCount > 0, "Iteration Count"))
Expand Down
2 changes: 1 addition & 1 deletion cs/remote/samples/FixedLenServer/Program.cs
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ static void Main(string[] args)
builder.SetMinimumLevel(LogLevel.Error);
});

using var server = new FixedLenServer<Key, Value, Input, Output, Functions>(opts.GetServerOptions(), () => new Functions(), lockingMode: LockingMode.Standard);
using var server = new FixedLenServer<Key, Value, Input, Output, Functions>(opts.GetServerOptions(), () => new Functions(), concurrencyControlMode: ConcurrencyControlMode.LockTable);
server.Start();
Console.WriteLine("Started server");

Expand Down
6 changes: 3 additions & 3 deletions cs/remote/src/FASTER.server/Servers/FixedLenServer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,11 @@ public sealed class FixedLenServer<Key, Value, Input, Output, Functions> : Gener
/// </summary>
/// <param name="opts"></param>
/// <param name="functionsGen"></param>
/// <param name="lockingMode"></param>
/// <param name="concurrencyControlMode"></param>
/// <param name="maxSizeSettings"></param>
/// <param name="loggerFactory"></param>
public FixedLenServer(ServerOptions opts, Func<Functions> functionsGen, LockingMode lockingMode, MaxSizeSettings maxSizeSettings = default, ILoggerFactory loggerFactory = null)
: base(opts, functionsGen, new FixedLenSerializer<Key, Value, Input, Output>(), new FixedLenKeySerializer<Key, Input>(), lockingMode: lockingMode, maxSizeSettings, loggerFactory)
public FixedLenServer(ServerOptions opts, Func<Functions> functionsGen, ConcurrencyControlMode concurrencyControlMode, MaxSizeSettings maxSizeSettings = default, ILoggerFactory loggerFactory = null)
: base(opts, functionsGen, new FixedLenSerializer<Key, Value, Input, Output>(), new FixedLenKeySerializer<Key, Input>(), concurrencyControlMode: concurrencyControlMode, maxSizeSettings, loggerFactory)
{
}
}
Expand Down
8 changes: 4 additions & 4 deletions cs/remote/src/FASTER.server/Servers/GenericServer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,11 @@ public class GenericServer<Key, Value, Input, Output, Functions, ParameterSerial
/// <param name="functionsGen"></param>
/// <param name="serializer"></param>
/// <param name="keyInputSerializer"></param>
/// <param name="lockingMode"></param>
/// <param name="concurrencyControlMode"></param>
/// <param name="maxSizeSettings"></param>
/// <param name="loggerFactory"></param>
public GenericServer(ServerOptions opts, Func<Functions> functionsGen, ParameterSerializer serializer, IKeyInputSerializer<Key, Input> keyInputSerializer,
LockingMode lockingMode, MaxSizeSettings maxSizeSettings = default, ILoggerFactory loggerFactory = null)
public GenericServer(ServerOptions opts, Func<Functions> functionsGen, ParameterSerializer serializer, IKeyInputSerializer<Key, Input> keyInputSerializer,
ConcurrencyControlMode concurrencyControlMode, MaxSizeSettings maxSizeSettings = default, ILoggerFactory loggerFactory = null)
{
this.opts = opts;

Expand All @@ -45,7 +45,7 @@ public GenericServer(ServerOptions opts, Func<Functions> functionsGen, Parameter
if (!Directory.Exists(opts.CheckpointDir))
Directory.CreateDirectory(opts.CheckpointDir);

store = new FasterKV<Key, Value>(indexSize, logSettings, checkpointSettings, lockingMode: lockingMode, loggerFactory: loggerFactory);
store = new FasterKV<Key, Value>(indexSize, logSettings, checkpointSettings, concurrencyControlMode: concurrencyControlMode, loggerFactory: loggerFactory);

if (opts.Recover)
{
Expand Down
2 changes: 1 addition & 1 deletion cs/remote/src/FASTER.server/Servers/VarLenServer.cs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public VarLenServer(ServerOptions opts, ILoggerFactory loggerFactory = null)
if (!Directory.Exists(opts.CheckpointDir))
Directory.CreateDirectory(opts.CheckpointDir);

store = new FasterKV<SpanByte, SpanByte>(indexSize, logSettings, checkpointSettings, lockingMode: LockingMode.Standard, loggerFactory: loggerFactory);
store = new FasterKV<SpanByte, SpanByte>(indexSize, logSettings, checkpointSettings, concurrencyControlMode: ConcurrencyControlMode.LockTable, loggerFactory: loggerFactory);

if (!opts.DisablePubSub)
{
Expand Down
2 changes: 1 addition & 1 deletion cs/remote/test/FASTER.remote.test/TestUtils.cs
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public static FixedLenServer<long, long, long, long, SimpleFunctions<long, long,
Recover = tryRecover,
IndexSize = "1m",
};
return new FixedLenServer<long, long, long, long, SimpleFunctions<long, long, long>>(opts, () => new SimpleFunctions<long, long, long>(merger), lockingMode: LockingMode.Standard);
return new FixedLenServer<long, long, long, long, SimpleFunctions<long, long, long>>(opts, () => new SimpleFunctions<long, long, long>(merger), concurrencyControlMode: ConcurrencyControlMode.LockTable);
}

/// <summary>
Expand Down
2 changes: 1 addition & 1 deletion cs/samples/StoreVarLenTypes/AsciiSumSample.cs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ public static void Run()
// For this test we require record-level locking
using var store = new FasterKV<SpanByte, SpanByte>(
size: 1L << 20,
logSettings: new LogSettings { LogDevice = log, MemorySizeBits = 15, PageSizeBits = 12 }, lockingMode: LockingMode.Standard);
logSettings: new LogSettings { LogDevice = log, MemorySizeBits = 15, PageSizeBits = 12 }, concurrencyControlMode: ConcurrencyControlMode.LockTable);

// Create session for ASCII sums. We require two callback function types to be provided:
// AsciiSumSpanByteFunctions implements RMW callback functions
Expand Down
4 changes: 2 additions & 2 deletions cs/src/core/ClientSession/ClientSession.cs
Original file line number Diff line number Diff line change
Expand Up @@ -241,7 +241,7 @@ public LockableUnsafeContext<Key, Value, Input, Output, Context, Functions> Lock
get
{
if (!this.fht.LockTable.IsEnabled)
throw new FasterException($"LockableUnsafeContext requires {nameof(LockingMode.Standard)}");
throw new FasterException($"LockableUnsafeContext requires {nameof(ConcurrencyControlMode.LockTable)}");
return luContext;
}
}
Expand All @@ -254,7 +254,7 @@ public LockableContext<Key, Value, Input, Output, Context, Functions> LockableCo
get
{
if (!this.fht.LockTable.IsEnabled)
throw new FasterException($"LockableContext requires {nameof(LockingMode.Standard)}");
throw new FasterException($"LockableContext requires {nameof(ConcurrencyControlMode.LockTable)}");
return lContext;
}
}
Expand Down
2 changes: 1 addition & 1 deletion cs/src/core/ClientSession/ILockableContext.cs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ public interface ILockableContext<TKey>
/// <summary>
/// If true, then keys must use one of the <see cref="IFasterContext{TKey}.GetKeyHash(ref TKey)"/> overloads to obtain a code by which groups of keys will be sorted for manual locking, to avoid deadlocks.
/// </summary>
/// <remarks>Whether this returns true depends on the <see cref="LockingMode"/> on <see cref="FasterKVSettings{Key, Value}"/>, or passed to the FasterKV constructor.</remarks>
/// <remarks>Whether this returns true depends on the <see cref="ConcurrencyControlMode"/> on <see cref="FasterKVSettings{Key, Value}"/>, or passed to the FasterKV constructor.</remarks>
bool NeedKeyHash { get; }

/// <summary>
Expand Down
4 changes: 2 additions & 2 deletions cs/src/core/Index/Common/FasterKVSettings.cs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ public sealed class FasterKVSettings<Key, Value> : IDisposable
/// <summary>
/// How FASTER should do record locking
/// </summary>
public LockingMode LockingMode;
public ConcurrencyControlMode ConcurrencyControlMode;

/// <summary>
/// Device used for main hybrid log
Expand Down Expand Up @@ -199,7 +199,7 @@ public override string ToString()
var retStr = $"index: {Utility.PrettySize(IndexSize)}; log memory: {Utility.PrettySize(MemorySize)}; log page: {Utility.PrettySize(PageSize)}; log segment: {Utility.PrettySize(SegmentSize)}";
retStr += $"; log device: {(LogDevice == null ? "null" : LogDevice.GetType().Name)}";
retStr += $"; obj log device: {(ObjectLogDevice == null ? "null" : ObjectLogDevice.GetType().Name)}";
retStr += $"; mutable fraction: {MutableFraction}; locking mode: {this.LockingMode}";
retStr += $"; mutable fraction: {MutableFraction}; locking mode: {this.ConcurrencyControlMode}";
retStr += $"; read cache (rc): {(ReadCacheEnabled ? "yes" : "no")}";
retStr += $"; read copy options: {ReadCopyOptions}";
if (ReadCacheEnabled)
Expand Down
12 changes: 6 additions & 6 deletions cs/src/core/Index/FASTER/FASTER.cs
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ public FasterKV(FasterKVSettings<Key, Value> fasterKVSettings) :
fasterKVSettings.GetIndexSizeCacheLines(), fasterKVSettings.GetLogSettings(),
fasterKVSettings.GetCheckpointSettings(), fasterKVSettings.GetSerializerSettings(),
fasterKVSettings.EqualityComparer, fasterKVSettings.GetVariableLengthStructSettings(),
fasterKVSettings.TryRecoverLatest, fasterKVSettings.LockingMode, null, fasterKVSettings.logger)
fasterKVSettings.TryRecoverLatest, fasterKVSettings.ConcurrencyControlMode, null, fasterKVSettings.logger)
{ }

/// <summary>
Expand All @@ -100,14 +100,14 @@ public FasterKV(FasterKVSettings<Key, Value> fasterKVSettings) :
/// <param name="comparer">FASTER equality comparer for key</param>
/// <param name="variableLengthStructSettings"></param>
/// <param name="tryRecoverLatest">Try to recover from latest checkpoint, if any</param>
/// <param name="lockingMode">How FASTER should do record locking</param>
/// <param name="concurrencyControlMode">How FASTER should do record locking</param>
/// <param name="loggerFactory">Logger factory to create an ILogger, if one is not passed in (e.g. from <see cref="FasterKVSettings{Key, Value}"/>).</param>
/// <param name="logger">Logger to use.</param>
/// <param name="lockTableSize">Number of buckets in the lock table</param>
public FasterKV(long size, LogSettings logSettings,
CheckpointSettings checkpointSettings = null, SerializerSettings<Key, Value> serializerSettings = null,
IFasterEqualityComparer<Key> comparer = null,
VariableLengthStructSettings<Key, Value> variableLengthStructSettings = null, bool tryRecoverLatest = false, LockingMode lockingMode = LockingMode.Standard,
VariableLengthStructSettings<Key, Value> variableLengthStructSettings = null, bool tryRecoverLatest = false, ConcurrencyControlMode concurrencyControlMode = ConcurrencyControlMode.LockTable,
ILoggerFactory loggerFactory = null, ILogger logger = null, int lockTableSize = Constants.kDefaultLockTableSize)
{
this.loggerFactory = loggerFactory;
Expand All @@ -134,8 +134,8 @@ public FasterKV(long size, LogSettings logSettings,
}
}

this.DoTransientLocking = lockingMode == LockingMode.Standard;
this.DoEphemeralLocking = lockingMode == LockingMode.Ephemeral;
this.DoTransientLocking = concurrencyControlMode == ConcurrencyControlMode.LockTable;
this.DoEphemeralLocking = concurrencyControlMode == ConcurrencyControlMode.RecordIsolation;

if (checkpointSettings is null)
checkpointSettings = new CheckpointSettings();
Expand Down Expand Up @@ -238,7 +238,7 @@ public FasterKV(long size, LogSettings logSettings,
sectorSize = (int)logSettings.LogDevice.SectorSize;
Initialize(size, sectorSize);

this.LockTable = new OverflowBucketLockTable<Key, Value>(lockingMode == LockingMode.Standard ? this : null);
this.LockTable = new OverflowBucketLockTable<Key, Value>(concurrencyControlMode == ConcurrencyControlMode.LockTable ? this : null);

systemState = SystemState.Make(Phase.REST, 1);

Expand Down
2 changes: 1 addition & 1 deletion cs/src/core/Index/FASTER/FASTERIterator.cs
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ public FasterKVIterator(FasterKV<Key, Value> fht, Functions functions, long unti
}

tempKv = new FasterKV<Key, Value>(fht.IndexSize, new LogSettings { LogDevice = new NullDevice(), ObjectLogDevice = new NullDevice(), MutableFraction = 1 }, comparer: fht.Comparer,
variableLengthStructSettings: variableLengthStructSettings, loggerFactory: loggerFactory, lockingMode: LockingMode.None);
variableLengthStructSettings: variableLengthStructSettings, loggerFactory: loggerFactory, concurrencyControlMode: ConcurrencyControlMode.None);
tempKvSession = tempKv.NewSession<Input, Output, Context, Functions>(functions);
mainKvIter = fht.Log.Scan(fht.Log.BeginAddress, untilAddress);
pushScanIterator = mainKvIter as IPushScanIterator<Key>;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,13 @@ internal struct OverflowBucketLockTable<TKey, TValue> : ILockTable<TKey>
internal OverflowBucketLockTable(FasterKV<TKey, TValue> f) => this.fht = f;

[Conditional("DEBUG")]
void AssertLockAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking lock when locking mode is not {LockingMode.Standard}");
void AssertLockAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking lock when locking mode is not {ConcurrencyControlMode.LockTable}");

[Conditional("DEBUG")]
void AssertUnlockAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking unlock when locking mode is not {LockingMode.Standard}");
void AssertUnlockAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking unlock when locking mode is not {ConcurrencyControlMode.LockTable}");

[Conditional("DEBUG")]
void AssertQueryAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking query when locking mode is not {LockingMode.Standard}");
void AssertQueryAllowed() => Debug.Assert(IsEnabled, $"Attempt to do Manual-locking query when locking mode is not {ConcurrencyControlMode.LockTable}");

internal long GetSize() => fht.state[fht.resizeInfo.version].size_mask;

Expand Down
Loading

0 comments on commit 32d04e6

Please sign in to comment.