├── .nuget
└── icon.png
├── .vscode
├── settings.json
├── tasks.json
└── launch.json
├── src
└── Knet.Kudu.Client
│ ├── Util
│ ├── CharUtil.cs
│ ├── ISystemClock.cs
│ ├── SystemClock.cs
│ ├── Murmur2.cs
│ ├── HybridTimeUtil.cs
│ ├── FastHash.cs
│ ├── EndpointParser.cs
│ └── EpochTime.cs
│ ├── ComparisonOp.cs
│ ├── MasterLeaderInfo.cs
│ ├── RangeSchema.cs
│ ├── ReplicaRole.cs
│ ├── Requests
│ ├── KuduMasterRpc.cs
│ ├── KuduTxnRpc.cs
│ ├── DeleteTableRequest.cs
│ ├── IsAlterTableDoneRequest.cs
│ ├── AbortTransactionRequest.cs
│ ├── GetTableLocationsRequest.cs
│ ├── IsCreateTableDoneRequest.cs
│ ├── BeginTransactionRequest.cs
│ ├── CommitTransactionRequest.cs
│ ├── GetTableStatisticsRequest.cs
│ ├── ListTabletServersRequest.cs
│ ├── GetTransactionStateRequest.cs
│ ├── KeepTransactionAliveRequest.cs
│ ├── CreateTableRequest.cs
│ ├── ListTablesRequest.cs
│ ├── KuduTabletRpc.cs
│ ├── ConnectToMasterRequest.cs
│ ├── GetTableSchemaRequest.cs
│ ├── KeepAliveRequest.cs
│ ├── AlterTableRequest.cs
│ ├── WriteRequest.cs
│ ├── SplitKeyRangeRequest.cs
│ └── KuduRpc.cs
│ ├── Scanner
│ └── PartitionKeyRange.cs
│ ├── AlterTableResponse.cs
│ ├── WriteResponse.cs
│ ├── Tablet
│ ├── KeyEncoder.netstandard.cs
│ ├── FindTabletResult.cs
│ ├── TableLocationEntry.cs
│ ├── KeyRange.cs
│ ├── RemoteTabletExtensions.cs
│ ├── KeyEncoder.sse.cs
│ └── RemoteTablet.cs
│ ├── Exceptions
│ ├── RpcRemoteException.cs
│ ├── FaultTolerantScannerExpiredException.cs
│ ├── InvalidAuthzTokenException.cs
│ ├── InvalidAuthnTokenException.cs
│ ├── RecoverableException.cs
│ ├── NonRecoverableException.cs
│ ├── KuduException.cs
│ ├── NoLeaderFoundException.cs
│ ├── KuduWriteException.cs
│ └── NonCoveredRangeException.cs
│ ├── KuduTransactionSerializationOptions.cs
│ ├── Protocol
│ ├── SidecarOffset.cs
│ ├── ParseStep.cs
│ └── KuduMessage.cs
│ ├── RangePartitionBound.cs
│ ├── HashBucketSchema.cs
│ ├── Assembly.cs
│ ├── Internal
│ ├── AvlTreeExtensions.cs
│ ├── ThreadSafeRandom.cs
│ ├── ArrayPoolBuffer.cs
│ ├── KuduTypeFlags.cs
│ ├── TaskCompletionSource.cs
│ ├── PeriodicTimer.cs
│ ├── SequenceReaderExtensions.cs
│ ├── AuthzTokenCache.cs
│ ├── SecurityUtil.cs
│ ├── KuduTypeValidation.cs
│ └── Netstandard2Extensions.cs
│ ├── SessionExceptionContext.cs
│ ├── CompressionType.cs
│ ├── TableInfo.cs
│ ├── EncodingType.cs
│ ├── TabletServerState.cs
│ ├── Connection
│ ├── IKuduConnectionFactory.cs
│ ├── HostAndPort.cs
│ ├── ServerInfo.cs
│ ├── RequestTracker.cs
│ └── ISecurityContext.cs
│ ├── EncryptionPolicy.cs
│ ├── KuduReplica.cs
│ ├── KuduTableStatistics.cs
│ ├── KuduType.cs
│ ├── Negotiate
│ └── SaslPlain.cs
│ ├── Mapper
│ ├── ResultSetMapper.cs
│ ├── DelegateCache.cs
│ └── ColumnNameMatcher.cs
│ ├── ReplicaSelection.cs
│ ├── PredicateType.cs
│ ├── KuduScannerBuilder.cs
│ ├── PartialRowOperation.cs
│ ├── Protos
│ └── kudu
│ │ ├── util
│ │ ├── hash.proto
│ │ ├── compression
│ │ │ └── compression.proto
│ │ ├── block_bloom_filter.proto
│ │ └── pb_util.proto
│ │ └── consensus
│ │ ├── opid.proto
│ │ └── replica_management.proto
│ ├── MasterManager.cs
│ ├── PartitionSchema.cs
│ ├── KuduClientOptions.cs
│ ├── HiveMetastoreConfig.cs
│ ├── KuduScannerExtensions.cs
│ ├── KuduSessionOptions.cs
│ ├── KuduBloomFilterBuilder.cs
│ ├── Logging
│ └── LoggerHelperExtensions.cs
│ ├── KuduOperation.cs
│ ├── TabletServerInfo.cs
│ ├── RowOperation.cs
│ ├── ResourceMetrics.cs
│ ├── IKuduSession.cs
│ ├── ColumnTypeAttributes.cs
│ ├── Knet.Kudu.Client.csproj
│ ├── ReadMode.cs
│ ├── KuduPartitioner.cs
│ └── ExternalConsistencyMode.cs
├── examples
└── InsertLoadgen
│ ├── InsertLoadgen.csproj
│ └── Program.cs
├── test
├── Knet.Kudu.Client.Tests
│ ├── MurmurHashTests.cs
│ ├── Knet.Kudu.Client.Tests.csproj
│ ├── FastHashTests.cs
│ ├── HostAndPortTests.cs
│ ├── DecimalUtilTests.cs
│ ├── EndpointParserTests.cs
│ ├── EpochTimeTests.cs
│ ├── KuduStatusTests.cs
│ ├── TableBuilderTests.cs
│ ├── PartitionTests.cs
│ └── RequestTrackerTests.cs
└── Knet.Kudu.Client.FunctionalTests
│ ├── Knet.Kudu.Client.FunctionalTests.csproj
│ ├── MiniCluster
│ ├── MiniKuduClusterTestAttribute.cs
│ └── MiniKuduClusterBuilder.cs
│ ├── DeleteTableTests.cs
│ ├── Util
│ └── TestExtensions.cs
│ ├── TimeoutTests.cs
│ ├── LeaderFailoverTests.cs
│ ├── HandleTooBusyTests.cs
│ ├── SessionTests.cs
│ ├── MasterFailoverTests.cs
│ ├── ClientTests.cs
│ └── MultipleLeaderFailoverTests.cs
├── .editorconfig
├── .github
└── workflows
│ └── ci.yml
└── README.md
/.nuget/icon.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/xqrzd/kudu-client-net/HEAD/.nuget/icon.png
--------------------------------------------------------------------------------
/.vscode/settings.json:
--------------------------------------------------------------------------------
1 | {
2 | "dotnet.defaultSolution": "kudu-client-net.sln",
3 | "files.exclude": {
4 | "**/bin": true,
5 | "**/obj": true
6 | }
7 | }
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Util/CharUtil.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Util;
2 |
3 | public static class CharUtil
4 | {
5 | public const int MinVarcharLength = 1;
6 | public const int MaxVarcharLength = 65535;
7 | }
8 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/ComparisonOp.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | ///
4 | /// The comparison operator of a predicate.
5 | ///
6 | public enum ComparisonOp
7 | {
8 | Greater,
9 | GreaterEqual,
10 | Equal,
11 | Less,
12 | LessEqual
13 | }
14 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/MasterLeaderInfo.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Connection;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | internal sealed record MasterLeaderInfo(
6 | string Location,
7 | string ClusterId,
8 | ServerInfo ServerInfo,
9 | HiveMetastoreConfig? HiveMetastoreConfig);
10 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/RangeSchema.cs:
--------------------------------------------------------------------------------
1 | using System.Collections.Generic;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | public class RangeSchema
6 | {
7 | public List ColumnIds { get; }
8 |
9 | public RangeSchema(List columnIds)
10 | {
11 | ColumnIds = columnIds;
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/ReplicaRole.cs:
--------------------------------------------------------------------------------
1 | using static Knet.Kudu.Client.Protobuf.Consensus.RaftPeerPB.Types;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | public enum ReplicaRole
6 | {
7 | Follower = Role.Follower,
8 | Leader = Role.Leader,
9 | Learner = Role.Learner,
10 | NonParticipant = Role.NonParticipant
11 | }
12 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Requests/KuduMasterRpc.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Protobuf.Master;
2 |
3 | namespace Knet.Kudu.Client.Requests;
4 |
5 | internal abstract class KuduMasterRpc : KuduRpc
6 | {
7 | public MasterErrorPB? Error { get; protected set; }
8 |
9 | public KuduMasterRpc()
10 | {
11 | ServiceName = MasterServiceName;
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Scanner/PartitionKeyRange.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Scanner;
2 |
3 | public readonly struct PartitionKeyRange
4 | {
5 | public byte[] Lower { get; }
6 |
7 | public byte[] Upper { get; }
8 |
9 | public PartitionKeyRange(byte[] lower, byte[] upper)
10 | {
11 | Lower = lower;
12 | Upper = upper;
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Requests/KuduTxnRpc.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Protobuf.Transactions;
2 |
3 | namespace Knet.Kudu.Client.Requests;
4 |
5 | internal abstract class KuduTxnRpc : KuduRpc
6 | {
7 | public TxnManagerErrorPB? Error { get; protected set; }
8 |
9 | public KuduTxnRpc()
10 | {
11 | ServiceName = TxnManagerServiceName;
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/AlterTableResponse.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | public class AlterTableResponse
4 | {
5 | public string TableId { get; }
6 |
7 | public uint SchemaVersion { get; }
8 |
9 | public AlterTableResponse(string tableId, uint schemaVersion)
10 | {
11 | TableId = tableId;
12 | SchemaVersion = schemaVersion;
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/WriteResponse.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | public class WriteResponse
4 | {
5 | ///
6 | /// The HybridTime-encoded write timestamp.
7 | ///
8 | public long WriteTimestampRaw { get; }
9 |
10 | public WriteResponse(long writeTimestampRaw)
11 | {
12 | WriteTimestampRaw = writeTimestampRaw;
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Util/ISystemClock.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Util;
2 |
3 | public interface ISystemClock
4 | {
5 | ///
6 | /// Retrieve the current milliseconds. This value should only
7 | /// be used to measure how much time has passed relative to
8 | /// another call to this property.
9 | ///
10 | long CurrentMilliseconds { get; }
11 | }
12 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Tablet/KeyEncoder.netstandard.cs:
--------------------------------------------------------------------------------
1 | #if !NETCOREAPP3_1_OR_GREATER
2 | using System;
3 |
4 | namespace Knet.Kudu.Client.Tablet;
5 |
6 | public static partial class KeyEncoder
7 | {
8 | private static int EncodeBinary(
9 | ReadOnlySpan source, Span destination)
10 | {
11 | return EncodeBinaryStandard(source, destination);
12 | }
13 | }
14 | #endif
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/RpcRemoteException.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Protobuf.Rpc;
2 |
3 | namespace Knet.Kudu.Client.Exceptions;
4 |
5 | public class RpcRemoteException : NonRecoverableException
6 | {
7 | public ErrorStatusPB ErrorPb { get; }
8 |
9 | public RpcRemoteException(KuduStatus status, ErrorStatusPB errorPb)
10 | : base(status)
11 | {
12 | ErrorPb = errorPb;
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/FaultTolerantScannerExpiredException.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Exceptions;
2 |
3 | ///
4 | /// A scanner expired exception only used for fault tolerant scanner.
5 | ///
6 | public class FaultTolerantScannerExpiredException : NonRecoverableException
7 | {
8 | public FaultTolerantScannerExpiredException(KuduStatus status)
9 | : base(status)
10 | {
11 | }
12 | }
13 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/KuduTransactionSerializationOptions.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | public record KuduTransactionSerializationOptions
4 | {
5 | ///
6 | /// Whether the created from these
7 | /// options will send keepalive messages to avoid automatic rollback
8 | /// of the underlying transaction.
9 | ///
10 | public bool EnableKeepalive { get; init; }
11 | }
12 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Protocol/SidecarOffset.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Protocol;
2 |
3 | public readonly struct SidecarOffset
4 | {
5 | public readonly int Start;
6 |
7 | public readonly int Length;
8 |
9 | public SidecarOffset(int start, int length)
10 | {
11 | Start = start;
12 | Length = length;
13 | }
14 |
15 | public override string ToString() =>
16 | $"Start: {Start}, Length: {Length}";
17 | }
18 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/RangePartitionBound.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | ///
4 | /// Specifies whether a range partition bound is inclusive or exclusive.
5 | ///
6 | public enum RangePartitionBound
7 | {
8 | ///
9 | /// An exclusive range partition bound.
10 | ///
11 | Exclusive,
12 | ///
13 | /// An inclusive range partition bound.
14 | ///
15 | Inclusive
16 | }
17 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Util/SystemClock.cs:
--------------------------------------------------------------------------------
1 | using System;
2 | using System.Diagnostics;
3 |
4 | namespace Knet.Kudu.Client.Util;
5 |
6 | public sealed class SystemClock : ISystemClock
7 | {
8 | #if NETCOREAPP3_1_OR_GREATER
9 | public long CurrentMilliseconds => Environment.TickCount64;
10 | #else
11 | private readonly Stopwatch _stopwatch = Stopwatch.StartNew();
12 |
13 | public long CurrentMilliseconds => _stopwatch.ElapsedMilliseconds;
14 | #endif
15 | }
16 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/HashBucketSchema.cs:
--------------------------------------------------------------------------------
1 | using System.Collections.Generic;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | public class HashBucketSchema
6 | {
7 | public List ColumnIds { get; }
8 |
9 | public int NumBuckets { get; }
10 |
11 | public uint Seed { get; }
12 |
13 | public HashBucketSchema(List columnIds, int numBuckets, uint seed)
14 | {
15 | ColumnIds = columnIds;
16 | NumBuckets = numBuckets;
17 | Seed = seed;
18 | }
19 | }
20 |
--------------------------------------------------------------------------------
/examples/InsertLoadgen/InsertLoadgen.csproj:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | Exe
5 | net7.0
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Assembly.cs:
--------------------------------------------------------------------------------
1 | using System.ComponentModel;
2 | using System.Runtime.CompilerServices;
3 |
4 | [assembly: InternalsVisibleTo("Knet.Kudu.Client.FunctionalTests")]
5 | [assembly: InternalsVisibleTo("Knet.Kudu.Client.Tests")]
6 |
7 | #if NET5_0_OR_GREATER
8 |
9 | [module: SkipLocalsInit]
10 |
11 | #else
12 |
13 | namespace System.Runtime.CompilerServices
14 | {
15 | [EditorBrowsable(EditorBrowsableState.Never)]
16 | internal static class IsExternalInit { }
17 | }
18 |
19 | #endif
20 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/InvalidAuthzTokenException.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Exceptions;
2 |
3 | ///
4 | /// Receiving this exception means the authorization token used to make a
5 | /// request is no longer valid and a new one is needed to make requests that
6 | /// access data.
7 | ///
8 | public class InvalidAuthzTokenException : RecoverableException
9 | {
10 | public InvalidAuthzTokenException(KuduStatus status)
11 | : base(status)
12 | {
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Internal/AvlTreeExtensions.cs:
--------------------------------------------------------------------------------
1 | using System;
2 | using Knet.Kudu.Client.Tablet;
3 |
4 | namespace Knet.Kudu.Client.Internal;
5 |
6 | internal static class AvlTreeExtensions
7 | {
8 | public static TableLocationEntry? FloorEntry(
9 | this AvlTree avlTree, ReadOnlySpan partitionKey)
10 | {
11 | avlTree.SearchLeftRight(
12 | partitionKey,
13 | out TableLocationEntry left,
14 | out _);
15 |
16 | return left;
17 | }
18 | }
19 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/SessionExceptionContext.cs:
--------------------------------------------------------------------------------
1 | using System;
2 | using System.Collections.Generic;
3 |
4 | namespace Knet.Kudu.Client;
5 |
6 | public sealed class SessionExceptionContext
7 | {
8 | public Exception Exception { get; }
9 |
10 | public IReadOnlyList Rows { get; }
11 |
12 | public SessionExceptionContext(
13 | Exception exception,
14 | IReadOnlyList rows)
15 | {
16 | Exception = exception;
17 | Rows = rows;
18 | }
19 | }
20 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/InvalidAuthnTokenException.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Exceptions;
2 |
3 | ///
4 | /// Receiving this exception means the current authentication token is no
5 | /// longer valid and a new one is needed to establish connections to the
6 | /// Kudu servers for sending RPCs.
7 | ///
8 | public class InvalidAuthnTokenException : RecoverableException
9 | {
10 | public InvalidAuthnTokenException(KuduStatus status)
11 | : base(status)
12 | {
13 | }
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/RecoverableException.cs:
--------------------------------------------------------------------------------
1 | using System;
2 |
3 | namespace Knet.Kudu.Client.Exceptions;
4 |
5 | ///
6 | /// An exception that's possible to retry.
7 | ///
8 | public class RecoverableException : KuduException
9 | {
10 | public RecoverableException(KuduStatus status)
11 | : base(status)
12 | {
13 | }
14 |
15 | public RecoverableException(KuduStatus status, Exception? innerException)
16 | : base(status, innerException)
17 | {
18 | }
19 | }
20 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/NonRecoverableException.cs:
--------------------------------------------------------------------------------
1 | using System;
2 |
3 | namespace Knet.Kudu.Client.Exceptions;
4 |
5 | ///
6 | /// An exception that cannot be retried.
7 | ///
8 | public class NonRecoverableException : KuduException
9 | {
10 | public NonRecoverableException(KuduStatus status)
11 | : base(status)
12 | {
13 | }
14 |
15 | public NonRecoverableException(KuduStatus status, Exception? innerException)
16 | : base(status, innerException)
17 | {
18 | }
19 | }
20 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/KuduException.cs:
--------------------------------------------------------------------------------
1 | using System;
2 |
3 | namespace Knet.Kudu.Client.Exceptions;
4 |
5 | public abstract class KuduException : Exception
6 | {
7 | public KuduStatus Status { get; }
8 |
9 | public KuduException(KuduStatus status)
10 | : base(status.Message)
11 | {
12 | Status = status;
13 | }
14 |
15 | public KuduException(KuduStatus status, Exception? innerException)
16 | : base(status.Message, innerException)
17 | {
18 | Status = status;
19 | }
20 | }
21 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/CompressionType.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | ///
4 | /// Supported compression for Kudu columns.
5 | /// See https://kudu.apache.org/docs/schema_design.html#compression
6 | ///
7 | public enum CompressionType
8 | {
9 | DefaultCompression = Protobuf.CompressionType.DefaultCompression,
10 | NoCompression = Protobuf.CompressionType.NoCompression,
11 | Snappy = Protobuf.CompressionType.Snappy,
12 | Lz4 = Protobuf.CompressionType.Lz4,
13 | Zlib = Protobuf.CompressionType.Zlib
14 | }
15 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Exceptions/NoLeaderFoundException.cs:
--------------------------------------------------------------------------------
1 | using System;
2 |
3 | namespace Knet.Kudu.Client.Exceptions;
4 |
5 | ///
6 | /// Indicates that the request failed because we couldn't find a leader.
7 | /// It is retried as long as the original call hasn't timed out.
8 | ///
9 | public class NoLeaderFoundException : RecoverableException
10 | {
11 | public NoLeaderFoundException(string message, Exception? innerException)
12 | : base(KuduStatus.NetworkError(message), innerException)
13 | {
14 | }
15 | }
16 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/TableInfo.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | public class TableInfo
4 | {
5 | ///
6 | /// The table name.
7 | ///
8 | public string TableName { get; }
9 |
10 | ///
11 | /// The table Id.
12 | ///
13 | public string TableId { get; }
14 |
15 | public TableInfo(string tableName, string tableId)
16 | {
17 | TableName = tableName;
18 | TableId = tableId;
19 | }
20 |
21 | public override string ToString() => TableName;
22 | }
23 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Internal/ThreadSafeRandom.cs:
--------------------------------------------------------------------------------
1 | using System;
2 | using System.Threading;
3 |
4 | namespace Knet.Kudu.Client.Internal;
5 |
6 | internal static class ThreadSafeRandom
7 | {
8 | #if NET6_0_OR_GREATER
9 | public static Random Instance => Random.Shared;
10 | #else
11 | private static int _seed = Environment.TickCount;
12 |
13 | private static readonly ThreadLocal _random =
14 | new(() => new Random(Interlocked.Increment(ref _seed)));
15 |
16 | public static Random Instance => _random.Value!;
17 | #endif
18 | }
19 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/EncodingType.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | ///
4 | /// Supported encoding for Kudu columns.
5 | /// See https://kudu.apache.org/docs/schema_design.html#encoding
6 | ///
7 | public enum EncodingType
8 | {
9 | AutoEncoding = Protobuf.EncodingType.AutoEncoding,
10 | PlainEncoding = Protobuf.EncodingType.PlainEncoding,
11 | PrefixEncoding = Protobuf.EncodingType.PrefixEncoding,
12 | Rle = Protobuf.EncodingType.Rle,
13 | DictEncoding = Protobuf.EncodingType.DictEncoding,
14 | BitShuffle = Protobuf.EncodingType.BitShuffle
15 | }
16 |
--------------------------------------------------------------------------------
/.vscode/tasks.json:
--------------------------------------------------------------------------------
1 | {
2 | "version": "2.0.0",
3 | "tasks": [
4 | {
5 | "label": "build",
6 | "command": "dotnet",
7 | "type": "process",
8 | "args": [
9 | "build",
10 | "${workspaceFolder}/kudu-client-net.sln",
11 | "/property:GenerateFullPaths=true",
12 | "/consoleloggerparameters:NoSummary"
13 | ],
14 | "problemMatcher": "$msCompile",
15 | "group": {
16 | "kind": "build",
17 | "isDefault": true
18 | }
19 | }
20 | ]
21 | }
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Internal/ArrayPoolBuffer.cs:
--------------------------------------------------------------------------------
1 | using System;
2 | using System.Buffers;
3 |
4 | namespace Knet.Kudu.Client.Internal;
5 |
6 | internal sealed class ArrayPoolBuffer : IDisposable
7 | {
8 | public T[] Buffer { get; private set; }
9 |
10 | public ArrayPoolBuffer(int minimumLength)
11 | {
12 | Buffer = ArrayPool.Shared.Rent(minimumLength);
13 | }
14 |
15 | public void Dispose()
16 | {
17 | var buffer = Buffer;
18 | if (buffer is not null)
19 | {
20 | Buffer = null!;
21 | ArrayPool.Shared.Return(buffer);
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/test/Knet.Kudu.Client.Tests/MurmurHashTests.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Internal;
2 | using Knet.Kudu.Client.Util;
3 | using Xunit;
4 |
5 | namespace Knet.Kudu.Client.Tests;
6 |
7 | public class MurmurHashTests
8 | {
9 | [Theory]
10 | [InlineData("ab", 0, 7115271465109541368UL)]
11 | [InlineData("abcdefg", 0, 2601573339036254301UL)]
12 | [InlineData("quick brown fox", 42, 3575930248840144026UL)]
13 | public void TestMurmur2Hash64(string data, ulong seed, ulong expectedHash)
14 | {
15 | ulong hash = Murmur2.Hash64(data.ToUtf8ByteArray(), seed);
16 | Assert.Equal(expectedHash, hash);
17 | }
18 | }
19 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/TabletServerState.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Protobuf.Master;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | public enum TabletServerState
6 | {
7 | ///
8 | /// Default value for backwards compatibility.
9 | ///
10 | Unknown = TServerStatePB.UnknownState,
11 | ///
12 | /// No state for the tserver.
13 | ///
14 | None = TServerStatePB.None,
15 | ///
16 | /// New replicas are not added to the tserver, and failed replicas on
17 | /// the tserver are not re-replicated.
18 | ///
19 | MaintenanceMode = TServerStatePB.MaintenanceMode,
20 | }
21 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Connection/IKuduConnectionFactory.cs:
--------------------------------------------------------------------------------
1 | using System.Collections.Generic;
2 | using System.Threading;
3 | using System.Threading.Tasks;
4 |
5 | namespace Knet.Kudu.Client.Connection;
6 |
7 | public interface IKuduConnectionFactory
8 | {
9 | Task ConnectAsync(
10 | ServerInfo serverInfo, CancellationToken cancellationToken = default);
11 |
12 | Task> GetMasterServerInfoAsync(
13 | HostAndPort hostPort, CancellationToken cancellationToken = default);
14 |
15 | Task GetTabletServerInfoAsync(
16 | HostAndPort hostPort, string uuid, string? location, CancellationToken cancellationToken = default);
17 | }
18 |
--------------------------------------------------------------------------------
/test/Knet.Kudu.Client.Tests/Knet.Kudu.Client.Tests.csproj:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | net7.0
5 | latest
6 | false
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
--------------------------------------------------------------------------------
/.vscode/launch.json:
--------------------------------------------------------------------------------
1 | {
2 | "version": "0.2.0",
3 | "configurations": [
4 | {
5 | "name": "Examples: InsertLoadgen",
6 | "type": "coreclr",
7 | "request": "launch",
8 | "preLaunchTask": "build",
9 | "program": "${workspaceFolder}/examples/InsertLoadgen/bin/Debug/net7.0/InsertLoadgen.dll",
10 | "args": [],
11 | "cwd": "${workspaceFolder}/examples/InsertLoadgen",
12 | "console": "internalConsole",
13 | "stopAtEntry": false
14 | },
15 | {
16 | "name": ".NET Core Attach",
17 | "type": "coreclr",
18 | "request": "attach"
19 | }
20 | ]
21 | }
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/EncryptionPolicy.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | public enum EncryptionPolicy
4 | {
5 | ///
6 | /// Optional, it uses encrypted connection if the server supports it,
7 | /// but it can connect to insecure servers too.
8 | ///
9 | Optional,
10 | ///
11 | /// Only connects to remote servers that support encryption, fails
12 | /// otherwise. It can connect to insecure servers only locally.
13 | ///
14 | RequiredRemote,
15 | ///
16 | /// Only connects to any server, including on the loopback interface,
17 | /// that support encryption, fails otherwise.
18 | ///
19 | Required
20 | }
21 |
--------------------------------------------------------------------------------
/.editorconfig:
--------------------------------------------------------------------------------
1 | # EditorConfig is awesome:http://EditorConfig.org
2 |
3 | # top-most EditorConfig file
4 | root = true
5 |
6 | [*]
7 | indent_style = space
8 | charset = utf-8
9 | trim_trailing_whitespace = true
10 | insert_final_newline = true
11 |
12 | # Dotnet code style settings:
13 | [*.{cs,vb}]
14 | indent_size = 4
15 | # Sort using and Import directives with System.* appearing first
16 | dotnet_sort_system_directives_first = true
17 |
18 | # Namespace settings
19 | csharp_style_namespace_declarations = file_scoped
20 |
21 | [*.{xml,config,*proj,nuspec,props,resx,targets,yml}]
22 | indent_size = 2
23 |
24 | [*.json]
25 | indent_size = 2
26 |
27 | [*.cs]
28 | # Use range operator
29 | dotnet_diagnostic.IDE0057.severity = silent
30 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/KuduReplica.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Connection;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | ///
6 | /// One of the replicas of the tablet.
7 | ///
8 | public class KuduReplica
9 | {
10 | public HostAndPort HostPort { get; }
11 |
12 | public ReplicaRole Role { get; }
13 |
14 | public string? DimensionLabel { get; }
15 |
16 | public KuduReplica(HostAndPort hostPort, ReplicaRole role, string? dimensionLabel)
17 | {
18 | HostPort = hostPort;
19 | Role = role;
20 | DimensionLabel = dimensionLabel;
21 | }
22 |
23 | public override string ToString() =>
24 | $"Replica(host={HostPort}, role={Role}, dimensionLabel={DimensionLabel})";
25 | }
26 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/KuduTableStatistics.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client;
2 |
3 | ///
4 | /// Represent statistics belongs to a specific kudu table.
5 | ///
6 | public class KuduTableStatistics
7 | {
8 | ///
9 | /// The table's on disk size in bytes, this statistic is pre-replication.
10 | ///
11 | public long OnDiskSize { get; }
12 |
13 | ///
14 | /// The table's live row count, this statistic is pre-replication.
15 | ///
16 | public long LiveRowCount { get; }
17 |
18 | public KuduTableStatistics(long onDiskSize, long liveRowCount)
19 | {
20 | OnDiskSize = onDiskSize;
21 | LiveRowCount = liveRowCount;
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Protocol/ParseStep.cs:
--------------------------------------------------------------------------------
1 | namespace Knet.Kudu.Client.Protocol;
2 |
3 | internal enum ParseStep
4 | {
5 | ///
6 | /// Total message length (4 bytes).
7 | ///
8 | TotalMessageLength,
9 | ///
10 | /// RPC Header protobuf length (variable encoding).
11 | ///
12 | HeaderLength,
13 | ///
14 | /// RPC Header protobuf.
15 | ///
16 | Header,
17 | ///
18 | /// Main message length (variable encoding),
19 | /// including sidecars (if any).
20 | ///
21 | MainMessageLength,
22 | ///
23 | /// Main message protobuf, including any sidecars.
24 | ///
25 | MainMessage
26 | }
27 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Internal/KuduTypeFlags.cs:
--------------------------------------------------------------------------------
1 | using System;
2 |
3 | namespace Knet.Kudu.Client.Internal;
4 |
5 | [Flags]
6 | internal enum KuduTypeFlags
7 | {
8 | Int8 = 1 << KuduType.Int8,
9 | Int16 = 1 << KuduType.Int16,
10 | Int32 = 1 << KuduType.Int32,
11 | Int64 = 1 << KuduType.Int64,
12 | String = 1 << KuduType.String,
13 | Bool = 1 << KuduType.Bool,
14 | Float = 1 << KuduType.Float,
15 | Double = 1 << KuduType.Double,
16 | Binary = 1 << KuduType.Binary,
17 | UnixtimeMicros = 1 << KuduType.UnixtimeMicros,
18 | Decimal32 = 1 << KuduType.Decimal32,
19 | Decimal64 = 1 << KuduType.Decimal64,
20 | Decimal128 = 1 << KuduType.Decimal128,
21 | Varchar = 1 << KuduType.Varchar,
22 | Date = 1 << KuduType.Date
23 | }
24 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/KuduType.cs:
--------------------------------------------------------------------------------
1 | using Knet.Kudu.Client.Protobuf;
2 |
3 | namespace Knet.Kudu.Client;
4 |
5 | ///
6 | /// Supported Kudu data types.
7 | /// See https://kudu.apache.org/docs/schema_design.html#column-design
8 | ///
9 | public enum KuduType
10 | {
11 | Int8 = DataType.Int8,
12 | Int16 = DataType.Int16,
13 | Int32 = DataType.Int32,
14 | Int64 = DataType.Int64,
15 | String = DataType.String,
16 | Bool = DataType.Bool,
17 | Float = DataType.Float,
18 | Double = DataType.Double,
19 | Binary = DataType.Binary,
20 | UnixtimeMicros = DataType.UnixtimeMicros,
21 | Decimal32 = DataType.Decimal32,
22 | Decimal64 = DataType.Decimal64,
23 | Decimal128 = DataType.Decimal128,
24 | Varchar = DataType.Varchar,
25 | Date = DataType.Date
26 | }
27 |
--------------------------------------------------------------------------------
/src/Knet.Kudu.Client/Internal/TaskCompletionSource.cs:
--------------------------------------------------------------------------------
1 | #if !NET5_0_OR_GREATER
2 |
3 | using System.Threading.Tasks;
4 |
5 | namespace Knet.Kudu.Client.Internal;
6 |
7 | internal class TaskCompletionSource : TaskCompletionSource