+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// where to send results
+ /// multi-record transaction
+ public void Commit(CommitListener listener, Tran tran)
+ {
+ tran.SetRollAttempted();
+
+ //AsyncTranRoll tm = new AsyncTranRoll(
+ //cluster, tranVerifyPolicyDefault, tranRollPolicyDefault, tran
+ //);
+ //tm.Commit(listener);
+ }
+
+ ///
+ /// Asynchronously abort and rollback the given multi-record transaction.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// where to send results
+ /// multi-record transaction
+ public void Abort(AbortListener listener, Tran tran)
+ {
+ tran.SetRollAttempted();
+
+ //AsyncTranRoll tm = new AsyncTranRoll(cluster, null, tranRollPolicyDefault, tran);
+ //tm.Abort(listener);
+ }
+
//-------------------------------------------------------
// Write Record Operations
//-------------------------------------------------------
@@ -1115,7 +1161,7 @@ public Task Operate(WritePolicy policy, CancellationToken token, Key key
/// if queue is full
public void Operate(WritePolicy policy, RecordListener listener, Key key, params Operation[] ops)
{
- OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, key, ops);
+ OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, ops);
AsyncOperate async = new AsyncOperate(cluster, listener, key, args);
async.Execute();
}
diff --git a/AerospikeClient/Async/IAsyncClient.cs b/AerospikeClient/Async/IAsyncClient.cs
index 3b82b58d..70d130aa 100644
--- a/AerospikeClient/Async/IAsyncClient.cs
+++ b/AerospikeClient/Async/IAsyncClient.cs
@@ -14,9 +14,6 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
-using System.Collections.Generic;
-using System.Threading;
-using System.Threading.Tasks;
namespace Aerospike.Client
{
@@ -39,7 +36,39 @@ namespace Aerospike.Client
///
///
public interface IAsyncClient : IAerospikeClient
- {
+ {
+ //-------------------------------------------------------
+ // Multi-Record Transactions
+ //-------------------------------------------------------
+
+ ///
+ /// Asynchronously attempt to commit the given multi-record transaction. First, the expected
+ /// record versions are sent to the server nodes for verification.If all nodes return success,
+ /// the transaction is committed.Otherwise, the transaction is aborted.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// where to send results
+ /// multi-record transaction
+ void Commit(CommitListener listener, Tran tran);
+
+ ///
+ /// Asynchronously abort and rollback the given multi-record transaction.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ ///
+ ///
+ void Abort(AbortListener listener, Tran tran);
+
//-------------------------------------------------------
// Write Record Operations
//-------------------------------------------------------
diff --git a/AerospikeClient/Command/Batch.cs b/AerospikeClient/Command/Batch.cs
index 4ecbb5c8..fee63f92 100644
--- a/AerospikeClient/Command/Batch.cs
+++ b/AerospikeClient/Command/Batch.cs
@@ -42,7 +42,7 @@ protected internal override void WriteBuffer()
{
if (batch.node != null && batch.node.HasBatchAny)
{
- SetBatchOperate(batchPolicy, records, batch);
+ SetBatchOperate(batchPolicy, null, null, null, records, batch);
}
else
{
@@ -52,10 +52,10 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
-
BatchRead record = records[batchIndex];
+ ParseFieldsRead(record.key);
+
if (resultCode == 0)
{
record.SetRecord(ParseRecord());
@@ -116,7 +116,7 @@ protected internal override void WriteBuffer()
{
if (batch.node != null && batch.node.HasBatchAny)
{
- BatchAttr attr = new BatchAttr(policy, readAttr, ops);
+ BatchAttr attr = new(policy, readAttr, ops);
SetBatchOperate(batchPolicy, keys, batch, binNames, ops, attr);
}
else
@@ -127,7 +127,7 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
+ ParseFieldsRead(keys[batchIndex]);
if (resultCode == 0)
{
@@ -174,7 +174,7 @@ protected internal override void WriteBuffer()
{
if (batch.node != null && batch.node.HasBatchAny)
{
- BatchAttr attr = new BatchAttr(policy, Command.INFO1_READ | Command.INFO1_NOBINDATA);
+ BatchAttr attr = new(policy, Command.INFO1_READ | Command.INFO1_NOBINDATA);
SetBatchOperate(batchPolicy, keys, batch, null, null, attr);
}
else
@@ -185,7 +185,7 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
+ ParseFieldsRead(keys[batchIndex]);
if (opCount > 0)
{
@@ -236,15 +236,15 @@ protected internal override bool IsWrite()
protected internal override void WriteBuffer()
{
- SetBatchOperate(batchPolicy, (IList)records, batch);
+ SetBatchOperate(batchPolicy, null, null, null, (IList)records, batch);
}
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
-
BatchRecord record = records[batchIndex];
+ ParseFields(record);
+
if (resultCode == 0)
{
record.SetRecord(ParseRecord());
@@ -342,10 +342,10 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
-
BatchRecord record = records[batchIndex];
+ ParseFields(record);
+
if (resultCode == 0)
{
record.SetRecord(ParseRecord());
@@ -434,10 +434,10 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
- SkipKey(fieldCount);
-
BatchRecord record = records[batchIndex];
+ ParseFields(record);
+
if (resultCode == 0)
{
record.SetRecord(ParseRecord());
@@ -494,6 +494,151 @@ protected internal override List GenerateBatchNodes()
}
}
+ //-------------------------------------------------------
+ // MRT
+ //-------------------------------------------------------
+
+ public sealed class BatchTranVerify : BatchCommand
+ {
+ private readonly Tran tran;
+ private readonly Key[] keys;
+ private readonly long[] versions;
+ private readonly BatchRecord[] records;
+
+ public BatchTranVerify(
+ Cluster cluster,
+ BatchNode batch,
+ BatchPolicy batchPolicy,
+ Tran tran,
+ Key[] keys,
+ long[] versions,
+ BatchRecord[] records,
+ BatchStatus status
+ ) : base(cluster, batch, batchPolicy, status, false)
+ {
+ this.tran = tran;
+ this.keys = keys;
+ this.versions = versions;
+ this.records = records;
+ }
+
+ protected internal override bool IsWrite()
+ {
+ return false;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetBatchTranVerify(batchPolicy, tran, keys, versions, batch);
+ }
+
+ protected internal override bool ParseRow()
+ {
+ SkipKey(fieldCount);
+
+ BatchRecord record = records[batchIndex];
+
+ if (resultCode == 0)
+ {
+ record.resultCode = resultCode;
+ }
+ else
+ {
+ record.SetError(resultCode, false);
+ status.SetRowError();
+ }
+ return true;
+ }
+
+ protected internal override BatchCommand CreateCommand(BatchNode batchNode)
+ {
+ return new BatchTranVerify(cluster, batchNode, batchPolicy, tran, keys, versions, records, status);
+ }
+
+ protected internal override List GenerateBatchNodes()
+ {
+ return BatchNode.GenerateList(cluster, batchPolicy, keys, records, sequenceAP, sequenceSC, batch, false, status);
+ }
+ }
+
+ public sealed class BatchTranRoll : BatchCommand
+ {
+ private readonly Key[] keys;
+ private readonly BatchRecord[] records;
+ private readonly BatchAttr attr;
+
+ public BatchTranRoll(
+ Cluster cluster,
+ BatchNode batch,
+ BatchPolicy batchPolicy,
+ Key[] keys,
+ BatchRecord[] records,
+ BatchAttr attr,
+ BatchStatus status
+ ) : base(cluster, batch, batchPolicy, status, false)
+ {
+ this.keys = keys;
+ this.records = records;
+ this.attr = attr;
+ }
+
+ protected internal override bool IsWrite()
+ {
+ return attr.hasWrite;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetBatchTranRoll(batchPolicy, keys, batch, attr);
+ }
+
+ protected internal override bool ParseRow()
+ {
+ SkipKey(fieldCount);
+
+ BatchRecord record = records[batchIndex];
+
+ if (resultCode == 0)
+ {
+ record.resultCode = resultCode;
+ }
+ else
+ {
+ record.SetError(resultCode, Command.BatchInDoubt(attr.hasWrite, commandSentCounter));
+ status.SetRowError();
+ }
+ return true;
+ }
+
+ protected internal override void SetInDoubt(bool inDoubt)
+ {
+ if (!inDoubt || !attr.hasWrite)
+ {
+ return;
+ }
+
+ foreach (int index in batch.offsets)
+ {
+ BatchRecord record = records[index];
+
+ if (record.resultCode == ResultCode.NO_RESPONSE)
+ {
+ record.inDoubt = true;
+ }
+ }
+ }
+
+ protected internal override BatchCommand CreateCommand(BatchNode batchNode)
+ {
+ return new BatchTranRoll(cluster, batchNode, batchPolicy, keys, records, attr, status);
+ }
+
+ protected internal override List GenerateBatchNodes()
+ {
+ return BatchNode.GenerateList(cluster, batchPolicy, keys, records, sequenceAP, sequenceSC, batch, attr.hasWrite, status);
+ }
+ }
+
//-------------------------------------------------------
// Batch Base Command
//-------------------------------------------------------
@@ -554,6 +699,40 @@ public void Run(object obj)
}
}
+ protected void ParseFieldsRead(Key key)
+ {
+ if (policy.Tran != null)
+ {
+ long? version = ParseVersion(fieldCount);
+ policy.Tran.OnRead(key, version);
+ }
+ else
+ {
+ SkipKey(fieldCount);
+ }
+ }
+
+ protected void ParseFields(BatchRecord br)
+ {
+ if (policy.Tran != null)
+ {
+ long? version = ParseVersion(fieldCount);
+
+ if (br.hasWrite)
+ {
+ policy.Tran.OnWrite(br.key, version, resultCode);
+ }
+ else
+ {
+ policy.Tran.OnRead(br.key, version);
+ }
+ }
+ else
+ {
+ SkipKey(fieldCount);
+ }
+ }
+
protected override Latency.LatencyType GetLatencyType()
{
return Latency.LatencyType.BATCH;
diff --git a/AerospikeClient/Command/BatchAttr.cs b/AerospikeClient/Command/BatchAttr.cs
index 65c13b13..42839e40 100644
--- a/AerospikeClient/Command/BatchAttr.cs
+++ b/AerospikeClient/Command/BatchAttr.cs
@@ -22,7 +22,9 @@ public sealed class BatchAttr
public int readAttr;
public int writeAttr;
public int infoAttr;
+ public int tranAttr;
public int expiration;
+ public int opSize;
public short generation;
public bool hasWrite;
public bool sendKey;
@@ -394,5 +396,30 @@ public void SetDelete(BatchDeletePolicy dp)
infoAttr |= Command.INFO3_COMMIT_MASTER;
}
}
+
+ public void SetOpSize(Operation[] ops)
+ {
+ int dataOffset = 0;
+
+ foreach (Operation op in ops)
+ {
+ dataOffset += ByteUtil.EstimateSizeUtf8(op.binName) + Command.OPERATION_HEADER_SIZE;
+ dataOffset += op.value.EstimateSize();
+ }
+ opSize = dataOffset;
+ }
+
+ public void SetTran(int attr)
+ {
+ filterExp = null;
+ readAttr = 0;
+ writeAttr = Command.INFO2_WRITE | Command.INFO2_RESPOND_ALL_OPS | Command.INFO2_DURABLE_DELETE;
+ infoAttr = 0;
+ tranAttr = attr;
+ expiration = 0;
+ generation = 0;
+ hasWrite = true;
+ sendKey = false;
+ }
}
}
diff --git a/AerospikeClient/Command/ByteUtil.cs b/AerospikeClient/Command/ByteUtil.cs
index fbb2a2e5..c01faea0 100644
--- a/AerospikeClient/Command/ByteUtil.cs
+++ b/AerospikeClient/Command/ByteUtil.cs
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2023 Aerospike, Inc.
+ * Copyright 2012-2024 Aerospike, Inc.
*
* Portions may be licensed to Aerospike, Inc. under one or more contributor
* license agreements.
@@ -424,15 +424,52 @@ public static long LittleBytesToLong(byte[] buf, int offset)
);
}
- //-------------------------------------------------------
- // 32 bit number conversions.
- //-------------------------------------------------------
-
- ///
- /// Convert int to big endian 32 bits.
- /// The bit pattern will be the same regardless of sign.
+ //-------------------------------------------------------
+ // Transaction version conversions.
+ //-------------------------------------------------------
+
+ ///
+ /// Convert long to a 7 byte record version for MRT.
+ ///
+ public static void LongToVersionBytes(long v, byte[] buf, int offset)
+ {
+ buf[offset++] = (byte)(v >> 0);
+ buf[offset++] = (byte)(v >> 8);
+ buf[offset++] = (byte)(v >> 16);
+ buf[offset++] = (byte)(v >> 24);
+ buf[offset++] = (byte)(v >> 32);
+ buf[offset++] = (byte)(v >> 40);
+ buf[offset] = (byte)(v >> 48);
+ }
+
+ ///
+ /// Convert 7 byte record version to a long for MRT.
///
- public static int IntToBytes(uint v, byte[] buf, int offset)
+ ///
+ ///
+ ///
+ public static long VersionBytesToLong(byte[] buf, int offset)
+ {
+ return (
+ ((long)(buf[offset] & 0xFF) << 0) |
+ ((long)(buf[offset + 1] & 0xFF) << 8) |
+ ((long)(buf[offset + 2] & 0xFF) << 16) |
+ ((long)(buf[offset + 3] & 0xFF) << 24) |
+ ((long)(buf[offset + 4] & 0xFF) << 32) |
+ ((long)(buf[offset + 5] & 0xFF) << 40) |
+ ((long)(buf[offset + 6] & 0xFF) << 48)
+ );
+ }
+
+ //-------------------------------------------------------
+ // 32 bit number conversions.
+ //-------------------------------------------------------
+
+ ///
+ /// Convert int to big endian 32 bits.
+ /// The bit pattern will be the same regardless of sign.
+ ///
+ public static int IntToBytes(uint v, byte[] buf, int offset)
{
// Benchmarks show that custom conversion is faster than System.BitConverter.GetBytes().
// Assume little endian machine and reverse/convert in one pass.
diff --git a/AerospikeClient/Command/Command.cs b/AerospikeClient/Command/Command.cs
index 0f0fe138..2b50cfe5 100644
--- a/AerospikeClient/Command/Command.cs
+++ b/AerospikeClient/Command/Command.cs
@@ -63,6 +63,10 @@ public abstract class Command
// 1 0 allow replica
// 1 1 allow unavailable
+ public const int INFO4_MRT_VERIFY_READ = (1 << 0); // Send MRT version to the server to be verified.
+ public const int INFO4_MRT_ROLL_FORWARD = (1 << 1); // Roll forward MRT.
+ public const int INFO4_MRT_ROLL_BACK = (1 << 2); // Roll back MRT.
+
public const byte STATE_READ_AUTH_HEADER = 1;
public const byte STATE_READ_HEADER = 2;
public const byte STATE_READ_DETAIL = 3;
@@ -73,12 +77,12 @@ public abstract class Command
public const byte BATCH_MSG_INFO = 0x2;
public const byte BATCH_MSG_GEN = 0x4;
public const byte BATCH_MSG_TTL = 0x8;
+ public const byte BATCH_MSG_INFO4 = 0x10;
public const int MSG_TOTAL_HEADER_SIZE = 30;
public const int FIELD_HEADER_SIZE = 5;
public const int OPERATION_HEADER_SIZE = 8;
public const int MSG_REMAINING_HEADER_SIZE = 22;
- public const int DIGEST_SIZE = 20;
public const int COMPRESS_THRESHOLD = 128;
public const ulong CL_MSG_VERSION = 2UL;
public const ulong AS_MSG_TYPE = 3UL;
@@ -90,6 +94,7 @@ public abstract class Command
internal readonly int serverTimeout;
internal int socketTimeout;
internal int totalTimeout;
+ internal long? Version;
public Command(int socketTimeout, int totalTimeout, int maxRetries)
{
@@ -108,6 +113,337 @@ public Command(int socketTimeout, int totalTimeout, int maxRetries)
}
}
+ //--------------------------------------------------
+ // Multi-record Transactions
+ //--------------------------------------------------
+
+ public void SetTranAddKeys(WritePolicy policy, Key key, OperateArgs args)
+ {
+ Begin();
+ int fieldCount = EstimateKeySize(key);
+ dataOffset += args.size;
+ WriteTranMonitor(key, args.readAttr, args.writeAttr, fieldCount, args.operations.Length);
+
+ foreach (Operation operation in args.operations)
+ {
+ WriteOperation(operation);
+ }
+ End(policy.compress);
+ }
+
+ public void SetTranVerify(Tran tran, Key key, long ver)
+ {
+ Begin();
+ int fieldCount = EstimateKeySize(key);
+
+ // Version field.
+ dataOffset += 7;
+ fieldCount++;
+
+ SizeBuffer();
+ dataBuffer[dataOffset++] = MSG_REMAINING_HEADER_SIZE;
+ dataBuffer[dataOffset++] = (byte)(Command.INFO1_READ | Command.INFO1_NOBINDATA);
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = (byte)Command.INFO3_SC_READ_TYPE;
+ dataBuffer[dataOffset++] = (byte)Command.INFO4_MRT_VERIFY_READ;
+ dataBuffer[dataOffset++] = 0;
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes((uint)serverTimeout, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)fieldCount, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes(0, dataBuffer, dataOffset);
+
+ WriteKey(key);
+ WriteFieldVersion(ver);
+ End();
+ }
+
+ public void SetBatchTranVerify(
+ BatchPolicy policy,
+ Tran tran,
+ Key[] keys,
+ long[] versions,
+ BatchNode batch
+ )
+ {
+ BatchOffsetsNative offsets = new(batch);
+ SetBatchTranVerify(policy, tran, keys, versions, offsets);
+ }
+
+ public void SetBatchTranVerify(
+ BatchPolicy policy,
+ Tran tran,
+ Key[] keys,
+ long[] versions,
+ BatchOffsets offsets
+ )
+ {
+ // Estimate buffer size.
+ Begin();
+
+ // Batch field
+ dataOffset += FIELD_HEADER_SIZE + 5;
+
+ Key prev = null;
+ int max = offsets.Size();
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions[offset];
+
+ dataOffset += key.digest.Length + 4;
+
+ if (CanRepeat(key, prev, ver))
+ {
+ // Can set repeat previous namespace/bin names to save space.
+ dataOffset++;
+ }
+ else
+ {
+ // Write full header and namespace/set/bin names.
+ dataOffset += 9; // header(4) + fieldCount(2) + opCount(2) = 9
+ dataOffset += ByteUtil.EstimateSizeUtf8(key.ns) + FIELD_HEADER_SIZE;
+ dataOffset += ByteUtil.EstimateSizeUtf8(key.setName) + FIELD_HEADER_SIZE;
+
+ if (ver != null)
+ {
+ dataOffset += 7 + FIELD_HEADER_SIZE;
+ }
+ prev = key;
+ }
+ }
+
+ SizeBuffer();
+
+ WriteBatchHeader(policy, totalTimeout, 1);
+
+ int fieldSizeOffset = dataOffset;
+ WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
+
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
+ dataBuffer[dataOffset++] = GetBatchFlags(policy);
+ prev = null;
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions[offset];
+
+ dataOffset += ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
+
+ byte[] digest = key.digest;
+ Array.Copy(digest, 0, dataBuffer, dataOffset, digest.Length);
+ dataOffset += digest.Length;
+
+ if (CanRepeat(key, prev, ver))
+ {
+ // Can set repeat previous namespace/bin names to save space.
+ dataBuffer[dataOffset++] = BATCH_MSG_REPEAT;
+ }
+ else
+ {
+ // Write full message.
+ dataBuffer[dataOffset++] = (byte)(BATCH_MSG_INFO | BATCH_MSG_INFO4);
+ dataBuffer[dataOffset++] = (byte)(Command.INFO1_READ | Command.INFO1_NOBINDATA);
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = (byte)Command.INFO3_SC_READ_TYPE;
+ dataBuffer[dataOffset++] = (byte)Command.INFO4_MRT_VERIFY_READ;
+
+ int fieldCount = 0;
+
+ if (ver.HasValue)
+ {
+ fieldCount++;
+ }
+
+ WriteBatchFields(key, fieldCount, 0);
+
+ if (ver.HasValue)
+ {
+ WriteFieldVersion(ver.Value);
+ }
+
+ prev = key;
+ }
+ }
+
+ // Write real field size.
+ ByteUtil.IntToBytes((uint)dataOffset - MSG_TOTAL_HEADER_SIZE - 4, dataBuffer, fieldSizeOffset);
+ End(policy.compress);
+ }
+
+ public void SetTranMarkRollForward(Tran tran, Key key)
+ {
+ Bin bin = new("fwd", true);
+
+ Begin();
+ int fieldCount = EstimateKeySize(key);
+ EstimateOperationSize(bin);
+ WriteTranMonitor(key, 0, Command.INFO2_WRITE, fieldCount, 1);
+ WriteOperation(bin, Operation.Type.WRITE);
+ End();
+ }
+
+ public void SetTranRoll(Key key, Tran tran, int tranAttr)
+ {
+ Begin();
+ int fieldCount = EstimateKeySize(key);
+
+ fieldCount += SizeTran(key, tran, false);
+
+ SizeBuffer();
+ dataBuffer[dataOffset++] = MSG_REMAINING_HEADER_SIZE;
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = (byte)(Command.INFO2_WRITE | Command.INFO2_DURABLE_DELETE);
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = (byte)tranAttr;
+ dataBuffer[dataOffset++] = 0; // clear the result code
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes((uint)serverTimeout, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)fieldCount, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes(0, dataBuffer, dataOffset);
+
+ WriteKey(key);
+ WriteTran(tran, false);
+ End();
+ }
+
+ public void SetBatchTranRoll(
+ BatchPolicy policy,
+ Key[] keys,
+ BatchNode batch,
+ BatchAttr attr
+ )
+ {
+ BatchOffsetsNative offsets = new(batch);
+ SetBatchTranRoll(policy, keys, attr, offsets);
+ }
+
+ public void SetBatchTranRoll(
+ BatchPolicy policy,
+ Key[] keys,
+ BatchAttr attr,
+ BatchOffsets offsets
+ )
+ {
+ // Estimate buffer size.
+ Begin();
+ int fieldCount = 1;
+ int max = offsets.Size();
+ Tran tran = policy.Tran;
+ long[] versions = new long[max];
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ versions[i] = tran.GetReadVersion(key);
+ }
+
+ // Batch field
+ dataOffset += FIELD_HEADER_SIZE + 5;
+
+ Key prev = null;
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long ver = versions[i];
+
+ dataOffset += key.digest.Length + 4;
+
+ if (CanRepeat(key, prev, ver))
+ {
+ // Can set repeat previous namespace/bin names to save space.
+ dataOffset++;
+ }
+ else
+ {
+ // Write full header and namespace/set/bin names.
+ dataOffset += 13; // header(4) + ttl(4) + fieldCount(2) + opCount(2) = 13
+ dataOffset += ByteUtil.EstimateSizeUtf8(key.ns) + FIELD_HEADER_SIZE;
+ dataOffset += ByteUtil.EstimateSizeUtf8(key.setName) + FIELD_HEADER_SIZE;
+ SizeTranBatch(tran, ver);
+ dataOffset += 2; // gen(2) = 2
+ prev = key;
+ }
+ }
+
+ SizeBuffer();
+
+ WriteBatchHeader(policy, totalTimeout, fieldCount);
+
+ int fieldSizeOffset = dataOffset;
+ WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
+
+ ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
+ dataOffset += 4;
+ dataBuffer[dataOffset++] = GetBatchFlags(policy);
+ prev = null;
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long ver = versions[i];
+
+ dataOffset += ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
+
+ byte[] digest = key.digest;
+ Array.Copy(digest, 0, dataBuffer, dataOffset, digest.Length);
+ dataOffset += digest.Length;
+
+ if (CanRepeat(key, prev, ver))
+ {
+ // Can set repeat previous namespace/bin names to save space.
+ dataBuffer[dataOffset++] = BATCH_MSG_REPEAT;
+ }
+ else
+ {
+ // Write full message.
+ WriteBatchWrite(key, tran, ver, attr, null, 0, 0);
+ prev = key;
+ }
+ }
+
+ // Write real field size.
+ ByteUtil.IntToBytes((uint)dataOffset - MSG_TOTAL_HEADER_SIZE - 4, dataBuffer, fieldSizeOffset);
+ End(policy.compress);
+ }
+
+ public void SetTranClose(Tran tran, Key key)
+ {
+ Begin();
+ int fieldCount = EstimateKeySize(key);
+ WriteTranMonitor(key, 0, Command.INFO2_WRITE | Command.INFO2_DELETE | Command.INFO2_DURABLE_DELETE,
+ fieldCount, 0);
+ End();
+ }
+
+ private void WriteTranMonitor(Key key, int readAttr, int writeAttr, int fieldCount, int opCount)
+ {
+ SizeBuffer();
+
+ dataBuffer[dataOffset++] = MSG_REMAINING_HEADER_SIZE;
+ dataBuffer[dataOffset++] = (byte)readAttr;
+ dataBuffer[dataOffset++] = (byte)writeAttr;
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = 0;
+ dataBuffer[dataOffset++] = 0;
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes(0, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes((uint)serverTimeout, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)fieldCount, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)opCount, dataBuffer, dataOffset);
+
+ WriteKey(key);
+ }
+
//--------------------------------------------------
// Writes
//--------------------------------------------------
@@ -115,7 +451,7 @@ public Command(int socketTimeout, int totalTimeout, int maxRetries)
public virtual void SetWrite(WritePolicy policy, Operation.Type operation, Key key, Bin[] bins)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, true);
if (policy.filterExp != null)
{
@@ -131,12 +467,9 @@ public virtual void SetWrite(WritePolicy policy, Operation.Type operation, Key k
bool compress = SizeBuffer(policy);
WriteHeaderWrite(policy, Command.INFO2_WRITE, fieldCount, bins.Length);
- WriteKey(policy, key);
+ WriteKey(policy, key, true);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
foreach (Bin bin in bins)
{
@@ -148,7 +481,7 @@ public virtual void SetWrite(WritePolicy policy, Operation.Type operation, Key k
public virtual void SetDelete(WritePolicy policy, Key key)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, true);
if (policy.filterExp != null)
{
@@ -157,19 +490,26 @@ public virtual void SetDelete(WritePolicy policy, Key key)
}
SizeBuffer();
WriteHeaderWrite(policy, Command.INFO2_WRITE | Command.INFO2_DELETE, fieldCount, 0);
- WriteKey(policy, key);
+ WriteKey(policy, key, true);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
+ End();
+ }
+
+ public void SetDelete(Policy policy, Key key, BatchAttr attr)
+ {
+ Begin();
+ Expression exp = GetBatchExpression(policy, attr);
+ int fieldCount = EstimateKeyAttrSize(policy, key, attr, exp);
+ SizeBuffer();
+ WriteKeyAttr(policy, key, attr, exp, fieldCount, 0);
End();
}
public virtual void SetTouch(WritePolicy policy, Key key)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, true);
if (policy.filterExp != null)
{
@@ -179,12 +519,9 @@ public virtual void SetTouch(WritePolicy policy, Key key)
EstimateOperationSize();
SizeBuffer();
WriteHeaderWrite(policy, Command.INFO2_WRITE, fieldCount, 1);
- WriteKey(policy, key);
+ WriteKey(policy, key, true);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
WriteOperation(Operation.Type.TOUCH);
End();
}
@@ -196,7 +533,7 @@ public virtual void SetTouch(WritePolicy policy, Key key)
public virtual void SetExists(Policy policy, Key key)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, false);
if (policy.filterExp != null)
{
@@ -205,93 +542,174 @@ public virtual void SetExists(Policy policy, Key key)
}
SizeBuffer();
WriteHeaderReadHeader(policy, Command.INFO1_READ | Command.INFO1_NOBINDATA, fieldCount, 0);
- WriteKey(policy, key);
+ WriteKey(policy, key, false);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
End();
}
- public virtual void SetRead(Policy policy, Key key)
+ public virtual void SetRead(Policy policy, Key key, string[] binNames)
{
+ int readAttr = Command.INFO1_READ;
+ int opCount = 0;
+
+ if (binNames != null && binNames.Length > 0)
+ {
+ opCount = binNames.Length;
+ }
+ else
+ {
+ readAttr |= Command.INFO1_GET_ALL;
+ }
+
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, false);
if (policy.filterExp != null)
{
dataOffset += policy.filterExp.Size();
fieldCount++;
}
+
+ if (opCount != 0)
+ {
+ foreach (string binName in binNames)
+ {
+ EstimateOperationSize(binName);
+ }
+ }
+
SizeBuffer();
- WriteHeaderRead(policy, serverTimeout, Command.INFO1_READ | Command.INFO1_GET_ALL, 0, 0, fieldCount, 0);
- WriteKey(policy, key);
+ WriteHeaderRead(policy, serverTimeout, readAttr, 0, 0, fieldCount, opCount);
+ WriteKey(policy, key, false);
- if (policy.filterExp != null)
+ policy.filterExp?.Write(this);
+
+ if (opCount != 0)
{
- policy.filterExp.Write(this);
+ foreach (string binName in binNames)
+ {
+ WriteOperation(binName, Operation.Type.READ);
+ }
}
End();
}
- public virtual void SetRead(Policy policy, Key key, string[] binNames)
+ public void SetRead(Policy policy, BatchRead br)
{
- if (binNames != null)
+ Begin();
+
+ BatchReadPolicy rp = br.policy;
+ BatchAttr attr = new();
+ Expression exp;
+ int opCount;
+
+ if (rp != null)
+ {
+ attr.SetRead(rp);
+ exp = rp.filterExp ?? policy.filterExp;
+ }
+ else
{
- Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ attr.SetRead(policy);
+ exp = policy.filterExp;
+ }
- if (policy.filterExp != null)
- {
- dataOffset += policy.filterExp.Size();
- fieldCount++;
- }
+ if (br.binNames != null)
+ {
+ opCount = br.binNames.Length;
- foreach (string binName in binNames)
+ foreach (string binName in br.binNames)
{
EstimateOperationSize(binName);
}
- SizeBuffer();
- WriteHeaderRead(policy, serverTimeout, Command.INFO1_READ, 0, 0, fieldCount, binNames.Length);
- WriteKey(policy, key);
+ }
+ else if (br.ops != null)
+ {
+ attr.AdjustRead(br.ops);
+ opCount = br.ops.Length;
- if (policy.filterExp != null)
+ foreach (Operation op in br.ops)
{
- policy.filterExp.Write(this);
+ if (Operation.IsWrite(op.type))
+ {
+ throw new AerospikeException(ResultCode.PARAMETER_ERROR, "Write operations not allowed in read");
+ }
+ EstimateOperationSize(op);
}
+ }
+ else
+ {
+ attr.AdjustRead(br.readAllBins);
+ opCount = 0;
+ }
- foreach (string binName in binNames)
+ int fieldCount = EstimateKeyAttrSize(policy, br.key, attr, exp);
+
+ SizeBuffer();
+ WriteKeyAttr(policy, br.key, attr, exp, fieldCount, opCount);
+
+ if (br.binNames != null)
+ {
+ foreach (string binName in br.binNames)
{
WriteOperation(binName, Operation.Type.READ);
}
- End();
}
- else
+ else if (br.ops != null)
+ {
+ foreach (Operation op in br.ops)
+ {
+ WriteOperation(op);
+ }
+ }
+ End();
+ }
+
+ public void SetRead(Policy policy, Key key, Operation[] ops)
+ {
+ Begin();
+
+ BatchAttr attr = new BatchAttr();
+ attr.SetRead(policy);
+ attr.AdjustRead(ops);
+
+ int fieldCount = EstimateKeyAttrSize(policy, key, attr, policy.filterExp);
+
+ foreach (Operation op in ops)
{
- SetRead(policy, key);
+ if (Operation.IsWrite(op.type))
+ {
+ throw new AerospikeException(ResultCode.PARAMETER_ERROR, "Write operations not allowed in read");
+ }
+ EstimateOperationSize(op);
}
+
+ SizeBuffer();
+ WriteKeyAttr(policy, key, attr, policy.filterExp, fieldCount, ops.Length);
+
+ foreach (Operation op in ops)
+ {
+ WriteOperation(op);
+ }
+ End();
}
public virtual void SetReadHeader(Policy policy, Key key)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, false);
if (policy.filterExp != null)
{
dataOffset += policy.filterExp.Size();
fieldCount++;
}
- EstimateOperationSize((string)null);
SizeBuffer();
WriteHeaderReadHeader(policy, Command.INFO1_READ | Command.INFO1_NOBINDATA, fieldCount, 0);
- WriteKey(policy, key);
+ WriteKey(policy, key, false);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
End();
}
@@ -302,7 +720,7 @@ public virtual void SetReadHeader(Policy policy, Key key)
public virtual void SetOperate(WritePolicy policy, Key key, OperateArgs args)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, args.hasWrite);
if (policy.filterExp != null)
{
@@ -314,12 +732,9 @@ public virtual void SetOperate(WritePolicy policy, Key key, OperateArgs args)
bool compress = SizeBuffer(policy);
WriteHeaderReadWrite(policy, args, fieldCount);
- WriteKey(policy, key);
+ WriteKey(policy, key, args.hasWrite);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
foreach (Operation operation in args.operations)
{
@@ -328,6 +743,24 @@ public virtual void SetOperate(WritePolicy policy, Key key, OperateArgs args)
End(compress);
}
+ public void SetOperate(Policy policy, BatchAttr attr, Key key, Operation[] ops)
+ {
+ Begin();
+ Expression exp = GetBatchExpression(policy, attr);
+ int fieldCount = EstimateKeyAttrSize(policy, key, attr, exp);
+
+ dataOffset += attr.opSize;
+ SizeBuffer();
+ WriteKeyAttr(policy, key, attr, exp, fieldCount, ops.Length);
+
+ foreach (Operation op in ops)
+ {
+ WriteOperation(op);
+ }
+ End(policy.compress);
+ }
+
+
//--------------------------------------------------
// UDF
//--------------------------------------------------
@@ -335,7 +768,7 @@ public virtual void SetOperate(WritePolicy policy, Key key, OperateArgs args)
public virtual void SetUdf(WritePolicy policy, Key key, string packageName, string functionName, Value[] args)
{
Begin();
- int fieldCount = EstimateKeySize(policy, key);
+ int fieldCount = EstimateKeySize(policy, key, true);
if (policy.filterExp != null)
{
@@ -348,7 +781,7 @@ public virtual void SetUdf(WritePolicy policy, Key key, string packageName, stri
bool compress = SizeBuffer(policy);
WriteHeaderWrite(policy, Command.INFO2_WRITE, fieldCount, 0);
- WriteKey(policy, key);
+ WriteKey(policy, key, true);
if (policy.filterExp != null)
{
@@ -360,6 +793,27 @@ public virtual void SetUdf(WritePolicy policy, Key key, string packageName, stri
End(compress);
}
+ public void SetUdf(Policy policy, BatchAttr attr, Key key, string packageName, string functionName, Value[] args)
+ {
+ byte[] argBytes = Packer.Pack(args);
+ SetUdf(policy, attr, key, packageName, functionName, argBytes);
+ }
+
+ public void SetUdf(Policy policy, BatchAttr attr, Key key, string packageName, string functionName, byte[] argBytes)
+ {
+ Begin();
+ Expression exp = GetBatchExpression(policy, attr);
+ int fieldCount = EstimateKeyAttrSize(policy, key, attr, exp);
+ fieldCount += EstimateUdfSize(packageName, functionName, argBytes);
+
+ SizeBuffer();
+ WriteKeyAttr(policy, key, attr, exp, fieldCount, 0);
+ WriteField(packageName, FieldType.UDF_PACKAGE_NAME);
+ WriteField(functionName, FieldType.UDF_FUNCTION);
+ WriteField(argBytes, FieldType.UDF_ARGLIST);
+ End(policy.compress);
+ }
+
//--------------------------------------------------
// Batch Read Only
//--------------------------------------------------
@@ -437,16 +891,12 @@ public virtual void SetBatchRead(BatchPolicy policy, List records, Ba
WriteHeaderRead(policy, totalTimeout, readAttr | Command.INFO1_BATCH, 0, 0, fieldCount, 0);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
int fieldSizeOffset = dataOffset;
WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
- ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = (policy.allowInline) ? (byte)1 : (byte)0;
prev = null;
@@ -582,24 +1032,19 @@ int readAttr
WriteHeaderRead(policy, totalTimeout, readAttr | Command.INFO1_BATCH, 0, 0, fieldCount, 0);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
int fieldSizeOffset = dataOffset;
WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
- ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = (policy.allowInline) ? (byte)1 : (byte)0;
prev = null;
for (int i = 0; i < max; i++)
{
int index = offsets[i];
- ByteUtil.IntToBytes((uint)index, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)index, dataBuffer, dataOffset);
Key key = keys[index];
byte[] digest = key.digest;
@@ -622,7 +1067,7 @@ int readAttr
dataBuffer[dataOffset++] = (byte)readAttr;
WriteBatchFields(key, 0, binNames.Length);
- foreach (String binName in binNames)
+ foreach (string binName in binNames)
{
WriteOperation(binName, Operation.Type.READ);
}
@@ -651,14 +1096,44 @@ int readAttr
// Batch Read/Write Operations
//--------------------------------------------------
- public virtual void SetBatchOperate(BatchPolicy policy, IList records, BatchNode batch)
+ public virtual void SetBatchOperate(
+ BatchPolicy policy,
+ BatchWritePolicy writePolicy,
+ BatchUDFPolicy udfPolicy,
+ BatchDeletePolicy deletePolicy,
+ IList records,
+ BatchNode batch)
{
- // Estimate full row size
- int[] offsets = batch.offsets;
- int max = batch.offsetsSize;
- BatchRecord prev = null;
+ BatchOffsetsNative offsets = new BatchOffsetsNative(batch);
+ SetBatchOperate(policy, writePolicy, udfPolicy, deletePolicy, records, offsets);
+ }
+ public void SetBatchOperate(
+ BatchPolicy policy,
+ BatchWritePolicy writePolicy,
+ BatchUDFPolicy udfPolicy,
+ BatchDeletePolicy deletePolicy,
+ IList records,
+ BatchOffsets offsets
+ )
+ {
Begin();
+ int max = offsets.Size();
+ Tran tran = policy.Tran;
+ long?[] versions = null;
+
+ if (tran != null)
+ {
+ versions = new long?[max];
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ BatchRecord record = (BatchRecord)records[offset];
+ versions[i] = tran.GetReadVersion(record.key);
+ }
+ }
+
int fieldCount = 1;
if (policy.filterExp != null)
@@ -669,19 +1144,18 @@ public virtual void SetBatchOperate(BatchPolicy policy, IList records, BatchNode
dataOffset += FIELD_HEADER_SIZE + 5;
+ BatchRecord prev = null;
+
for (int i = 0; i < max; i++)
{
- BatchRecord record = (BatchRecord)records[offsets[i]];
+ int offset = offsets.Get(i);
+ BatchRecord record = (BatchRecord)records[offset];
Key key = record.key;
+ long? ver = versions?[i];
dataOffset += key.digest.Length + 4;
- // Avoid relatively expensive full equality checks for performance reasons.
- // Use reference equality only in hope that common namespaces/bin names are set from
- // fixed variables. It's fine if equality not determined correctly because it just
- // results in more space used. The batch will still be correct.
- if (!policy.sendKey && prev != null && prev.key.ns == key.ns &&
- prev.key.setName == key.setName && record.Equals(prev))
+ if (CanRepeat(policy, key, record, prev, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataOffset++;
@@ -689,51 +1163,43 @@ public virtual void SetBatchOperate(BatchPolicy policy, IList records, BatchNode
else
{
// Estimate full header, namespace and bin names.
- dataOffset += 12;
+ dataOffset += 13;
dataOffset += ByteUtil.EstimateSizeUtf8(key.ns) + FIELD_HEADER_SIZE;
dataOffset += ByteUtil.EstimateSizeUtf8(key.setName) + FIELD_HEADER_SIZE;
+ SizeTranBatch(tran, ver);
dataOffset += record.Size(policy);
prev = record;
}
}
-
- bool compress = SizeBuffer(policy);
+ SizeBuffer();
WriteBatchHeader(policy, totalTimeout, fieldCount);
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
int fieldSizeOffset = dataOffset;
- WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
+ WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
- ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = GetBatchFlags(policy);
- BatchAttr attr = new BatchAttr();
+ BatchAttr attr = new();
prev = null;
for (int i = 0; i < max; i++)
{
- int index = offsets[i];
- ByteUtil.IntToBytes((uint)index, dataBuffer, dataOffset);
- dataOffset += 4;
+ int offset = offsets.Get(i);
+ BatchRecord record = (BatchRecord)records[offset];
+ long? ver = versions?[i];
+
+ dataOffset += ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
- BatchRecord record = (BatchRecord)records[index];
Key key = record.key;
byte[] digest = key.digest;
Array.Copy(digest, 0, dataBuffer, dataOffset, digest.Length);
dataOffset += digest.Length;
- // Avoid relatively expensive full equality checks for performance reasons.
- // Use reference equality only in hope that common namespaces/bin names are set from
- // fixed variables. It's fine if equality not determined correctly because it just
- // results in more space used. The batch will still be correct.
- if (!policy.sendKey && prev != null && prev.key.ns == key.ns &&
- prev.key.setName == key.setName && record.Equals(prev))
+ if (CanRepeat(policy, key, record, prev, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataBuffer[dataOffset++] = BATCH_MSG_REPEAT;
@@ -744,94 +1210,84 @@ public virtual void SetBatchOperate(BatchPolicy policy, IList records, BatchNode
switch (record.GetBatchType())
{
case BatchRecord.Type.BATCH_READ:
- {
- BatchRead br = (BatchRead)record;
-
- if (br.policy != null)
{
- attr.SetRead(br.policy);
- }
- else
- {
- attr.SetRead(policy);
- }
+ BatchRead br = (BatchRead)record;
- if (br.binNames != null)
- {
- WriteBatchBinNames(key, br.binNames, attr, attr.filterExp);
- }
- else if (br.ops != null)
- {
- attr.AdjustRead(br.ops);
- WriteBatchOperations(key, br.ops, attr, attr.filterExp);
- }
- else
- {
- attr.AdjustRead(br.readAllBins);
- WriteBatchRead(key, attr, attr.filterExp, 0);
+ if (br.policy != null)
+ {
+ attr.SetRead(br.policy);
+ }
+ else
+ {
+ attr.SetRead(policy);
+ }
+
+ if (br.binNames != null)
+ {
+ if (br.binNames.Length > 0)
+ {
+ WriteBatchBinNames(key, tran, ver, br.binNames, attr, attr.filterExp);
+ }
+ else
+ {
+ attr.AdjustRead(true);
+ WriteBatchRead(key, tran, ver, attr, attr.filterExp, 0);
+ }
+ }
+ else if (br.ops != null)
+ {
+ attr.AdjustRead(br.ops);
+ WriteBatchOperations(key, tran, ver, br.ops, attr, attr.filterExp);
+ }
+ else
+ {
+ attr.AdjustRead(br.readAllBins);
+ WriteBatchRead(key, tran, ver, attr, attr.filterExp, 0);
+ }
+ break;
}
- break;
- }
case BatchRecord.Type.BATCH_WRITE:
- {
- BatchWrite bw = (BatchWrite)record;
-
- if (bw.policy != null)
{
- attr.SetWrite(bw.policy);
- }
- else
- {
- attr.SetWrite(policy);
+ BatchWrite bw = (BatchWrite)record;
+ BatchWritePolicy bwp = (bw.policy != null) ? bw.policy : writePolicy;
+
+ attr.SetWrite(bwp);
+ attr.AdjustWrite(bw.ops);
+ WriteBatchOperations(key, tran, ver, bw.ops, attr, attr.filterExp);
+ break;
}
- attr.AdjustWrite(bw.ops);
- WriteBatchOperations(key, bw.ops, attr, attr.filterExp);
- break;
- }
case BatchRecord.Type.BATCH_UDF:
- {
- BatchUDF bu = (BatchUDF)record;
-
- if (bu.policy != null)
- {
- attr.SetUDF(bu.policy);
- }
- else
{
- attr.SetUDF(policy);
+ BatchUDF bu = (BatchUDF)record;
+ BatchUDFPolicy bup = (bu.policy != null) ? bu.policy : udfPolicy;
+
+ attr.SetUDF(bup);
+ WriteBatchWrite(key, tran, ver, attr, attr.filterExp, 3, 0);
+ WriteField(bu.packageName, FieldType.UDF_PACKAGE_NAME);
+ WriteField(bu.functionName, FieldType.UDF_FUNCTION);
+ WriteField(bu.argBytes, FieldType.UDF_ARGLIST);
+ break;
}
- WriteBatchWrite(key, attr, attr.filterExp, 3, 0);
- WriteField(bu.packageName, FieldType.UDF_PACKAGE_NAME);
- WriteField(bu.functionName, FieldType.UDF_FUNCTION);
- WriteField(bu.argBytes, FieldType.UDF_ARGLIST);
- break;
- }
case BatchRecord.Type.BATCH_DELETE:
- {
- BatchDelete bd = (BatchDelete)record;
-
- if (bd.policy != null)
{
- attr.SetDelete(bd.policy);
- }
- else
- {
- attr.SetDelete(policy);
+ BatchDelete bd = (BatchDelete)record;
+ BatchDeletePolicy bdp = (bd.policy != null) ? bd.policy : deletePolicy;
+
+ attr.SetDelete(bdp);
+ WriteBatchWrite(key, tran, ver, attr, attr.filterExp, 0, 0);
+ break;
}
- WriteBatchWrite(key, attr, attr.filterExp, 0, 0);
- break;
- }
}
prev = record;
}
}
// Write real field size.
- ByteUtil.IntToBytes((uint)(dataOffset - MSG_TOTAL_HEADER_SIZE - 4), dataBuffer, fieldSizeOffset);
- End(compress);
+ ByteUtil.IntToBytes((uint)dataOffset - MSG_TOTAL_HEADER_SIZE - 4, dataBuffer, fieldSizeOffset);
+ End(policy.compress);
}
public virtual void SetBatchOperate
@@ -843,15 +1299,41 @@ public virtual void SetBatchOperate
Operation[] ops,
BatchAttr attr
)
+ {
+ BatchOffsetsNative offsets = new BatchOffsetsNative(batch);
+ SetBatchOperate(policy, keys, binNames, ops, attr, offsets);
+ }
+
+ public void SetBatchOperate(
+ BatchPolicy policy,
+ Key[] keys,
+ string[] binNames,
+ Operation[] ops,
+ BatchAttr attr,
+ BatchOffsets offsets
+ )
{
// Estimate full row size
- int[] offsets = batch.offsets;
- int max = batch.offsetsSize;
+ int max = offsets.Size();
+ Tran tran = policy.Tran;
+ long?[] versions = null;
- // Estimate dataBuffer size.
Begin();
- int fieldCount = 1;
+
+ if (tran != null)
+ {
+ versions = new long?[max];
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ versions[i] = tran.GetReadVersion(key);
+ }
+ }
+
Expression exp = GetBatchExpression(policy, attr);
+ int fieldCount = 1;
if (exp != null)
{
@@ -865,13 +1347,13 @@ BatchAttr attr
for (int i = 0; i < max; i++)
{
- Key key = keys[offsets[i]];
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions?[i];
dataOffset += key.digest.Length + 4;
- // Try reference equality in hope that namespace/set for all keys is set from fixed variables.
- if (!attr.sendKey && prev != null && prev.ns == key.ns &&
- prev.setName == key.setName)
+ if (CanRepeat(key, prev, attr, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataOffset++;
@@ -879,18 +1361,19 @@ BatchAttr attr
else
{
// Write full header and namespace/set/bin names.
- dataOffset += 12; // header(4) + ttl(4) + fielCount(2) + opCount(2) = 12
+ dataOffset += 13; // header(5) + ttl(4) + fieldCount(2) + opCount(2) = 13
dataOffset += ByteUtil.EstimateSizeUtf8(key.ns) + FIELD_HEADER_SIZE;
dataOffset += ByteUtil.EstimateSizeUtf8(key.setName) + FIELD_HEADER_SIZE;
+ SizeTranBatch(tran, ver);
- if (attr.sendKey)
+ if (attr.sendKey)
{
dataOffset += key.userKey.EstimateSize() + FIELD_HEADER_SIZE + 1;
}
- if (binNames != null)
+ if (binNames != null)
{
- foreach (string binName in binNames)
+ foreach (string binName in binNames)
{
EstimateOperationSize(binName);
}
@@ -918,37 +1401,32 @@ BatchAttr attr
}
}
- bool compress = SizeBuffer(policy);
+ SizeBuffer();
WriteBatchHeader(policy, totalTimeout, fieldCount);
- if (exp != null)
- {
- exp.Write(this);
- }
+ exp?.Write(this);
int fieldSizeOffset = dataOffset;
- WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
+ WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
- ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = GetBatchFlags(policy);
prev = null;
for (int i = 0; i < max; i++)
{
- int index = offsets[i];
- ByteUtil.IntToBytes((uint)index, dataBuffer, dataOffset);
- dataOffset += 4;
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions?[i];
+
+ dataOffset += ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
- Key key = keys[index];
byte[] digest = key.digest;
Array.Copy(digest, 0, dataBuffer, dataOffset, digest.Length);
dataOffset += digest.Length;
- // Try reference equality in hope that namespace/set for all keys is set from fixed variables.
- if (!attr.sendKey && prev != null && prev.ns == key.ns &&
- prev.setName == key.setName)
+ if (CanRepeat(key, prev, attr, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataBuffer[dataOffset++] = BATCH_MSG_REPEAT;
@@ -958,31 +1436,30 @@ BatchAttr attr
// Write full message.
if (binNames != null)
{
- WriteBatchBinNames(key, binNames, attr, null);
+ WriteBatchBinNames(key, tran, ver, binNames, attr, null);
}
else if (ops != null)
{
- WriteBatchOperations(key, ops, attr, null);
+ WriteBatchOperations(key, tran, ver, ops, attr, null);
}
else if ((attr.writeAttr & Command.INFO2_DELETE) != 0)
{
- WriteBatchWrite(key, attr, null, 0, 0);
+ WriteBatchWrite(key, tran, ver, attr, null, 0, 0);
}
else
{
- WriteBatchRead(key, attr, null, 0);
+ WriteBatchRead(key, tran, ver, attr, null, 0);
}
prev = key;
}
}
// Write real field size.
- ByteUtil.IntToBytes((uint)(dataOffset - MSG_TOTAL_HEADER_SIZE - 4), dataBuffer, fieldSizeOffset);
- End(compress);
+ ByteUtil.IntToBytes((uint)dataOffset - MSG_TOTAL_HEADER_SIZE - 4, dataBuffer, fieldSizeOffset);
+ End(policy.compress);
}
- public virtual void SetBatchUDF
- (
+ public virtual void SetBatchUDF(
BatchPolicy policy,
Key[] keys,
BatchNode batch,
@@ -992,14 +1469,41 @@ public virtual void SetBatchUDF
BatchAttr attr
)
{
- // Estimate full row size
- int[] offsets = batch.offsets;
- int max = batch.offsetsSize;
+ BatchOffsetsNative offsets = new BatchOffsetsNative(batch);
+ SetBatchUDF(policy, keys, packageName, functionName, argBytes, attr, offsets);
+ }
- // Estimate dataBuffer size.
+ public virtual void SetBatchUDF
+ (
+ BatchPolicy policy,
+ Key[] keys,
+ string packageName,
+ string functionName,
+ byte[] argBytes,
+ BatchAttr attr,
+ BatchOffsets offsets
+ )
+ {
+ // Estimate buffer size.
Begin();
- int fieldCount = 1;
+ int max = offsets.Size();
+ Tran tran = policy.Tran;
+ long?[] versions = null;
+
+ if (tran != null)
+ {
+ versions = new long?[max];
+
+ for (int i = 0; i < max; i++)
+ {
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ versions[i] = tran.GetReadVersion(key);
+ }
+ }
+
Expression exp = GetBatchExpression(policy, attr);
+ int fieldCount = 1;
if (exp != null)
{
@@ -1013,13 +1517,13 @@ BatchAttr attr
for (int i = 0; i < max; i++)
{
- Key key = keys[offsets[i]];
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions?[i];
dataOffset += key.digest.Length + 4;
- // Try reference equality in hope that namespace/set for all keys is set from fixed variables.
- if (!attr.sendKey && prev != null && prev.ns == key.ns &&
- prev.setName == key.setName)
+ if (CanRepeat(key, prev, attr, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataOffset++;
@@ -1027,51 +1531,47 @@ BatchAttr attr
else
{
// Write full header and namespace/set/bin names.
- dataOffset += 12; // header(4) + ttl(4) + fielCount(2) + opCount(2) = 12
+ dataOffset += 13; // header(4) + ttl(4) + fieldCount(2) + opCount(2) = 13
dataOffset += ByteUtil.EstimateSizeUtf8(key.ns) + FIELD_HEADER_SIZE;
dataOffset += ByteUtil.EstimateSizeUtf8(key.setName) + FIELD_HEADER_SIZE;
+ SizeTranBatch(tran, ver);
- if (attr.sendKey)
+ if (attr.sendKey)
{
dataOffset += key.userKey.EstimateSize() + FIELD_HEADER_SIZE + 1;
}
- dataOffset += 2; // gen(2) = 6
+ dataOffset += 2; // gen(2) = 2
EstimateUdfSize(packageName, functionName, argBytes);
prev = key;
}
}
- bool compress = SizeBuffer(policy);
+ SizeBuffer();
WriteBatchHeader(policy, totalTimeout, fieldCount);
- if (exp != null)
- {
- exp.Write(this);
- }
+ exp?.Write(this);
int fieldSizeOffset = dataOffset;
- WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
+ WriteFieldHeader(0, FieldType.BATCH_INDEX); // Need to update size at end
- ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)max, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = GetBatchFlags(policy);
prev = null;
for (int i = 0; i < max; i++)
{
- int index = offsets[i];
- ByteUtil.IntToBytes((uint)index, dataBuffer, dataOffset);
- dataOffset += 4;
+ int offset = offsets.Get(i);
+ Key key = keys[offset];
+ long? ver = versions?[i];
+
+ dataOffset += ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
- Key key = keys[index];
byte[] digest = key.digest;
Array.Copy(digest, 0, dataBuffer, dataOffset, digest.Length);
dataOffset += digest.Length;
- // Try reference equality in hope that namespace/set for all keys is set from fixed variables.
- if (!attr.sendKey && prev != null && prev.ns == key.ns &&
- prev.setName == key.setName)
+ if (CanRepeat(key, prev, attr, ver))
{
// Can set repeat previous namespace/bin names to save space.
dataBuffer[dataOffset++] = BATCH_MSG_REPEAT;
@@ -1079,7 +1579,7 @@ BatchAttr attr
else
{
// Write full message.
- WriteBatchWrite(key, attr, null, 3, 0);
+ WriteBatchWrite(key, tran, ver, attr, null, 3, 0);
WriteField(packageName, FieldType.UDF_PACKAGE_NAME);
WriteField(functionName, FieldType.UDF_FUNCTION);
WriteField(argBytes, FieldType.UDF_ARGLIST);
@@ -1088,8 +1588,36 @@ BatchAttr attr
}
// Write real field size.
- ByteUtil.IntToBytes((uint)(dataOffset - MSG_TOTAL_HEADER_SIZE - 4), dataBuffer, fieldSizeOffset);
- End(compress);
+ ByteUtil.IntToBytes((uint)dataOffset - MSG_TOTAL_HEADER_SIZE - 4, dataBuffer, fieldSizeOffset);
+ End(policy.compress);
+ }
+
+ private static bool CanRepeat(
+ Policy policy,
+ Key key,
+ BatchRecord record,
+ BatchRecord prev,
+ long? ver
+ )
+ {
+ // Avoid relatively expensive full equality checks for performance reasons.
+ // Use reference equality only in hope that common namespaces/bin names are set from
+ // fixed variables. It's fine if equality not determined correctly because it just
+ // results in more space used. The batch will still be correct.
+ return ver == null && !policy.sendKey && prev != null && prev.key.ns == key.ns &&
+ prev.key.setName == key.setName && record.Equals(prev);
+ }
+
+ private static bool CanRepeat(Key key, Key prev, BatchAttr attr, long? ver)
+ {
+ return ver == null && !attr.sendKey && prev != null && prev.ns == key.ns &&
+ prev.setName == key.setName;
+ }
+
+ private static bool CanRepeat(Key key, Key prev, long? ver)
+ {
+ return ver == null && prev != null && prev.ns == key.ns &&
+ prev.setName == key.setName;
}
private static Expression GetBatchExpression(Policy policy, BatchAttr attr)
@@ -1118,6 +1646,24 @@ private static byte GetBatchFlags(BatchPolicy policy)
return flags;
}
+ private void SizeTranBatch(Tran tran, long? ver)
+ {
+ if (tran != null)
+ {
+ dataOffset += 8 + FIELD_HEADER_SIZE;
+
+ if (ver != null)
+ {
+ dataOffset += 7 + FIELD_HEADER_SIZE;
+ }
+
+ if (tran.Deadline != 0)
+ {
+ dataOffset += 4 + FIELD_HEADER_SIZE;
+ }
+ }
+ }
+
private void WriteBatchHeader(Policy policy, int timeout, int fieldCount)
{
int readAttr = Command.INFO1_BATCH;
@@ -1131,18 +1677,21 @@ private void WriteBatchHeader(Policy policy, int timeout, int fieldCount)
dataOffset += 8;
dataBuffer[dataOffset++] = MSG_REMAINING_HEADER_SIZE; // Message header length.
dataBuffer[dataOffset++] = (byte)readAttr;
+ dataBuffer[dataOffset++] = (byte)0;
+ dataBuffer[dataOffset++] = (byte)0;
- Array.Clear(dataBuffer, dataOffset, 12);
- dataOffset += 12;
-
+ for (int i = 0; i < 10; i++)
+ {
+ dataBuffer[dataOffset++] = 0;
+ }
dataOffset += ByteUtil.IntToBytes((uint)timeout, dataBuffer, dataOffset);
dataOffset += ByteUtil.ShortToBytes((ushort)fieldCount, dataBuffer, dataOffset);
dataOffset += ByteUtil.ShortToBytes(0, dataBuffer, dataOffset);
}
- private void WriteBatchBinNames(Key key, string[] binNames, BatchAttr attr, Expression filter)
+ private void WriteBatchBinNames(Key key, Tran tran, long? ver, string[] binNames, BatchAttr attr, Expression filter)
{
- WriteBatchRead(key, attr, filter, binNames.Length);
+ WriteBatchRead(key, tran, ver, attr, filter, binNames.Length);
foreach (string binName in binNames)
{
@@ -1150,15 +1699,15 @@ private void WriteBatchBinNames(Key key, string[] binNames, BatchAttr attr, Expr
}
}
- private void WriteBatchOperations(Key key, Operation[] ops, BatchAttr attr, Expression filter)
+ private void WriteBatchOperations(Key key, Tran tran, long? ver, Operation[] ops, BatchAttr attr, Expression filter)
{
if (attr.hasWrite)
{
- WriteBatchWrite(key, attr, filter, 0, ops.Length);
+ WriteBatchWrite(key, tran, ver, attr, filter, 0, ops.Length);
}
else
{
- WriteBatchRead(key, attr, filter, ops.Length);
+ WriteBatchRead(key, tran, ver, attr, filter, ops.Length);
}
foreach (Operation op in ops)
@@ -1167,48 +1716,78 @@ private void WriteBatchOperations(Key key, Operation[] ops, BatchAttr attr, Expr
}
}
- private void WriteBatchRead(Key key, BatchAttr attr, Expression filter, int opCount)
+ private void WriteBatchRead(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int opCount)
{
dataBuffer[dataOffset++] = (byte)(BATCH_MSG_INFO | BATCH_MSG_TTL);
dataBuffer[dataOffset++] = (byte)attr.readAttr;
dataBuffer[dataOffset++] = (byte)attr.writeAttr;
dataBuffer[dataOffset++] = (byte)attr.infoAttr;
+ dataBuffer[dataOffset++] = (byte)attr.tranAttr;
dataOffset += ByteUtil.IntToBytes((uint)attr.expiration, dataBuffer, dataOffset);
- WriteBatchFields(key, filter, 0, opCount);
+ WriteBatchFields(key, tran, ver, attr,filter, 0, opCount);
}
- private void WriteBatchWrite(Key key, BatchAttr attr, Expression filter, int fieldCount, int opCount)
+ private void WriteBatchWrite(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
{
dataBuffer[dataOffset++] = (byte)(BATCH_MSG_INFO | BATCH_MSG_GEN | BATCH_MSG_TTL);
dataBuffer[dataOffset++] = (byte)attr.readAttr;
dataBuffer[dataOffset++] = (byte)attr.writeAttr;
dataBuffer[dataOffset++] = (byte)attr.infoAttr;
+ dataBuffer[dataOffset++] = (byte)attr.tranAttr;
dataOffset += ByteUtil.ShortToBytes((ushort)attr.generation, dataBuffer, dataOffset);
dataOffset += ByteUtil.IntToBytes((uint)attr.expiration, dataBuffer, dataOffset);
+ WriteBatchFields(key, tran, ver, attr, filter, fieldCount, opCount);
+ }
- if (attr.sendKey)
+ private void WriteBatchFields(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
+ {
+ if (tran != null)
{
fieldCount++;
- WriteBatchFields(key, filter, fieldCount, opCount);
- WriteField(key.userKey, FieldType.KEY);
+
+ if (ver != null)
+ {
+ fieldCount++;
+ }
+
+ if (attr.hasWrite && tran.Deadline != 0)
+ {
+ fieldCount++;
+ }
}
- else
+
+ if (filter != null)
{
- WriteBatchFields(key, filter, fieldCount, opCount);
+ fieldCount++;
}
- }
- private void WriteBatchFields(Key key, Expression filter, int fieldCount, int opCount)
- {
- if (filter != null)
+ if (attr.sendKey)
{
fieldCount++;
- WriteBatchFields(key, fieldCount, opCount);
- filter.Write(this);
}
- else
+
+ WriteBatchFields(key, fieldCount, opCount);
+
+ if (tran != null)
+ {
+ WriteFieldLE(tran.Id, FieldType.MRT_ID);
+
+ if (ver.HasValue)
+ {
+ WriteFieldVersion(ver.Value);
+ }
+
+ if (attr.hasWrite && tran.Deadline != 0)
+ {
+ WriteFieldLE(tran.Deadline, FieldType.MRT_DEADLINE);
+ }
+ }
+
+ filter?.Write(this);
+
+ if (attr.sendKey)
{
- WriteBatchFields(key, fieldCount, opCount);
+ WriteField(key.userKey, FieldType.KEY);
}
}
@@ -1328,8 +1907,7 @@ NodePartitions nodePartitions
foreach (PartitionStatus part in nodePartitions.partsFull)
{
- ByteUtil.ShortToLittleBytes((ushort)part.id, dataBuffer, dataOffset);
- dataOffset += 2;
+ dataOffset += ByteUtil.ShortToLittleBytes((ushort)part.id, dataBuffer, dataOffset);
}
}
@@ -1362,7 +1940,7 @@ NodePartitions nodePartitions
WriteField(policy.socketTimeout, FieldType.SOCKET_TIMEOUT);
// Write taskId field
- WriteField(taskId, FieldType.TRAN_ID);
+ WriteField(taskId, FieldType.QUERY_ID);
if (binNames != null)
{
@@ -1622,7 +2200,7 @@ NodePartitions nodePartitions
WriteField(policy.socketTimeout, FieldType.SOCKET_TIMEOUT);
// Write taskId field
- WriteField(taskId, FieldType.TRAN_ID);
+ WriteField(taskId, FieldType.QUERY_ID);
if (statement.filter != null)
{
@@ -1673,10 +2251,7 @@ NodePartitions nodePartitions
WriteField(functionArgBuffer, FieldType.UDF_ARGLIST);
}
- if (policy.filterExp != null)
- {
- policy.filterExp.Write(this);
- }
+ policy.filterExp?.Write(this);
if (partsFullSize > 0)
{
@@ -1684,8 +2259,7 @@ NodePartitions nodePartitions
foreach (PartitionStatus part in nodePartitions.partsFull)
{
- ByteUtil.ShortToLittleBytes((ushort)part.id, dataBuffer, dataOffset);
- dataOffset += 2;
+ dataOffset += ByteUtil.ShortToLittleBytes((ushort)part.id, dataBuffer, dataOffset);
}
}
@@ -1706,8 +2280,7 @@ NodePartitions nodePartitions
foreach (PartitionStatus part in nodePartitions.partsPartial)
{
- ByteUtil.LongToLittleBytes(part.bval, dataBuffer, dataOffset);
- dataOffset += 8;
+ dataOffset += ByteUtil.LongToLittleBytes(part.bval, dataBuffer, dataOffset);
}
}
@@ -1737,7 +2310,33 @@ NodePartitions nodePartitions
// Command Sizing
//--------------------------------------------------
- private int EstimateKeySize(Policy policy, Key key)
+ private int EstimateKeyAttrSize(Policy policy, Key key, BatchAttr attr, Expression filterExp)
+ {
+ int fieldCount = EstimateKeySize(policy, key, attr.hasWrite);
+
+ if (filterExp != null)
+ {
+ dataOffset += filterExp.Size();
+ fieldCount++;
+ }
+ return fieldCount;
+ }
+
+ private int EstimateKeySize(Policy policy, Key key, bool sendDeadline)
+ {
+ int fieldCount = EstimateKeySize(key);
+
+ fieldCount += SizeTran(key, policy.Tran, sendDeadline);
+
+ if (policy.sendKey)
+ {
+ dataOffset += key.userKey.EstimateSize() + FIELD_HEADER_SIZE + 1;
+ fieldCount++;
+ }
+ return fieldCount;
+ }
+
+ private int EstimateKeySize(Key key)
{
int fieldCount = 0;
@@ -1756,11 +2355,6 @@ private int EstimateKeySize(Policy policy, Key key)
dataOffset += key.digest.Length + FIELD_HEADER_SIZE;
fieldCount++;
- if (policy.sendKey)
- {
- dataOffset += key.userKey.EstimateSize() + FIELD_HEADER_SIZE + 1;
- fieldCount++;
- }
return fieldCount;
}
@@ -2078,7 +2672,50 @@ private void WriteHeaderReadHeader(Policy policy, int readAttr, int fieldCount,
dataOffset += ByteUtil.ShortToBytes((ushort)operationCount, dataBuffer, dataOffset);
}
- private void WriteKey(Policy policy, Key key)
+ ///
+ /// Header write for batch single commands.
+ ///
+ private void WriteKeyAttr(
+ Policy policy,
+ Key key,
+ BatchAttr attr,
+ Expression filterExp,
+ int fieldCount,
+ int operationCount
+ )
+ {
+ dataOffset += 8;
+
+ // Write all header data except total size which must be written last.
+ dataBuffer[dataOffset++] = MSG_REMAINING_HEADER_SIZE; // Message header length.
+ dataBuffer[dataOffset++] = (byte)attr.readAttr;
+ dataBuffer[dataOffset++] = (byte)attr.writeAttr;
+ dataBuffer[dataOffset++] = (byte)attr.infoAttr;
+ dataBuffer[dataOffset++] = 0; // unused
+ dataBuffer[dataOffset++] = 0; // clear the result code
+ dataOffset += ByteUtil.IntToBytes((uint)attr.generation, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes((uint)attr.expiration, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.IntToBytes((uint)serverTimeout, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)fieldCount, dataBuffer, dataOffset);
+ dataOffset += ByteUtil.ShortToBytes((ushort)operationCount, dataBuffer, dataOffset);
+
+ WriteKey(policy, key, attr.hasWrite);
+
+ filterExp?.Write(this);
+ }
+
+ private void WriteKey(Policy policy, Key key, bool sendDeadline)
+ {
+ WriteKey(key);
+ WriteTran(policy.Tran, sendDeadline);
+
+ if (policy.sendKey)
+ {
+ WriteField(key.userKey, FieldType.KEY);
+ }
+ }
+
+ private void WriteKey(Key key)
{
// Write key into dataBuffer.
if (key.ns != null)
@@ -2092,11 +2729,6 @@ private void WriteKey(Policy policy, Key key)
}
WriteField(key.digest, FieldType.DIGEST_RIPE);
-
- if (policy.sendKey)
- {
- WriteField(key.userKey, FieldType.KEY);
- }
}
private int WriteReadOnlyOperations(Operation[] ops, int readAttr)
@@ -2139,8 +2771,7 @@ private void WriteOperation(Bin bin, Operation.Type operationType)
int nameLength = ByteUtil.StringToUtf8(bin.name, dataBuffer, dataOffset + OPERATION_HEADER_SIZE);
int valueLength = bin.value.Write(dataBuffer, dataOffset + OPERATION_HEADER_SIZE + nameLength);
- ByteUtil.IntToBytes((uint)(nameLength + valueLength + 4), dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)(nameLength + valueLength + 4), dataBuffer, dataOffset);
dataBuffer[dataOffset++] = Operation.GetProtocolType(operationType);
dataBuffer[dataOffset++] = (byte) bin.value.Type;
dataBuffer[dataOffset++] = (byte) 0;
@@ -2153,8 +2784,7 @@ private void WriteOperation(Operation operation)
int nameLength = ByteUtil.StringToUtf8(operation.binName, dataBuffer, dataOffset + OPERATION_HEADER_SIZE);
int valueLength = operation.value.Write(dataBuffer, dataOffset + OPERATION_HEADER_SIZE + nameLength);
- ByteUtil.IntToBytes((uint)(nameLength + valueLength + 4), dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)(nameLength + valueLength + 4), dataBuffer, dataOffset);
dataBuffer[dataOffset++] = Operation.GetProtocolType(operation.type);
dataBuffer[dataOffset++] = (byte) operation.value.Type;
dataBuffer[dataOffset++] = (byte) 0;
@@ -2166,8 +2796,7 @@ private void WriteOperation(string name, Operation.Type operationType)
{
int nameLength = ByteUtil.StringToUtf8(name, dataBuffer, dataOffset + OPERATION_HEADER_SIZE);
- ByteUtil.IntToBytes((uint)(nameLength + 4), dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes((uint)(nameLength + 4), dataBuffer, dataOffset);
dataBuffer[dataOffset++] = Operation.GetProtocolType(operationType);
dataBuffer[dataOffset++] = (byte) 0;
dataBuffer[dataOffset++] = (byte) 0;
@@ -2177,14 +2806,64 @@ private void WriteOperation(string name, Operation.Type operationType)
private void WriteOperation(Operation.Type operationType)
{
- ByteUtil.IntToBytes(4, dataBuffer, dataOffset);
- dataOffset += 4;
+ dataOffset += ByteUtil.IntToBytes(4, dataBuffer, dataOffset);
dataBuffer[dataOffset++] = Operation.GetProtocolType(operationType);
dataBuffer[dataOffset++] = 0;
dataBuffer[dataOffset++] = 0;
dataBuffer[dataOffset++] = 0;
}
+ private int SizeTran(Key key, Tran tran, bool sendDeadline)
+ {
+ int fieldCount = 0;
+
+ if (tran != null)
+ {
+ dataOffset += 8 + FIELD_HEADER_SIZE;
+ fieldCount++;
+
+ Version = tran.GetReadVersion(key);
+
+ if (Version != null)
+ {
+ dataOffset += 7 + FIELD_HEADER_SIZE;
+ fieldCount++;
+ }
+
+ if (sendDeadline && tran.Deadline != 0)
+ {
+ dataOffset += 4 + FIELD_HEADER_SIZE;
+ fieldCount++;
+ }
+ }
+ return fieldCount;
+ }
+
+ private void WriteTran(Tran tran, bool sendDeadline)
+ {
+ if (tran != null)
+ {
+ WriteFieldLE(tran.Id, FieldType.MRT_ID);
+
+ if (Version.HasValue)
+ {
+ WriteFieldVersion(Version.Value);
+ }
+
+ if (sendDeadline && tran.Deadline != 0)
+ {
+ WriteFieldLE(tran.Deadline, FieldType.MRT_DEADLINE);
+ }
+ }
+ }
+
+ private void WriteFieldVersion(long ver)
+ {
+ WriteFieldHeader(7, FieldType.RECORD_VERSION);
+ ByteUtil.LongToVersionBytes(ver, dataBuffer, dataOffset);
+ dataOffset += 7;
+ }
+
private void WriteField(Value value, int type)
{
int offset = dataOffset + FIELD_HEADER_SIZE;
@@ -2214,12 +2893,24 @@ private void WriteField(int val, int type)
dataOffset += ByteUtil.IntToBytes((uint)val, dataBuffer, dataOffset);
}
+ private void WriteFieldLE(int val, int type)
+ {
+ WriteFieldHeader(4, type);
+ dataOffset += ByteUtil.IntToLittleBytes((uint)val, dataBuffer, dataOffset);
+ }
+
private void WriteField(ulong val, int type)
{
WriteFieldHeader(8, type);
dataOffset += ByteUtil.LongToBytes(val, dataBuffer, dataOffset);
}
+ private void WriteFieldLE(long val, int type)
+ {
+ WriteFieldHeader(8, type);
+ dataOffset += ByteUtil.LongToLittleBytes((ulong)val, dataBuffer, dataOffset);
+ }
+
private void WriteFieldHeader(int size, int type)
{
dataOffset += ByteUtil.IntToBytes((uint)size + 1, dataBuffer, dataOffset);
@@ -2353,10 +3044,59 @@ internal virtual Key ParseKey(int fieldCount, out ulong bval)
return new Key(ns, digest, setName, userKey);
}
+ public long? ParseVersion(int fieldCount)
+ {
+ long? version = null;
+
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int type = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (type == FieldType.RECORD_VERSION && size == 7)
+ {
+ version = ByteUtil.VersionBytesToLong(dataBuffer, dataOffset);
+ }
+ dataOffset += size;
+ }
+ return version;
+ }
+
public static bool BatchInDoubt(bool isWrite, int commandSentCounter)
{
return isWrite && commandSentCounter > 1;
}
+
+ public interface BatchOffsets
+ {
+ int Size();
+ int Get(int i);
+ }
+
+ private class BatchOffsetsNative : BatchOffsets
+ {
+ private int size;
+ private int[] offsets;
+
+ public BatchOffsetsNative(BatchNode batch)
+ {
+ this.size = batch.offsetsSize;
+ this.offsets = batch.offsets;
+ }
+
+ public int Size()
+ {
+ return size;
+ }
+
+ public int Get(int i)
+ {
+ return offsets[i];
+ }
+ }
}
}
#pragma warning restore 0618
diff --git a/AerospikeClient/Command/FieldType.cs b/AerospikeClient/Command/FieldType.cs
index 75cd5fa3..7b4b01d0 100644
--- a/AerospikeClient/Command/FieldType.cs
+++ b/AerospikeClient/Command/FieldType.cs
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2022 Aerospike, Inc.
+ * Copyright 2012-2024 Aerospike, Inc.
*
* Portions may be licensed to Aerospike, Inc. under one or more contributor
* license agreements.
@@ -21,8 +21,11 @@ public sealed class FieldType
public const int NAMESPACE = 0;
public const int TABLE = 1;
public const int KEY = 2;
+ public const int RECORD_VERSION = 3;
public const int DIGEST_RIPE = 4;
- public const int TRAN_ID = 7; // user supplied transaction id, which is simply passed back
+ public const int MRT_ID = 5;
+ public const int MRT_DEADLINE = 6;
+ public const int QUERY_ID = 7;
public const int SOCKET_TIMEOUT = 9;
public const int RECORDS_PER_SECOND = 10;
public const int PID_ARRAY = 11;
diff --git a/AerospikeClient/Command/OperateArgs.cs b/AerospikeClient/Command/OperateArgs.cs
index c13edab7..21ce2e4e 100644
--- a/AerospikeClient/Command/OperateArgs.cs
+++ b/AerospikeClient/Command/OperateArgs.cs
@@ -30,7 +30,6 @@ public OperateArgs
WritePolicy policy,
WritePolicy writeDefault,
WritePolicy readDefault,
- Key key,
Operation[] operations
)
{
diff --git a/AerospikeClient/Command/SyncWriteCommand.cs b/AerospikeClient/Command/SyncWriteCommand.cs
new file mode 100644
index 00000000..5380f2e3
--- /dev/null
+++ b/AerospikeClient/Command/SyncWriteCommand.cs
@@ -0,0 +1,162 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public abstract class SyncWriteCommand : SyncCommand
+ {
+ protected readonly WritePolicy writePolicy;
+ protected readonly Key key;
+ private readonly Partition partition;
+
+ public SyncWriteCommand(Cluster cluster, WritePolicy writePolicy, Key key)
+ : base(cluster, writePolicy)
+ {
+ this.writePolicy = writePolicy;
+ this.key = key;
+ this.partition = Partition.Write(cluster, writePolicy, key);
+ cluster.AddTran();
+ }
+
+ protected internal override bool IsWrite()
+ {
+ return true;
+ }
+
+ protected internal override Node GetNode()
+ {
+ return partition.GetNodeWrite(cluster);
+ }
+
+ protected override Latency.LatencyType GetLatencyType()
+ {
+ return Latency.LatencyType.WRITE;
+ }
+
+ protected internal override bool PrepareRetry(bool timeout)
+ {
+ partition.PrepareRetryWrite(timeout);
+ return true;
+ }
+
+ protected int ParseHeader(IConnection conn)
+ {
+ // Read header.
+ conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
+
+ long sz = ByteUtil.BytesToLong(dataBuffer, 0);
+ int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
+
+ if (receiveSize <= 0)
+ {
+ throw new AerospikeException("Invalid receive size: " + receiveSize);
+ }
+
+ SizeBuffer(receiveSize);
+ conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
+ conn.UpdateLastUsed();
+
+ ulong type = (ulong)(sz >> 48) & 0xff;
+
+ if (type == Command.AS_MSG_TYPE)
+ {
+ dataOffset = 5;
+ }
+ else if (type == Command.MSG_TYPE_COMPRESSED)
+ {
+ int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
+ byte[] ubuf = new byte[usize];
+
+ ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
+ dataBuffer = ubuf;
+ dataOffset = 13;
+ }
+ else
+ {
+ throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
+ }
+
+ int resultCode = dataBuffer[dataOffset] & 0xFF;
+ dataOffset++;
+ int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+ int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 8;
+ int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+ int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+
+ if (policy.Tran == null)
+ {
+ SkipFields(fieldCount);
+ if (opCount > 0)
+ {
+ throw new AerospikeException("Unexpected write response opCount: " + opCount + ',' + resultCode);
+ }
+ return resultCode;
+ }
+
+ long? version = null;
+
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int fieldType = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (fieldType == FieldType.RECORD_VERSION)
+ {
+ if (size == 7)
+ {
+ version = ByteUtil.VersionBytesToLong(dataBuffer, dataOffset);
+ }
+ else
+ {
+ throw new AerospikeException("Record version field has invalid size: " + size);
+ }
+ }
+ dataOffset += size;
+ }
+
+ policy.Tran.OnWrite(key, version, resultCode);
+
+ if (opCount > 0)
+ {
+ throw new AerospikeException("Unexpected write response opCount: " + opCount + ',' + resultCode);
+ }
+ return resultCode;
+ }
+
+ private void SkipFields(int fieldCount)
+ {
+ // There can be fields in the response (setname etc).
+ // But for now, ignore them. Expose them to the API if needed in the future.
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int fieldlen = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4 + fieldlen;
+ }
+ }
+
+ protected internal abstract override void WriteBuffer();
+
+ protected internal abstract override void ParseResult(IConnection conn);
+ }
+}
diff --git a/AerospikeClient/Command/TranAddKeys.cs b/AerospikeClient/Command/TranAddKeys.cs
new file mode 100644
index 00000000..39d71567
--- /dev/null
+++ b/AerospikeClient/Command/TranAddKeys.cs
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class TranAddKeys : SyncWriteCommand
+ {
+ private readonly OperateArgs args;
+
+ public TranAddKeys (Cluster cluster, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, key)
+ {
+ this.args = args;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTranAddKeys(args.writePolicy, key, args);
+ }
+
+ protected internal override void ParseResult(IConnection conn)
+ {
+ // Read header.
+ conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
+
+ long sz = ByteUtil.BytesToLong(dataBuffer, 0);
+ int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
+
+ if (receiveSize <= 0)
+ {
+ throw new AerospikeException("Invalid receive size: " + receiveSize);
+ }
+
+ SizeBuffer(receiveSize);
+ conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
+ conn.UpdateLastUsed();
+
+ ulong type = (ulong)(sz >> 48) & 0xff;
+
+ if (type == Command.AS_MSG_TYPE)
+ {
+ dataOffset = 5;
+ }
+ else if (type == Command.MSG_TYPE_COMPRESSED)
+ {
+ int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
+ byte[] ubuf = new byte[usize];
+
+ ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
+ dataBuffer = ubuf;
+ dataOffset = 13;
+ }
+ else
+ {
+ throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
+ }
+
+ int resultCode = dataBuffer[dataOffset] & 0xFF;
+ dataOffset++;
+ int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+ int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 8;
+ int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+ int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int fieldType = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (fieldType == FieldType.MRT_DEADLINE)
+ {
+ int deadline = ByteUtil.LittleBytesToInt(dataBuffer, dataOffset);
+ policy.Tran.Deadline = deadline;
+ }
+ dataOffset += size;
+ }
+
+ if (resultCode == ResultCode.OK)
+ {
+ return;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+ }
+}
diff --git a/AerospikeClient/Command/TranClose.cs b/AerospikeClient/Command/TranClose.cs
new file mode 100644
index 00000000..4bd1ff49
--- /dev/null
+++ b/AerospikeClient/Command/TranClose.cs
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class TranClose : SyncWriteCommand
+ {
+ private readonly Tran tran;
+
+ public TranClose(Cluster cluster, Tran tran, WritePolicy writePolicy, Key key)
+ : base(cluster, writePolicy, key)
+ {
+ this.tran = tran;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTranClose(tran, key);
+ }
+
+ protected internal override void ParseResult(IConnection conn)
+ {
+ int resultCode = ParseHeader(conn);
+
+ if (resultCode == ResultCode.OK || resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
+ {
+ return;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+ }
+}
diff --git a/AerospikeClient/Command/TranMarkRollForward.cs b/AerospikeClient/Command/TranMarkRollForward.cs
new file mode 100644
index 00000000..dde97904
--- /dev/null
+++ b/AerospikeClient/Command/TranMarkRollForward.cs
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class TranMarkRollForward : SyncWriteCommand
+ {
+ private readonly Tran tran;
+
+ public TranMarkRollForward(Cluster cluster, Tran tran, WritePolicy writePolicy, Key key)
+ : base(cluster, writePolicy, key)
+ {
+ this.tran = tran;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTranMarkRollForward(tran, key);
+ }
+
+ protected internal override void ParseResult(IConnection conn)
+ {
+ int resultCode = ParseHeader(conn);
+
+ if (resultCode == ResultCode.OK || resultCode == ResultCode.BIN_EXISTS_ERROR)
+ {
+ return;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+ }
+}
diff --git a/AerospikeClient/Command/TranMonitor.cs b/AerospikeClient/Command/TranMonitor.cs
new file mode 100644
index 00000000..7fa7584c
--- /dev/null
+++ b/AerospikeClient/Command/TranMonitor.cs
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class TranMonitor
+ {
+ private static readonly ListPolicy OrderedListPolicy = new(ListOrder.ORDERED,
+ ListWriteFlags.ADD_UNIQUE | ListWriteFlags.NO_FAIL | ListWriteFlags.PARTIAL);
+
+ private static readonly string BinNameId = "id";
+ private static readonly string BinNameDigests = "keyds";
+
+ public static void AddKey(Cluster cluster, WritePolicy policy, Key cmdKey)
+ {
+ Tran tran = policy.Tran;
+
+ if (tran.Writes.Contains(cmdKey))
+ {
+ // Transaction monitor already contains this key.
+ return;
+ }
+
+ Operation[] ops = GetTranOps(tran, cmdKey);
+ AddWriteKeys(cluster, policy, ops);
+ }
+
+ public static void AddKeys(Cluster cluster, BatchPolicy policy, Key[] keys)
+ {
+ Operation[] ops = GetTranOps(policy.Tran, keys);
+ AddWriteKeys(cluster, policy, ops);
+ }
+
+ public static void AddKeys(Cluster cluster, BatchPolicy policy, List records)
+ {
+ Operation[] ops = GetTranOps(policy.Tran, records);
+
+ if (ops != null)
+ {
+ AddWriteKeys(cluster, policy, ops);
+ }
+ }
+
+ public static Operation[] GetTranOps(Tran tran, Key cmdKey)
+ {
+ tran.Ns = cmdKey.ns;
+
+ if (tran.Deadline == 0)
+ {
+ // No existing monitor record.
+ return new Operation[] {
+ Operation.Put(new Bin(BinNameId, tran.Id)),
+ ListOperation.Append(OrderedListPolicy, BinNameDigests, Value.Get(cmdKey.digest))
+ };
+ }
+ else
+ {
+ return new Operation[] {
+ ListOperation.Append(OrderedListPolicy, BinNameDigests, Value.Get(cmdKey.digest))
+ };
+ }
+ }
+
+ public static Operation[] GetTranOps(Tran tran, Key[] keys)
+ {
+ List list = new(keys.Length);
+
+ foreach (Key key in keys)
+ {
+ tran.Ns = key.ns;
+ list.Add(Value.Get(key.digest));
+ }
+ return GetTranOps(tran, list);
+ }
+
+ public static Operation[] GetTranOps(Tran tran, List records)
+ {
+ List list = new(records.Count);
+
+ foreach (BatchRecord br in records) {
+ tran.Ns = br.key.ns;
+
+ if (br.hasWrite)
+ {
+ list.Add(Value.Get(br.key.digest));
+ }
+ }
+
+ if (list.Count == 0)
+ {
+ // Readonly batch does not need to add key digests.
+ return null;
+ }
+ return GetTranOps(tran, list);
+ }
+
+ private static Operation[] GetTranOps(Tran tran, List list)
+ {
+ if (tran.Deadline == 0)
+ {
+ // No existing monitor record.
+ return new Operation[] {
+ Operation.Put(new Bin(BinNameId, tran.Id)),
+ ListOperation.AppendItems(OrderedListPolicy, BinNameDigests, list)
+ };
+ }
+ else
+ {
+ return new Operation[] {
+ ListOperation.AppendItems(OrderedListPolicy, BinNameDigests, list)
+ };
+ }
+ }
+
+ private static void AddWriteKeys(Cluster cluster, Policy policy, Operation[] ops)
+ {
+ Key tranKey = GetTranMonitorKey(policy.Tran);
+ WritePolicy wp = CopyTimeoutPolicy(policy);
+ OperateArgs args = new(wp, null, null, ops);
+ TranAddKeys cmd = new(cluster, tranKey, args);
+ cmd.Execute();
+ }
+
+ public static Key GetTranMonitorKey(Tran tran)
+ {
+ return new Key(tran.Ns, " keySet = tran.Writes;
+
+ if (keySet.Count != 0)
+ {
+ // Tell MRT monitor that a roll-forward will commence.
+ try
+ {
+ MarkRollForward(writePolicy, tranKey);
+ }
+ catch (Exception t)
+ {
+ throw new AerospikeException.Commit(CommitErrorType.MARK_ROLL_FORWARD_ABANDONED, verifyRecords, rollRecords, t);
+ }
+
+ // Roll-forward writes in batch.
+ try
+ {
+ Roll(rollPolicy, Command.INFO4_MRT_ROLL_FORWARD);
+ }
+ catch (Exception t)
+ {
+ throw new AerospikeException.Commit(CommitErrorType.ROLL_FORWARD_ABANDONED, verifyRecords, rollRecords, t);
+ }
+ }
+
+ if (tran.Deadline != 0)
+ {
+ // Remove MRT monitor.
+ try
+ {
+ Close(writePolicy, tranKey);
+ }
+ catch (Exception t)
+ {
+ throw new AerospikeException.Commit(CommitErrorType.CLOSE_ABANDONED, verifyRecords, rollRecords, t);
+ }
+ }
+ }
+
+ public void Abort(BatchPolicy rollPolicy)
+ {
+ HashSet keySet = tran.Writes;
+
+ if (keySet.Count != 0)
+ {
+ try
+ {
+ Roll(rollPolicy, Command.INFO4_MRT_ROLL_BACK);
+ }
+ catch (Exception t)
+ {
+ throw new AerospikeException.Abort(AbortErrorType.ROLL_BACK_ABANDONED, rollRecords, t);
+ }
+ }
+
+ if (tran.Deadline != 0)
+ {
+ try
+ {
+ WritePolicy writePolicy = new(rollPolicy);
+ Key tranKey = TranMonitor.GetTranMonitorKey(tran);
+ Close(writePolicy, tranKey);
+ }
+ catch (Exception t)
+ {
+ throw new AerospikeException.Abort(AbortErrorType.CLOSE_ABANDONED, rollRecords, t);
+ }
+ }
+ }
+ private void Verify(BatchPolicy verifyPolicy)
+ {
+ // Validate record versions in a batch.
+ HashSet> reads = tran.Reads.ToHashSet>();
+ int max = reads.Count;
+ if (max == 0)
+ {
+ return;
+ }
+
+ BatchRecord[] records = new BatchRecord[max];
+ Key[] keys = new Key[max];
+ long[] versions = new long[max];
+ int count = 0;
+
+ foreach (KeyValuePair entry in reads)
+ {
+ Key key = entry.Key;
+ keys[count] = key;
+ records[count] = new BatchRecord(key, false);
+ versions[count] = entry.Value;
+ count++;
+ }
+ this.verifyRecords = records;
+
+ BatchStatus status = new(true);
+ List bns = BatchNode.GenerateList(cluster, verifyPolicy, keys, records, false, status);
+ BatchCommand[] commands = new BatchCommand[bns.Count];
+
+ count = 0;
+
+ foreach (BatchNode bn in bns)
+ {
+ commands[count++] = new BatchTranVerify(
+ cluster, bn, verifyPolicy, tran, keys, versions, records, status);
+ }
+
+ BatchExecutor.Execute(cluster, verifyPolicy, commands, status);
+
+ if (!status.GetStatus())
+ {
+ throw new AerospikeException("Failed to verify one or more record versions");
+ }
+ }
+
+ private void MarkRollForward(WritePolicy writePolicy, Key tranKey)
+ {
+ // Tell MRT monitor that a roll-forward will commence.
+ TranMarkRollForward cmd = new(cluster, tran, writePolicy, tranKey);
+ cmd.Execute();
+ }
+
+ private void Roll(BatchPolicy rollPolicy, int tranAttr)
+ {
+ HashSet keySet = tran.Writes;
+
+ if (keySet.Count == 0)
+ {
+ return;
+ }
+
+ Key[] keys = keySet.ToArray();
+ BatchRecord[] records = new BatchRecord[keys.Length];
+
+ for (int i = 0; i < keys.Length; i++)
+ {
+ records[i] = new BatchRecord(keys[i], true);
+ }
+
+ this.rollRecords = records;
+
+ // Copy tran roll policy because it needs to be modified.
+ BatchPolicy batchPolicy = new(rollPolicy);
+
+ BatchAttr attr = new();
+ attr.SetTran(tranAttr);
+ BatchStatus status = new(true);
+
+ // generate() requires a null tran instance.
+ List bns = BatchNode.GenerateList(cluster, batchPolicy, keys, records, true, status);
+ BatchCommand[] commands = new BatchCommand[bns.Count];
+
+ // Batch roll forward requires the tran instance.
+ batchPolicy.Tran = tran;
+
+ int count = 0;
+
+ foreach (BatchNode bn in bns)
+ {
+ commands[count++] = new BatchTranRoll(
+ cluster, bn, batchPolicy, keys, records, attr, status);
+ }
+ BatchExecutor.Execute(cluster, batchPolicy, commands, status);
+
+ if (!status.GetStatus())
+ {
+ string rollString = tranAttr == Command.INFO4_MRT_ROLL_FORWARD ? "commit" : "abort";
+ throw new AerospikeException("Failed to " + rollString + " one or more records");
+ }
+ }
+
+ private void Close(WritePolicy writePolicy, Key tranKey)
+ {
+ // Delete MRT monitor on server.
+ TranClose cmd = new(cluster, tran, writePolicy, tranKey);
+ cmd.Execute();
+
+ // Reset MRT on client.
+ tran.Clear();
+ }
+ }
+}
diff --git a/AerospikeClient/Command/WriteCommand.cs b/AerospikeClient/Command/WriteCommand.cs
index 07984aff..05b9eb91 100644
--- a/AerospikeClient/Command/WriteCommand.cs
+++ b/AerospikeClient/Command/WriteCommand.cs
@@ -17,40 +17,19 @@
namespace Aerospike.Client
{
- public sealed class WriteCommand : SyncCommand
+ public sealed class WriteCommand : SyncWriteCommand
{
- private readonly WritePolicy writePolicy;
- private readonly Key key;
- private readonly Partition partition;
private readonly Bin[] bins;
private readonly Operation.Type operation;
public WriteCommand(Cluster cluster, WritePolicy writePolicy, Key key, Bin[] bins, Operation.Type operation)
- : base(cluster, writePolicy)
+ : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
- this.key = key;
- this.partition = Partition.Write(cluster, writePolicy, key);
this.bins = bins;
this.operation = operation;
cluster.AddTran();
}
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode()
- {
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
-
protected internal override void WriteBuffer()
{
SetWrite(writePolicy, operation, key, bins);
@@ -80,11 +59,5 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
-
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
}
}
diff --git a/AerospikeClient/Listener/AbortListener.cs b/AerospikeClient/Listener/AbortListener.cs
new file mode 100644
index 00000000..3a86682e
--- /dev/null
+++ b/AerospikeClient/Listener/AbortListener.cs
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+namespace Aerospike.Client
+{
+ ///
+ /// Asynchronous result notifications for multi-record transaction (MRT) aborts.
+ ///
+ public interface AbortListener
+ {
+ ///
+ /// This method is called when the abort succeeds.
+ ///
+ void OnSuccess();
+
+ ///
+ /// This method is called when the abort fails.
+ ///
+ /// error that occurred
+ void OnFailure(AerospikeException exception);
+ }
+}
diff --git a/AerospikeClient/Listener/CommitListener.cs b/AerospikeClient/Listener/CommitListener.cs
new file mode 100644
index 00000000..f77bc571
--- /dev/null
+++ b/AerospikeClient/Listener/CommitListener.cs
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+namespace Aerospike.Client
+{
+ ///
+ /// Asynchronous result notifications for multi-record transaction (MRT) commits.
+ ///
+ public interface CommitListener
+ {
+ ///
+ /// This method is called when the records are verified and the commit succeeds.
+ ///
+ void OnSuccess();
+
+ ///
+ /// This method is called when the commit fails.
+ ///
+ /// error that occurred
+ void OnFailure(AerospikeException exception);
+ }
+}
diff --git a/AerospikeClient/Main/AbortError.cs b/AerospikeClient/Main/AbortError.cs
new file mode 100644
index 00000000..83997c91
--- /dev/null
+++ b/AerospikeClient/Main/AbortError.cs
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) abort error status.
+ ///
+ public static class AbortError
+ {
+ public enum AbortErrorType
+ {
+ ROLL_BACK_ABANDONED,
+ CLOSE_ABANDONED
+ }
+
+ public static string AbortErrorToString(AbortErrorType error)
+ {
+ return error switch
+ {
+ AbortErrorType.ROLL_BACK_ABANDONED => "MRT client roll back abandoned. Server will eventually abort the MRT.",
+ AbortErrorType.CLOSE_ABANDONED => "MRT has been rolled back, but MRT client close was abandoned. Server will eventually close the MRT.",
+ _ => "Unexpected AbortErrorType"
+ };
+ }
+ }
+}
diff --git a/AerospikeClient/Main/AerospikeClient.cs b/AerospikeClient/Main/AerospikeClient.cs
index 0ceecc27..852ebb87 100644
--- a/AerospikeClient/Main/AerospikeClient.cs
+++ b/AerospikeClient/Main/AerospikeClient.cs
@@ -94,6 +94,17 @@ public class AerospikeClient : IDisposable, IAerospikeClient
///
public BatchUDFPolicy batchUDFPolicyDefault;
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ public BatchPolicy tranVerifyPolicyDefault;
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ public BatchPolicy tranRollPolicyDefault;
+
///
/// Default info policy that is used when info command policy is null.
///
@@ -190,6 +201,8 @@ public AerospikeClient(ClientPolicy policy, params Host[] hosts)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
+ this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
+ this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -214,6 +227,8 @@ protected internal AerospikeClient(ClientPolicy policy)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
+ this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
+ this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
}
else
@@ -227,6 +242,8 @@ protected internal AerospikeClient(ClientPolicy policy)
this.batchWritePolicyDefault = new BatchWritePolicy();
this.batchDeletePolicyDefault = new BatchDeletePolicy();
this.batchUDFPolicyDefault = new BatchUDFPolicy();
+ this.tranVerifyPolicyDefault = new TranVerifyPolicy();
+ this.tranRollPolicyDefault= new TranRollPolicy();
this.infoPolicyDefault = new InfoPolicy();
}
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -320,6 +337,25 @@ public BatchUDFPolicy BatchUDFPolicyDefault
set { batchUDFPolicyDefault = value; }
}
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ public BatchPolicy TranVerifyPolicyDefault
+ {
+ get { return tranVerifyPolicyDefault; }
+ set { tranVerifyPolicyDefault = value; }
+ }
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ public BatchPolicy TranRollPolicyDefault
+ {
+ get { return tranRollPolicyDefault; }
+ set { tranRollPolicyDefault = value; }
+ }
+
///
/// Default info policy that is used when info command policy is null.
///
@@ -422,6 +458,42 @@ public ClusterStats GetClusterStats()
return cluster.GetStats();
}
+ //-------------------------------------------------------
+ // Multi-Record Transactions
+ //-------------------------------------------------------
+
+ ///
+ /// Attempt to commit the given multi-record transaction. First, the expected record versions are
+ /// sent to the server nodes for verification.If all nodes return success, the transaction is
+ /// committed.Otherwise, the transaction is aborted.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ public void Commit(Tran tran)
+ {
+ tran.SetRollAttempted();
+
+ TranRoll tm = new TranRoll(cluster, tran);
+ tm.Commit(tranVerifyPolicyDefault, tranRollPolicyDefault);
+ }
+
+ ///
+ /// Abort and rollback the given multi-record transaction.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ public void Abort(Tran tran)
+ {
+ tran.SetRollAttempted();
+
+ TranRoll tm = new TranRoll(cluster, tran);
+ tm.Abort(tranRollPolicyDefault);
+ }
+
//-------------------------------------------------------
// Write Record Operations
//-------------------------------------------------------
@@ -1137,7 +1209,7 @@ public Record Join(BatchPolicy policy, Key key, params Join[] joins)
/// if command fails
public Record Operate(WritePolicy policy, Key key, params Operation[] operations)
{
- OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, key, operations);
+ OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, operations);
OperateCommand command = new OperateCommand(cluster, key, args);
command.Execute();
return command.Record;
@@ -2027,7 +2099,7 @@ params CTX[] ctx
sb.Append(";indexdata=");
sb.Append(binName);
- sb.Append(",");
+ sb.Append(',');
sb.Append(indexType);
// Send index command to one node. That node will distribute the command to other nodes.
diff --git a/AerospikeClient/Main/AerospikeException.cs b/AerospikeClient/Main/AerospikeException.cs
index 257a35fc..ec7268b0 100644
--- a/AerospikeClient/Main/AerospikeException.cs
+++ b/AerospikeClient/Main/AerospikeException.cs
@@ -15,6 +15,8 @@
* the License.
*/
using System.Text;
+using static Aerospike.Client.AbortError;
+using static Aerospike.Client.CommitError;
namespace Aerospike.Client
{
@@ -41,6 +43,12 @@ public AerospikeException(int resultCode, Exception e)
this.resultCode = resultCode;
}
+ public AerospikeException(int resultCode, string message)
+ : base(message)
+ {
+ this.resultCode = resultCode;
+ }
+
public AerospikeException(int resultCode)
: base("")
{
@@ -76,7 +84,7 @@ public override string Message
{
get
{
- StringBuilder sb = new StringBuilder(512);
+ StringBuilder sb = new(512);
sb.Append("Error ");
sb.Append(resultCode);
@@ -299,7 +307,7 @@ public override string Message
return "Client timeout: " + totalTimeout;
}
- StringBuilder sb = new StringBuilder(512);
+ StringBuilder sb = new(512);
if (client)
{
@@ -487,6 +495,12 @@ public BatchRecordArray(BatchRecord[] records, Exception e)
{
this.records = records;
}
+
+ public BatchRecordArray(BatchRecord[] records, String message, Exception e)
+ : base(ResultCode.BATCH_FAILED, message, e)
+ {
+ this.records = records;
+ }
}
///
@@ -588,5 +602,138 @@ public EndOfGRPCStream(int resultCode)
ResultCode = resultCode;
}
}
+
+ ///
+ /// Exception thrown when {@link AerospikeClient#commit(com.aerospike.client.Tran)} fails.
+ ///
+ public sealed class Commit : AerospikeException
+ {
+ ///
+ /// Error status of the attempted commit.
+ ///
+ public readonly CommitErrorType Error;
+
+ ///
+ /// Verify result for each read key in the MRT. May be null if failure occurred before verify.
+ ///
+ public readonly BatchRecord[] VerifyRecords;
+
+ ///
+ /// Roll forward/backward result for each write key in the MRT. May be null if failure occurred before
+ /// roll forward/backward.
+ ///
+ public readonly BatchRecord[] RollRecords;
+
+ public Commit(CommitErrorType error, BatchRecord[] verifyRecords, BatchRecord[] rollRecords)
+ : base(ResultCode.TRAN_FAILED, CommitErrorToString(error))
+ {
+ this.Error = error;
+ this.VerifyRecords = verifyRecords;
+ this.RollRecords = rollRecords;
+ }
+
+ public Commit(CommitErrorType error, BatchRecord[] verifyRecords, BatchRecord[] rollRecords, Exception cause)
+ : base(ResultCode.TRAN_FAILED, CommitErrorToString(error), cause)
+ {
+ this.Error = error;
+ this.VerifyRecords = verifyRecords;
+ this.RollRecords = rollRecords;
+ }
+
+ ///
+ /// Get Commit message with records.
+ ///
+ public override string Message
+ {
+ get
+ {
+ StringBuilder sb = new(1024);
+ RecordsToString(sb, "verify errors:", VerifyRecords);
+ RecordsToString(sb, "roll errors:", RollRecords);
+ return BaseMessage + sb.ToString();
+ }
+ }
+ }
+
+ ///
+ /// Exception thrown when {@link AerospikeClient#abort(com.aerospike.client.Tran)} fails.
+ ///
+ public sealed class Abort : AerospikeException
+ {
+ ///
+ /// Error status of the attempted abort.
+ ///
+ public readonly AbortErrorType Error;
+
+ ///
+ /// Roll backward result for each write key in the MRT. May be null if failure occurred before roll backward.
+ ///
+ public readonly BatchRecord[] RollRecords;
+
+ public Abort(AbortErrorType error, BatchRecord[] rollRecords)
+ : base(ResultCode.TRAN_FAILED, AbortErrorToString(error))
+ {
+ this.Error = error;
+ this.RollRecords = rollRecords;
+ }
+
+ public Abort(AbortErrorType error, BatchRecord[] rollRecords, Exception cause)
+ : base(ResultCode.TRAN_FAILED, AbortErrorToString(error), cause)
+ {
+ this.Error = error;
+ this.RollRecords = rollRecords;
+ }
+
+ ///
+ /// Get Commit message with records.
+ ///
+ public override string Message
+ {
+ get
+ {
+ StringBuilder sb = new(1024);
+ RecordsToString(sb, "roll errors:", RollRecords);
+ return BaseMessage + sb.ToString();
+ }
+ }
+ }
+
+ private static void RecordsToString(StringBuilder sb, String title, BatchRecord[] records)
+ {
+ if (records == null)
+ {
+ return;
+ }
+
+ int count = 0;
+
+ foreach (BatchRecord br in records) {
+ // Only show results with an error response.
+ if (!(br.resultCode == ResultCode.OK || br.resultCode == ResultCode.NO_RESPONSE))
+ {
+ // Only show first 3 errors.
+ if (count >= 3)
+ {
+ sb.Append(System.Environment.NewLine);
+ sb.Append("...");
+ break;
+ }
+
+ if (count == 0)
+ {
+ sb.Append(System.Environment.NewLine);
+ sb.Append(title);
+ }
+
+ sb.Append(System.Environment.NewLine);
+ sb.Append(br.key);
+ sb.Append(',');
+ sb.Append(br.resultCode);
+ sb.Append(',');
+ sb.Append(br.inDoubt);
+ count++;
+ }
+ }
+ }
}
}
diff --git a/AerospikeClient/Main/CommitError.cs b/AerospikeClient/Main/CommitError.cs
new file mode 100644
index 00000000..6a166eb5
--- /dev/null
+++ b/AerospikeClient/Main/CommitError.cs
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) error status.
+ ///
+ public static class CommitError
+ {
+ public enum CommitErrorType
+ {
+ VERIFY_FAIL,
+ VERIFY_FAIL_CLOSE_ABANDONED,
+ VERIFY_FAIL_ABORT_ABANDONED,
+ MARK_ROLL_FORWARD_ABANDONED,
+ ROLL_FORWARD_ABANDONED,
+ CLOSE_ABANDONED
+ }
+
+ public static string CommitErrorToString(CommitErrorType type)
+ {
+ return type switch
+ {
+ CommitErrorType.VERIFY_FAIL => "MRT verify failed. MRT aborted.",
+ CommitErrorType.VERIFY_FAIL_CLOSE_ABANDONED => "MRT verify failed. MRT aborted. MRT client close abandoned. Server will eventually close the MRT.",
+ CommitErrorType.VERIFY_FAIL_ABORT_ABANDONED => "MRT verify failed. MRT client abort abandoned. Server will eventually abort the MRT.",
+ CommitErrorType.MARK_ROLL_FORWARD_ABANDONED => "MRT client mark roll forward abandoned. Server will eventually abort the MRT.",
+ CommitErrorType.ROLL_FORWARD_ABANDONED => "MRT client roll forward abandoned. Server will eventually commit the MRT.",
+ CommitErrorType.CLOSE_ABANDONED => "MRT has been rolled forward, but MRT client close was abandoned. Server will eventually close the MRT.",
+ _ => "Unexpected CommitErrorType"
+ };
+ }
+ }
+}
diff --git a/AerospikeClient/Main/IAerospikeClient.cs b/AerospikeClient/Main/IAerospikeClient.cs
index f8180f88..8e39e632 100644
--- a/AerospikeClient/Main/IAerospikeClient.cs
+++ b/AerospikeClient/Main/IAerospikeClient.cs
@@ -73,6 +73,17 @@ public interface IAerospikeClient
///
BatchUDFPolicy BatchUDFPolicyDefault { get; set; }
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ BatchPolicy TranVerifyPolicyDefault { get; set; }
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ BatchPolicy TranRollPolicyDefault { get; set; }
+
///
/// Default info policy that is used when info command policy is null.
///
@@ -118,6 +129,30 @@ public interface IAerospikeClient
///
ClusterStats GetClusterStats();
+ //-------------------------------------------------------
+ // Multi-Record Transactions
+ //-------------------------------------------------------
+
+ ///
+ /// Attempt to commit the given multi-record transaction. First, the expected record versions are
+ /// sent to the server nodes for verification.If all nodes return success, the transaction is
+ /// committed.Otherwise, the transaction is aborted.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ void Commit(Tran tran);
+
+ ///
+ /// Abort and rollback the given multi-record transaction.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ void Abort(Tran tran);
+
//-------------------------------------------------------
// Write Record Operations
//-------------------------------------------------------
diff --git a/AerospikeClient/Main/ResultCode.cs b/AerospikeClient/Main/ResultCode.cs
index e1093899..83a0416b 100644
--- a/AerospikeClient/Main/ResultCode.cs
+++ b/AerospikeClient/Main/ResultCode.cs
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2023 Aerospike, Inc.
+ * Copyright 2012-2024 Aerospike, Inc.
*
* Portions may be licensed to Aerospike, Inc. under one or more contributor
* license agreements.
@@ -21,6 +21,12 @@ namespace Aerospike.Client
///
public sealed class ResultCode
{
+ ///
+ /// Multi-record transaction failed
+ /// Value: -17
+ ///
+ public const int TRAN_FAILED = -17;
+
///
/// One or more keys failed in a batch.
/// Value: -16
@@ -276,6 +282,24 @@ public sealed class ResultCode
///
public const int LOST_CONFLICT = 28;
+ ///
+ /// MRT record blocked by a different transaction.
+ /// Value: 29
+ ///
+ public const int MRT_BLOCKED = 29;
+
+ ///
+ /// MRT read verify failed. Some other command changed record outside of the transaction.
+ /// Value: 30
+ ///
+ public const int MRT_CONFLICT = 30;
+
+ ///
+ /// MRT deadline reached without a successful commit or abort.
+ /// Value: 31
+ ///
+ public const int MRT_EXPIRED = 31;
+
///
/// There are no more records left for query.
/// Value: 50
@@ -542,6 +566,9 @@ public static string GetResultString(int resultCode)
{
switch (resultCode)
{
+ case TRAN_FAILED:
+ return "Multi-record transaction failed";
+
case BATCH_FAILED:
return "One or more keys failed in a batch";
@@ -668,6 +695,15 @@ public static string GetResultString(int resultCode)
case LOST_CONFLICT:
return "Transaction failed due to conflict with XDR";
+ case MRT_BLOCKED:
+ return "MRT record blocked by a different transaction";
+
+ case MRT_CONFLICT:
+ return "MRT verify failed";
+
+ case MRT_EXPIRED:
+ return "MRT expired";
+
case QUERY_END:
return "Query end";
diff --git a/AerospikeClient/Main/Tran.cs b/AerospikeClient/Main/Tran.cs
new file mode 100644
index 00000000..009cae98
--- /dev/null
+++ b/AerospikeClient/Main/Tran.cs
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+
+namespace Aerospike.Client
+{
+ ///
+ /// Mutli-record transaction (MRT). Each command in the MRT must use the same namespace.
+ ///
+ public class Tran
+ {
+ public long Id { get; private set; }
+ public ConcurrentDictionary Reads { get; private set; }
+ public HashSet Writes { get; private set; }
+ public string Ns { get; set; }
+ public int Deadline { get; set; }
+
+ private bool rollAttempted;
+
+ ///
+ /// Create MRT, assign random transaction id and initialize reads/writes hashmaps with default capacities.
+ ///
+ public Tran()
+ {
+ Id = CreateId();
+ Reads = new ConcurrentDictionary();
+ Writes = new HashSet();
+ }
+
+ ///
+ /// Create MRT, assign random transaction id and initialize reads/writes hashmaps with given capacities.
+ ///
+ /// expected number of record reads in the MRT. Minimum value is 16.
+ /// expected number of record writes in the MRT. Minimum value is 16.
+ public Tran(int readsCapacity, int writesCapacity)
+ {
+ if (readsCapacity < 16)
+ {
+ readsCapacity = 16;
+ }
+
+ if (writesCapacity < 16)
+ {
+ writesCapacity = 16;
+ }
+
+ Id = CreateId();
+ Reads = new ConcurrentDictionary(-1, readsCapacity); // TODO: concurrency level
+ Writes = new HashSet(writesCapacity);
+ }
+
+ private static long CreateId()
+ {
+ // An id of zero is considered invalid. Create random numbers
+ // in a loop until non-zero is returned.
+ Random r = new();
+ long id = r.NextInt64();
+
+ while (id == 0)
+ {
+ id = r.NextInt64();
+ }
+ return id;
+ }
+
+ ///
+ /// Process the results of a record read. For internal use only.
+ ///
+ ///
+ ///
+ internal void OnRead(Key key, long? version)
+ {
+ Ns = key.ns;
+
+ if (version.HasValue)
+ {
+ Reads.TryAdd(key, version.Value);
+ }
+ }
+
+ ///
+ /// Get record version for a given key.
+ ///
+ ///
+ ///
+ public long GetReadVersion(Key key)
+ {
+ return Reads[key];
+ }
+
+ ///
+ /// Process the results of a record write. For internal use only.
+ ///
+ ///
+ ///
+ ///
+ public void OnWrite(Key key, long? version, int resultCode)
+ {
+ // Write commands set namespace prior to sending the command, so there is
+ // no need to call it here when receiving the response.
+ if (version.HasValue)
+ {
+ Reads.TryAdd(key, version.Value);
+ }
+ else
+ {
+ if (resultCode == ResultCode.OK)
+ {
+ Reads.Remove(key, out _);
+ Writes.Add(key);
+ }
+ }
+ }
+
+ public void SetRollAttempted()
+ {
+ if (rollAttempted)
+ {
+ throw new AerospikeException(ResultCode.PARAMETER_ERROR,
+ "commit() or abort() may only be called once for a given MRT");
+ }
+ rollAttempted = true;
+ }
+
+ public void Clear()
+ {
+ Ns = null;
+ Deadline = 0;
+ Reads.Clear();
+ Writes.Clear();
+ }
+ }
+}
diff --git a/AerospikeClient/Policy/BatchPolicy.cs b/AerospikeClient/Policy/BatchPolicy.cs
index 46b54fb7..0e040327 100644
--- a/AerospikeClient/Policy/BatchPolicy.cs
+++ b/AerospikeClient/Policy/BatchPolicy.cs
@@ -23,7 +23,7 @@ namespace Aerospike.Client
///
/// Batch parent policy.
///
- public sealed class BatchPolicy : Policy
+ public class BatchPolicy : Policy
{
///
/// Maximum number of concurrent synchronous batch node request threads to server nodes.
diff --git a/AerospikeClient/Policy/ClientPolicy.cs b/AerospikeClient/Policy/ClientPolicy.cs
index 9947761f..be3dca01 100644
--- a/AerospikeClient/Policy/ClientPolicy.cs
+++ b/AerospikeClient/Policy/ClientPolicy.cs
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2022 Aerospike, Inc.
+ * Copyright 2012-2024 Aerospike, Inc.
*
* Portions may be licensed to Aerospike, Inc. under one or more contributor
* license agreements.
@@ -225,7 +225,18 @@ public class ClientPolicy
/// Default user defined function policy used in batch UDF excecute commands.
///
public BatchUDFPolicy batchUDFPolicyDefault = new BatchUDFPolicy();
-
+
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ public TranVerifyPolicy tranVerifyPolicyDefault = new TranVerifyPolicy();
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ public TranRollPolicy tranRollPolicyDefault = new TranRollPolicy();
+
///
/// Default info policy that is used when info command's policy is null.
///
@@ -326,6 +337,8 @@ public ClientPolicy(ClientPolicy other)
this.batchWritePolicyDefault = new BatchWritePolicy(other.batchWritePolicyDefault);
this.batchDeletePolicyDefault = new BatchDeletePolicy(other.batchDeletePolicyDefault);
this.batchUDFPolicyDefault = new BatchUDFPolicy(other.batchUDFPolicyDefault);
+ this.tranVerifyPolicyDefault = new TranVerifyPolicy(other.tranVerifyPolicyDefault);
+ this.tranRollPolicyDefault = new TranRollPolicy(other.tranRollPolicyDefault);
this.infoPolicyDefault = new InfoPolicy(other.infoPolicyDefault);
this.tlsPolicy = (other.tlsPolicy != null) ? new TlsPolicy(other.tlsPolicy) : null;
this.ipMap = other.ipMap;
diff --git a/AerospikeClient/Policy/Policy.cs b/AerospikeClient/Policy/Policy.cs
index 4fbe4f93..757f2129 100644
--- a/AerospikeClient/Policy/Policy.cs
+++ b/AerospikeClient/Policy/Policy.cs
@@ -25,6 +25,14 @@ namespace Aerospike.Client
///
public class Policy
{
+ ///
+ /// Multi-record transaction identifier.
+ ///
+ /// Default: null
+ ///
+ ///
+ public Tran Tran { get; set; }
+
///
/// Read policy for AP (availability) namespaces.
///
@@ -275,6 +283,7 @@ public class Policy
///
public Policy(Policy other)
{
+ this.Tran = other.Tran;
this.readModeAP = other.readModeAP;
this.readModeSC = other.readModeSC;
this.replica = other.replica;
diff --git a/AerospikeClient/Policy/TranRollPolicy.cs b/AerospikeClient/Policy/TranRollPolicy.cs
new file mode 100644
index 00000000..c123c2fa
--- /dev/null
+++ b/AerospikeClient/Policy/TranRollPolicy.cs
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) policy fields used to batch roll forward/backward records on
+ /// commit or abort.Used a placeholder for now as there are no additional fields beyond BatchPolicy.
+ ///
+ public sealed class TranRollPolicy : BatchPolicy
+ {
+ ///
+ /// Copy policy from another policy.
+ ///
+ public TranRollPolicy(TranRollPolicy other) :
+ base(other)
+ {
+ }
+
+ ///
+ /// Default constructor.
+ ///
+ public TranRollPolicy()
+ {
+ replica = Replica.MASTER;
+ maxRetries = 5;
+ totalTimeout = 10000;
+ sleepBetweenRetries = 1000;
+ }
+ }
+}
diff --git a/AerospikeClient/Policy/TranVerifyPolicy.cs b/AerospikeClient/Policy/TranVerifyPolicy.cs
new file mode 100644
index 00000000..e641ba33
--- /dev/null
+++ b/AerospikeClient/Policy/TranVerifyPolicy.cs
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) policy fields used to batch verify record versions on commit.
+ /// Used a placeholder for now as there are no additional fields beyond BatchPolicy.
+ ///
+ public sealed class TranVerifyPolicy : BatchPolicy
+ {
+ ///
+ /// Copy policy from another policy.
+ ///
+ public TranVerifyPolicy(TranVerifyPolicy other) :
+ base(other)
+ {
+ }
+
+ ///
+ /// Default constructor.
+ ///
+ public TranVerifyPolicy()
+ {
+ readModeSC = ReadModeSC.LINEARIZE;
+ replica = Replica.MASTER;
+ maxRetries = 5;
+ totalTimeout = 10000;
+ sleepBetweenRetries = 1000;
+ }
+ }
+}
diff --git a/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs b/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
index afa3028e..e19597df 100644
--- a/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
+++ b/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
@@ -100,6 +100,17 @@ public class AerospikeClientProxy : IDisposable, IAerospikeClient
///
public BatchUDFPolicy batchUDFPolicyDefault { get; set; }
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ public BatchPolicy tranVerifyPolicyDefault { get; set; }
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ public BatchPolicy tranRollPolicyDefault { get; set; }
+
///
/// Default info policy that is used when info command policy is null.
///
@@ -150,6 +161,8 @@ public AerospikeClientProxy(ClientPolicy policy, params Host[] hosts)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
+ this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
+ this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -304,6 +317,25 @@ public BatchUDFPolicy BatchUDFPolicyDefault
set { batchUDFPolicyDefault = value; }
}
+ ///
+ /// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
+ ///
+ public BatchPolicy TranVerifyPolicyDefault
+ {
+ get { return tranVerifyPolicyDefault; }
+ set { tranVerifyPolicyDefault = value; }
+ }
+
+ ///
+ /// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
+ /// or back(abort) in a batch.
+ ///
+ public BatchPolicy TranRollPolicyDefault
+ {
+ get { return tranRollPolicyDefault; }
+ set { tranRollPolicyDefault = value; }
+ }
+
///
/// Default info policy that is used when info command policy is null.
///
@@ -424,6 +456,36 @@ private string GetVersion()
return response.Version;
}
+ //-------------------------------------------------------
+ // Multi-Record Transactions
+ //-------------------------------------------------------
+
+ ///
+ /// Attempt to commit the given multi-record transaction. First, the expected record versions are
+ /// sent to the server nodes for verification.If all nodes return success, the transaction is
+ /// committed.Otherwise, the transaction is aborted.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ public void Commit(Tran tran)
+ {
+
+ }
+
+ ///
+ /// Abort and rollback the given multi-record transaction.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// multi-record transaction
+ public void Abort(Tran tran)
+ {
+
+ }
+
//-------------------------------------------------------
// Write Record Operations
//-------------------------------------------------------
@@ -979,7 +1041,7 @@ public Record Join(BatchPolicy policy, Key key, params Join[] joins)
/// if command fails
public Record Operate(WritePolicy policy, Key key, params Operation[] operations)
{
- OperateArgs args = new(policy, writePolicyDefault, operatePolicyReadDefault, key, operations);
+ OperateArgs args = new(policy, writePolicyDefault, operatePolicyReadDefault, operations);
Buffer buffer = new();
OperateCommandProxy command = new(buffer, Channel, key, args);
command.Execute();
diff --git a/AerospikeClientProxy/Proxy/AsyncClientProxy.cs b/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
index 2af97038..b9990631 100644
--- a/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
+++ b/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
@@ -111,6 +111,44 @@ public AsyncClientProxy(AsyncClientPolicy policy, params Host[] hosts)
{
}
+ //-------------------------------------------------------
+ // Multi-Record Transactions
+ //-------------------------------------------------------
+
+ ///
+ /// Asynchronously attempt to commit the given multi-record transaction. First, the expected
+ /// record versions are sent to the server nodes for verification.If all nodes return success,
+ /// the transaction is committed.Otherwise, the transaction is aborted.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
+ ///
+ /// where to send results
+ /// multi-record transaction
+ public void Commit(CommitListener listener, Tran tran)
+ {
+
+ }
+
+ ///
+ /// Asynchronously abort and rollback the given multi-record transaction.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ ///
/// This method registers the command with an event loop and returns.
/// The event loop thread will process the command and send the results to the listener.
- ///
+ ///
/// Requires server version 8.0+
- ///
+ ///
///
/// where to send results
- /// multi-record transaction
- public void Commit(CommitListener listener, Tran tran)
+ /// multi-record transaction
+ public void Commit(CommitListener listener, Txn txn)
{
- tran.SetRollAttempted();
+ if (!txn.SetRollAttempted())
+ {
+ listener.OnSuccess(CommitStatus.CommitStatusType.ALREADY_ATTEMPTED);
+ }
- //AsyncTranRoll tm = new AsyncTranRoll(
- //cluster, tranVerifyPolicyDefault, tranRollPolicyDefault, tran
- //);
- //tm.Commit(listener);
+ AsyncTxnRoll atr = new(
+ cluster, txnVerifyPolicyDefault, txnRollPolicyDefault, txn
+ );
+ atr.Commit(listener);
}
///
/// Asynchronously abort and rollback the given multi-record transaction.
- ///
- /// This method registers the command with an event loop and returns.
- /// The event loop thread will process the command and send the results to the listener.
- ///
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
/// Requires server version 8.0+
- ///
+ ///
///
/// where to send results
- /// multi-record transaction
- public void Abort(AbortListener listener, Tran tran)
+ /// multi-record transaction
+ public void Abort(AbortListener listener, Txn txn)
{
- tran.SetRollAttempted();
+ if (!txn.SetRollAttempted())
+ {
+ listener.OnSuccess(AbortStatus.AbortStatusType.ALREADY_ATTEMPTED);
+ }
- //AsyncTranRoll tm = new AsyncTranRoll(cluster, null, tranRollPolicyDefault, tran);
- //tm.Abort(listener);
+ AsyncTxnRoll atr = new(cluster, null, txnRollPolicyDefault, txn);
+ atr.Abort(listener);
}
//-------------------------------------------------------
@@ -219,7 +225,7 @@ public void Put(WritePolicy policy, WriteListener listener, Key key, params Bin[
policy = writePolicyDefault;
}
AsyncWrite async = new AsyncWrite(cluster, policy, listener, key, bins, Operation.Type.WRITE);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
//-------------------------------------------------------
@@ -269,7 +275,7 @@ public void Append(WritePolicy policy, WriteListener listener, Key key, params B
policy = writePolicyDefault;
}
AsyncWrite async = new AsyncWrite(cluster, policy, listener, key, bins, Operation.Type.APPEND);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
///
@@ -315,7 +321,7 @@ public void Prepend(WritePolicy policy, WriteListener listener, Key key, params
policy = writePolicyDefault;
}
AsyncWrite async = new AsyncWrite(cluster, policy, listener, key, bins, Operation.Type.PREPEND);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
//-------------------------------------------------------
@@ -363,7 +369,7 @@ public void Add(WritePolicy policy, WriteListener listener, Key key, params Bin[
policy = writePolicyDefault;
}
AsyncWrite async = new AsyncWrite(cluster, policy, listener, key, bins, Operation.Type.ADD);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
//-------------------------------------------------------
@@ -401,7 +407,7 @@ public void Delete(WritePolicy policy, DeleteListener listener, Key key)
policy = writePolicyDefault;
}
AsyncDelete async = new AsyncDelete(cluster, policy, key, listener);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
///
@@ -457,7 +463,8 @@ public void Delete(BatchPolicy batchPolicy, BatchDeletePolicy deletePolicy, Batc
BatchAttr attr = new BatchAttr();
attr.SetDelete(deletePolicy);
- new AsyncBatchOperateRecordArrayExecutor(cluster, batchPolicy, listener, keys, null, attr);
+ AsyncBatchOperateRecordArrayExecutor executor = new(cluster, batchPolicy, listener, keys, null, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
///
@@ -497,7 +504,8 @@ public void Delete(BatchPolicy batchPolicy, BatchDeletePolicy deletePolicy, Batc
BatchAttr attr = new BatchAttr();
attr.SetDelete(deletePolicy);
- new AsyncBatchOperateRecordSequenceExecutor(cluster, batchPolicy, listener, keys, null, attr);
+ AsyncBatchOperateRecordSequenceExecutor executor = new(cluster, batchPolicy, listener, keys, null, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
//-------------------------------------------------------
@@ -537,7 +545,7 @@ public void Touch(WritePolicy policy, WriteListener listener, Key key)
policy = writePolicyDefault;
}
AsyncTouch async = new AsyncTouch(cluster, policy, listener, key);
- async.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, async);
}
//-------------------------------------------------------
@@ -574,6 +582,9 @@ public void Exists(Policy policy, ExistsListener listener, Key key)
{
policy = readPolicyDefault;
}
+
+ policy.Txn?.SetNamespace(key.ns);
+
AsyncExists async = new AsyncExists(cluster, policy, key, listener);
async.Execute();
}
@@ -613,7 +624,10 @@ public void Exists(BatchPolicy policy, ExistsArrayListener listener, Key[] keys)
{
policy = batchPolicyDefault;
}
- new AsyncBatchExistsArrayExecutor(cluster, policy, keys, listener);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchExistsArrayExecutor executor = new(cluster, policy, keys, listener);
+ executor.Execute();
}
///
@@ -636,7 +650,10 @@ public void Exists(BatchPolicy policy, ExistsSequenceListener listener, Key[] ke
{
policy = batchPolicyDefault;
}
- new AsyncBatchExistsSequenceExecutor(cluster, policy, keys, listener);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchExistsSequenceExecutor executor = new(cluster, policy, keys, listener);
+ executor.Execute();
}
//-------------------------------------------------------
@@ -673,6 +690,9 @@ public void Get(Policy policy, RecordListener listener, Key key)
{
policy = readPolicyDefault;
}
+
+ policy.Txn?.SetNamespace(key.ns);
+
AsyncRead async = new AsyncRead(cluster, policy, listener, key, (string[])null);
async.Execute();
}
@@ -709,6 +729,9 @@ public void Get(Policy policy, RecordListener listener, Key key, params string[]
{
policy = readPolicyDefault;
}
+
+ policy.Txn?.SetNamespace(key.ns);
+
AsyncRead async = new AsyncRead(cluster, policy, listener, key, binNames);
async.Execute();
}
@@ -743,6 +766,9 @@ public void GetHeader(Policy policy, RecordListener listener, Key key)
{
policy = readPolicyDefault;
}
+
+ policy.Txn?.SetNamespace(key.ns);
+
AsyncReadHeader async = new AsyncReadHeader(cluster, policy, listener, key);
async.Execute();
}
@@ -795,7 +821,10 @@ public void Get(BatchPolicy policy, BatchListListener listener, List
{
policy = batchPolicyDefault;
}
- new AsyncBatchReadListExecutor(cluster, policy, listener, records);
+ policy.Txn?.SetNamespace(records);
+
+ AsyncBatchReadListExecutor executor = new(cluster, policy, listener, records);
+ executor.Execute();
}
///
@@ -823,7 +852,10 @@ public void Get(BatchPolicy policy, BatchSequenceListener listener, List
@@ -867,7 +899,10 @@ public void Get(BatchPolicy policy, RecordArrayListener listener, Key[] keys)
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetArrayExecutor(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_GET_ALL, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetArrayExecutor executor = new(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_GET_ALL, false);
+ executor.Execute();
}
///
@@ -893,7 +928,10 @@ public void Get(BatchPolicy policy, RecordSequenceListener listener, Key[] keys)
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetSequenceExecutor(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_GET_ALL, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_GET_ALL, false);
+ executor.Execute();
}
///
@@ -939,7 +977,10 @@ public void Get(BatchPolicy policy, RecordArrayListener listener, Key[] keys, pa
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetArrayExecutor(cluster, policy, listener, keys, binNames, null, Command.INFO1_READ, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetArrayExecutor executor = new(cluster, policy, listener, keys, binNames, null, Command.INFO1_READ, false);
+ executor.Execute();
}
///
@@ -966,7 +1007,10 @@ public void Get(BatchPolicy policy, RecordSequenceListener listener, Key[] keys,
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetSequenceExecutor(cluster, policy, listener, keys, binNames, null, Command.INFO1_READ, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, binNames, null, Command.INFO1_READ, false);
+ executor.Execute();
}
///
@@ -1014,7 +1058,11 @@ public void Get(BatchPolicy policy, RecordArrayListener listener, Key[] keys, pa
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetArrayExecutor(cluster, policy, listener, keys, null, ops, Command.INFO1_READ, true);
+
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetArrayExecutor executor = new(cluster, policy, listener, keys, null, ops, Command.INFO1_READ, true);
+ executor.Execute();
}
///
@@ -1043,7 +1091,11 @@ public void Get(BatchPolicy policy, RecordSequenceListener listener, Key[] keys,
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetSequenceExecutor(cluster, policy, listener, keys, null, ops, Command.INFO1_READ, true);
+
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, null, ops, Command.INFO1_READ, true);
+ executor.Execute();
}
///
@@ -1087,7 +1139,10 @@ public void GetHeader(BatchPolicy policy, RecordArrayListener listener, Key[] ke
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetArrayExecutor(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_NOBINDATA, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetArrayExecutor executor = new(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_NOBINDATA, false);
+ executor.Execute();
}
///
@@ -1113,7 +1168,10 @@ public void GetHeader(BatchPolicy policy, RecordSequenceListener listener, Key[]
{
policy = batchPolicyDefault;
}
- new AsyncBatchGetSequenceExecutor(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_NOBINDATA, false);
+ policy.Txn?.SetNamespace(keys);
+
+ AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, null, null, Command.INFO1_READ | Command.INFO1_NOBINDATA, false);
+ executor.Execute();
}
//-------------------------------------------------------
@@ -1162,8 +1220,20 @@ public Task Operate(WritePolicy policy, CancellationToken token, Key key
public void Operate(WritePolicy policy, RecordListener listener, Key key, params Operation[] ops)
{
OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, ops);
- AsyncOperate async = new AsyncOperate(cluster, listener, key, args);
- async.Execute();
+ policy = args.writePolicy;
+
+ if (args.hasWrite)
+ {
+ AsyncOperateWrite async = new(cluster, listener, key, args);
+ AsyncTxnMonitor.Execute(cluster, policy, async);
+ }
+ else
+ {
+ policy.Txn?.SetNamespace(key.ns);
+
+ AsyncOperateRead async = new(cluster, listener, key, args);
+ async.Execute();
+ }
}
//-------------------------------------------------------
@@ -1217,7 +1287,8 @@ public void Operate(BatchPolicy policy, BatchOperateListListener listener, List<
{
policy = batchParentPolicyWriteDefault;
}
- new AsyncBatchOperateListExecutor(cluster, policy, listener, records);
+ AsyncBatchOperateListExecutor executor = new(cluster, policy, listener, records);
+ AsyncTxnMonitor.ExecuteBatch(policy, executor, records);
}
///
@@ -1251,7 +1322,8 @@ public void Operate(BatchPolicy policy, BatchRecordSequenceListener listener, Li
{
policy = batchParentPolicyWriteDefault;
}
- new AsyncBatchOperateSequenceExecutor(cluster, policy, listener, records);
+ AsyncBatchOperateSequenceExecutor executor = new(cluster, policy, listener, records);
+ AsyncTxnMonitor.ExecuteBatch(policy, executor, records);
}
///
@@ -1315,7 +1387,8 @@ public void Operate(BatchPolicy batchPolicy, BatchWritePolicy writePolicy, Batch
}
BatchAttr attr = new BatchAttr(batchPolicy, writePolicy, ops);
- new AsyncBatchOperateRecordArrayExecutor(cluster, batchPolicy, listener, keys, ops, attr);
+ AsyncBatchOperateRecordArrayExecutor executor = new(cluster, batchPolicy, listener, keys, ops, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
///
@@ -1358,7 +1431,8 @@ public void Operate(BatchPolicy batchPolicy, BatchWritePolicy writePolicy, Batch
}
BatchAttr attr = new BatchAttr(batchPolicy, writePolicy, ops);
- new AsyncBatchOperateRecordSequenceExecutor(cluster, batchPolicy, listener, keys, ops, attr);
+ AsyncBatchOperateRecordSequenceExecutor executor = new(cluster, batchPolicy, listener, keys, ops, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
//-------------------------------------------------------
@@ -1468,7 +1542,7 @@ public void Execute(WritePolicy policy, ExecuteListener listener, Key key, strin
policy = writePolicyDefault;
}
AsyncExecute command = new AsyncExecute(cluster, policy, listener, key, packageName, functionName, functionArgs);
- command.Execute();
+ AsyncTxnMonitor.Execute(cluster, policy, command);
}
///
@@ -1534,7 +1608,8 @@ public void Execute(BatchPolicy batchPolicy, BatchUDFPolicy udfPolicy, BatchReco
BatchAttr attr = new BatchAttr();
attr.SetUDF(udfPolicy);
- new AsyncBatchUDFArrayExecutor(cluster, batchPolicy, listener, keys, packageName, functionName, argBytes, attr);
+ AsyncBatchUDFArrayExecutor executor = new(cluster, batchPolicy, listener, keys, packageName, functionName, argBytes, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
///
@@ -1581,7 +1656,8 @@ public void Execute(BatchPolicy batchPolicy, BatchUDFPolicy udfPolicy, BatchReco
BatchAttr attr = new BatchAttr();
attr.SetUDF(udfPolicy);
- new AsyncBatchUDFSequenceExecutor(cluster, batchPolicy, listener, keys, packageName, functionName, argBytes, attr);
+ AsyncBatchUDFSequenceExecutor executor = new(cluster, batchPolicy, listener, keys, packageName, functionName, argBytes, attr);
+ AsyncTxnMonitor.ExecuteBatch(batchPolicy, executor, keys);
}
//-------------------------------------------------------
diff --git a/AerospikeClient/Async/AsyncCommand.cs b/AerospikeClient/Async/AsyncCommand.cs
index 4a295901..8a70a0c0 100644
--- a/AerospikeClient/Async/AsyncCommand.cs
+++ b/AerospikeClient/Async/AsyncCommand.cs
@@ -908,6 +908,12 @@ private void NotifyFailure(AerospikeException ae)
ae.Policy = policy;
ae.Iteration = iteration;
ae.SetInDoubt(IsWrite(), commandSentCounter);
+
+ if (ae.InDoubt)
+ {
+ OnInDoubt();
+ }
+
OnFailure(ae);
}
catch (Exception e)
@@ -943,6 +949,12 @@ internal void ReleaseBuffer()
}
}
+ // Do nothing by default. Write commands will override this method.
+ protected internal virtual void OnInDoubt()
+ {
+
+ }
+
protected internal virtual bool RetryBatch()
{
return false;
diff --git a/AerospikeClient/Async/AsyncDelete.cs b/AerospikeClient/Async/AsyncDelete.cs
index cd63221b..99e79ea1 100644
--- a/AerospikeClient/Async/AsyncDelete.cs
+++ b/AerospikeClient/Async/AsyncDelete.cs
@@ -17,31 +17,22 @@
namespace Aerospike.Client
{
- public sealed class AsyncDelete : AsyncSingleCommand
+ public sealed class AsyncDelete : AsyncWriteBase
{
- private readonly WritePolicy writePolicy;
private readonly DeleteListener listener;
- private readonly Key key;
- private readonly Partition partition;
private bool existed;
public AsyncDelete(AsyncCluster cluster, WritePolicy writePolicy, Key key, DeleteListener listener)
- : base(cluster, writePolicy)
+ : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
this.listener = listener;
- this.key = key;
- this.partition = Partition.Write(cluster, policy, key);
- cluster.AddTran();
+ cluster.AddCommand();
}
public AsyncDelete(AsyncDelete other)
: base(other)
{
- this.writePolicy = other.writePolicy;
this.listener = other.listener;
- this.key = other.key;
- this.partition = other.partition;
}
protected internal override AsyncCommand CloneCommand()
@@ -49,40 +40,25 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncDelete(this);
}
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
-
protected internal override void WriteBuffer()
{
- SetDelete(writePolicy, key);
+ SetDelete(writePolicy, Key);
}
- protected internal override void ParseResult()
+ protected internal override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
+ ParseHeader();
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
existed = true;
- return;
+ return true;
}
if (resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
existed = false;
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -92,23 +68,17 @@ protected internal override void ParseResult()
throw new AerospikeException(resultCode);
}
existed = true;
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
{
- listener.OnSuccess(key, existed);
+ listener.OnSuccess(Key, existed);
}
}
diff --git a/AerospikeClient/Async/AsyncExecute.cs b/AerospikeClient/Async/AsyncExecute.cs
index d3cea1f3..01503c9d 100644
--- a/AerospikeClient/Async/AsyncExecute.cs
+++ b/AerospikeClient/Async/AsyncExecute.cs
@@ -15,15 +15,17 @@
* the License.
*/
+using System;
+
namespace Aerospike.Client
{
- public sealed class AsyncExecute : AsyncRead
+ public sealed class AsyncExecute : AsyncWriteBase
{
- private readonly WritePolicy writePolicy;
private readonly ExecuteListener executeListener;
private readonly string packageName;
private readonly string functionName;
private readonly Value[] args;
+ private Record record;
public AsyncExecute
(
@@ -36,7 +38,6 @@ public AsyncExecute
Value[] args
) : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
this.executeListener = listener;
this.packageName = packageName;
this.functionName = functionName;
@@ -46,7 +47,6 @@ Value[] args
public AsyncExecute(AsyncExecute other)
: base(other)
{
- this.writePolicy = other.writePolicy;
this.executeListener = other.executeListener;
this.packageName = other.packageName;
this.functionName = other.functionName;
@@ -58,35 +58,66 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncExecute(this);
}
- protected internal override bool IsWrite()
+ protected internal override void WriteBuffer()
{
- return true;
+ SetUdf(writePolicy, Key, packageName, functionName, args);
}
- protected internal override Node GetNode(Cluster cluster)
+ protected internal override bool ParseResult()
{
- return partition.GetNodeWrite(cluster);
- }
+ ParseHeader();
+ ParseFields(policy.Txn, Key, true);
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
+ if (resultCode == ResultCode.OK)
+ {
+ record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, false);
+ return true;
+ }
- protected internal override void WriteBuffer()
- {
- SetUdf(writePolicy, key, packageName, functionName, args);
- }
+ if (resultCode == ResultCode.UDF_BAD_RESPONSE)
+ {
+ record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, false);
+ HandleUdfError(resultCode);
+ return true;
+ }
+
+ if (resultCode == ResultCode.FILTERED_OUT)
+ {
+ if (policy.failOnFilteredOut)
+ {
+ throw new AerospikeException(resultCode);
+ }
+ return true;
+ }
- protected internal override void HandleNotFound(int resultCode)
- {
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
+ private void HandleUdfError(int resultCode)
{
- partition.PrepareRetryWrite(timeout);
- return true;
+ string ret = (string)record.bins["FAILURE"];
+
+ if (ret == null)
+ {
+ throw new AerospikeException(resultCode);
+ }
+
+ String message;
+ int code;
+
+ try
+ {
+ string[] list = ret.Split(":");
+ Int32.TryParse(list[2].Trim(), out code);
+ message = list[0] + ':' + list[1] + ' ' + list[3];
+ }
+ catch (Exception e)
+ {
+ // Use generic exception if parse error occurs.
+ throw new AerospikeException(resultCode, ret);
+ }
+
+ throw new AerospikeException(code, message);
}
protected internal override void OnSuccess()
@@ -94,7 +125,7 @@ protected internal override void OnSuccess()
if (executeListener != null)
{
object obj = ParseEndResult();
- executeListener.OnSuccess(key, obj);
+ executeListener.OnSuccess(Key, obj);
}
}
diff --git a/AerospikeClient/Async/AsyncExists.cs b/AerospikeClient/Async/AsyncExists.cs
index 4217acb5..e57daa30 100644
--- a/AerospikeClient/Async/AsyncExists.cs
+++ b/AerospikeClient/Async/AsyncExists.cs
@@ -17,28 +17,21 @@
namespace Aerospike.Client
{
- public sealed class AsyncExists : AsyncSingleCommand
+ public sealed class AsyncExists : AsyncReadBase
{
private readonly ExistsListener listener;
- private readonly Key key;
- private readonly Partition partition;
private bool exists;
public AsyncExists(AsyncCluster cluster, Policy policy, Key key, ExistsListener listener)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
this.listener = listener;
- this.key = key;
- this.partition = Partition.Read(cluster, policy, key);
- cluster.AddTran();
}
public AsyncExists(AsyncExists other)
: base(other)
{
this.listener = other.listener;
- this.key = other.key;
- this.partition = other.partition;
}
protected internal override AsyncCommand CloneCommand()
@@ -46,35 +39,25 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncExists(this);
}
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
- }
-
protected internal override void WriteBuffer()
{
SetExists(policy, key);
}
- protected internal override void ParseResult()
+ protected internal override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
+ ParseHeader();
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
exists = true;
- return;
+ return true;
}
if (resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
exists = false;
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -84,18 +67,12 @@ protected internal override void ParseResult()
throw new AerospikeException(resultCode);
}
exists = true;
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
diff --git a/AerospikeClient/Main/AbortError.cs b/AerospikeClient/Async/AsyncOperateRead.cs
similarity index 56%
rename from AerospikeClient/Main/AbortError.cs
rename to AerospikeClient/Async/AsyncOperateRead.cs
index 83997c91..076fc4ca 100644
--- a/AerospikeClient/Main/AbortError.cs
+++ b/AerospikeClient/Async/AsyncOperateRead.cs
@@ -17,25 +17,25 @@
namespace Aerospike.Client
{
- ///
- /// Multi-record transaction (MRT) abort error status.
- ///
- public static class AbortError
+ public sealed class AsyncOperateRead : AsyncRead
{
- public enum AbortErrorType
+ private readonly OperateArgs args;
+
+ public AsyncOperateRead(AsyncCluster cluster, RecordListener listener, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, listener, key, true)
+ {
+ this.args = args;
+ }
+
+ public AsyncOperateRead(AsyncOperateRead other)
+ : base(other)
{
- ROLL_BACK_ABANDONED,
- CLOSE_ABANDONED
+ this.args = other.args;
}
- public static string AbortErrorToString(AbortErrorType error)
+ protected internal override void WriteBuffer()
{
- return error switch
- {
- AbortErrorType.ROLL_BACK_ABANDONED => "MRT client roll back abandoned. Server will eventually abort the MRT.",
- AbortErrorType.CLOSE_ABANDONED => "MRT has been rolled back, but MRT client close was abandoned. Server will eventually close the MRT.",
- _ => "Unexpected AbortErrorType"
- };
+ SetOperate(args.writePolicy, key, args);
}
}
}
diff --git a/AerospikeClient/Async/AsyncOperateWrite.cs b/AerospikeClient/Async/AsyncOperateWrite.cs
new file mode 100644
index 00000000..2e2093be
--- /dev/null
+++ b/AerospikeClient/Async/AsyncOperateWrite.cs
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+using System;
+
+namespace Aerospike.Client
+{
+ public sealed class AsyncOperateWrite : AsyncWriteBase
+ {
+ private readonly RecordListener listener;
+ private readonly OperateArgs args;
+ private Record record;
+
+ public AsyncOperateWrite(AsyncCluster cluster, RecordListener listener, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, key)
+ {
+ this.args = args;
+ }
+
+ public AsyncOperateWrite(AsyncOperateWrite other)
+ : base(other)
+ {
+ this.args = other.args;
+ }
+
+ protected internal override AsyncCommand CloneCommand()
+ {
+ return new AsyncOperateWrite(this);
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetOperate(args.writePolicy, Key, args);
+ }
+
+ protected internal override bool ParseResult()
+ {
+ ParseHeader();
+ ParseFields(policy.Txn, Key, true);
+
+ if (resultCode == ResultCode.OK)
+ {
+ record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, true);
+ return true;
+ }
+
+ if (resultCode == ResultCode.FILTERED_OUT)
+ {
+ if (policy.failOnFilteredOut)
+ {
+ throw new AerospikeException(resultCode);
+ }
+ return true;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+
+ protected internal override void OnSuccess()
+ {
+ if (listener != null)
+ {
+ listener.OnSuccess(Key, record);
+ }
+ }
+
+ protected internal override void OnFailure(AerospikeException e)
+ {
+ if (listener != null)
+ {
+ listener.OnFailure(e);
+ }
+ }
+ }
+}
diff --git a/AerospikeClient/Async/AsyncQueryPartitionExecutor.cs b/AerospikeClient/Async/AsyncQueryPartitionExecutor.cs
index 65e4d5d5..747c100c 100644
--- a/AerospikeClient/Async/AsyncQueryPartitionExecutor.cs
+++ b/AerospikeClient/Async/AsyncQueryPartitionExecutor.cs
@@ -41,7 +41,7 @@ PartitionTracker tracker
this.statement = statement;
this.tracker = tracker;
- cluster.AddTran();
+ cluster.AddCommand();
tracker.SleepBetweenRetries = 0;
taskId = statement.PrepareTaskId();
QueryPartitions();
diff --git a/AerospikeClient/Async/AsyncRead.cs b/AerospikeClient/Async/AsyncRead.cs
index 5cb5ebb8..b83b3a22 100644
--- a/AerospikeClient/Async/AsyncRead.cs
+++ b/AerospikeClient/Async/AsyncRead.cs
@@ -17,59 +17,37 @@
namespace Aerospike.Client
{
- public class AsyncRead : AsyncSingleCommand
+ public class AsyncRead : AsyncReadBase
{
private readonly RecordListener listener;
- protected internal readonly Key key;
private readonly string[] binNames;
private readonly bool isOperation;
- protected readonly Partition partition;
protected Record record;
// Read constructor.
public AsyncRead(AsyncCluster cluster, Policy policy, RecordListener listener, Key key, string[] binNames)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
this.listener = listener;
- this.key = key;
this.binNames = binNames;
this.isOperation = false;
- this.partition = Partition.Read(cluster, policy, key);
- cluster.AddTran();
- }
-
- // UDF constructor.
- public AsyncRead(AsyncCluster cluster, WritePolicy policy, Key key)
- : base(cluster, policy)
- {
- this.listener = null;
- this.key = key;
- this.binNames = null;
- this.isOperation = false;
- this.partition = Partition.Write(cluster, policy, key);
- cluster.AddTran();
}
// Operate constructor.
- public AsyncRead(AsyncCluster cluster, Policy policy, RecordListener listener, Key key, Partition partition, bool isOperation)
- : base(cluster, policy)
+ public AsyncRead(AsyncCluster cluster, Policy policy, RecordListener listener, Key key, bool isOperation)
+ : base(cluster, policy, key)
{
this.listener = listener;
- this.key = key;
this.binNames = null;
this.isOperation = isOperation;
- this.partition = partition;
- cluster.AddTran();
}
public AsyncRead(AsyncRead other)
: base(other)
{
this.listener = other.listener;
- this.key = other.key;
this.binNames = other.binNames;
this.isOperation = other.isOperation;
- this.partition = other.partition;
}
protected internal override AsyncCommand CloneCommand()
@@ -77,47 +55,25 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncRead(this);
}
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
- }
-
protected internal override void WriteBuffer()
{
SetRead(policy, key, binNames);
}
- protected internal sealed override void ParseResult()
+ protected internal sealed override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
- int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset + 6);
- int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset + 10);
- int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset + 18);
- int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset + 20);
- dataOffset += Command.MSG_REMAINING_HEADER_SIZE;
+ ParseHeader();
+ ParseFields(policy.Txn, key, false);
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
- if (opCount == 0)
- {
- // Bin data was not returned.
- record = new Record(null, generation, expiration);
- return;
- }
- SkipKey(fieldCount);
record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, isOperation);
- return;
+ return true;
}
if (resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
- HandleNotFound(resultCode);
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -126,59 +82,12 @@ record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, ge
{
throw new AerospikeException(resultCode);
}
- return;
- }
-
- if (resultCode == ResultCode.UDF_BAD_RESPONSE)
- {
- SkipKey(fieldCount);
- record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, isOperation);
- HandleUdfError(resultCode);
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
- protected internal virtual void HandleNotFound(int resultCode)
- {
- // Do nothing in default case. Record will be null.
- }
-
- private void HandleUdfError(int resultCode)
- {
- object obj;
-
- if (!record.bins.TryGetValue("FAILURE", out obj))
- {
- throw new AerospikeException(resultCode);
- }
-
- string ret = (string)obj;
- string message;
- int code;
-
- try
- {
- string[] list = ret.Split(':');
- code = Convert.ToInt32(list[2].Trim());
- message = list[0] + ':' + list[1] + ' ' + list[3];
- }
- catch (Exception)
- {
- // Use generic exception if parse error occurs.
- throw new AerospikeException(resultCode, ret);
- }
-
- throw new AerospikeException(code, message);
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
diff --git a/AerospikeClient/Async/AsyncReadBase.cs b/AerospikeClient/Async/AsyncReadBase.cs
new file mode 100644
index 00000000..2229adba
--- /dev/null
+++ b/AerospikeClient/Async/AsyncReadBase.cs
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public abstract class AsyncReadBase : AsyncSingleCommand
+ {
+ protected internal readonly Key key;
+ protected readonly Partition partition;
+
+ public AsyncReadBase(AsyncCluster cluster, Policy policy, Key key)
+ : base(cluster, policy)
+ {
+ this.key = key;
+ this.partition = Partition.Read(cluster, policy, key);
+ cluster.AddCommand();
+ }
+
+ public AsyncReadBase(AsyncReadBase other)
+ : base(other)
+ {
+ this.key = other.key;
+ this.partition = other.partition;
+ }
+
+ protected internal override bool IsWrite()
+ {
+ return false;
+ }
+
+ protected internal override Node GetNode(Cluster cluster)
+ {
+ return partition.GetNodeRead(cluster);
+ }
+
+ protected override Latency.LatencyType GetLatencyType()
+ {
+ return Latency.LatencyType.READ;
+ }
+
+ protected internal override bool PrepareRetry(bool timeout)
+ {
+ partition.PrepareRetryRead(timeout);
+ return true;
+ }
+
+ protected internal abstract override void WriteBuffer();
+
+ protected internal abstract override bool ParseResult();
+ }
+}
diff --git a/AerospikeClient/Async/AsyncReadHeader.cs b/AerospikeClient/Async/AsyncReadHeader.cs
index f65f4e35..cc75781a 100644
--- a/AerospikeClient/Async/AsyncReadHeader.cs
+++ b/AerospikeClient/Async/AsyncReadHeader.cs
@@ -17,28 +17,22 @@
namespace Aerospike.Client
{
- public sealed class AsyncReadHeader : AsyncSingleCommand
+ public sealed class AsyncReadHeader : AsyncReadBase
{
private readonly RecordListener listener;
- private readonly Key key;
- private readonly Partition partition;
private Record record;
public AsyncReadHeader(AsyncCluster cluster, Policy policy, RecordListener listener, Key key)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
this.listener = listener;
- this.key = key;
- this.partition = Partition.Read(cluster, policy, key);
- cluster.AddTran();
+ cluster.AddCommand();
}
public AsyncReadHeader(AsyncReadHeader other)
: base(other)
{
this.listener = other.listener;
- this.key = other.key;
- this.partition = other.partition;
}
protected internal override AsyncCommand CloneCommand()
@@ -46,37 +40,25 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncReadHeader(this);
}
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
- }
-
protected internal override void WriteBuffer()
{
SetReadHeader(policy, key);
}
- protected internal override void ParseResult()
+ protected internal override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
+ ParseHeader();
+ ParseFields(policy.Txn, key, false);
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
- int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset + 6);
- int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset + 10);
-
record = new Record(null, generation, expiration);
- return;
+ return true;
}
if (resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -85,18 +67,12 @@ protected internal override void ParseResult()
{
throw new AerospikeException(resultCode);
}
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
diff --git a/AerospikeClient/Async/AsyncScanPartitionExecutor.cs b/AerospikeClient/Async/AsyncScanPartitionExecutor.cs
index fd43e5d9..120fd7e2 100644
--- a/AerospikeClient/Async/AsyncScanPartitionExecutor.cs
+++ b/AerospikeClient/Async/AsyncScanPartitionExecutor.cs
@@ -46,7 +46,7 @@ PartitionTracker tracker
this.binNames = binNames;
this.tracker = tracker;
- cluster.AddTran();
+ cluster.AddCommand();
tracker.SleepBetweenRetries = 0;
ScanPartitions();
}
diff --git a/AerospikeClient/Async/AsyncSingleCommand.cs b/AerospikeClient/Async/AsyncSingleCommand.cs
index 6ce6abf2..bad45ec1 100644
--- a/AerospikeClient/Async/AsyncSingleCommand.cs
+++ b/AerospikeClient/Async/AsyncSingleCommand.cs
@@ -19,6 +19,12 @@ namespace Aerospike.Client
{
public abstract class AsyncSingleCommand : AsyncCommand
{
+ protected int resultCode;
+ protected int generation;
+ protected int expiration;
+ protected int fieldCount;
+ protected int opCount;
+
public AsyncSingleCommand(AsyncCluster cluster, Policy policy)
: base(cluster, policy)
{
@@ -35,6 +41,88 @@ protected internal sealed override void ParseCommand()
Finish();
}
- protected internal abstract void ParseResult();
+ protected void ParseHeader()
+ {
+ resultCode = dataBuffer[dataOffset + 5];
+ generation = ByteUtil.BytesToInt(dataBuffer, dataOffset + 6);
+ expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset + 10);
+ fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset + 18);
+ opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset + 20);
+ dataOffset += Command.MSG_REMAINING_HEADER_SIZE;
+ }
+
+ protected void ParseFields(Txn tran, Key key, bool hasWrite)
+ {
+ if (tran == null)
+ {
+ SkipFields(fieldCount);
+ return;
+ }
+
+ long? version = null;
+
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int type = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (type == FieldType.RECORD_VERSION)
+ {
+ if (size == 7)
+ {
+ version = ByteUtil.VersionBytesToLong(dataBuffer, dataOffset);
+ }
+ else
+ {
+ throw new AerospikeException("Record version field has invalid size: " + size);
+ }
+ }
+ dataOffset += size;
+ }
+
+ if (hasWrite)
+ {
+ tran.OnWrite(key, version, resultCode);
+ }
+ else
+ {
+ tran.OnRead(key, version);
+ }
+ }
+
+ protected void SkipFields(int fieldCount)
+ {
+ // There can be fields in the response (setname etc).
+ // But for now, ignore them. Expose them to the API if needed in the future.
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int fieldlen = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4 + fieldlen;
+ }
+ }
+
+ protected void ParseTranDeadline(Txn txn)
+ {
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int type = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (type == FieldType.MRT_DEADLINE)
+ {
+ int deadline = ByteUtil.LittleBytesToInt(dataBuffer, dataOffset);
+ txn.Deadline = deadline;
+ }
+ dataOffset += size;
+ }
+ }
+
+ protected internal abstract bool ParseResult();
}
}
diff --git a/AerospikeClient/Async/AsyncTouch.cs b/AerospikeClient/Async/AsyncTouch.cs
index d38f5251..bd998ff3 100644
--- a/AerospikeClient/Async/AsyncTouch.cs
+++ b/AerospikeClient/Async/AsyncTouch.cs
@@ -17,30 +17,20 @@
namespace Aerospike.Client
{
- public sealed class AsyncTouch : AsyncSingleCommand
+ public sealed class AsyncTouch : AsyncWriteBase
{
- private readonly WritePolicy writePolicy;
private readonly WriteListener listener;
- private readonly Key key;
- private readonly Partition partition;
public AsyncTouch(AsyncCluster cluster, WritePolicy writePolicy, WriteListener listener, Key key)
- : base(cluster, writePolicy)
+ : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
this.listener = listener;
- this.key = key;
- this.partition = Partition.Write(cluster, policy, key);
- cluster.AddTran();
}
public AsyncTouch(AsyncTouch other)
: base(other)
{
- this.writePolicy = other.writePolicy;
this.listener = other.listener;
- this.key = other.key;
- this.partition = other.partition;
}
protected internal override AsyncCommand CloneCommand()
@@ -48,33 +38,18 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncTouch(this);
}
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
-
protected internal override void WriteBuffer()
{
- SetTouch(writePolicy, key);
+ SetTouch(writePolicy, Key);
}
- protected internal override void ParseResult()
+ protected internal override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
+ ParseHeader();
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -83,23 +58,17 @@ protected internal override void ParseResult()
{
throw new AerospikeException(resultCode);
}
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
{
- listener.OnSuccess(key);
+ listener.OnSuccess(Key);
}
}
diff --git a/AerospikeClient/Async/AsyncOperate.cs b/AerospikeClient/Async/AsyncTxnAddKeys.cs
similarity index 51%
rename from AerospikeClient/Async/AsyncOperate.cs
rename to AerospikeClient/Async/AsyncTxnAddKeys.cs
index 736c7ec7..66cdfc68 100644
--- a/AerospikeClient/Async/AsyncOperate.cs
+++ b/AerospikeClient/Async/AsyncTxnAddKeys.cs
@@ -17,68 +17,79 @@
namespace Aerospike.Client
{
- public sealed class AsyncOperate : AsyncRead
+ public sealed class AsyncTxnAddKeys : AsyncWriteBase
{
+ private readonly RecordListener listener;
private readonly OperateArgs args;
- public AsyncOperate(AsyncCluster cluster, RecordListener listener, Key key, OperateArgs args)
- : base(cluster, args.writePolicy, listener, key, args.GetPartition(cluster, key), true)
+ public AsyncTxnAddKeys
+ (
+ AsyncCluster cluster,
+ RecordListener listener,
+ Key key,
+ OperateArgs args
+ ) : base(cluster, args.writePolicy, key)
{
+ this.listener = listener;
this.args = args;
}
- public AsyncOperate(AsyncOperate other)
+ public AsyncTxnAddKeys(AsyncTxnAddKeys other)
: base(other)
{
+ this.listener = other.listener;
this.args = other.args;
}
protected internal override AsyncCommand CloneCommand()
{
- return new AsyncOperate(this);
+ return new AsyncTxnAddKeys(this);
}
- protected internal override bool IsWrite()
+ protected internal override void WriteBuffer()
{
- return args.hasWrite;
+ SetTxnAddKeys(args.writePolicy, Key, args);
}
- protected internal override Node GetNode(Cluster cluster)
+ protected internal override bool ParseResult()
{
- return args.hasWrite ? partition.GetNodeWrite(cluster) : partition.GetNodeRead(cluster);
+ ParseHeader();
+ ParseTranDeadline(policy.Txn);
+
+ if (resultCode == ResultCode.OK)
+ {
+ return true;
+ }
+
+ throw new AerospikeException(resultCode);
}
- protected override Latency.LatencyType GetLatencyType()
+ protected internal override bool PrepareRetry(bool timeout)
{
- return args.hasWrite ? Latency.LatencyType.WRITE : Latency.LatencyType.READ;
+ partition.PrepareRetryWrite(timeout);
+ return true;
}
- protected internal override void WriteBuffer()
+ protected internal override void OnInDoubt()
{
- SetOperate(args.writePolicy, key, args);
+ policy.Txn.SetMonitorInDoubt();
}
- protected internal override void HandleNotFound(int resultCode)
+ protected internal override void OnSuccess()
{
- // Only throw not found exception for command with write operations.
- // Read-only command operations return a null record.
- if (args.hasWrite)
+ if (listener != null)
{
- throw new AerospikeException(resultCode);
+ listener.OnSuccess(Key, null);
}
}
- protected internal override bool PrepareRetry(bool timeout)
+ protected internal override void OnFailure(AerospikeException e)
{
- if (args.hasWrite)
- {
- partition.PrepareRetryWrite(timeout);
- }
- else
+ if (listener != null)
{
- partition.PrepareRetryRead(timeout);
+ listener.OnFailure(e);
}
- return true;
}
}
}
+
diff --git a/AerospikeClient/Async/AsyncTxnClose.cs b/AerospikeClient/Async/AsyncTxnClose.cs
new file mode 100644
index 00000000..e982e9a0
--- /dev/null
+++ b/AerospikeClient/Async/AsyncTxnClose.cs
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class AsyncTxnClose : AsyncWriteBase
+ {
+ private readonly Txn txn;
+ private readonly DeleteListener listener;
+
+ public AsyncTxnClose
+ (
+ AsyncCluster cluster,
+ Txn txn,
+ DeleteListener listener,
+ WritePolicy writePolicy,
+ Key key
+ ) : base(cluster, writePolicy, key)
+ {
+ this.txn = txn;
+ this.listener = listener;
+ }
+
+ public AsyncTxnClose(AsyncTxnClose other)
+ : base(other)
+ {
+ this.txn = other.txn;
+ this.listener = other.listener;
+ }
+
+ protected internal override AsyncCommand CloneCommand()
+ {
+ return new AsyncTxnClose(this);
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTxnClose(txn, Key);
+ }
+
+ protected internal override bool ParseResult()
+ {
+ ParseHeader();
+
+ if (resultCode == ResultCode.OK || resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
+ {
+ return true;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+
+ protected internal override void OnInDoubt()
+ {
+ }
+
+ protected internal override void OnSuccess()
+ {
+ if (listener != null)
+ {
+ listener.OnSuccess(Key, true);
+ }
+ }
+
+ protected internal override void OnFailure(AerospikeException e)
+ {
+ if (listener != null)
+ {
+ listener.OnFailure(e);
+ }
+ }
+ }
+}
+
diff --git a/AerospikeClient/Async/AsyncTxnMarkRollForward.cs b/AerospikeClient/Async/AsyncTxnMarkRollForward.cs
new file mode 100644
index 00000000..6ebd03f4
--- /dev/null
+++ b/AerospikeClient/Async/AsyncTxnMarkRollForward.cs
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class AsyncTxnMarkRollForward : AsyncWriteBase
+ {
+ private readonly Txn txn;
+ private readonly WriteListener listener;
+
+ public AsyncTxnMarkRollForward
+ (
+ AsyncCluster cluster,
+ Txn txn,
+ WriteListener listener,
+ WritePolicy writePolicy,
+ Key key
+ ) : base(cluster, writePolicy, key)
+ {
+ this.txn = txn;
+ this.listener = listener;
+ }
+
+ public AsyncTxnMarkRollForward(AsyncTxnMarkRollForward other)
+ : base(other)
+ {
+ this.txn = other.txn;
+ this.listener = other.listener;
+ }
+
+ protected internal override AsyncCommand CloneCommand()
+ {
+ return new AsyncTxnMarkRollForward(this);
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTxnMarkRollForward(txn, Key);
+ }
+
+ protected internal override bool ParseResult()
+ {
+ ParseHeader();
+
+ if (resultCode == ResultCode.OK || resultCode == ResultCode.BIN_EXISTS_ERROR)
+ {
+ return true;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+
+ protected internal override void OnInDoubt()
+ {
+ }
+
+ protected internal override void OnSuccess()
+ {
+ if (listener != null)
+ {
+ listener.OnSuccess(Key);
+ }
+ }
+
+ protected internal override void OnFailure(AerospikeException e)
+ {
+ if (listener != null)
+ {
+ listener.OnFailure(e);
+ }
+ }
+ }
+}
+
diff --git a/AerospikeClient/Async/AsyncTxnMonitor.cs b/AerospikeClient/Async/AsyncTxnMonitor.cs
new file mode 100644
index 00000000..f7f20c36
--- /dev/null
+++ b/AerospikeClient/Async/AsyncTxnMonitor.cs
@@ -0,0 +1,207 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+using Aerospike.Client;
+using System.Diagnostics.Metrics;
+using System.Diagnostics;
+
+namespace Aerospike.Client
+{
+ public abstract class AsyncTxnMonitor
+ {
+ public static void Execute(AsyncCluster cluster, WritePolicy policy, AsyncWriteBase command)
+ {
+ if (policy.Txn == null)
+ {
+ // Command is not run under a MRT monitor. Run original command.
+ command.Execute();
+ return;
+ }
+
+ Txn txn = policy.Txn;
+ Key cmdKey = command.Key;
+
+ if (txn.Writes.Contains(cmdKey))
+ {
+ // MRT monitor already contains this key. Run original command.
+ command.Execute();
+ return;
+ }
+
+ // Add key to MRT monitor and then run original command.
+ Operation[] ops = TxnMonitor.GetTranOps(txn, cmdKey);
+ SingleTxnMonitor stm = new(cluster, command);
+ stm.Execute(policy, ops);
+ }
+
+ public static void ExecuteBatch(
+ BatchPolicy policy,
+ AsyncBatchExecutor executor,
+ Key[] keys
+ )
+ {
+ if (policy.Txn == null)
+ {
+ // Command is not run under a MRT monitor. Run original command.
+ executor.Execute(executor.commands);
+ return;
+ }
+
+ // Add write keys to MRT monitor and then run original command.
+ Operation[] ops = TxnMonitor.GetTranOps(policy.Txn, keys);
+ BatchTxnMonitor ate = new(executor);
+ ate.Execute(policy, ops);
+ }
+
+ public static void ExecuteBatch(
+ BatchPolicy policy,
+ AsyncBatchExecutor executor,
+ List records
+ )
+ {
+ if (policy.Txn == null)
+ {
+ // Command is not run under a MRT monitor. Run original command.
+ executor.Execute();
+ return;
+ }
+
+ // Add write keys to MRT monitor and then run original command.
+ Operation[] ops = TxnMonitor.GetTranOps(policy.Txn, records);
+
+ if (ops == null)
+ {
+ // Readonly batch does not need to add key digests. Run original command.
+ executor.Execute();
+ return;
+ }
+
+ BatchTxnMonitor ate = new(executor);
+ ate.Execute(policy, ops);
+ }
+
+ public sealed class SingleTxnMonitor : AsyncTxnMonitor
+ {
+ public SingleTxnMonitor(AsyncCluster cluster, AsyncWriteBase command)
+ : base(command, cluster)
+ {
+ }
+
+ public override void RunCommand()
+ {
+ command.Execute();
+ }
+
+ public override void OnFailure(AerospikeException ae)
+ {
+ command.OnFailure(ae);
+ }
+ }
+
+ public sealed class BatchTxnMonitor : AsyncTxnMonitor
+ {
+ private readonly AsyncBatchExecutor executor;
+ private readonly AsyncBatchCommand[] commands;
+
+ public BatchTxnMonitor(AsyncBatchExecutor executor)
+ : base(null, null)
+ {
+ this.executor = executor;
+ this.commands = executor.commands;
+ }
+
+ public override void RunCommand()
+ {
+ executor.Execute(commands);
+ }
+
+ public override void OnFailure(AerospikeException ae)
+ {
+ executor.OnFailure(ae);
+ }
+ }
+
+ readonly AsyncCommand command;
+ readonly AsyncCluster cluster;
+
+ private AsyncTxnMonitor(AsyncCommand command, AsyncCluster cluster)
+ {
+ this.command = command;
+ this.cluster = cluster;
+ }
+
+ void Execute(Policy policy, Operation[] ops)
+ {
+ Key tranKey = TxnMonitor.GetTxnMonitorKey(policy.Txn);
+ WritePolicy wp = TxnMonitor.CopyTimeoutPolicy(policy);
+
+ ExecuteRecordListener tranListener = new(this);
+
+ // Add write key(s) to MRT monitor.
+ OperateArgs args = new(wp, null, null, ops);
+ AsyncTxnAddKeys tranCommand = new(cluster, tranListener, tranKey, args);
+ tranCommand.Execute();
+ }
+
+ private void NotifyFailure(AerospikeException ae)
+ {
+ try
+ {
+ OnFailure(ae);
+ }
+ catch (Exception t)
+ {
+ Log.Error("notifyCommandFailure onFailure() failed: " + t.StackTrace);
+ }
+ }
+
+ public abstract void OnFailure(AerospikeException ae);
+ public abstract void RunCommand();
+
+ private sealed class ExecuteRecordListener : RecordListener
+ {
+ private readonly AsyncTxnMonitor monitor;
+
+ public ExecuteRecordListener(AsyncTxnMonitor monitor)
+ {
+ this.monitor = monitor;
+ }
+
+ public void OnSuccess(Key key, Record record)
+ {
+ try
+ {
+ // Run original command.
+ monitor.RunCommand();
+ }
+ catch (AerospikeException ae)
+ {
+ monitor.NotifyFailure(ae);
+ }
+ catch (Exception t)
+ {
+ monitor.NotifyFailure(new AerospikeException(t));
+ }
+ }
+
+ public void OnFailure(AerospikeException ae)
+ {
+ monitor.NotifyFailure(new AerospikeException(ResultCode.TRAN_FAILED, "Failed to add key(s) to MRT monitor", ae));
+ }
+ }
+ }
+}
+
diff --git a/AerospikeClient/Async/AsyncTxnRoll.cs b/AerospikeClient/Async/AsyncTxnRoll.cs
new file mode 100644
index 00000000..10bdad1a
--- /dev/null
+++ b/AerospikeClient/Async/AsyncTxnRoll.cs
@@ -0,0 +1,430 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+using static Aerospike.Client.AbortStatus;
+using static Aerospike.Client.CommitError;
+using static Aerospike.Client.CommitStatus;
+
+namespace Aerospike.Client
+{
+ public sealed class AsyncTxnRoll
+ {
+ private readonly AsyncCluster cluster;
+ private readonly BatchPolicy verifyPolicy;
+ private readonly BatchPolicy rollPolicy;
+ private readonly WritePolicy writePolicy;
+ private readonly Txn txn;
+ private readonly Key tranKey;
+ private CommitListener commitListener;
+ private AbortListener abortListener;
+ private BatchRecord[] verifyRecords;
+ private BatchRecord[] rollRecords;
+ private AerospikeException verifyException;
+
+ public AsyncTxnRoll
+ (
+ AsyncCluster cluster,
+ BatchPolicy verifyPolicy,
+ BatchPolicy rollPolicy,
+ Txn txn
+ )
+ {
+ this.cluster = cluster;
+ this.verifyPolicy = verifyPolicy;
+ this.rollPolicy = rollPolicy;
+ this.writePolicy = new WritePolicy(rollPolicy);
+ this.txn = txn;
+ this.tranKey = TxnMonitor.GetTxnMonitorKey(txn);
+ }
+
+ public void Commit(CommitListener listener)
+ {
+ commitListener = listener;
+ Verify(new VerifyListener(this));
+ }
+
+ public void Abort(AbortListener listener)
+ {
+ abortListener = listener;
+
+ Roll(new RollListener(this), Command.INFO4_MRT_ROLL_BACK);
+ }
+
+ private void Verify(BatchRecordArrayListener verifyListener)
+ {
+ // Validate record versions in a batch.
+ HashSet> reads = txn.Reads.ToHashSet>();
+ int max = reads.Count;
+ if (max == 0)
+ {
+ return;
+ }
+
+ BatchRecord[] records = new BatchRecord[max];
+ Key[] keys = new Key[max];
+ long[] versions = new long[max];
+ int count = 0;
+
+ foreach (KeyValuePair entry in reads)
+ {
+ Key key = entry.Key;
+ keys[count] = key;
+ records[count] = new BatchRecord(key, false);
+ versions[count] = entry.Value;
+ count++;
+ }
+ this.verifyRecords = records;
+
+ new AsyncBatchTxnVerifyExecutor(cluster, verifyPolicy, verifyListener, keys, versions, records);
+ }
+
+ private void MarkRollForward()
+ {
+ // Tell MRT monitor that a roll-forward will commence.
+ try
+ {
+ MarkRollForwardListener writeListener = new(this);
+ AsyncTxnMarkRollForward command = new(cluster, txn, writeListener, writePolicy, tranKey);
+ command.Execute();
+ }
+ catch (Exception t)
+ {
+ NotifyCommitFailure(CommitErrorType.MARK_ROLL_FORWARD_ABANDONED, t, false);
+ }
+ }
+
+ private void RollForward()
+ {
+ try
+ {
+ RollForwardListener rollListener = new(this);
+ Roll(rollListener, Command.INFO4_MRT_ROLL_FORWARD);
+ }
+ catch (Exception t)
+ {
+ NotifyCommitSuccess(CommitStatusType.ROLL_FORWARD_ABANDONED);
+ }
+ }
+
+ private void RollBack()
+ {
+ try
+ {
+ RollForwardListener rollListener = new(this);
+ Roll(rollListener, Command.INFO4_MRT_ROLL_BACK);
+ }
+ catch (Exception t)
+ {
+ NotifyCommitFailure(CommitErrorType.VERIFY_FAIL_ABORT_ABANDONED, t, false);
+ }
+ }
+
+ private void Roll(BatchRecordArrayListener rollListener, int txnAttr)
+ {
+ HashSet keySet = txn.Writes;
+
+ if (keySet.Count == 0)
+ {
+ return;
+ }
+
+ Key[] keys = keySet.ToArray();
+ BatchRecord[] records = new BatchRecord[keys.Length];
+
+ for (int i = 0; i < keys.Length; i++)
+ {
+ records[i] = new BatchRecord(keys[i], true);
+ }
+
+ this.rollRecords = records;
+
+ // Copy txn roll policy because it needs to be modified.
+ BatchPolicy batchPolicy = new(rollPolicy);
+
+ BatchAttr attr = new();
+ attr.SetTxn(txnAttr);
+
+ new AsyncBatchTxnRollExecutor(cluster, verifyPolicy, rollListener, keys, records, attr);
+ }
+
+ private void CloseOnCommit(bool verified)
+ {
+ if (!txn.MonitorMightExist())
+ {
+ if (verified)
+ {
+ NotifyCommitSuccess(CommitStatusType.OK);
+ }
+ else
+ {
+ NotifyCommitFailure(CommitErrorType.VERIFY_FAIL, null, false);
+ }
+ }
+ }
+
+ private void CloseOnAbort()
+ {
+ if (!txn.MonitorMightExist())
+ {
+ // There is no MRT monitor record to remove.
+ NotifyAbortSuccess(AbortStatusType.OK);
+ return;
+ }
+
+ try
+ {
+ CloseOnAbortListener deleteListener = new(this);
+ AsyncTxnClose command = new(cluster, txn, deleteListener, writePolicy, tranKey);
+ command.Execute();
+ }
+ catch (Exception t)
+ {
+ NotifyAbortSuccess(AbortStatusType.CLOSE_ABANDONED);
+ }
+ }
+
+ private void NotifyCommitSuccess(CommitStatusType status)
+ {
+ txn.Clear();
+
+ try
+ {
+ commitListener.OnSuccess(status);
+ }
+ catch (Exception t)
+ {
+ Log.Error("CommitListener onSuccess() failed: " + t.StackTrace);
+ }
+ }
+
+ private void NotifyCommitFailure(CommitErrorType error, Exception cause, bool setInDoubt)
+ {
+ try
+ {
+ AerospikeException.Commit aec = (cause == null) ?
+ new AerospikeException.Commit(error, verifyRecords, rollRecords) :
+ new AerospikeException.Commit(error, verifyRecords, rollRecords, cause);
+
+ if (verifyException != null)
+ {
+ //aec.AddSuppressed(verifyException); TODO
+ }
+
+ if (cause is AerospikeException) {
+ AerospikeException src = (AerospikeException)cause;
+ aec.Node = src.Node;
+ aec.Policy = src.Policy;
+ aec.Iteration = src.Iteration;
+
+ if (setInDoubt)
+ {
+ aec.SetInDoubt(src.InDoubt);
+ }
+ }
+
+ commitListener.OnFailure(aec);
+ }
+ catch (Exception t)
+ {
+ Log.Error("CommitListener onFailure() failed: " + t.StackTrace);
+ }
+ }
+
+ private void NotifyAbortSuccess(AbortStatusType status)
+ {
+ txn.Clear();
+
+ try
+ {
+ abortListener.OnSuccess(status);
+ }
+ catch (Exception t)
+ {
+ Log.Error("AbortListener onSuccess() failed: " + t.StackTrace);
+ }
+ }
+
+ private sealed class VerifyListener : BatchRecordArrayListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public VerifyListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(BatchRecord[] records, bool status)
+ {
+ command.verifyRecords = records;
+
+ if (status)
+ {
+ if (command.txn.MonitorExists())
+ {
+ command.MarkRollForward();
+ }
+ else
+ {
+ // There is nothing to roll-forward.
+ command.CloseOnCommit(true);
+ }
+ }
+ else
+ {
+ command.RollBack();
+ }
+ }
+
+ public void OnFailure(BatchRecord[] records, AerospikeException ae)
+ {
+ command.verifyRecords = records;
+ command.verifyException = ae;
+ command.RollBack();
+ }
+ };
+
+ private sealed class RollListener : BatchRecordArrayListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public RollListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(BatchRecord[] records, bool status)
+ {
+ command.rollRecords = records;
+
+ if (status)
+ {
+ command.CloseOnAbort();
+ }
+ else
+ {
+ command.NotifyAbortSuccess(AbortStatusType.ROLL_BACK_ABANDONED);
+ }
+ }
+
+ public void OnFailure(BatchRecord[] records, AerospikeException ae)
+ {
+ command.rollRecords = records;
+ command.NotifyAbortSuccess(AbortStatusType.ROLL_BACK_ABANDONED);
+ }
+ };
+
+ private sealed class MarkRollForwardListener : WriteListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public MarkRollForwardListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(Key key)
+ {
+ command.RollForward();
+ }
+
+ public void OnFailure(AerospikeException ae)
+ {
+ command.NotifyCommitFailure(CommitErrorType.MARK_ROLL_FORWARD_ABANDONED, ae, true);
+ }
+ };
+
+ private sealed class RollForwardListener : BatchRecordArrayListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public RollForwardListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(BatchRecord[] records, bool status)
+ {
+ command.rollRecords = records;
+
+ if (status)
+ {
+ command.CloseOnCommit(true);
+ }
+ else
+ {
+ command.NotifyCommitSuccess(CommitStatusType.ROLL_FORWARD_ABANDONED);
+ }
+ }
+
+ public void OnFailure(BatchRecord[] records, AerospikeException ae)
+ {
+ command.rollRecords = records;
+ command.NotifyCommitSuccess(CommitStatusType.ROLL_FORWARD_ABANDONED);
+ }
+ };
+
+ private sealed class RollBackListener : BatchRecordArrayListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public RollBackListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(BatchRecord[] records, bool status)
+ {
+ command.rollRecords = records;
+
+ if (status)
+ {
+ command.CloseOnCommit(false);
+ }
+ else
+ {
+ command.NotifyCommitFailure(CommitErrorType.VERIFY_FAIL_ABORT_ABANDONED, null, false);
+ }
+ }
+
+ public void OnFailure(BatchRecord[] records, AerospikeException ae)
+ {
+ command.rollRecords = records;
+ command.NotifyCommitFailure(CommitErrorType.VERIFY_FAIL_ABORT_ABANDONED, ae, false);
+ }
+ };
+
+ private sealed class CloseOnAbortListener : DeleteListener
+ {
+ private readonly AsyncTxnRoll command;
+
+ public CloseOnAbortListener(AsyncTxnRoll command)
+ {
+ this.command = command;
+ }
+
+ public void OnSuccess(Key key, bool existed)
+ {
+ command.NotifyAbortSuccess(AbortStatusType.OK);
+ }
+
+ public void OnFailure(AerospikeException ae)
+ {
+ command.NotifyAbortSuccess(AbortStatusType.CLOSE_ABANDONED);
+ }
+ };
+ }
+}
+
diff --git a/AerospikeClient/Async/AsyncWrite.cs b/AerospikeClient/Async/AsyncWrite.cs
index f95ce6a1..923a5dec 100644
--- a/AerospikeClient/Async/AsyncWrite.cs
+++ b/AerospikeClient/Async/AsyncWrite.cs
@@ -17,12 +17,9 @@
namespace Aerospike.Client
{
- public sealed class AsyncWrite : AsyncSingleCommand
+ public sealed class AsyncWrite : AsyncWriteBase
{
- private readonly WritePolicy writePolicy;
private readonly WriteListener listener;
- private readonly Key key;
- private readonly Partition partition;
private readonly Bin[] bins;
private readonly Operation.Type operation;
@@ -34,24 +31,18 @@ public AsyncWrite
Key key,
Bin[] bins,
Operation.Type operation
- ) : base(cluster, writePolicy)
+ ) : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
this.listener = listener;
- this.key = key;
- this.partition = Partition.Write(cluster, policy, key);
this.bins = bins;
this.operation = operation;
- cluster.AddTran();
+ cluster.AddCommand();
}
public AsyncWrite(AsyncWrite other)
: base(other)
{
- this.writePolicy = other.writePolicy;
this.listener = other.listener;
- this.key = other.key;
- this.partition = other.partition;
this.bins = other.bins;
this.operation = other.operation;
}
@@ -61,33 +52,18 @@ protected internal override AsyncCommand CloneCommand()
return new AsyncWrite(this);
}
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode(Cluster cluster)
- {
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
-
protected internal override void WriteBuffer()
{
- SetWrite(writePolicy, operation, key, bins);
+ SetWrite(writePolicy, operation, Key, bins);
}
- protected internal override void ParseResult()
+ protected internal override bool ParseResult()
{
- int resultCode = dataBuffer[dataOffset + 5];
+ ParseHeader();
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
- return;
+ return true;
}
if (resultCode == ResultCode.FILTERED_OUT)
@@ -96,23 +72,17 @@ protected internal override void ParseResult()
{
throw new AerospikeException(resultCode);
}
- return;
+ return true;
}
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
-
protected internal override void OnSuccess()
{
if (listener != null)
{
- listener.OnSuccess(key);
+ listener.OnSuccess(Key);
}
}
@@ -125,3 +95,4 @@ protected internal override void OnFailure(AerospikeException e)
}
}
}
+
diff --git a/AerospikeClient/Async/AsyncWriteBase.cs b/AerospikeClient/Async/AsyncWriteBase.cs
new file mode 100644
index 00000000..638951f1
--- /dev/null
+++ b/AerospikeClient/Async/AsyncWriteBase.cs
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public abstract class AsyncWriteBase : AsyncSingleCommand
+ {
+ protected readonly WritePolicy writePolicy;
+ public Key Key { get; private set; }
+ protected readonly Partition partition;
+
+ public AsyncWriteBase
+ (
+ AsyncCluster cluster,
+ WritePolicy writePolicy,
+ Key key
+ ) : base(cluster, writePolicy)
+ {
+ this.writePolicy = writePolicy;
+ this.Key = key;
+ this.partition = Partition.Write(cluster, policy, key);
+ cluster.AddCommand();
+ }
+
+ public AsyncWriteBase(AsyncWriteBase other)
+ : base(other)
+ {
+ this.writePolicy = other.writePolicy;
+ this.Key = other.Key;
+ this.partition = other.partition;
+ }
+
+ protected internal override bool IsWrite()
+ {
+ return true;
+ }
+
+ protected internal override Node GetNode(Cluster cluster)
+ {
+ return partition.GetNodeWrite(cluster);
+ }
+
+ protected override Latency.LatencyType GetLatencyType()
+ {
+ return Latency.LatencyType.WRITE;
+ }
+
+ protected internal override bool PrepareRetry(bool timeout)
+ {
+ partition.PrepareRetryWrite(timeout);
+ return true;
+ }
+
+ protected internal override void OnInDoubt()
+ {
+ if (writePolicy.Txn != null)
+ {
+ writePolicy.Txn.OnWriteInDoubt(Key);
+ }
+ }
+
+ protected internal abstract override void WriteBuffer();
+
+ protected internal abstract override bool ParseResult();
+ }
+}
\ No newline at end of file
diff --git a/AerospikeClient/Async/IAsyncClient.cs b/AerospikeClient/Async/IAsyncClient.cs
index 70d130aa..57172fcd 100644
--- a/AerospikeClient/Async/IAsyncClient.cs
+++ b/AerospikeClient/Async/IAsyncClient.cs
@@ -45,29 +45,29 @@ public interface IAsyncClient : IAerospikeClient
/// Asynchronously attempt to commit the given multi-record transaction. First, the expected
/// record versions are sent to the server nodes for verification.If all nodes return success,
/// the transaction is committed.Otherwise, the transaction is aborted.
- ///
+ ///
/// This method registers the command with an event loop and returns.
/// The event loop thread will process the command and send the results to the listener.
- ///
+ ///
/// Requires server version 8.0+
- ///
+ ///
///
/// where to send results
/// multi-record transaction
- void Commit(CommitListener listener, Tran tran);
+ void Commit(CommitListener listener, Txn txn);
///
- /// Asynchronously abort and rollback the given multi-record transaction.
- ///
- /// This method registers the command with an event loop and returns.
- /// The event loop thread will process the command and send the results to the listener.
- ///
- /// Requires server version 8.0+
- ///
+ /// Asynchronously abort and rollback the given multi-record transaction.
+ ///
+ /// This method registers the command with an event loop and returns.
+ /// The event loop thread will process the command and send the results to the listener.
+ ///
+ /// Requires server version 8.0+
+ ///
///
///
///
- void Abort(AbortListener listener, Tran tran);
+ void Abort(AbortListener listener, Txn tran);
//-------------------------------------------------------
// Write Record Operations
diff --git a/AerospikeClient/Cluster/Cluster.cs b/AerospikeClient/Cluster/Cluster.cs
index 32107bea..9f8b5342 100644
--- a/AerospikeClient/Cluster/Cluster.cs
+++ b/AerospikeClient/Cluster/Cluster.cs
@@ -14,10 +14,7 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
-using System;
-using System.Collections.Generic;
using System.Text;
-using System.Threading;
namespace Aerospike.Client
{
@@ -97,7 +94,7 @@ public class Cluster
// Login timeout.
protected internal readonly int loginTimeout;
- // Maximum socket idle to validate connections in transactions.
+ // Maximum socket idle to validate connections in commands.
private readonly double maxSocketIdleMillisTran;
// Maximum socket idle to trim peak connections to min connections.
@@ -137,7 +134,7 @@ public class Cluster
public MetricsPolicy MetricsPolicy;
private volatile IMetricsListener metricsListener;
private volatile int retryCount;
- private volatile int tranCount;
+ private volatile int commandCount;
private volatile int delayQueueTimeoutCount;
public Cluster(ClientPolicy policy, Host[] hosts)
@@ -1201,26 +1198,26 @@ private static bool SupportsPartitionQuery(Node[] nodes)
}
///
- /// Increment transaction count when metrics are enabled.
+ /// Increment command count when metrics are enabled.
///
- public void AddTran()
+ public void AddCommand()
{
if (MetricsEnabled)
{
- Interlocked.Increment(ref tranCount);
+ Interlocked.Increment(ref commandCount);
}
}
///
- /// Return transaction count. The value is cumulative and not reset per metrics interval.
+ /// Return command count. The value is cumulative and not reset per metrics interval.
///
- public int GetTranCount()
+ public int GetCommandCount()
{
- return tranCount;
+ return commandCount;
}
///
- /// Increment transaction retry count. There can be multiple retries for a single transaction.
+ /// Increment command retry count. There can be multiple retries for a single command.
///
public void AddRetry()
{
@@ -1228,7 +1225,7 @@ public void AddRetry()
}
///
- /// Add transaction retry count. There can be multiple retries for a single transaction.
+ /// Add command retry count. There can be multiple retries for a single command.
///
public void AddRetries(int count)
{
@@ -1236,7 +1233,7 @@ public void AddRetries(int count)
}
///
- /// Return transaction retry count. The value is cumulative and not reset per metrics interval.
+ /// Return command retry count. The value is cumulative and not reset per metrics interval.
///
public int GetRetryCount()
{
diff --git a/AerospikeClient/Cluster/ClusterStats.cs b/AerospikeClient/Cluster/ClusterStats.cs
index 0196fdf3..3aa19513 100644
--- a/AerospikeClient/Cluster/ClusterStats.cs
+++ b/AerospikeClient/Cluster/ClusterStats.cs
@@ -51,7 +51,7 @@ public sealed class ClusterStats
public readonly int invalidNodeCount;
///
- /// Count of transaction retires since cluster was started.
+ /// Count of command retires since cluster was started.
///
public readonly long RetryCount;
@@ -126,14 +126,14 @@ public sealed class NodeStats
public readonly ConnectionStats asyncStats;
///
- /// Transaction error count since node was initialized. If the error is retryable, multiple errors per
- /// transaction may occur.
+ /// Command error count since node was initialized. If the error is retryable, multiple errors per
+ /// Command may occur.
///
public readonly long ErrorCount;
///
- /// Transaction timeout count since node was initialized. If the timeout is retryable (ie socketTimeout),
- /// multiple timeouts per transaction may occur.
+ /// Command timeout count since node was initialized. If the timeout is retryable (ie socketTimeout),
+ /// multiple timeouts per Command may occur.
///
public readonly long TimeoutCount;
diff --git a/AerospikeClient/Cluster/Node.cs b/AerospikeClient/Cluster/Node.cs
index eaeb440d..f28017de 100644
--- a/AerospikeClient/Cluster/Node.cs
+++ b/AerospikeClient/Cluster/Node.cs
@@ -976,8 +976,8 @@ public void ValidateErrorCount()
}
///
- /// Increment transaction error count. If the error is retryable, multiple errors per
- /// transaction may occur.
+ /// Increment command error count. If the error is retryable, multiple errors per
+ /// command may occur.
///
public void AddError()
@@ -986,8 +986,8 @@ public void AddError()
}
///
- /// Increment transaction timeout count. If the timeout is retryable (ie socketTimeout),
- /// multiple timeouts per transaction may occur.
+ /// Increment command timeout count. If the timeout is retryable (ie socketTimeout),
+ /// multiple timeouts per command may occur.
///
public void AddTimeout()
{
@@ -995,7 +995,7 @@ public void AddTimeout()
}
///
- /// Return transaction error count. The value is cumulative and not reset per metrics interval.
+ /// Return command error count. The value is cumulative and not reset per metrics interval.
///
public int GetErrorCount()
{
@@ -1003,7 +1003,7 @@ public int GetErrorCount()
}
///
- /// Return transaction timeout count. The value is cumulative and not reset per metrics interval.
+ /// Return command timeout count. The value is cumulative and not reset per metrics interval.
///
public int GetTimeoutCount()
{
diff --git a/AerospikeClient/Command/Batch.cs b/AerospikeClient/Command/Batch.cs
index 79b4b39c..aec9cdf1 100644
--- a/AerospikeClient/Command/Batch.cs
+++ b/AerospikeClient/Command/Batch.cs
@@ -186,12 +186,6 @@ protected internal override void WriteBuffer()
protected internal override bool ParseRow()
{
ParseFieldsRead(keys[batchIndex]);
-
- if (opCount > 0)
- {
- throw new AerospikeException.Parse("Received bins that were not requested!");
- }
-
existsArray[batchIndex] = resultCode == 0;
return true;
}
@@ -286,6 +280,10 @@ protected internal override void SetInDoubt(bool inDoubt)
if (record.resultCode == ResultCode.NO_RESPONSE)
{
record.inDoubt = record.hasWrite;
+
+ if (record.inDoubt && policy.Txn != null) {
+ policy.Txn.OnWriteInDoubt(record.key);
+ }
}
}
}
@@ -372,6 +370,10 @@ protected internal override void SetInDoubt(bool inDoubt)
if (record.resultCode == ResultCode.NO_RESPONSE)
{
record.inDoubt = inDoubt;
+
+ if (record.inDoubt && policy.Txn != null) {
+ policy.Txn.OnWriteInDoubt(record.key);
+ }
}
}
}
@@ -479,6 +481,10 @@ protected internal override void SetInDoubt(bool inDoubt)
if (record.resultCode == ResultCode.NO_RESPONSE)
{
record.inDoubt = inDoubt;
+
+ if (record.inDoubt && policy.Txn != null) {
+ policy.Txn.OnWriteInDoubt(record.key);
+ }
}
}
}
@@ -498,25 +504,25 @@ protected internal override List GenerateBatchNodes()
// MRT
//-------------------------------------------------------
- public sealed class BatchTranVerify : BatchCommand
+ public sealed class BatchTxnVerify : BatchCommand
{
- private readonly Tran tran;
+ private readonly Txn txn;
private readonly Key[] keys;
private readonly long[] versions;
private readonly BatchRecord[] records;
- public BatchTranVerify(
+ public BatchTxnVerify(
Cluster cluster,
BatchNode batch,
BatchPolicy batchPolicy,
- Tran tran,
+ Txn tran,
Key[] keys,
long[] versions,
BatchRecord[] records,
BatchStatus status
) : base(cluster, batch, batchPolicy, status, false)
{
- this.tran = tran;
+ this.txn = tran;
this.keys = keys;
this.versions = versions;
this.records = records;
@@ -529,7 +535,7 @@ protected internal override bool IsWrite()
protected internal override void WriteBuffer()
{
- SetBatchTranVerify(batchPolicy, tran, keys, versions, batch);
+ SetBatchTxnVerify(batchPolicy, txn, keys, versions, batch);
}
protected internal override bool ParseRow()
@@ -552,7 +558,7 @@ protected internal override bool ParseRow()
protected internal override BatchCommand CreateCommand(BatchNode batchNode)
{
- return new BatchTranVerify(cluster, batchNode, batchPolicy, tran, keys, versions, records, status);
+ return new BatchTxnVerify(cluster, batchNode, batchPolicy, txn, keys, versions, records, status);
}
protected internal override List GenerateBatchNodes()
@@ -561,13 +567,13 @@ protected internal override List GenerateBatchNodes()
}
}
- public sealed class BatchTranRoll : BatchCommand
+ public sealed class BatchTxnRoll : BatchCommand
{
private readonly Key[] keys;
private readonly BatchRecord[] records;
private readonly BatchAttr attr;
- public BatchTranRoll(
+ public BatchTxnRoll(
Cluster cluster,
BatchNode batch,
BatchPolicy batchPolicy,
@@ -589,7 +595,7 @@ protected internal override bool IsWrite()
protected internal override void WriteBuffer()
{
- SetBatchTranRoll(batchPolicy, keys, batch, attr);
+ SetBatchTxnRoll(batchPolicy, keys, batch, attr);
}
protected internal override bool ParseRow()
@@ -630,7 +636,7 @@ protected internal override void SetInDoubt(bool inDoubt)
protected internal override BatchCommand CreateCommand(BatchNode batchNode)
{
- return new BatchTranRoll(cluster, batchNode, batchPolicy, keys, records, attr, status);
+ return new BatchTxnRoll(cluster, batchNode, batchPolicy, keys, records, attr, status);
}
protected internal override List GenerateBatchNodes()
@@ -701,10 +707,10 @@ public void Run(object obj)
protected void ParseFieldsRead(Key key)
{
- if (policy.Tran != null)
+ if (policy.Txn != null)
{
long? version = ParseVersion(fieldCount);
- policy.Tran.OnRead(key, version);
+ policy.Txn.OnRead(key, version);
}
else
{
@@ -714,17 +720,17 @@ protected void ParseFieldsRead(Key key)
protected void ParseFields(BatchRecord br)
{
- if (policy.Tran != null)
+ if (policy.Txn != null)
{
long? version = ParseVersion(fieldCount);
if (br.hasWrite)
{
- policy.Tran.OnWrite(br.key, version, resultCode);
+ policy.Txn.OnWrite(br.key, version, resultCode);
}
else
{
- policy.Tran.OnRead(br.key, version);
+ policy.Txn.OnRead(br.key, version);
}
}
else
diff --git a/AerospikeClient/Command/BatchAttr.cs b/AerospikeClient/Command/BatchAttr.cs
index 42839e40..723a6880 100644
--- a/AerospikeClient/Command/BatchAttr.cs
+++ b/AerospikeClient/Command/BatchAttr.cs
@@ -409,7 +409,7 @@ public void SetOpSize(Operation[] ops)
opSize = dataOffset;
}
- public void SetTran(int attr)
+ public void SetTxn(int attr)
{
filterExp = null;
readAttr = 0;
diff --git a/AerospikeClient/Command/BatchExecutor.cs b/AerospikeClient/Command/BatchExecutor.cs
index 7fc82ac4..6da7f1ca 100644
--- a/AerospikeClient/Command/BatchExecutor.cs
+++ b/AerospikeClient/Command/BatchExecutor.cs
@@ -23,7 +23,7 @@ public sealed class BatchExecutor
{
public static void Execute(Cluster cluster, BatchPolicy policy, BatchCommand[] commands, BatchStatus status)
{
- cluster.AddTran();
+ cluster.AddCommand();
if (policy.maxConcurrentThreads == 1 || commands.Length <= 1)
{
diff --git a/AerospikeClient/Command/Command.cs b/AerospikeClient/Command/Command.cs
index fcba2a1e..9940f7d5 100644
--- a/AerospikeClient/Command/Command.cs
+++ b/AerospikeClient/Command/Command.cs
@@ -117,7 +117,7 @@ public Command(int socketTimeout, int totalTimeout, int maxRetries)
// Multi-record Transactions
//--------------------------------------------------
- public void SetTranAddKeys(WritePolicy policy, Key key, OperateArgs args)
+ public void SetTxnAddKeys(WritePolicy policy, Key key, OperateArgs args)
{
Begin();
int fieldCount = EstimateKeySize(key);
@@ -131,7 +131,7 @@ public void SetTranAddKeys(WritePolicy policy, Key key, OperateArgs args)
End(policy.compress);
}
- public void SetTranVerify(Tran tran, Key key, long ver)
+ public void SetTranVerify(Txn tran, Key key, long ver)
{
Begin();
int fieldCount = EstimateKeySize(key);
@@ -160,9 +160,9 @@ public void SetTranVerify(Tran tran, Key key, long ver)
End();
}
- public void SetBatchTranVerify(
+ public void SetBatchTxnVerify(
BatchPolicy policy,
- Tran tran,
+ Txn tran,
Key[] keys,
long[] versions,
BatchNode batch
@@ -174,7 +174,7 @@ BatchNode batch
public void SetBatchTranVerify(
BatchPolicy policy,
- Tran tran,
+ Txn tran,
Key[] keys,
long[] versions,
BatchOffsets offsets
@@ -279,7 +279,7 @@ BatchOffsets offsets
End(compress);
}
- public void SetTranMarkRollForward(Tran tran, Key key)
+ public void SetTxnMarkRollForward(Txn tran, Key key)
{
Bin bin = new("fwd", true);
@@ -291,7 +291,7 @@ public void SetTranMarkRollForward(Tran tran, Key key)
End();
}
- public void SetTranRoll(Key key, Tran tran, int tranAttr)
+ public void SetTranRoll(Key key, Txn tran, int tranAttr)
{
Begin();
int fieldCount = EstimateKeySize(key);
@@ -317,7 +317,7 @@ public void SetTranRoll(Key key, Tran tran, int tranAttr)
End();
}
- public void SetBatchTranRoll(
+ public void SetBatchTxnRoll(
BatchPolicy policy,
Key[] keys,
BatchNode batch,
@@ -339,8 +339,8 @@ BatchOffsets offsets
Begin();
int fieldCount = 1;
int max = offsets.Size();
- Tran tran = policy.Tran;
- long[] versions = new long[max];
+ Txn tran = policy.Txn;
+ long?[] versions = new long?[max];
for (int i = 0; i < max; i++)
{
@@ -358,7 +358,7 @@ BatchOffsets offsets
{
int offset = offsets.Get(i);
Key key = keys[offset];
- long ver = versions[i];
+ long? ver = versions[i];
dataOffset += key.digest.Length + 4;
@@ -395,7 +395,7 @@ BatchOffsets offsets
{
int offset = offsets.Get(i);
Key key = keys[offset];
- long ver = versions[i];
+ long? ver = versions[i];
ByteUtil.IntToBytes((uint)offset, dataBuffer, dataOffset);
dataOffset += 4;
@@ -422,7 +422,7 @@ BatchOffsets offsets
End(compress);
}
- public void SetTranClose(Tran tran, Key key)
+ public void SetTxnClose(Txn tran, Key key)
{
Begin();
int fieldCount = EstimateKeySize(key);
@@ -1121,7 +1121,7 @@ public void SetBatchOperate(
{
Begin();
int max = offsets.Size();
- Tran tran = policy.Tran;
+ Txn tran = policy.Txn;
long?[] versions = null;
if (tran != null)
@@ -1280,7 +1280,7 @@ public void SetBatchOperate(
{
attr.SetUDF(policy);
}
- WriteBatchWrite(key, policy.Tran, null, attr, attr.filterExp, 3, 0);
+ WriteBatchWrite(key, policy.Txn, null, attr, attr.filterExp, 3, 0);
WriteField(bu.packageName, FieldType.UDF_PACKAGE_NAME);
WriteField(bu.functionName, FieldType.UDF_FUNCTION);
WriteField(bu.argBytes, FieldType.UDF_ARGLIST);
@@ -1337,7 +1337,7 @@ BatchOffsets offsets
{
// Estimate full row size
int max = offsets.Size();
- Tran tran = policy.Tran;
+ Txn tran = policy.Txn;
long?[] versions = null;
Begin();
@@ -1511,7 +1511,7 @@ BatchOffsets offsets
// Estimate buffer size.
Begin();
int max = offsets.Size();
- Tran tran = policy.Tran;
+ Txn tran = policy.Txn;
long?[] versions = null;
if (tran != null)
@@ -1672,7 +1672,7 @@ private static byte GetBatchFlags(BatchPolicy policy)
return flags;
}
- private void SizeTranBatch(Tran tran, long? ver)
+ private void SizeTranBatch(Txn tran, long? ver)
{
if (tran != null)
{
@@ -1711,7 +1711,7 @@ private void WriteBatchHeader(Policy policy, int timeout, int fieldCount)
dataOffset += ByteUtil.ShortToBytes(0, dataBuffer, dataOffset);
}
- private void WriteBatchBinNames(Key key, Tran tran, long? ver, string[] binNames, BatchAttr attr, Expression filter)
+ private void WriteBatchBinNames(Key key, Txn tran, long? ver, string[] binNames, BatchAttr attr, Expression filter)
{
WriteBatchRead(key, tran, ver, attr, filter, binNames.Length);
@@ -1721,7 +1721,7 @@ private void WriteBatchBinNames(Key key, Tran tran, long? ver, string[] binNames
}
}
- private void WriteBatchOperations(Key key, Tran tran, long? ver, Operation[] ops, BatchAttr attr, Expression filter)
+ private void WriteBatchOperations(Key key, Txn tran, long? ver, Operation[] ops, BatchAttr attr, Expression filter)
{
if (attr.hasWrite)
{
@@ -1738,7 +1738,7 @@ private void WriteBatchOperations(Key key, Tran tran, long? ver, Operation[] ops
}
}
- private void WriteBatchRead(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int opCount)
+ private void WriteBatchRead(Key key, Txn tran, long? ver, BatchAttr attr, Expression filter, int opCount)
{
dataBuffer[dataOffset++] = (byte)(BATCH_MSG_INFO | BATCH_MSG_INFO4 | BATCH_MSG_TTL);
dataBuffer[dataOffset++] = (byte)attr.readAttr;
@@ -1750,7 +1750,7 @@ private void WriteBatchRead(Key key, Tran tran, long? ver, BatchAttr attr, Expre
WriteBatchFields(key, tran, ver, attr,filter, 0, opCount);
}
- private void WriteBatchWrite(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
+ private void WriteBatchWrite(Key key, Txn tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
{
dataBuffer[dataOffset++] = (byte)(BATCH_MSG_INFO | BATCH_MSG_INFO4 | BATCH_MSG_GEN | BATCH_MSG_TTL);
dataBuffer[dataOffset++] = (byte)attr.readAttr;
@@ -1764,7 +1764,7 @@ private void WriteBatchWrite(Key key, Tran tran, long? ver, BatchAttr attr, Expr
WriteBatchFields(key, tran, ver, attr, filter, fieldCount, opCount);
}
- private void WriteBatchFields(Key key, Tran tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
+ private void WriteBatchFields(Key key, Txn tran, long? ver, BatchAttr attr, Expression filter, int fieldCount, int opCount)
{
if (tran != null)
{
@@ -2356,7 +2356,7 @@ private int EstimateKeySize(Policy policy, Key key, bool sendDeadline)
{
int fieldCount = EstimateKeySize(key);
- fieldCount += SizeTran(key, policy.Tran, sendDeadline);
+ fieldCount += SizeTran(key, policy.Txn, sendDeadline);
if (policy.sendKey)
{
@@ -2737,7 +2737,7 @@ int operationCount
private void WriteKey(Policy policy, Key key, bool sendDeadline)
{
WriteKey(key);
- WriteTran(policy.Tran, sendDeadline);
+ WriteTran(policy.Txn, sendDeadline);
if (policy.sendKey)
{
@@ -2847,7 +2847,7 @@ private void WriteOperation(Operation.Type operationType)
dataBuffer[dataOffset++] = 0;
}
- private int SizeTran(Key key, Tran tran, bool sendDeadline)
+ private int SizeTran(Key key, Txn tran, bool sendDeadline)
{
int fieldCount = 0;
@@ -2873,7 +2873,7 @@ private int SizeTran(Key key, Tran tran, bool sendDeadline)
return fieldCount;
}
- private void WriteTran(Tran tran, bool sendDeadline)
+ private void WriteTran(Txn tran, bool sendDeadline)
{
if (tran != null)
{
diff --git a/AerospikeClient/Command/DeleteCommand.cs b/AerospikeClient/Command/DeleteCommand.cs
index d2b17713..2d87cf6e 100644
--- a/AerospikeClient/Command/DeleteCommand.cs
+++ b/AerospikeClient/Command/DeleteCommand.cs
@@ -15,37 +15,18 @@
* the License.
*/
+using System;
+
namespace Aerospike.Client
{
- public sealed class DeleteCommand : SyncCommand
+ public sealed class DeleteCommand : SyncWriteCommand
{
- private readonly WritePolicy writePolicy;
- private readonly Key key;
- private readonly Partition partition;
private bool existed;
public DeleteCommand(Cluster cluster, WritePolicy writePolicy, Key key)
- : base(cluster, writePolicy)
- {
- this.writePolicy = writePolicy;
- this.key = key;
- this.partition = Partition.Write(cluster, writePolicy, key);
- cluster.AddTran();
- }
-
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode()
+ : base(cluster, writePolicy, key)
{
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
+ cluster.AddCommand();
}
protected internal override void WriteBuffer()
@@ -55,11 +36,7 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, MSG_TOTAL_HEADER_SIZE, Command.STATE_READ_HEADER);
- conn.UpdateLastUsed();
-
- int resultCode = dataBuffer[13];
+ ParseHeader(conn);
if (resultCode == 0)
{
@@ -86,12 +63,6 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
-
public bool Existed()
{
return existed;
diff --git a/AerospikeClient/Command/ExecuteCommand.cs b/AerospikeClient/Command/ExecuteCommand.cs
index a92bfe3b..0c7a7bb4 100644
--- a/AerospikeClient/Command/ExecuteCommand.cs
+++ b/AerospikeClient/Command/ExecuteCommand.cs
@@ -15,14 +15,16 @@
* the License.
*/
+using Aerospike.Client;
+
namespace Aerospike.Client
{
- public sealed class ExecuteCommand : ReadCommand
+ public sealed class ExecuteCommand : SyncWriteCommand
{
- private readonly WritePolicy writePolicy;
private readonly string packageName;
private readonly string functionName;
private readonly Value[] args;
+ public Record Record { get; private set; }
public ExecuteCommand
(
@@ -32,43 +34,78 @@ public ExecuteCommand
string packageName,
string functionName,
Value[] args
- ) : base(cluster, writePolicy, key, Partition.Write(cluster, writePolicy, key), false)
+ ) : base(cluster, writePolicy, key)
{
- this.writePolicy = writePolicy;
this.packageName = packageName;
this.functionName = functionName;
this.args = args;
}
- protected internal override bool IsWrite()
+ protected internal override void WriteBuffer()
{
- return true;
+ SetUdf(writePolicy, key, packageName, functionName, args);
}
- protected internal override Node GetNode()
+ protected internal override void ParseResult(IConnection conn)
{
- return partition.GetNodeWrite(cluster);
- }
+ ParseHeader(conn);
+ ParseFields(policy.Txn, key, true);
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
- }
+ if (resultCode == ResultCode.OK)
+ {
+ Record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, false);
+ return;
+ }
- protected internal override void WriteBuffer()
- {
- SetUdf(writePolicy, key, packageName, functionName, args);
- }
+ if (resultCode == ResultCode.UDF_BAD_RESPONSE)
+ {
+ Record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, false);
+ HandleUdfError(resultCode);
+ return;
+ }
+
+ if (opCount > 0)
+ {
+ throw new AerospikeException("Unexpected UDF opCount on error: " + opCount + ',' + resultCode);
+ }
+
+ if (resultCode == ResultCode.FILTERED_OUT)
+ {
+ if (policy.failOnFilteredOut)
+ {
+ throw new AerospikeException(resultCode);
+ }
+ return;
+ }
- protected internal override void HandleNotFound(int resultCode)
- {
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
+ private void HandleUdfError(int resultCode)
{
- partition.PrepareRetryWrite(timeout);
- return true;
+ string ret = (string)Record.bins["FAILURE"];
+
+ if (ret == null)
+ {
+ throw new AerospikeException(resultCode);
+ }
+
+ string message;
+ int code;
+
+ try
+ {
+ string[] list = ret.Split(":");
+ Int32.TryParse(list[2].Trim(), out code);
+ message = list[0] + ':' + list[1] + ' ' + list[3];
+ }
+ catch (Exception e)
+ {
+ // Use generic exception if parse error occurs.
+ throw new AerospikeException(resultCode, ret);
+ }
+
+ throw new AerospikeException(code, message);
}
}
}
diff --git a/AerospikeClient/Command/ExistsCommand.cs b/AerospikeClient/Command/ExistsCommand.cs
index 146a4a1e..9269c9b2 100644
--- a/AerospikeClient/Command/ExistsCommand.cs
+++ b/AerospikeClient/Command/ExistsCommand.cs
@@ -17,28 +17,13 @@
namespace Aerospike.Client
{
- public sealed class ExistsCommand : SyncCommand
+ public sealed class ExistsCommand : SyncReadCommand
{
- private readonly Key key;
- private readonly Partition partition;
private bool exists;
public ExistsCommand(Cluster cluster, Policy policy, Key key)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
- this.key = key;
- this.partition = Partition.Read(cluster, policy, key);
- cluster.AddTran();
- }
-
- protected internal override Node GetNode()
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
}
protected internal override void WriteBuffer()
@@ -48,13 +33,10 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, MSG_TOTAL_HEADER_SIZE, Command.STATE_READ_HEADER);
- conn.UpdateLastUsed();
-
- int resultCode = dataBuffer[13];
+ ParseHeader(conn);
+ ParseFields(policy.Txn, key, false);
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
exists = true;
return;
@@ -79,12 +61,6 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
public bool Exists()
{
return exists;
diff --git a/AerospikeClient/Command/MultiCommand.cs b/AerospikeClient/Command/MultiCommand.cs
index db235eee..04473c39 100644
--- a/AerospikeClient/Command/MultiCommand.cs
+++ b/AerospikeClient/Command/MultiCommand.cs
@@ -27,12 +27,7 @@ public abstract class MultiCommand : SyncCommand
protected internal readonly String ns;
private readonly ulong clusterKey;
protected internal int info3;
- protected internal int resultCode;
- protected internal int generation;
- protected internal int expiration;
protected internal int batchIndex;
- protected internal int fieldCount;
- protected internal int opCount;
protected internal readonly bool isOperation;
private readonly bool first;
protected internal volatile bool valid = true;
diff --git a/AerospikeClient/Command/OperateCommand.cs b/AerospikeClient/Command/OperateCommand.cs
deleted file mode 100644
index 45737711..00000000
--- a/AerospikeClient/Command/OperateCommand.cs
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2012-2024 Aerospike, Inc.
- *
- * Portions may be licensed to Aerospike, Inc. under one or more contributor
- * license agreements.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not
- * use this file except in compliance with the License. You may obtain a copy of
- * the License at http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-namespace Aerospike.Client
-{
- public sealed class OperateCommand : ReadCommand
- {
- private readonly OperateArgs args;
-
- public OperateCommand(Cluster cluster, Key key, OperateArgs args)
- : base(cluster, args.writePolicy, key, args.GetPartition(cluster, key), true)
- {
- this.args = args;
- }
-
- protected internal override bool IsWrite()
- {
- return args.hasWrite;
- }
-
- protected internal override Node GetNode()
- {
- return args.hasWrite ? partition.GetNodeWrite(cluster) : partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return args.hasWrite ? Latency.LatencyType.WRITE : Latency.LatencyType.READ;
- }
-
- protected internal override void WriteBuffer()
- {
- SetOperate(args.writePolicy, key, args);
- }
-
- protected internal override void HandleNotFound(int resultCode)
- {
- // Only throw not found exception for command with write operations.
- // Read-only command operations return a null record.
- if (args.hasWrite)
- {
- throw new AerospikeException(resultCode);
- }
- }
-
- protected internal override bool PrepareRetry(bool timeout)
- {
- if (args.hasWrite)
- {
- partition.PrepareRetryWrite(timeout);
- }
- else
- {
- partition.PrepareRetryRead(timeout);
- }
- return true;
- }
- }
-}
diff --git a/AerospikeClient/Command/OperateCommandRead.cs b/AerospikeClient/Command/OperateCommandRead.cs
new file mode 100644
index 00000000..9e15efb1
--- /dev/null
+++ b/AerospikeClient/Command/OperateCommandRead.cs
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class OperateCommandRead : ReadCommand
+ {
+ private readonly OperateArgs args;
+
+ public OperateCommandRead(Cluster cluster, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, key, args.GetPartition(cluster, key), true)
+ {
+ this.args = args;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetOperate(args.writePolicy, key, args);
+ }
+ }
+}
diff --git a/AerospikeClient/Command/OperateCommandWrite.cs b/AerospikeClient/Command/OperateCommandWrite.cs
new file mode 100644
index 00000000..a7144f8d
--- /dev/null
+++ b/AerospikeClient/Command/OperateCommandWrite.cs
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+using Aerospike.Client;
+
+namespace Aerospike.Client
+{
+ public sealed class OperateCommandWrite : SyncWriteCommand
+ {
+ private readonly OperateArgs args;
+ public Record Record { get; private set; }
+
+ public OperateCommandWrite(Cluster cluster, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, key)
+ {
+ this.args = args;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetOperate(args.writePolicy, key, args);
+ }
+
+ protected internal override void ParseResult(IConnection conn)
+ {
+ ParseHeader(conn);
+ ParseFields(policy.Txn, key, true);
+
+ if (resultCode == ResultCode.OK) {
+ Record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, true);
+ return;
+ }
+
+ if (opCount > 0) {
+ throw new AerospikeException("Unexpected operate opCount on error: " + opCount + ',' + resultCode);
+ }
+
+ if (resultCode == ResultCode.FILTERED_OUT)
+ {
+ if (policy.failOnFilteredOut)
+ {
+ throw new AerospikeException(resultCode);
+ }
+ return;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+
+ }
+}
diff --git a/AerospikeClient/Command/ReadCommand.cs b/AerospikeClient/Command/ReadCommand.cs
index 79171f0d..a4b00eb8 100644
--- a/AerospikeClient/Command/ReadCommand.cs
+++ b/AerospikeClient/Command/ReadCommand.cs
@@ -17,52 +17,34 @@
namespace Aerospike.Client
{
- public class ReadCommand : SyncCommand
+ public class ReadCommand : SyncReadCommand
{
- protected readonly Key key;
- protected readonly Partition partition;
private readonly string[] binNames;
private readonly bool isOperation;
private Record record;
public ReadCommand(Cluster cluster, Policy policy, Key key)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
- this.key = key;
this.binNames = null;
- this.partition = Partition.Read(cluster, policy, key);
this.isOperation = false;
- cluster.AddTran();
+ cluster.AddCommand();
}
public ReadCommand(Cluster cluster, Policy policy, Key key, String[] binNames)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
- this.key = key;
this.binNames = binNames;
- this.partition = Partition.Read(cluster, policy, key);
this.isOperation = false;
- cluster.AddTran();
+ cluster.AddCommand();
}
public ReadCommand(Cluster cluster, Policy policy, Key key, Partition partition, bool isOperation)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
- this.key = key;
this.binNames = null;
- this.partition = partition;
this.isOperation = isOperation;
- cluster.AddTran();
- }
-
- protected internal override Node GetNode()
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
+ cluster.AddCommand();
}
protected internal override void WriteBuffer()
@@ -72,68 +54,22 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
-
- long sz = ByteUtil.BytesToLong(dataBuffer, 0);
- int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
-
- if (receiveSize <= 0)
- {
- throw new AerospikeException("Invalid receive size: " + receiveSize);
- }
-
- SizeBuffer(receiveSize);
- conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
- conn.UpdateLastUsed();
-
- ulong type = (ulong)((sz >> 48) & 0xff);
+ ParseHeader(conn);
+ ParseFields(policy.Txn, key, false);
- if (type == Command.AS_MSG_TYPE)
+ if (resultCode == ResultCode.OK)
{
- dataOffset = 5;
+ this.record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, isOperation);
+ return;
}
- else if (type == Command.MSG_TYPE_COMPRESSED)
- {
- int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
- byte[] ubuf = new byte[usize];
- ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
- dataBuffer = ubuf;
- dataOffset = 13;
- }
- else
+ if (opCount > 0)
{
- throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
- }
-
- int resultCode = dataBuffer[dataOffset];
- dataOffset++;
- int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4;
- int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 8;
- int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
- int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
-
- if (resultCode == 0)
- {
- if (opCount == 0)
- {
- // Bin data was not returned.
- record = new Record(null, generation, expiration);
- return;
- }
- SkipKey(fieldCount);
- record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, isOperation);
- return;
+ throw new AerospikeException("Unexpected read opCount on error: " + opCount + ',' + resultCode);
}
if (resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
- HandleNotFound(resultCode);
return;
}
@@ -146,56 +82,8 @@ record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, ge
return;
}
- if (resultCode == ResultCode.UDF_BAD_RESPONSE)
- {
- SkipKey(fieldCount);
- record = policy.recordParser.ParseRecord(dataBuffer, ref dataOffset, opCount, generation, expiration, isOperation);
- HandleUdfError(resultCode);
- return;
- }
-
throw new AerospikeException(resultCode);
}
-
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
- protected internal virtual void HandleNotFound(int resultCode)
- {
- // Do nothing in default case. Record will be null.
- }
-
- private void HandleUdfError(int resultCode)
- {
- object obj;
-
- if (!record.bins.TryGetValue("FAILURE", out obj))
- {
- throw new AerospikeException(resultCode);
- }
-
- string ret = (string)obj;
- string message;
- int code;
-
- try
- {
- string[] list = ret.Split(':');
- code = Convert.ToInt32(list[2].Trim());
- message = list[0] + ':' + list[1] + ' ' + list[3];
- }
- catch (Exception e)
- {
- // Use generic exception if parse error occurs.
- throw new AerospikeException(resultCode, ret, e);
- }
-
- throw new AerospikeException(code, message);
- }
-
public Record Record
{
get
diff --git a/AerospikeClient/Command/ReadHeaderCommand.cs b/AerospikeClient/Command/ReadHeaderCommand.cs
index 14e41178..cbf443f4 100644
--- a/AerospikeClient/Command/ReadHeaderCommand.cs
+++ b/AerospikeClient/Command/ReadHeaderCommand.cs
@@ -17,28 +17,13 @@
namespace Aerospike.Client
{
- public sealed class ReadHeaderCommand : SyncCommand
+ public sealed class ReadHeaderCommand : SyncReadCommand
{
- private readonly Key key;
- private readonly Partition partition;
private Record record;
public ReadHeaderCommand(Cluster cluster, Policy policy, Key key)
- : base(cluster, policy)
+ : base(cluster, policy, key)
{
- this.key = key;
- this.partition = Partition.Read(cluster, policy, key);
- cluster.AddTran();
- }
-
- protected internal override Node GetNode()
- {
- return partition.GetNodeRead(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.READ;
}
protected internal override void WriteBuffer()
@@ -48,16 +33,11 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, MSG_TOTAL_HEADER_SIZE, Command.STATE_READ_HEADER);
- conn.UpdateLastUsed();
-
- int resultCode = dataBuffer[13];
+ ParseHeader(conn);
+ ParseFields(policy.Txn, key, false);
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
- int generation = ByteUtil.BytesToInt(dataBuffer, 14);
- int expiration = ByteUtil.BytesToInt(dataBuffer, 18);
record = new Record(null, generation, expiration);
return;
}
@@ -79,12 +59,6 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryRead(timeout);
- return true;
- }
-
public Record Record
{
get
diff --git a/AerospikeClient/Command/ScanExecutor.cs b/AerospikeClient/Command/ScanExecutor.cs
index 8380d849..d9cbe187 100644
--- a/AerospikeClient/Command/ScanExecutor.cs
+++ b/AerospikeClient/Command/ScanExecutor.cs
@@ -24,7 +24,7 @@ public sealed class ScanExecutor
{
public static void ScanPartitions(Cluster cluster, ScanPolicy policy, string ns, string setName, string[] binNames, ScanCallback callback, PartitionTracker tracker)
{
- cluster.AddTran();
+ cluster.AddCommand();
while (true)
{
diff --git a/AerospikeClient/Command/SyncCommand.cs b/AerospikeClient/Command/SyncCommand.cs
index 0f74c3d6..7333516a 100644
--- a/AerospikeClient/Command/SyncCommand.cs
+++ b/AerospikeClient/Command/SyncCommand.cs
@@ -14,7 +14,9 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
+using System;
using System.Net.Sockets;
+using System.Runtime.InteropServices;
using static Aerospike.Client.Latency;
namespace Aerospike.Client
@@ -26,6 +28,11 @@ public abstract class SyncCommand : Command
internal int iteration = 1;
internal int commandSentCounter;
internal DateTime deadline;
+ protected int resultCode;
+ protected int generation;
+ protected int expiration;
+ protected int fieldCount;
+ protected int opCount;
///
/// Default constructor.
@@ -339,6 +346,17 @@ protected internal void SizeBuffer(int size)
}
}
+ protected void SkipFields(int fieldCount)
+ {
+ // There can be fields in the response (setname etc).
+ // But for now, ignore them. Expose them to the API if needed in the future.
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int fieldlen = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4 + fieldlen;
+ }
+ }
+
protected internal sealed override void End()
{
// Write total size of message.
@@ -346,11 +364,127 @@ protected internal sealed override void End()
ByteUtil.LongToBytes(size, dataBuffer, 0);
}
+ protected void ParseHeader(IConnection conn)
+ {
+ // Read header.
+ conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
+
+ long sz = ByteUtil.BytesToLong(dataBuffer, 0);
+ int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
+
+ if (receiveSize <= 0)
+ {
+ throw new AerospikeException("Invalid receive size: " + receiveSize);
+ }
+
+ SizeBuffer(receiveSize);
+ conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
+ conn.UpdateLastUsed();
+
+ ulong type = (ulong)(sz >> 48) & 0xff;
+
+ if (type == Command.AS_MSG_TYPE)
+ {
+ dataOffset = 5;
+ }
+ else if (type == Command.MSG_TYPE_COMPRESSED)
+ {
+ int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
+ byte[] ubuf = new byte[usize];
+
+ ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
+ dataBuffer = ubuf;
+ dataOffset = 13;
+ }
+ else
+ {
+ throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
+ }
+
+ this.resultCode = dataBuffer[dataOffset] & 0xFF;
+ dataOffset++;
+ this.generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+ this.expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 8;
+ this.fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+ this.opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
+ dataOffset += 2;
+ }
+
+ protected void ParseFields(Txn tran, Key key, bool hasWrite)
+ {
+ if (tran == null)
+ {
+ SkipFields(fieldCount);
+ return;
+ }
+
+ long? version = null;
+
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int type = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (type == FieldType.RECORD_VERSION)
+ {
+ if (size == 7)
+ {
+ version = ByteUtil.VersionBytesToLong(dataBuffer, dataOffset);
+ }
+ else
+ {
+ throw new AerospikeException("Record version field has invalid size: " + size);
+ }
+ }
+ dataOffset += size;
+ }
+
+ if (hasWrite)
+ {
+ tran.OnWrite(key, version, resultCode);
+ }
+ else
+ {
+ tran.OnRead(key, version);
+ }
+ }
+
+ protected void ParseTranDeadline(Txn txn)
+ {
+ for (int i = 0; i < fieldCount; i++)
+ {
+ int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
+ dataOffset += 4;
+
+ int type = dataBuffer[dataOffset++];
+ int size = len - 1;
+
+ if (type == FieldType.MRT_DEADLINE)
+ {
+ int deadline = ByteUtil.LittleBytesToInt(dataBuffer, dataOffset);
+ txn.Deadline = deadline;
+ }
+ dataOffset += size;
+ }
+ }
+
protected internal sealed override void SetLength(int length)
{
dataOffset = length;
}
+ // Do nothing by default. Write commands will override this method.
+ protected internal virtual void OnInDoubt()
+ {
+
+ }
+
protected internal virtual bool RetryBatch
(
Cluster cluster,
diff --git a/AerospikeClient/Command/SyncReadCommand.cs b/AerospikeClient/Command/SyncReadCommand.cs
new file mode 100644
index 00000000..dd258924
--- /dev/null
+++ b/AerospikeClient/Command/SyncReadCommand.cs
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public abstract class SyncReadCommand : SyncCommand
+ {
+ protected readonly Key key;
+ private readonly Partition partition;
+
+ public SyncReadCommand(Cluster cluster, Policy policy, Key key)
+ : base(cluster, policy)
+ {
+ this.key = key;
+ this.partition = Partition.Read(cluster, policy, key);
+ cluster.AddCommand();
+ }
+
+ protected internal override Node GetNode()
+ {
+ return partition.GetNodeRead(cluster);
+ }
+
+ protected override Latency.LatencyType GetLatencyType()
+ {
+ return Latency.LatencyType.READ;
+ }
+
+ protected internal override bool PrepareRetry(bool timeout)
+ {
+ partition.PrepareRetryRead(timeout);
+ return true;
+ }
+
+ protected internal abstract override void WriteBuffer();
+
+ protected internal abstract override void ParseResult(IConnection conn);
+ }
+}
diff --git a/AerospikeClient/Command/SyncWriteCommand.cs b/AerospikeClient/Command/SyncWriteCommand.cs
index 5380f2e3..dd5cb31a 100644
--- a/AerospikeClient/Command/SyncWriteCommand.cs
+++ b/AerospikeClient/Command/SyncWriteCommand.cs
@@ -15,6 +15,8 @@
* the License.
*/
+using System.Runtime.InteropServices;
+
namespace Aerospike.Client
{
public abstract class SyncWriteCommand : SyncCommand
@@ -29,7 +31,7 @@ public SyncWriteCommand(Cluster cluster, WritePolicy writePolicy, Key key)
this.writePolicy = writePolicy;
this.key = key;
this.partition = Partition.Write(cluster, writePolicy, key);
- cluster.AddTran();
+ cluster.AddCommand();
}
protected internal override bool IsWrite()
@@ -53,105 +55,11 @@ protected internal override bool PrepareRetry(bool timeout)
return true;
}
- protected int ParseHeader(IConnection conn)
- {
- // Read header.
- conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
-
- long sz = ByteUtil.BytesToLong(dataBuffer, 0);
- int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
-
- if (receiveSize <= 0)
- {
- throw new AerospikeException("Invalid receive size: " + receiveSize);
- }
-
- SizeBuffer(receiveSize);
- conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
- conn.UpdateLastUsed();
-
- ulong type = (ulong)(sz >> 48) & 0xff;
-
- if (type == Command.AS_MSG_TYPE)
- {
- dataOffset = 5;
- }
- else if (type == Command.MSG_TYPE_COMPRESSED)
- {
- int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
- byte[] ubuf = new byte[usize];
-
- ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
- dataBuffer = ubuf;
- dataOffset = 13;
- }
- else
- {
- throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
- }
-
- int resultCode = dataBuffer[dataOffset] & 0xFF;
- dataOffset++;
- int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4;
- int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 8;
- int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
- int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
-
- if (policy.Tran == null)
- {
- SkipFields(fieldCount);
- if (opCount > 0)
- {
- throw new AerospikeException("Unexpected write response opCount: " + opCount + ',' + resultCode);
- }
- return resultCode;
- }
-
- long? version = null;
-
- for (int i = 0; i < fieldCount; i++)
- {
- int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4;
-
- int fieldType = dataBuffer[dataOffset++];
- int size = len - 1;
-
- if (fieldType == FieldType.RECORD_VERSION)
- {
- if (size == 7)
- {
- version = ByteUtil.VersionBytesToLong(dataBuffer, dataOffset);
- }
- else
- {
- throw new AerospikeException("Record version field has invalid size: " + size);
- }
- }
- dataOffset += size;
- }
-
- policy.Tran.OnWrite(key, version, resultCode);
-
- if (opCount > 0)
- {
- throw new AerospikeException("Unexpected write response opCount: " + opCount + ',' + resultCode);
- }
- return resultCode;
- }
-
- private void SkipFields(int fieldCount)
+ protected internal override void OnInDoubt()
{
- // There can be fields in the response (setname etc).
- // But for now, ignore them. Expose them to the API if needed in the future.
- for (int i = 0; i < fieldCount; i++)
+ if (writePolicy.Txn != null)
{
- int fieldlen = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4 + fieldlen;
+ writePolicy.Txn.OnWriteInDoubt(key);
}
}
diff --git a/AerospikeClient/Command/TouchCommand.cs b/AerospikeClient/Command/TouchCommand.cs
index ecad9438..d2206286 100644
--- a/AerospikeClient/Command/TouchCommand.cs
+++ b/AerospikeClient/Command/TouchCommand.cs
@@ -17,34 +17,12 @@
namespace Aerospike.Client
{
- public sealed class TouchCommand : SyncCommand
+ public sealed class TouchCommand : SyncWriteCommand
{
- private readonly WritePolicy writePolicy;
- private readonly Key key;
- private readonly Partition partition;
-
public TouchCommand(Cluster cluster, WritePolicy writePolicy, Key key)
- : base(cluster, writePolicy)
- {
- this.writePolicy = writePolicy;
- this.key = key;
- this.partition = Partition.Write(cluster, writePolicy, key);
- cluster.AddTran();
- }
-
- protected internal override bool IsWrite()
- {
- return true;
- }
-
- protected internal override Node GetNode()
+ : base(cluster, writePolicy, key)
{
- return partition.GetNodeWrite(cluster);
- }
-
- protected override Latency.LatencyType GetLatencyType()
- {
- return Latency.LatencyType.WRITE;
+ cluster.AddCommand();
}
protected internal override void WriteBuffer()
@@ -54,13 +32,9 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, MSG_TOTAL_HEADER_SIZE, Command.STATE_READ_HEADER);
- conn.UpdateLastUsed();
-
- int resultCode = dataBuffer[13];
+ ParseHeader(conn);
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
return;
}
@@ -76,11 +50,5 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
-
- protected internal override bool PrepareRetry(bool timeout)
- {
- partition.PrepareRetryWrite(timeout);
- return true;
- }
}
}
diff --git a/AerospikeClient/Command/TranAddKeys.cs b/AerospikeClient/Command/TranAddKeys.cs
deleted file mode 100644
index 39d71567..00000000
--- a/AerospikeClient/Command/TranAddKeys.cs
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Copyright 2012-2024 Aerospike, Inc.
- *
- * Portions may be licensed to Aerospike, Inc. under one or more contributor
- * license agreements.
- *
- * Licensed under the Apache License, Version 2.0 (the "License"); you may not
- * use this file except in compliance with the License. You may obtain a copy of
- * the License at http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-namespace Aerospike.Client
-{
- public sealed class TranAddKeys : SyncWriteCommand
- {
- private readonly OperateArgs args;
-
- public TranAddKeys (Cluster cluster, Key key, OperateArgs args)
- : base(cluster, args.writePolicy, key)
- {
- this.args = args;
- }
-
- protected internal override void WriteBuffer()
- {
- SetTranAddKeys(args.writePolicy, key, args);
- }
-
- protected internal override void ParseResult(IConnection conn)
- {
- // Read header.
- conn.ReadFully(dataBuffer, 8, Command.STATE_READ_HEADER);
-
- long sz = ByteUtil.BytesToLong(dataBuffer, 0);
- int receiveSize = (int)(sz & 0xFFFFFFFFFFFFL);
-
- if (receiveSize <= 0)
- {
- throw new AerospikeException("Invalid receive size: " + receiveSize);
- }
-
- SizeBuffer(receiveSize);
- conn.ReadFully(dataBuffer, receiveSize, Command.STATE_READ_DETAIL);
- conn.UpdateLastUsed();
-
- ulong type = (ulong)(sz >> 48) & 0xff;
-
- if (type == Command.AS_MSG_TYPE)
- {
- dataOffset = 5;
- }
- else if (type == Command.MSG_TYPE_COMPRESSED)
- {
- int usize = (int)ByteUtil.BytesToLong(dataBuffer, 0);
- byte[] ubuf = new byte[usize];
-
- ByteUtil.Decompress(dataBuffer, 8, receiveSize, ubuf, usize);
- dataBuffer = ubuf;
- dataOffset = 13;
- }
- else
- {
- throw new AerospikeException("Invalid proto type: " + type + " Expected: " + Command.AS_MSG_TYPE);
- }
-
- int resultCode = dataBuffer[dataOffset] & 0xFF;
- dataOffset++;
- int generation = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4;
- int expiration = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 8;
- int fieldCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
- int opCount = ByteUtil.BytesToShort(dataBuffer, dataOffset);
- dataOffset += 2;
-
- for (int i = 0; i < fieldCount; i++)
- {
- int len = ByteUtil.BytesToInt(dataBuffer, dataOffset);
- dataOffset += 4;
-
- int fieldType = dataBuffer[dataOffset++];
- int size = len - 1;
-
- if (fieldType == FieldType.MRT_DEADLINE)
- {
- int deadline = ByteUtil.LittleBytesToInt(dataBuffer, dataOffset);
- policy.Tran.Deadline = deadline;
- }
- dataOffset += size;
- }
-
- if (resultCode == ResultCode.OK)
- {
- return;
- }
-
- throw new AerospikeException(resultCode);
- }
- }
-}
diff --git a/AerospikeClient/Command/TxnAddKeys.cs b/AerospikeClient/Command/TxnAddKeys.cs
new file mode 100644
index 00000000..85846e6c
--- /dev/null
+++ b/AerospikeClient/Command/TxnAddKeys.cs
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ public sealed class TxnAddKeys : SyncWriteCommand
+ {
+ private readonly OperateArgs args;
+
+ public TxnAddKeys (Cluster cluster, Key key, OperateArgs args)
+ : base(cluster, args.writePolicy, key)
+ {
+ this.args = args;
+ }
+
+ protected internal override void WriteBuffer()
+ {
+ SetTxnAddKeys(args.writePolicy, key, args);
+ }
+
+ protected internal override void ParseResult(IConnection conn)
+ {
+ ParseHeader(conn);
+ ParseTranDeadline(policy.Txn);
+
+ if (resultCode == ResultCode.OK)
+ {
+ return;
+ }
+
+ throw new AerospikeException(resultCode);
+ }
+
+ protected internal override void OnInDoubt()
+ {
+ policy.Txn.SetMonitorInDoubt();
+ }
+ }
+}
diff --git a/AerospikeClient/Command/TranClose.cs b/AerospikeClient/Command/TxnClose.cs
similarity index 80%
rename from AerospikeClient/Command/TranClose.cs
rename to AerospikeClient/Command/TxnClose.cs
index 4bd1ff49..2c8319b9 100644
--- a/AerospikeClient/Command/TranClose.cs
+++ b/AerospikeClient/Command/TxnClose.cs
@@ -17,11 +17,11 @@
namespace Aerospike.Client
{
- public sealed class TranClose : SyncWriteCommand
+ public sealed class TxnClose : SyncWriteCommand
{
- private readonly Tran tran;
+ private readonly Txn tran;
- public TranClose(Cluster cluster, Tran tran, WritePolicy writePolicy, Key key)
+ public TxnClose(Cluster cluster, Txn tran, WritePolicy writePolicy, Key key)
: base(cluster, writePolicy, key)
{
this.tran = tran;
@@ -29,12 +29,12 @@ public TranClose(Cluster cluster, Tran tran, WritePolicy writePolicy, Key key)
protected internal override void WriteBuffer()
{
- SetTranClose(tran, key);
+ SetTxnClose(tran, key);
}
protected internal override void ParseResult(IConnection conn)
{
- int resultCode = ParseHeader(conn);
+ ParseHeader(conn);
if (resultCode == ResultCode.OK || resultCode == ResultCode.KEY_NOT_FOUND_ERROR)
{
@@ -43,5 +43,9 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
+
+ protected internal override void OnInDoubt()
+ {
+ }
}
}
diff --git a/AerospikeClient/Command/TranMarkRollForward.cs b/AerospikeClient/Command/TxnMarkRollForward.cs
similarity index 70%
rename from AerospikeClient/Command/TranMarkRollForward.cs
rename to AerospikeClient/Command/TxnMarkRollForward.cs
index dde97904..838ce457 100644
--- a/AerospikeClient/Command/TranMarkRollForward.cs
+++ b/AerospikeClient/Command/TxnMarkRollForward.cs
@@ -17,11 +17,11 @@
namespace Aerospike.Client
{
- public sealed class TranMarkRollForward : SyncWriteCommand
+ public sealed class TxnMarkRollForward : SyncWriteCommand
{
- private readonly Tran tran;
+ private readonly Txn tran;
- public TranMarkRollForward(Cluster cluster, Tran tran, WritePolicy writePolicy, Key key)
+ public TxnMarkRollForward(Cluster cluster, Txn tran, WritePolicy writePolicy, Key key)
: base(cluster, writePolicy, key)
{
this.tran = tran;
@@ -29,13 +29,15 @@ public TranMarkRollForward(Cluster cluster, Tran tran, WritePolicy writePolicy,
protected internal override void WriteBuffer()
{
- SetTranMarkRollForward(tran, key);
+ SetTxnMarkRollForward(tran, key);
}
protected internal override void ParseResult(IConnection conn)
{
- int resultCode = ParseHeader(conn);
+ ParseHeader(conn);
+ // BIN_EXISTS_ERROR is considered a success because it means a previous attempt already
+ // succeeded in notifying the server that the MRT will be rolled forward.
if (resultCode == ResultCode.OK || resultCode == ResultCode.BIN_EXISTS_ERROR)
{
return;
@@ -43,5 +45,9 @@ protected internal override void ParseResult(IConnection conn)
throw new AerospikeException(resultCode);
}
+
+ protected internal override void OnInDoubt()
+ {
+ }
}
}
diff --git a/AerospikeClient/Command/TranMonitor.cs b/AerospikeClient/Command/TxnMonitor.cs
similarity index 79%
rename from AerospikeClient/Command/TranMonitor.cs
rename to AerospikeClient/Command/TxnMonitor.cs
index 7fa7584c..c20873f3 100644
--- a/AerospikeClient/Command/TranMonitor.cs
+++ b/AerospikeClient/Command/TxnMonitor.cs
@@ -17,7 +17,7 @@
namespace Aerospike.Client
{
- public sealed class TranMonitor
+ public sealed class TxnMonitor
{
private static readonly ListPolicy OrderedListPolicy = new(ListOrder.ORDERED,
ListWriteFlags.ADD_UNIQUE | ListWriteFlags.NO_FAIL | ListWriteFlags.PARTIAL);
@@ -27,27 +27,27 @@ public sealed class TranMonitor
public static void AddKey(Cluster cluster, WritePolicy policy, Key cmdKey)
{
- Tran tran = policy.Tran;
+ Txn txn = policy.Txn;
- if (tran.Writes.Contains(cmdKey))
+ if (txn.Writes.Contains(cmdKey))
{
// Transaction monitor already contains this key.
return;
}
- Operation[] ops = GetTranOps(tran, cmdKey);
+ Operation[] ops = GetTranOps(txn, cmdKey);
AddWriteKeys(cluster, policy, ops);
}
public static void AddKeys(Cluster cluster, BatchPolicy policy, Key[] keys)
{
- Operation[] ops = GetTranOps(policy.Tran, keys);
+ Operation[] ops = GetTranOps(policy.Txn, keys);
AddWriteKeys(cluster, policy, ops);
}
public static void AddKeys(Cluster cluster, BatchPolicy policy, List records)
{
- Operation[] ops = GetTranOps(policy.Tran, records);
+ Operation[] ops = GetTranOps(policy.Txn, records);
if (ops != null)
{
@@ -55,9 +55,9 @@ public static void AddKeys(Cluster cluster, BatchPolicy policy, List list = new(keys.Length);
foreach (Key key in keys)
{
- tran.Ns = key.ns;
+ tran.SetNamespace(key.ns);
list.Add(Value.Get(key.digest));
}
return GetTranOps(tran, list);
}
- public static Operation[] GetTranOps(Tran tran, List records)
+ public static Operation[] GetTranOps(Txn tran, List records)
{
List list = new(records.Count);
foreach (BatchRecord br in records) {
- tran.Ns = br.key.ns;
+ tran.SetNamespace(br.key.ns);
if (br.hasWrite)
{
@@ -108,7 +108,7 @@ public static Operation[] GetTranOps(Tran tran, List records)
return GetTranOps(tran, list);
}
- private static Operation[] GetTranOps(Tran tran, List list)
+ private static Operation[] GetTranOps(Txn tran, List list)
{
if (tran.Deadline == 0)
{
@@ -128,14 +128,14 @@ private static Operation[] GetTranOps(Tran tran, List list)
private static void AddWriteKeys(Cluster cluster, Policy policy, Operation[] ops)
{
- Key tranKey = GetTranMonitorKey(policy.Tran);
+ Key tranKey = GetTxnMonitorKey(policy.Txn);
WritePolicy wp = CopyTimeoutPolicy(policy);
OperateArgs args = new(wp, null, null, ops);
- TranAddKeys cmd = new(cluster, tranKey, args);
+ TxnAddKeys cmd = new(cluster, tranKey, args);
cmd.Execute();
}
- public static Key GetTranMonitorKey(Tran tran)
+ public static Key GetTxnMonitorKey(Txn tran)
{
return new Key(tran.Ns, " keySet = tran.Writes;
+ txnKey = TxnMonitor.GetTxnMonitorKey(txn);
+ HashSet keySet = txn.Writes;
if (keySet.Count != 0)
{
// Tell MRT monitor that a roll-forward will commence.
try
{
- MarkRollForward(writePolicy, tranKey);
+ MarkRollForward(writePolicy, txnKey);
}
catch (Exception t)
{
@@ -104,23 +105,25 @@ public void Commit(BatchPolicy verifyPolicy, BatchPolicy rollPolicy)
}
}
- if (tran.Deadline != 0)
+ if (txn.Deadline != 0)
{
// Remove MRT monitor.
try
{
- Close(writePolicy, tranKey);
+ Close(writePolicy, txnKey);
}
catch (Exception t)
{
- throw new AerospikeException.Commit(CommitErrorType.CLOSE_ABANDONED, verifyRecords, rollRecords, t);
+ return CommitStatusType.CLOSE_ABANDONED;
}
}
+
+ return CommitStatusType.OK;
}
- public void Abort(BatchPolicy rollPolicy)
+ public AbortStatusType Abort(BatchPolicy rollPolicy)
{
- HashSet keySet = tran.Writes;
+ HashSet keySet = txn.Writes;
if (keySet.Count != 0)
{
@@ -128,30 +131,32 @@ public void Abort(BatchPolicy rollPolicy)
{
Roll(rollPolicy, Command.INFO4_MRT_ROLL_BACK);
}
- catch (Exception t)
+ catch (Exception)
{
- throw new AerospikeException.Abort(AbortErrorType.ROLL_BACK_ABANDONED, rollRecords, t);
+ return AbortStatusType.ROLL_BACK_ABANDONED;
}
}
- if (tran.Deadline != 0)
+ if (txn.Deadline != 0)
{
try
{
WritePolicy writePolicy = new(rollPolicy);
- Key tranKey = TranMonitor.GetTranMonitorKey(tran);
- Close(writePolicy, tranKey);
+ Key txnKey = TxnMonitor.GetTxnMonitorKey(txn);
+ Close(writePolicy, txnKey);
}
catch (Exception t)
{
- throw new AerospikeException.Abort(AbortErrorType.CLOSE_ABANDONED, rollRecords, t);
+ return AbortStatusType.CLOSE_ABANDONED;
}
}
+
+ return AbortStatusType.OK;
}
private void Verify(BatchPolicy verifyPolicy)
{
// Validate record versions in a batch.
- HashSet> reads = tran.Reads.ToHashSet>();
+ HashSet> reads = txn.Reads.ToHashSet>();
int max = reads.Count;
if (max == 0)
{
@@ -181,8 +186,8 @@ private void Verify(BatchPolicy verifyPolicy)
foreach (BatchNode bn in bns)
{
- commands[count++] = new BatchTranVerify(
- cluster, bn, verifyPolicy, tran, keys, versions, records, status);
+ commands[count++] = new BatchTxnVerify(
+ cluster, bn, verifyPolicy, txn, keys, versions, records, status);
}
BatchExecutor.Execute(cluster, verifyPolicy, commands, status);
@@ -193,16 +198,16 @@ private void Verify(BatchPolicy verifyPolicy)
}
}
- private void MarkRollForward(WritePolicy writePolicy, Key tranKey)
+ private void MarkRollForward(WritePolicy writePolicy, Key txnKey)
{
// Tell MRT monitor that a roll-forward will commence.
- TranMarkRollForward cmd = new(cluster, tran, writePolicy, tranKey);
+ TxnMarkRollForward cmd = new(cluster, txn, writePolicy, txnKey);
cmd.Execute();
}
- private void Roll(BatchPolicy rollPolicy, int tranAttr)
+ private void Roll(BatchPolicy rollPolicy, int txnAttr)
{
- HashSet keySet = tran.Writes;
+ HashSet keySet = txn.Writes;
if (keySet.Count == 0)
{
@@ -219,44 +224,44 @@ private void Roll(BatchPolicy rollPolicy, int tranAttr)
this.rollRecords = records;
- // Copy tran roll policy because it needs to be modified.
+ // Copy txn roll policy because it needs to be modified.
BatchPolicy batchPolicy = new(rollPolicy);
BatchAttr attr = new();
- attr.SetTran(tranAttr);
+ attr.SetTxn(txnAttr);
BatchStatus status = new(true);
- // generate() requires a null tran instance.
+ // generate() requires a null txn instance.
List bns = BatchNode.GenerateList(cluster, batchPolicy, keys, records, true, status);
BatchCommand[] commands = new BatchCommand[bns.Count];
- // Batch roll forward requires the tran instance.
- batchPolicy.Tran = tran;
+ // Batch roll forward requires the txn instance.
+ batchPolicy.Txn = txn;
int count = 0;
foreach (BatchNode bn in bns)
{
- commands[count++] = new BatchTranRoll(
+ commands[count++] = new BatchTxnRoll(
cluster, bn, batchPolicy, keys, records, attr, status);
}
BatchExecutor.Execute(cluster, batchPolicy, commands, status);
if (!status.GetStatus())
{
- string rollString = tranAttr == Command.INFO4_MRT_ROLL_FORWARD ? "commit" : "abort";
+ string rollString = txnAttr == Command.INFO4_MRT_ROLL_FORWARD ? "commit" : "abort";
throw new AerospikeException("Failed to " + rollString + " one or more records");
}
}
- private void Close(WritePolicy writePolicy, Key tranKey)
+ private void Close(WritePolicy writePolicy, Key txnKey)
{
// Delete MRT monitor on server.
- TranClose cmd = new(cluster, tran, writePolicy, tranKey);
+ TxnClose cmd = new(cluster, txn, writePolicy, txnKey);
cmd.Execute();
// Reset MRT on client.
- tran.Clear();
+ txn.Clear();
}
}
}
diff --git a/AerospikeClient/Command/WriteCommand.cs b/AerospikeClient/Command/WriteCommand.cs
index 05b9eb91..4363427b 100644
--- a/AerospikeClient/Command/WriteCommand.cs
+++ b/AerospikeClient/Command/WriteCommand.cs
@@ -27,7 +27,7 @@ public WriteCommand(Cluster cluster, WritePolicy writePolicy, Key key, Bin[] bin
{
this.bins = bins;
this.operation = operation;
- cluster.AddTran();
+ cluster.AddCommand();
}
protected internal override void WriteBuffer()
@@ -37,13 +37,9 @@ protected internal override void WriteBuffer()
protected internal override void ParseResult(IConnection conn)
{
- // Read header.
- conn.ReadFully(dataBuffer, MSG_TOTAL_HEADER_SIZE, Command.STATE_READ_HEADER);
- conn.UpdateLastUsed();
+ ParseHeader(conn);
- int resultCode = dataBuffer[13];
-
- if (resultCode == 0)
+ if (resultCode == ResultCode.OK)
{
return;
}
diff --git a/AerospikeClient/Listener/AbortListener.cs b/AerospikeClient/Listener/AbortListener.cs
index 3a86682e..4e7f29b5 100644
--- a/AerospikeClient/Listener/AbortListener.cs
+++ b/AerospikeClient/Listener/AbortListener.cs
@@ -14,6 +14,8 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
+using static Aerospike.Client.AbortStatus;
+
namespace Aerospike.Client
{
///
@@ -24,7 +26,7 @@ public interface AbortListener
///
/// This method is called when the abort succeeds.
///
- void OnSuccess();
+ void OnSuccess(AbortStatusType status);
///
/// This method is called when the abort fails.
diff --git a/AerospikeClient/Listener/CommitListener.cs b/AerospikeClient/Listener/CommitListener.cs
index f77bc571..6e32a04d 100644
--- a/AerospikeClient/Listener/CommitListener.cs
+++ b/AerospikeClient/Listener/CommitListener.cs
@@ -14,6 +14,8 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
+using static Aerospike.Client.CommitStatus;
+
namespace Aerospike.Client
{
///
@@ -24,7 +26,7 @@ public interface CommitListener
///
/// This method is called when the records are verified and the commit succeeds.
///
- void OnSuccess();
+ void OnSuccess(CommitStatusType status);
///
/// This method is called when the commit fails.
diff --git a/AerospikeClient/Main/AbortStatus.cs b/AerospikeClient/Main/AbortStatus.cs
new file mode 100644
index 00000000..3bbe81e1
--- /dev/null
+++ b/AerospikeClient/Main/AbortStatus.cs
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) abort error status code
+ ///
+ public static class AbortStatus
+ {
+ public enum AbortStatusType
+ {
+ OK,
+ ALREADY_ATTEMPTED,
+ ROLL_BACK_ABANDONED,
+ CLOSE_ABANDONED
+ }
+
+ public static string AbortErrorToString(AbortStatusType status)
+ {
+ return status switch
+ {
+ AbortStatusType.OK => "Abort succeeded.",
+ AbortStatusType.ALREADY_ATTEMPTED => "Abort or commit already attempted.",
+ AbortStatusType.ROLL_BACK_ABANDONED => "MRT client roll back abandoned. Server will eventually abort the MRT.",
+ AbortStatusType.CLOSE_ABANDONED => "MRT has been rolled back, but MRT client close was abandoned. Server will eventually close the MRT.",
+ _ => "Unexpected AbortStatusType."
+ };
+ }
+ }
+}
diff --git a/AerospikeClient/Main/AerospikeClient.cs b/AerospikeClient/Main/AerospikeClient.cs
index 94d3200d..d656a1bf 100644
--- a/AerospikeClient/Main/AerospikeClient.cs
+++ b/AerospikeClient/Main/AerospikeClient.cs
@@ -97,13 +97,13 @@ public class AerospikeClient : IDisposable, IAerospikeClient
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- public BatchPolicy tranVerifyPolicyDefault;
+ public TxnVerifyPolicy txnVerifyPolicyDefault;
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- public BatchPolicy tranRollPolicyDefault;
+ public TxnRollPolicy txnRollPolicyDefault;
///
/// Default info policy that is used when info command policy is null.
@@ -201,8 +201,8 @@ public AerospikeClient(ClientPolicy policy, params Host[] hosts)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
- this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
- this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
+ this.txnVerifyPolicyDefault = policy.txnVerifyPolicyDefault;
+ this.txnRollPolicyDefault = policy.txnRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -227,8 +227,8 @@ protected internal AerospikeClient(ClientPolicy policy)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
- this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
- this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
+ this.txnVerifyPolicyDefault = policy.txnVerifyPolicyDefault;
+ this.txnRollPolicyDefault = policy.txnRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
}
else
@@ -242,8 +242,8 @@ protected internal AerospikeClient(ClientPolicy policy)
this.batchWritePolicyDefault = new BatchWritePolicy();
this.batchDeletePolicyDefault = new BatchDeletePolicy();
this.batchUDFPolicyDefault = new BatchUDFPolicy();
- this.tranVerifyPolicyDefault = new TranVerifyPolicy();
- this.tranRollPolicyDefault= new TranRollPolicy();
+ this.txnVerifyPolicyDefault = new TxnVerifyPolicy();
+ this.txnRollPolicyDefault= new TxnRollPolicy();
this.infoPolicyDefault = new InfoPolicy();
}
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -340,20 +340,20 @@ public BatchUDFPolicy BatchUDFPolicyDefault
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- public BatchPolicy TranVerifyPolicyDefault
+ public TxnVerifyPolicy TxnVerifyPolicyDefault
{
- get { return new BatchPolicy(tranVerifyPolicyDefault); }
- set { tranVerifyPolicyDefault = value; }
+ get { return new TxnVerifyPolicy(txnVerifyPolicyDefault); }
+ set { txnVerifyPolicyDefault = value; }
}
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- public BatchPolicy TranRollPolicyDefault
+ public TxnRollPolicy TxnRollPolicyDefault
{
- get { return new BatchPolicy(tranRollPolicyDefault); }
- set { tranRollPolicyDefault = value; }
+ get { return new TxnRollPolicy(txnRollPolicyDefault); }
+ set { txnRollPolicyDefault = value; }
}
///
@@ -464,19 +464,22 @@ public ClusterStats GetClusterStats()
///
/// Attempt to commit the given multi-record transaction. First, the expected record versions are
- /// sent to the server nodes for verification.If all nodes return success, the transaction is
+ /// sent to the server nodes for verification. If all nodes return success, the transaction is
/// committed.Otherwise, the transaction is aborted.
///
/// Requires server version 8.0+
///
///
- /// multi-record transaction
- public void Commit(Tran tran)
+ /// multi-record transaction
+ public CommitStatus.CommitStatusType Commit(Txn txn)
{
- tran.SetRollAttempted();
+ if (!txn.SetRollAttempted())
+ {
+ return CommitStatus.CommitStatusType.ALREADY_ATTEMPTED;
+ }
- TranRoll tm = new TranRoll(cluster, tran);
- tm.Commit(tranVerifyPolicyDefault, tranRollPolicyDefault);
+ TxnRoll tm = new TxnRoll(cluster, txn);
+ return tm.Commit(txnVerifyPolicyDefault, txnRollPolicyDefault);
}
///
@@ -485,13 +488,16 @@ public void Commit(Tran tran)
/// Requires server version 8.0+
///
///
- /// multi-record transaction
- public void Abort(Tran tran)
+ /// multi-record transaction
+ public AbortStatus.AbortStatusType Abort(Txn txn)
{
- tran.SetRollAttempted();
+ if (!txn.SetRollAttempted())
+ {
+ return AbortStatus.AbortStatusType.ALREADY_ATTEMPTED;
+ }
- TranRoll tm = new TranRoll(cluster, tran);
- tm.Abort(tranRollPolicyDefault);
+ TxnRoll tm = new TxnRoll(cluster, txn);
+ return tm.Abort(txnRollPolicyDefault);
}
//-------------------------------------------------------
@@ -500,7 +506,7 @@ public void Abort(Tran tran)
///
/// Write record bin(s).
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeouts, record expiration and how the command is
/// handled when the record already exists.
///
/// write configuration parameters, pass in null for defaults
@@ -513,6 +519,12 @@ public void Put(WritePolicy policy, Key key, params Bin[] bins)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
WriteCommand command = new WriteCommand(cluster, policy, key, bins, Operation.Type.WRITE);
command.Execute();
}
@@ -523,7 +535,7 @@ public void Put(WritePolicy policy, Key key, params Bin[] bins)
///
/// Append bin string values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call only works for string values.
///
@@ -537,13 +549,19 @@ public void Append(WritePolicy policy, Key key, params Bin[] bins)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
WriteCommand command = new WriteCommand(cluster, policy, key, bins, Operation.Type.APPEND);
command.Execute();
}
///
/// Prepend bin string values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call works only for string values.
///
@@ -557,6 +575,12 @@ public void Prepend(WritePolicy policy, Key key, params Bin[] bins)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
WriteCommand command = new WriteCommand(cluster, policy, key, bins, Operation.Type.PREPEND);
command.Execute();
}
@@ -567,7 +591,7 @@ public void Prepend(WritePolicy policy, Key key, params Bin[] bins)
///
/// Add integer/double bin values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
/// write configuration parameters, pass in null for defaults
@@ -580,6 +604,12 @@ public void Add(WritePolicy policy, Key key, params Bin[] bins)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
WriteCommand command = new WriteCommand(cluster, policy, key, bins, Operation.Type.ADD);
command.Execute();
}
@@ -591,7 +621,7 @@ public void Add(WritePolicy policy, Key key, params Bin[] bins)
///
/// Delete record for specified key.
/// Return whether record existed on server before deletion.
- /// The policy specifies the transaction timeout.
+ /// The policy specifies the command timeout.
///
/// delete configuration parameters, pass in null for defaults
/// unique record identifier
@@ -602,6 +632,12 @@ public bool Delete(WritePolicy policy, Key key)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
DeleteCommand command = new DeleteCommand(cluster, policy, key);
command.Execute();
return command.Existed();
@@ -635,6 +671,11 @@ public BatchResults Delete(BatchPolicy batchPolicy, BatchDeletePolicy deletePoli
deletePolicy = batchDeletePolicyDefault;
}
+ if (batchPolicy.Txn != null)
+ {
+ TxnMonitor.AddKeys(cluster, batchPolicy, keys);
+ }
+
BatchAttr attr = new BatchAttr();
attr.SetDelete(deletePolicy);
@@ -744,6 +785,12 @@ public void Touch(WritePolicy policy, Key key)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
TouchCommand command = new TouchCommand(cluster, policy, key);
command.Execute();
}
@@ -766,6 +813,12 @@ public bool Exists(Policy policy, Key key)
{
policy = readPolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(key.ns);
+ }
+
ExistsCommand command = new ExistsCommand(cluster, policy, key);
command.Execute();
return command.Exists();
@@ -790,6 +843,10 @@ public bool[] Exists(BatchPolicy policy, Key[] keys)
policy = batchPolicyDefault;
}
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(keys);
+ }
bool[] existsArray = new bool[keys.Length];
@@ -842,6 +899,9 @@ public Record Get(Policy policy, Key key)
{
policy = readPolicyDefault;
}
+
+ policy.Txn?.SetNamespace(key.ns);
+
ReadCommand command = new ReadCommand(cluster, policy, key);
command.Execute();
return command.Record;
@@ -862,6 +922,12 @@ public Record Get(Policy policy, Key key, params string[] binNames)
{
policy = readPolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(key.ns);
+ }
+
ReadCommand command = new ReadCommand(cluster, policy, key, binNames);
command.Execute();
return command.Record;
@@ -881,6 +947,12 @@ public Record GetHeader(Policy policy, Key key)
{
policy = readPolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(key.ns);
+ }
+
ReadHeaderCommand command = new ReadHeaderCommand(cluster, policy, key);
command.Execute();
return command.Record;
@@ -913,6 +985,11 @@ public bool Get(BatchPolicy policy, List records)
policy = batchPolicyDefault;
}
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(records);
+ }
+
BatchStatus status = new BatchStatus(true);
List batchNodes = BatchNode.GenerateList(cluster, policy, records, status);
BatchCommand[] commands = new BatchCommand[batchNodes.Count];
@@ -946,6 +1023,13 @@ public Record[] Get(BatchPolicy policy, Key[] keys)
policy = batchPolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(keys);
+ }
+
+
Record[] records = new Record[keys.Length];
try
@@ -1000,6 +1084,11 @@ public Record[] Get(BatchPolicy policy, Key[] keys, params string[] binNames)
policy = batchPolicyDefault;
}
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(keys);
+ }
+
Record[] records = new Record[keys.Length];
try
@@ -1054,6 +1143,11 @@ public Record[] Get(BatchPolicy policy, Key[] keys, params Operation[] ops)
policy = batchPolicyDefault;
}
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(keys);
+ }
+
Record[] records = new Record[keys.Length];
try
@@ -1107,6 +1201,13 @@ public Record[] GetHeader(BatchPolicy policy, Key[] keys)
policy = batchPolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ policy.Txn.SetNamespace(keys);
+ }
+
+
Record[] records = new Record[keys.Length];
try
@@ -1210,9 +1311,31 @@ public Record Join(BatchPolicy policy, Key key, params Join[] joins)
public Record Operate(WritePolicy policy, Key key, params Operation[] operations)
{
OperateArgs args = new OperateArgs(policy, writePolicyDefault, operatePolicyReadDefault, operations);
- OperateCommand command = new OperateCommand(cluster, key, args);
- command.Execute();
- return command.Record;
+
+ if (args.hasWrite)
+ {
+ policy = args.writePolicy;
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
+ OperateCommandWrite command = new(cluster, key, args);
+ command.Execute();
+ return command.Record;
+ }
+ else
+ {
+ if (policy?.Txn != null)
+ {
+ policy.Txn.SetNamespace(key.ns);
+ }
+
+ OperateCommandRead command = new(cluster, key, args);
+ command.Execute();
+ return command.Record;
+ }
}
//-------------------------------------------------------
@@ -1247,6 +1370,11 @@ public bool Operate(BatchPolicy policy, List records)
policy = batchParentPolicyWriteDefault;
}
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKeys(cluster, policy, records);
+ }
+
BatchStatus status = new BatchStatus(true);
List batchNodes = BatchNode.GenerateList(cluster, policy, records, status);
BatchCommand[] commands = new BatchCommand[batchNodes.Count];
@@ -1293,6 +1421,11 @@ public BatchResults Operate(BatchPolicy batchPolicy, BatchWritePolicy writePolic
writePolicy = batchWritePolicyDefault;
}
+ if (batchPolicy.Txn != null)
+ {
+ TxnMonitor.AddKeys(cluster, batchPolicy, keys);
+ }
+
BatchAttr attr = new BatchAttr(batchPolicy, writePolicy, ops);
BatchRecord[] records = new BatchRecord[keys.Length];
@@ -1567,13 +1700,19 @@ public void RemoveUdf(InfoPolicy policy, string serverPath)
/// server package name where user defined function resides
/// user defined function
/// arguments passed in to user defined function
- /// if transaction fails
+ /// if command fails
public object Execute(WritePolicy policy, Key key, string packageName, string functionName, params Value[] args)
{
if (policy == null)
{
policy = writePolicyDefault;
}
+
+ if (policy.Txn != null)
+ {
+ TxnMonitor.AddKey(cluster, policy, key);
+ }
+
ExecuteCommand command = new ExecuteCommand(cluster, policy, key, packageName, functionName, args);
command.Execute();
@@ -1633,6 +1772,11 @@ public BatchResults Execute(BatchPolicy batchPolicy, BatchUDFPolicy udfPolicy, K
udfPolicy = batchUDFPolicyDefault;
}
+ if (batchPolicy.Txn != null)
+ {
+ TxnMonitor.AddKeys(cluster, batchPolicy, keys);
+ }
+
byte[] argBytes = Packer.Pack(functionArgs);
BatchAttr attr = new BatchAttr();
@@ -1695,7 +1839,7 @@ public ExecuteTask Execute(WritePolicy policy, Statement statement, string packa
statement.FunctionName = functionName;
statement.FunctionArgs = functionArgs;
- cluster.AddTran();
+ cluster.AddCommand();
ulong taskId = statement.PrepareTaskId();
Node[] nodes = cluster.ValidateNodes();
@@ -1734,7 +1878,7 @@ public ExecuteTask Execute(WritePolicy policy, Statement statement, params Opera
statement.Operations = operations;
}
- cluster.AddTran();
+ cluster.AddCommand();
ulong taskId = statement.PrepareTaskId();
Node[] nodes = cluster.ValidateNodes();
diff --git a/AerospikeClient/Main/AerospikeException.cs b/AerospikeClient/Main/AerospikeException.cs
index ec7268b0..312fd4ae 100644
--- a/AerospikeClient/Main/AerospikeException.cs
+++ b/AerospikeClient/Main/AerospikeException.cs
@@ -15,7 +15,7 @@
* the License.
*/
using System.Text;
-using static Aerospike.Client.AbortError;
+using static Aerospike.Client.AbortStatus;
using static Aerospike.Client.CommitError;
namespace Aerospike.Client
@@ -167,7 +167,7 @@ public bool ShouldSerializeNode()
}
///
- /// Transaction policy.
+ /// Command policy.
///
public Policy Policy
{
@@ -208,7 +208,7 @@ public int Iteration
}
///
- /// Is it possible that write transaction may have completed.
+ /// Is it possible that write command may have completed.
///
public bool InDoubt
{
@@ -219,7 +219,7 @@ public bool InDoubt
}
///
- /// Set whether it is possible that the write transaction may have completed
+ /// Set whether it is possible that the write command may have completed
/// even though this exception was generated. This may be the case when a
/// client error occurs (like timeout) after the command was sent to the server.
///
@@ -604,7 +604,7 @@ public EndOfGRPCStream(int resultCode)
}
///
- /// Exception thrown when {@link AerospikeClient#commit(com.aerospike.client.Tran)} fails.
+ /// Exception thrown when {@link AerospikeClient#commit(com.aerospike.client.Txn)} fails.
///
public sealed class Commit : AerospikeException
{
@@ -656,28 +656,28 @@ public override string Message
}
///
- /// Exception thrown when {@link AerospikeClient#abort(com.aerospike.client.Tran)} fails.
+ /// Exception thrown when {@link AerospikeClient#abort(com.aerospike.client.Txn)} fails.
///
public sealed class Abort : AerospikeException
{
///
/// Error status of the attempted abort.
///
- public readonly AbortErrorType Error;
+ public readonly AbortStatusType Error;
///
/// Roll backward result for each write key in the MRT. May be null if failure occurred before roll backward.
///
public readonly BatchRecord[] RollRecords;
- public Abort(AbortErrorType error, BatchRecord[] rollRecords)
+ public Abort(AbortStatusType error, BatchRecord[] rollRecords)
: base(ResultCode.TRAN_FAILED, AbortErrorToString(error))
{
this.Error = error;
this.RollRecords = rollRecords;
}
- public Abort(AbortErrorType error, BatchRecord[] rollRecords, Exception cause)
+ public Abort(AbortStatusType error, BatchRecord[] rollRecords, Exception cause)
: base(ResultCode.TRAN_FAILED, AbortErrorToString(error), cause)
{
this.Error = error;
diff --git a/AerospikeClient/Main/BatchRecord.cs b/AerospikeClient/Main/BatchRecord.cs
index cc59de11..9196404b 100644
--- a/AerospikeClient/Main/BatchRecord.cs
+++ b/AerospikeClient/Main/BatchRecord.cs
@@ -39,7 +39,7 @@ public class BatchRecord
public int resultCode;
///
- /// Is it possible that the write transaction may have completed even though an error
+ /// Is it possible that the write command may have completed even though an error
/// occurred for this record. This may be the case when a client error occurs (like timeout)
/// after the command was sent to the server.
///
diff --git a/AerospikeClient/Main/CommitStatus.cs b/AerospikeClient/Main/CommitStatus.cs
new file mode 100644
index 00000000..021b88f7
--- /dev/null
+++ b/AerospikeClient/Main/CommitStatus.cs
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2012-2024 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+using static Aerospike.Client.AbortStatus;
+
+namespace Aerospike.Client
+{
+ ///
+ /// Multi-record transaction (MRT) commit status code.
+ ///
+ public static class CommitStatus
+ {
+ public enum CommitStatusType
+ {
+ OK,
+ ALREADY_ATTEMPTED,
+ ROLL_FORWARD_ABANDONED,
+ CLOSE_ABANDONED
+ }
+
+ public static string CommitErrorToString(CommitStatusType status)
+ {
+ return status switch
+ {
+ CommitStatusType.OK => "Commit succeeded.",
+ CommitStatusType.ALREADY_ATTEMPTED => "Commit or abort already attempted.",
+ CommitStatusType.ROLL_FORWARD_ABANDONED => "MRT client roll forward abandoned. Server will eventually commit the MRT.",
+ CommitStatusType.CLOSE_ABANDONED => "MRT has been rolled back, but MRT client close was abandoned. Server will eventually close the MRT.",
+ _ => "Unexpected AbortStatusType."
+ };
+ }
+ }
+}
diff --git a/AerospikeClient/Main/IAerospikeClient.cs b/AerospikeClient/Main/IAerospikeClient.cs
index 8e39e632..5adfca00 100644
--- a/AerospikeClient/Main/IAerospikeClient.cs
+++ b/AerospikeClient/Main/IAerospikeClient.cs
@@ -14,7 +14,6 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
-using System.Diagnostics.Metrics;
using System.Reflection;
namespace Aerospike.Client
@@ -76,13 +75,13 @@ public interface IAerospikeClient
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- BatchPolicy TranVerifyPolicyDefault { get; set; }
+ TxnVerifyPolicy TxnVerifyPolicyDefault { get; set; }
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- BatchPolicy TranRollPolicyDefault { get; set; }
+ TxnRollPolicy TxnRollPolicyDefault { get; set; }
///
/// Default info policy that is used when info command policy is null.
@@ -135,23 +134,23 @@ public interface IAerospikeClient
///
/// Attempt to commit the given multi-record transaction. First, the expected record versions are
- /// sent to the server nodes for verification.If all nodes return success, the transaction is
- /// committed.Otherwise, the transaction is aborted.
+ /// sent to the server nodes for verification.If all nodes return success, the command is
+ /// committed.Otherwise, the command is aborted.
///
/// Requires server version 8.0+
///
///
- /// multi-record transaction
- void Commit(Tran tran);
+ /// multi-record transaction
+ CommitStatus.CommitStatusType Commit(Txn txn);
///
/// Abort and rollback the given multi-record transaction.
- ///
- /// Requires server version 8.0+
+ ///
+ /// Requires server version 8.0+
///
///
- /// multi-record transaction
- void Abort(Tran tran);
+ /// multi-record transaction
+ AbortStatus.AbortStatusType Abort(Txn txn);
//-------------------------------------------------------
// Write Record Operations
@@ -159,7 +158,7 @@ public interface IAerospikeClient
///
/// Write record bin(s).
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
/// write configuration parameters, pass in null for defaults
@@ -174,7 +173,7 @@ public interface IAerospikeClient
///
/// Append bin string values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call only works for string values.
///
@@ -186,7 +185,7 @@ public interface IAerospikeClient
///
/// Prepend bin string values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call works only for string values.
///
@@ -202,7 +201,7 @@ public interface IAerospikeClient
///
/// Add integer bin values to existing record bin values.
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call only works for integer values.
///
@@ -219,7 +218,7 @@ public interface IAerospikeClient
///
/// Delete record for specified key.
/// Return whether record existed on server before deletion.
- /// The policy specifies the transaction timeout.
+ /// The policy specifies the command timeout.
///
/// delete configuration parameters, pass in null for defaults
/// unique record identifier
@@ -627,7 +626,7 @@ public interface IAerospikeClient
/// server package name where user defined function resides
/// user defined function
/// arguments passed in to user defined function
- /// if transaction fails
+ /// if command fails
object Execute(WritePolicy policy, Key key, string packageName, string functionName, params Value[] args);
///
diff --git a/AerospikeClient/Main/Key.cs b/AerospikeClient/Main/Key.cs
index 16f05bed..f929545b 100644
--- a/AerospikeClient/Main/Key.cs
+++ b/AerospikeClient/Main/Key.cs
@@ -1,5 +1,5 @@
/*
- * Copyright 2012-2018 Aerospike, Inc.
+ * Copyright 2012-2024 Aerospike, Inc.
*
* Portions may be licensed to Aerospike, Inc. under one or more contributor
* license agreements.
@@ -14,7 +14,6 @@
* License for the specific language governing permissions and limitations under
* the License.
*/
-using System.Linq;
namespace Aerospike.Client
{
@@ -378,11 +377,9 @@ public Key(string ns, byte[] digest, string setName, Value userKey)
///
public override int GetHashCode()
{
- int result = 1;
- foreach (byte element in digest)
- {
- result = 31 * result + element;
- }
+ // The digest is already a hash, so pick 4 bytes from the 20 byte digest at a
+ // random offset (in this case 8).
+ int result = ByteUtil.LittleBytesToInt(digest, 8) + 31;
return 31 * result + ns.GetHashCode();
}
diff --git a/AerospikeClient/Main/ResultCode.cs b/AerospikeClient/Main/ResultCode.cs
index 83a0416b..edd6e470 100644
--- a/AerospikeClient/Main/ResultCode.cs
+++ b/AerospikeClient/Main/ResultCode.cs
@@ -22,7 +22,7 @@ namespace Aerospike.Client
public sealed class ResultCode
{
///
- /// Multi-record transaction failed
+ /// Multi-record transaction failed.
/// Value: -17
///
public const int TRAN_FAILED = -17;
@@ -271,7 +271,7 @@ public sealed class ResultCode
public const int OP_NOT_APPLICABLE = 26;
///
- /// The transaction was not performed because the filter was false.
+ /// The command was not performed because the filter was false.
/// Value: 27
///
public const int FILTERED_OUT = 27;
@@ -690,7 +690,7 @@ public static string GetResultString(int resultCode)
return "Operation not applicable";
case FILTERED_OUT:
- return "Transaction filtered out";
+ return "Command filtered out";
case LOST_CONFLICT:
return "Transaction failed due to conflict with XDR";
diff --git a/AerospikeClient/Main/Tran.cs b/AerospikeClient/Main/Txn.cs
similarity index 63%
rename from AerospikeClient/Main/Tran.cs
rename to AerospikeClient/Main/Txn.cs
index 009cae98..03b3f156 100644
--- a/AerospikeClient/Main/Tran.cs
+++ b/AerospikeClient/Main/Txn.cs
@@ -15,28 +15,30 @@
* the License.
*/
+using System;
using System.Collections.Concurrent;
-using System.Collections.Generic;
namespace Aerospike.Client
{
///
/// Mutli-record transaction (MRT). Each command in the MRT must use the same namespace.
///
- public class Tran
+ public class Txn
{
public long Id { get; private set; }
public ConcurrentDictionary Reads { get; private set; }
public HashSet Writes { get; private set; }
- public string Ns { get; set; }
+ public string Ns { get; private set; }
public int Deadline { get; set; }
+ private bool monitorInDoubt;
+
private bool rollAttempted;
///
/// Create MRT, assign random transaction id and initialize reads/writes hashmaps with default capacities.
///
- public Tran()
+ public Txn()
{
Id = CreateId();
Reads = new ConcurrentDictionary();
@@ -48,7 +50,7 @@ public Tran()
///
/// expected number of record reads in the MRT. Minimum value is 16.
/// expected number of record writes in the MRT. Minimum value is 16.
- public Tran(int readsCapacity, int writesCapacity)
+ public Txn(int readsCapacity, int writesCapacity)
{
if (readsCapacity < 16)
{
@@ -86,8 +88,6 @@ private static long CreateId()
///
internal void OnRead(Key key, long? version)
{
- Ns = key.ns;
-
if (version.HasValue)
{
Reads.TryAdd(key, version.Value);
@@ -99,9 +99,16 @@ internal void OnRead(Key key, long? version)
///
///
///
- public long GetReadVersion(Key key)
+ public long? GetReadVersion(Key key)
{
- return Reads[key];
+ if (Reads.ContainsKey(key))
+ {
+ return Reads[key];
+ }
+ else
+ {
+ return null;
+ }
}
///
@@ -128,14 +135,87 @@ public void OnWrite(Key key, long? version, int resultCode)
}
}
- public void SetRollAttempted()
+ ///
+ /// Add key to write hash when write command is in doubt (usually caused by timeout).
+ ///
+ public void OnWriteInDoubt(Key key)
+ {
+ Reads.Remove(key, out _);
+ Writes.Add(key);
+ }
+
+ ///
+ /// Set MRT namespace only if doesn't already exist.
+ /// If namespace already exists, verify new namespace is the same.
+ ///
+ public void SetNamespace(string ns)
+ {
+ if (Ns == null)
+ {
+ Ns = ns;
+ }
+ else if (!Ns.Equals(ns)) {
+ throw new AerospikeException("Namespace must be the same for all commands in the MRT. orig: " +
+ Ns + " new: " + ns);
+ }
+ }
+
+ ///
+ /// Set MRT namespaces for each key only if doesn't already exist.
+ /// If namespace already exists, verify new namespace is the same.
+ ///
+ public void SetNamespace(Key[] keys)
+ {
+ foreach (Key key in keys)
+ {
+ SetNamespace(key.ns);
+ }
+ }
+
+ ///
+ /// Set MRT namespaces for each key only if doesn't already exist.
+ /// If namespace already exists, verify new namespace is the same.
+ ///
+ public void SetNamespace(List records)
+ {
+ foreach (BatchRead br in records)
+ {
+ SetNamespace(br.key.ns);
+ }
+ }
+
+ ///
+ /// Set that the MRT monitor existence is in doubt.
+ ///
+ public void SetMonitorInDoubt()
+ {
+ this.monitorInDoubt = true;
+ }
+
+ ///
+ /// Does MRT monitor record exist or is in doubt.
+ ///
+ public bool MonitorMightExist()
+ {
+ return Deadline != 0 || monitorInDoubt;
+ }
+
+ ///
+ /// Does MRT monitor record exist.
+ ///
+ public bool MonitorExists()
+ {
+ return Deadline != 0;
+ }
+
+ public bool SetRollAttempted()
{
if (rollAttempted)
{
- throw new AerospikeException(ResultCode.PARAMETER_ERROR,
- "commit() or abort() may only be called once for a given MRT");
+ return false;
}
rollAttempted = true;
+ return true;
}
public void Clear()
diff --git a/AerospikeClient/Metrics/MetricsWriter.cs b/AerospikeClient/Metrics/MetricsWriter.cs
index e3b54d20..7d94989a 100644
--- a/AerospikeClient/Metrics/MetricsWriter.cs
+++ b/AerospikeClient/Metrics/MetricsWriter.cs
@@ -148,7 +148,7 @@ private void Open()
sb.Append(now.ToString(timestampFormat));
sb.Append(" header(1)");
- sb.Append(" cluster[name,cpu,mem,recoverQueueSize,invalidNodeCount,tranCount,retryCount,delayQueueTimeoutCount,asyncThreadsInUse,asyncCompletionPortsInUse,node[]]");
+ sb.Append(" cluster[name,cpu,mem,recoverQueueSize,invalidNodeCount,commandCount,retryCount,delayQueueTimeoutCount,asyncThreadsInUse,asyncCompletionPortsInUse,node[]]");
sb.Append(" node[name,address,port,syncConn,asyncConn,errors,timeouts,latency[]]");
sb.Append(" conn[inUse,inPool,opened,closed]");
sb.Append(" latency(");
@@ -180,7 +180,7 @@ private void WriteCluster(Cluster cluster)
sb.Append(',');
sb.Append(cluster.InvalidNodeCount); // Cumulative. Not reset on each interval.
sb.Append(',');
- sb.Append(cluster.GetTranCount()); // Cumulative. Not reset on each interval.
+ sb.Append(cluster.GetCommandCount()); // Cumulative. Not reset on each interval.
sb.Append(',');
sb.Append(cluster.GetRetryCount()); // Cumulative. Not reset on each interval.
sb.Append(',');
diff --git a/AerospikeClient/Policy/ClientPolicy.cs b/AerospikeClient/Policy/ClientPolicy.cs
index be3dca01..2e9c5edd 100644
--- a/AerospikeClient/Policy/ClientPolicy.cs
+++ b/AerospikeClient/Policy/ClientPolicy.cs
@@ -229,13 +229,13 @@ public class ClientPolicy
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- public TranVerifyPolicy tranVerifyPolicyDefault = new TranVerifyPolicy();
+ public TxnVerifyPolicy txnVerifyPolicyDefault = new TxnVerifyPolicy();
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- public TranRollPolicy tranRollPolicyDefault = new TranRollPolicy();
+ public TxnRollPolicy txnRollPolicyDefault = new TxnRollPolicy();
///
/// Default info policy that is used when info command's policy is null.
@@ -337,8 +337,8 @@ public ClientPolicy(ClientPolicy other)
this.batchWritePolicyDefault = new BatchWritePolicy(other.batchWritePolicyDefault);
this.batchDeletePolicyDefault = new BatchDeletePolicy(other.batchDeletePolicyDefault);
this.batchUDFPolicyDefault = new BatchUDFPolicy(other.batchUDFPolicyDefault);
- this.tranVerifyPolicyDefault = new TranVerifyPolicy(other.tranVerifyPolicyDefault);
- this.tranRollPolicyDefault = new TranRollPolicy(other.tranRollPolicyDefault);
+ this.txnVerifyPolicyDefault = new TxnVerifyPolicy(other.txnVerifyPolicyDefault);
+ this.txnRollPolicyDefault = new TxnRollPolicy(other.txnRollPolicyDefault);
this.infoPolicyDefault = new InfoPolicy(other.infoPolicyDefault);
this.tlsPolicy = (other.tlsPolicy != null) ? new TlsPolicy(other.tlsPolicy) : null;
this.ipMap = other.ipMap;
diff --git a/AerospikeClient/Policy/Policy.cs b/AerospikeClient/Policy/Policy.cs
index aa431247..cd1962e9 100644
--- a/AerospikeClient/Policy/Policy.cs
+++ b/AerospikeClient/Policy/Policy.cs
@@ -31,7 +31,7 @@ public class Policy
/// Default: null
///
///
- public Tran Tran { get; set; }
+ public Txn Txn { get; set; }
///
/// Read policy for AP (availability) namespaces.
@@ -283,7 +283,7 @@ public class Policy
///
public Policy(Policy other)
{
- this.Tran = other.Tran;
+ this.Txn = other.Txn;
this.readModeAP = other.readModeAP;
this.readModeSC = other.readModeSC;
this.replica = other.replica;
@@ -305,7 +305,7 @@ public Policy(Policy other)
///
public Policy()
{
- Tran = null;
+ Txn = null;
}
///
diff --git a/AerospikeClient/Policy/TranRollPolicy.cs b/AerospikeClient/Policy/TxnRollPolicy.cs
similarity index 90%
rename from AerospikeClient/Policy/TranRollPolicy.cs
rename to AerospikeClient/Policy/TxnRollPolicy.cs
index c123c2fa..e2a90034 100644
--- a/AerospikeClient/Policy/TranRollPolicy.cs
+++ b/AerospikeClient/Policy/TxnRollPolicy.cs
@@ -21,12 +21,12 @@ namespace Aerospike.Client
/// Multi-record transaction (MRT) policy fields used to batch roll forward/backward records on
/// commit or abort.Used a placeholder for now as there are no additional fields beyond BatchPolicy.
///
- public sealed class TranRollPolicy : BatchPolicy
+ public sealed class TxnRollPolicy : BatchPolicy
{
///
/// Copy policy from another policy.
///
- public TranRollPolicy(TranRollPolicy other) :
+ public TxnRollPolicy(TxnRollPolicy other) :
base(other)
{
}
@@ -34,7 +34,7 @@ public TranRollPolicy(TranRollPolicy other) :
///
/// Default constructor.
///
- public TranRollPolicy()
+ public TxnRollPolicy()
{
replica = Replica.MASTER;
maxRetries = 5;
diff --git a/AerospikeClient/Policy/TranVerifyPolicy.cs b/AerospikeClient/Policy/TxnVerifyPolicy.cs
similarity index 90%
rename from AerospikeClient/Policy/TranVerifyPolicy.cs
rename to AerospikeClient/Policy/TxnVerifyPolicy.cs
index e641ba33..4130fc98 100644
--- a/AerospikeClient/Policy/TranVerifyPolicy.cs
+++ b/AerospikeClient/Policy/TxnVerifyPolicy.cs
@@ -21,12 +21,12 @@ namespace Aerospike.Client
/// Multi-record transaction (MRT) policy fields used to batch verify record versions on commit.
/// Used a placeholder for now as there are no additional fields beyond BatchPolicy.
///
- public sealed class TranVerifyPolicy : BatchPolicy
+ public sealed class TxnVerifyPolicy : BatchPolicy
{
///
/// Copy policy from another policy.
///
- public TranVerifyPolicy(TranVerifyPolicy other) :
+ public TxnVerifyPolicy(TxnVerifyPolicy other) :
base(other)
{
}
@@ -34,7 +34,7 @@ public TranVerifyPolicy(TranVerifyPolicy other) :
///
/// Default constructor.
///
- public TranVerifyPolicy()
+ public TxnVerifyPolicy()
{
readModeSC = ReadModeSC.LINEARIZE;
replica = Replica.MASTER;
diff --git a/AerospikeClient/Query/QueryExecutor.cs b/AerospikeClient/Query/QueryExecutor.cs
index 44325952..0fc72986 100644
--- a/AerospikeClient/Query/QueryExecutor.cs
+++ b/AerospikeClient/Query/QueryExecutor.cs
@@ -45,7 +45,7 @@ public QueryExecutor(Cluster cluster, QueryPolicy policy, Statement statement, N
// Initialize maximum number of nodes to query in parallel.
this.maxConcurrentNodes = (policy.maxConcurrentNodes == 0 || policy.maxConcurrentNodes >= threads.Length) ? threads.Length : policy.maxConcurrentNodes;
- cluster.AddTran();
+ cluster.AddCommand();
}
protected internal void InitializeThreads()
diff --git a/AerospikeClient/Query/QueryListenerExecutor.cs b/AerospikeClient/Query/QueryListenerExecutor.cs
index da72f62b..b215e835 100644
--- a/AerospikeClient/Query/QueryListenerExecutor.cs
+++ b/AerospikeClient/Query/QueryListenerExecutor.cs
@@ -29,7 +29,7 @@ public static void execute
PartitionTracker tracker
)
{
- cluster.AddTran();
+ cluster.AddCommand();
ulong taskId = statement.PrepareTaskId();
diff --git a/AerospikeClient/Query/QueryPartitionExecutor.cs b/AerospikeClient/Query/QueryPartitionExecutor.cs
index 3c00b4ce..f970055b 100644
--- a/AerospikeClient/Query/QueryPartitionExecutor.cs
+++ b/AerospikeClient/Query/QueryPartitionExecutor.cs
@@ -51,7 +51,7 @@ PartitionTracker tracker
this.cancel = new CancellationTokenSource();
this.tracker = tracker;
this.recordSet = new RecordSet(this, policy.recordQueueSize, cancel.Token);
- cluster.AddTran();
+ cluster.AddCommand();
ThreadPool.UnsafeQueueUserWorkItem(this.Run, null);
}
diff --git a/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs b/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
index e19597df..2275256f 100644
--- a/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
+++ b/AerospikeClientProxy/Proxy/AerospikeClientProxy.cs
@@ -103,13 +103,13 @@ public class AerospikeClientProxy : IDisposable, IAerospikeClient
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- public BatchPolicy tranVerifyPolicyDefault { get; set; }
+ public TxnVerifyPolicy txnVerifyPolicyDefault { get; set; }
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- public BatchPolicy tranRollPolicyDefault { get; set; }
+ public TxnRollPolicy txnRollPolicyDefault { get; set; }
///
/// Default info policy that is used when info command policy is null.
@@ -161,8 +161,8 @@ public AerospikeClientProxy(ClientPolicy policy, params Host[] hosts)
this.batchWritePolicyDefault = policy.batchWritePolicyDefault;
this.batchDeletePolicyDefault = policy.batchDeletePolicyDefault;
this.batchUDFPolicyDefault = policy.batchUDFPolicyDefault;
- this.tranVerifyPolicyDefault = policy.tranVerifyPolicyDefault;
- this.tranRollPolicyDefault = policy.tranRollPolicyDefault;
+ this.txnVerifyPolicyDefault = policy.txnVerifyPolicyDefault;
+ this.txnRollPolicyDefault = policy.txnRollPolicyDefault;
this.infoPolicyDefault = policy.infoPolicyDefault;
this.operatePolicyReadDefault = new WritePolicy(this.readPolicyDefault);
@@ -320,20 +320,20 @@ public BatchUDFPolicy BatchUDFPolicyDefault
///
/// Default multi-record transactions (MRT) policy when verifying record versions in a batch on a commit.
///
- public BatchPolicy TranVerifyPolicyDefault
+ public TxnVerifyPolicy TxnVerifyPolicyDefault
{
- get { return tranVerifyPolicyDefault; }
- set { tranVerifyPolicyDefault = value; }
+ get { return txnVerifyPolicyDefault; }
+ set { txnVerifyPolicyDefault = value; }
}
///
/// Default multi-record transactions (MRT) policy when rolling the transaction records forward (commit)
/// or back(abort) in a batch.
///
- public BatchPolicy TranRollPolicyDefault
+ public TxnRollPolicy TxnRollPolicyDefault
{
- get { return tranRollPolicyDefault; }
- set { tranRollPolicyDefault = value; }
+ get { return txnRollPolicyDefault; }
+ set { txnRollPolicyDefault = value; }
}
///
@@ -469,9 +469,9 @@ private string GetVersion()
///
///
/// multi-record transaction
- public void Commit(Tran tran)
+ public CommitStatus.CommitStatusType Commit(Txn tran)
{
-
+ return CommitStatus.CommitStatusType.OK;
}
///
@@ -481,9 +481,9 @@ public void Commit(Tran tran)
///
///
/// multi-record transaction
- public void Abort(Tran tran)
+ public AbortStatus.AbortStatusType Abort(Txn tran)
{
-
+ return AbortStatus.AbortStatusType.OK;
}
//-------------------------------------------------------
diff --git a/AerospikeClientProxy/Proxy/AsyncClientProxy.cs b/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
index b9990631..b40b4cc5 100644
--- a/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
+++ b/AerospikeClientProxy/Proxy/AsyncClientProxy.cs
@@ -127,10 +127,9 @@ public AsyncClientProxy(AsyncClientPolicy policy, params Host[] hosts)
///
///
/// where to send results
- /// multi-record transaction
- public void Commit(CommitListener listener, Tran tran)
+ /// multi-record transaction
+ public void Commit(CommitListener listener, Txn txn)
{
-
}
///
@@ -143,10 +142,9 @@ public void Commit(CommitListener listener, Tran tran)
///
///
/// where to send results
- /// multi-record transaction
- public void Abort(AbortListener listener, Tran tran)
+ /// multi-record transaction
+ public void Abort(AbortListener listener, Txn txn)
{
-
}
//-------------------------------------------------------
diff --git a/AerospikeTest/Args.cs b/AerospikeTest/Args.cs
index 40c4c28f..e534f3e2 100644
--- a/AerospikeTest/Args.cs
+++ b/AerospikeTest/Args.cs
@@ -43,6 +43,7 @@ public class Args
public string clusterName;
public string ns;
public string set;
+ public bool useServicesAlternate;
public string tlsName;
public string proxyTlsName;
public TlsPolicy tlsPolicy;
@@ -70,6 +71,7 @@ public Args()
ns = section.GetSection("Namespace").Value;
set = section.GetSection("Set").Value;
authMode = (AuthMode)Enum.Parse(typeof(AuthMode), section.GetSection("AuthMode").Value, true);
+ useServicesAlternate = bool.Parse(section.GetSection("UseServicesAlternate").Value);
bool tlsEnable = bool.Parse(section.GetSection("TlsEnable").Value);
@@ -130,6 +132,7 @@ private void ConnectSync()
policy.tlsPolicy = tlsPolicy;
policy.authMode = authMode;
policy.timeout = timeout;
+ policy.useServicesAlternate = useServicesAlternate;
if (user != null && user.Length > 0)
{
@@ -187,6 +190,11 @@ private void ConnectProxy()
proxyAsyncPolicy.maxConnsPerNode = 100;
proxyPolicy.timeout = timeout;
proxyAsyncPolicy.timeout = timeout;
+ policy.useServicesAlternate = useServicesAlternate;
+ proxyPolicy.useServicesAlternate = useServicesAlternate;
+ asyncPolicy.useServicesAlternate = useServicesAlternate;
+ proxyAsyncPolicy.useServicesAlternate = useServicesAlternate;
+
if (user != null && user.Length > 0)
{
@@ -287,6 +295,7 @@ private void ConnectAsync()
policy.authMode = authMode;
policy.asyncMaxCommands = 300;
policy.timeout = timeout;
+ policy.useServicesAlternate = useServicesAlternate;
if (user != null && user.Length > 0)
{
diff --git a/AerospikeTest/Async/TestAsyncTxn.cs b/AerospikeTest/Async/TestAsyncTxn.cs
new file mode 100644
index 00000000..da53dadc
--- /dev/null
+++ b/AerospikeTest/Async/TestAsyncTxn.cs
@@ -0,0 +1,492 @@
+/*
+ * Copyright 2012-2018 Aerospike, Inc.
+ *
+ * Portions may be licensed to Aerospike, Inc. under one or more contributor
+ * license agreements.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Aerospike.Client;
+using System.Reflection;
+using System.Text;
+
+namespace Aerospike.Test
+{
+ [TestClass]
+ public class TestAsyncTxn : TestAsync
+ {
+ private static readonly string binName = "bin";
+
+ [ClassInitialize()]
+ public static void Prepare(TestContext testContext)
+ {
+ if (!args.testProxy || (args.testProxy && nativeClient != null))
+ {
+ Assembly assembly = Assembly.GetExecutingAssembly();
+ RegisterTask task = nativeClient.Register(null, assembly, "Aerospike.Test.LuaResources.record_example.lua", "record_example.lua", Language.LUA);
+ task.Wait();
+ }
+ }
+
+ [TestMethod]
+ public void AsyncTxnWrite()
+ {
+ Key key = new(args.ns, args.set, "asyncTxnWrite");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Put(wp, key, new Bin(binName, "val2"));
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val2");
+ }
+
+ [TestMethod]
+ public void AsyncTxnWriteTwice()
+ {
+ Key key = new(args.ns, args.set, "mrtkey2");
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Put(wp, key, new Bin(binName, "val1"));
+ client.Put(wp, key, new Bin(binName, "val2"));
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val2");
+ }
+
+ [TestMethod]
+ public void AsyncTxnWriteConflict()
+ {
+ Key key = new(args.ns, args.set, "mrtkey21");
+
+ Txn txn1 = new();
+ Txn txn2 = new();
+
+ WritePolicy wp1 = client.WritePolicyDefault;
+ WritePolicy wp2 = client.WritePolicyDefault;
+ wp1.Txn = txn1;
+ wp2.Txn = txn2;
+
+ client.Put(wp1, key, new Bin(binName, "val1"));
+
+ try
+ {
+ client.Put(wp2, key, new Bin(binName, "val2"));
+ }
+ catch (AerospikeException ae)
+ {
+ if (ae.Result != ResultCode.MRT_BLOCKED)
+ {
+ throw ae;
+ }
+ }
+
+ client.Commit(txn1);
+ client.Commit(txn2);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnWriteBlock()
+ {
+ Key key = new(args.ns, args.set, "mrtkey3");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Put(wp, key, new Bin(binName, "val2"));
+
+ try
+ {
+ // This write should be blocked.
+ client.Put(null, key, new Bin(binName, "val3"));
+ throw new AerospikeException("Unexpected success");
+ }
+ catch (AerospikeException e)
+ {
+ if (e.Result != ResultCode.MRT_BLOCKED)
+ {
+ throw e;
+ }
+ }
+
+ client.Commit(txn);
+ }
+
+ [TestMethod]
+ public void AsyncTxnWriteRead()
+ {
+ Key key = new(args.ns, args.set, "mrtkey4");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Put(wp, key, new Bin(binName, "val2"));
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+
+ client.Commit(txn);
+
+ record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val2");
+ }
+
+ [TestMethod]
+ public void AsyncTxnWriteAbort()
+ {
+ Key key = new(args.ns, args.set, "mrtkey5");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Put(wp, key, new Bin(binName, "val2"));
+
+ Policy p = client.ReadPolicyDefault;
+ p.Txn = txn;
+ Record record = client.Get(p, key);
+ AssertBinEqual(key, record, binName, "val2");
+
+ client.Abort(txn);
+
+ record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnDelete()
+ {
+ Key key = new(args.ns, args.set, "mrtkey6");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ wp.durableDelete = true;
+ client.Delete(wp, key);
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ Assert.IsNull(record);
+ }
+
+ [TestMethod]
+ public void AsyncTxnDeleteAbort()
+ {
+ Key key = new(args.ns, args.set, "mrtkey7");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ wp.durableDelete = true;
+ client.Delete(wp, key);
+
+ client.Abort(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnDeleteTwice()
+ {
+ Key key = new(args.ns, args.set, "mrtkey8");
+
+ Txn txn = new();
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ wp.durableDelete = true;
+ client.Delete(wp, key);
+ client.Delete(wp, key);
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ Assert.IsNull(record);
+ }
+
+ [TestMethod]
+ public void AsyncTxnTouch()
+ {
+ Key key = new(args.ns, args.set, "mrtkey9");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Touch(wp, key);
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnTouchAbort()
+ {
+ Key key = new(args.ns, args.set, "mrtkey10");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Touch(wp, key);
+
+ client.Abort(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnOperateWrite()
+ {
+ Key key = new(args.ns, args.set, "mrtkey11");
+
+ client.Put(null, key, new Bin(binName, "val1"), new Bin("bin2", "bal1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ Record record = client.Operate(wp, key,
+ Operation.Put(new Bin(binName, "val2")),
+ Operation.Get("bin2")
+ );
+ AssertBinEqual(key, record, "bin2", "bal1");
+
+ client.Commit(txn);
+
+ record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val2");
+ }
+
+ [TestMethod]
+ public void AsyncTxnOperateWriteAbort()
+ {
+ Key key = new(args.ns, args.set, "mrtkey12");
+
+ client.Put(null, key, new Bin(binName, "val1"), new Bin("bin2", "bal1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ Record record = client.Operate(wp, key,
+ Operation.Put(new Bin(binName, "val2")),
+ Operation.Get("bin2")
+ );
+ AssertBinEqual(key, record, "bin2", "bal1");
+
+ client.Abort(txn);
+
+ record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnUDF()
+ {
+ Key key = new(args.ns, args.set, "mrtkey13");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Execute(wp, key, "record_example", "writeBin", Value.Get(binName), Value.Get("val2"));
+
+ client.Commit(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val2");
+ }
+
+ [TestMethod]
+ public void AsyncTxnUDFAbort()
+ {
+ Key key = new(args.ns, args.set, "mrtkey14");
+
+ client.Put(null, key, new Bin(binName, "val1"));
+
+ Txn txn = new();
+
+ WritePolicy wp = client.WritePolicyDefault;
+ wp.Txn = txn;
+ client.Execute(wp, key, "record_example", "writeBin", Value.Get(binName), Value.Get("val2"));
+
+ client.Abort(txn);
+
+ Record record = client.Get(null, key);
+ AssertBinEqual(key, record, binName, "val1");
+ }
+
+ [TestMethod]
+ public void AsyncTxnBatch()
+ {
+ Key[] keys = new Key[10];
+ Bin bin = new(binName, 1);
+
+ for (int i = 0; i < keys.Length; i++)
+ {
+ Key key = new(args.ns, args.set, i);
+ keys[i] = key;
+
+ client.Put(null, key, bin);
+ }
+
+ Record[] recs = client.Get(null, keys);
+ AssertBatchEqual(keys, recs, 1);
+
+ Txn txn = new();
+
+ bin = new(binName, 2);
+
+ BatchPolicy bp = BatchPolicy.WriteDefault();
+ bp.Txn = txn;
+
+ BatchResults bresults = client.Operate(bp, null, keys, Operation.Put(bin));
+
+ if (!bresults.status)
+ {
+ StringBuilder sb = new StringBuilder();
+ sb.Append("Batch failed:");
+ sb.Append(System.Environment.NewLine);
+
+ foreach (BatchRecord br in bresults.records)
+ {
+ if (br.resultCode == 0)
+ {
+ sb.Append("Record: " + br.record);
+ }
+ else
+ {
+ sb.Append("ResultCode: " + br.resultCode);
+ }
+ sb.Append(System.Environment.NewLine);
+ }
+
+ throw new AerospikeException(sb.ToString());
+ }
+
+ client.Commit(txn);
+
+ recs = client.Get(null, keys);
+ AssertBatchEqual(keys, recs, 2);
+ }
+
+ [TestMethod]
+ public void AsyncTxnBatchAbort()
+ {
+ var keys = new Key[10];
+ Bin bin = new(binName, 1);
+
+ for (int i = 0; i < keys.Length; i++)
+ {
+ Key key = new(args.ns, args.set, i);
+ keys[i] = key;
+
+ client.Put(null, key, bin);
+ }
+
+ Record[] recs = client.Get(null, keys);
+ AssertBatchEqual(keys, recs, 1);
+
+ Txn txn = new();
+
+ bin = new Bin(binName, 2);
+
+ BatchPolicy bp = BatchPolicy.WriteDefault();
+ bp.Txn = txn;
+
+ BatchResults bresults = client.Operate(bp, null, keys, Operation.Put(bin));
+
+ if (!bresults.status)
+ {
+ StringBuilder sb = new StringBuilder();
+ sb.Append("Batch failed:");
+ sb.Append(System.Environment.NewLine);
+
+ foreach (BatchRecord br in bresults.records)
+ {
+ if (br.resultCode == 0)
+ {
+ sb.Append("Record: " + br.record);
+ }
+ else
+ {
+ sb.Append("ResultCode: " + br.resultCode);
+ }
+ sb.Append(System.Environment.NewLine);
+ }
+
+ throw new AerospikeException(sb.ToString());
+ }
+
+ client.Abort(txn);
+
+ recs = client.Get(null, keys);
+ AssertBatchEqual(keys, recs, 1);
+ }
+
+ private void AsyncAssertBatchEqual(Key[] keys, Record[] recs, int expected)
+ {
+ for (int i = 0; i < keys.Length; i++)
+ {
+ Key key = keys[i];
+ Record rec = recs[i];
+
+ Assert.IsNotNull(rec);
+
+ int received = rec.GetInt(binName);
+ Assert.AreEqual(expected, received);
+ }
+ }
+ }
+}
diff --git a/AerospikeTest/Sync/Basic/TestTran.cs b/AerospikeTest/Sync/Basic/TestTxn.cs
similarity index 74%
rename from AerospikeTest/Sync/Basic/TestTran.cs
rename to AerospikeTest/Sync/Basic/TestTxn.cs
index ab30ae82..3a3784fa 100644
--- a/AerospikeTest/Sync/Basic/TestTran.cs
+++ b/AerospikeTest/Sync/Basic/TestTxn.cs
@@ -22,7 +22,7 @@
namespace Aerospike.Test
{
[TestClass]
- public class TestTran : TestSync
+ public class TestTxn : TestSync
{
private static readonly string binName = "bin";
@@ -38,54 +38,54 @@ public static void Prepare(TestContext testContext)
}
[TestMethod]
- public void TranWrite()
+ public void TxnWrite()
{
- Key key = new Key(args.ns, args.set, "mrtkey1");
+ Key key = new(args.ns, args.set, "mrtkey1");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Put(wp, key, new Bin(binName, "val2"));
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val2");
}
[TestMethod]
- public void TranWriteTwice()
+ public void TxnWriteTwice()
{
- Key key = new Key(args.ns, args.set, "mrtkey2");
+ Key key = new(args.ns, args.set, "mrtkey2");
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Put(wp, key, new Bin(binName, "val1"));
client.Put(wp, key, new Bin(binName, "val2"));
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val2");
}
[TestMethod]
- public void tranWriteConflict()
+ public void TxnWriteConflict()
{
- Key key = new Key(args.ns, args.set, "mrtkey21");
+ Key key = new(args.ns, args.set, "mrtkey21");
- Tran tran1 = new Tran();
- Tran tran2 = new Tran();
+ Txn txn1 = new();
+ Txn txn2 = new();
WritePolicy wp1 = client.WritePolicyDefault;
WritePolicy wp2 = client.WritePolicyDefault;
- wp1.Tran = tran1;
- wp2.Tran = tran2;
+ wp1.Txn = txn1;
+ wp2.Txn = txn2;
client.Put(wp1, key, new Bin(binName, "val1"));
@@ -101,24 +101,24 @@ public void tranWriteConflict()
}
}
- client.Commit(tran1);
- client.Commit(tran2);
+ client.Commit(txn1);
+ client.Commit(txn2);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranWriteBlock()
+ public void TxnWriteBlock()
{
- Key key = new Key(args.ns, args.set, "mrtkey3");
+ Key key = new(args.ns, args.set, "mrtkey3");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Put(wp, key, new Bin(binName, "val2"));
try
@@ -135,247 +135,247 @@ public void TranWriteBlock()
}
}
- client.Commit(tran);
+ client.Commit(txn);
}
[TestMethod]
- public void TranWriteRead()
+ public void TxnWriteRead()
{
- Key key = new Key(args.ns, args.set, "mrtkey4");
+ Key key = new(args.ns, args.set, "mrtkey4");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Put(wp, key, new Bin(binName, "val2"));
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
- client.Commit(tran);
+ client.Commit(txn);
record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val2");
}
[TestMethod]
- public void TranWriteAbort()
+ public void TxnWriteAbort()
{
- Key key = new Key(args.ns, args.set, "mrtkey5");
+ Key key = new(args.ns, args.set, "mrtkey5");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Put(wp, key, new Bin(binName, "val2"));
Policy p = client.ReadPolicyDefault;
- p.Tran = tran;
+ p.Txn = txn;
Record record = client.Get(p, key);
AssertBinEqual(key, record, binName, "val2");
- client.Abort(tran);
+ client.Abort(txn);
record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranDelete()
+ public void TxnDelete()
{
- Key key = new Key(args.ns, args.set, "mrtkey6");
+ Key key = new(args.ns, args.set, "mrtkey6");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
wp.durableDelete = true;
client.Delete(wp, key);
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
Assert.IsNull(record);
}
[TestMethod]
- public void TranDeleteAbort()
+ public void TxnDeleteAbort()
{
- Key key = new Key(args.ns, args.set, "mrtkey7");
+ Key key = new(args.ns, args.set, "mrtkey7");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
wp.durableDelete = true;
client.Delete(wp, key);
- client.Abort(tran);
+ client.Abort(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranDeleteTwice()
+ public void TxnDeleteTwice()
{
- Key key = new Key(args.ns, args.set, "mrtkey8");
+ Key key = new(args.ns, args.set, "mrtkey8");
- Tran tran = new Tran();
+ Txn txn = new();
client.Put(null, key, new Bin(binName, "val1"));
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
wp.durableDelete = true;
client.Delete(wp, key);
client.Delete(wp, key);
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
Assert.IsNull(record);
}
[TestMethod]
- public void TranTouch()
+ public void TxnTouch()
{
- Key key = new Key(args.ns, args.set, "mrtkey9");
+ Key key = new(args.ns, args.set, "mrtkey9");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Touch(wp, key);
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranTouchAbort()
+ public void TxnTouchAbort()
{
- Key key = new Key(args.ns, args.set, "mrtkey10");
+ Key key = new(args.ns, args.set, "mrtkey10");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Touch(wp, key);
- client.Abort(tran);
+ client.Abort(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranOperateWrite()
+ public void TxnOperateWrite()
{
- Key key = new Key(args.ns, args.set, "mrtkey11");
+ Key key = new(args.ns, args.set, "mrtkey11");
client.Put(null, key, new Bin(binName, "val1"), new Bin("bin2", "bal1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
Record record = client.Operate(wp, key,
Operation.Put(new Bin(binName, "val2")),
Operation.Get("bin2")
);
AssertBinEqual(key, record, "bin2", "bal1");
- client.Commit(tran);
+ client.Commit(txn);
record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val2");
}
[TestMethod]
- public void TranOperateWriteAbort()
+ public void TxnOperateWriteAbort()
{
- Key key = new Key(args.ns, args.set, "mrtkey12");
+ Key key = new(args.ns, args.set, "mrtkey12");
client.Put(null, key, new Bin(binName, "val1"), new Bin("bin2", "bal1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
Record record = client.Operate(wp, key,
Operation.Put(new Bin(binName, "val2")),
Operation.Get("bin2")
);
AssertBinEqual(key, record, "bin2", "bal1");
- client.Abort(tran);
+ client.Abort(txn);
record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranUDF()
+ public void TxnUDF()
{
- Key key = new Key(args.ns, args.set, "mrtkey13");
+ Key key = new(args.ns, args.set, "mrtkey13");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Execute(wp, key, "record_example", "writeBin", Value.Get(binName), Value.Get("val2"));
- client.Commit(tran);
+ client.Commit(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val2");
}
[TestMethod]
- public void TranUDFAbort()
+ public void TxnUDFAbort()
{
- Key key = new Key(args.ns, args.set, "mrtkey14");
+ Key key = new(args.ns, args.set, "mrtkey14");
client.Put(null, key, new Bin(binName, "val1"));
- Tran tran = new Tran();
+ Txn txn = new();
WritePolicy wp = client.WritePolicyDefault;
- wp.Tran = tran;
+ wp.Txn = txn;
client.Execute(wp, key, "record_example", "writeBin", Value.Get(binName), Value.Get("val2"));
- client.Abort(tran);
+ client.Abort(txn);
Record record = client.Get(null, key);
AssertBinEqual(key, record, binName, "val1");
}
[TestMethod]
- public void TranBatch()
+ public void TxnBatch()
{
Key[] keys = new Key[10];
- Bin bin = new Bin(binName, 1);
+ Bin bin = new(binName, 1);
for (int i = 0; i < keys.Length; i++)
{
- Key key = new Key(args.ns, args.set, i);
+ Key key = new(args.ns, args.set, i);
keys[i] = key;
client.Put(null, key, bin);
@@ -384,12 +384,12 @@ public void TranBatch()
Record[] recs = client.Get(null, keys);
AssertBatchEqual(keys, recs, 1);
- Tran tran = new Tran();
+ Txn txn = new();
- bin = new Bin(binName, 2);
+ bin = new(binName, 2);
BatchPolicy bp = BatchPolicy.WriteDefault();
- bp.Tran = tran;
+ bp.Txn = txn;
BatchResults bresults = client.Operate(bp, null, keys, Operation.Put(bin));
@@ -415,21 +415,21 @@ public void TranBatch()
throw new AerospikeException(sb.ToString());
}
- client.Commit(tran);
+ client.Commit(txn);
recs = client.Get(null, keys);
AssertBatchEqual(keys, recs, 2);
}
[TestMethod]
- public void TranBatchAbort()
+ public void TxnBatchAbort()
{
- Key[] keys = new Key[10];
- Bin bin = new Bin(binName, 1);
+ var keys = new Key[10];
+ Bin bin = new(binName, 1);
for (int i = 0; i < keys.Length; i++)
{
- Key key = new Key(args.ns, args.set, i);
+ Key key = new(args.ns, args.set, i);
keys[i] = key;
client.Put(null, key, bin);
@@ -438,12 +438,12 @@ public void TranBatchAbort()
Record[] recs = client.Get(null, keys);
AssertBatchEqual(keys, recs, 1);
- Tran tran = new Tran();
+ Txn txn = new();
bin = new Bin(binName, 2);
BatchPolicy bp = BatchPolicy.WriteDefault();
- bp.Tran = tran;
+ bp.Txn = txn;
BatchResults bresults = client.Operate(bp, null, keys, Operation.Put(bin));
@@ -469,7 +469,7 @@ public void TranBatchAbort()
throw new AerospikeException(sb.ToString());
}
- client.Abort(tran);
+ client.Abort(txn);
recs = client.Get(null, keys);
AssertBatchEqual(keys, recs, 1);
diff --git a/AerospikeTest/settings.json b/AerospikeTest/settings.json
index 1f22cf00..7ca33c77 100644
--- a/AerospikeTest/settings.json
+++ b/AerospikeTest/settings.json
@@ -1,7 +1,7 @@
{
"Host": "localhost",
- "Port": 3000,
- "ProxyHost": "localhost",
+ "Port": 3100,
+ "ProxyHost": "localhost",
"ProxyPort": 4000,
"TestProxy": false,
"ClusterName": "",
@@ -10,13 +10,14 @@
"User": "",
"Password": "",
"Timeout": 25000,
+ "UseServicesAlternate": true,
"TlsEnable": false,
"TlsName": "",
"TlsProtocols": "",
"TlsRevoke": "",
"TlsClientCertFile": "",
"TlsLoginOnly": false,
- "ProxyTlsEnable": false,
+ "ProxyTlsEnable": true,
"ProxyTlsName": "",
"ProxyTlsProtocols": "",
"ProxyTlsRevoke": "",
From 69812d933f2a5f5fd885c377abf9c83b5ee88efb Mon Sep 17 00:00:00 2001
From: Shannon Klaus
Date: Wed, 28 Aug 2024 15:05:49 -0600
Subject: [PATCH 04/41] Go through reference again, fixing some bugs
---
AerospikeClient/Async/AsyncBatch.cs | 4 +-
AerospikeClient/Async/AsyncClient.cs | 27 +-
AerospikeClient/Async/AsyncClientPolicy.cs | 2 +-
AerospikeClient/Async/AsyncCommand.cs | 10 +-
AerospikeClient/Async/AsyncConnector.cs | 2 +-
AerospikeClient/Async/AsyncDelete.cs | 1 -
AerospikeClient/Async/AsyncExists.cs | 1 +
AerospikeClient/Async/AsyncOperateWrite.cs | 1 +
.../Async/AsyncQueryPartitionExecutor.cs | 2 +-
AerospikeClient/Async/AsyncReadBase.cs | 2 +-
AerospikeClient/Async/AsyncReadHeader.cs | 2 +-
.../Async/AsyncScanPartitionExecutor.cs | 2 +-
AerospikeClient/Async/AsyncSingleCommand.cs | 10 +-
AerospikeClient/Async/AsyncTxnAddKeys.cs | 2 +-
.../Async/AsyncTxnMarkRollForward.cs | 2 +
AerospikeClient/Async/AsyncTxnMonitor.cs | 24 +-
AerospikeClient/Async/AsyncTxnRoll.cs | 8 +-
AerospikeClient/Async/AsyncWrite.cs | 1 -
AerospikeClient/Async/AsyncWriteBase.cs | 2 +-
AerospikeClient/Async/IAsyncClient.cs | 28 +-
.../BatchOperateListListenerAdapter.cs | 2 +-
AerospikeClient/Cluster/Cluster.cs | 2 +-
AerospikeClient/Cluster/ClusterStats.cs | 2 +-
AerospikeClient/Command/Batch.cs | 4 +-
AerospikeClient/Command/BatchAttr.cs | 4 +-
AerospikeClient/Command/BatchExecutor.cs | 2 +-
AerospikeClient/Command/ByteUtil.cs | 2 +-
AerospikeClient/Command/Command.cs | 366 +++++++-----
AerospikeClient/Command/DeleteCommand.cs | 2 +-
AerospikeClient/Command/ExecuteCommand.cs | 5 -
AerospikeClient/Command/OperateArgs.cs | 12 -
AerospikeClient/Command/OperateCommandRead.cs | 2 +-
.../Command/OperateCommandWrite.cs | 4 -
AerospikeClient/Command/ReadCommand.cs | 10 +-
AerospikeClient/Command/ScanExecutor.cs | 2 +-
AerospikeClient/Command/SyncCommand.cs | 10 +-
AerospikeClient/Command/SyncReadCommand.cs | 2 +-
AerospikeClient/Command/SyncWriteCommand.cs | 2 +-
AerospikeClient/Command/TouchCommand.cs | 1 -
AerospikeClient/Command/TxnAddKeys.cs | 2 +-
AerospikeClient/Command/TxnClose.cs | 8 +-
AerospikeClient/Command/TxnMarkRollForward.cs | 8 +-
AerospikeClient/Command/TxnMonitor.cs | 42 +-
AerospikeClient/Command/TxnRoll.cs | 34 +-
AerospikeClient/Command/WriteCommand.cs | 1 -
AerospikeClient/Listener/AbortListener.cs | 8 +-
AerospikeClient/Listener/CommitListener.cs | 2 +-
AerospikeClient/Main/AerospikeClient.cs | 17 +-
AerospikeClient/Main/AerospikeException.cs | 47 +-
AerospikeClient/Main/CommitError.cs | 6 +-
AerospikeClient/Main/CommitStatus.cs | 2 +-
AerospikeClient/Main/IAerospikeClient.cs | 2 +-
AerospikeClient/Main/Key.cs | 1 +
AerospikeClient/Main/ResultCode.cs | 8 +-
AerospikeClient/Metrics/LatencyBuckets.cs | 2 +-
AerospikeClient/Policy/BatchDeletePolicy.cs | 4 +-
AerospikeClient/Policy/BatchPolicy.cs | 4 +-
AerospikeClient/Policy/BatchUDFPolicy.cs | 4 +-
AerospikeClient/Policy/BatchWritePolicy.cs | 4 +-
AerospikeClient/Policy/ClientPolicy.cs | 4 +-
AerospikeClient/Policy/CommitLevel.cs | 2 +-
AerospikeClient/Policy/Policy.cs | 39 +-
AerospikeClient/Policy/QueryPolicy.cs | 4 +
AerospikeClient/Policy/ScanPolicy.cs | 4 +
AerospikeClient/Policy/WritePolicy.cs | 4 +-
AerospikeClient/Query/QueryExecutor.cs | 2 +-
.../Query/QueryListenerExecutor.cs | 2 +-
.../Query/QueryPartitionExecutor.cs | 2 +-
AerospikeClient/Query/RecordSet.cs | 2 +-
AerospikeClient/Query/ResultSet.cs | 2 +-
.../Proxy/AerospikeClientProxy.cs | 22 +-
.../Proxy/AsyncClientProxy.cs | 12 +-
AerospikeTest/Async/TestAsync.cs | 35 ++
AerospikeTest/Async/TestAsyncTxn.cs | 534 ++++++++++++------
AerospikeTest/settings.json | 4 +-
75 files changed, 834 insertions(+), 615 deletions(-)
diff --git a/AerospikeClient/Async/AsyncBatch.cs b/AerospikeClient/Async/AsyncBatch.cs
index 83586343..1d1a808e 100644
--- a/AerospikeClient/Async/AsyncBatch.cs
+++ b/AerospikeClient/Async/AsyncBatch.cs
@@ -1887,11 +1887,13 @@ public abstract class AsyncBatchExecutor : IBatchStatus
private readonly bool hasResultCode;
private bool error;
public AsyncBatchCommand[] commands;
+ public AsyncCluster cluster;
public AsyncBatchExecutor(AsyncCluster cluster, bool hasResultCode)
{
this.hasResultCode = hasResultCode;
- cluster.AddCommand();
+ this.cluster = cluster;
+ cluster.AddCommandCount();
}
public void Execute()
diff --git a/AerospikeClient/Async/AsyncClient.cs b/AerospikeClient/Async/AsyncClient.cs
index da41e32c..1922f3d7 100644
--- a/AerospikeClient/Async/AsyncClient.cs
+++ b/AerospikeClient/Async/AsyncClient.cs
@@ -135,7 +135,7 @@ public AsyncClient(AsyncClientPolicy policy, params Host[] hosts)
///
/// Asynchronously attempt to commit the given multi-record transaction. First, the expected
/// record versions are sent to the server nodes for verification.If all nodes return success,
- /// the transaction is committed.Otherwise, the transaction is aborted.
+ /// the transaction is committed. Otherwise, the transaction is aborted.
///
/// This method registers the command with an event loop and returns.
/// The event loop thread will process the command and send the results to the listener.
@@ -188,7 +188,7 @@ public void Abort(AbortListener listener, Txn txn)
/// Asynchronously write record bin(s).
/// Create listener, call asynchronous put and return task monitor.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
///
@@ -203,13 +203,13 @@ public Task Put(WritePolicy policy, CancellationToken token, Key key, params Bin
Put(policy, listener, key, bins);
return listener.Task;
}
-
+
///
/// Asynchronously write record bin(s).
/// Schedules the put command with a channel selector and return.
/// Another thread will process the command and send the results to the listener.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
///
@@ -236,7 +236,7 @@ public void Put(WritePolicy policy, WriteListener listener, Key key, params Bin[
/// Asynchronously append bin string values to existing record bin values.
/// Create listener, call asynchronous append and return task monitor.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call only works for string values.
///
@@ -258,7 +258,7 @@ public Task Append(WritePolicy policy, CancellationToken token, Key key, params
/// Schedule the append command with a channel selector and return.
/// Another thread will process the command and send the results to the listener.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call only works for string values.
///
@@ -282,7 +282,7 @@ public void Append(WritePolicy policy, WriteListener listener, Key key, params B
/// Asynchronously prepend bin string values to existing record bin values.
/// Create listener, call asynchronous prepend and return task monitor.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call works only for string values.
///
@@ -304,7 +304,7 @@ public Task Prepend(WritePolicy policy, CancellationToken token, Key key, params
/// Schedule the prepend command with a channel selector and return.
/// Another thread will process the command and send the results to the listener.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
/// This call works only for string values.
///
@@ -332,7 +332,7 @@ public void Prepend(WritePolicy policy, WriteListener listener, Key key, params
/// Asynchronously add integer/double bin values to existing record bin values.
/// Create listener, call asynchronous add and return task monitor.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
///
@@ -353,7 +353,7 @@ public Task Add(WritePolicy policy, CancellationToken token, Key key, params Bin
/// Schedule the add command with a channel selector and return.
/// Another thread will process the command and send the results to the listener.
///
- /// The policy specifies the transaction timeout, record expiration and how the transaction is
+ /// The policy specifies the command timeout, record expiration and how the command is
/// handled when the record already exists.
///
///
@@ -1009,7 +1009,10 @@ public void Get(BatchPolicy policy, RecordSequenceListener listener, Key[] keys,
}
policy.Txn?.SetNamespace(keys);
- AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, binNames, null, Command.INFO1_READ, false);
+ int readAttr = (binNames == null || binNames.Length == 0)?
+ Command.INFO1_READ | Command.INFO1_GET_ALL : Command.INFO1_READ;
+
+ AsyncBatchGetSequenceExecutor executor = new(cluster, policy, listener, keys, binNames, null, readAttr, false);
executor.Execute();
}
@@ -1534,7 +1537,7 @@ public Task