diff --git a/.gitignore b/.gitignore
index 70e4a4485..f1880c096 100644
--- a/.gitignore
+++ b/.gitignore
@@ -193,3 +193,8 @@ packages/
*.lib
nativebin/
/cs/**/launchSettings.json
+
+# JetBrains
+cs/.idea/
+cs/remote/.idea
+cs/libdpr/.idea
diff --git a/cs/remote/src/FASTER.client/ClientSession.cs b/cs/remote/src/FASTER.client/ClientSession.cs
index ac1227220..a49f2290f 100644
--- a/cs/remote/src/FASTER.client/ClientSession.cs
+++ b/cs/remote/src/FASTER.client/ClientSession.cs
@@ -186,7 +186,9 @@ public void Flush()
if (offset > sendObject.obj.bufferPtr + sizeof(int) + BatchHeader.Size)
{
int payloadSize = (int)(offset - sendObject.obj.bufferPtr);
- ((BatchHeader*)(sendObject.obj.bufferPtr + sizeof(int)))->numMessages = numMessages;
+
+ // seqNo and wire format are default (0)
+ ((BatchHeader*)(sendObject.obj.bufferPtr + sizeof(int)))->NumMessages = numMessages;
Interlocked.Increment(ref numPendingBatches);
// Set packet size in header
@@ -255,8 +257,8 @@ internal void ProcessReplies(byte[] buf, int offset)
fixed (byte* b = &buf[offset])
{
var src = b;
- var seqNo = ((BatchHeader*)src)->seqNo;
- var count = ((BatchHeader*)src)->numMessages;
+ var seqNo = ((BatchHeader*)src)->SeqNo;
+ var count = ((BatchHeader*)src)->NumMessages;
if (seqNo != lastSeqNo + 1)
throw new Exception("Out of order message within session");
lastSeqNo = seqNo;
diff --git a/cs/remote/src/FASTER.common/BatchHeader.cs b/cs/remote/src/FASTER.common/BatchHeader.cs
index fbaf47292..af2a95dd2 100644
--- a/cs/remote/src/FASTER.common/BatchHeader.cs
+++ b/cs/remote/src/FASTER.common/BatchHeader.cs
@@ -1,15 +1,17 @@
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT license.
+using System.Runtime.CompilerServices;
using System.Runtime.InteropServices;
namespace FASTER.common
{
///
- /// Header for message batch
+ /// Header for message batch (Little Endian server)
+ /// [4 byte seqNo][1 byte protocol][3 byte numMessages]
///
[StructLayout(LayoutKind.Explicit, Size = 8)]
- public unsafe struct BatchHeader
+ public struct BatchHeader
{
///
/// Size
@@ -17,15 +19,37 @@ public unsafe struct BatchHeader
public const int Size = 8;
///
- /// Sequence number for batch
+ /// Sequence number.
///
[FieldOffset(0)]
- public int seqNo;
+ public int SeqNo;
///
- /// Number of messsages packed in batch
+ /// Lower-order 8 bits are protocol type, higher-order 24 bits are num messages.
///
[FieldOffset(4)]
- public int numMessages;
+ private int numMessagesAndProtocolType;
+
+ ///
+ /// Number of messages packed in batch
+ ///
+ public int NumMessages
+ {
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ get => (int)((uint)numMessagesAndProtocolType >> 8);
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ set { numMessagesAndProtocolType = (value << 8) | (numMessagesAndProtocolType & 0xFF); }
+ }
+
+ ///
+ /// Wire protocol this batch is written in
+ ///
+ public WireFormat Protocol
+ {
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ get => (WireFormat)(numMessagesAndProtocolType & 0xFF);
+ [MethodImpl(MethodImplOptions.AggressiveInlining)]
+ set { numMessagesAndProtocolType = (numMessagesAndProtocolType & ~0xFF) | ((int)value & 0xFF); }
+ }
}
}
\ No newline at end of file
diff --git a/cs/remote/src/FASTER.server/WireFormat.cs b/cs/remote/src/FASTER.common/WireFormat.cs
similarity index 82%
rename from cs/remote/src/FASTER.server/WireFormat.cs
rename to cs/remote/src/FASTER.common/WireFormat.cs
index 96d6a7f8e..26b562b09 100644
--- a/cs/remote/src/FASTER.server/WireFormat.cs
+++ b/cs/remote/src/FASTER.common/WireFormat.cs
@@ -1,12 +1,12 @@
// Copyright (c) Microsoft Corporation. All rights reserved.
// Licensed under the MIT license.
-namespace FASTER.server
+namespace FASTER.common
{
///
/// Wire format
///
- public enum WireFormat
+ public enum WireFormat : byte
{
///
/// Custom binary format
diff --git a/cs/remote/src/FASTER.server/BinaryServerSession.cs b/cs/remote/src/FASTER.server/BinaryServerSession.cs
index db4a2efea..ed84faf14 100644
--- a/cs/remote/src/FASTER.server/BinaryServerSession.cs
+++ b/cs/remote/src/FASTER.server/BinaryServerSession.cs
@@ -10,7 +10,7 @@
namespace FASTER.server
{
internal unsafe sealed class BinaryServerSession
- : ServerSessionBase
+ : FasterKVServerSessionBase
where Functions : IFunctions
where ParameterSerializer : IServerSerializer
{
@@ -48,7 +48,7 @@ public override int TryConsumeMessages(byte[] buf)
return bytesRead;
}
- public override void CompleteRead(ref Output output, long ctx, core.Status status)
+ public override void CompleteRead(ref Output output, long ctx, Status status)
{
byte* d = responseObject.obj.bufferPtr;
var dend = d + responseObject.obj.buffer.Length;
@@ -60,12 +60,12 @@ public override void CompleteRead(ref Output output, long ctx, core.Status statu
hrw.Write((MessageType)(ctx >> 32), ref dcurr, (int)(dend - dcurr));
Write((int)(ctx & 0xffffffff), ref dcurr, (int)(dend - dcurr));
Write(ref status, ref dcurr, (int)(dend - dcurr));
- if (status != core.Status.NOTFOUND)
+ if (status != Status.NOTFOUND)
serializer.Write(ref output, ref dcurr, (int)(dend - dcurr));
msgnum++;
}
- public override void CompleteRMW(long ctx, core.Status status)
+ public override void CompleteRMW(long ctx, Status status)
{
byte* d = responseObject.obj.bufferPtr;
var dend = d + responseObject.obj.buffer.Length;
@@ -113,13 +113,15 @@ private unsafe void ProcessBatch(byte[] buf, int offset)
var src = b;
ref var header = ref Unsafe.AsRef(src);
+ var num = header.NumMessages;
src += BatchHeader.Size;
- core.Status status = default;
+ Status status = default;
dcurr += BatchHeader.Size;
start = 0;
msgnum = 0;
- for (msgnum = 0; msgnum < header.numMessages; msgnum++)
+
+ for (msgnum = 0; msgnum < num; msgnum++)
{
var message = (MessageType)(*src++);
switch (message)
@@ -146,9 +148,9 @@ private unsafe void ProcessBatch(byte[] buf, int offset)
hrw.Write(message, ref dcurr, (int)(dend - dcurr));
Write(ref status, ref dcurr, (int)(dend - dcurr));
- if (status == core.Status.PENDING)
+ if (status == Status.PENDING)
Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr));
- else if (status == core.Status.OK)
+ else if (status == Status.OK)
serializer.SkipOutput(ref dcurr);
break;
@@ -162,7 +164,7 @@ private unsafe void ProcessBatch(byte[] buf, int offset)
hrw.Write(message, ref dcurr, (int)(dend - dcurr));
Write(ref status, ref dcurr, (int)(dend - dcurr));
- if (status == core.Status.PENDING)
+ if (status == Status.PENDING)
Write(pendingSeqNo++, ref dcurr, (int)(dend - dcurr));
break;
@@ -193,7 +195,7 @@ private unsafe void ProcessBatch(byte[] buf, int offset)
}
[MethodImpl(MethodImplOptions.AggressiveInlining)]
- private unsafe bool Write(ref core.Status s, ref byte* dst, int length)
+ private unsafe bool Write(ref Status s, ref byte* dst, int length)
{
if (length < 1) return false;
*dst++ = (byte)s;
@@ -224,8 +226,8 @@ private void SendAndReset(ref byte* d, ref byte* dend)
private void Send(byte* d)
{
var dstart = d + sizeof(int);
- Unsafe.AsRef(dstart).numMessages = msgnum - start;
- Unsafe.AsRef(dstart).seqNo = seqNo++;
+ Unsafe.AsRef(dstart).NumMessages = msgnum - start;
+ Unsafe.AsRef(dstart).SeqNo = seqNo++;
int payloadSize = (int)(dcurr - d);
// Set packet size in header
*(int*)responseObject.obj.bufferPtr = -(payloadSize - sizeof(int));
diff --git a/cs/remote/src/FASTER.server/ConnectionArgs.cs b/cs/remote/src/FASTER.server/ConnectionArgs.cs
index 61b9a8dfc..3d26f4db0 100644
--- a/cs/remote/src/FASTER.server/ConnectionArgs.cs
+++ b/cs/remote/src/FASTER.server/ConnectionArgs.cs
@@ -2,21 +2,15 @@
// Licensed under the MIT license.
using System.Net.Sockets;
-using FASTER.core;
using FASTER.common;
-using System;
namespace FASTER.server
{
- class ConnectionArgs
- where Functions : IFunctions
- where ParameterSerializer : IServerSerializer
+ internal class ConnectionArgs
{
public Socket socket;
- public FasterKV store;
- public Func functionsGen;
- public ParameterSerializer serializer;
- public ServerSessionBase session;
+ public ServerSessionBase session;
public MaxSizeSettings maxSizeSettings;
+ public IFasterRemoteBackendProvider provider;
}
}
diff --git a/cs/remote/src/FASTER.server/FasterKVServer.cs b/cs/remote/src/FASTER.server/FasterKVServer.cs
index 8e9ee8085..abe63ec16 100644
--- a/cs/remote/src/FASTER.server/FasterKVServer.cs
+++ b/cs/remote/src/FASTER.server/FasterKVServer.cs
@@ -9,6 +9,7 @@
using System;
using System.Runtime.CompilerServices;
using System.Collections.Concurrent;
+using System.Security.Cryptography;
using System.Threading;
namespace FASTER.server
@@ -23,22 +24,20 @@ public sealed class FasterKVServer, byte> activeSessions;
+ readonly ConcurrentDictionary activeSessions;
int activeSessionCount;
bool disposed;
-
+
///
/// Constructor
///
- /// Instance of FasterKV store to use in server
- /// Functions generator (based on wire format)
+ /// Backend provider for this server
/// IP address
/// Port
- /// Parameter serializer
/// Max size settings
- public FasterKVServer(FasterKV store, Func functionsGen, string address, int port, ParameterSerializer serializer = default, MaxSizeSettings maxSizeSettings = default) : base()
+ public FasterKVServer(IFasterRemoteBackendProvider backendProvider, string address, int port, MaxSizeSettings maxSizeSettings = default)
{
- activeSessions = new ConcurrentDictionary, byte>();
+ activeSessions = new ConcurrentDictionary();
activeSessionCount = 0;
disposed = false;
@@ -50,11 +49,26 @@ public FasterKVServer(FasterKV store, Func fu
servSocket.Listen(512);
acceptEventArg = new SocketAsyncEventArgs
{
- UserToken = (store, functionsGen, serializer)
+ UserToken = backendProvider
};
acceptEventArg.Completed += AcceptEventArg_Completed;
}
+ ///
+ /// Constructor
+ ///
+ /// Instance of FasterKV store to use in server
+ /// Functions generator (based on wire format)
+ /// IP address
+ /// Port
+ /// Parameter serializer
+ /// Max size settings
+ public FasterKVServer(FasterKV store, Func functionsGen, string address,
+ int port, ParameterSerializer serializer = default, MaxSizeSettings maxSizeSettings = default)
+ : this(new FasterKVBackendProvider(store, functionsGen, serializer), address, port, maxSizeSettings)
+ {
+ }
+
///
/// Start server
///
@@ -88,15 +102,13 @@ private bool HandleNewConnection(SocketAsyncEventArgs e)
var buffer = new byte[bufferSize];
receiveEventArgs.SetBuffer(buffer, 0, bufferSize);
- var (store, functionsGen, serializer) = ((FasterKV, Func, ParameterSerializer))e.UserToken;
+ var backendProvider = (IFasterRemoteBackendProvider) e.UserToken;
- var args = new ConnectionArgs
+ var args = new ConnectionArgs
{
socket = e.AcceptSocket,
- store = store,
- functionsGen = functionsGen,
- serializer = serializer,
- maxSizeSettings = maxSizeSettings
+ maxSizeSettings = maxSizeSettings,
+ provider = backendProvider
};
receiveEventArgs.UserToken = args;
@@ -141,7 +153,7 @@ private void DisposeActiveSessions()
[MethodImpl(MethodImplOptions.AggressiveInlining)]
private bool HandleReceiveCompletion(SocketAsyncEventArgs e)
{
- var connArgs = (ConnectionArgs)e.UserToken;
+ var connArgs = (ConnectionArgs) e.UserToken;
if (e.BytesTransferred == 0 || e.SocketError != SocketError.Success || disposed)
{
DisposeConnectionSession(e);
@@ -160,22 +172,30 @@ private bool HandleReceiveCompletion(SocketAsyncEventArgs e)
return true;
}
- private bool CreateSession(SocketAsyncEventArgs e)
+ private unsafe bool CreateSession(SocketAsyncEventArgs e)
{
- var connArgs = (ConnectionArgs)e.UserToken;
+ var connArgs = (ConnectionArgs) e.UserToken;
- if (e.BytesTransferred < 4)
- {
- e.SetBuffer(0, e.Buffer.Length);
- return true;
- }
+ if (e.BytesTransferred < 4) return false;
+
+ // FASTER's protocol family is identified by inverted size field in the start of a packet
+ if (e.Buffer[3] <= 127)
+ throw new FasterException("Unexpected wire format");
+
+ if (e.BytesTransferred < 4 + BatchHeader.Size) return false;
- if (e.Buffer[3] > 127)
+ fixed (void* bh = &e.Buffer[4])
{
- connArgs.session = new BinaryServerSession(connArgs.socket, connArgs.store, connArgs.functionsGen(WireFormat.Binary), connArgs.serializer, connArgs.maxSizeSettings);
+ switch (((BatchHeader *) bh)->Protocol)
+ {
+ case WireFormat.Binary:
+ var backend = connArgs.provider.GetBackendForProtocol>(WireFormat.Binary);
+ connArgs.session = new BinaryServerSession(connArgs.socket, backend.store, backend.functionsGen(WireFormat.Binary), backend.serializer, connArgs.maxSizeSettings);
+ break;
+ default:
+ throw new FasterException("Unexpected wire format");
+ }
}
- else
- throw new FasterException("Unexpected wire format");
if (activeSessions.TryAdd(connArgs.session, default))
Interlocked.Increment(ref activeSessionCount);
@@ -192,7 +212,7 @@ private bool CreateSession(SocketAsyncEventArgs e)
private void DisposeConnectionSession(SocketAsyncEventArgs e)
{
- var connArgs = (ConnectionArgs)e.UserToken;
+ var connArgs = (ConnectionArgs) e.UserToken;
connArgs.socket.Dispose();
e.Dispose();
var _session = connArgs.session;
@@ -208,7 +228,7 @@ private void DisposeConnectionSession(SocketAsyncEventArgs e)
private void RecvEventArg_Completed(object sender, SocketAsyncEventArgs e)
{
- var connArgs = (ConnectionArgs)e.UserToken;
+ var connArgs = (ConnectionArgs) e.UserToken;
do
{
// No more things to receive
diff --git a/cs/remote/src/FASTER.server/FasterKVServerSessionBase.cs b/cs/remote/src/FASTER.server/FasterKVServerSessionBase.cs
new file mode 100644
index 000000000..32e7bf440
--- /dev/null
+++ b/cs/remote/src/FASTER.server/FasterKVServerSessionBase.cs
@@ -0,0 +1,33 @@
+using System.Net.Sockets;
+using FASTER.common;
+using FASTER.core;
+
+namespace FASTER.server
+{
+ internal abstract class FasterKVServerSessionBase : ServerSessionBase
+ where Functions : IFunctions
+ where ParameterSerializer : IServerSerializer
+ {
+ protected readonly ClientSession> session;
+ protected readonly ParameterSerializer serializer;
+
+ public FasterKVServerSessionBase(Socket socket, FasterKV store, Functions functions,
+ ParameterSerializer serializer, MaxSizeSettings maxSizeSettings)
+ : base(socket, maxSizeSettings)
+ {
+ session = store.For(new ServerKVFunctions(functions, this))
+ .NewSession>();
+ this.serializer = serializer;
+ }
+
+ public abstract void CompleteRead(ref Output output, long ctx, Status status);
+ public abstract void CompleteRMW(long ctx, Status status);
+
+
+ public override void Dispose()
+ {
+ session.Dispose();
+ base.Dispose();
+ }
+ }
+}
\ No newline at end of file
diff --git a/cs/remote/src/FASTER.server/FasterRemoteBackendProvider.cs b/cs/remote/src/FASTER.server/FasterRemoteBackendProvider.cs
new file mode 100644
index 000000000..734d4a759
--- /dev/null
+++ b/cs/remote/src/FASTER.server/FasterRemoteBackendProvider.cs
@@ -0,0 +1,80 @@
+using System;
+using System.Runtime.CompilerServices;
+using FASTER.common;
+using FASTER.core;
+
+namespace FASTER.server
+{
+ ///
+ /// Provides backend store (e.g., FasterLog, FasterKV) of the Faster Server
+ ///
+ public interface IFasterRemoteBackendProvider : IDisposable
+ {
+ ///
+ /// Given messages of wire format type, returns the backend object for that type. If no backend is configured
+ /// for the given type or if the supplied BackendType does not match the type of object configured, will
+ /// throw an exception
+ ///
+ /// the wire format type
+ /// object type of the configured backend for given type
+ /// configured backend object
+ BackendType GetBackendForProtocol(WireFormat type);
+ }
+
+ public class FasterKVBackend : IDisposable
+ {
+ internal FasterKV store;
+ internal Func functionsGen;
+ internal ParameterSerializer serializer;
+
+ internal FasterKVBackend(FasterKV store, Func functionsGen,
+ ParameterSerializer serializer = default)
+ {
+ this.store = store;
+ this.functionsGen = functionsGen;
+ this.serializer = serializer;
+ }
+
+ public void Dispose()
+ {
+ store.Dispose();
+ }
+ }
+
+ public class FasterKVBackendProvider : IFasterRemoteBackendProvider
+ {
+ private FasterKVBackend backend;
+
+ public FasterKVBackendProvider(FasterKVBackend backend)
+ {
+ this.backend = backend;
+ }
+
+ public FasterKVBackendProvider(FasterKV store, Func functionsGen,
+ ParameterSerializer serializer = default)
+ {
+ backend =
+ new FasterKVBackend(store, functionsGen, serializer);
+ }
+
+ public BackendType GetBackendForProtocol(WireFormat type)
+ {
+ if (type != WireFormat.Binary)
+ throw new InvalidOperationException(
+ $"Backend required for protocol {type.ToString()} is not supported on this server");
+ if (typeof(BackendType) != typeof(FasterKVBackend))
+ {
+ throw new InvalidCastException(
+ $"Backend is configured to provide {typeof(FasterKVBackend)} to" +
+ $"protocol {type.ToString()}, instead, {typeof(BackendType)} was requested");
+ }
+ // Throws invalid cast exception if user calls with wrong type.
+ return (BackendType) (object) backend;
+ }
+
+ public void Dispose()
+ {
+ backend.Dispose();
+ }
+ }
+}
\ No newline at end of file
diff --git a/cs/remote/src/FASTER.server/ServerFunctions.cs b/cs/remote/src/FASTER.server/ServerKVFunctions.cs
similarity index 86%
rename from cs/remote/src/FASTER.server/ServerFunctions.cs
rename to cs/remote/src/FASTER.server/ServerKVFunctions.cs
index 5895bb084..9cff220d7 100644
--- a/cs/remote/src/FASTER.server/ServerFunctions.cs
+++ b/cs/remote/src/FASTER.server/ServerKVFunctions.cs
@@ -6,16 +6,16 @@
namespace FASTER.server
{
- internal struct ServerFunctions : IFunctions
+ internal struct ServerKVFunctions : IFunctions
where Functions : IFunctions
where ParameterSerializer : IServerSerializer
{
private readonly Functions functions;
- private readonly ServerSessionBase serverNetworkSession;
+ private readonly FasterKVServerSessionBase serverNetworkSession;
public bool SupportsLocking => functions.SupportsLocking;
- public ServerFunctions(Functions functions, ServerSessionBase serverNetworkSession)
+ public ServerKVFunctions(Functions functions, FasterKVServerSessionBase serverNetworkSession)
{
this.functions = functions;
this.serverNetworkSession = serverNetworkSession;
@@ -45,13 +45,13 @@ public void InitialUpdater(ref Key key, ref Input input, ref Value value)
public bool InPlaceUpdater(ref Key key, ref Input input, ref Value value)
=> functions.InPlaceUpdater(ref key, ref input, ref value);
- public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, long ctx, core.Status status)
+ public void ReadCompletionCallback(ref Key key, ref Input input, ref Output output, long ctx, Status status)
{
serverNetworkSession.CompleteRead(ref output, ctx, status);
functions.ReadCompletionCallback(ref key, ref input, ref output, ctx, status);
}
- public void RMWCompletionCallback(ref Key key, ref Input input, long ctx, core.Status status)
+ public void RMWCompletionCallback(ref Key key, ref Input input, long ctx, Status status)
{
serverNetworkSession.CompleteRMW(ctx, status);
functions.RMWCompletionCallback(ref key, ref input, ctx, status);
diff --git a/cs/remote/src/FASTER.server/ServerSessionBase.cs b/cs/remote/src/FASTER.server/ServerSessionBase.cs
index 272635d44..4d499ee2d 100644
--- a/cs/remote/src/FASTER.server/ServerSessionBase.cs
+++ b/cs/remote/src/FASTER.server/ServerSessionBase.cs
@@ -8,36 +8,27 @@
namespace FASTER.server
{
- internal abstract class ServerSessionBase : IDisposable
- where Functions : IFunctions
- where ParameterSerializer : IServerSerializer
+ internal abstract class ServerSessionBase : IDisposable
{
protected readonly Socket socket;
- protected readonly ClientSession> session;
- protected readonly ParameterSerializer serializer;
protected readonly MaxSizeSettings maxSizeSettings;
-
private readonly NetworkSender messageManager;
private readonly int serverBufferSize;
protected ReusableObject responseObject;
protected int bytesRead;
- public ServerSessionBase(Socket socket, FasterKV store, Functions functions, ParameterSerializer serializer, MaxSizeSettings maxSizeSettings)
+ public ServerSessionBase(Socket socket, MaxSizeSettings maxSizeSettings)
{
this.socket = socket;
- session = store.For(new ServerFunctions(functions, this)).NewSession>();
this.maxSizeSettings = maxSizeSettings;
serverBufferSize = BufferSizeUtils.ServerBufferSize(maxSizeSettings);
messageManager = new NetworkSender(serverBufferSize);
- this.serializer = serializer;
-
bytesRead = 0;
}
public abstract int TryConsumeMessages(byte[] buf);
- public abstract void CompleteRead(ref Output output, long ctx, core.Status status);
- public abstract void CompleteRMW(long ctx, core.Status status);
+
protected void GetResponseObject() { if (responseObject.obj == null) responseObject = messageManager.GetReusableSeaaBuffer(); }
@@ -65,15 +56,12 @@ protected void SendResponse(int offset, int size)
}
public void AddBytesRead(int bytesRead) => this.bytesRead += bytesRead;
-
-
-
+
///
/// Dispose
///
public virtual void Dispose()
{
- session.Dispose();
socket.Dispose();
if (responseObject.obj != null)
responseObject.Dispose();
diff --git a/cs/remote/src/FASTER.server/SpanByteFunctionsForServer.cs b/cs/remote/src/FASTER.server/SpanByteFunctionsForServer.cs
index 22f05687e..babb1a752 100644
--- a/cs/remote/src/FASTER.server/SpanByteFunctionsForServer.cs
+++ b/cs/remote/src/FASTER.server/SpanByteFunctionsForServer.cs
@@ -2,6 +2,7 @@
// Licensed under the MIT license.
using System.Buffers;
+using FASTER.common;
using FASTER.core;
namespace FASTER.server
diff --git a/cs/remote/test/FASTER.remote.test/FixedLenServer.cs b/cs/remote/test/FASTER.remote.test/FixedLenServer.cs
index 52140b7c9..0b8e915f0 100644
--- a/cs/remote/test/FASTER.remote.test/FixedLenServer.cs
+++ b/cs/remote/test/FASTER.remote.test/FixedLenServer.cs
@@ -2,6 +2,7 @@
using FASTER.server;
using System;
using System.IO;
+using System.Security.Cryptography;
namespace FASTER.remote.test
{
@@ -21,10 +22,14 @@ public FixedLenServer(string folderName, Func merger, strin
// We use blittable structs Key and Value to construct a costomized server for fixed-length types
store = new FasterKV(indexSize, logSettings, checkpointSettings);
+ var provider =
+ new FasterKVBackendProvider,
+ FixedLenSerializer>(
+ store, e => new FixedLenServerFunctions(merger));
// We specify FixedLenSerializer as our in-built serializer for blittable (fixed length) types
// This server can be used with compatible clients such as FixedLenClient and FASTER.benchmark
server = new FasterKVServer, FixedLenSerializer>
- (store, e => new FixedLenServerFunctions(merger), address, port);
+ (provider, address, port);
server.Start();
}
diff --git a/cs/remote/test/FASTER.remote.test/VarLenServer.cs b/cs/remote/test/FASTER.remote.test/VarLenServer.cs
index 0760bd94c..6bf608147 100644
--- a/cs/remote/test/FASTER.remote.test/VarLenServer.cs
+++ b/cs/remote/test/FASTER.remote.test/VarLenServer.cs
@@ -19,10 +19,13 @@ public VarLenServer(string folderName, string address = "127.0.0.1", int port =
// We use blittable structs Key and Value to construct a costomized server for fixed-length types
store = new FasterKV(indexSize, logSettings, checkpointSettings);
+ var provider =
+ new FasterKVBackendProvider, SpanByteSerializer>(
+ store, wp => new SpanByteFunctionsForServer(wp), new SpanByteSerializer());
// We specify FixedLenSerializer as our in-built serializer for blittable (fixed length) types
// This server can be used with compatible clients such as FixedLenClient and FASTER.benchmark
server = server = new FasterKVServer, SpanByteSerializer>
- (store, wp => new SpanByteFunctionsForServer(wp), address, port, new SpanByteSerializer(), default);
+ (provider, address, port);
server.Start();
}