From 1175f23460c6e77158d6bff5e5c2c563e8f52f29 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Thu, 2 Apr 2015 18:06:46 +0100 Subject: [PATCH 01/14] Support version 0 (zookeeper) and version 1 (kafka) protocol for commit offsets. --- src/kafka-net/Interfaces/IKafkaRequest.cs | 4 ++ src/kafka-net/Protocol/BaseRequest.cs | 14 +++++- src/kafka-net/Protocol/OffsetCommitRequest.cs | 28 +++++++++-- src/kafka-net/Protocol/OffsetFetchRequest.cs | 5 ++ .../Integration/OffsetManagementTests.cs | 48 ++++++++++++------- 5 files changed, 75 insertions(+), 24 deletions(-) diff --git a/src/kafka-net/Interfaces/IKafkaRequest.cs b/src/kafka-net/Interfaces/IKafkaRequest.cs index 4ac0653c..cbf1f0bb 100644 --- a/src/kafka-net/Interfaces/IKafkaRequest.cs +++ b/src/kafka-net/Interfaces/IKafkaRequest.cs @@ -19,6 +19,10 @@ public interface IKafkaRequest /// string ClientId { get; set; } /// + /// The API Version used for this request + /// + short ApiVersion { get; } + /// /// Id which will be echoed back by Kafka to correlate responses to this request. Usually automatically assigned by driver. /// int CorrelationId { get; set; } diff --git a/src/kafka-net/Protocol/BaseRequest.cs b/src/kafka-net/Protocol/BaseRequest.cs index 7ffde936..45a840b7 100644 --- a/src/kafka-net/Protocol/BaseRequest.cs +++ b/src/kafka-net/Protocol/BaseRequest.cs @@ -14,10 +14,15 @@ public abstract class BaseRequest /// https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol /// protected const int ReplicaId = -1; - protected const Int16 ApiVersion = 0; + private readonly short _apiVersion; private string _clientId = "Kafka-Net"; private int _correlationId = 1; + protected BaseRequest(short apiVersion = 0) + { + _apiVersion = apiVersion; + } + /// /// Descriptive name of the source of the messages sent to kafka /// @@ -29,6 +34,11 @@ public abstract class BaseRequest /// public int CorrelationId { get { return _correlationId; } set { _correlationId = value; } } + /// + /// Get the API Version for this request + /// + public short ApiVersion { get { return _apiVersion; } } + /// /// Flag which tells the broker call to expect a response for this request. /// @@ -43,7 +53,7 @@ public static KafkaMessagePacker EncodeHeader(IKafkaRequest request) { return new KafkaMessagePacker() .Pack(((Int16)request.ApiKey)) - .Pack(ApiVersion) + .Pack(request.ApiVersion) .Pack(request.CorrelationId) .Pack(request.ClientId, StringPrefixEncoding.Int16); } diff --git a/src/kafka-net/Protocol/OffsetCommitRequest.cs b/src/kafka-net/Protocol/OffsetCommitRequest.cs index 8a65fcd6..57f7037b 100644 --- a/src/kafka-net/Protocol/OffsetCommitRequest.cs +++ b/src/kafka-net/Protocol/OffsetCommitRequest.cs @@ -6,13 +6,20 @@ namespace KafkaNet.Protocol { /// + /// https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetFetchRequest /// Class that represents the api call to commit a specific set of offsets for a given topic. The offset is saved under the /// arbitrary ConsumerGroup name provided by the call. + /// This now supports version 0 and 1 of the protocol /// public class OffsetCommitRequest : BaseRequest, IKafkaRequest { + public OffsetCommitRequest(Int16 version = 1) : base(version) + { + } public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetCommit; } } public string ConsumerGroup { get; set; } + public int ConsumerGroupGenerationId { get; set; } + public string ConsumerId { get; set; } public List OffsetCommits { get; set; } public byte[] Encode() @@ -31,6 +38,12 @@ private byte[] EncodeOffsetCommitRequest(OffsetCommitRequest request) using (var message = EncodeHeader(request).Pack(request.ConsumerGroup, StringPrefixEncoding.Int16)) { + if (ApiVersion == 1) + { + message + .Pack(ConsumerGroupGenerationId) + .Pack(ConsumerId, StringPrefixEncoding.Int16); + } var topicGroups = request.OffsetCommits.GroupBy(x => x.Topic).ToList(); message.Pack(topicGroups.Count); @@ -44,10 +57,17 @@ private byte[] EncodeOffsetCommitRequest(OffsetCommitRequest request) { foreach (var commit in partition) { - message.Pack(partition.Key) - .Pack(commit.Offset) - .Pack(commit.TimeStamp) - .Pack(commit.Metadata, StringPrefixEncoding.Int16); + message + .Pack(partition.Key) + .Pack(commit.Offset); + + if (ApiVersion == 1) + { + message.Pack(commit.TimeStamp); + } + + message + .Pack(commit.Metadata, StringPrefixEncoding.Int16); } } } diff --git a/src/kafka-net/Protocol/OffsetFetchRequest.cs b/src/kafka-net/Protocol/OffsetFetchRequest.cs index 773d8c7d..da327502 100644 --- a/src/kafka-net/Protocol/OffsetFetchRequest.cs +++ b/src/kafka-net/Protocol/OffsetFetchRequest.cs @@ -10,9 +10,14 @@ namespace KafkaNet.Protocol /// Class that represents both the request and the response from a kafka server of requesting a stored offset value /// for a given consumer group. Essentially this part of the api allows a user to save/load a given offset position /// under any abritrary name. + /// This now supports version 1 of the protocol /// public class OffsetFetchRequest : BaseRequest, IKafkaRequest { + public OffsetFetchRequest(short version = 1) : base(version) + { + + } public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetFetch; } } public string ConsumerGroup { get; set; } public List Topics { get; set; } diff --git a/src/kafka-tests/Integration/OffsetManagementTests.cs b/src/kafka-tests/Integration/OffsetManagementTests.cs index 673cfda0..9f907d5b 100644 --- a/src/kafka-tests/Integration/OffsetManagementTests.cs +++ b/src/kafka-tests/Integration/OffsetManagementTests.cs @@ -22,7 +22,7 @@ public void Setup() } [Test] - public void OffsetFetchRequestOfNonExistingGroupShouldReturnNoError() + public void OffsetFetchRequestOfNonExistingGroupShouldReturnNoError([Values(0,1)] int version) { //From documentation: https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetFetchRequest //Note that if there is no offset associated with a topic-partition under that consumer group the broker does not set an error code @@ -30,27 +30,36 @@ public void OffsetFetchRequestOfNonExistingGroupShouldReturnNoError() const int partitionId = 0; using (var router = new BrokerRouter(Options)) { - var request = CreateOffsetFetchRequest(Guid.NewGuid().ToString(), partitionId); + var request = CreateOffsetFetchRequest(version, Guid.NewGuid().ToString(), partitionId); var conn = router.SelectBrokerRoute(IntegrationConfig.IntegrationTopic, partitionId); var response = conn.Connection.SendAsync(request).Result.FirstOrDefault(); Assert.That(response, Is.Not.Null); - Assert.That(response.Error, Is.EqualTo((int)ErrorResponseCode.NoError)); + if (version == 0) + { + // Version 0 (storing in zookeeper) results in unknown topic or partition as the consumer group + // and partition are used to make up the string, and when it is missing it results in an error + Assert.That(response.Error, Is.EqualTo((int)ErrorResponseCode.UnknownTopicOrPartition)); + } + else + { + Assert.That(response.Error, Is.EqualTo((int)ErrorResponseCode.NoError)); + } Assert.That(response.Offset, Is.EqualTo(-1)); } } [Test] - public void OffsetCommitShouldStoreAndReturnSuccess() + public void OffsetCommitShouldStoreAndReturnSuccess([Values(0, 1)] int version) { const int partitionId = 0; using (var router = new BrokerRouter(Options)) { var conn = router.SelectBrokerRoute(IntegrationConfig.IntegrationTopic, partitionId); - var commit = CreateOffsetCommitRequest(IntegrationConfig.IntegrationConsumer, partitionId, 10); + var commit = CreateOffsetCommitRequest(version, IntegrationConfig.IntegrationConsumer, partitionId, 10); var response = conn.Connection.SendAsync(commit).Result.FirstOrDefault(); Assert.That(response, Is.Not.Null); @@ -59,7 +68,7 @@ public void OffsetCommitShouldStoreAndReturnSuccess() } [Test] - public void OffsetCommitShouldStoreOffsetValue() + public void OffsetCommitShouldStoreOffsetValue([Values(0, 1)] int version) { const int partitionId = 0; const long offset = 99; @@ -69,13 +78,13 @@ public void OffsetCommitShouldStoreOffsetValue() var conn = router.SelectBrokerRoute(IntegrationConfig.IntegrationTopic, partitionId); - var commit = CreateOffsetCommitRequest(IntegrationConfig.IntegrationConsumer, partitionId, offset); + var commit = CreateOffsetCommitRequest(version, IntegrationConfig.IntegrationConsumer, partitionId, offset); var commitResponse = conn.Connection.SendAsync(commit).Result.FirstOrDefault(); Assert.That(commitResponse, Is.Not.Null); Assert.That(commitResponse.Error, Is.EqualTo((int)ErrorResponseCode.NoError)); - var fetch = CreateOffsetFetchRequest(IntegrationConfig.IntegrationConsumer, partitionId); + var fetch = CreateOffsetFetchRequest(version, IntegrationConfig.IntegrationConsumer, partitionId); var fetchResponse = conn.Connection.SendAsync(fetch).Result.FirstOrDefault(); Assert.That(fetchResponse, Is.Not.Null); @@ -85,8 +94,7 @@ public void OffsetCommitShouldStoreOffsetValue() } [Test] - [Ignore("The response does not seem to return metadata information. Not supported yet in kafka?")] - public void OffsetCommitShouldStoreMetadata() + public void OffsetCommitShouldStoreMetadata([Values(0, 1)] int version) { const int partitionId = 0; const long offset = 101; @@ -96,24 +104,28 @@ public void OffsetCommitShouldStoreMetadata() { var conn = router.SelectBrokerRoute(IntegrationConfig.IntegrationTopic, partitionId); - var commit = CreateOffsetCommitRequest(IntegrationConfig.IntegrationConsumer, partitionId, offset, metadata); + var commit = CreateOffsetCommitRequest(version, IntegrationConfig.IntegrationConsumer, partitionId, offset, metadata); var commitResponse = conn.Connection.SendAsync(commit).Result.FirstOrDefault(); Assert.That(commitResponse, Is.Not.Null); Assert.That(commitResponse.Error, Is.EqualTo((int)ErrorResponseCode.NoError)); - var fetch = CreateOffsetFetchRequest(IntegrationConfig.IntegrationConsumer, partitionId); + var fetch = CreateOffsetFetchRequest(version, IntegrationConfig.IntegrationConsumer, partitionId); var fetchResponse = conn.Connection.SendAsync(fetch).Result.FirstOrDefault(); Assert.That(fetchResponse, Is.Not.Null); Assert.That(fetchResponse.Error, Is.EqualTo((int)ErrorResponseCode.NoError)); Assert.That(fetchResponse.Offset, Is.EqualTo(offset)); - Assert.That(fetchResponse.MetaData, Is.EqualTo(metadata)); + + // metadata is only stored with version 1. Zookeeper doesn't store metadata + if (version == 1) + { + Assert.That(fetchResponse.MetaData, Is.EqualTo(metadata)); + } } } [Test] - [Ignore("Not supported currently in 8.1.1?")] public void ConsumerMetadataRequestShouldReturnWithoutError() { using (var router = new BrokerRouter(Options)) @@ -129,9 +141,9 @@ public void ConsumerMetadataRequestShouldReturnWithoutError() } } - private OffsetFetchRequest CreateOffsetFetchRequest(string consumerGroup, int partitionId) + private OffsetFetchRequest CreateOffsetFetchRequest(int version, string consumerGroup, int partitionId) { - var request = new OffsetFetchRequest + var request = new OffsetFetchRequest((short)version) { ConsumerGroup = consumerGroup, Topics = new List @@ -147,9 +159,9 @@ private OffsetFetchRequest CreateOffsetFetchRequest(string consumerGroup, int pa return request; } - private OffsetCommitRequest CreateOffsetCommitRequest(string consumerGroup, int partitionId, long offset, string metadata = null) + private OffsetCommitRequest CreateOffsetCommitRequest(int version, string consumerGroup, int partitionId, long offset, string metadata = null) { - var commit = new OffsetCommitRequest + var commit = new OffsetCommitRequest((short)version) { ConsumerGroup = consumerGroup, OffsetCommits = new List From 1cd8447bb3ed7b8d8dd4d0efa36b679a506fe511 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Wed, 8 Apr 2015 20:38:43 +0100 Subject: [PATCH 02/14] New SimpleKafka derived from kafka-net --- src/SimpleKafka/BigEndianReader.cs | 57 ++++ src/SimpleKafka/BigEndianWriter.cs | 42 +++ .../Common/BigEndianBinaryReader.cs | 218 +++++++++++++ .../Common/BigEndianBinaryWriter.cs | 174 ++++++++++ src/SimpleKafka/Common/Crc32Provider.cs | 83 +++++ src/SimpleKafka/Common/Extensions.cs | 32 ++ src/SimpleKafka/Common/KafkaMessagePacker.cs | 106 +++++++ src/SimpleKafka/Extensions.cs | 37 +++ src/SimpleKafka/IKafkaMessagePartitioner.cs | 13 + src/SimpleKafka/IKafkaSerializer.cs | 15 + src/SimpleKafka/Interfaces/IKafkaRequest.cs | 45 +++ src/SimpleKafka/KafkaBrokers.cs | 298 ++++++++++++++++++ src/SimpleKafka/KafkaConnection.cs | 108 +++++++ src/SimpleKafka/KafkaConnectionFactory.cs | 56 ++++ src/SimpleKafka/KafkaMessage.cs | 26 ++ src/SimpleKafka/KafkaProducer.cs | 182 +++++++++++ src/SimpleKafka/LoadBalancedPartitioner.cs | 26 ++ src/SimpleKafka/NullSerializer.cs | 21 ++ src/SimpleKafka/Protocol/BaseRequest.cs | 61 ++++ src/SimpleKafka/Protocol/Broker.cs | 23 ++ .../Protocol/ConsumerMetadataRequest.cs | 66 ++++ src/SimpleKafka/Protocol/FetchRequest.cs | 160 ++++++++++ src/SimpleKafka/Protocol/Message.cs | 199 ++++++++++++ src/SimpleKafka/Protocol/MetadataRequest.cs | 88 ++++++ .../Protocol/OffsetCommitRequest.cs | 152 +++++++++ .../Protocol/OffsetFetchRequest.cs | 137 ++++++++ src/SimpleKafka/Protocol/OffsetRequest.cs | 154 +++++++++ src/SimpleKafka/Protocol/ProduceRequest.cs | 155 +++++++++ src/SimpleKafka/Protocol/Protocol.cs | 211 +++++++++++++ src/SimpleKafka/Protocol/Topic.cs | 99 ++++++ src/SimpleKafka/SimpleKafka.csproj | 101 ++++++ src/SimpleKafka/StringSerializer.cs | 21 ++ src/SimpleKafka/packages.config | 4 + src/kafka-net/Consumer.cs | 0 34 files changed, 3170 insertions(+) create mode 100644 src/SimpleKafka/BigEndianReader.cs create mode 100644 src/SimpleKafka/BigEndianWriter.cs create mode 100644 src/SimpleKafka/Common/BigEndianBinaryReader.cs create mode 100644 src/SimpleKafka/Common/BigEndianBinaryWriter.cs create mode 100644 src/SimpleKafka/Common/Crc32Provider.cs create mode 100644 src/SimpleKafka/Common/Extensions.cs create mode 100644 src/SimpleKafka/Common/KafkaMessagePacker.cs create mode 100644 src/SimpleKafka/Extensions.cs create mode 100644 src/SimpleKafka/IKafkaMessagePartitioner.cs create mode 100644 src/SimpleKafka/IKafkaSerializer.cs create mode 100644 src/SimpleKafka/Interfaces/IKafkaRequest.cs create mode 100644 src/SimpleKafka/KafkaBrokers.cs create mode 100644 src/SimpleKafka/KafkaConnection.cs create mode 100644 src/SimpleKafka/KafkaConnectionFactory.cs create mode 100644 src/SimpleKafka/KafkaMessage.cs create mode 100644 src/SimpleKafka/KafkaProducer.cs create mode 100644 src/SimpleKafka/LoadBalancedPartitioner.cs create mode 100644 src/SimpleKafka/NullSerializer.cs create mode 100644 src/SimpleKafka/Protocol/BaseRequest.cs create mode 100644 src/SimpleKafka/Protocol/Broker.cs create mode 100644 src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs create mode 100644 src/SimpleKafka/Protocol/FetchRequest.cs create mode 100644 src/SimpleKafka/Protocol/Message.cs create mode 100644 src/SimpleKafka/Protocol/MetadataRequest.cs create mode 100644 src/SimpleKafka/Protocol/OffsetCommitRequest.cs create mode 100644 src/SimpleKafka/Protocol/OffsetFetchRequest.cs create mode 100644 src/SimpleKafka/Protocol/OffsetRequest.cs create mode 100644 src/SimpleKafka/Protocol/ProduceRequest.cs create mode 100644 src/SimpleKafka/Protocol/Protocol.cs create mode 100644 src/SimpleKafka/Protocol/Topic.cs create mode 100644 src/SimpleKafka/SimpleKafka.csproj create mode 100644 src/SimpleKafka/StringSerializer.cs create mode 100644 src/SimpleKafka/packages.config mode change 100644 => 100755 src/kafka-net/Consumer.cs diff --git a/src/SimpleKafka/BigEndianReader.cs b/src/SimpleKafka/BigEndianReader.cs new file mode 100644 index 00000000..be61a424 --- /dev/null +++ b/src/SimpleKafka/BigEndianReader.cs @@ -0,0 +1,57 @@ +using System; +using System.Collections.Generic; +using System.IO; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class BigEndianReader + { + private readonly Stream stream; + private readonly byte[] commonBuffer = new byte[8]; + + + public BigEndianReader(Stream stream) + { + this.stream = stream; + } + + private async Task ReadFullyAsync(byte[] buffer, int offset, int numberOfBytes, CancellationToken token) + { + while (numberOfBytes > 0) + { + var bytesRead = await stream.ReadAsync(buffer, offset, numberOfBytes, token).ConfigureAwait(false); + if (bytesRead <= 0) + { + throw new EndOfStreamException(); + } + numberOfBytes -= bytesRead; + offset += bytesRead; + } + } + + public async Task ReadInt32Async(CancellationToken token) + { + var buffer = commonBuffer; + await ReadFullyAsync(buffer, 0, 4, token).ConfigureAwait(false); + unchecked + { + return (buffer[0] << 24) | + (buffer[1] << 16) | + (buffer[2] << 8) | + (buffer[3]); + } + } + + public async Task ReadBytesAsync(int numberOfBytes, CancellationToken token) + { + var buffer = new byte[numberOfBytes]; + await ReadFullyAsync(buffer, 0, numberOfBytes, token).ConfigureAwait(false); + return buffer; + } + + } +} diff --git a/src/SimpleKafka/BigEndianWriter.cs b/src/SimpleKafka/BigEndianWriter.cs new file mode 100644 index 00000000..ba3969e9 --- /dev/null +++ b/src/SimpleKafka/BigEndianWriter.cs @@ -0,0 +1,42 @@ +using System; +using System.Collections.Generic; +using System.IO; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class BigEndianWriter + { + private readonly Stream stream; + private readonly byte[] commonBuffer = new byte[8]; + + public BigEndianWriter(Stream stream) + { + this.stream = stream; + } + + public async Task WriteAsync(byte[] buffer, int offset, int count, CancellationToken token) + { + await stream.WriteAsync(buffer, offset, count, token).ConfigureAwait(false); + } + + public async Task WriteAsync(int value, CancellationToken token) + { + var buffer = commonBuffer; + unchecked + { + buffer[3] = (byte)(value & 0x0ff); + value = value >> 8; + buffer[2] = (byte)(value & 0x0ff); + value = value >> 8; + buffer[1] = (byte)(value & 0x0ff); + value = value >> 8; + buffer[0] = (byte)(value & 0x0ff); + } + await stream.WriteAsync(buffer, 0, 4, token).ConfigureAwait(false); + } + } +} diff --git a/src/SimpleKafka/Common/BigEndianBinaryReader.cs b/src/SimpleKafka/Common/BigEndianBinaryReader.cs new file mode 100644 index 00000000..668a3224 --- /dev/null +++ b/src/SimpleKafka/Common/BigEndianBinaryReader.cs @@ -0,0 +1,218 @@ +using System; +using System.Collections.Generic; +using System.Diagnostics.Contracts; +using System.IO; +using System.Linq; +using System.Text; + +namespace SimpleKafka.Common +{ + /// + /// A BinaryReader that is BigEndian aware binary reader. + /// + /// + /// Booleans, bytes and byte arrays will be written directly. + /// All other values will be converted to a byte array in BigEndian byte order and written. + /// Characters and Strings will all be encoded in UTF-8 (which is byte order independent). + /// + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// + /// The code was modified to provide Kafka specific logic and helper functions. + /// + public class BigEndianBinaryReader : BinaryReader + { + private const int KafkaNullSize = -1; + + public BigEndianBinaryReader(IEnumerable payload) : base(new MemoryStream(payload.ToArray()), Encoding.UTF8) + { + + } + + public long Length{get{return base.BaseStream.Length;}} + public long Position { get { return base.BaseStream.Position; } set { base.BaseStream.Position = 0; } } + public bool HasData { get { return base.BaseStream.Position < base.BaseStream.Length; } } + + public bool Available(int dataSize) + { + return (base.BaseStream.Length - base.BaseStream.Position) >= dataSize; + } + + public override Decimal ReadDecimal() + { + var bytes = GetNextBytesNativeEndian(16); + + var ints = new Int32[4]; + ints[0] = (Int32)bytes[0] << 0 + | (Int32)bytes[1] << 8 + | (Int32)bytes[2] << 16 + | (Int32)bytes[3] << 24; + ints[1] = (Int32)bytes[4] << 0 + | (Int32)bytes[5] << 8 + | (Int32)bytes[6] << 16 + | (Int32)bytes[7] << 24; + ints[2] = (Int32)bytes[8] << 0 + | (Int32)bytes[9] << 8 + | (Int32)bytes[10] << 16 + | (Int32)bytes[11] << 24; + ints[3] = (Int32)bytes[12] << 0 + | (Int32)bytes[13] << 8 + | (Int32)bytes[14] << 16 + | (Int32)bytes[15] << 24; + + return new Decimal(ints); + } + + public override Single ReadSingle() + { + return EndianAwareRead(4, BitConverter.ToSingle); + } + + public override Double ReadDouble() + { + return EndianAwareRead(8, BitConverter.ToDouble); + } + + public override Int16 ReadInt16() + { + return EndianAwareRead(2, BitConverter.ToInt16); + } + + public override Int32 ReadInt32() + { + return EndianAwareRead(4, BitConverter.ToInt32); + } + + public override Int64 ReadInt64() + { + return EndianAwareRead(8, BitConverter.ToInt64); + } + + public override UInt16 ReadUInt16() + { + return EndianAwareRead(2, BitConverter.ToUInt16); + } + + public override UInt32 ReadUInt32() + { + return EndianAwareRead(4, BitConverter.ToUInt32); + } + + public override UInt64 ReadUInt64() + { + return EndianAwareRead(8, BitConverter.ToUInt64); + } + + public string ReadInt16String() + { + var size = ReadInt16(); + if (size == KafkaNullSize) return null; + return Encoding.UTF8.GetString(RawRead(size), 0, size); + } + + public string ReadIntString() + { + var size = ReadInt32(); + if (size == KafkaNullSize) return null; + return Encoding.UTF8.GetString(RawRead(size), 0, size); + } + + public byte[] ReadInt16PrefixedBytes() + { + var size = ReadInt16(); + if (size == KafkaNullSize) { return null; } + return RawRead(size); + } + + public byte[] ReadIntPrefixedBytes() + { + var size = ReadInt32(); + if (size == KafkaNullSize) { return null; } + return RawRead(size); + } + + public byte[] ReadToEnd() + { + var size = (int)(base.BaseStream.Length - base.BaseStream.Position); + var buffer = new byte[size]; + base.BaseStream.Read(buffer, 0, size); + return buffer; + } + + public byte[] CrcHash() + { + var currentPosition = base.BaseStream.Position; + try + { + base.BaseStream.Position = 0; + return Crc32Provider.ComputeHash(ReadToEnd()); + } + finally + { + base.BaseStream.Position = currentPosition; + } + } + + public uint Crc() + { + var currentPosition = base.BaseStream.Position; + try + { + base.BaseStream.Position = 0; + return Crc32Provider.Compute(ReadToEnd()); + } + finally + { + base.BaseStream.Position = currentPosition; + } + } + + public byte[] RawRead(int size) + { + if (size <= 0) { return new byte[0]; } + + var buffer = new byte[size]; + + base.Read(buffer, 0, size); + + return buffer; + } + + private T EndianAwareRead(Int32 size, Func converter) where T : struct + { + Contract.Requires(size >= 0); + Contract.Requires(converter != null); + + var bytes = GetNextBytesNativeEndian(size); + return converter(bytes, 0); + } + + private Byte[] GetNextBytesNativeEndian(Int32 count) + { + Contract.Requires(count >= 0); + Contract.Ensures(Contract.Result() != null); + Contract.Ensures(Contract.Result().Length == count); + + var bytes = GetNextBytes(count); + if (BitConverter.IsLittleEndian) + Array.Reverse(bytes); + return bytes; + } + + private Byte[] GetNextBytes(Int32 count) + { + Contract.Requires(count >= 0); + Contract.Ensures(Contract.Result() != null); + Contract.Ensures(Contract.Result().Length == count); + + var buffer = new Byte[count]; + var bytesRead = BaseStream.Read(buffer, 0, count); + + if (bytesRead != count) + throw new EndOfStreamException(); + + return buffer; + } + } +} diff --git a/src/SimpleKafka/Common/BigEndianBinaryWriter.cs b/src/SimpleKafka/Common/BigEndianBinaryWriter.cs new file mode 100644 index 00000000..ebab24b8 --- /dev/null +++ b/src/SimpleKafka/Common/BigEndianBinaryWriter.cs @@ -0,0 +1,174 @@ +using System; +using System.Diagnostics.Contracts; +using System.IO; +using System.Text; + +namespace SimpleKafka.Common +{ + /// + /// A BinaryWriter that stores values in BigEndian format. + /// + /// + /// Booleans, bytes and byte arrays will be written directly. + /// All other values will be converted to a byte array in BigEndian byte order and written. + /// Characters and Strings will all be encoded in UTF-8 (which is byte order independent). + /// + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// The code was modified to implement Kafka specific byte handling. + /// + public class BigEndianBinaryWriter : BinaryWriter + { + public BigEndianBinaryWriter(Stream stream) + : base(stream, Encoding.UTF8) + { + Contract.Requires(stream != null); + } + + public BigEndianBinaryWriter(Stream stream, Boolean leaveOpen) + : base(stream, Encoding.UTF8, leaveOpen) + { + Contract.Requires(stream != null); + } + + public override void Write(Decimal value) + { + var ints = Decimal.GetBits(value); + Contract.Assume(ints != null); + Contract.Assume(ints.Length == 4); + + if (BitConverter.IsLittleEndian) + Array.Reverse(ints); + + for (var i = 0; i < 4; ++i) + { + var bytes = BitConverter.GetBytes(ints[i]); + if (BitConverter.IsLittleEndian) + Array.Reverse(bytes); + + Write(bytes); + } + } + + public override void Write(Single value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(Double value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(Int16 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(Int32 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(Int64 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(UInt16 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(UInt32 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(UInt64 value) + { + var bytes = BitConverter.GetBytes(value); + WriteBigEndian(bytes); + } + + public override void Write(string value) + { + throw new NotSupportedException("Kafka requires specific string length prefix encoding."); + } + + public void Write(byte[] value, StringPrefixEncoding encoding) + { + if (value == null) + { + Write(-1); + return; + } + + switch (encoding) + { + case StringPrefixEncoding.Int16: + Write((Int16)value.Length); + break; + case StringPrefixEncoding.Int32: + Write(value.Length); + break; + } + + Write(value); + } + + public void Write(string value, StringPrefixEncoding encoding) + { + if (value == null) + { + switch (encoding) + { + case StringPrefixEncoding.Int16: + Write((Int16)(-1)); + return; + default: + Write(-1); + return; + } + } + + switch (encoding) + { + case StringPrefixEncoding.Int16: + Write((Int16)value.Length); + break; + case StringPrefixEncoding.Int32: + Write(value.Length); + break; + } + + Write(Encoding.UTF8.GetBytes(value)); + } + + + private void WriteBigEndian(Byte[] bytes) + { + Contract.Requires(bytes != null); + + if (BitConverter.IsLittleEndian) + Array.Reverse(bytes); + + Write(bytes); + } + } + + public enum StringPrefixEncoding + { + Int16, + Int32, + None + }; +} diff --git a/src/SimpleKafka/Common/Crc32Provider.cs b/src/SimpleKafka/Common/Crc32Provider.cs new file mode 100644 index 00000000..5cb19339 --- /dev/null +++ b/src/SimpleKafka/Common/Crc32Provider.cs @@ -0,0 +1,83 @@ +// Copyright (c) Damien Guard. All rights reserved. +// 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 +// Originally published at http://damieng.com/blog/2006/08/08/calculating_crc32_in_c_and_net + +using System; + +namespace SimpleKafka.Common +{ + /// + /// This code was originally from the copyrighted code listed above but was modified significantly + /// as the original code was not thread safe and did not match was was required of this driver. This + /// class now provides a static lib which will do the simple CRC calculation required by Kafka servers. + /// + public static class Crc32Provider + { + public const UInt32 DefaultPolynomial = 0xedb88320u; + public const UInt32 DefaultSeed = 0xffffffffu; + private static readonly UInt32[] PolynomialTable; + + static Crc32Provider() + { + PolynomialTable = InitializeTable(DefaultPolynomial); + } + + public static UInt32 Compute(byte[] buffer) + { + return ~CalculateHash(buffer, 0, buffer.Length); + } + + public static UInt32 Compute(byte[] buffer, int offset, int length) + { + return ~CalculateHash(buffer, offset, length); + } + + public static byte[] ComputeHash(byte[] buffer) + { + return UInt32ToBigEndianBytes(Compute(buffer)); + } + + public static byte[] ComputeHash(byte[] buffer, int offset, int length) + { + return UInt32ToBigEndianBytes(Compute(buffer, offset, length)); + } + + private static UInt32[] InitializeTable(UInt32 polynomial) + { + var createTable = new UInt32[256]; + for (var i = 0; i < 256; i++) + { + var entry = (UInt32)i; + for (var j = 0; j < 8; j++) + if ((entry & 1) == 1) + entry = (entry >> 1) ^ polynomial; + else + entry = entry >> 1; + createTable[i] = entry; + } + + return createTable; + } + + private static UInt32 CalculateHash(byte[] buffer, int offset, int length) + { + var crc = DefaultSeed; + for (var i = offset; i < length; i++) + { + crc = (crc >> 8) ^ PolynomialTable[buffer[i] ^ crc & 0xff]; + } + return crc; + } + + private static byte[] UInt32ToBigEndianBytes(UInt32 uint32) + { + var result = BitConverter.GetBytes(uint32); + + if (BitConverter.IsLittleEndian) + Array.Reverse(result); + + return result; + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Common/Extensions.cs b/src/SimpleKafka/Common/Extensions.cs new file mode 100644 index 00000000..3bcf592f --- /dev/null +++ b/src/SimpleKafka/Common/Extensions.cs @@ -0,0 +1,32 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka.Common +{ + public static class Extensions + { + public static string ToUtf8String(this byte[] value) + { + if (value == null) return string.Empty; + + return Encoding.UTF8.GetString(value); + } + + public static byte[] ToBytes(this string value) + { + if (string.IsNullOrEmpty(value)) return (-1).ToBytes(); + + //UTF8 is array of bytes, no endianness + return Encoding.UTF8.GetBytes(value); + } + + public static byte[] ToBytes(this int value) + { + return BitConverter.GetBytes(value).Reverse().ToArray(); + } + + } +} diff --git a/src/SimpleKafka/Common/KafkaMessagePacker.cs b/src/SimpleKafka/Common/KafkaMessagePacker.cs new file mode 100644 index 00000000..67ef5819 --- /dev/null +++ b/src/SimpleKafka/Common/KafkaMessagePacker.cs @@ -0,0 +1,106 @@ +using System; +using System.Collections.Generic; +using System.IO; + +namespace SimpleKafka.Common +{ + public class KafkaMessagePacker : IDisposable + { + private const int IntegerByteSize = 4; + private readonly BigEndianBinaryWriter _stream; + + public KafkaMessagePacker() + { + _stream = new BigEndianBinaryWriter(new MemoryStream()); + Pack(IntegerByteSize); //pre-allocate space for buffer length + } + + public KafkaMessagePacker Pack(byte value) + { + _stream.Write(value); + return this; + } + + public KafkaMessagePacker Pack(Int32 ints) + { + _stream.Write(ints); + return this; + } + + public KafkaMessagePacker Pack(Int16 ints) + { + _stream.Write(ints); + return this; + } + + public KafkaMessagePacker Pack(Int64 ints) + { + _stream.Write(ints); + return this; + } + + public KafkaMessagePacker Pack(byte[] buffer, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + { + _stream.Write(buffer, encoding); + return this; + } + + public KafkaMessagePacker Pack(string data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + { + _stream.Write(data, encoding); + return this; + } + + public KafkaMessagePacker Pack(IEnumerable data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + { + foreach (var item in data) + { + _stream.Write(item, encoding); + } + + return this; + } + + public byte[] Payload() + { + var buffer = new byte[_stream.BaseStream.Length]; + _stream.BaseStream.Position = 0; + Pack((Int32)(_stream.BaseStream.Length - IntegerByteSize)); + _stream.BaseStream.Position = 0; + _stream.BaseStream.Read(buffer, 0, (int)_stream.BaseStream.Length); + return buffer; + } + + public byte[] PayloadNoLength() + { + var payloadLength = _stream.BaseStream.Length - IntegerByteSize; + var buffer = new byte[payloadLength]; + _stream.BaseStream.Position = IntegerByteSize; + _stream.BaseStream.Read(buffer, 0, (int)payloadLength); + return buffer; + } + + public byte[] CrcPayload() + { + var buffer = new byte[_stream.BaseStream.Length]; + + //copy the payload over + _stream.BaseStream.Position = 0; + _stream.BaseStream.Read(buffer, 0, (int)_stream.BaseStream.Length); + + //calculate the crc + var crc = Crc32Provider.ComputeHash(buffer, IntegerByteSize, buffer.Length); + buffer[0] = crc[0]; + buffer[1] = crc[1]; + buffer[2] = crc[2]; + buffer[3] = crc[3]; + + return buffer; + } + + public void Dispose() + { + using (_stream) { } + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Extensions.cs b/src/SimpleKafka/Extensions.cs new file mode 100644 index 00000000..a72cf88e --- /dev/null +++ b/src/SimpleKafka/Extensions.cs @@ -0,0 +1,37 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + internal static class Extensions + { + public static TValue FindOrCreate(this Dictionary map, TKey key) + where TValue : new() + { + TValue result; + if (!map.TryGetValue(key, out result)) + { + result = new TValue(); + map.Add(key, result); + } + return result; + } + + public static TValue TryGetValue(this Dictionary map, TKey key) + where TValue : class + { + TValue result; + if (map.TryGetValue(key, out result)) + { + return result; + } + else + { + return null; + } + } + } +} diff --git a/src/SimpleKafka/IKafkaMessagePartitioner.cs b/src/SimpleKafka/IKafkaMessagePartitioner.cs new file mode 100644 index 00000000..5ee67623 --- /dev/null +++ b/src/SimpleKafka/IKafkaMessagePartitioner.cs @@ -0,0 +1,13 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public interface IKafkaMessagePartitioner + { + int CalculatePartition(KafkaMessage message); + } +} diff --git a/src/SimpleKafka/IKafkaSerializer.cs b/src/SimpleKafka/IKafkaSerializer.cs new file mode 100644 index 00000000..0d9f6caa --- /dev/null +++ b/src/SimpleKafka/IKafkaSerializer.cs @@ -0,0 +1,15 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public interface IKafkaSerializer + { + byte[] Serialize(T value); + + T Deserialize(byte[] serialized); + } +} diff --git a/src/SimpleKafka/Interfaces/IKafkaRequest.cs b/src/SimpleKafka/Interfaces/IKafkaRequest.cs new file mode 100644 index 00000000..fd50e1ad --- /dev/null +++ b/src/SimpleKafka/Interfaces/IKafkaRequest.cs @@ -0,0 +1,45 @@ +using System.Collections.Generic; +using SimpleKafka.Protocol; + +namespace SimpleKafka +{ + /// + /// KafkaRequest represents a Kafka request messages as an object which can Encode itself into the appropriate + /// binary request and Decode any responses to that request. + /// + /// The type of the KafkaResponse expected back from the request. + public interface IKafkaRequest + { + /// + /// Indicates this request should wait for a response from the broker + /// + bool ExpectResponse { get; } + /// + /// Descriptive name used to identify the source of this request. + /// + string ClientId { get; set; } + /// + /// The API Version used for this request + /// + short ApiVersion { get; } + /// + /// Id which will be echoed back by Kafka to correlate responses to this request. Usually automatically assigned by driver. + /// + int CorrelationId { get; set; } + /// + /// Enum identifying the specific type of request message being represented. + /// + ApiKeyRequestType ApiKey { get; } + /// + /// Encode this request into the Kafka wire protocol. + /// + /// Byte[] representing the binary wire protocol of this request. + byte[] Encode(); + /// + /// Decode a response payload from Kafka into an enumerable of T responses. + /// + /// Payload data returned by Kafka servers. + /// + IEnumerable Decode(byte[] payload); + } +} \ No newline at end of file diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs new file mode 100644 index 00000000..309cd908 --- /dev/null +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -0,0 +1,298 @@ +using Serilog; +using SimpleKafka.Protocol; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class KafkaBrokers : IDisposable + { + private readonly Random backoffGenerator = new Random(); + private readonly HashSet brokers = new HashSet(); + private readonly Dictionary topicToPartitions = new Dictionary(StringComparer.CurrentCultureIgnoreCase); + + private readonly Dictionary connections = new Dictionary(); + public KafkaConnection this[int brokerId] + { + get + { + var connection = connections.TryGetValue(brokerId); + if (connection == null) + { + throw new KeyNotFoundException("Failed to find broker " + brokerId); + } + return connection; + } + } + + public KafkaBrokers(params Uri[] addresses) + { + foreach (var address in addresses) + { + brokers.Add(address); + } + } + + + public async Task RefreshAsync(CancellationToken token) + { + while (await TryToRefreshAsync(token).ConfigureAwait(false)) + { + + if (!IsLeaderElectionTakingPlace) + { + return true; + } + Log.Verbose("Leader election taking place"); + await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); + } + return false; + + } + + private bool IsLeaderElectionTakingPlace + { + get + { + foreach (var topicKvp in topicToPartitions) + { + foreach (var partition in topicKvp.Value) + { + if (partition.LeaderId == -1) + { + return true; + } + } + } + + return false; + } + } + + private async Task TryToRefreshAsync(CancellationToken token) + { + if (brokers.Count == 0) + { + throw new InvalidOperationException("No brokers defined"); + } + + if (connections.Count > 0) + { + await TryToRefreshFromCurrentConnectionsAsync(token).ConfigureAwait(false); + } + + if (connections.Count == 0) + { + await TryToInitialiseFromBrokersAsync(brokers, token).ConfigureAwait(false); + } + + return (connections.Count > 0); + } + + + private async Task TryToInitialiseFromBrokersAsync(IEnumerable brokers, CancellationToken token) + { + foreach (var broker in brokers) + { + try { + var newConnection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(broker, token).ConfigureAwait(false); + var success = await TryToRefreshFromConnectionAsync(newConnection, token).ConfigureAwait(false); + if (success) + { + return; + } else + { + newConnection.Dispose(); + } + } + catch (Exception ex) + { + Console.WriteLine(ex); + } + } + } + + internal void AddTopic(string topic) + { + if (!topicToPartitions.ContainsKey(topic)) + { + topicToPartitions.Add(topic, null); + } + } + + internal Partition[] GetPartitionsForTopic(string topic) + { + return topicToPartitions.TryGetValue(topic); + } + + private async Task TryToRefreshFromCurrentConnectionsAsync(CancellationToken token) + { + foreach (var connectionKvp in connections.ToList()) + { + var connection = connectionKvp.Value; + var success = await TryToRefreshFromConnectionAsync(connection, token).ConfigureAwait(false); + if (success) + { + return; + } + else + { + connection.Dispose(); + connections.Remove(connectionKvp.Key); + } + } + + } + + private async Task TryToRefreshFromConnectionAsync(KafkaConnection connection, CancellationToken token) + { + var request = new MetadataRequest + { + Topics = topicToPartitions.Keys.ToList() + }; + + try { + var response = await connection.SendRequestAsync(request, token).ConfigureAwait(false); + await RefreshBrokersAsync(response.Brokers, token).ConfigureAwait(false); + RefreshTopics(response.Topics); + return true; + } + catch (Exception ex) + { + Log.Error(ex, "Error refreshing connection"); + return false; + } + } + + + private void RefreshTopics(List topics) + { + var previousTopics = new HashSet(topicToPartitions.Keys); + + + foreach (var topic in topics) + { + if (topic.ErrorCode != (short)ErrorResponseCode.NoError) + { + Log.Information("Topic {topic} has error {error}", topic.Name, (ErrorResponseCode)topic.ErrorCode); + } + else + { + var currentPartitions = topicToPartitions.TryGetValue(topic.Name); + if ((currentPartitions == null) || (currentPartitions.Length != topic.Partitions.Count)) + { + currentPartitions = new Partition[topic.Partitions.Count]; + topicToPartitions[topic.Name] = currentPartitions; + } + + foreach (var partition in topic.Partitions) + { + if (partition.ErrorCode != (short)ErrorResponseCode.NoError) + { + Log.Verbose("Topic {topic} partition {partition} has error {error}", topic.Name, partition.PartitionId, (ErrorResponseCode)partition.ErrorCode); + } + currentPartitions[partition.PartitionId] = partition; + } + + previousTopics.Remove(topic.Name); + } + } + + foreach (var oldTopic in previousTopics) + { + topicToPartitions.Remove(oldTopic); + } + } + + private async Task RefreshBrokersAsync(List latestBrokers, CancellationToken token) + { + var previousBrokers = new HashSet(brokers); + var previousConnections = connections.ToDictionary(kvp => kvp.Key, kvp => kvp.Value); + foreach (var broker in latestBrokers) + { + var uri = broker.Address; + + if (!brokers.Contains(uri)) { + brokers.Add(uri); + } else + { + previousBrokers.Remove(uri); + } + + var currentConnection = connections.TryGetValue(broker.BrokerId); + if (currentConnection == null) + { + var newConnection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(uri, token).ConfigureAwait(false); + connections.Add(broker.BrokerId, newConnection); + } else + { + previousConnections.Remove(broker.BrokerId); + } + } + + foreach (var oldBroker in previousBrokers) + { + brokers.Remove(oldBroker); + } + + foreach (var oldConnectionKvp in previousConnections) + { + connections.Remove(oldConnectionKvp.Key); + oldConnectionKvp.Value.Dispose(); + } + } + + public override string ToString() + { + var sb = new StringBuilder(); + sb.Append("Brokers: ").Append(String.Join(", ", brokers)).AppendLine(); + + sb.Append("Connections: ").Append(String.Join(", ", + connections + .OrderBy(kvp => kvp.Key) + .Select(kvp => kvp.Key + ":" + kvp.Value.ServerEndpoint))).AppendLine(); + + sb.Append("Partitions:").AppendLine(); + foreach (var topicKvp in topicToPartitions) + { + sb.Append(topicKvp.Key).Append(String.Join(", ", + topicKvp.Value.Select(p => p.PartitionId + "@" + p.LeaderId))) + .AppendLine(); + + } + + return sb.ToString(); + } + + #region IDisposable Support + private bool disposedValue = false; // To detect redundant calls + + protected virtual void Dispose(bool disposing) + { + if (!disposedValue) + { + if (disposing) + { + foreach (var connection in connections.Values) + { + connection.Dispose(); + } + } + + disposedValue = true; + } + } + + // This code added to correctly implement the disposable pattern. + public void Dispose() + { + // Do not change this code. Put cleanup code in Dispose(bool disposing) above. + Dispose(true); + } + #endregion + } +} diff --git a/src/SimpleKafka/KafkaConnection.cs b/src/SimpleKafka/KafkaConnection.cs new file mode 100644 index 00000000..937977c3 --- /dev/null +++ b/src/SimpleKafka/KafkaConnection.cs @@ -0,0 +1,108 @@ +using System; +using System.Linq; +using System.Net; +using System.Net.Sockets; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class KafkaConnection : IDisposable + { + internal static async Task CreateAsync(IPEndPoint serverEndpoint, CancellationToken token) + { + var client = new TcpClient(serverEndpoint.AddressFamily); + await client.ConnectAsync(serverEndpoint.Address, serverEndpoint.Port).ConfigureAwait(false); + return new KafkaConnection(serverEndpoint, client); + + } + private readonly SemaphoreSlim clientLock = new SemaphoreSlim(1); + + private readonly IPEndPoint serverEndpoint; + public IPEndPoint ServerEndpoint { get { return serverEndpoint; } } + private readonly TcpClient client; + private readonly BigEndianReader reader; + private readonly BigEndianWriter writer; + + private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client) + { + this.serverEndpoint = serverEndpoint; + this.client = client; + var stream = client.GetStream(); + this.reader = new BigEndianReader(stream); + this.writer = new BigEndianWriter(stream); + } + + + private async Task ReceiveResponseAsync(CancellationToken token) + { + var length = await reader.ReadInt32Async(token).ConfigureAwait(false); + var buffer = await reader.ReadBytesAsync(length, token).ConfigureAwait(false); + return buffer; + } + + private async Task CommunicateWithClientAsync(byte[] buffer, int offset, int length, bool expectResponse, CancellationToken token) + { + await clientLock.WaitAsync(token).ConfigureAwait(false); + try + { + + await writer.WriteAsync(buffer, offset, length, token).ConfigureAwait(false); + if (expectResponse) + { + var resultBuffer = await ReceiveResponseAsync(token).ConfigureAwait(false); + return resultBuffer; + } + else + { + return null; + } + } + finally + { + clientLock.Release(); + } + + } + + + public async Task SendRequestAsync(IKafkaRequest request, CancellationToken token) + { + var encoded = request.Encode(); + var resultBuffer = await CommunicateWithClientAsync(encoded, 0, encoded.Length, request.ExpectResponse, token).ConfigureAwait(false); + if (request.ExpectResponse) + { + var result = request.Decode(resultBuffer).Single(); + return result; + } + else + { + return default(T); + } + } + + #region IDisposable Support + private bool disposedValue = false; // To detect redundant calls + + protected virtual void Dispose(bool disposing) + { + if (!disposedValue) + { + if (disposing) + { + client.Close(); + } + disposedValue = true; + } + } + + // This code added to correctly implement the disposable pattern. + public void Dispose() + { + // Do not change this code. Put cleanup code in Dispose(bool disposing) above. + Dispose(true); + } + #endregion + + } +} diff --git a/src/SimpleKafka/KafkaConnectionFactory.cs b/src/SimpleKafka/KafkaConnectionFactory.cs new file mode 100644 index 00000000..d23e313d --- /dev/null +++ b/src/SimpleKafka/KafkaConnectionFactory.cs @@ -0,0 +1,56 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Net; +using System.Text; +using System.Threading; +using System.Threading.Tasks; +using Serilog; +using System.Net.Sockets; +using SimpleKafka.Protocol; + +namespace SimpleKafka +{ + public static class KafkaConnectionFactory + { + public static async Task CreateSimpleKafkaConnectionAsync(Uri address) + { + return await CreateSimpleKafkaConnectionAsync(address, CancellationToken.None).ConfigureAwait(false); + } + + public static async Task CreateSimpleKafkaConnectionAsync(Uri address, CancellationToken token) + { + var ipAddress = await GetFirstAddressAsync(address.Host, token); + var endpoint = new IPEndPoint(ipAddress, address.Port); + var connection = await KafkaConnection.CreateAsync(endpoint, token).ConfigureAwait(false); + return connection; + } + + private static async Task GetFirstAddressAsync(string hostname, CancellationToken token) + { + try + { + //lookup the IP address from the provided host name + var addresses = await Dns.GetHostAddressesAsync(hostname); + + if (addresses.Length > 0) + { + Array.ForEach(addresses, address => Log.Debug("Found address {address} for {hostname}", address, hostname)); + + var selectedAddress = addresses.FirstOrDefault(item => item.AddressFamily == AddressFamily.InterNetwork) ?? addresses.First(); + + Log.Debug("Using address {address} for {hostname}", selectedAddress, hostname); + + return selectedAddress; + } + } + catch + { + throw new UnresolvedHostnameException("Could not resolve the following hostname: {0}", hostname); + } + + throw new UnresolvedHostnameException("Could not resolve the following hostname: {0}", hostname); + } + + } +} diff --git a/src/SimpleKafka/KafkaMessage.cs b/src/SimpleKafka/KafkaMessage.cs new file mode 100644 index 00000000..8fa1d352 --- /dev/null +++ b/src/SimpleKafka/KafkaMessage.cs @@ -0,0 +1,26 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class KafkaMessage + { + private readonly string topic; + public string Topic { get { return topic; } } + private readonly TKey key; + public TKey Key { get { return key; } } + private readonly TValue value; + public TValue Value { get { return value; } } + + public KafkaMessage(string topic, TKey key, TValue value) + { + this.topic = topic; + this.key = key; + this.value = value; + } + + } +} diff --git a/src/SimpleKafka/KafkaProducer.cs b/src/SimpleKafka/KafkaProducer.cs new file mode 100644 index 00000000..765a4e20 --- /dev/null +++ b/src/SimpleKafka/KafkaProducer.cs @@ -0,0 +1,182 @@ +using SimpleKafka.Protocol; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class KafkaProducer + { + private class MessageAndPartition + { + readonly KafkaMessage message; + readonly int partition; + + public MessageAndPartition(KafkaMessage message, int partition) + { + + this.message = message; + this.partition = partition; + } + } + + private readonly KafkaBrokers brokers; + private readonly IKafkaSerializer keySerializer; + private readonly IKafkaSerializer valueSerializer; + private readonly IKafkaMessagePartitioner messagePartitioner; + private readonly int acks = 1; + private readonly int timeoutMs = 10000; + private readonly MessageCodec codec = MessageCodec.CodecNone; + + public KafkaProducer(KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, + IKafkaMessagePartitioner messagePartitioner) + { + this.brokers = brokers; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.messagePartitioner = messagePartitioner; + } + + public async Task SendAsync(KafkaMessage message, CancellationToken token) + { + await SendAsync(new KafkaMessage[] { message }, token); + } + + public async Task SendAsync(IEnumerable> messages, CancellationToken token) + { + var topicMap = BuildTopicMap(messages); + + while (topicMap.Count > 0) + { + var brokerMap = await BuildBrokerMap(token, topicMap); + + var completed = await SendMessagesToBrokersAsync(token, topicMap, brokerMap).ConfigureAwait(false); + if (!completed) + { + var refreshed = await brokers.RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new InvalidOperationException("Failed to refresh"); + } + } + + } + } + + private async Task SendMessagesToBrokersAsync(CancellationToken token, Dictionary>> topicMap, Dictionary, List>> brokerMap) + { + foreach (var brokerKvp in brokerMap) + { + var responses = await ProduceMessagesToBroker(brokerKvp.Key, brokerKvp.Value, token).ConfigureAwait(false); + foreach (var response in responses) + { + switch ((ErrorResponseCode)response.Error) + { + case ErrorResponseCode.NoError: + var partitions = topicMap[response.Topic]; + partitions.Remove(response.PartitionId); + if (partitions.Count == 0) + { + topicMap.Remove(response.Topic); + } + break; + + + case ErrorResponseCode.LeaderNotAvailable: + case ErrorResponseCode.NotLeaderForPartition: + break; + + default: + throw new InvalidOperationException("Unhandled error " + (ErrorResponseCode)response.Error + ", " + response.Topic + ":" + response.PartitionId); + } + } + } + + return brokerMap.Count > 0; + } + + private async Task, List>>> BuildBrokerMap(CancellationToken token, Dictionary>> topicMap) + { + var brokerMap = new Dictionary, List>>(); + foreach (var topicKvp in topicMap) + { + var topic = topicKvp.Key; + var partitions = brokers.GetPartitionsForTopic(topic); + if (partitions == null) + { + brokers.AddTopic(topic); + var refreshed = await brokers.RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new KeyNotFoundException("Failed to refresh brokers"); + } + partitions = brokers.GetPartitionsForTopic(topic); + if (partitions == null) + { + throw new KeyNotFoundException("Failed to find topic: " + topic); + } + } + + foreach (var partitionKvp in topicKvp.Value) + { + var partitionNumber = partitionKvp.Key; + if (partitionNumber >= partitions.Length) + { + throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionNumber + " is too big. Only have " + partitions.Length + " partitions"); + } + + var partition = partitions[partitionNumber]; + var brokerTopics = brokerMap.FindOrCreate(partition.LeaderId); + var topicMessages = brokerTopics.FindOrCreate(Tuple.Create(topic, partitionNumber)); + topicMessages.AddRange(partitionKvp.Value); + } + } + + return brokerMap; + } + + private async Task> ProduceMessagesToBroker(int brokerId, Dictionary,List> topicMessages, CancellationToken token) + { + var payload = new List(topicMessages.Count); + foreach (var kvp in topicMessages) + { + payload.Add(new Payload + { + Topic = kvp.Key.Item1, + Partition = kvp.Key.Item2, + Codec = codec, + Messages = kvp.Value + }); + } + var request = new ProduceRequest + { + Acks = (short)acks, + TimeoutMS = timeoutMs, + Payload = payload, + }; + var response = await brokers[brokerId].SendRequestAsync(request, token).ConfigureAwait(false); + return response; + } + + private Dictionary>> BuildTopicMap(IEnumerable> messages) + { + var topicMap = new Dictionary>>(); + foreach (var message in messages) + { + var partitionMap = topicMap.FindOrCreate(message.Topic); + var partition = messagePartitioner.CalculatePartition(message); + var messageList = partitionMap.FindOrCreate(partition); + var encodedMessage = new Message + { + Key = keySerializer.Serialize(message.Key), + Value = valueSerializer.Serialize(message.Value), + }; + messageList.Add(encodedMessage); + } + return topicMap; + } + } +} diff --git a/src/SimpleKafka/LoadBalancedPartitioner.cs b/src/SimpleKafka/LoadBalancedPartitioner.cs new file mode 100644 index 00000000..630af3ae --- /dev/null +++ b/src/SimpleKafka/LoadBalancedPartitioner.cs @@ -0,0 +1,26 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class LoadBalancedPartitioner : IKafkaMessagePartitioner + { + private readonly int numberOfPartitions; + public LoadBalancedPartitioner(int numberOfPartitions) + { + this.numberOfPartitions = numberOfPartitions; + } + + private int current; + + public int CalculatePartition(KafkaMessage message) + { + var partition = current; + current = (current + 1) % numberOfPartitions; + return partition; + } + } +} diff --git a/src/SimpleKafka/NullSerializer.cs b/src/SimpleKafka/NullSerializer.cs new file mode 100644 index 00000000..2cc5a94d --- /dev/null +++ b/src/SimpleKafka/NullSerializer.cs @@ -0,0 +1,21 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class NullSerializer : IKafkaSerializer + { + public T Deserialize(byte[] serialized) + { + return default(T); + } + + public byte[] Serialize(T value) + { + return null; + } + } +} diff --git a/src/SimpleKafka/Protocol/BaseRequest.cs b/src/SimpleKafka/Protocol/BaseRequest.cs new file mode 100644 index 00000000..35e0081a --- /dev/null +++ b/src/SimpleKafka/Protocol/BaseRequest.cs @@ -0,0 +1,61 @@ +using System; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public abstract class BaseRequest + { + /// + /// From Documentation: + /// The replica id indicates the node id of the replica initiating this request. Normal client consumers should always specify this as -1 as they have no node id. + /// Other brokers set this to be their own node id. The value -2 is accepted to allow a non-broker to issue fetch requests as if it were a replica broker for debugging purposes. + /// + /// Kafka Protocol implementation: + /// https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol + /// + protected const int ReplicaId = -1; + private readonly short _apiVersion; + private string _clientId = "Kafka-Net"; + private int _correlationId = 1; + + protected BaseRequest(short apiVersion = 0) + { + _apiVersion = apiVersion; + } + + /// + /// Descriptive name of the source of the messages sent to kafka + /// + public string ClientId { get { return _clientId; } set { _clientId = value; } } + + /// + /// Value supplied will be passed back in the response by the server unmodified. + /// It is useful for matching request and response between the client and server. + /// + public int CorrelationId { get { return _correlationId; } set { _correlationId = value; } } + + /// + /// Get the API Version for this request + /// + public short ApiVersion { get { return _apiVersion; } } + + /// + /// Flag which tells the broker call to expect a response for this request. + /// + public virtual bool ExpectResponse { get { return true; } } + + /// + /// Encode the common head for kafka request. + /// + /// KafkaMessagePacker with header populated + /// Format: (hhihs) + public static KafkaMessagePacker EncodeHeader(IKafkaRequest request) + { + return new KafkaMessagePacker() + .Pack(((Int16)request.ApiKey)) + .Pack(request.ApiVersion) + .Pack(request.CorrelationId) + .Pack(request.ClientId, StringPrefixEncoding.Int16); + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Broker.cs b/src/SimpleKafka/Protocol/Broker.cs new file mode 100644 index 00000000..85fef582 --- /dev/null +++ b/src/SimpleKafka/Protocol/Broker.cs @@ -0,0 +1,23 @@ +using System; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public class Broker + { + public int BrokerId { get; set; } + public string Host { get; set; } + public int Port { get; set; } + public Uri Address { get { return new Uri(string.Format("http://{0}:{1}", Host, Port));} } + + public static Broker FromStream(BigEndianBinaryReader stream) + { + return new Broker + { + BrokerId = stream.ReadInt32(), + Host = stream.ReadInt16String(), + Port = stream.ReadInt32() + }; + } + } +} diff --git a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs new file mode 100644 index 00000000..5de3c34b --- /dev/null +++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs @@ -0,0 +1,66 @@ +using System; +using System.Collections.Generic; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + /// + /// https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetFetchRequest + /// The offsets for a given consumer group is maintained by a specific broker called the offset coordinator. i.e., a consumer needs + /// to issue its offset commit and fetch requests to this specific broker. It can discover the current offset coordinator by issuing a consumer metadata request. + /// + public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest + { + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.ConsumerMetadataRequest; } } + public string ConsumerGroup { get; set; } + + public byte[] Encode() + { + return EncodeConsumerMetadataRequest(this); + } + + + public IEnumerable Decode(byte[] payload) + { + return DecodeConsumerMetadataResponse(payload); + } + + private byte[] EncodeConsumerMetadataRequest(ConsumerMetadataRequest request) + { + using (var message = EncodeHeader(request).Pack(request.ConsumerGroup, StringPrefixEncoding.Int16)) + { + return message.Payload(); + } + } + + private IEnumerable DecodeConsumerMetadataResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var response = new ConsumerMetadataResponse + { + Error = stream.ReadInt16(), + CoordinatorId = stream.ReadInt32(), + CoordinatorHost = stream.ReadInt16String(), + CoordinatorPort = stream.ReadInt32() + }; + + yield return response; + } + } + } + + public class ConsumerMetadataResponse + { + /// + /// Error code of exception that occured during the request. Zero if no error. + /// + public Int16 Error; + + public int CoordinatorId; + public string CoordinatorHost; + public int CoordinatorPort; + } +} diff --git a/src/SimpleKafka/Protocol/FetchRequest.cs b/src/SimpleKafka/Protocol/FetchRequest.cs new file mode 100644 index 00000000..9bc350de --- /dev/null +++ b/src/SimpleKafka/Protocol/FetchRequest.cs @@ -0,0 +1,160 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public class FetchRequest : BaseRequest, IKafkaRequest + { + internal const int DefaultMinBlockingByteBufferSize = 4096; + internal const int DefaultBufferSize = DefaultMinBlockingByteBufferSize * 8; + internal const int DefaultMaxBlockingWaitTime = 5000; + + /// + /// Indicates the type of kafka encoding this request is + /// + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Fetch; } } + /// + /// The max wait time is the maximum amount of time in milliseconds to block waiting if insufficient data is available at the time the request is issued. + /// + public int MaxWaitTime = DefaultMaxBlockingWaitTime; + /// + /// This is the minimum number of bytes of messages that must be available to give a response. + /// If the client sets this to 0 the server will always respond immediately, however if there is no new data since their last request they will just get back empty message sets. + /// If this is set to 1, the server will respond as soon as at least one partition has at least 1 byte of data or the specified timeout occurs. + /// By setting higher values in combination with the timeout the consumer can tune for throughput and trade a little additional latency for reading only large chunks of data + /// (e.g. setting MaxWaitTime to 100 ms and setting MinBytes to 64k would allow the server to wait up to 100ms to try to accumulate 64k of data before responding). + /// + public int MinBytes = DefaultMinBlockingByteBufferSize; + + public List Fetches { get; set; } + + public byte[] Encode() + { + return EncodeFetchRequest(this); + } + + public IEnumerable Decode(byte[] payload) + { + return DecodeFetchResponse(payload); + } + + private byte[] EncodeFetchRequest(FetchRequest request) + { + if (request.Fetches == null) request.Fetches = new List(); + + using (var message = EncodeHeader(request)) + { + var topicGroups = request.Fetches.GroupBy(x => x.Topic).ToList(); + message.Pack(ReplicaId) + .Pack(request.MaxWaitTime) + .Pack(request.MinBytes) + .Pack(topicGroups.Count); + + foreach (var topicGroup in topicGroups) + { + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) + .Pack(partitions.Count); + + foreach (var partition in partitions) + { + foreach (var fetch in partition) + { + message.Pack(partition.Key) + .Pack(fetch.Offset) + .Pack(fetch.MaxBytes); + } + } + } + + return message.Payload(); + } + } + + private IEnumerable DecodeFetchResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var topicCount = stream.ReadInt32(); + for (int i = 0; i < topicCount; i++) + { + var topic = stream.ReadInt16String(); + + var partitionCount = stream.ReadInt32(); + for (int j = 0; j < partitionCount; j++) + { + var partitionId = stream.ReadInt32(); + var response = new FetchResponse + { + Topic = topic, + PartitionId = partitionId, + Error = stream.ReadInt16(), + HighWaterMark = stream.ReadInt64() + }; + //note: dont use initializer here as it breaks stream position. + response.Messages = Message.DecodeMessageSet(stream.ReadIntPrefixedBytes()) + .Select(x => { x.Meta.PartitionId = partitionId; return x; }) + .ToList(); + yield return response; + } + } + } + } + } + + public class Fetch + { + public Fetch() + { + MaxBytes = FetchRequest.DefaultMinBlockingByteBufferSize * 8; + } + + /// + /// The name of the topic. + /// + public string Topic { get; set; } + /// + /// The id of the partition the fetch is for. + /// + public int PartitionId { get; set; } + /// + /// The offset to begin this fetch from. + /// + public long Offset { get; set; } + /// + /// The maximum bytes to include in the message set for this partition. This helps bound the size of the response. + /// + public int MaxBytes { get; set; } + } + + public class FetchResponse + { + /// + /// The name of the topic this response entry is for. + /// + public string Topic { get; set; } + /// + /// The id of the partition this response is for. + /// + public int PartitionId { get; set; } + /// + /// Error code of exception that occured during the request. Zero if no error. + /// + public Int16 Error { get; set; } + /// + /// The offset at the end of the log for this partition. This can be used by the client to determine how many messages behind the end of the log they are. + /// + public long HighWaterMark { get; set; } + + public List Messages { get; set; } + + public FetchResponse() + { + Messages = new List(); + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Message.cs b/src/SimpleKafka/Protocol/Message.cs new file mode 100644 index 00000000..66d4528d --- /dev/null +++ b/src/SimpleKafka/Protocol/Message.cs @@ -0,0 +1,199 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + /// + /// Payload represents a collection of messages to be posted to a specified Topic on specified Partition. + /// + public class Payload + { + public Payload() + { + Codec = MessageCodec.CodecNone; + } + + public string Topic { get; set; } + public int Partition { get; set; } + public MessageCodec Codec { get; set; } + public List Messages { get; set; } + } + + /// + /// Message represents the data from a single event occurance. + /// + public class Message + { + private const int MessageHeaderSize = 12; + private const long InitialMessageOffset = 0; + + /// + /// Metadata on source offset and partition location for this message. + /// + public MessageMetadata Meta { get; set; } + /// + /// This is a version id used to allow backwards compatible evolution of the message binary format. Reserved for future use. + /// + public byte MagicNumber { get; set; } + /// + /// Attribute value outside message body used for added codec/compression info. + /// + public byte Attribute { get; set; } + /// + /// Key value used for routing message to partitions. + /// + public byte[] Key { get; set; } + /// + /// The message body contents. Can contain compress message set. + /// + public byte[] Value { get; set; } + + /// + /// Construct an empty message. + /// + public Message() { } + + /// + /// Convenience constructor will encode both the key and message to byte streams. + /// Most of the time a message will be string based. + /// + /// The key value for the message. Can be null. + /// The main content data of this message. + public Message(string value, string key = null) + { + Key = key == null ? null : key.ToBytes(); + Value = value.ToBytes(); + } + + /// + /// Encodes a collection of messages into one byte[]. Encoded in order of list. + /// + /// The collection of messages to encode together. + /// Encoded byte[] representing the collection of messages. + public static byte[] EncodeMessageSet(IEnumerable messages) + { + using (var stream = new KafkaMessagePacker()) + { + foreach (var message in messages) + { + stream.Pack(InitialMessageOffset) + .Pack(EncodeMessage(message)); + } + + return stream.PayloadNoLength(); + } + } + + /// + /// Decode a byte[] that represents a collection of messages. + /// + /// The byte[] encode as a message set from kafka. + /// Enumerable representing stream of messages decoded from byte[] + public static IEnumerable DecodeMessageSet(byte[] messageSet) + { + using (var stream = new BigEndianBinaryReader(messageSet)) + { + while (stream.HasData) + { + //this checks that we have at least the minimum amount of data to retrieve a header + if (stream.Available(MessageHeaderSize) == false) + yield break; + + var offset = stream.ReadInt64(); + var messageSize = stream.ReadInt32(); + + //if messagessize is greater than the total payload, our max buffer is insufficient. + if ((stream.Length - MessageHeaderSize) < messageSize) + throw new BufferUnderRunException(MessageHeaderSize, messageSize); + + //if the stream does not have enough left in the payload, we got only a partial message + if (stream.Available(messageSize) == false) + yield break; + + foreach (var message in DecodeMessage(offset, stream.RawRead(messageSize))) + { + yield return message; + } + } + } + } + + /// + /// Encodes a message object to byte[] + /// + /// Message data to encode. + /// Encoded byte[] representation of the message object. + /// + /// Format: + /// Crc (Int32), MagicByte (Byte), Attribute (Byte), Key (Byte[]), Value (Byte[]) + /// + public static byte[] EncodeMessage(Message message) + { + using(var stream = new KafkaMessagePacker()) + { + return stream.Pack(message.MagicNumber) + .Pack(message.Attribute) + .Pack(message.Key) + .Pack(message.Value) + .CrcPayload(); + } + } + + /// + /// Decode messages from a payload and assign it a given kafka offset. + /// + /// The offset represting the log entry from kafka of this message. + /// The byte[] encode as a message from kafka. + /// Enumerable representing stream of messages decoded from byte[]. + /// The return type is an Enumerable as the message could be a compressed message set. + public static IEnumerable DecodeMessage(long offset, byte[] payload) + { + var crc = payload.Take(4).ToArray(); + using (var stream = new BigEndianBinaryReader(payload.Skip(4))) + { + if (crc.SequenceEqual(stream.CrcHash()) == false) + throw new FailCrcCheckException("Payload did not match CRC validation."); + + var message = new Message + { + Meta = new MessageMetadata { Offset = offset }, + MagicNumber = stream.ReadByte(), + Attribute = stream.ReadByte(), + Key = stream.ReadIntPrefixedBytes() + }; + + var codec = (MessageCodec)(ProtocolConstants.AttributeCodeMask & message.Attribute); + switch (codec) + { + case MessageCodec.CodecNone: + message.Value = stream.ReadIntPrefixedBytes(); + yield return message; + break; + default: + throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", codec)); + } + } + } + } + + /// + /// Provides metadata about the message received from the FetchResponse + /// + /// + /// The purpose of this metadata is to allow client applications to track their own offset information about messages received from Kafka. + /// + /// + public class MessageMetadata + { + /// + /// The log offset of this message as stored by the Kafka server. + /// + public long Offset { get; set; } + /// + /// The partition id this offset is from. + /// + public int PartitionId { get; set; } + } +} diff --git a/src/SimpleKafka/Protocol/MetadataRequest.cs b/src/SimpleKafka/Protocol/MetadataRequest.cs new file mode 100644 index 00000000..22e1b64e --- /dev/null +++ b/src/SimpleKafka/Protocol/MetadataRequest.cs @@ -0,0 +1,88 @@ +using System.Collections.Generic; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public class MetadataRequest : BaseRequest, IKafkaRequest + { + /// + /// Indicates the type of kafka encoding this request is + /// + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.MetaData; } } + + /// + /// The list of topics to get metadata for. + /// + public List Topics { get; set; } + + public byte[] Encode() + { + return EncodeMetadataRequest(this); + } + + public IEnumerable Decode(byte[] payload) + { + return new[] { DecodeMetadataResponse(payload) }; + } + + /// + /// Encode a request for metadata about topic and broker information. + /// + /// The MetaDataRequest to encode. + /// Encoded byte[] representing the request. + /// Format: (MessageSize), Header, ix(hs) + private byte[] EncodeMetadataRequest(MetadataRequest request) + { + if (request.Topics == null) request.Topics = new List(); + + using (var message = EncodeHeader(request) + .Pack(request.Topics.Count) + .Pack(request.Topics, StringPrefixEncoding.Int16)) + { + return message.Payload(); + } + } + + /// + /// Decode the metadata response from kafka server. + /// + /// + /// + private MetadataResponse DecodeMetadataResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var response = new MetadataResponse(); + response.CorrelationId = stream.ReadInt32(); + + var brokerCount = stream.ReadInt32(); + for (var i = 0; i < brokerCount; i++) + { + response.Brokers.Add(Broker.FromStream(stream)); + } + + var topicCount = stream.ReadInt32(); + for (var i = 0; i < topicCount; i++) + { + response.Topics.Add(Topic.FromStream(stream)); + } + + return response; + } + } + + } + + public class MetadataResponse + { + public int CorrelationId { get; set; } + public MetadataResponse() + { + Brokers = new List(); + Topics = new List(); + } + + public List Brokers { get; set; } + public List Topics { get; set; } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs new file mode 100644 index 00000000..16469206 --- /dev/null +++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs @@ -0,0 +1,152 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + /// + /// https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetFetchRequest + /// Class that represents the api call to commit a specific set of offsets for a given topic. The offset is saved under the + /// arbitrary ConsumerGroup name provided by the call. + /// This now supports version 0 and 1 of the protocol + /// + public class OffsetCommitRequest : BaseRequest, IKafkaRequest + { + public OffsetCommitRequest(Int16 version = 1) : base(version) + { + } + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetCommit; } } + public string ConsumerGroup { get; set; } + public int ConsumerGroupGenerationId { get; set; } + public string ConsumerId { get; set; } + public List OffsetCommits { get; set; } + + public byte[] Encode() + { + return EncodeOffsetCommitRequest(this); + } + + public IEnumerable Decode(byte[] payload) + { + return DecodeOffsetCommitResponse(payload); + } + + private byte[] EncodeOffsetCommitRequest(OffsetCommitRequest request) + { + if (request.OffsetCommits == null) request.OffsetCommits = new List(); + + using (var message = EncodeHeader(request).Pack(request.ConsumerGroup, StringPrefixEncoding.Int16)) + { + if (ApiVersion == 1) + { + message + .Pack(ConsumerGroupGenerationId) + .Pack(ConsumerId, StringPrefixEncoding.Int16); + } + var topicGroups = request.OffsetCommits.GroupBy(x => x.Topic).ToList(); + message.Pack(topicGroups.Count); + + foreach (var topicGroup in topicGroups) + { + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) + .Pack(partitions.Count); + + foreach (var partition in partitions) + { + foreach (var commit in partition) + { + message + .Pack(partition.Key) + .Pack(commit.Offset); + + if (ApiVersion == 1) + { + message.Pack(commit.TimeStamp); + } + + message + .Pack(commit.Metadata, StringPrefixEncoding.Int16); + } + } + } + + return message.Payload(); + } + } + + private IEnumerable DecodeOffsetCommitResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var topicCount = stream.ReadInt32(); + for (int i = 0; i < topicCount; i++) + { + var topic = stream.ReadInt16String(); + + var partitionCount = stream.ReadInt32(); + for (int j = 0; j < partitionCount; j++) + { + var response = new OffsetCommitResponse() + { + Topic = topic, + PartitionId = stream.ReadInt32(), + Error = stream.ReadInt16() + }; + + yield return response; + } + } + } + } + } + + public class OffsetCommit + { + /// + /// The topic the offset came from. + /// + public string Topic { get; set; } + /// + /// The partition the offset came from. + /// + public int PartitionId { get; set; } + /// + /// The offset number to commit as completed. + /// + public long Offset { get; set; } + /// + /// If the time stamp field is set to -1, then the broker sets the time stamp to the receive time before committing the offset. + /// + public long TimeStamp { get; set; } + /// + /// Descriptive metadata about this commit. + /// + public string Metadata { get; set; } + + public OffsetCommit() + { + TimeStamp = -1; + } + + } + + public class OffsetCommitResponse + { + /// + /// The name of the topic this response entry is for. + /// + public string Topic; + /// + /// The id of the partition this response is for. + /// + public Int32 PartitionId; + /// + /// Error code of exception that occured during the request. Zero if no error. + /// + public Int16 Error; + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs new file mode 100644 index 00000000..88b8b354 --- /dev/null +++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs @@ -0,0 +1,137 @@ +using System; +using System.Collections.Generic; +using System.Linq; + +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + /// + /// Class that represents both the request and the response from a kafka server of requesting a stored offset value + /// for a given consumer group. Essentially this part of the api allows a user to save/load a given offset position + /// under any abritrary name. + /// This now supports version 1 of the protocol + /// + public class OffsetFetchRequest : BaseRequest, IKafkaRequest + { + public OffsetFetchRequest(short version = 1) : base(version) + { + + } + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetFetch; } } + public string ConsumerGroup { get; set; } + public List Topics { get; set; } + + public byte[] Encode() + { + return EncodeOffsetFetchRequest(this); + } + + protected byte[] EncodeOffsetFetchRequest(OffsetFetchRequest request) + { + if (request.Topics == null) request.Topics = new List(); + + using (var message = EncodeHeader(request)) + { + var topicGroups = request.Topics.GroupBy(x => x.Topic).ToList(); + + message.Pack(ConsumerGroup, StringPrefixEncoding.Int16) + .Pack(topicGroups.Count); + + foreach (var topicGroup in topicGroups) + { + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) + .Pack(partitions.Count); + + foreach (var partition in partitions) + { + foreach (var offset in partition) + { + message.Pack(offset.PartitionId); + } + } + } + + return message.Payload(); + } + } + + public IEnumerable Decode(byte[] payload) + { + return DecodeOffsetFetchResponse(payload); + } + + + protected IEnumerable DecodeOffsetFetchResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var topicCount = stream.ReadInt32(); + for (int i = 0; i < topicCount; i++) + { + var topic = stream.ReadInt16String(); + + var partitionCount = stream.ReadInt32(); + for (int j = 0; j < partitionCount; j++) + { + var response = new OffsetFetchResponse() + { + Topic = topic, + PartitionId = stream.ReadInt32(), + Offset = stream.ReadInt64(), + MetaData = stream.ReadInt16String(), + Error = stream.ReadInt16() + }; + yield return response; + } + } + } + } + + } + + public class OffsetFetch + { + /// + /// The topic the offset came from. + /// + public string Topic { get; set; } + /// + /// The partition the offset came from. + /// + public int PartitionId { get; set; } + } + + public class OffsetFetchResponse + { + /// + /// The name of the topic this response entry is for. + /// + public string Topic; + /// + /// The id of the partition this response is for. + /// + public Int32 PartitionId; + /// + /// The offset position saved to the server. + /// + public Int64 Offset; + /// + /// Any arbitrary metadata stored during a CommitRequest. + /// + public string MetaData; + /// + /// Error code of exception that occured during the request. Zero if no error. + /// + public Int16 Error; + + public override string ToString() + { + return string.Format("[OffsetFetchResponse TopicName={0}, PartitionID={1}, Offset={2}, MetaData={3}, ErrorCode={4}]", Topic, PartitionId, Offset, MetaData, Error); + } + + } +} diff --git a/src/SimpleKafka/Protocol/OffsetRequest.cs b/src/SimpleKafka/Protocol/OffsetRequest.cs new file mode 100644 index 00000000..926ddc8b --- /dev/null +++ b/src/SimpleKafka/Protocol/OffsetRequest.cs @@ -0,0 +1,154 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + /// + /// A funky Protocol for requesting the starting offset of each segment for the requested partition + /// + public class OffsetRequest : BaseRequest, IKafkaRequest + { + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Offset; } } + public List Offsets { get; set; } + + public byte[] Encode() + { + return EncodeOffsetRequest(this); + } + + public IEnumerable Decode(byte[] payload) + { + return DecodeOffsetResponse(payload); + } + + private byte[] EncodeOffsetRequest(OffsetRequest request) + { + if (request.Offsets == null) request.Offsets = new List(); + using (var message = EncodeHeader(request)) + { + var topicGroups = request.Offsets.GroupBy(x => x.Topic).ToList(); + message.Pack(ReplicaId) + .Pack(topicGroups.Count); + + foreach (var topicGroup in topicGroups) + { + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) + .Pack(partitions.Count); + + foreach (var partition in partitions) + { + foreach (var offset in partition) + { + message.Pack(partition.Key) + .Pack(offset.Time) + .Pack(offset.MaxOffsets); + } + } + } + + return message.Payload(); + } + } + + + private IEnumerable DecodeOffsetResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var topicCount = stream.ReadInt32(); + for (int i = 0; i < topicCount; i++) + { + var topic = stream.ReadInt16String(); + + var partitionCount = stream.ReadInt32(); + for (int j = 0; j < partitionCount; j++) + { + var response = new OffsetResponse() + { + Topic = topic, + PartitionId = stream.ReadInt32(), + Error = stream.ReadInt16(), + Offsets = new List() + }; + var offsetCount = stream.ReadInt32(); + for (int k = 0; k < offsetCount; k++) + { + response.Offsets.Add(stream.ReadInt64()); + } + + yield return response; + } + } + } + } + } + + public class Offset + { + public Offset() + { + Time = -1; + MaxOffsets = 1; + } + public string Topic { get; set; } + public int PartitionId { get; set; } + /// + /// Used to ask for all messages before a certain time (ms). There are two special values. + /// Specify -1 to receive the latest offsets and -2 to receive the earliest available offset. + /// Note that because offsets are pulled in descending order, asking for the earliest offset will always return you a single element. + /// + public long Time { get; set; } + public int MaxOffsets { get; set; } + } + + public class OffsetResponse + { + public string Topic { get; set; } + public int PartitionId { get; set; } + public Int16 Error { get; set; } + public List Offsets { get; set; } + } + + public class OffsetPosition + { + public OffsetPosition() { } + public OffsetPosition(int partitionId, long offset) + { + PartitionId = partitionId; + Offset = offset; + } + public int PartitionId { get; set; } + public long Offset { get; set; } + + public override string ToString() + { + return string.Format("PartitionId:{0}, Offset:{1}", PartitionId, Offset); + } + + public override bool Equals(object obj) + { + if (ReferenceEquals(null, obj)) return false; + if (ReferenceEquals(this, obj)) return true; + if (obj.GetType() != this.GetType()) return false; + return Equals((OffsetPosition)obj); + } + + protected bool Equals(OffsetPosition other) + { + return PartitionId == other.PartitionId && Offset == other.Offset; + } + + public override int GetHashCode() + { + unchecked + { + return (PartitionId * 397) ^ Offset.GetHashCode(); + } + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/ProduceRequest.cs b/src/SimpleKafka/Protocol/ProduceRequest.cs new file mode 100644 index 00000000..d7ed4e96 --- /dev/null +++ b/src/SimpleKafka/Protocol/ProduceRequest.cs @@ -0,0 +1,155 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public class ProduceRequest : BaseRequest, IKafkaRequest> + { + /// + /// Provide a hint to the broker call not to expect a response for requests without Acks. + /// + public override bool ExpectResponse { get { return Acks > 0; } } + /// + /// Indicates the type of kafka encoding this request is. + /// + public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Produce; } } + /// + /// Time kafka will wait for requested ack level before returning. + /// + public int TimeoutMS = 1000; + /// + /// Level of ack required by kafka. 0 immediate, 1 written to leader, 2+ replicas synced, -1 all replicas + /// + public Int16 Acks = 1; + /// + /// Collection of payloads to post to kafka + /// + public List Payload = new List(); + + + public byte[] Encode() + { + return EncodeProduceRequest(this); + } + + public IEnumerable> Decode(byte[] payload) + { + yield return DecodeProduceResponse(payload); + } + + #region Protocol... + private byte[] EncodeProduceRequest(ProduceRequest request) + { + if (request.Payload == null) request.Payload = new List(); + + var groupedPayloads = (from p in request.Payload + group p by new + { + p.Topic, + p.Partition, + p.Codec + } into tpc + select tpc).ToList(); + + using (var message = EncodeHeader(request) + .Pack(request.Acks) + .Pack(request.TimeoutMS) + .Pack(groupedPayloads.Count)) + { + foreach (var groupedPayload in groupedPayloads) + { + var payloads = groupedPayload.ToList(); + message.Pack(groupedPayload.Key.Topic, StringPrefixEncoding.Int16) + .Pack(payloads.Count) + .Pack(groupedPayload.Key.Partition); + + switch (groupedPayload.Key.Codec) + { + case MessageCodec.CodecNone: + message.Pack(Message.EncodeMessageSet(payloads.SelectMany(x => x.Messages))); + break; + default: + throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", groupedPayload.Key.Codec)); + } + } + + return message.Payload(); + } + } + private IEnumerable DecodeProduceResponse(byte[] data) + { + using (var stream = new BigEndianBinaryReader(data)) + { + var correlationId = stream.ReadInt32(); + + var topicCount = stream.ReadInt32(); + for (int i = 0; i < topicCount; i++) + { + var topic = stream.ReadInt16String(); + + var partitionCount = stream.ReadInt32(); + for (int j = 0; j < partitionCount; j++) + { + var response = new ProduceResponse() + { + Topic = topic, + PartitionId = stream.ReadInt32(), + Error = stream.ReadInt16(), + Offset = stream.ReadInt64() + }; + + yield return response; + } + } + } + } + #endregion + } + + public class ProduceResponse + { + /// + /// The topic the offset came from. + /// + public string Topic { get; set; } + /// + /// The partition the offset came from. + /// + public int PartitionId { get; set; } + /// + /// Error response code. 0 is success. + /// + public Int16 Error { get; set; } + /// + /// The offset number to commit as completed. + /// + public long Offset { get; set; } + + public override bool Equals(object obj) + { + if (ReferenceEquals(null, obj)) return false; + if (ReferenceEquals(this, obj)) return true; + if (obj.GetType() != this.GetType()) return false; + return Equals((ProduceResponse)obj); + } + + protected bool Equals(ProduceResponse other) + { + return string.Equals(Topic, other.Topic) && PartitionId == other.PartitionId && Error == other.Error && Offset == other.Offset; + } + + public override int GetHashCode() + { + unchecked + { + int hashCode = (Topic != null ? Topic.GetHashCode() : 0); + hashCode = (hashCode * 397) ^ PartitionId; + hashCode = (hashCode * 397) ^ Error.GetHashCode(); + hashCode = (hashCode * 397) ^ Offset.GetHashCode(); + return hashCode; + } + } + } +} \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Protocol.cs b/src/SimpleKafka/Protocol/Protocol.cs new file mode 100644 index 00000000..d9ad314a --- /dev/null +++ b/src/SimpleKafka/Protocol/Protocol.cs @@ -0,0 +1,211 @@ +using System; +using System.IO; + +namespace SimpleKafka.Protocol +{ + /// + /// Enumeration of numeric codes that the ApiKey in the request can take for each request types. + /// + public enum ApiKeyRequestType + { + Produce = 0, + Fetch = 1, + Offset = 2, + MetaData = 3, + OffsetCommit = 8, + OffsetFetch = 9, + ConsumerMetadataRequest = 10 + } + + /// + /// Enumeration of error codes that might be returned from a Kafka server + /// + public enum ErrorResponseCode : short + { + /// + /// No error--it worked! + /// + NoError = 0, + + /// + /// An unexpected server error + /// + Unknown = -1, + + /// + /// The requested offset is outside the range of offsets maintained by the server for the given topic/partition. + /// + OffsetOutOfRange = 1, + + /// + /// This indicates that a message contents does not match its CRC + /// + InvalidMessage = 2, + + /// + /// This request is for a topic or partition that does not exist on this broker. + /// + UnknownTopicOrPartition = 3, + + /// + /// The message has a negative size + /// + InvalidMessageSize = 4, + + /// + /// This error is thrown if we are in the middle of a leadership election and there is currently no leader for this partition and hence it is unavailable for writes. + /// + LeaderNotAvailable = 5, + + /// + /// This error is thrown if the client attempts to send messages to a replica that is not the leader for some partition. It indicates that the clients metadata is out of date. + /// + NotLeaderForPartition = 6, + + /// + /// This error is thrown if the request exceeds the user-specified time limit in the request. + /// + RequestTimedOut = 7, + + /// + /// This is not a client facing error and is used only internally by intra-cluster broker communication. + /// + BrokerNotAvailable = 8, + + /// + /// If replica is expected on a broker, but is not. + /// + ReplicaNotAvailable = 9, + + /// + /// The server has a configurable maximum message size to avoid unbounded memory allocation. This error is thrown if the client attempt to produce a message larger than this maximum. + /// + MessageSizeTooLarge = 10, + + /// + /// Internal error code for broker-to-broker communication. + /// + StaleControllerEpochCode = 11, + + /// + /// If you specify a string larger than configured maximum for offset metadata + /// + OffsetMetadataTooLargeCode = 12, + + /// + /// The broker returns this error code for an offset fetch request if it is still loading offsets (after a leader change for that offsets topic partition). + /// + OffsetsLoadInProgressCode = 14, + + /// + /// The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has not yet been created. + /// + ConsumerCoordinatorNotAvailableCode = 15, + + /// + /// The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is not a coordinator for. + /// + NotCoordinatorForConsumerCode = 16 + } + + /// + /// Protocol specific constants + /// + public struct ProtocolConstants + { + /// + /// The lowest 2 bits contain the compression codec used for the message. The other bits should be set to 0. + /// + public static byte AttributeCodeMask = 0x03; + } + + /// + /// Enumeration which specifies the compression type of messages + /// + public enum MessageCodec + { + CodecNone = 0x00, + CodecGzip = 0x01, + CodecSnappy = 0x02 + } + + #region Exceptions... + public class FailCrcCheckException : Exception + { + public FailCrcCheckException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class ResponseTimeoutException : Exception + { + public ResponseTimeoutException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class InvalidPartitionException : Exception + { + public InvalidPartitionException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class ServerDisconnectedException : Exception + { + public ServerDisconnectedException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class ServerUnreachableException : Exception + { + public ServerUnreachableException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class InvalidTopicMetadataException : Exception + { + public InvalidTopicMetadataException(ErrorResponseCode code, string message, params object[] args) + : base(string.Format(message, args)) + { + ErrorResponseCode = code; + } + public ErrorResponseCode ErrorResponseCode { get; private set; } + } + + public class LeaderNotFoundException : Exception + { + public LeaderNotFoundException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class UnresolvedHostnameException : Exception + { + public UnresolvedHostnameException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class InvalidMetadataException : Exception + { + public int ErrorCode { get; set; } + public InvalidMetadataException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class OffsetOutOfRangeException : Exception + { + public Fetch FetchRequest { get; set; } + public OffsetOutOfRangeException(string message, params object[] args) : base(string.Format(message, args)) { } + } + + public class BufferUnderRunException : Exception + { + public int MessageHeaderSize { get; set; } + public int RequiredBufferSize { get; set; } + + public BufferUnderRunException(int messageHeaderSize, int requiredBufferSize) + : base("The size of the message from Kafka exceeds the provide buffer size.") + { + MessageHeaderSize = messageHeaderSize; + RequiredBufferSize = requiredBufferSize; + } + } + + public class KafkaException : Exception + { + public int ErrorCode { get; set; } + public KafkaException(string message, params object[] args) : base(string.Format(message, args)) { } + } + #endregion + + +} diff --git a/src/SimpleKafka/Protocol/Topic.cs b/src/SimpleKafka/Protocol/Topic.cs new file mode 100644 index 00000000..8abc1167 --- /dev/null +++ b/src/SimpleKafka/Protocol/Topic.cs @@ -0,0 +1,99 @@ +using System; +using System.Collections.Generic; +using SimpleKafka.Common; + +namespace SimpleKafka.Protocol +{ + public class Topic + { + public Int16 ErrorCode { get; set; } + public string Name { get; set; } + public List Partitions { get; set; } + + public static Topic FromStream(BigEndianBinaryReader stream) + { + var topic = new Topic + { + ErrorCode = stream.ReadInt16(), + Name = stream.ReadInt16String(), + Partitions = new List() + }; + + var numPartitions = stream.ReadInt32(); + for (int i = 0; i < numPartitions; i++) + { + topic.Partitions.Add(Partition.FromStream(stream)); + } + + return topic; + } + } + + public class Partition + { + /// + /// Error code. 0 indicates no error occured. + /// + public Int16 ErrorCode { get; set; } + /// + /// The Id of the partition that this metadata describes. + /// + public int PartitionId { get; set; } + /// + /// The node id for the kafka broker currently acting as leader for this partition. If no leader exists because we are in the middle of a leader election this id will be -1. + /// + public int LeaderId { get; set; } + /// + /// The set of alive nodes that currently acts as slaves for the leader for this partition. + /// + public List Replicas { get; set; } + /// + /// The set subset of the replicas that are "caught up" to the leader + /// + public List Isrs { get; set; } + + public static Partition FromStream(BigEndianBinaryReader stream) + { + var partition = new Partition { + ErrorCode = stream.ReadInt16(), + PartitionId = stream.ReadInt32(), + LeaderId = stream.ReadInt32(), + Replicas = new List(), + Isrs = new List() + }; + + var numReplicas = stream.ReadInt32(); + for (int i = 0; i < numReplicas; i++) + { + partition.Replicas.Add(stream.ReadInt32()); + } + + var numIsr = stream.ReadInt32(); + for (int i = 0; i < numIsr; i++) + { + partition.Isrs.Add(stream.ReadInt32()); + } + + return partition; + } + + protected bool Equals(Partition other) + { + return PartitionId == other.PartitionId; + } + + public override int GetHashCode() + { + return PartitionId; + } + + public override bool Equals(object obj) + { + if (ReferenceEquals(null, obj)) return false; + if (ReferenceEquals(this, obj)) return true; + if (obj.GetType() != this.GetType()) return false; + return Equals((Partition) obj); + } + } + +} diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj new file mode 100644 index 00000000..960f922e --- /dev/null +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -0,0 +1,101 @@ + + + + + Debug + AnyCPU + {DB719BEC-843A-4A5E-A151-EC47FEC2EE48} + Library + Properties + SimpleKafka + SimpleKafka + v4.5 + 512 + ..\ + true + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + + + + ..\packages\Serilog.1.4.214\lib\net45\Serilog.dll + + + ..\packages\Serilog.1.4.214\lib\net45\Serilog.FullNetFx.dll + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This project references NuGet package(s) that are missing on this computer. Use NuGet Package Restore to download them. For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}. + + + + + \ No newline at end of file diff --git a/src/SimpleKafka/StringSerializer.cs b/src/SimpleKafka/StringSerializer.cs new file mode 100644 index 00000000..1136a574 --- /dev/null +++ b/src/SimpleKafka/StringSerializer.cs @@ -0,0 +1,21 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class StringSerializer : IKafkaSerializer + { + public string Deserialize(byte[] serialized) + { + return Encoding.UTF8.GetString(serialized); + } + + public byte[] Serialize(string value) + { + return Encoding.UTF8.GetBytes(value); + } + } +} diff --git a/src/SimpleKafka/packages.config b/src/SimpleKafka/packages.config new file mode 100644 index 00000000..592b22a0 --- /dev/null +++ b/src/SimpleKafka/packages.config @@ -0,0 +1,4 @@ + + + + \ No newline at end of file diff --git a/src/kafka-net/Consumer.cs b/src/kafka-net/Consumer.cs old mode 100644 new mode 100755 From 2637cb474aa90c214c055ef72da596706832a242 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Wed, 8 Apr 2015 20:39:11 +0100 Subject: [PATCH 03/14] Tests for simple kafka - derived from kafka-net --- src/SimpleKafkaTests/App.config | 7 + .../Helpers/IntegrationConfig.cs | 26 + src/SimpleKafkaTests/Helpers/MessageHelper.cs | 27 + .../Integration/SimpleTests.cs | 475 ++++++++++++++++++ .../Properties/AssemblyInfo.cs | 36 ++ src/SimpleKafkaTests/SimpleKafkaTests.csproj | 85 ++++ .../Unit/BigEndianBinaryReaderTests.cs | 160 ++++++ .../Unit/BigEndianBinaryWriterTests.cs | 236 +++++++++ .../Unit/ProtocolBaseRequestTests.cs | 23 + .../Unit/ProtocolMessageTests.cs | 115 +++++ src/SimpleKafkaTests/packages.config | 4 + 11 files changed, 1194 insertions(+) create mode 100644 src/SimpleKafkaTests/App.config create mode 100644 src/SimpleKafkaTests/Helpers/IntegrationConfig.cs create mode 100644 src/SimpleKafkaTests/Helpers/MessageHelper.cs create mode 100644 src/SimpleKafkaTests/Integration/SimpleTests.cs create mode 100644 src/SimpleKafkaTests/Properties/AssemblyInfo.cs create mode 100644 src/SimpleKafkaTests/SimpleKafkaTests.csproj create mode 100644 src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs create mode 100644 src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs create mode 100644 src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs create mode 100644 src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs create mode 100644 src/SimpleKafkaTests/packages.config diff --git a/src/SimpleKafkaTests/App.config b/src/SimpleKafkaTests/App.config new file mode 100644 index 00000000..a2c5cec4 --- /dev/null +++ b/src/SimpleKafkaTests/App.config @@ -0,0 +1,7 @@ + + + + + + + \ No newline at end of file diff --git a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs new file mode 100644 index 00000000..ab7c214d --- /dev/null +++ b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs @@ -0,0 +1,26 @@ +using System; +using System.Collections.Generic; +using System.Configuration; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Helpers +{ + public static class IntegrationConfig + { + public static string IntegrationCompressionTopic = "IntegrationCompressionTopic"; + public static string IntegrationTopic = "IntegrationTopic"; + public static string IntegrationConsumer = "IntegrationConsumer"; + public static Uri IntegrationUri = new Uri("http://server.home:9092"); + public static Uri IntegrationUriOld + { + get + { + var url = ConfigurationManager.AppSettings["IntegrationKafkaServerUrl"]; + if (url == null) throw new ConfigurationErrorsException("IntegrationKafkaServerUrl must be specified in the app.config file."); + return new Uri(url); + } + } + } +} diff --git a/src/SimpleKafkaTests/Helpers/MessageHelper.cs b/src/SimpleKafkaTests/Helpers/MessageHelper.cs new file mode 100644 index 00000000..24b1f89e --- /dev/null +++ b/src/SimpleKafkaTests/Helpers/MessageHelper.cs @@ -0,0 +1,27 @@ +using System; +using SimpleKafka.Common; +using SimpleKafka.Protocol; + +namespace SimpleKafkaTests.Helpers +{ + public static class MessageHelper + { + public static byte[] FetchResponseMaxBytesOverflow = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 18, 89, 42, 71, 135, 0, 0, 255, 255, 255, 255, 0, 0, 0, 4, 116, 101, 115, 116, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 50, 2, 216, 134, 114, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 55, 49, 54, 53, 49, 100, 51, 45, 101, 98, 98, 56, 45, 52, 54, 100, 101, 45, 97, 99, 99, 102, 45, 57, 56, 57, 100, 98, 49, 54, 51, 98, 49, 53, 98, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 50, 201, 74, 111, 65, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 56, 97, 49, 53, 49, 99, 52, 45, 99, 48, 49, 52, 45, 52, 57, 97, 49, 45, 56, 49, 49, 49, 45, 52, 48, 99, 56, 52, 48, 100, 48, 55, 49, 97, 49, 0, 0, 0, 0, 0, 0, 0, 3, 0, 0, 0, 50, 108, 133, 219, 125, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 101, 52, 102, 100, 52, 52, 100, 45, 56, 55, 57, 51, 45, 52, 98, 97, 51, 45, 98, 50, 100, 97, 45, 99, 54, 102, 55, 102, 99, 50, 99, 52, 55, 102, 101, 0, 0, 0, 0, 0, 0, 0, 4, 0, 0, 0, 50, 13, 180, 181, 64, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 102, 101, 53, 49, 97, 53, 102, 45, 50, 102, 54, 48, 45, 52, 49, 100, 49, 45, 97, 97, 57, 54, 45, 56, 100, 52, 97, 52, 53, 98, 50, 57, 102, 97, 51, 0, 0, 0, 0, 0, 0, 0, 5, 0, 0, 0, 50, 17, 47, 29, 202, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 54, 49, 50, 52, 50, 99, 57, 45, 54, 52, 51, 100, 45, 52, 48, 53, 50, 45, 57, 54, 97, 51, 45, 56, 56, 50, 100, 98, 49, 99, 52, 56, 53, 52, 100, 0, 0, 0, 0, 0, 0, 0, 6, 0, 0, 0, 50, 206, 168, 194, 202, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 51, 98, 97, 51, 99, 48, 55, 45, 100, 54, 98, 57, 45, 52, 98, 98, 54, 45, 56, 55, 99, 100, 45, 55, 54, 101, 97, 57, 53, 97, 52, 51, 48, 48, 53, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 50, 244, 216, 183, 43, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 55, 56, 51, 54, 57, 100, 99, 45, 50, 99, 52, 55, 45, 52, 50, 100, 57, 45, 97, 52, 56, 55, 45, 50, 98, 102, 51, 55, 102, 98, 55, 97, 50, 55, 57, 0, 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, 50, 247, 214, 81, 136, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 97, 98, 54, 52, 48, 49, 55, 45, 100, 101, 48, 98, 45, 52, 98, 49, 56, 45, 57, 55, 56, 54, 45, 49, 53, 100, 54, 97, 100, 57, 99, 50, 102, 51, 97, 0, 0, 0, 0, 0, 0, 0, 9, 0, 0, 0, 50, 12, 160, 252, 213, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 52, 51, 51, 56, 49, 52, 97, 45, 54, 54, 52, 101, 45, 52, 54, 53, 97, 45, 56, 49, 98, 52, 45, 102, 54, 51, 54, 100, 48, 98, 49, 52, 49, 97, 99, 0, 0, 0, 0, 0, 0, 0, 10, 0, 0, 0, 50, 108, 175, 147, 38, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 54, 52, 54, 57, 98, 56, 54, 45, 54, 55, 49, 56, 45, 52, 54, 48, 56, 45, 57, 57, 53, 48, 45, 54, 48, 101, 97, 52, 55, 55, 50, 54, 55, 102, 51, 0, 0, 0, 0, 0, 0, 0, 11, 0, 0, 0, 50, 178, 100, 177, 129, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 98, 56, 50, 48, 53, 53, 48, 45, 54, 51, 52, 98, 45, 52, 50, 48, 98, 45, 57, 50, 57, 100, 45, 101, 53, 99, 100, 56, 56, 101, 51, 98, 97, 51, 98, 0, 0, 0, 0, 0, 0, 0, 12, 0, 0, 0, 50, 37, 241, 90, 32, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 99, 100, 99, 102, 99, 51, 53, 45, 51, 56, 100, 49, 45, 52, 51, 102, 52, 45, 98, 99, 49, 49, 45, 102, 100, 101, 100, 97, 101, 51, 100, 101, 99, 48, 50, 0, 0, 0, 0, 0, 0, 0, 13, 0, 0, 0, 50, 211, 149, 163, 233, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 57, 100, 56, 100, 101, 52, 53, 45, 55, 51, 99, 56, 45, 52, 55, 56, 49, 45, 98, 101, 56, 52, 45, 102, 56, 99, 56, 98, 100, 99, 100, 56, 49, 102, 48, 0, 0, 0, 0, 0, 0, 0, 14, 0, 0, 0, 50, 225, 104, 33, 46, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 102, 49, 101, 56, 55, 101, 54, 45, 97, 101, 102, 99, 45, 52, 99, 50, 51, 45, 98, 54, 102, 100, 45, 101, 97, 50, 53, 98, 54, 97, 48, 57, 98, 56, 97, 0, 0, 0, 0, 0, 0, 0, 15, 0, 0, 0, 50, 34, 55, 211, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 97, 56, 102, 55, 53, 55, 97, 45, 50, 50, 97, 52, 45, 52, 50, 57, 53, 45, 56, 100, 97, 54, 45, 56, 98, 56, 102, 53, 100, 54, 55, 54, 50, 57, 57, 0, 0, 0, 0, 0, 0, 0, 16, 0, 0, 0, 50, 128, 172, 178, 94, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 98, 51, 100, 54, 100, 52, 101, 45, 98, 51, 55, 52, 45, 52, 48, 57, 49, 45, 98, 53, 99, 99, 45, 100, 54, 100, 51, 53, 53, 49, 51, 48, 50, 57, 55, 0, 0, 0, 0, 0, 0, 0, 17, 0, 0, 0, 50, 175, 37, 198, 220, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 48, 54, 48, 52, 101, 50, 100, 45, 54, 50, 48, 53, 45, 52, 51, 56, 56, 45, 57, 54, 99, 49, 45, 54, 57, 49, 53, 101, 50, 55, 98, 50, 49, 100, 52, 0, 0, 0, 0, 0, 0, 0, 18, 0, 0, 0, 50, 176, 223, 58, 185, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 101, 55, 102, 48, 99, 54, 101, 45, 52, 51, 53, 101, 45, 52, 55, 54, 52, 45, 56, 99, 102, 55, 45, 98, 52, 100, 54, 53, 98, 101, 50, 51, 52, 52, 56, 0, 0, 0, 0, 0, 0, 0, 19, 0, 0, 0, 50, 5, 160, 86, 154, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 51, 97, 51, 53, 100, 101, 54, 45, 54, 101, 53, 52, 45, 52, 102, 53, 56, 45, 57, 49, 102, 49, 45, 102, 98, 97, 102, 54, 53, 51, 102, 52, 56, 51, 50, 0, 0, 0, 0, 0, 0, 0, 20, 0, 0, 0, 50, 211, 25, 232, 236, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 49, 100, 48, 53, 56, 49, 99, 45, 97, 101, 101, 102, 45, 52, 98, 53, 51, 45, 57, 56, 50, 53, 45, 56, 55, 49, 56, 54, 54, 98, 52, 57, 52, 97, 98, 0, 0, 0, 0, 0, 0, 0, 21, 0, 0, 0, 50, 32, 119, 95, 169, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 100, 97, 99, 48, 52, 101, 51, 45, 100, 100, 100, 52, 45, 52, 101, 99, 98, 45, 97, 54, 53, 100, 45, 52, 50, 97, 54, 54, 53, 54, 52, 51, 51, 101, 55, 0, 0, 0, 0, 0, 0, 0, 22, 0, 0, 0, 50, 211, 170, 191, 186, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 53, 57, 48, 50, 50, 97, 45, 102, 48, 53, 101, 45, 52, 57, 57, 97, 45, 56, 51, 55, 53, 45, 48, 100, 55, 55, 97, 49, 57, 56, 102, 102, 55, 101, 0, 0, 0, 0, 0, 0, 0, 23, 0, 0, 0, 50, 142, 88, 203, 71, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 52, 55, 51, 97, 97, 102, 48, 45, 56, 52, 48, 53, 45, 52, 48, 53, 51, 45, 97, 55, 102, 57, 45, 53, 54, 57, 57, 57, 49, 50, 56, 54, 54, 52, 98, 0, 0, 0, 0, 0, 0, 0, 24, 0, 0, 0, 50, 251, 143, 70, 88, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 100, 102, 97, 53, 51, 100, 99, 45, 50, 100, 99, 48, 45, 52, 102, 97, 54, 45, 98, 48, 53, 98, 45, 51, 56, 101, 49, 56, 102, 49, 99, 48, 56, 100, 53, 0, 0, 0, 0, 0, 0, 0, 25, 0, 0, 0, 50, 235, 160, 167, 132, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 100, 98, 99, 50, 56, 102, 50, 45, 52, 97, 100, 98, 45, 52, 100, 98, 101, 45, 56, 100, 53, 97, 45, 54, 57, 51, 48, 99, 102, 48, 50, 97, 48, 100, 100, 0, 0, 0, 0, 0, 0, 0, 26, 0, 0, 0, 50, 245, 164, 170, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 55, 48, 97, 55, 56, 100, 48, 45, 48, 99, 50, 56, 45, 52, 97, 54, 101, 45, 57, 53, 50, 56, 45, 98, 55, 97, 53, 49, 101, 100, 51, 54, 97, 52, 55, 0, 0, 0, 0, 0, 0, 0, 27, 0, 0, 0, 50, 48, 99, 158, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 102, 97, 55, 55, 97, 98, 48, 45, 100, 102, 97, 100, 45, 52, 100, 98, 50, 45, 57, 50, 48, 49, 45, 52, 56, 99, 48, 99, 53, 56, 55, 52, 101, 54, 97, 0, 0, 0, 0, 0, 0, 0, 28, 0, 0, 0, 50, 41, 193, 178, 44, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 99, 56, 99, 57, 50, 97, 50, 45, 50, 54, 99, 56, 45, 52, 100, 57, 100, 45, 56, 97, 101, 56, 45, 48, 102, 100, 102, 55, 50, 56, 56, 97, 49, 97, 98, 0, 0, 0, 0, 0, 0, 0, 29, 0, 0, 0, 50, 88, 149, 18, 29, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 49, 99, 102, 55, 51, 98, 53, 45, 98, 99, 102, 102, 45, 52, 54, 56, 51, 45, 56, 52, 56, 56, 45, 99, 55, 50, 48, 52, 99, 55, 54, 55, 48, 57, 50, 0, 0, 0, 0, 0, 0, 0, 30, 0, 0, 0, 50, 34, 89, 128, 5, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 101, 56, 52, 54, 48, 102, 100, 45, 51, 57, 56, 50, 45, 52, 50, 97, 98, 45, 98, 55, 99, 48, 45, 48, 50, 99, 97, 51, 97, 48, 99, 101, 48, 54, 97, 0, 0, 0, 0, 0, 0, 0, 31, 0, 0, 0, 50, 26, 45, 195, 128, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 56, 49, 52, 48, 57, 55, 100, 45, 57, 57, 99, 54, 45, 52, 55, 53, 50, 45, 57, 54, 99, 51, 45, 56, 50, 54, 101, 98, 100, 49, 97, 57, 56, 57, 56, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 50, 185, 4, 117, 24, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 52, 56, 51, 50, 53, 49, 45, 52, 56, 56, 99, 45, 52, 56, 55, 51, 45, 56, 49, 55, 100, 45, 49, 56, 53, 52, 101, 54, 49, 55, 100, 49, 101, 50, 0, 0, 0, 0, 0, 0, 0, 33, 0, 0, 0, 50, 70, 197, 55, 55, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 53, 49, 50, 48, 52, 57, 100, 45, 100, 100, 100, 97, 45, 52, 57, 100, 99, 45, 97, 48, 52, 101, 45, 53, 53, 102, 53, 102, 49, 54, 102, 97, 102, 53, 48, 0, 0, 0, 0, 0, 0, 0, 34, 0, 0, 0, 50, 201, 166, 201, 125, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 53, 52, 52, 51, 101, 102, 99, 45, 98, 102, 48, 48, 45, 52, 102, 101, 97, 45, 97, 48, 52, 52, 45, 53, 101, 102, 101, 98, 98, 98, 54, 101, 54, 49, 99, 0, 0, 0, 0, 0, 0, 0, 35, 0, 0, 0, 50, 223, 99, 126, 4, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 48, 101, 50, 99, 98, 57, 55, 45, 53, 49, 52, 102, 45, 52, 56, 51, 97, 45, 57, 52, 97, 97, 45, 51, 57, 51, 101, 102, 50, 51, 97, 52, 53, 49, 49, 0, 0, 0, 0, 0, 0, 0, 36, 0, 0, 0, 50, 155, 181, 254, 91, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 53, 97, 51, 98, 102, 49, 56, 45, 53, 98, 97, 54, 45, 52, 97, 49, 53, 45, 57, 101, 97, 100, 45, 102, 56, 52, 101, 101, 54, 56, 102, 54, 50, 52, 53, 0, 0, 0, 0, 0, 0, 0, 37, 0, 0, 0, 50, 197, 98, 158, 34, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 56, 57, 48, 52, 56, 49, 99, 45, 99, 101, 52, 102, 45, 52, 98, 54, 48, 45, 56, 51, 101, 57, 45, 51, 57, 98, 53, 57, 57, 97, 100, 57, 55, 55, 51, 0, 0, 0, 0, 0, 0, 0, 38, 0, 0, 0, 50, 70, 114, 198, 61, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 51, 57, 53, 48, 57, 49, 97, 45, 99, 48, 55, 102, 45, 52, 98, 97, 53, 45, 98, 97, 98, 98, 45, 101, 53, 98, 98, 54, 49, 100, 51, 53, 52, 53, 101, 0, 0, 0, 0, 0, 0, 0, 39, 0, 0, 0, 50, 67, 119, 50, 9, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 52, 51, 102, 101, 57, 98, 100, 45, 99, 49, 100, 54, 45, 52, 56, 51, 55, 45, 98, 53, 51, 99, 45, 55, 55, 57, 49, 56, 99, 57, 50, 52, 54, 97, 100, 0, 0, 0, 0, 0, 0, 0, 40, 0, 0, 0, 50, 160, 81, 66, 91, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 97, 53, 54, 57, 57, 55, 56, 45, 48, 48, 97, 97, 45, 52, 101, 99, 57, 45, 57, 57, 49, 98, 45, 51, 48, 51, 101, 48, 102, 48, 102, 98, 49, 56, 51, 0, 0, 0, 0, 0, 0, 0, 41, 0, 0, 0, 50, 128, 87, 11, 55, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 98, 97, 98, 100, 52, 49, 100, 45, 49, 97, 99, 49, 45, 52, 102, 56, 50, 45, 97, 50, 48, 57, 45, 53, 51, 55, 57, 98, 54, 56, 57, 98, 49, 57, 55, 0, 0, 0, 0, 0, 0, 0, 42, 0, 0, 0, 50, 159, 251, 253, 174, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 50, 56, 50, 97, 97, 57, 99, 45, 48, 48, 98, 98, 45, 52, 52, 98, 101, 45, 56, 51, 54, 56, 45, 49, 97, 101, 101, 53, 97, 52, 56, 52, 100, 97, 55, 0, 0, 0, 0, 0, 0, 0, 43, 0, 0, 0, 50, 60, 176, 164, 60, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 57, 99, 56, 53, 101, 52, 51, 45, 102, 57, 53, 102, 45, 52, 52, 97, 52, 45, 98, 48, 49, 97, 45, 56, 49, 51, 52, 57, 97, 50, 101, 54, 49, 101, 57, 0, 0, 0, 0, 0, 0, 0, 44, 0, 0, 0, 50, 19, 177, 197, 123, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 52, 99, 50, 50, 97, 101, 49, 45, 56, 102, 56, 56, 45, 52, 51, 49, 57, 45, 56, 100, 55, 100, 45, 56, 51, 52, 98, 98, 48, 98, 54, 98, 51, 48, 55, 0, 0, 0, 0, 0, 0, 0, 45, 0, 0, 0, 50, 30, 193, 152, 239, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 99, 52, 56, 99, 50, 101, 98, 45, 98, 48, 101, 51, 45, 52, 102, 55, 49, 45, 97, 98, 55, 52, 45, 99, 99, 56, 54, 48, 57, 99, 101, 53, 98, 52, 102, 0, 0, 0, 0, 0, 0, 0, 46, 0, 0, 0, 50, 15, 232, 170, 254, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 101, 97, 56, 56, 50, 53, 57, 45, 51, 51, 52, 98, 45, 52, 51, 54, 57, 45, 97, 52, 97, 99, 45, 52, 52, 53, 99, 54, 51, 101, 53, 57, 101, 55, 57, 0, 0, 0, 0, 0, 0, 0, 47, 0, 0, 0, 50, 122, 77, 157, 250, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 100, 51, 99, 102, 100, 52, 99, 45, 97, 54, 100, 56, 45, 52, 101, 52, 56, 45, 98, 102, 99, 54, 45, 55, 55, 101, 98, 100, 97, 55, 53, 55, 48, 100, 101, 0, 0, 0, 0, 0, 0, 0, 48, 0, 0, 0, 50, 111, 186, 232, 214, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 55, 50, 98, 99, 101, 57, 52, 45, 97, 102, 57, 55, 45, 52, 53, 54, 48, 45, 56, 56, 48, 52, 45, 57, 49, 98, 98, 101, 55, 57, 57, 98, 48, 50, 51, 0, 0, 0, 0, 0, 0, 0, 49, 0, 0, 0, 50, 131, 211, 167, 141, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 48, 53, 100, 52, 52, 101, 48, 45, 97, 51, 49, 50, 45, 52, 99, 49, 97, 45, 98, 57, 54, 98, 45, 56, 97, 101, 50, 54, 48, 101, 50, 50, 57, 99, 49, 0, 0, 0, 0, 0, 0, 0, 50, 0, 0, 0, 50, 211, 208, 93, 175, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 54, 56, 102, 56, 55, 54, 97, 45, 98, 101, 98, 100, 45, 52, 53, 52, 56, 45, 56, 102, 54, 48, 45, 55, 98, 102, 51, 57, 57, 55, 98, 97, 102, 50, 57, 0, 0, 0, 0, 0, 0, 0, 51, 0, 0, 0, 50, 159, 169, 159, 18, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 102, 99, 53, 51, 97, 102, 57, 45, 102, 49, 101, 56, 45, 52, 100, 54, 54, 45, 56, 50, 53, 101, 45, 52, 50, 48, 99, 102, 48, 48, 102, 99, 97, 51, 98, 0, 0, 0, 0, 0, 0, 0, 52, 0, 0, 0, 50, 222, 43, 203, 214, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 50, 55, 53, 57, 99, 54, 100, 45, 55, 54, 53, 97, 45, 52, 48, 98, 99, 45, 98, 49, 100, 51, 45, 54, 53, 98, 51, 55, 53, 97, 97, 52, 49, 49, 102, 0, 0, 0, 0, 0, 0, 0, 53, 0, 0, 0, 50, 206, 183, 105, 146, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 51, 100, 101, 100, 102, 97, 51, 45, 101, 50, 53, 57, 45, 52, 50, 53, 55, 45, 97, 49, 98, 57, 45, 100, 52, 55, 52, 52, 52, 51, 55, 53, 101, 100, 49, 0, 0, 0, 0, 0, 0, 0, 54, 0, 0, 0, 50, 87, 21, 65, 131, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 97, 50, 50, 50, 99, 52, 100, 45, 102, 54, 57, 99, 45, 52, 101, 55, 102, 45, 56, 55, 98, 99, 45, 50, 97, 102, 52, 57, 57, 99, 53, 49, 50, 53, 100, 0, 0, 0, 0, 0, 0, 0, 55, 0, 0, 0, 50, 156, 157, 75, 129, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 52, 55, 57, 98, 52, 99, 49, 45, 99, 55, 49, 51, 45, 52, 102, 48, 53, 45, 97, 97, 53, 54, 45, 49, 57, 51, 100, 50, 57, 55, 48, 52, 102, 98, 50, 0, 0, 0, 0, 0, 0, 0, 56, 0, 0, 0, 50, 42, 124, 189, 35, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 102, 97, 53, 98, 54, 102, 48, 45, 53, 49, 56, 101, 45, 52, 54, 97, 49, 45, 97, 98, 55, 54, 45, 50, 50, 51, 55, 99, 49, 52, 97, 54, 57, 54, 52, 0, 0, 0, 0, 0, 0, 0, 57, 0, 0, 0, 50, 139, 54, 224, 142, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 54, 53, 52, 52, 52, 99, 98, 45, 99, 54, 98, 99, 45, 52, 49, 54, 50, 45, 97, 56, 101, 49, 45, 99, 57, 99, 100, 57, 48, 57, 51, 99, 54, 100, 49, 0, 0, 0, 0, 0, 0, 0, 58, 0, 0, 0, 50, 211, 197, 70, 39, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 50, 48, 55, 102, 55, 57, 50, 45, 54, 98, 51, 57, 45, 52, 52, 51, 54, 45, 56, 52, 57, 98, 45, 49, 101, 51, 102, 102, 52, 48, 56, 98, 56, 99, 49, 0, 0, 0, 0, 0, 0, 0, 59, 0, 0, 0, 50, 208, 140, 82, 10, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 50, 101, 52, 55, 57, 97, 97, 45, 56, 50, 102, 56, 45, 52, 50, 97, 52, 45, 98, 102, 100, 54, 45, 102, 48, 101, 50, 54, 99, 98, 50, 49, 49, 52, 49, 0, 0, 0, 0, 0, 0, 0, 60, 0, 0, 0, 50, 74, 56, 203, 193, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 99, 49, 101, 49, 101, 57, 53, 45, 52, 49, 99, 53, 45, 52, 51, 53, 102, 45, 97, 99, 51, 97, 45, 49, 57, 54, 49, 50, 56, 102, 54, 50, 48, 57, 97, 0, 0, 0, 0, 0, 0, 0, 61, 0, 0, 0, 50, 115, 228, 40, 70, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 51, 102, 50, 102, 97, 50, 53, 45, 51, 52, 54, 54, 45, 52, 48, 50, 101, 45, 57, 48, 102, 100, 45, 56, 50, 55, 99, 48, 100, 49, 97, 50, 51, 51, 53, 0, 0, 0, 0, 0, 0, 0, 62, 0, 0, 0, 50, 255, 245, 47, 192, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 99, 53, 97, 55, 49, 52, 54, 45, 48, 52, 51, 49, 45, 52, 101, 50, 53, 45, 56, 50, 49, 101, 45, 48, 56, 97, 99, 100, 51, 54, 49, 50, 99, 101, 55, 0, 0, 0, 0, 0, 0, 0, 63, 0, 0, 0, 50, 178, 132, 104, 105, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 100, 53, 99, 53, 50, 98, 51, 45, 57, 97, 57, 49, 45, 52, 99, 100, 52, 45, 56, 48, 54, 48, 45, 56, 50, 48, 49, 50, 102, 57, 56, 50, 54, 54, 51, 0, 0, 0, 0, 0, 0, 0, 64, 0, 0, 0, 50, 240, 56, 176, 197, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 51, 53, 57, 51, 51, 102, 99, 45, 50, 54, 51, 97, 45, 52, 99, 57, 101, 45, 97, 50, 50, 48, 45, 51, 51, 53, 97, 53, 55, 53, 102, 56, 57, 98, 98, 0, 0, 0, 0, 0, 0, 0, 65, 0, 0, 0, 50, 120, 22, 202, 167, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 98, 102, 56, 99, 53, 51, 57, 45, 102, 48, 57, 102, 45, 52, 51, 102, 57, 45, 98, 54, 101, 57, 45, 102, 49, 56, 52, 55, 56, 97, 50, 49, 99, 49, 101, 0, 0, 0, 0, 0, 0, 0, 66, 0, 0, 0, 50, 237, 158, 144, 204, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 49, 55, 51, 53, 52, 101, 99, 45, 48, 99, 48, 98, 45, 52, 55, 98, 99, 45, 57, 51, 97, 56, 45, 53, 100, 53, 97, 53, 52, 98, 53, 54, 49, 99, 50, 0, 0, 0, 0, 0, 0, 0, 67, 0, 0, 0, 50, 121, 30, 58, 106, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 55, 102, 52, 48, 55, 50, 56, 45, 102, 51, 51, 53, 45, 52, 98, 102, 53, 45, 56, 49, 101, 100, 45, 54, 54, 54, 50, 48, 54, 98, 101, 57, 57, 102, 48, 0, 0, 0, 0, 0, 0, 0, 68, 0, 0, 0, 50, 255, 10, 177, 6, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 102, 51, 100, 99, 55, 99, 56, 45, 49, 53, 102, 101, 45, 52, 51, 101, 98, 45, 56, 100, 101, 97, 45, 51, 54, 56, 99, 101, 50, 56, 51, 53, 49, 52, 55, 0, 0, 0, 0, 0, 0, 0, 69, 0, 0, 0, 50, 212, 161, 169, 112, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 52, 49, 53, 56, 56, 101, 52, 45, 49, 99, 48, 101, 45, 52, 48, 51, 102, 45, 57, 97, 49, 100, 45, 101, 56, 100, 97, 97, 49, 102, 48, 55, 99, 98, 101, 0, 0, 0, 0, 0, 0, 0, 70, 0, 0, 0, 50, 162, 164, 222, 140, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 52, 97, 100, 48, 100, 53, 99, 45, 56, 48, 57, 52, 45, 52, 99, 51, 55, 45, 56, 99, 49, 57, 45, 55, 54, 49, 102, 49, 52, 56, 56, 101, 52, 55, 101, 0, 0, 0, 0, 0, 0, 0, 71, 0, 0, 0, 50, 31, 211, 53, 71, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 50, 98, 49, 51, 49, 50, 99, 45, 48, 101, 99, 55, 45, 52, 53, 55, 55, 45, 98, 100, 102, 101, 45, 51, 53, 100, 100, 50, 97, 99, 54, 53, 97, 51, 100, 0, 0, 0, 0, 0, 0, 0, 72, 0, 0, 0, 50, 155, 89, 184, 23, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 56, 99, 53, 98, 102, 48, 100, 45, 52, 50, 102, 54, 45, 52, 102, 98, 48, 45, 98, 53, 97, 56, 45, 56, 51, 56, 101, 49, 54, 50, 57, 57, 55, 57, 48, 0, 0, 0, 0, 0, 0, 0, 73, 0, 0, 0, 50, 183, 138, 251, 65, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 53, 55, 49, 56, 54, 48, 53, 45, 53, 101, 53, 50, 45, 52, 51, 99, 53, 45, 56, 99, 100, 48, 45, 50, 48, 97, 52, 102, 53, 56, 52, 97, 51, 101, 51, 0, 0, 0, 0, 0, 0, 0, 74, 0, 0, 0, 50, 246, 168, 90, 249, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 49, 54, 99, 98, 53, 55, 52, 45, 53, 100, 101, 48, 45, 52, 52, 50, 57, 45, 97, 56, 56, 101, 45, 54, 49, 102, 51, 51, 53, 54, 57, 99, 97, 56, 99, 0, 0, 0, 0, 0, 0, 0, 75, 0, 0, 0, 50, 59, 145, 36, 10, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 100, 98, 99, 100, 55, 97, 99, 45, 53, 55, 57, 99, 45, 52, 98, 57, 99, 45, 97, 52, 100, 102, 45, 57, 49, 51, 100, 50, 98, 101, 99, 49, 55, 54, 49, 0, 0, 0, 0, 0, 0, 0, 76, 0, 0, 0, 50, 10, 150, 95, 173, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 51, 97, 50, 48, 57, 51, 49, 45, 49, 102, 49, 100, 45, 52, 101, 53, 52, 45, 98, 57, 49, 101, 45, 99, 52, 53, 56, 100, 56, 101, 100, 49, 49, 99, 53, 0, 0, 0, 0, 0, 0, 0, 77, 0, 0, 0, 50, 20, 34, 94, 34, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 54, 56, 99, 50, 55, 55, 99, 45, 57, 97, 57, 54, 45, 52, 52, 102, 99, 45, 57, 99, 97, 55, 45, 99, 97, 49, 51, 54, 102, 54, 98, 49, 50, 99, 50, 0, 0, 0, 0, 0, 0, 0, 78, 0, 0, 0, 50, 219, 107, 25, 212, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 57, 101, 100, 48, 102, 53, 52, 45, 56, 101, 55, 54, 45, 52, 54, 98, 102, 45, 98, 52, 53, 100, 45, 97, 50, 101, 98, 102, 100, 52, 54, 98, 97, 56, 56, 0, 0, 0, 0, 0, 0, 0, 79, 0, 0, 0, 50, 255, 31, 206, 237, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 97, 48, 48, 56, 98, 100, 56, 45, 54, 98, 98, 54, 45, 52, 49, 57, 51, 45, 97, 101, 56, 56, 45, 51, 53, 57, 97, 100, 49, 98, 51, 50, 55, 97, 48, 0, 0, 0, 0, 0, 0, 0, 80, 0, 0, 0, 50, 93, 111, 170, 228, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 99, 54, 54, 54, 57, 48, 100, 45, 48, 55, 54, 50, 45, 52, 51, 54, 50, 45, 97, 51, 102, 48, 45, 53, 50, 51, 98, 52, 48, 51, 49, 50, 54, 56, 57, 0, 0, 0, 0, 0, 0, 0, 81, 0, 0, 0, 50, 44, 173, 87, 28, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 48, 55, 98, 97, 57, 53, 52, 45, 100, 99, 53, 98, 45, 52, 54, 97, 53, 45, 98, 102, 49, 97, 45, 100, 55, 57, 102, 49, 50, 99, 49, 49, 48, 102, 48, 0, 0, 0, 0, 0, 0, 0, 82, 0, 0, 0, 50, 162, 243, 5, 116, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 55, 99, 50, 97, 98, 52, 101, 45, 54, 51, 56, 101, 45, 52, 51, 50, 101, 45, 97, 55, 102, 100, 45, 102, 55, 101, 49, 51, 52, 52, 101, 53, 99, 97, 55, 0, 0, 0, 0, 0, 0, 0, 83, 0, 0, 0, 50, 133, 187, 134, 145, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 49, 57, 102, 55, 54, 102, 54, 45, 100, 102, 57, 101, 45, 52, 102, 102, 55, 45, 57, 50, 55, 55, 45, 49, 57, 100, 99, 102, 100, 51, 49, 99, 99, 53, 98, 0, 0, 0, 0, 0, 0, 0, 84, 0, 0, 0, 50, 240, 125, 240, 22, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 97, 50, 54, 49, 54, 55, 100, 45, 54, 53, 48, 102, 45, 52, 48, 102, 50, 45, 56, 101, 51, 49, 45, 52, 102, 49, 52, 101, 102, 100, 101, 98, 52, 101, 100, 0, 0, 0, 0, 0, 0, 0, 85, 0, 0, 0, 50, 201, 176, 72, 64, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 102, 51, 55, 97, 100, 100, 101, 45, 49, 49, 52, 102, 45, 52, 99, 52, 52, 45, 97, 101, 49, 97, 45, 50, 97, 55, 56, 51, 56, 55, 49, 98, 97, 54, 53, 0, 0, 0, 0, 0, 0, 0, 86, 0, 0, 0, 50, 205, 57, 34, 43, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 56, 51, 51, 54, 48, 57, 56, 45, 100, 51, 99, 100, 45, 52, 97, 102, 98, 45, 57, 55, 57, 99, 45, 97, 57, 101, 49, 49, 97, 56, 99, 51, 98, 100, 57, 0, 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, 50, 137, 129, 165, 183, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 102, 102, 100, 49, 51, 55, 102, 45, 100, 101, 55, 101, 45, 52, 56, 52, 53, 45, 57, 101, 49, 101, 45, 50, 49, 99, 51, 99, 49, 49, 50, 97, 97, 55, 52, 0, 0, 0, 0, 0, 0, 0, 88, 0, 0, 0, 50, 191, 49, 182, 161, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 57, 101, 97, 100, 48, 54, 55, 45, 48, 53, 100, 50, 45, 52, 100, 53, 51, 45, 97, 97, 56, 52, 45, 53, 100, 55, 54, 55, 102, 48, 53, 101, 97, 57, 50, 0, 0, 0, 0, 0, 0, 0, 89, 0, 0, 0, 50, 36, 218, 17, 34, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 49, 101, 100, 56, 49, 99, 48, 45, 53, 54, 48, 102, 45, 52, 102, 50, 100, 45, 98, 53, 97, 54, 45, 49, 102, 52, 101, 100, 52, 98, 53, 48, 54, 51, 55, 0, 0, 0, 0, 0, 0, 0, 90, 0, 0, 0, 50, 16, 14, 19, 90, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 56, 99, 50, 56, 101, 56, 53, 45, 101, 97, 56, 54, 45, 52, 55, 55, 57, 45, 98, 48, 50, 57, 45, 102, 57, 50, 49, 48, 97, 102, 98, 52, 98, 53, 48, 0, 0, 0, 0, 0, 0, 0, 91, 0, 0, 0, 50, 71, 20, 34, 77, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 49, 51, 53, 51, 52, 57, 50, 45, 51, 54, 53, 48, 45, 52, 51, 51, 53, 45, 98, 100, 57, 52, 45, 51, 101, 51, 53, 102, 101, 100, 49, 101, 50, 97, 54, 0, 0, 0, 0, 0, 0, 0, 92, 0, 0, 0, 50, 27, 109, 31, 124, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 53, 101, 54, 50, 98, 52, 54, 45, 100, 54, 48, 101, 45, 52, 56, 98, 102, 45, 57, 52, 49, 57, 45, 99, 99, 56, 98, 48, 48, 55, 53, 48, 54, 50, 99, 0, 0, 0, 0, 0, 0, 0, 93, 0, 0, 0, 50, 177, 174, 79, 198, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 102, 100, 52, 57, 97, 52, 53, 45, 55, 52, 49, 51, 45, 52, 55, 100, 53, 45, 97, 97, 55, 50, 45, 52, 52, 50, 102, 48, 48, 101, 54, 97, 52, 48, 48, 0, 0, 0, 0, 0, 0, 0, 94, 0, 0, 0, 50, 231, 5, 110, 91, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 50, 99, 53, 55, 97, 53, 54, 45, 97, 48, 57, 102, 45, 52, 50, 53, 52, 45, 98, 102, 49, 99, 45, 51, 99, 57, 57, 54, 54, 100, 52, 99, 53, 100, 99, 0, 0, 0, 0, 0, 0, 0, 95, 0, 0, 0, 50, 205, 14, 85, 20, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 97, 51, 97, 100, 98, 57, 56, 45, 56, 56, 97, 57, 45, 52, 55, 101, 102, 45, 98, 100, 102, 49, 45, 102, 56, 49, 101, 53, 54, 51, 56, 57, 99, 52, 51, 0, 0, 0, 0, 0, 0, 0, 96, 0, 0, 0, 50, 82, 231, 17, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 98, 99, 99, 98, 48, 99, 45, 56, 97, 54, 102, 45, 52, 54, 54, 100, 45, 98, 50, 49, 99, 45, 55, 57, 49, 49, 52, 54, 52, 97, 97, 98, 101, 56, 0, 0, 0, 0, 0, 0, 0, 97, 0, 0, 0, 50, 236, 6, 179, 100, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 52, 101, 50, 55, 48, 49, 51, 45, 102, 98, 56, 49, 45, 52, 97, 98, 54, 45, 97, 55, 48, 57, 45, 51, 48, 48, 52, 101, 102, 48, 97, 52, 99, 102, 53, 0, 0, 0, 0, 0, 0, 0, 98, 0, 0, 0, 50, 193, 163, 205, 2, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 53, 49, 101, 102, 56, 99, 52, 45, 102, 49, 99, 50, 45, 52, 55, 56, 102, 45, 97, 49, 97, 98, 45, 99, 56, 48, 48, 48, 48, 51, 54, 101, 98, 54, 101, 0, 0, 0, 0, 0, 0, 0, 99, 0, 0, 0, 50, 117, 202, 96, 133, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 99, 51, 49, 56, 50, 99, 57, 45, 99, 48, 100, 98, 45, 52, 101, 53, 97, 45, 97, 48, 102, 55, 45, 55, 100, 100, 49, 98, 98, 100, 54, 51, 55, 56, 49, 0, 0, 0, 0, 0, 0, 0, 100, 0, 0, 0, 50, 198, 169, 125, 142, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 53, 56, 102, 51, 54, 100, 52, 45, 97, 53, 52, 100, 45, 52, 54, 56, 102, 45, 97, 51, 57, 101, 45, 51, 49, 97, 49, 101, 51, 97, 53, 102, 49, 99, 97, 0, 0, 0, 0, 0, 0, 0, 101, 0, 0, 0, 50, 25, 24, 61, 163, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 48, 102, 50, 48, 57, 99, 99, 45, 51, 102, 54, 52, 45, 52, 50, 51, 100, 45, 97, 53, 48, 100, 45, 52, 57, 57, 99, 51, 53, 100, 101, 98, 102, 48, 99, 0, 0, 0, 0, 0, 0, 0, 102, 0, 0, 0, 50, 103, 143, 14, 164, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 52, 98, 57, 101, 54, 48, 54, 45, 102, 102, 97, 48, 45, 52, 100, 99, 50, 45, 98, 52, 51, 97, 45, 48, 50, 57, 56, 48, 102, 53, 97, 99, 49, 51, 55, 0, 0, 0, 0, 0, 0, 0, 103, 0, 0, 0, 50, 31, 29, 139, 58, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 52, 100, 99, 53, 55, 97, 53, 45, 98, 57, 97, 54, 45, 52, 102, 51, 102, 45, 97, 51, 99, 57, 45, 98, 102, 99, 48, 51, 48, 97, 97, 57, 97, 53, 99, 0, 0, 0, 0, 0, 0, 0, 104, 0, 0, 0, 50, 234, 234, 72, 237, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 52, 53, 52, 102, 101, 57, 53, 45, 52, 54, 51, 98, 45, 52, 99, 100, 49, 45, 56, 101, 56, 99, 45, 48, 48, 56, 100, 101, 49, 100, 57, 99, 54, 102, 50, 0, 0, 0, 0, 0, 0, 0, 105, 0, 0, 0, 50, 219, 81, 57, 77, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 48, 56, 53, 57, 51, 55, 51, 45, 56, 55, 55, 52, 45, 52, 98, 55, 99, 45, 57, 50, 49, 98, 45, 53, 53, 55, 54, 57, 50, 99, 55, 97, 101, 57, 98, 0, 0, 0, 0, 0, 0, 0, 106, 0, 0, 0, 50, 136, 90, 145, 148, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 54, 98, 97, 51, 53, 51, 57, 45, 56, 52, 50, 101, 45, 52, 56, 48, 52, 45, 97, 57, 49, 48, 45, 51, 55, 53, 97, 99, 98, 99, 97, 53, 102, 53, 97, 0, 0, 0, 0, 0, 0, 0, 107, 0, 0, 0, 50, 0, 149, 108, 33, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 99, 56, 52, 51, 57, 49, 49, 45, 55, 49, 100, 97, 45, 52, 53, 56, 55, 45, 57, 53, 50, 100, 45, 54, 57, 57, 51, 99, 102, 48, 51, 52, 48, 100, 100, 0, 0, 0, 0, 0, 0, 0, 108, 0, 0, 0, 50, 110, 250, 207, 155, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 101, 54, 50, 55, 102, 99, 57, 45, 99, 53, 50, 53, 45, 52, 51, 56, 100, 45, 97, 99, 98, 56, 45, 57, 49, 102, 101, 50, 53, 52, 55, 49, 98, 48, 101, 0, 0, 0, 0, 0, 0, 0, 109, 0, 0, 0, 50, 7, 218, 245, 51, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 53, 48, 101, 56, 100, 53, 49, 45, 56, 57, 53, 57, 45, 52, 97, 52, 51, 45, 57, 54, 102, 54, 45, 99, 50, 98, 52, 97, 57, 57, 51, 55, 54, 98, 49, 0, 0, 0, 0, 0, 0, 0, 110, 0, 0, 0, 50, 155, 215, 185, 22, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 48, 54, 54, 97, 52, 97, 98, 45, 100, 55, 101, 100, 45, 52, 97, 50, 50, 45, 97, 102, 101, 52, 45, 100, 53, 99, 98, 51, 54, 52, 57, 97, 56, 100, 102, 0, 0, 0, 0, 0, 0, 0, 111, 0, 0, 0, 50, 193, 252, 241, 234, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 50, 99, 54, 101, 54, 100, 45, 48, 99, 52, 97, 45, 52, 100, 52, 52, 45, 97, 57, 48, 50, 45, 56, 57, 54, 51, 48, 55, 99, 54, 49, 48, 54, 51, 0, 0, 0, 0, 0, 0, 0, 112, 0, 0, 0, 50, 82, 162, 160, 20, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 50, 56, 54, 50, 56, 54, 97, 45, 52, 54, 101, 51, 45, 52, 52, 52, 57, 45, 57, 99, 101, 57, 45, 100, 98, 57, 98, 98, 102, 52, 53, 49, 56, 50, 51, 0, 0, 0, 0, 0, 0, 0, 113, 0, 0, 0, 50, 166, 83, 25, 179, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 48, 98, 51, 48, 50, 50, 102, 45, 100, 55, 55, 57, 45, 52, 50, 49, 51, 45, 56, 102, 97, 99, 45, 101, 99, 101, 99, 57, 99, 52, 50, 101, 98, 98, 53, 0, 0, 0, 0, 0, 0, 0, 114, 0, 0, 0, 50, 63, 250, 207, 31, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 97, 57, 48, 102, 50, 54, 55, 45, 52, 100, 55, 97, 45, 52, 50, 53, 53, 45, 97, 98, 55, 97, 45, 97, 98, 102, 50, 100, 54, 53, 53, 53, 53, 48, 55, 0, 0, 0, 0, 0, 0, 0, 115, 0, 0, 0, 50, 171, 94, 24, 226, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 57, 50, 56, 55, 49, 98, 45, 99, 48, 48, 48, 45, 52, 56, 55, 52, 45, 56, 100, 51, 49, 45, 53, 49, 53, 99, 101, 49, 57, 102, 54, 51, 98, 49, 0, 0, 0, 0, 0, 0, 0, 116, 0, 0, 0, 50, 118, 160, 228, 183, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 102, 100, 50, 55, 98, 52, 100, 45, 99, 100, 100, 100, 45, 52, 48, 48, 102, 45, 57, 50, 53, 101, 45, 99, 56, 98, 49, 102, 99, 100, 51, 50, 56, 102, 97, 0, 0, 0, 0, 0, 0, 0, 117, 0, 0, 0, 50, 47, 98, 25, 176, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 56, 52, 99, 49, 52, 52, 57, 45, 101, 48, 48, 97, 45, 52, 57, 53, 48, 45, 56, 54, 53, 48, 45, 50, 101, 53, 51, 57, 98, 102, 100, 51, 98, 99, 102, 0, 0, 0, 0, 0, 0, 0, 118, 0, 0, 0, 50, 37, 198, 137, 172, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 48, 49, 101, 50, 99, 51, 98, 45, 51, 101, 53, 102, 45, 52, 49, 50, 102, 45, 56, 98, 48, 48, 45, 99, 102, 102, 98, 54, 102, 51, 102, 52, 56, 54, 50, 0, 0, 0, 0, 0, 0, 0, 119, 0, 0, 0, 50, 210, 49, 40, 165, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 50, 102, 52, 102, 101, 54, 54, 45, 54, 99, 97, 56, 45, 52, 50, 57, 50, 45, 97, 51, 51, 55, 45, 99, 57, 102, 99, 49, 97, 101, 50, 51, 56, 101, 49, 0, 0, 0, 0, 0, 0, 0, 120, 0, 0, 0, 50, 89, 245, 90, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 54, 98, 55, 100, 100, 100, 45, 51, 100, 100, 97, 45, 52, 49, 56, 48, 45, 56, 99, 54, 97, 45, 97, 101, 51, 99, 49, 54, 49, 100, 97, 97, 98, 54, 0, 0, 0, 0, 0, 0, 0, 121, 0, 0, 0, 50, 13, 225, 231, 143, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 56, 56, 100, 55, 52, 53, 50, 45, 48, 56, 56, 56, 45, 52, 57, 49, 56, 45, 57, 99, 53, 102, 45, 53, 49, 99, 57, 51, 98, 98, 55, 52, 97, 97, 52, 0, 0, 0, 0, 0, 0, 0, 122, 0, 0, 0, 50, 104, 48, 107, 246, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 50, 100, 56, 49, 100, 51, 48, 45, 100, 56, 50, 100, 45, 52, 98, 50, 55, 45, 97, 57, 54, 97, 45, 98, 50, 50, 56, 99, 98, 98, 50, 49, 99, 52, 49, 0, 0, 0, 0, 0, 0, 0, 123, 0, 0, 0, 50, 121, 51, 60, 53, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 49, 49, 54, 57, 56, 56, 55, 45, 51, 56, 97, 53, 45, 52, 56, 51, 50, 45, 56, 57, 98, 101, 45, 97, 52, 57, 101, 48, 102, 50, 53, 54, 52, 48, 97, 0, 0, 0, 0, 0, 0, 0, 124, 0, 0, 0, 50, 19, 233, 70, 8, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 52, 54, 51, 97, 100, 51, 98, 45, 53, 55, 97, 50, 45, 52, 101, 55, 100, 45, 57, 101, 49, 102, 45, 49, 101, 54, 53, 97, 52, 100, 101, 97, 54, 50, 102, 0, 0, 0, 0, 0, 0, 0, 125, 0, 0, 0, 50, 156, 155, 105, 234, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 101, 50, 56, 101, 50, 51, 54, 45, 55, 50, 100, 98, 45, 52, 102, 56, 55, 45, 56, 99, 56, 102, 45, 99, 97, 52, 98, 99, 53, 102, 54, 52, 56, 55, 55, 0, 0, 0, 0, 0, 0, 0, 126, 0, 0, 0, 50, 161, 177, 175, 56, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 101, 50, 55, 97, 53, 53, 101, 45, 52, 97, 100, 52, 45, 52, 55, 102, 97, 45, 98, 57, 102, 102, 45, 55, 48, 51, 57, 54, 102, 49, 50, 56, 56, 102, 52, 0, 0, 0, 0, 0, 0, 0, 127, 0, 0, 0, 50, 99, 25, 135, 168, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 101, 53, 102, 52, 57, 57, 49, 45, 48, 99, 97, 51, 45, 52, 98, 98, 50, 45, 57, 101, 57, 99, 45, 55, 98, 50, 57, 52, 55, 49, 49, 53, 100, 51, 55, 0, 0, 0, 0, 0, 0, 0, 128, 0, 0, 0, 50, 112, 144, 166, 27, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 56, 57, 52, 102, 97, 102, 55, 45, 102, 102, 56, 55, 45, 52, 57, 57, 97, 45, 57, 50, 51, 97, 45, 97, 57, 102, 57, 98, 56, 55, 55, 101, 53, 102, 50, 0, 0, 0, 0, 0, 0, 0, 129, 0, 0, 0, 50, 120, 159, 112, 161, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 53, 55, 101, 56, 55, 56, 102, 45, 51, 53, 99, 100, 45, 52, 49, 102, 50, 45, 56, 98, 56, 56, 45, 50, 54, 54, 101, 102, 53, 50, 98, 100, 55, 100, 55, 0, 0, 0, 0, 0, 0, 0, 130, 0, 0, 0, 50, 218, 7, 165, 26, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 57, 51, 54, 49, 102, 50, 99, 45, 55, 51, 97, 102, 45, 52, 97, 100, 100, 45, 57, 53, 100, 101, 45, 56, 50, 53, 98, 53, 49, 102, 101, 48, 102, 53, 52, 0, 0, 0, 0, 0, 0, 0, 131, 0, 0, 0, 50, 209, 235, 195, 247, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 52, 52, 50, 98, 54, 53, 45, 101, 49, 53, 57, 45, 52, 50, 51, 101, 45, 57, 52, 52, 99, 45, 97, 53, 102, 57, 48, 99, 57, 52, 97, 97, 55, 100, 0, 0, 0, 0, 0, 0, 0, 132, 0, 0, 0, 50, 186, 43, 113, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 102, 57, 100, 55, 100, 97, 99, 45, 50, 48, 51, 57, 45, 52, 100, 53, 98, 45, 56, 99, 100, 50, 45, 48, 55, 98, 51, 53, 51, 98, 50, 57, 54, 97, 53, 0, 0, 0, 0, 0, 0, 0, 133, 0, 0, 0, 50, 253, 86, 135, 27, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 51, 55, 100, 51, 97, 97, 101, 45, 99, 50, 54, 57, 45, 52, 54, 98, 53, 45, 57, 49, 99, 54, 45, 56, 98, 49, 53, 53, 54, 97, 102, 51, 52, 99, 57, 0, 0, 0, 0, 0, 0, 0, 134, 0, 0, 0, 50, 176, 29, 240, 3, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 99, 52, 55, 55, 101, 48, 98, 45, 100, 97, 101, 102, 45, 52, 56, 57, 57, 45, 98, 98, 48, 50, 45, 100, 51, 55, 53, 102, 99, 100, 101, 54, 57, 51, 54, 0, 0, 0, 0, 0, 0, 0, 135, 0, 0, 0, 50, 122, 60, 131, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 98, 98, 50, 100, 55, 98, 102, 45, 57, 48, 57, 101, 45, 52, 102, 56, 53, 45, 57, 50, 97, 49, 45, 54, 55, 52, 99, 50, 99, 56, 52, 53, 50, 48, 56, 0, 0, 0, 0, 0, 0, 0, 136, 0, 0, 0, 50, 205, 137, 13, 76, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 56, 56, 48, 57, 100, 50, 53, 45, 56, 51, 57, 101, 45, 52, 57, 56, 49, 45, 97, 52, 54, 50, 45, 51, 98, 98, 56, 51, 57, 49, 100, 54, 49, 100, 50, 0, 0, 0, 0, 0, 0, 0, 137, 0, 0, 0, 50, 156, 57, 105, 6, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 102, 102, 51, 53, 57, 56, 49, 45, 53, 49, 102, 54, 45, 52, 101, 99, 48, 45, 97, 53, 53, 102, 45, 50, 50, 98, 49, 99, 98, 51, 102, 101, 99, 101, 51, 0, 0, 0, 0, 0, 0, 0, 138, 0, 0, 0, 50, 178, 224, 249, 167, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 100, 97, 57, 49, 53, 56, 51, 45, 56, 101, 101, 98, 45, 52, 50, 56, 48, 45, 97, 53, 100, 100, 45, 100, 53, 98, 51, 49, 99, 52, 55, 52, 100, 49, 53, 0, 0, 0, 0, 0, 0, 0, 139, 0, 0, 0, 50, 216, 223, 186, 108, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 98, 100, 49, 98, 51, 101, 56, 45, 99, 54, 53, 50, 45, 52, 102, 102, 99, 45, 97, 102, 52, 49, 45, 101, 50, 99, 54, 99, 48, 53, 49, 98, 49, 50, 98, 0, 0, 0, 0, 0, 0, 0, 140, 0, 0, 0, 50, 248, 190, 93, 235, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 101, 99, 54, 50, 48, 57, 54, 45, 55, 100, 53, 99, 45, 52, 56, 50, 55, 45, 56, 52, 100, 55, 45, 98, 56, 54, 50, 48, 55, 49, 50, 48, 50, 53, 57, 0, 0, 0, 0, 0, 0, 0, 141, 0, 0, 0, 50, 10, 207, 68, 98, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 50, 53, 49, 56, 102, 50, 57, 45, 100, 51, 49, 51, 45, 52, 49, 49, 55, 45, 98, 97, 56, 101, 45, 53, 98, 98, 102, 52, 97, 51, 102, 102, 98, 50, 51, 0, 0, 0, 0, 0, 0, 0, 142, 0, 0, 0, 50, 86, 11, 29, 35, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 55, 57, 49, 98, 57, 54, 98, 45, 51, 98, 97, 101, 45, 52, 48, 49, 100, 45, 98, 102, 54, 54, 45, 101, 99, 57, 55, 101, 55, 49, 54, 48, 51, 53, 97, 0, 0, 0, 0, 0, 0, 0, 143, 0, 0, 0, 50, 239, 30, 237, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 57, 50, 102, 99, 102, 51, 57, 45, 53, 50, 99, 100, 45, 52, 49, 102, 54, 45, 97, 57, 49, 50, 45, 99, 50, 49, 98, 100, 98, 99, 57, 51, 53, 98, 50, 0, 0, 0, 0, 0, 0, 0, 144, 0, 0, 0, 50, 8, 255, 207, 212, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 52, 51, 100, 55, 49, 99, 100, 45, 51, 49, 101, 101, 45, 52, 55, 55, 98, 45, 98, 97, 98, 54, 45, 50, 53, 98, 97, 48, 50, 54, 102, 97, 56, 52, 102, 0, 0, 0, 0, 0, 0, 0, 145, 0, 0, 0, 50, 29, 250, 233, 84, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 57, 98, 53, 99, 56, 99, 54, 45, 55, 57, 99, 102, 45, 52, 50, 54, 56, 45, 56, 100, 102, 100, 45, 55, 53, 55, 50, 50, 97, 52, 50, 52, 54, 99, 101, 0, 0, 0, 0, 0, 0, 0, 146, 0, 0, 0, 50, 44, 21, 191, 62, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 54, 99, 97, 99, 100, 50, 97, 45, 100, 97, 54, 56, 45, 52, 98, 49, 54, 45, 57, 56, 54, 97, 45, 53, 57, 53, 56, 54, 52, 101, 48, 48, 55, 102, 49, 0, 0, 0, 0, 0, 0, 0, 147, 0, 0, 0, 50, 221, 11, 204, 237, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 99, 48, 52, 55, 49, 101, 97, 45, 54, 51, 51, 57, 45, 52, 99, 51, 48, 45, 98, 49, 99, 53, 45, 101, 101, 48, 49, 49, 55, 55, 99, 101, 100, 48, 101, 0, 0, 0, 0, 0, 0, 0, 148, 0, 0, 0, 50, 237, 132, 163, 115, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 51, 53, 98, 99, 51, 48, 52, 45, 52, 97, 48, 56, 45, 52, 100, 55, 51, 45, 97, 57, 53, 51, 45, 57, 56, 49, 55, 54, 97, 100, 49, 54, 48, 99, 56, 0, 0, 0, 0, 0, 0, 0, 149, 0, 0, 0, 50, 190, 199, 10, 71, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 97, 101, 49, 52, 55, 100, 49, 45, 55, 51, 102, 57, 45, 52, 53, 99, 57, 45, 57, 50, 52, 53, 45, 56, 54, 102, 54, 49, 101, 50, 54, 48, 53, 101, 98, 0, 0, 0, 0, 0, 0, 0, 150, 0, 0, 0, 50, 11, 215, 240, 24, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 49, 53, 55, 98, 102, 48, 50, 45, 57, 53, 48, 56, 45, 52, 53, 54, 56, 45, 56, 97, 49, 51, 45, 55, 48, 56, 49, 50, 57, 54, 98, 48, 54, 100, 51, 0, 0, 0, 0, 0, 0, 0, 151, 0, 0, 0, 50, 94, 170, 6, 26, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 101, 56, 56, 49, 51, 99, 101, 45, 49, 48, 56, 99, 45, 52, 50, 56, 54, 45, 98, 54, 48, 56, 45, 102, 100, 97, 56, 48, 102, 101, 49, 97, 50, 54, 98, 0, 0, 0, 0, 0, 0, 0, 152, 0, 0, 0, 50, 39, 107, 65, 105, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 48, 102, 56, 54, 54, 98, 45, 101, 97, 49, 55, 45, 52, 50, 49, 53, 45, 98, 48, 56, 50, 45, 49, 102, 97, 102, 56, 48, 102, 52, 101, 99, 98, 49, 0, 0, 0, 0, 0, 0, 0, 153, 0, 0, 0, 50, 211, 249, 32, 197, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 51, 97, 99, 48, 101, 51, 101, 45, 99, 50, 102, 53, 45, 52, 100, 97, 98, 45, 98, 50, 100, 101, 45, 54, 101, 101, 51, 55, 57, 48, 100, 99, 54, 53, 53, 0, 0, 0, 0, 0, 0, 0, 154, 0, 0, 0, 50, 212, 9, 200, 130, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 53, 100, 50, 49, 55, 51, 100, 45, 55, 98, 49, 55, 45, 52, 49, 49, 101, 45, 97, 97, 52, 101, 45, 48, 52, 53, 99, 98, 54, 52, 51, 49, 97, 52, 98, 0, 0, 0, 0, 0, 0, 0, 155, 0, 0, 0, 50, 133, 2, 202, 216, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 99, 102, 100, 56, 52, 49, 52, 45, 50, 99, 49, 102, 45, 52, 99, 101, 101, 45, 98, 101, 99, 101, 45, 51, 97, 54, 54, 100, 98, 53, 49, 97, 57, 99, 53, 0, 0, 0, 0, 0, 0, 0, 156, 0, 0, 0, 50, 119, 236, 161, 79, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 57, 102, 97, 49, 99, 99, 50, 45, 56, 53, 51, 99, 45, 52, 53, 97, 100, 45, 97, 100, 50, 49, 45, 51, 52, 97, 49, 57, 100, 50, 99, 53, 50, 51, 51, 0, 0, 0, 0, 0, 0, 0, 157, 0, 0, 0, 50, 11, 41, 86, 10, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 99, 99, 52, 49, 101, 102, 55, 45, 97, 98, 49, 97, 45, 52, 52, 51, 52, 45, 57, 101, 100, 55, 45, 100, 99, 101, 98, 52, 97, 56, 56, 102, 97, 102, 99, 0, 0, 0, 0, 0, 0, 0, 158, 0, 0, 0, 50, 202, 128, 224, 185, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 54, 54, 100, 55, 102, 57, 98, 45, 99, 53, 56, 51, 45, 52, 100, 57, 98, 45, 57, 49, 101, 50, 45, 54, 102, 97, 97, 52, 53, 98, 55, 49, 99, 53, 97, 0, 0, 0, 0, 0, 0, 0, 159, 0, 0, 0, 50, 213, 119, 97, 203, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 98, 100, 52, 99, 102, 50, 99, 45, 56, 48, 56, 98, 45, 52, 48, 99, 51, 45, 97, 48, 98, 97, 45, 50, 51, 52, 54, 54, 98, 100, 99, 100, 56, 56, 54, 0, 0, 0, 0, 0, 0, 0, 160, 0, 0, 0, 50, 125, 253, 133, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 54, 97, 54, 98, 100, 52, 54, 45, 101, 97, 102, 55, 45, 52, 100, 99, 100, 45, 57, 53, 54, 48, 45, 53, 56, 102, 97, 56, 98, 52, 55, 53, 56, 50, 102, 0, 0, 0, 0, 0, 0, 0, 161, 0, 0, 0, 50, 69, 82, 168, 139, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 97, 55, 56, 49, 98, 101, 57, 45, 57, 57, 51, 56, 45, 52, 51, 48, 101, 45, 97, 50, 53, 56, 45, 53, 99, 52, 57, 97, 98, 97, 48, 56, 100, 48, 55, 0, 0, 0, 0, 0, 0, 0, 162, 0, 0, 0, 50, 225, 173, 241, 11, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 52, 53, 50, 97, 100, 49, 54, 45, 101, 100, 102, 57, 45, 52, 49, 100, 100, 45, 98, 49, 55, 52, 45, 52, 51, 57, 57, 51, 50, 100, 52, 50, 99, 53, 52, 0, 0, 0, 0, 0, 0, 0, 163, 0, 0, 0, 50, 72, 226, 76, 107, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 49, 100, 52, 97, 98, 56, 45, 49, 99, 55, 99, 45, 52, 49, 97, 56, 45, 98, 54, 53, 102, 45, 54, 55, 102, 56, 101, 50, 99, 51, 50, 98, 50, 56, 0, 0, 0, 0, 0, 0, 0, 164, 0, 0, 0, 50, 194, 252, 138, 123, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 49, 48, 52, 50, 50, 54, 50, 45, 53, 52, 98, 53, 45, 52, 49, 101, 56, 45, 56, 50, 52, 56, 45, 50, 101, 52, 50, 52, 50, 97, 98, 101, 49, 52, 57, 0, 0, 0, 0, 0, 0, 0, 165, 0, 0, 0, 50, 90, 42, 216, 19, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 51, 54, 54, 54, 55, 52, 102, 45, 54, 57, 101, 49, 45, 52, 98, 48, 98, 45, 56, 102, 50, 100, 45, 54, 100, 57, 101, 54, 102, 55, 56, 51, 52, 51, 101, 0, 0, 0, 0, 0, 0, 0, 166, 0, 0, 0, 50, 226, 131, 31, 23, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 98, 51, 98, 98, 98, 98, 99, 45, 55, 102, 99, 53, 45, 52, 51, 56, 101, 45, 97, 51, 50, 101, 45, 53, 56, 53, 57, 98, 99, 102, 56, 51, 50, 98, 99, 0, 0, 0, 0, 0, 0, 0, 167, 0, 0, 0, 50, 241, 246, 238, 187, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 99, 57, 51, 98, 54, 55, 48, 45, 52, 55, 98, 55, 45, 52, 50, 53, 52, 45, 97, 102, 56, 48, 45, 54, 100, 100, 56, 54, 52, 53, 102, 97, 100, 50, 99, 0, 0, 0, 0, 0, 0, 0, 168, 0, 0, 0, 50, 25, 94, 164, 113, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 102, 51, 56, 48, 100, 57, 55, 45, 54, 54, 50, 100, 45, 52, 100, 102, 53, 45, 56, 56, 55, 57, 45, 48, 53, 51, 52, 57, 51, 52, 56, 49, 53, 54, 100, 0, 0, 0, 0, 0, 0, 0, 169, 0, 0, 0, 50, 22, 17, 193, 48, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 48, 100, 52, 57, 52, 54, 53, 45, 101, 52, 52, 51, 45, 52, 99, 56, 49, 45, 97, 100, 99, 56, 45, 101, 55, 98, 50, 98, 48, 56, 102, 99, 54, 48, 100, 0, 0, 0, 0, 0, 0, 0, 170, 0, 0, 0, 50, 6, 250, 144, 159, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 50, 51, 100, 52, 54, 48, 52, 45, 57, 98, 57, 50, 45, 52, 54, 49, 100, 45, 98, 100, 53, 49, 45, 51, 49, 50, 55, 100, 55, 48, 54, 52, 52, 49, 50, 0, 0, 0, 0, 0, 0, 0, 171, 0, 0, 0, 50, 133, 161, 44, 45, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 99, 98, 52, 99, 99, 55, 53, 45, 102, 54, 50, 101, 45, 52, 97, 55, 50, 45, 56, 97, 52, 51, 45, 52, 52, 99, 54, 52, 100, 98, 51, 98, 57, 52, 101, 0, 0, 0, 0, 0, 0, 0, 172, 0, 0, 0, 50, 122, 182, 88, 65, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 51, 48, 48, 56, 98, 98, 102, 45, 49, 48, 53, 52, 45, 52, 57, 102, 49, 45, 97, 49, 51, 56, 45, 56, 55, 50, 48, 55, 57, 99, 54, 48, 99, 55, 56, 0, 0, 0, 0, 0, 0, 0, 173, 0, 0, 0, 50, 44, 4, 24, 230, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 50, 101, 100, 48, 101, 56, 99, 45, 51, 99, 48, 55, 45, 52, 49, 55, 51, 45, 98, 52, 52, 54, 45, 56, 97, 50, 49, 48, 51, 98, 56, 48, 97, 48, 50, 0, 0, 0, 0, 0, 0, 0, 174, 0, 0, 0, 50, 80, 105, 78, 150, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 102, 98, 54, 52, 56, 56, 101, 45, 100, 56, 97, 54, 45, 52, 54, 99, 50, 45, 56, 102, 52, 99, 45, 102, 53, 52, 53, 97, 101, 55, 57, 100, 57, 101, 52, 0, 0, 0, 0, 0, 0, 0, 175, 0, 0, 0, 50, 163, 105, 195, 181, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 97, 99, 55, 51, 98, 56, 98, 45, 52, 98, 49, 99, 45, 52, 53, 97, 48, 45, 57, 51, 48, 101, 45, 50, 100, 99, 51, 57, 100, 101, 56, 97, 52, 100, 51, 0, 0, 0, 0, 0, 0, 0, 176, 0, 0, 0, 50, 135, 53, 5, 35, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 49, 100, 99, 48, 102, 100, 45, 97, 50, 97, 97, 45, 52, 54, 97, 102, 45, 98, 49, 54, 57, 45, 48, 52, 97, 57, 49, 56, 53, 102, 55, 100, 97, 57, 0, 0, 0, 0, 0, 0, 0, 177, 0, 0, 0, 50, 198, 68, 119, 192, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 54, 50, 56, 99, 101, 97, 101, 45, 55, 100, 48, 52, 45, 52, 101, 53, 53, 45, 57, 52, 52, 51, 45, 100, 100, 48, 102, 97, 98, 102, 57, 55, 50, 57, 53, 0, 0, 0, 0, 0, 0, 0, 178, 0, 0, 0, 50, 95, 62, 140, 57, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 97, 97, 97, 55, 100, 52, 57, 45, 101, 55, 100, 101, 45, 52, 51, 55, 53, 45, 97, 52, 57, 54, 45, 101, 53, 54, 54, 98, 54, 52, 55, 98, 56, 48, 100, 0, 0, 0, 0, 0, 0, 0, 179, 0, 0, 0, 50, 169, 162, 24, 246, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 102, 98, 51, 48, 101, 53, 102, 45, 49, 49, 98, 54, 45, 52, 100, 98, 51, 45, 97, 51, 102, 99, 45, 100, 53, 56, 52, 97, 54, 101, 100, 55, 54, 98, 48, 0, 0, 0, 0, 0, 0, 0, 180, 0, 0, 0, 50, 94, 231, 226, 180, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 48, 101, 55, 53, 50, 99, 52, 45, 98, 55, 100, 97, 45, 52, 49, 102, 52, 45, 97, 98, 48, 99, 45, 50, 49, 102, 102, 50, 100, 51, 49, 53, 97, 55, 99, 0, 0, 0, 0, 0, 0, 0, 181, 0, 0, 0, 50, 35, 132, 197, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 54, 55, 100, 99, 49, 49, 45, 53, 102, 53, 51, 45, 52, 99, 99, 98, 45, 97, 56, 49, 100, 45, 56, 52, 52, 53, 100, 56, 99, 49, 55, 55, 54, 98, 0, 0, 0, 0, 0, 0, 0, 182, 0, 0, 0, 50, 113, 10, 179, 163, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 50, 99, 100, 50, 52, 53, 48, 45, 102, 56, 49, 100, 45, 52, 55, 51, 101, 45, 57, 97, 55, 56, 45, 51, 100, 57, 98, 101, 54, 50, 55, 97, 102, 99, 57, 0, 0, 0, 0, 0, 0, 0, 183, 0, 0, 0, 50, 169, 191, 8, 70, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 50, 52, 57, 51, 54, 101, 101, 45, 100, 52, 55, 98, 45, 52, 100, 53, 50, 45, 97, 56, 53, 51, 45, 97, 55, 97, 50, 50, 57, 101, 101, 97, 53, 97, 102, 0, 0, 0, 0, 0, 0, 0, 184, 0, 0, 0, 50, 24, 84, 185, 103, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 48, 98, 48, 99, 101, 102, 100, 45, 100, 52, 100, 101, 45, 52, 99, 99, 97, 45, 57, 55, 52, 102, 45, 100, 102, 48, 55, 53, 48, 56, 49, 54, 99, 101, 99, 0, 0, 0, 0, 0, 0, 0, 185, 0, 0, 0, 50, 35, 29, 5, 110, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 55, 101, 99, 53, 49, 52, 49, 45, 102, 57, 100, 55, 45, 52, 98, 56, 98, 45, 98, 54, 51, 100, 45, 97, 100, 50, 101, 56, 55, 98, 100, 56, 97, 56, 98, 0, 0, 0, 0, 0, 0, 0, 186, 0, 0, 0, 50, 122, 141, 255, 44, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 51, 56, 100, 52, 50, 52, 53, 45, 101, 54, 57, 97, 45, 52, 49, 100, 57, 45, 98, 56, 54, 97, 45, 51, 55, 100, 100, 56, 53, 55, 54, 50, 52, 100, 98, 0, 0, 0, 0, 0, 0, 0, 187, 0, 0, 0, 50, 2, 18, 218, 249, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 99, 102, 49, 100, 56, 51, 48, 45, 97, 101, 53, 101, 45, 52, 99, 48, 53, 45, 56, 50, 98, 55, 45, 57, 57, 56, 56, 101, 97, 57, 52, 52, 98, 55, 54, 0, 0, 0, 0, 0, 0, 0, 188, 0, 0, 0, 50, 206, 187, 54, 252, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 57, 101, 49, 54, 97, 100, 49, 45, 55, 98, 50, 99, 45, 52, 101, 102, 97, 45, 57, 54, 100, 99, 45, 57, 57, 97, 56, 48, 55, 55, 50, 99, 54, 54, 52, 0, 0, 0, 0, 0, 0, 0, 189, 0, 0, 0, 50, 10, 115, 253, 55, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 56, 97, 102, 52, 100, 52, 57, 45, 56, 56, 53, 100, 45, 52, 55, 53, 52, 45, 57, 52, 52, 56, 45, 100, 50, 48, 52, 56, 52, 97, 99, 101, 54, 100, 100, 0, 0, 0, 0, 0, 0, 0, 190, 0, 0, 0, 50, 196, 138, 155, 215, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 56, 57, 48, 49, 55, 49, 100, 45, 97, 100, 51, 53, 45, 52, 54, 54, 52, 45, 98, 51, 99, 99, 45, 49, 55, 54, 49, 102, 55, 54, 98, 97, 51, 56, 102, 0, 0, 0, 0, 0, 0, 0, 191, 0, 0, 0, 50, 74, 1, 3, 211, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 97, 49, 101, 56, 48, 98, 99, 45, 52, 53, 50, 55, 45, 52, 54, 48, 102, 45, 97, 49, 48, 51, 45, 51, 56, 50, 48, 100, 102, 57, 53, 55, 101, 52, 98, 0, 0, 0, 0, 0, 0, 0, 192, 0, 0, 0, 50, 234, 191, 127, 192, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 56, 101, 50, 98, 101, 98, 52, 45, 97, 48, 55, 56, 45, 52, 102, 100, 48, 45, 56, 54, 100, 56, 45, 48, 52, 101, 54, 97, 48, 57, 56, 98, 98, 48, 49, 0, 0, 0, 0, 0, 0, 0, 193, 0, 0, 0, 50, 235, 87, 96, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 50, 53, 55, 48, 99, 52, 51, 45, 99, 51, 101, 56, 45, 52, 100, 49, 97, 45, 97, 54, 49, 54, 45, 52, 101, 50, 102, 101, 49, 102, 52, 97, 54, 99, 49, 0, 0, 0, 0, 0, 0, 0, 194, 0, 0, 0, 50, 64, 244, 184, 118, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 48, 55, 56, 53, 54, 53, 53, 45, 49, 102, 97, 98, 45, 52, 54, 101, 48, 45, 56, 53, 51, 49, 45, 55, 100, 51, 56, 98, 50, 101, 48, 48, 49, 50, 54, 0, 0, 0, 0, 0, 0, 0, 195, 0, 0, 0, 50, 139, 102, 116, 39, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 97, 100, 54, 101, 52, 51, 99, 45, 100, 54, 57, 98, 45, 52, 53, 55, 101, 45, 57, 97, 99, 101, 45, 55, 48, 50, 52, 54, 54, 52, 101, 98, 55, 56, 50, 0, 0, 0, 0, 0, 0, 0, 196, 0, 0, 0, 50, 146, 55, 113, 77, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 51, 56, 55, 99, 51, 53, 97, 45, 52, 52, 52, 101, 45, 52, 52, 51, 97, 45, 57, 50, 57, 50, 45, 57, 52, 55, 53, 55, 99, 99, 98, 55, 56, 53, 99, 0, 0, 0, 0, 0, 0, 0, 197, 0, 0, 0, 50, 184, 43, 223, 231, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 98, 51, 100, 49, 97, 57, 45, 53, 98, 57, 97, 45, 52, 50, 51, 55, 45, 57, 98, 97, 55, 45, 97, 100, 57, 48, 52, 101, 55, 48, 101, 101, 57, 49, 0, 0, 0, 0, 0, 0, 0, 198, 0, 0, 0, 50, 1, 250, 58, 229, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 98, 56, 50, 100, 101, 54, 52, 45, 99, 97, 49, 101, 45, 52, 57, 52, 50, 45, 56, 54, 48, 52, 45, 52, 48, 55, 57, 56, 51, 97, 99, 98, 101, 99, 56, 0, 0, 0, 0, 0, 0, 0, 199, 0, 0, 0, 50, 1, 147, 73, 56, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 56, 101, 50, 56, 54, 50, 45, 99, 56, 98, 101, 45, 52, 101, 52, 51, 45, 97, 54, 48, 56, 45, 52, 56, 102, 50, 102, 99, 53, 52, 51, 54, 98, 49, 0, 0, 0, 0, 0, 0, 0, 200, 0, 0, 0, 50, 238, 194, 70, 209, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 97, 97, 102, 99, 51, 100, 45, 56, 53, 51, 49, 45, 52, 55, 101, 49, 45, 97, 97, 50, 100, 45, 52, 97, 100, 102, 100, 52, 102, 97, 98, 52, 48, 54, 0, 0, 0, 0, 0, 0, 0, 201, 0, 0, 0, 50, 49, 31, 37, 151, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 50, 100, 50, 53, 53, 55, 52, 45, 49, 99, 97, 56, 45, 52, 56, 52, 52, 45, 56, 49, 99, 55, 45, 97, 52, 56, 57, 48, 54, 56, 52, 49, 51, 102, 51, 0, 0, 0, 0, 0, 0, 0, 202, 0, 0, 0, 50, 90, 114, 136, 48, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 48, 56, 51, 49, 52, 98, 50, 45, 99, 48, 102, 53, 45, 52, 100, 97, 100, 45, 56, 55, 102, 51, 45, 52, 97, 54, 54, 52, 54, 53, 56, 54, 51, 57, 50, 0, 0, 0, 0, 0, 0, 0, 203, 0, 0, 0, 50, 56, 75, 243, 176, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 52, 49, 97, 97, 49, 102, 99, 45, 102, 53, 99, 56, 45, 52, 100, 53, 99, 45, 97, 50, 52, 98, 45, 49, 97, 56, 51, 98, 97, 56, 56, 99, 49, 48, 53, 0, 0, 0, 0, 0, 0, 0, 204, 0, 0, 0, 50, 120, 165, 176, 61, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 57, 49, 50, 56, 50, 98, 54, 45, 100, 49, 98, 98, 45, 52, 98, 56, 56, 45, 97, 57, 54, 57, 45, 99, 54, 101, 51, 51, 98, 99, 100, 98, 100, 97, 52, 0, 0, 0, 0, 0, 0, 0, 205, 0, 0, 0, 50, 8, 189, 27, 75, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 97, 100, 55, 56, 55, 48, 54, 45, 48, 97, 57, 56, 45, 52, 97, 99, 101, 45, 97, 54, 97, 53, 45, 54, 56, 49, 50, 102, 55, 55, 57, 52, 49, 55, 101, 0, 0, 0, 0, 0, 0, 0, 206, 0, 0, 0, 50, 210, 70, 239, 205, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 53, 57, 55, 97, 97, 97, 99, 45, 51, 100, 99, 48, 45, 52, 49, 54, 101, 45, 56, 54, 99, 54, 45, 98, 53, 50, 97, 100, 102, 48, 100, 57, 55, 102, 97, 0, 0, 0, 0, 0, 0, 0, 207, 0, 0, 0, 50, 36, 221, 26, 47, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 49, 55, 99, 57, 97, 55, 53, 45, 101, 53, 57, 54, 45, 52, 54, 50, 52, 45, 56, 50, 98, 99, 45, 98, 53, 57, 54, 56, 98, 51, 54, 97, 99, 99, 52, 0, 0, 0, 0, 0, 0, 0, 208, 0, 0, 0, 50, 114, 236, 245, 181, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 97, 57, 100, 50, 52, 97, 53, 45, 54, 50, 51, 50, 45, 52, 99, 51, 97, 45, 57, 97, 102, 56, 45, 52, 99, 97, 53, 53, 52, 55, 98, 98, 98, 57, 49, 0, 0, 0, 0, 0, 0, 0, 209, 0, 0, 0, 50, 193, 244, 89, 214, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 52, 55, 50, 98, 52, 49, 57, 45, 97, 99, 52, 49, 45, 52, 98, 98, 97, 45, 56, 55, 51, 52, 45, 100, 49, 51, 57, 48, 99, 51, 51, 52, 101, 52, 57, 0, 0, 0, 0, 0, 0, 0, 210, 0, 0, 0, 50, 172, 129, 72, 53, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 48, 98, 98, 50, 51, 50, 57, 45, 53, 99, 57, 102, 45, 52, 55, 98, 54, 45, 97, 102, 99, 52, 45, 101, 50, 101, 97, 55, 52, 52, 48, 51, 102, 51, 98, 0, 0, 0, 0, 0, 0, 0, 211, 0, 0, 0, 50, 155, 4, 25, 248, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 100, 56, 52, 102, 101, 52, 51, 45, 50, 102, 99, 97, 45, 52, 48, 57, 54, 45, 97, 51, 53, 54, 45, 53, 48, 55, 49, 49, 98, 101, 48, 54, 56, 51, 51, 0, 0, 0, 0, 0, 0, 0, 212, 0, 0, 0, 50, 134, 34, 130, 114, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 52, 100, 55, 99, 99, 57, 54, 45, 56, 52, 102, 102, 45, 52, 57, 97, 51, 45, 97, 101, 57, 54, 45, 101, 52, 53, 48, 98, 102, 54, 102, 97, 48, 56, 100, 0, 0, 0, 0, 0, 0, 0, 213, 0, 0, 0, 50, 159, 92, 112, 242, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 56, 49, 53, 57, 53, 51, 101, 45, 50, 55, 48, 50, 45, 52, 50, 53, 55, 45, 56, 52, 53, 49, 45, 50, 97, 55, 102, 53, 52, 57, 97, 97, 97, 51, 53, 0, 0, 0, 0, 0, 0, 0, 214, 0, 0, 0, 50, 76, 88, 107, 216, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 51, 98, 102, 49, 49, 51, 102, 45, 99, 97, 49, 52, 45, 52, 49, 56, 53, 45, 57, 54, 54, 101, 45, 97, 56, 98, 53, 51, 52, 101, 57, 102, 57, 100, 52, 0, 0, 0, 0, 0, 0, 0, 215, 0, 0, 0, 50, 148, 192, 11, 41, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 99, 50, 53, 98, 49, 50, 55, 45, 50, 100, 100, 52, 45, 52, 52, 97, 102, 45, 56, 48, 100, 57, 45, 98, 56, 97, 53, 49, 98, 99, 99, 53, 54, 50, 54, 0, 0, 0, 0, 0, 0, 0, 216, 0, 0, 0, 50, 81, 198, 0, 232, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 98, 51, 54, 97, 56, 48, 52, 45, 53, 100, 102, 51, 45, 52, 100, 102, 53, 45, 56, 50, 100, 57, 45, 48, 52, 97, 51, 101, 102, 52, 56, 50, 53, 100, 52, 0, 0, 0, 0, 0, 0, 0, 217, 0, 0, 0, 50, 250, 80, 70, 6, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 50, 54, 102, 56, 56, 99, 98, 45, 52, 52, 57, 57, 45, 52, 102, 51, 52, 45, 57, 97, 100, 52, 45, 100, 49, 57, 101, 49, 100, 97, 56, 99, 49, 57, 49, 0, 0, 0, 0, 0, 0, 0, 218, 0, 0, 0, 50, 80, 149, 72, 144, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 101, 102, 52, 101, 48, 51, 99, 45, 52, 98, 48, 50, 45, 52, 52, 98, 51, 45, 57, 52, 51, 102, 45, 56, 52, 101, 49, 56, 55, 56, 100, 48, 49, 101, 99, 0, 0, 0, 0, 0, 0, 0, 219, 0, 0, 0, 50, 136, 18, 92, 17, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 100, 54, 101, 98, 97, 57, 51, 45, 51, 54, 102, 53, 45, 52, 54, 101, 48, 45, 56, 48, 102, 50, 45, 56, 99, 55, 49, 52, 52, 56, 50, 56, 97, 52, 50, 0, 0, 0, 0, 0, 0, 0, 220, 0, 0, 0, 50, 70, 209, 208, 235, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 57, 102, 57, 57, 50, 97, 56, 45, 98, 55, 55, 51, 45, 52, 101, 97, 54, 45, 98, 51, 99, 57, 45, 99, 54, 53, 54, 51, 55, 98, 52, 100, 54, 57, 53, 0, 0, 0, 0, 0, 0, 0, 221, 0, 0, 0, 50, 245, 45, 118, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 50, 50, 97, 55, 53, 100, 98, 45, 56, 97, 50, 51, 45, 52, 99, 53, 101, 45, 97, 99, 101, 99, 45, 48, 53, 102, 51, 97, 99, 49, 56, 56, 52, 56, 53, 0, 0, 0, 0, 0, 0, 0, 222, 0, 0, 0, 50, 81, 76, 107, 28, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 101, 54, 102, 50, 102, 57, 45, 51, 99, 49, 97, 45, 52, 56, 55, 50, 45, 97, 49, 101, 97, 45, 53, 49, 53, 48, 97, 56, 99, 99, 48, 50, 56, 48, 0, 0, 0, 0, 0, 0, 0, 223, 0, 0, 0, 50, 28, 116, 112, 237, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 100, 97, 48, 97, 98, 99, 52, 45, 101, 102, 50, 57, 45, 52, 52, 53, 97, 45, 98, 54, 56, 55, 45, 48, 99, 55, 102, 50, 97, 49, 57, 101, 97, 102, 97, 0, 0, 0, 0, 0, 0, 0, 224, 0, 0, 0, 50, 8, 13, 133, 135, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 97, 55, 102, 99, 54, 57, 56, 45, 97, 99, 52, 57, 45, 52, 50, 102, 53, 45, 56, 98, 56, 50, 45, 49, 101, 51, 99, 98, 50, 57, 98, 51, 102, 49, 54, 0, 0, 0, 0, 0, 0, 0, 225, 0, 0, 0, 50, 59, 179, 124, 175, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 100, 52, 50, 48, 48, 98, 45, 55, 54, 53, 99, 45, 52, 50, 100, 98, 45, 57, 101, 57, 99, 45, 50, 101, 57, 54, 51, 49, 99, 51, 53, 97, 55, 98, 0, 0, 0, 0, 0, 0, 0, 226, 0, 0, 0, 50, 48, 174, 214, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 50, 49, 101, 99, 56, 100, 52, 45, 56, 98, 53, 51, 45, 52, 97, 57, 55, 45, 97, 101, 56, 51, 45, 51, 50, 98, 97, 55, 101, 54, 99, 55, 54, 51, 53, 0, 0, 0, 0, 0, 0, 0, 227, 0, 0, 0, 50, 17, 252, 244, 184, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 55, 57, 100, 100, 99, 99, 53, 45, 49, 50, 54, 57, 45, 52, 97, 100, 54, 45, 98, 52, 49, 48, 45, 99, 55, 54, 53, 55, 54, 48, 49, 100, 102, 52, 52, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 50, 123, 15, 37, 26, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 56, 55, 53, 51, 56, 98, 97, 45, 100, 102, 52, 48, 45, 52, 57, 56, 53, 45, 97, 100, 54, 49, 45, 53, 55, 49, 57, 100, 97, 102, 98, 49, 54, 98, 101, 0, 0, 0, 0, 0, 0, 0, 229, 0, 0, 0, 50, 128, 166, 190, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 100, 48, 54, 101, 51, 100, 99, 45, 48, 97, 53, 49, 45, 52, 48, 97, 97, 45, 97, 102, 48, 48, 45, 56, 57, 100, 48, 57, 98, 52, 98, 102, 97, 49, 101, 0, 0, 0, 0, 0, 0, 0, 230, 0, 0, 0, 50, 4, 80, 158, 25, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 50, 51, 100, 100, 56, 48, 54, 45, 99, 48, 102, 98, 45, 52, 97, 50, 52, 45, 97, 99, 101, 101, 45, 57, 99, 97, 50, 53, 97, 54, 48, 101, 50, 50, 56, 0, 0, 0, 0, 0, 0, 0, 231, 0, 0, 0, 50, 23, 139, 53, 93, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 48, 99, 54, 49, 97, 48, 98, 45, 52, 52, 99, 50, 45, 52, 57, 49, 55, 45, 57, 98, 49, 97, 45, 100, 102, 52, 100, 52, 97, 50, 98, 99, 100, 53, 53, 0, 0, 0, 0, 0, 0, 0, 232, 0, 0, 0, 50, 218, 178, 115, 31, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 97, 57, 51, 53, 100, 102, 55, 45, 53, 101, 98, 56, 45, 52, 98, 49, 101, 45, 98, 97, 54, 53, 45, 102, 51, 51, 57, 102, 53, 102, 97, 101, 51, 55, 102, 0, 0, 0, 0, 0, 0, 0, 233, 0, 0, 0, 50, 116, 187, 194, 39, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 57, 97, 52, 56, 102, 57, 97, 45, 57, 54, 97, 53, 45, 52, 53, 57, 100, 45, 56, 48, 53, 102, 45, 102, 98, 102, 53, 102, 98, 50, 102, 51, 52, 53, 102, 0, 0, 0, 0, 0, 0, 0, 234, 0, 0, 0, 50, 43, 205, 125, 236, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 57, 49, 99, 97, 97, 54, 100, 45, 56, 100, 53, 48, 45, 52, 57, 48, 57, 45, 57, 98, 99, 54, 45, 50, 55, 99, 57, 101, 100, 57, 55, 53, 99, 101, 97, 0, 0, 0, 0, 0, 0, 0, 235, 0, 0, 0, 50, 28, 95, 118, 161, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 52, 98, 99, 101, 55, 52, 102, 45, 51, 54, 100, 48, 45, 52, 98, 52, 48, 45, 97, 102, 102, 48, 45, 100, 100, 48, 98, 51, 99, 98, 53, 56, 57, 48, 48, 0, 0, 0, 0, 0, 0, 0, 236, 0, 0, 0, 50, 102, 48, 87, 149, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 99, 101, 53, 100, 50, 56, 98, 45, 49, 48, 55, 98, 45, 52, 51, 101, 98, 45, 98, 48, 99, 53, 45, 49, 101, 55, 50, 51, 50, 53, 54, 51, 57, 55, 57, 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, 0, 50, 88, 163, 66, 227, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 52, 56, 99, 54, 52, 50, 101, 45, 99, 51, 54, 50, 45, 52, 54, 100, 51, 45, 97, 57, 48, 48, 45, 97, 57, 102, 102, 53, 51, 57, 100, 100, 55, 57, 101, 0, 0, 0, 0, 0, 0, 0, 238, 0, 0, 0, 50, 251, 150, 227, 67, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 54, 52, 49, 99, 50, 53, 101, 45, 98, 97, 97, 98, 45, 52, 101, 51, 52, 45, 97, 97, 100, 48, 45, 52, 55, 50, 97, 48, 49, 52, 97, 53, 99, 54, 54, 0, 0, 0, 0, 0, 0, 0, 239, 0, 0, 0, 50, 223, 203, 98, 32, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 52, 102, 55, 49, 49, 56, 45, 55, 102, 52, 101, 45, 52, 49, 52, 100, 45, 57, 51, 48, 56, 45, 54, 48, 52, 102, 100, 55, 55, 98, 51, 50, 54, 100, 0, 0, 0, 0, 0, 0, 0, 240, 0, 0, 0, 50, 16, 75, 207, 150, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 99, 50, 56, 57, 53, 54, 48, 45, 54, 49, 100, 97, 45, 52, 57, 57, 101, 45, 56, 50, 49, 55, 45, 102, 51, 50, 55, 101, 54, 51, 98, 98, 54, 100, 97, 0, 0, 0, 0, 0, 0, 0, 241, 0, 0, 0, 50, 81, 150, 196, 67, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 97, 98, 51, 52, 53, 50, 51, 45, 50, 49, 49, 56, 45, 52, 54, 50, 57, 45, 56, 53, 50, 101, 45, 56, 55, 53, 51, 53, 102, 53, 51, 54, 55, 101, 99, 0, 0, 0, 0, 0, 0, 0, 242, 0, 0, 0, 50, 195, 79, 11, 216, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 100, 99, 56, 97, 102, 53, 53, 45, 97, 52, 102, 52, 45, 52, 99, 48, 50, 45, 56, 55, 57, 100, 45, 97, 53, 101, 52, 56, 101, 97, 102, 57, 50, 54, 54, 0, 0, 0, 0, 0, 0, 0, 243, 0, 0, 0, 50, 124, 26, 186, 12, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 100, 49, 51, 52, 57, 98, 57, 45, 102, 55, 53, 55, 45, 52, 50, 56, 100, 45, 57, 56, 51, 55, 45, 100, 52, 97, 97, 100, 99, 56, 98, 51, 53, 102, 98, 0, 0, 0, 0, 0, 0, 0, 244, 0, 0, 0, 50, 143, 72, 43, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 56, 57, 48, 48, 55, 97, 100, 45, 51, 53, 99, 53, 45, 52, 97, 52, 55, 45, 56, 48, 99, 55, 45, 56, 50, 56, 98, 50, 53, 101, 53, 54, 49, 99, 48, 0, 0, 0, 0, 0, 0, 0, 245, 0, 0, 0, 50, 230, 230, 69, 128, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 57, 102, 98, 102, 57, 57, 97, 45, 55, 100, 102, 55, 45, 52, 57, 50, 57, 45, 56, 54, 99, 56, 45, 48, 99, 57, 48, 51, 55, 56, 56, 101, 56, 54, 98, 0, 0, 0, 0, 0, 0, 0, 246, 0, 0, 0, 50, 253, 241, 239, 34, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 102, 51, 102, 48, 99, 54, 55, 45, 57, 54, 52, 53, 45, 52, 99, 53, 100, 45, 57, 57, 50, 99, 45, 98, 54, 49, 97, 56, 56, 99, 57, 49, 54, 100, 51, 0, 0, 0, 0, 0, 0, 0, 247, 0, 0, 0, 50, 40, 177, 231, 250, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 102, 100, 48, 54, 52, 97, 101, 45, 102, 57, 52, 49, 45, 52, 51, 54, 54, 45, 57, 98, 100, 51, 45, 53, 100, 57, 50, 57, 48, 100, 50, 50, 57, 102, 101, 0, 0, 0, 0, 0, 0, 0, 248, 0, 0, 0, 50, 33, 177, 215, 133, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 56, 100, 48, 53, 98, 57, 48, 45, 48, 54, 49, 50, 45, 52, 99, 55, 53, 45, 56, 98, 100, 57, 45, 50, 51, 50, 54, 99, 99, 48, 57, 51, 55, 97, 97, 0, 0, 0, 0, 0, 0, 0, 249, 0, 0, 0, 50, 123, 22, 84, 157, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 50, 102, 99, 98, 54, 53, 48, 45, 102, 51, 50, 102, 45, 52, 54, 49, 53, 45, 97, 56, 53, 54, 45, 48, 101, 55, 56, 55, 57, 56, 102, 52, 97, 51, 57, 0, 0, 0, 0, 0, 0, 0, 250, 0, 0, 0, 50, 103, 48, 222, 220, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 97, 98, 52, 53, 102, 49, 56, 45, 57, 49, 54, 99, 45, 52, 98, 100, 54, 45, 57, 101, 49, 50, 45, 56, 51, 53, 102, 98, 102, 100, 53, 51, 54, 50, 56, 0, 0, 0, 0, 0, 0, 0, 251, 0, 0, 0, 50, 119, 44, 87, 29, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 53, 48, 57, 55, 98, 51, 48, 45, 48, 53, 57, 56, 45, 52, 102, 55, 48, 45, 97, 99, 102, 97, 45, 48, 97, 50, 102, 98, 56, 102, 50, 56, 97, 56, 49, 0, 0, 0, 0, 0, 0, 0, 252, 0, 0, 0, 50, 53, 141, 251, 240, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 53, 101, 101, 48, 99, 53, 48, 45, 55, 99, 54, 99, 45, 52, 52, 99, 100, 45, 97, 48, 101, 102, 45, 101, 52, 49, 48, 99, 101, 102, 48, 56, 100, 57, 55, 0, 0, 0, 0, 0, 0, 0, 253, 0, 0, 0, 50, 95, 75, 163, 194, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 99, 50, 100, 54, 102, 56, 53, 45, 52, 49, 49, 48, 45, 52, 53, 102, 99, 45, 56, 49, 98, 48, 45, 101, 101, 57, 52, 100, 51, 51, 52, 52, 57, 101, 52, 0, 0, 0, 0, 0, 0, 0, 254, 0, 0, 0, 50, 26, 30, 88, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 98, 48, 55, 51, 51, 54, 49, 45, 48, 48, 98, 98, 45, 52, 57, 56, 99, 45, 56, 101, 49, 53, 45, 97, 49, 48, 49, 100, 97, 49, 97, 102, 57, 98, 48, 0, 0, 0, 0, 0, 0, 0, 255, 0, 0, 0, 50, 208, 141, 57, 155, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 48, 102, 55, 97, 101, 48, 54, 45, 48, 48, 102, 102, 45, 52, 97, 102, 52, 45, 56, 97, 54, 56, 45, 48, 50, 98, 51, 102, 57, 49, 97, 52, 51, 51, 101, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 50, 162, 225, 150, 217, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 55, 57, 99, 48, 55, 100, 98, 45, 97, 48, 98, 52, 45, 52, 50, 100, 102, 45, 57, 99, 97, 52, 45, 52, 102, 56, 50, 48, 100, 56, 52, 56, 102, 49, 51, 0, 0, 0, 0, 0, 0, 1, 1, 0, 0, 0, 50, 174, 72, 184, 192, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 99, 49, 48, 52, 99, 55, 53, 45, 55, 100, 54, 48, 45, 52, 54, 54, 97, 45, 97, 53, 55, 50, 45, 51, 99, 100, 49, 49, 49, 49, 57, 101, 97, 102, 98, 0, 0, 0, 0, 0, 0, 1, 2, 0, 0, 0, 50, 13, 229, 220, 98, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 56, 53, 51, 53, 55, 98, 97, 45, 101, 102, 100, 102, 45, 52, 56, 55, 49, 45, 57, 98, 99, 54, 45, 102, 53, 100, 54, 54, 102, 98, 102, 48, 101, 50, 48, 0, 0, 0, 0, 0, 0, 1, 3, 0, 0, 0, 50, 154, 163, 92, 201, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 54, 97, 55, 55, 56, 97, 51, 45, 54, 55, 99, 54, 45, 52, 53, 50, 57, 45, 56, 51, 100, 49, 45, 49, 99, 101, 53, 99, 51, 56, 55, 100, 101, 51, 56, 0, 0, 0, 0, 0, 0, 1, 4, 0, 0, 0, 50, 84, 255, 35, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 48, 98, 99, 57, 48, 57, 48, 45, 100, 50, 52, 102, 45, 52, 102, 98, 54, 45, 56, 97, 52, 49, 45, 48, 98, 102, 48, 102, 99, 56, 55, 97, 56, 51, 56, 0, 0, 0, 0, 0, 0, 1, 5, 0, 0, 0, 50, 244, 87, 93, 208, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 56, 100, 50, 48, 101, 56, 50, 45, 102, 97, 57, 99, 45, 52, 97, 51, 52, 45, 57, 53, 52, 57, 45, 54, 50, 97, 48, 101, 101, 101, 101, 55, 101, 53, 97, 0, 0, 0, 0, 0, 0, 1, 6, 0, 0, 0, 50, 223, 116, 187, 99, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 98, 48, 102, 57, 53, 101, 101, 45, 54, 51, 52, 53, 45, 52, 51, 50, 48, 45, 98, 97, 55, 48, 45, 51, 98, 57, 53, 56, 57, 98, 48, 57, 100, 100, 55, 0, 0, 0, 0, 0, 0, 1, 7, 0, 0, 0, 50, 91, 103, 130, 206, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 56, 54, 55, 57, 53, 97, 56, 45, 101, 99, 99, 54, 45, 52, 100, 57, 57, 45, 57, 55, 57, 53, 45, 100, 50, 53, 48, 57, 53, 98, 53, 100, 55, 57, 98, 0, 0, 0, 0, 0, 0, 1, 8, 0, 0, 0, 50, 78, 62, 154, 27, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 52, 102, 48, 52, 100, 50, 55, 45, 57, 53, 56, 52, 45, 52, 53, 55, 49, 45, 97, 48, 54, 48, 45, 101, 54, 98, 99, 99, 99, 53, 97, 54, 52, 52, 51, 0, 0, 0, 0, 0, 0, 1, 9, 0, 0, 0, 50, 164, 136, 193, 70, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 52, 50, 50, 52, 55, 101, 101, 45, 56, 53, 98, 99, 45, 52, 53, 49, 53, 45, 57, 53, 52, 98, 45, 102, 53, 50, 99, 48, 57, 49, 53, 98, 53, 55, 102, 0, 0, 0, 0, 0, 0, 1, 10, 0, 0, 0, 50, 222, 128, 202, 178, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 54, 53, 55, 52, 101, 55, 48, 45, 100, 51, 52, 54, 45, 52, 49, 49, 98, 45, 97, 102, 100, 57, 45, 57, 57, 51, 99, 97, 54, 50, 99, 52, 98, 55, 48, 0, 0, 0, 0, 0, 0, 1, 11, 0, 0, 0, 50, 208, 126, 241, 240, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 48, 52, 51, 100, 48, 57, 48, 45, 52, 102, 53, 57, 45, 52, 102, 53, 51, 45, 56, 100, 55, 54, 45, 50, 50, 98, 99, 98, 56, 51, 97, 56, 53, 56, 52, 0, 0, 0, 0, 0, 0, 1, 12, 0, 0, 0, 50, 147, 128, 185, 172, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 51, 56, 51, 57, 50, 50, 52, 45, 54, 102, 56, 53, 45, 52, 102, 55, 54, 45, 97, 51, 48, 99, 45, 101, 100, 56, 102, 102, 99, 52, 100, 55, 102, 53, 54, 0, 0, 0, 0, 0, 0, 1, 13, 0, 0, 0, 50, 69, 166, 55, 70, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 97, 48, 51, 97, 52, 49, 99, 45, 51, 55, 55, 100, 45, 52, 101, 49, 53, 45, 57, 57, 98, 102, 45, 51, 48, 97, 100, 56, 100, 48, 57, 49, 49, 97, 52, 0, 0, 0, 0, 0, 0, 1, 14, 0, 0, 0, 50, 3, 110, 108, 140, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 55, 51, 100, 49, 102, 52, 53, 45, 53, 99, 57, 55, 45, 52, 97, 56, 50, 45, 98, 50, 57, 48, 45, 101, 53, 56, 51, 51, 55, 54, 57, 48, 99, 101, 48, 0, 0, 0, 0, 0, 0, 1, 15, 0, 0, 0, 50, 166, 38, 72, 52, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 101, 50, 52, 97, 49, 57, 52, 45, 101, 99, 52, 48, 45, 52, 48, 97, 100, 45, 97, 49, 51, 101, 45, 101, 52, 99, 52, 98, 49, 56, 50, 55, 102, 57, 97, 0, 0, 0, 0, 0, 0, 1, 16, 0, 0, 0, 50, 246, 182, 109, 169, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 102, 98, 49, 50, 48, 52, 45, 100, 57, 100, 99, 45, 52, 50, 97, 98, 45, 57, 51, 51, 100, 45, 54, 54, 57, 101, 48, 48, 101, 101, 52, 98, 54, 54, 0, 0, 0, 0, 0, 0, 1, 17, 0, 0, 0, 50, 190, 73, 2, 49, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 49, 102, 52, 53, 99, 98, 52, 45, 57, 100, 49, 48, 45, 52, 50, 52, 100, 45, 57, 56, 55, 97, 45, 101, 49, 49, 54, 52, 55, 99, 100, 52, 98, 48, 51, 0, 0, 0, 0, 0, 0, 1, 18, 0, 0, 0, 50, 53, 112, 198, 141, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 51, 99, 52, 56, 100, 97, 99, 45, 100, 48, 101, 56, 45, 52, 97, 101, 53, 45, 57, 48, 101, 102, 45, 50, 102, 54, 100, 51, 102, 51, 98, 54, 50, 53, 56, 0, 0, 0, 0, 0, 0, 1, 19, 0, 0, 0, 50, 255, 66, 208, 230, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 53, 49, 49, 50, 102, 49, 48, 45, 49, 55, 53, 51, 45, 52, 49, 52, 54, 45, 98, 97, 51, 56, 45, 97, 101, 101, 99, 52, 100, 55, 52, 100, 102, 48, 48, 0, 0, 0, 0, 0, 0, 1, 20, 0, 0, 0, 50, 188, 179, 74, 46, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 52, 97, 55, 52, 52, 54, 48, 45, 50, 99, 48, 52, 45, 52, 100, 102, 48, 45, 57, 98, 54, 98, 45, 53, 57, 100, 49, 99, 55, 55, 99, 98, 50, 100, 54, 0, 0, 0, 0, 0, 0, 1, 21, 0, 0, 0, 50, 207, 210, 142, 220, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 49, 53, 98, 56, 55, 55, 57, 45, 51, 48, 57, 55, 45, 52, 101, 102, 102, 45, 56, 102, 52, 51, 45, 97, 56, 50, 98, 53, 56, 100, 56, 99, 49, 98, 56, 0, 0, 0, 0, 0, 0, 1, 22, 0, 0, 0, 50, 126, 125, 88, 192, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 53, 57, 49, 48, 53, 102, 51, 45, 98, 55, 51, 48, 45, 52, 54, 50, 49, 45, 57, 56, 55, 53, 45, 49, 101, 52, 54, 49, 52, 97, 102, 54, 50, 55, 97, 0, 0, 0, 0, 0, 0, 1, 23, 0, 0, 0, 50, 247, 92, 228, 42, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 102, 53, 51, 57, 49, 55, 52, 45, 48, 102, 97, 51, 45, 52, 57, 101, 54, 45, 57, 54, 97, 55, 45, 56, 97, 55, 49, 97, 49, 52, 50, 101, 48, 52, 48, 0, 0, 0, 0, 0, 0, 1, 24, 0, 0, 0, 50, 98, 160, 28, 164, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 57, 50, 49, 51, 54, 97, 53, 45, 101, 56, 101, 97, 45, 52, 49, 49, 51, 45, 97, 50, 57, 52, 45, 49, 98, 54, 102, 100, 54, 102, 48, 54, 101, 101, 57, 0, 0, 0, 0, 0, 0, 1, 25, 0, 0, 0, 50, 61, 181, 108, 62, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 49, 54, 50, 57, 54, 52, 52, 45, 97, 98, 99, 54, 45, 52, 48, 54, 51, 45, 98, 101, 49, 49, 45, 99, 101, 48, 101, 50, 54, 53, 101, 100, 51, 53, 57, 0, 0, 0, 0, 0, 0, 1, 26, 0, 0, 0, 50, 140, 123, 176, 41, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 56, 101, 52, 97, 98, 57, 50, 45, 98, 55, 101, 101, 45, 52, 48, 100, 55, 45, 97, 52, 49, 49, 45, 53, 99, 99, 50, 98, 54, 54, 100, 101, 55, 50, 53, 0, 0, 0, 0, 0, 0, 1, 27, 0, 0, 0, 50, 25, 195, 92, 164, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 57, 51, 55, 48, 101, 48, 56, 45, 50, 56, 54, 101, 45, 52, 98, 48, 52, 45, 56, 53, 99, 101, 45, 49, 48, 48, 54, 53, 57, 52, 102, 100, 99, 57, 102, 0, 0, 0, 0, 0, 0, 1, 28, 0, 0, 0, 50, 121, 200, 243, 220, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 56, 97, 100, 54, 99, 51, 57, 45, 51, 50, 97, 48, 45, 52, 97, 50, 55, 45, 98, 48, 55, 48, 45, 54, 51, 53, 101, 48, 55, 50, 51, 101, 100, 97, 97, 0, 0, 0, 0, 0, 0, 1, 29, 0, 0, 0, 50, 127, 56, 29, 68, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 56, 53, 99, 100, 97, 55, 55, 45, 51, 101, 50, 56, 45, 52, 99, 50, 49, 45, 57, 48, 100, 102, 45, 57, 98, 102, 97, 101, 101, 97, 100, 52, 57, 55, 52, 0, 0, 0, 0, 0, 0, 1, 30, 0, 0, 0, 50, 212, 64, 98, 216, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 55, 48, 100, 48, 56, 53, 55, 45, 53, 54, 98, 97, 45, 52, 97, 53, 53, 45, 97, 51, 54, 55, 45, 52, 102, 102, 50, 49, 48, 101, 52, 98, 48, 101, 51, 0, 0, 0, 0, 0, 0, 1, 31, 0, 0, 0, 50, 49, 186, 49, 18, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 99, 50, 102, 53, 48, 97, 57, 45, 101, 52, 54, 100, 45, 52, 99, 55, 101, 45, 98, 100, 99, 48, 45, 101, 101, 57, 100, 50, 100, 99, 101, 54, 49, 57, 101, 0, 0, 0, 0, 0, 0, 1, 32, 0, 0, 0, 50, 217, 38, 196, 40, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 51, 99, 51, 49, 50, 100, 57, 45, 50, 51, 57, 57, 45, 52, 101, 99, 52, 45, 98, 53, 54, 49, 45, 98, 99, 101, 53, 101, 97, 99, 99, 100, 56, 102, 97, 0, 0, 0, 0, 0, 0, 1, 33, 0, 0, 0, 50, 204, 49, 76, 37, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 49, 52, 101, 49, 56, 101, 102, 45, 52, 55, 55, 51, 45, 52, 54, 51, 98, 45, 56, 49, 99, 56, 45, 54, 51, 57, 53, 97, 52, 52, 102, 53, 98, 98, 98, 0, 0, 0, 0, 0, 0, 1, 34, 0, 0, 0, 50, 65, 236, 183, 22, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 101, 101, 49, 53, 50, 50, 100, 45, 101, 54, 99, 50, 45, 52, 102, 97, 101, 45, 56, 57, 55, 102, 45, 56, 99, 50, 101, 54, 53, 57, 49, 51, 101, 52, 101, 0, 0, 0, 0, 0, 0, 1, 35, 0, 0, 0, 50, 214, 21, 237, 154, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 97, 102, 51, 50, 51, 97, 54, 45, 99, 57, 48, 57, 45, 52, 52, 51, 49, 45, 56, 52, 101, 101, 45, 53, 97, 50, 57, 57, 57, 102, 57, 54, 49, 56, 53, 0, 0, 0, 0, 0, 0, 1, 36, 0, 0, 0, 50, 129, 17, 183, 202, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 52, 100, 98, 97, 49, 54, 52, 45, 55, 56, 57, 97, 45, 52, 50, 53, 48, 45, 97, 55, 57, 48, 45, 99, 97, 57, 97, 57, 52, 50, 54, 53, 55, 50, 55, 0, 0, 0, 0, 0, 0, 1, 37, 0, 0, 0, 50, 243, 86, 173, 202, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 101, 48, 51, 49, 56, 55, 53, 45, 98, 56, 98, 49, 45, 52, 101, 54, 99, 45, 57, 53, 53, 49, 45, 99, 57, 49, 98, 57, 101, 50, 102, 49, 99, 56, 51, 0, 0, 0, 0, 0, 0, 1, 38, 0, 0, 0, 50, 135, 190, 114, 62, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 53, 54, 99, 97, 56, 50, 98, 45, 100, 57, 54, 52, 45, 52, 54, 49, 57, 45, 97, 97, 56, 56, 45, 50, 101, 56, 53, 102, 49, 48, 52, 53, 51, 49, 52, 0, 0, 0, 0, 0, 0, 1, 39, 0, 0, 0, 50, 20, 203, 38, 4, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 97, 55, 48, 48, 51, 48, 52, 45, 98, 99, 55, 50, 45, 52, 102, 55, 49, 45, 56, 50, 55, 100, 45, 55, 55, 50, 55, 55, 57, 53, 52, 97, 98, 97, 53, 0, 0, 0, 0, 0, 0, 1, 40, 0, 0, 0, 50, 117, 66, 195, 3, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 56, 52, 55, 53, 53, 52, 48, 45, 98, 55, 50, 53, 45, 52, 97, 50, 101, 45, 56, 100, 51, 100, 45, 56, 97, 55, 50, 98, 53, 49, 55, 51, 99, 101, 51, 0, 0, 0, 0, 0, 0, 1, 41, 0, 0, 0, 50, 28, 247, 85, 219, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 54, 52, 50, 54, 55, 100, 55, 45, 99, 48, 54, 51, 45, 52, 49, 99, 56, 45, 57, 56, 52, 102, 45, 49, 56, 57, 49, 57, 50, 52, 97, 99, 98, 55, 99, 0, 0, 0, 0, 0, 0, 1, 42, 0, 0, 0, 50, 245, 55, 40, 213, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 102, 101, 102, 53, 99, 49, 100, 45, 100, 51, 55, 98, 45, 52, 98, 53, 102, 45, 97, 57, 53, 50, 45, 97, 101, 53, 52, 100, 53, 101, 50, 57, 53, 50, 50, 0, 0, 0, 0, 0, 0, 1, 43, 0, 0, 0, 50, 20, 59, 205, 211, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 97, 51, 56, 56, 101, 54, 51, 45, 98, 56, 100, 53, 45, 52, 51, 49, 52, 45, 98, 49, 102, 98, 45, 57, 98, 57, 99, 54, 51, 57, 98, 57, 55, 101, 57, 0, 0, 0, 0, 0, 0, 1, 44, 0, 0, 0, 50, 113, 47, 227, 228, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 102, 48, 51, 57, 102, 49, 55, 45, 49, 51, 100, 49, 45, 52, 49, 49, 49, 45, 97, 99, 54, 99, 45, 99, 102, 98, 50, 53, 100, 97, 49, 53, 98, 53, 57, 0, 0, 0, 0, 0, 0, 1, 45, 0, 0, 0, 50, 122, 189, 98, 102, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 51, 100, 50, 54, 48, 56, 57, 45, 49, 101, 49, 52, 45, 52, 102, 100, 99, 45, 56, 51, 56, 100, 45, 99, 97, 49, 57, 56, 54, 48, 55, 55, 52, 53, 50, 0, 0, 0, 0, 0, 0, 1, 46, 0, 0, 0, 50, 159, 139, 100, 123, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 57, 99, 97, 56, 48, 56, 56, 45, 55, 56, 99, 57, 45, 52, 48, 54, 52, 45, 97, 97, 52, 98, 45, 56, 49, 49, 97, 51, 51, 50, 50, 102, 102, 52, 55, 0, 0, 0, 0, 0, 0, 1, 47, 0, 0, 0, 50, 0, 83, 121, 179, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 100, 98, 101, 98, 48, 97, 98, 45, 102, 57, 102, 51, 45, 52, 54, 102, 57, 45, 98, 52, 102, 97, 45, 48, 98, 50, 101, 99, 98, 49, 52, 55, 48, 54, 97, 0, 0, 0, 0, 0, 0, 1, 48, 0, 0, 0, 50, 87, 233, 131, 236, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 51, 100, 101, 54, 53, 48, 57, 45, 56, 54, 53, 97, 45, 52, 98, 102, 51, 45, 57, 53, 100, 99, 45, 53, 51, 56, 51, 53, 54, 97, 97, 98, 52, 98, 56, 0, 0, 0, 0, 0, 0, 1, 49, 0, 0, 0, 50, 11, 6, 24, 54, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 49, 102, 51, 99, 101, 50, 102, 45, 97, 97, 102, 54, 45, 52, 51, 98, 100, 45, 57, 54, 53, 53, 45, 50, 97, 101, 55, 57, 54, 55, 99, 50, 99, 98, 57, 0, 0, 0, 0, 0, 0, 1, 50, 0, 0, 0, 50, 23, 118, 203, 83, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 55, 48, 101, 56, 49, 57, 97, 45, 56, 54, 50, 50, 45, 52, 50, 50, 56, 45, 56, 101, 102, 54, 45, 51, 98, 50, 98, 54, 57, 99, 101, 98, 54, 98, 97, 0, 0, 0, 0, 0, 0, 1, 51, 0, 0, 0, 50, 111, 244, 75, 102, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 53, 54, 99, 102, 52, 97, 48, 45, 52, 48, 55, 57, 45, 52, 56, 56, 102, 45, 98, 97, 57, 53, 45, 54, 102, 50, 56, 49, 57, 50, 101, 53, 56, 99, 54, 0, 0, 0, 0, 0, 0, 1, 52, 0, 0, 0, 50, 178, 212, 150, 197, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 48, 100, 101, 97, 48, 56, 48, 45, 57, 101, 101, 55, 45, 52, 49, 100, 55, 45, 57, 100, 52, 99, 45, 57, 52, 101, 100, 51, 97, 98, 54, 99, 97, 98, 48, 0, 0, 0, 0, 0, 0, 1, 53, 0, 0, 0, 50, 32, 173, 253, 164, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 50, 97, 97, 57, 56, 57, 101, 45, 52, 56, 97, 102, 45, 52, 48, 99, 53, 45, 57, 102, 55, 97, 45, 56, 50, 55, 100, 101, 48, 57, 54, 51, 54, 56, 97, 0, 0, 0, 0, 0, 0, 1, 54, 0, 0, 0, 50, 220, 140, 117, 14, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 49, 56, 56, 101, 55, 55, 45, 102, 101, 102, 48, 45, 52, 57, 56, 51, 45, 97, 53, 49, 53, 45, 100, 98, 97, 102, 97, 49, 102, 97, 48, 56, 99, 57, 0, 0, 0, 0, 0, 0, 1, 55, 0, 0, 0, 50, 185, 131, 119, 245, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 56, 52, 53, 51, 55, 99, 48, 45, 55, 57, 100, 102, 45, 52, 55, 101, 51, 45, 57, 55, 56, 98, 45, 100, 100, 97, 51, 102, 100, 57, 53, 52, 51, 56, 48, 0, 0, 0, 0, 0, 0, 1, 56, 0, 0, 0, 50, 163, 47, 130, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 53, 53, 53, 51, 98, 51, 98, 45, 51, 48, 102, 53, 45, 52, 98, 51, 98, 45, 98, 53, 49, 97, 45, 57, 101, 102, 50, 54, 55, 50, 55, 97, 57, 51, 97, 0, 0, 0, 0, 0, 0, 1, 57, 0, 0, 0, 50, 3, 155, 212, 255, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 48, 99, 57, 57, 97, 100, 54, 45, 97, 52, 53, 56, 45, 52, 54, 97, 50, 45, 57, 97, 51, 54, 45, 97, 50, 57, 53, 55, 99, 98, 101, 52, 100, 50, 52, 0, 0, 0, 0, 0, 0, 1, 58, 0, 0, 0, 50, 102, 191, 207, 117, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 102, 48, 55, 101, 54, 48, 99, 45, 101, 97, 100, 54, 45, 52, 99, 97, 101, 45, 97, 50, 48, 52, 45, 51, 99, 50, 54, 53, 102, 99, 56, 99, 97, 99, 56, 0, 0, 0, 0, 0, 0, 1, 59, 0, 0, 0, 50, 36, 198, 111, 6, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 100, 53, 99, 53, 53, 102, 54, 45, 52, 51, 53, 51, 45, 52, 50, 100, 48, 45, 98, 51, 55, 97, 45, 97, 57, 53, 99, 97, 101, 102, 50, 56, 53, 100, 57, 0, 0, 0, 0, 0, 0, 1, 60, 0, 0, 0, 50, 212, 220, 45, 42, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 99, 97, 53, 52, 54, 52, 102, 45, 54, 49, 100, 57, 45, 52, 57, 97, 54, 45, 98, 52, 55, 56, 45, 56, 57, 97, 51, 102, 100, 51, 98, 53, 57, 53, 54, 0, 0, 0, 0, 0, 0, 1, 61, 0, 0, 0, 50, 241, 158, 82, 134, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 50, 100, 50, 97, 53, 102, 49, 45, 51, 49, 56, 50, 45, 52, 98, 102, 97, 45, 98, 54, 56, 102, 45, 57, 55, 102, 55, 54, 53, 49, 100, 50, 51, 51, 97, 0, 0, 0, 0, 0, 0, 1, 62, 0, 0, 0, 50, 203, 216, 222, 221, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 54, 102, 100, 57, 57, 57, 52, 45, 54, 50, 55, 57, 45, 52, 53, 52, 56, 45, 56, 55, 49, 57, 45, 48, 99, 98, 101, 49, 48, 56, 56, 53, 102, 48, 97, 0, 0, 0, 0, 0, 0, 1, 63, 0, 0, 0, 50, 108, 220, 79, 8, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 99, 55, 55, 57, 55, 97, 53, 45, 49, 49, 100, 57, 45, 52, 52, 101, 56, 45, 98, 52, 49, 53, 45, 56, 49, 53, 52, 99, 100, 55, 54, 54, 56, 100, 98, 0, 0, 0, 0, 0, 0, 1, 64, 0, 0, 0, 50, 163, 38, 80, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 54, 54, 50, 48, 101, 98, 97, 45, 101, 51, 98, 101, 45, 52, 54, 51, 97, 45, 57, 100, 49, 56, 45, 101, 100, 101, 100, 53, 102, 102, 102, 54, 54, 52, 102, 0, 0, 0, 0, 0, 0, 1, 65, 0, 0, 0, 50, 52, 201, 138, 42, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 49, 51, 49, 101, 50, 100, 57, 45, 98, 52, 99, 102, 45, 52, 99, 51, 51, 45, 97, 101, 48, 48, 45, 102, 99, 49, 97, 98, 102, 101, 100, 99, 55, 98, 99, 0, 0, 0, 0, 0, 0, 1, 66, 0, 0, 0, 50, 149, 50, 246, 144, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 54, 101, 52, 55, 101, 54, 57, 45, 53, 56, 48, 97, 45, 52, 54, 54, 98, 45, 57, 99, 55, 102, 45, 98, 97, 49, 57, 52, 50, 102, 52, 99, 100, 57, 55, 0, 0, 0, 0, 0, 0, 1, 67, 0, 0, 0, 50, 66, 104, 10, 76, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 99, 53, 50, 101, 54, 98, 55, 45, 102, 53, 54, 53, 45, 52, 50, 56, 53, 45, 56, 97, 101, 56, 45, 100, 52, 50, 101, 102, 97, 55, 101, 48, 55, 52, 102, 0, 0, 0, 0, 0, 0, 1, 68, 0, 0, 0, 50, 10, 101, 113, 82, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 97, 99, 54, 49, 54, 98, 57, 45, 49, 98, 54, 97, 45, 52, 100, 49, 57, 45, 56, 102, 102, 97, 45, 101, 99, 54, 52, 50, 53, 53, 102, 53, 48, 99, 101, 0, 0, 0, 0, 0, 0, 1, 69, 0, 0, 0, 50, 158, 85, 40, 130, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 50, 52, 57, 51, 100, 48, 98, 45, 100, 102, 102, 51, 45, 52, 50, 52, 57, 45, 57, 51, 97, 55, 45, 48, 54, 101, 102, 56, 51, 48, 56, 49, 102, 98, 48, 0, 0, 0, 0, 0, 0, 1, 70, 0, 0, 0, 50, 136, 196, 140, 219, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 99, 48, 57, 100, 53, 57, 52, 45, 99, 56, 57, 98, 45, 52, 57, 52, 54, 45, 56, 50, 48, 56, 45, 102, 57, 50, 55, 51, 101, 56, 57, 52, 51, 101, 52, 0, 0, 0, 0, 0, 0, 1, 71, 0, 0, 0, 50, 247, 157, 175, 159, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 50, 57, 97, 48, 102, 51, 51, 45, 57, 98, 102, 97, 45, 52, 100, 98, 50, 45, 97, 56, 48, 51, 45, 56, 97, 99, 100, 48, 57, 99, 54, 98, 97, 101, 49, 0, 0, 0, 0, 0, 0, 1, 72, 0, 0, 0, 50, 106, 170, 183, 11, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 54, 51, 55, 51, 102, 49, 53, 45, 97, 99, 100, 48, 45, 52, 100, 53, 102, 45, 57, 97, 101, 100, 45, 55, 98, 102, 57, 48, 57, 57, 57, 100, 53, 49, 51, 0, 0, 0, 0, 0, 0, 1, 73, 0, 0, 0, 50, 217, 150, 88, 229, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 100, 49, 56, 50, 97, 56, 50, 45, 101, 49, 57, 53, 45, 52, 102, 54, 100, 45, 97, 56, 98, 51, 45, 55, 57, 56, 52, 99, 55, 100, 55, 97, 55, 53, 97, 0, 0, 0, 0, 0, 0, 1, 74, 0, 0, 0, 50, 102, 204, 181, 178, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 56, 54, 48, 54, 98, 52, 54, 45, 54, 52, 57, 98, 45, 52, 98, 51, 98, 45, 57, 57, 101, 50, 45, 97, 52, 54, 97, 48, 98, 56, 102, 52, 101, 52, 48, 0, 0, 0, 0, 0, 0, 1, 75, 0, 0, 0, 50, 246, 165, 77, 73, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 101, 57, 56, 100, 98, 56, 99, 45, 97, 57, 56, 98, 45, 52, 56, 56, 50, 45, 97, 57, 100, 49, 45, 97, 101, 49, 57, 98, 51, 52, 102, 101, 99, 98, 102, 0, 0, 0, 0, 0, 0, 1, 76, 0, 0, 0, 50, 243, 137, 181, 124, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 49, 101, 57, 50, 53, 52, 51, 45, 48, 100, 98, 49, 45, 52, 56, 50, 100, 45, 57, 102, 57, 102, 45, 57, 49, 98, 101, 101, 56, 102, 57, 57, 55, 57, 97, 0, 0, 0, 0, 0, 0, 1, 77, 0, 0, 0, 50, 241, 50, 86, 27, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 51, 55, 102, 48, 52, 49, 54, 45, 52, 97, 55, 48, 45, 52, 102, 97, 97, 45, 57, 55, 53, 102, 45, 55, 102, 54, 54, 51, 51, 51, 100, 101, 99, 97, 101, 0, 0, 0, 0, 0, 0, 1, 78, 0, 0, 0, 50, 181, 56, 56, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 52, 48, 51, 51, 99, 54, 57, 45, 50, 50, 55, 54, 45, 52, 99, 50, 50, 45, 57, 53, 49, 98, 45, 98, 48, 98, 52, 102, 99, 102, 54, 57, 52, 98, 51, 0, 0, 0, 0, 0, 0, 1, 79, 0, 0, 0, 50, 181, 178, 105, 187, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 52, 102, 100, 55, 102, 52, 102, 45, 99, 55, 99, 56, 45, 52, 98, 101, 102, 45, 98, 100, 56, 101, 45, 53, 55, 51, 54, 51, 51, 51, 51, 55, 52, 55, 49, 0, 0, 0, 0, 0, 0, 1, 80, 0, 0, 0, 50, 116, 176, 208, 187, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 53, 55, 53, 48, 98, 54, 97, 45, 52, 48, 53, 49, 45, 52, 97, 51, 57, 45, 56, 98, 53, 56, 45, 49, 56, 50, 54, 49, 51, 99, 97, 102, 56, 55, 51, 0, 0, 0, 0, 0, 0, 1, 81, 0, 0, 0, 50, 100, 123, 33, 166, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 56, 55, 56, 99, 51, 55, 50, 45, 98, 50, 100, 57, 45, 52, 97, 53, 56, 45, 97, 57, 49, 100, 45, 101, 50, 98, 99, 102, 55, 56, 99, 52, 49, 54, 52, 0, 0, 0, 0, 0, 0, 1, 82, 0, 0, 0, 50, 154, 254, 113, 236, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 102, 50, 101, 99, 53, 54, 50, 45, 99, 53, 52, 97, 45, 52, 53, 98, 101, 45, 57, 98, 53, 56, 45, 48, 100, 97, 54, 97, 101, 97, 51, 53, 99, 97, 48, 0, 0, 0, 0, 0, 0, 1, 83, 0, 0, 0, 50, 39, 98, 182, 0, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 102, 52, 98, 52, 49, 100, 99, 45, 97, 101, 97, 52, 45, 52, 101, 53, 101, 45, 97, 102, 50, 99, 45, 51, 52, 98, 102, 49, 48, 56, 56, 97, 57, 49, 52, 0, 0, 0, 0, 0, 0, 1, 84, 0, 0, 0, 50, 97, 172, 119, 186, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 49, 55, 49, 97, 53, 52, 48, 45, 56, 57, 51, 49, 45, 52, 100, 98, 100, 45, 98, 100, 102, 49, 45, 55, 52, 55, 99, 98, 49, 99, 49, 102, 101, 48, 51, 0, 0, 0, 0, 0, 0, 1, 85, 0, 0, 0, 50, 17, 113, 102, 156, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 99, 53, 49, 101, 98, 101, 53, 45, 102, 53, 100, 98, 45, 52, 55, 50, 52, 45, 97, 57, 57, 53, 45, 52, 101, 54, 98, 53, 99, 99, 100, 48, 97, 102, 53, 0, 0, 0, 0, 0, 0, 1, 86, 0, 0, 0, 50, 21, 164, 134, 234, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 102, 50, 102, 56, 97, 53, 100, 45, 57, 52, 102, 56, 45, 52, 54, 51, 101, 45, 57, 48, 48, 52, 45, 57, 55, 50, 101, 98, 101, 49, 102, 57, 101, 51, 50, 0, 0, 0, 0, 0, 0, 1, 87, 0, 0, 0, 50, 212, 240, 108, 224, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 49, 56, 52, 53, 53, 101, 48, 45, 53, 51, 57, 48, 45, 52, 50, 48, 51, 45, 57, 57, 99, 57, 45, 54, 98, 56, 54, 53, 54, 52, 99, 48, 98, 54, 52, 0, 0, 0, 0, 0, 0, 1, 88, 0, 0, 0, 50, 8, 3, 43, 11, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 54, 48, 53, 51, 57, 56, 57, 45, 99, 102, 50, 50, 45, 52, 50, 54, 97, 45, 57, 53, 100, 55, 45, 97, 99, 101, 49, 55, 51, 55, 57, 100, 100, 49, 55, 0, 0, 0, 0, 0, 0, 1, 89, 0, 0, 0, 50, 96, 235, 122, 198, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 54, 52, 49, 57, 54, 100, 55, 45, 98, 102, 98, 102, 45, 52, 48, 98, 51, 45, 98, 48, 52, 99, 45, 98, 100, 101, 51, 53, 54, 97, 51, 101, 48, 50, 53, 0, 0, 0, 0, 0, 0, 1, 90, 0, 0, 0, 50, 72, 54, 201, 187, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 101, 97, 102, 102, 97, 54, 50, 45, 55, 49, 97, 101, 45, 52, 50, 100, 57, 45, 97, 99, 98, 56, 45, 98, 51, 100, 48, 97, 101, 48, 102, 56, 98, 52, 48, 0, 0, 0, 0, 0, 0, 1, 91, 0, 0, 0, 50, 237, 52, 188, 26, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 99, 101, 101, 51, 99, 52, 99, 45, 54, 57, 102, 48, 45, 52, 57, 100, 50, 45, 97, 98, 49, 101, 45, 57, 57, 51, 51, 55, 54, 102, 50, 100, 101, 97, 56, 0, 0, 0, 0, 0, 0, 1, 92, 0, 0, 0, 50, 222, 208, 9, 245, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 98, 53, 101, 49, 52, 100, 100, 45, 102, 57, 97, 101, 45, 52, 99, 98, 48, 45, 57, 56, 97, 53, 45, 52, 57, 48, 54, 97, 52, 100, 56, 51, 52, 100, 53, 0, 0, 0, 0, 0, 0, 1, 93, 0, 0, 0, 50, 114, 72, 93, 188, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 57, 55, 99, 97, 55, 54, 45, 49, 97, 99, 52, 45, 52, 57, 56, 57, 45, 98, 97, 57, 51, 45, 98, 97, 97, 100, 56, 53, 99, 100, 52, 54, 101, 56, 0, 0, 0, 0, 0, 0, 1, 94, 0, 0, 0, 50, 131, 166, 1, 163, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 102, 51, 98, 102, 51, 56, 100, 45, 51, 98, 52, 98, 45, 52, 55, 102, 99, 45, 97, 49, 99, 48, 45, 57, 53, 57, 55, 52, 48, 54, 100, 99, 56, 52, 101, 0, 0, 0, 0, 0, 0, 1, 95, 0, 0, 0, 50, 141, 218, 156, 195, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 53, 99, 48, 98, 52, 98, 51, 45, 55, 56, 57, 57, 45, 52, 98, 97, 55, 45, 56, 48, 56, 49, 45, 99, 49, 98, 56, 48, 102, 97, 56, 52, 53, 56, 53, 0, 0, 0, 0, 0, 0, 1, 96, 0, 0, 0, 50, 254, 152, 172, 229, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 99, 49, 50, 54, 50, 57, 56, 45, 57, 55, 102, 52, 45, 52, 54, 97, 55, 45, 57, 97, 99, 53, 45, 49, 97, 54, 52, 54, 99, 99, 97, 97, 53, 51, 100, 0, 0, 0, 0, 0, 0, 1, 97, 0, 0, 0, 50, 48, 29, 177, 104, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 57, 98, 48, 50, 55, 98, 45, 57, 57, 99, 101, 45, 52, 51, 98, 52, 45, 97, 102, 97, 55, 45, 50, 49, 101, 56, 57, 98, 97, 50, 100, 51, 99, 100, 0, 0, 0, 0, 0, 0, 1, 98, 0, 0, 0, 50, 163, 160, 24, 134, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 49, 56, 56, 54, 55, 48, 99, 45, 52, 51, 49, 101, 45, 52, 100, 53, 52, 45, 97, 54, 54, 99, 45, 99, 52, 100, 52, 97, 51, 54, 53, 99, 49, 100, 48, 0, 0, 0, 0, 0, 0, 1, 99, 0, 0, 0, 50, 116, 199, 94, 230, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 53, 102, 56, 57, 98, 98, 101, 45, 56, 102, 48, 57, 45, 52, 49, 57, 49, 45, 57, 52, 101, 54, 45, 98, 49, 52, 52, 101, 97, 52, 49, 52, 52, 100, 55, 0, 0, 0, 0, 0, 0, 1, 100, 0, 0, 0, 50, 1, 17, 235, 175, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 97, 50, 57, 50, 50, 52, 53, 45, 57, 99, 56, 54, 45, 52, 55, 101, 102, 45, 97, 98, 97, 102, 45, 53, 57, 102, 53, 100, 56, 51, 55, 56, 102, 101, 56, 0, 0, 0, 0, 0, 0, 1, 101, 0, 0, 0, 50, 246, 172, 233, 44, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 50, 48, 50, 54, 99, 50, 48, 45, 53, 53, 53, 99, 45, 52, 100, 50, 57, 45, 98, 55, 49, 52, 45, 98, 53, 54, 99, 49, 52, 98, 52, 56, 49, 102, 100, 0, 0, 0, 0, 0, 0, 1, 102, 0, 0, 0, 50, 82, 134, 154, 74, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 100, 55, 100, 57, 53, 55, 54, 45, 49, 102, 97, 48, 45, 52, 51, 53, 53, 45, 57, 52, 56, 99, 45, 52, 55, 53, 101, 100, 97, 99, 54, 55, 51, 55, 48, 0, 0, 0, 0, 0, 0, 1, 103, 0, 0, 0, 50, 6, 210, 242, 228, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 52, 52, 100, 98, 102, 56, 102, 45, 102, 100, 48, 54, 45, 52, 51, 54, 55, 45, 98, 99, 102, 102, 45, 57, 97, 99, 99, 54, 53, 48, 102, 49, 57, 102, 49, 0, 0, 0, 0, 0, 0, 1, 104, 0, 0, 0, 50, 106, 87, 150, 81, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 55, 51, 53, 48, 54, 54, 53, 45, 56, 55, 54, 99, 45, 52, 53, 99, 56, 45, 98, 51, 56, 100, 45, 52, 99, 56, 56, 98, 99, 52, 50, 50, 55, 51, 102, 0, 0, 0, 0, 0, 0, 1, 105, 0, 0, 0, 50, 247, 186, 169, 115, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 102, 101, 99, 50, 52, 56, 53, 45, 56, 102, 55, 57, 45, 52, 97, 49, 54, 45, 98, 48, 54, 97, 45, 57, 102, 101, 102, 99, 100, 102, 100, 55, 98, 48, 100, 0, 0, 0, 0, 0, 0, 1, 106, 0, 0, 0, 50, 21, 69, 180, 228, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 99, 56, 56, 55, 100, 52, 50, 45, 101, 56, 55, 55, 45, 52, 51, 49, 51, 45, 57, 98, 53, 53, 45, 48, 97, 98, 101, 48, 97, 54, 101, 54, 52, 56, 54, 0, 0, 0, 0, 0, 0, 1, 107, 0, 0, 0, 50, 137, 7, 139, 60, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 57, 55, 50, 50, 48, 49, 98, 45, 98, 57, 51, 102, 45, 52, 50, 50, 54, 45, 98, 55, 48, 55, 45, 99, 52, 100, 50, 102, 57, 99, 55, 57, 102, 51, 54, 0, 0, 0, 0, 0, 0, 1, 108, 0, 0, 0, 50, 3, 146, 6, 243, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 54, 49, 100, 48, 51, 48, 45, 102, 97, 98, 57, 45, 52, 100, 97, 50, 45, 56, 50, 48, 102, 45, 56, 52, 56, 55, 99, 102, 55, 54, 97, 51, 52, 51, 0, 0, 0, 0, 0, 0, 1, 109, 0, 0, 0, 50, 227, 47, 35, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 55, 56, 101, 48, 54, 57, 54, 45, 53, 98, 100, 55, 45, 52, 97, 52, 99, 45, 56, 54, 53, 48, 45, 57, 99, 102, 99, 52, 97, 52, 100, 52, 48, 57, 99, 0, 0, 0, 0, 0, 0, 1, 110, 0, 0, 0, 50, 66, 236, 248, 149, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 56, 101, 57, 54, 51, 101, 56, 45, 54, 98, 97, 49, 45, 52, 56, 51, 57, 45, 56, 50, 54, 56, 45, 57, 98, 56, 98, 57, 98, 50, 52, 97, 98, 101, 98, 0, 0, 0, 0, 0, 0, 1, 111, 0, 0, 0, 50, 171, 157, 92, 129, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 57, 49, 57, 53, 53, 52, 50, 45, 54, 49, 101, 48, 45, 52, 98, 97, 55, 45, 97, 48, 56, 48, 45, 57, 99, 51, 50, 52, 100, 56, 97, 98, 51, 53, 51, 0, 0, 0, 0, 0, 0, 1, 112, 0, 0, 0, 50, 167, 35, 248, 233, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 100, 99, 51, 98, 57, 97, 100, 45, 49, 52, 99, 99, 45, 52, 98, 49, 57, 45, 57, 53, 55, 98, 45, 101, 102, 56, 55, 49, 48, 100, 99, 100, 102, 49, 102, 0, 0, 0, 0, 0, 0, 1, 113, 0, 0, 0, 50, 77, 181, 125, 61, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 49, 48, 51, 48, 54, 56, 101, 45, 57, 54, 56, 100, 45, 52, 51, 57, 53, 45, 97, 57, 99, 50, 45, 101, 100, 56, 102, 102, 50, 102, 51, 97, 53, 49, 101, 0, 0, 0, 0, 0, 0, 1, 114, 0, 0, 0, 50, 21, 227, 197, 31, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 55, 55, 54, 55, 54, 49, 54, 45, 99, 53, 51, 53, 45, 52, 57, 54, 49, 45, 56, 51, 50, 52, 45, 55, 99, 51, 48, 57, 97, 56, 50, 56, 98, 101, 98, 0, 0, 0, 0, 0, 0, 1, 115, 0, 0, 0, 50, 243, 220, 104, 94, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 100, 53, 48, 49, 51, 101, 49, 45, 100, 57, 101, 97, 45, 52, 100, 100, 52, 45, 56, 97, 50, 55, 45, 98, 55, 53, 97, 100, 99, 101, 52, 49, 55, 51, 56, 0, 0, 0, 0, 0, 0, 1, 116, 0, 0, 0, 50, 231, 179, 103, 249, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 53, 49, 54, 52, 54, 49, 52, 45, 102, 51, 49, 57, 45, 52, 55, 55, 56, 45, 97, 54, 53, 52, 45, 48, 53, 55, 53, 49, 102, 100, 51, 54, 50, 51, 53, 0, 0, 0, 0, 0, 0, 1, 117, 0, 0, 0, 50, 126, 115, 202, 53, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 100, 97, 56, 54, 56, 48, 98, 45, 48, 57, 48, 54, 45, 52, 50, 49, 97, 45, 57, 48, 99, 99, 45, 54, 102, 99, 100, 48, 56, 102, 50, 102, 55, 54, 49, 0, 0, 0, 0, 0, 0, 1, 118, 0, 0, 0, 50, 217, 144, 58, 241, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 101, 99, 49, 56, 56, 101, 55, 45, 98, 52, 55, 100, 45, 52, 101, 102, 53, 45, 56, 54, 50, 50, 45, 50, 101, 55, 100, 51, 55, 53, 57, 51, 57, 57, 49, 0, 0, 0, 0, 0, 0, 1, 119, 0, 0, 0, 50, 60, 97, 250, 8, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 56, 100, 57, 101, 100, 48, 97, 45, 98, 49, 97, 48, 45, 52, 51, 53, 53, 45, 98, 56, 55, 99, 45, 53, 56, 56, 98, 57, 48, 55, 50, 100, 51, 51, 98, 0, 0, 0, 0, 0, 0, 1, 120, 0, 0, 0, 50, 48, 102, 87, 83, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 55, 55, 49, 55, 101, 56, 55, 45, 57, 55, 101, 100, 45, 52, 57, 54, 56, 45, 97, 55, 50, 52, 45, 50, 52, 54, 48, 99, 102, 57, 52, 51, 100, 49, 57, 0, 0, 0, 0, 0, 0, 1, 121, 0, 0, 0, 50, 25, 244, 0, 156, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 57, 99, 100, 101, 51, 52, 51, 45, 57, 49, 101, 52, 45, 52, 51, 102, 54, 45, 57, 49, 56, 50, 45, 53, 100, 101, 56, 100, 98, 101, 102, 48, 51, 102, 49, 0, 0, 0, 0, 0, 0, 1, 122, 0, 0, 0, 50, 246, 237, 87, 68, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 97, 99, 102, 54, 99, 56, 50, 45, 56, 101, 97, 99, 45, 52, 51, 49, 50, 45, 57, 56, 53, 48, 45, 101, 99, 57, 53, 99, 55, 55, 51, 53, 57, 100, 54, 0, 0, 0, 0, 0, 0, 1, 123, 0, 0, 0, 50, 204, 3, 125, 4, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 54, 101, 48, 97, 102, 102, 54, 45, 99, 100, 53, 57, 45, 52, 52, 50, 55, 45, 98, 101, 100, 100, 45, 97, 54, 50, 101, 97, 49, 100, 53, 101, 54, 100, 101, 0, 0, 0, 0, 0, 0, 1, 124, 0, 0, 0, 50, 170, 108, 213, 29, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 52, 52, 51, 102, 102, 49, 100, 45, 99, 53, 97, 98, 45, 52, 100, 100, 52, 45, 97, 54, 57, 51, 45, 102, 57, 57, 99, 100, 49, 99, 101, 102, 102, 48, 55, 0, 0, 0, 0, 0, 0, 1, 125, 0, 0, 0, 50, 128, 43, 232, 174, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 49, 49, 48, 102, 97, 101, 97, 45, 55, 101, 52, 55, 45, 52, 55, 101, 48, 45, 56, 55, 52, 100, 45, 97, 55, 54, 51, 56, 53, 52, 56, 48, 53, 48, 56, 0, 0, 0, 0, 0, 0, 1, 126, 0, 0, 0, 50, 145, 117, 101, 173, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 57, 97, 99, 55, 98, 51, 102, 45, 100, 102, 50, 52, 45, 52, 51, 52, 101, 45, 97, 100, 102, 48, 45, 57, 52, 55, 97, 98, 97, 50, 49, 57, 102, 48, 100, 0, 0, 0, 0, 0, 0, 1, 127, 0, 0, 0, 50, 13, 187, 149, 76, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 51, 100, 54, 97, 98, 54, 56, 45, 49, 98, 56, 54, 45, 52, 56, 57, 101, 45, 97, 53, 48, 55, 45, 49, 102, 56, 102, 97, 102, 54, 54, 100, 102, 54, 102, 0, 0, 0, 0, 0, 0, 1, 128, 0, 0, 0, 50, 111, 112, 163, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 50, 53, 57, 99, 49, 101, 102, 45, 100, 54, 99, 48, 45, 52, 98, 54, 52, 45, 57, 50, 98, 48, 45, 56, 99, 49, 51, 52, 102, 48, 48, 102, 56, 97, 98, 0, 0, 0, 0, 0, 0, 1, 129, 0, 0, 0, 50, 142, 122, 220, 234, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 55, 50, 100, 51, 53, 101, 56, 45, 102, 49, 99, 101, 45, 52, 52, 99, 49, 45, 98, 51, 51, 100, 45, 97, 55, 57, 99, 54, 101, 53, 49, 100, 48, 51, 54, 0, 0, 0, 0, 0, 0, 1, 130, 0, 0, 0, 50, 82, 99, 182, 51, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 102, 48, 100, 100, 51, 54, 52, 45, 54, 97, 53, 48, 45, 52, 51, 50, 57, 45, 56, 102, 97, 55, 45, 53, 52, 56, 54, 56, 56, 97, 56, 97, 50, 48, 100, 0, 0, 0, 0, 0, 0, 1, 131, 0, 0, 0, 50, 10, 163, 189, 253, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 101, 53, 53, 98, 102, 100, 53, 45, 100, 49, 54, 101, 45, 52, 98, 53, 50, 45, 98, 54, 55, 53, 45, 54, 52, 49, 98, 53, 49, 102, 50, 55, 99, 100, 97, 0, 0, 0, 0, 0, 0, 1, 132, 0, 0, 0, 50, 61, 101, 41, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 50, 49, 97, 102, 56, 54, 102, 45, 52, 57, 49, 57, 45, 52, 56, 57, 53, 45, 97, 57, 99, 97, 45, 101, 53, 50, 100, 55, 48, 98, 57, 98, 49, 98, 100, 0, 0, 0, 0, 0, 0, 1, 133, 0, 0, 0, 50, 127, 196, 41, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 99, 54, 51, 99, 52, 98, 101, 45, 54, 99, 99, 52, 45, 52, 99, 56, 98, 45, 98, 48, 99, 102, 45, 99, 101, 48, 50, 57, 53, 50, 100, 101, 52, 56, 53, 0, 0, 0, 0, 0, 0, 1, 134, 0, 0, 0, 50, 77, 23, 131, 224, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 50, 101, 99, 51, 52, 52, 56, 45, 98, 52, 99, 99, 45, 52, 50, 50, 98, 45, 97, 102, 57, 101, 45, 57, 55, 100, 48, 48, 49, 57, 54, 50, 52, 51, 101, 0, 0, 0, 0, 0, 0, 1, 135, 0, 0, 0, 50, 1, 165, 253, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 102, 51, 54, 57, 54, 100, 51, 45, 48, 55, 101, 98, 45, 52, 57, 98, 97, 45, 97, 51, 101, 52, 45, 97, 100, 56, 52, 54, 101, 55, 57, 48, 97, 100, 53, 0, 0, 0, 0, 0, 0, 1, 136, 0, 0, 0, 50, 173, 222, 134, 76, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 101, 99, 99, 99, 98, 99, 54, 45, 56, 53, 48, 49, 45, 52, 51, 53, 99, 45, 57, 100, 51, 99, 45, 56, 102, 55, 50, 99, 53, 52, 56, 100, 102, 97, 48, 0, 0, 0, 0, 0, 0, 1, 137, 0, 0, 0, 50, 4, 125, 181, 134, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 100, 48, 100, 50, 51, 53, 49, 45, 53, 98, 52, 101, 45, 52, 56, 50, 53, 45, 57, 99, 55, 100, 45, 51, 50, 56, 52, 101, 53, 56, 49, 102, 102, 98, 98, 0, 0, 0, 0, 0, 0, 1, 138, 0, 0, 0, 50, 167, 49, 98, 115, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 97, 51, 53, 99, 53, 52, 48, 45, 52, 57, 97, 53, 45, 52, 97, 98, 56, 45, 98, 51, 98, 97, 45, 53, 52, 102, 98, 102, 50, 48, 101, 101, 50, 98, 102, 0, 0, 0, 0, 0, 0, 1, 139, 0, 0, 0, 50, 84, 208, 197, 136, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 49, 54, 99, 100, 100, 99, 99, 45, 55, 53, 98, 54, 45, 52, 99, 97, 52, 45, 98, 102, 54, 53, 45, 57, 102, 101, 48, 102, 100, 51, 102, 97, 48, 53, 53, 0, 0, 0, 0, 0, 0, 1, 140, 0, 0, 0, 50, 216, 46, 54, 40, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 52, 48, 50, 50, 52, 99, 53, 45, 48, 57, 52, 100, 45, 52, 101, 57, 53, 45, 97, 55, 48, 98, 45, 51, 102, 57, 53, 101, 49, 99, 51, 55, 49, 99, 56, 0, 0, 0, 0, 0, 0, 1, 141, 0, 0, 0, 50, 50, 20, 83, 47, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 56, 51, 102, 50, 102, 51, 50, 45, 99, 49, 55, 100, 45, 52, 99, 98, 53, 45, 97, 50, 52, 55, 45, 102, 50, 98, 55, 99, 99, 52, 98, 48, 100, 48, 50, 0, 0, 0, 0, 0, 0, 1, 142, 0, 0, 0, 50, 185, 253, 153, 148, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 101, 102, 56, 100, 97, 54, 51, 45, 53, 57, 48, 53, 45, 52, 54, 102, 54, 45, 97, 55, 99, 54, 45, 100, 57, 55, 55, 55, 101, 97, 56, 50, 48, 57, 56, 0, 0, 0, 0, 0, 0, 1, 143, 0, 0, 0, 50, 163, 180, 88, 168, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 51, 100, 99, 48, 100, 98, 53, 45, 102, 57, 56, 99, 45, 52, 52, 50, 101, 45, 57, 99, 52, 53, 45, 97, 53, 53, 49, 97, 53, 57, 98, 102, 55, 49, 49, 0, 0, 0, 0, 0, 0, 1, 144, 0, 0, 0, 50, 170, 85, 3, 97, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 53, 49, 98, 100, 102, 54, 102, 45, 53, 50, 54, 51, 45, 52, 55, 55, 49, 45, 57, 57, 53, 48, 45, 55, 50, 51, 100, 98, 54, 53, 99, 101, 56, 100, 50, 0, 0, 0, 0, 0, 0, 1, 145, 0, 0, 0, 50, 123, 208, 22, 66, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 54, 57, 57, 52, 49, 98, 101, 45, 48, 49, 52, 51, 45, 52, 102, 56, 98, 45, 57, 50, 49, 56, 45, 52, 53, 98, 52, 99, 101, 52, 99, 57, 101, 52, 49, 0, 0, 0, 0, 0, 0, 1, 146, 0, 0, 0, 50, 68, 51, 175, 221, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 56, 57, 48, 55, 97, 99, 51, 45, 54, 99, 100, 102, 45, 52, 98, 49, 56, 45, 98, 97, 53, 99, 45, 102, 50, 55, 100, 56, 55, 101, 48, 55, 99, 54, 56, 0, 0, 0, 0, 0, 0, 1, 147, 0, 0, 0, 50, 68, 53, 204, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 53, 54, 97, 48, 51, 56, 48, 45, 56, 52, 57, 49, 45, 52, 99, 51, 97, 45, 97, 100, 48, 99, 45, 100, 54, 55, 55, 56, 51, 55, 48, 49, 102, 54, 53, 0, 0, 0, 0, 0, 0, 1, 148, 0, 0, 0, 50, 175, 46, 110, 246, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 52, 51, 102, 53, 52, 57, 54, 45, 102, 55, 57, 56, 45, 52, 52, 57, 97, 45, 57, 50, 102, 97, 45, 101, 48, 51, 54, 54, 99, 100, 55, 50, 57, 49, 102, 0, 0, 0, 0, 0, 0, 1, 149, 0, 0, 0, 50, 22, 35, 133, 246, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 100, 53, 54, 97, 55, 54, 99, 45, 99, 50, 56, 48, 45, 52, 56, 100, 53, 45, 98, 49, 53, 100, 45, 53, 102, 57, 55, 48, 98, 54, 51, 97, 99, 50, 53, 0, 0, 0, 0, 0, 0, 1, 150, 0, 0, 0, 50, 32, 59, 175, 182, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 48, 99, 99, 52, 102, 52, 53, 45, 97, 102, 48, 52, 45, 52, 100, 51, 55, 45, 97, 53, 101, 102, 45, 53, 57, 54, 52, 49, 101, 54, 55, 48, 49, 53, 100, 0, 0, 0, 0, 0, 0, 1, 151, 0, 0, 0, 50, 176, 228, 53, 15, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 57, 97, 56, 52, 101, 97, 99, 45, 53, 101, 52, 98, 45, 52, 57, 55, 101, 45, 98, 49, 48, 48, 45, 50, 48, 97, 53, 97, 55, 56, 99, 55, 100, 48, 100, 0, 0, 0, 0, 0, 0, 1, 152, 0, 0, 0, 50, 225, 73, 37, 21, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 54, 49, 54, 55, 52, 102, 99, 45, 51, 57, 97, 97, 45, 52, 54, 51, 55, 45, 98, 48, 99, 100, 45, 56, 102, 101, 52, 50, 50, 48, 102, 54, 49, 102, 102, 0, 0, 0, 0, 0, 0, 1, 153, 0, 0, 0, 50, 194, 89, 218, 214, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 97, 99, 102, 98, 53, 52, 50, 45, 48, 98, 100, 99, 45, 52, 97, 99, 53, 45, 56, 53, 48, 49, 45, 98, 55, 51, 53, 56, 99, 101, 99, 50, 55, 53, 51, 0, 0, 0, 0, 0, 0, 1, 154, 0, 0, 0, 50, 5, 189, 205, 153, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 100, 99, 57, 51, 57, 101, 45, 50, 54, 98, 54, 45, 52, 55, 49, 55, 45, 97, 101, 51, 48, 45, 56, 50, 97, 48, 50, 101, 99, 54, 99, 48, 52, 50, 0, 0, 0, 0, 0, 0, 1, 155, 0, 0, 0, 50, 53, 14, 164, 11, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 57, 101, 54, 57, 56, 49, 48, 45, 48, 51, 49, 54, 45, 52, 48, 49, 99, 45, 98, 51, 51, 52, 45, 97, 53, 49, 99, 57, 100, 101, 52, 53, 49, 97, 101, 0, 0, 0, 0, 0, 0, 1, 156, 0, 0, 0, 50, 0, 188, 136, 200, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 48, 53, 55, 53, 101, 50, 101, 45, 53, 57, 49, 55, 45, 52, 55, 57, 51, 45, 97, 48, 50, 51, 45, 48, 100, 53, 57, 57, 56, 48, 51, 49, 102, 102, 53, 0, 0, 0, 0, 0, 0, 1, 157, 0, 0, 0, 50, 175, 6, 176, 101, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 102, 102, 100, 55, 54, 57, 52, 45, 99, 48, 49, 97, 45, 52, 99, 97, 55, 45, 97, 54, 98, 100, 45, 54, 50, 102, 55, 50, 51, 100, 57, 52, 48, 52, 98, 0, 0, 0, 0, 0, 0, 1, 158, 0, 0, 0, 50, 120, 180, 209, 105, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 100, 57, 53, 50, 48, 55, 57, 45, 48, 98, 97, 56, 45, 52, 98, 51, 97, 45, 56, 56, 53, 51, 45, 100, 102, 52, 99, 52, 55, 50, 102, 99, 102, 53, 49, 0, 0, 0, 0, 0, 0, 1, 159, 0, 0, 0, 50, 40, 102, 128, 47, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 49, 102, 48, 99, 56, 49, 54, 45, 52, 53, 51, 98, 45, 52, 53, 48, 102, 45, 98, 101, 54, 100, 45, 98, 57, 99, 55, 49, 55, 99, 97, 50, 55, 97, 102, 0, 0, 0, 0, 0, 0, 1, 160, 0, 0, 0, 50, 205, 49, 110, 66, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 48, 50, 50, 102, 50, 49, 56, 45, 50, 101, 101, 55, 45, 52, 53, 54, 53, 45, 98, 55, 99, 57, 45, 48, 57, 98, 97, 48, 56, 51, 100, 52, 50, 100, 99, 0, 0, 0, 0, 0, 0, 1, 161, 0, 0, 0, 50, 206, 188, 106, 81, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 102, 48, 100, 52, 99, 99, 98, 45, 97, 102, 98, 100, 45, 52, 99, 56, 99, 45, 98, 49, 48, 102, 45, 53, 52, 101, 99, 51, 51, 102, 54, 55, 51, 52, 99, 0, 0, 0, 0, 0, 0, 1, 162, 0, 0, 0, 50, 0, 86, 38, 175, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 100, 102, 51, 101, 53, 98, 55, 45, 53, 55, 102, 54, 45, 52, 99, 53, 100, 45, 57, 51, 49, 48, 45, 57, 53, 51, 50, 98, 51, 102, 52, 99, 55, 101, 49, 0, 0, 0, 0, 0, 0, 1, 163, 0, 0, 0, 50, 176, 29, 118, 60, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 101, 99, 53, 97, 97, 55, 102, 45, 97, 97, 54, 50, 45, 52, 49, 57, 56, 45, 56, 55, 102, 100, 45, 100, 48, 97, 51, 55, 99, 49, 56, 57, 53, 49, 49, 0, 0, 0, 0, 0, 0, 1, 164, 0, 0, 0, 50, 104, 190, 105, 251, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 49, 50, 99, 49, 101, 101, 54, 45, 51, 53, 50, 98, 45, 52, 101, 100, 55, 45, 97, 102, 55, 55, 45, 55, 54, 53, 51, 100, 57, 53, 52, 56, 55, 48, 100, 0, 0, 0, 0, 0, 0, 1, 165, 0, 0, 0, 50, 119, 193, 121, 107, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 57, 51, 56, 48, 98, 100, 49, 45, 97, 50, 52, 48, 45, 52, 51, 101, 54, 45, 57, 53, 50, 99, 45, 53, 101, 56, 52, 51, 50, 55, 54, 97, 53, 100, 48, 0, 0, 0, 0, 0, 0, 1, 166, 0, 0, 0, 50, 236, 109, 240, 169, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 51, 100, 101, 48, 99, 52, 55, 45, 49, 51, 55, 100, 45, 52, 49, 51, 53, 45, 97, 54, 49, 99, 45, 100, 53, 53, 55, 51, 55, 97, 50, 54, 54, 50, 57, 0, 0, 0, 0, 0, 0, 1, 167, 0, 0, 0, 50, 143, 100, 66, 27, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 53, 56, 50, 51, 99, 98, 98, 45, 102, 51, 100, 98, 45, 52, 54, 101, 56, 45, 57, 49, 55, 101, 45, 54, 97, 55, 48, 54, 49, 99, 56, 102, 100, 54, 54, 0, 0, 0, 0, 0, 0, 1, 168, 0, 0, 0, 50, 105, 155, 87, 149, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 52, 51, 98, 52, 100, 52, 54, 45, 97, 97, 52, 52, 45, 52, 49, 48, 55, 45, 97, 101, 97, 97, 45, 99, 53, 49, 48, 53, 55, 51, 49, 101, 54, 52, 50, 0, 0, 0, 0, 0, 0, 1, 169, 0, 0, 0, 50, 144, 108, 71, 162, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 53, 49, 55, 55, 101, 49, 99, 45, 102, 56, 99, 98, 45, 52, 56, 101, 52, 45, 57, 48, 50, 51, 45, 50, 49, 101, 53, 102, 56, 50, 53, 55, 55, 50, 102, 0, 0, 0, 0, 0, 0, 1, 170, 0, 0, 0, 50, 155, 205, 208, 19, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 100, 102, 56, 99, 57, 55, 53, 45, 52, 50, 55, 53, 45, 52, 51, 52, 100, 45, 98, 98, 100, 49, 45, 56, 99, 51, 56, 97, 102, 100, 57, 98, 54, 51, 100, 0, 0, 0, 0, 0, 0, 1, 171, 0, 0, 0, 50, 230, 83, 183, 245, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 54, 101, 101, 99, 53, 53, 51, 45, 48, 55, 55, 102, 45, 52, 55, 57, 56, 45, 98, 50, 56, 54, 45, 51, 52, 57, 102, 50, 97, 50, 102, 49, 57, 102, 57, 0, 0, 0, 0, 0, 0, 1, 172, 0, 0, 0, 50, 39, 168, 173, 85, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 50, 50, 53, 49, 102, 98, 97, 45, 49, 52, 53, 101, 45, 52, 99, 51, 55, 45, 97, 98, 52, 53, 45, 102, 55, 57, 48, 100, 98, 57, 53, 100, 100, 48, 99, 0, 0, 0, 0, 0, 0, 1, 173, 0, 0, 0, 50, 235, 110, 60, 58, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 52, 101, 101, 56, 51, 49, 45, 51, 48, 101, 100, 45, 52, 102, 48, 102, 45, 56, 98, 55, 55, 45, 56, 57, 102, 55, 51, 53, 98, 98, 101, 102, 48, 50, 0, 0, 0, 0, 0, 0, 1, 174, 0, 0, 0, 50, 236, 18, 101, 174, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 57, 102, 51, 101, 54, 97, 49, 45, 55, 55, 53, 53, 45, 52, 101, 50, 101, 45, 97, 51, 98, 53, 45, 55, 100, 56, 54, 48, 97, 51, 99, 56, 102, 98, 98, 0, 0, 0, 0, 0, 0, 1, 175, 0, 0, 0, 50, 110, 100, 172, 137, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 55, 97, 99, 100, 49, 97, 102, 45, 56, 98, 52, 57, 45, 52, 101, 57, 98, 45, 56, 48, 53, 102, 45, 48, 100, 55, 97, 53, 100, 102, 56, 98, 55, 51, 102, 0, 0, 0, 0, 0, 0, 1, 176, 0, 0, 0, 50, 63, 129, 5, 49, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 55, 101, 101, 49, 99, 52, 97, 45, 50, 57, 51, 101, 45, 52, 48, 55, 48, 45, 56, 55, 49, 53, 45, 55, 50, 54, 101, 102, 102, 57, 52, 53, 54, 102, 102, 0, 0, 0, 0, 0, 0, 1, 177, 0, 0, 0, 50, 138, 37, 36, 56, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 97, 102, 55, 57, 99, 102, 55, 45, 57, 101, 50, 102, 45, 52, 49, 98, 52, 45, 57, 100, 50, 56, 45, 49, 100, 101, 53, 57, 101, 100, 101, 48, 101, 99, 100, 0, 0, 0, 0, 0, 0, 1, 178, 0, 0, 0, 50, 184, 117, 92, 59, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 102, 101, 57, 98, 53, 53, 56, 45, 100, 56, 53, 56, 45, 52, 56, 102, 51, 45, 97, 101, 54, 100, 45, 51, 50, 99, 97, 56, 53, 102, 99, 54, 98, 97, 100, 0, 0, 0, 0, 0, 0, 1, 179, 0, 0, 0, 50, 123, 9, 252, 56, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 48, 55, 101, 99, 56, 55, 53, 45, 102, 57, 49, 100, 45, 52, 99, 98, 98, 45, 97, 97, 53, 101, 45, 98, 100, 50, 99, 102, 52, 53, 51, 57, 102, 50, 100, 0, 0, 0, 0, 0, 0, 1, 180, 0, 0, 0, 50, 149, 107, 34, 250, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 51, 51, 52, 102, 55, 57, 100, 45, 54, 54, 50, 48, 45, 52, 97, 98, 54, 45, 56, 53, 53, 99, 45, 49, 100, 50, 56, 97, 101, 100, 55, 102, 48, 100, 102, 0, 0, 0, 0, 0, 0, 1, 181, 0, 0, 0, 50, 203, 51, 33, 48, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 55, 98, 55, 98, 51, 55, 53, 45, 55, 55, 102, 49, 45, 52, 101, 99, 99, 45, 97, 53, 99, 56, 45, 100, 56, 48, 55, 97, 49, 49, 50, 98, 100, 51, 48, 0, 0, 0, 0, 0, 0, 1, 182, 0, 0, 0, 50, 88, 24, 24, 17, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 49, 51, 50, 55, 98, 52, 97, 45, 98, 50, 101, 54, 45, 52, 54, 48, 51, 45, 97, 50, 102, 52, 45, 99, 55, 53, 57, 102, 101, 100, 49, 52, 51, 48, 101, 0, 0, 0, 0, 0, 0, 1, 183, 0, 0, 0, 50, 65, 40, 97, 211, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 99, 97, 102, 55, 100, 57, 55, 45, 48, 51, 57, 49, 45, 52, 48, 53, 102, 45, 57, 53, 101, 49, 45, 100, 99, 53, 53, 99, 52, 57, 57, 55, 51, 52, 101, 0, 0, 0, 0, 0, 0, 1, 184, 0, 0, 0, 50, 150, 87, 45, 38, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 101, 53, 98, 100, 50, 49, 53, 45, 56, 52, 97, 48, 45, 52, 53, 101, 51, 45, 97, 56, 97, 102, 45, 97, 50, 56, 98, 52, 52, 51, 102, 53, 56, 52, 50, 0, 0, 0, 0, 0, 0, 1, 185, 0, 0, 0, 50, 21, 231, 69, 18, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 52, 98, 49, 98, 49, 55, 53, 45, 98, 52, 100, 55, 45, 52, 52, 99, 101, 45, 56, 49, 99, 100, 45, 48, 51, 54, 49, 55, 52, 48, 101, 48, 54, 100, 97, 0, 0, 0, 0, 0, 0, 1, 186, 0, 0, 0, 50, 153, 70, 249, 17, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 56, 101, 51, 100, 102, 49, 99, 45, 97, 57, 49, 57, 45, 52, 49, 49, 57, 45, 57, 101, 53, 51, 45, 49, 53, 55, 49, 49, 53, 54, 50, 53, 50, 102, 50, 0, 0, 0, 0, 0, 0, 1, 187, 0, 0, 0, 50, 26, 188, 133, 241, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 53, 49, 50, 50, 49, 101, 97, 45, 99, 54, 49, 101, 45, 52, 51, 99, 50, 45, 56, 50, 55, 97, 45, 51, 57, 54, 52, 55, 54, 56, 55, 49, 48, 102, 99, 0, 0, 0, 0, 0, 0, 1, 188, 0, 0, 0, 50, 98, 177, 245, 221, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 54, 98, 53, 102, 99, 56, 99, 45, 98, 56, 51, 99, 45, 52, 55, 54, 102, 45, 56, 98, 50, 55, 45, 54, 52, 55, 98, 48, 54, 48, 48, 50, 97, 56, 49, 0, 0, 0, 0, 0, 0, 1, 189, 0, 0, 0, 50, 104, 112, 210, 110, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 99, 101, 99, 98, 49, 50, 54, 45, 55, 49, 54, 97, 45, 52, 52, 99, 54, 45, 98, 97, 51, 49, 45, 56, 98, 52, 53, 102, 52, 51, 51, 98, 48, 102, 51, 0, 0, 0, 0, 0, 0, 1, 190, 0, 0, 0, 50, 67, 58, 62, 253, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 48, 100, 49, 48, 55, 54, 55, 45, 54, 99, 57, 102, 45, 52, 56, 48, 56, 45, 56, 51, 102, 55, 45, 56, 55, 52, 51, 50, 102, 51, 52, 101, 99, 50, 100, 0, 0, 0, 0, 0, 0, 1, 191, 0, 0, 0, 50, 255, 33, 193, 53, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 53, 101, 100, 51, 97, 51, 98, 45, 49, 102, 102, 48, 45, 52, 102, 56, 56, 45, 57, 99, 55, 99, 45, 101, 56, 52, 53, 97, 99, 52, 48, 98, 49, 99, 52, 0, 0, 0, 0, 0, 0, 1, 192, 0, 0, 0, 50, 40, 164, 103, 82, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 49, 50, 54, 55, 100, 102, 102, 45, 102, 51, 49, 100, 45, 52, 52, 54, 102, 45, 97, 97, 97, 52, 45, 98, 48, 98, 97, 56, 53, 98, 100, 50, 101, 97, 57, 0, 0, 0, 0, 0, 0, 1, 193, 0, 0, 0, 50, 180, 175, 114, 47, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 55, 49, 102, 100, 98, 99, 55, 45, 53, 49, 48, 98, 45, 52, 99, 48, 51, 45, 97, 51, 98, 57, 45, 102, 54, 56, 101, 55, 51, 97, 54, 102, 97, 49, 50, 0, 0, 0, 0, 0, 0, 1, 194, 0, 0, 0, 50, 196, 57, 103, 225, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 52, 50, 50, 97, 97, 49, 49, 45, 100, 98, 98, 54, 45, 52, 53, 101, 54, 45, 57, 100, 55, 57, 45, 97, 101, 98, 50, 102, 50, 52, 101, 49, 98, 48, 57, 0, 0, 0, 0, 0, 0, 1, 195, 0, 0, 0, 50, 165, 117, 136, 201, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 48, 51, 99, 49, 51, 48, 48, 45, 53, 99, 52, 48, 45, 52, 97, 57, 50, 45, 57, 48, 57, 54, 45, 99, 102, 97, 56, 54, 99, 54, 99, 53, 50, 57, 55, 0, 0, 0, 0, 0, 0, 1, 196, 0, 0, 0, 50, 255, 64, 239, 216, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 57, 49, 48, 98, 48, 55, 56, 45, 101, 57, 101, 101, 45, 52, 53, 50, 56, 45, 57, 48, 101, 54, 45, 48, 56, 54, 53, 54, 49, 49, 50, 57, 97, 53, 50, 0, 0, 0, 0, 0, 0, 1, 197, 0, 0, 0, 50, 159, 217, 33, 172, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 48, 99, 54, 49, 49, 56, 100, 45, 51, 56, 97, 50, 45, 52, 101, 99, 98, 45, 97, 101, 102, 98, 45, 53, 55, 57, 99, 48, 97, 51, 48, 51, 99, 48, 51, 0, 0, 0, 0, 0, 0, 1, 198, 0, 0, 0, 50, 181, 119, 48, 89, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 52, 98, 99, 51, 97, 100, 52, 45, 48, 54, 99, 50, 45, 52, 53, 48, 100, 45, 98, 51, 102, 55, 45, 102, 55, 97, 101, 53, 97, 56, 53, 55, 98, 55, 52, 0, 0, 0, 0, 0, 0, 1, 199, 0, 0, 0, 50, 104, 109, 145, 57, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 57, 102, 54, 52, 97, 56, 100, 45, 97, 102, 56, 52, 45, 52, 100, 54, 100, 45, 57, 51, 53, 53, 45, 51, 53, 49, 52, 102, 101, 54, 51, 48, 49, 55, 49, 0, 0, 0, 0, 0, 0, 1, 200, 0, 0, 0, 50, 20, 30, 103, 108, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 101, 51, 56, 51, 100, 100, 56, 45, 56, 53, 98, 54, 45, 52, 57, 53, 48, 45, 56, 48, 55, 99, 45, 53, 97, 101, 55, 52, 49, 51, 102, 51, 51, 99, 98, 0, 0, 0, 0, 0, 0, 1, 201, 0, 0, 0, 50, 97, 220, 49, 208, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 50, 101, 52, 101, 49, 55, 99, 45, 102, 50, 54, 53, 45, 52, 102, 49, 50, 45, 97, 57, 49, 52, 45, 49, 49, 98, 52, 55, 48, 57, 51, 49, 56, 101, 53, 0, 0, 0, 0, 0, 0, 1, 202, 0, 0, 0, 50, 145, 246, 24, 12, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 56, 97, 50, 99, 55, 101, 102, 45, 52, 54, 97, 48, 45, 52, 50, 102, 56, 45, 57, 101, 100, 99, 45, 52, 53, 100, 55, 49, 102, 52, 49, 55, 102, 98, 55, 0, 0, 0, 0, 0, 0, 1, 203, 0, 0, 0, 50, 150, 15, 188, 145, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 54, 98, 102, 97, 101, 55, 100, 45, 101, 51, 100, 55, 45, 52, 57, 97, 100, 45, 56, 53, 50, 57, 45, 50, 53, 100, 57, 51, 54, 98, 55, 53, 57, 97, 55, 0, 0, 0, 0, 0, 0, 1, 204, 0, 0, 0, 50, 10, 80, 2, 194, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 99, 53, 52, 55, 102, 49, 50, 45, 57, 101, 52, 51, 45, 52, 102, 98, 56, 45, 56, 54, 49, 48, 45, 49, 54, 57, 57, 98, 50, 99, 49, 55, 50, 100, 50, 0, 0, 0, 0, 0, 0, 1, 205, 0, 0, 0, 50, 197, 71, 162, 131, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 49, 53, 48, 51, 50, 57, 56, 45, 56, 56, 99, 52, 45, 52, 48, 100, 49, 45, 97, 50, 50, 99, 45, 56, 100, 100, 97, 57, 49, 99, 54, 50, 50, 51, 50, 0, 0, 0, 0, 0, 0, 1, 206, 0, 0, 0, 50, 104, 155, 246, 20, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 48, 56, 98, 100, 51, 98, 97, 45, 56, 53, 99, 98, 45, 52, 99, 99, 53, 45, 57, 56, 102, 101, 45, 100, 55, 98, 52, 55, 101, 99, 50, 51, 55, 101, 53, 0, 0, 0, 0, 0, 0, 1, 207, 0, 0, 0, 50, 94, 254, 116, 82, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 56, 98, 97, 49, 99, 55, 97, 45, 98, 49, 100, 102, 45, 52, 48, 101, 53, 45, 98, 56, 49, 101, 45, 99, 101, 56, 52, 57, 53, 49, 99, 56, 101, 53, 100, 0, 0, 0, 0, 0, 0, 1, 208, 0, 0, 0, 50, 169, 40, 127, 55, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 97, 97, 52, 100, 101, 50, 100, 45, 51, 101, 101, 97, 45, 52, 102, 52, 98, 45, 56, 99, 55, 50, 45, 49, 52, 102, 54, 100, 54, 55, 56, 48, 50, 97, 97, 0, 0, 0, 0, 0, 0, 1, 209, 0, 0, 0, 50, 27, 116, 117, 201, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 51, 51, 56, 52, 52, 57, 54, 45, 56, 101, 50, 57, 45, 52, 52, 51, 54, 45, 98, 50, 99, 101, 45, 52, 48, 97, 100, 99, 50, 56, 55, 99, 55, 100, 55, 0, 0, 0, 0, 0, 0, 1, 210, 0, 0, 0, 50, 102, 136, 148, 111, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 49, 99, 54, 99, 99, 97, 54, 45, 52, 52, 97, 99, 45, 52, 51, 50, 52, 45, 98, 97, 50, 50, 45, 99, 101, 101, 51, 57, 99, 101, 101, 49, 49, 49, 98, 0, 0, 0, 0, 0, 0, 1, 211, 0, 0, 0, 50, 179, 82, 191, 167, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 102, 49, 99, 97, 49, 99, 48, 45, 50, 51, 102, 57, 45, 52, 48, 49, 48, 45, 97, 55, 101, 49, 45, 54, 98, 57, 102, 101, 100, 48, 99, 52, 101, 48, 54, 0, 0, 0, 0, 0, 0, 1, 212, 0, 0, 0, 50, 207, 130, 46, 170, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 56, 56, 51, 51, 101, 98, 57, 45, 49, 98, 101, 52, 45, 52, 102, 101, 48, 45, 57, 101, 52, 97, 45, 100, 97, 52, 98, 53, 102, 50, 56, 56, 48, 54, 56, 0, 0, 0, 0, 0, 0, 1, 213, 0, 0, 0, 50, 98, 141, 167, 245, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 49, 55, 97, 55, 100, 97, 57, 45, 102, 98, 54, 102, 45, 52, 100, 99, 98, 45, 98, 53, 51, 99, 45, 51, 56, 102, 98, 102, 99, 102, 56, 101, 101, 102, 100, 0, 0, 0, 0, 0, 0, 1, 214, 0, 0, 0, 50, 249, 191, 157, 147, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 50, 52, 97, 100, 98, 52, 97, 45, 53, 49, 50, 49, 45, 52, 52, 54, 51, 45, 56, 57, 97, 52, 45, 100, 53, 49, 97, 102, 48, 99, 51, 55, 98, 53, 101, 0, 0, 0, 0, 0, 0, 1, 215, 0, 0, 0, 50, 34, 250, 155, 2, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 100, 97, 97, 48, 57, 57, 50, 45, 99, 49, 55, 51, 45, 52, 102, 55, 101, 45, 97, 55, 54, 98, 45, 98, 48, 49, 49, 52, 100, 53, 98, 53, 54, 49, 98, 0, 0, 0, 0, 0, 0, 1, 216, 0, 0, 0, 50, 23, 163, 106, 238, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 101, 98, 97, 51, 49, 48, 100, 45, 55, 56, 55, 57, 45, 52, 54, 99, 55, 45, 97, 52, 52, 98, 45, 49, 48, 56, 99, 50, 49, 99, 54, 57, 97, 99, 99, 0, 0, 0, 0, 0, 0, 1, 217, 0, 0, 0, 50, 80, 104, 237, 19, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 100, 98, 52, 51, 48, 57, 99, 45, 51, 100, 49, 54, 45, 52, 54, 54, 100, 45, 97, 55, 50, 52, 45, 51, 56, 55, 50, 52, 53, 98, 102, 48, 102, 55, 49, 0, 0, 0, 0, 0, 0, 1, 218, 0, 0, 0, 50, 5, 46, 211, 144, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 102, 101, 99, 53, 56, 55, 52, 45, 50, 52, 97, 50, 45, 52, 54, 101, 98, 45, 57, 98, 99, 101, 45, 99, 101, 55, 48, 51, 101, 56, 102, 49, 54, 52, 56, 0, 0, 0, 0, 0, 0, 1, 219, 0, 0, 0, 50, 236, 151, 198, 251, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 102, 98, 52, 52, 54, 99, 97, 45, 49, 100, 50, 49, 45, 52, 53, 51, 54, 45, 56, 49, 98, 51, 45, 99, 52, 97, 55, 54, 98, 97, 54, 56, 52, 98, 53, 0, 0, 0, 0, 0, 0, 1, 220, 0, 0, 0, 50, 143, 138, 30, 82, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 51, 99, 49, 54, 53, 50, 55, 45, 102, 48, 50, 51, 45, 52, 52, 48, 97, 45, 56, 51, 57, 102, 45, 52, 49, 52, 48, 99, 50, 102, 57, 100, 55, 48, 97, 0, 0, 0, 0, 0, 0, 1, 221, 0, 0, 0, 50, 92, 145, 123, 118, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 50, 48, 49, 97, 49, 99, 51, 45, 54, 57, 50, 50, 45, 52, 101, 49, 100, 45, 97, 101, 49, 54, 45, 52, 56, 98, 100, 54, 98, 49, 50, 99, 49, 53, 49, 0, 0, 0, 0, 0, 0, 1, 222, 0, 0, 0, 50, 156, 99, 76, 172, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 57, 102, 100, 56, 55, 50, 54, 45, 48, 48, 51, 56, 45, 52, 98, 49, 53, 45, 98, 53, 52, 51, 45, 101, 51, 52, 102, 56, 55, 98, 49, 97, 56, 57, 48, 0, 0, 0, 0, 0, 0, 1, 223, 0, 0, 0, 50, 229, 7, 26, 178, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 53, 98, 51, 48, 102, 102, 55, 45, 55, 98, 57, 101, 45, 52, 102, 102, 57, 45, 56, 99, 57, 101, 45, 57, 51, 98, 99, 101, 54, 57, 50, 57, 52, 101, 101, 0, 0, 0, 0, 0, 0, 1, 224, 0, 0, 0, 50, 193, 234, 51, 74, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 54, 56, 52, 48, 48, 97, 100, 45, 57, 49, 49, 97, 45, 52, 99, 51, 50, 45, 56, 99, 52, 97, 45, 98, 101, 52, 57, 53, 101, 56, 57, 51, 53, 98, 50, 0, 0, 0, 0, 0, 0, 1, 225, 0, 0, 0, 50, 95, 205, 40, 221, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 54, 102, 57, 56, 99, 54, 101, 45, 48, 100, 100, 99, 45, 52, 48, 52, 48, 45, 56, 57, 54, 50, 45, 49, 98, 48, 98, 98, 51, 100, 52, 97, 54, 101, 102, 0, 0, 0, 0, 0, 0, 1, 226, 0, 0, 0, 50, 169, 253, 162, 49, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 99, 98, 52, 55, 101, 100, 49, 45, 57, 48, 52, 97, 45, 52, 97, 48, 51, 45, 57, 52, 48, 57, 45, 52, 53, 51, 101, 55, 100, 57, 97, 97, 98, 98, 53, 0, 0, 0, 0, 0, 0, 1, 227, 0, 0, 0, 50, 91, 59, 66, 167, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 48, 98, 50, 100, 100, 101, 53, 45, 100, 55, 49, 98, 45, 52, 99, 49, 50, 45, 97, 102, 98, 56, 45, 98, 100, 102, 54, 48, 50, 99, 52, 57, 100, 100, 50, 0, 0, 0, 0, 0, 0, 1, 228, 0, 0, 0, 50, 56, 38, 154, 252, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 51, 101, 56, 55, 99, 52, 50, 45, 50, 99, 98, 49, 45, 52, 50, 51, 55, 45, 97, 49, 56, 102, 45, 48, 100, 57, 48, 48, 56, 98, 48, 97, 100, 102, 101, 0, 0, 0, 0, 0, 0, 1, 229, 0, 0, 0, 50, 197, 37, 22, 80, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 97, 50, 51, 99, 97, 55, 98, 45, 48, 98, 51, 56, 45, 52, 51, 51, 57, 45, 56, 102, 54, 55, 45, 49, 97, 52, 55, 101, 99, 48, 99, 49, 56, 53, 54, 0, 0, 0, 0, 0, 0, 1, 230, 0, 0, 0, 50, 71, 148, 131, 69, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 97, 102, 100, 55, 55, 54, 53, 45, 98, 57, 48, 56, 45, 52, 102, 54, 53, 45, 98, 50, 56, 54, 45, 101, 100, 101, 101, 48, 50, 48, 99, 55, 57, 97, 102, 0, 0, 0, 0, 0, 0, 1, 231, 0, 0, 0, 50, 100, 110, 217, 209, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 55, 99, 53, 102, 99, 100, 53, 45, 50, 97, 98, 101, 45, 52, 55, 102, 101, 45, 97, 52, 100, 102, 45, 50, 53, 49, 48, 55, 51, 102, 98, 51, 100, 97, 100, 0, 0, 0, 0, 0, 0, 1, 232, 0, 0, 0, 50, 104, 174, 133, 244, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 99, 99, 49, 51, 98, 49, 99, 45, 97, 97, 97, 99, 45, 52, 100, 50, 99, 45, 97, 55, 54, 97, 45, 100, 55, 50, 97, 54, 57, 102, 53, 97, 52, 99, 101, 0, 0, 0, 0, 0, 0, 1, 233, 0, 0, 0, 50, 205, 206, 197, 54, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 52, 52, 97, 100, 50, 56, 57, 45, 56, 56, 54, 49, 45, 52, 101, 98, 100, 45, 97, 101, 48, 48, 45, 57, 55, 56, 98, 97, 98, 49, 53, 102, 52, 52, 51, 0, 0, 0, 0, 0, 0, 1, 234, 0, 0, 0, 50, 35, 235, 157, 98, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 99, 99, 55, 52, 48, 98, 57, 97, 45, 97, 48, 98, 98, 45, 52, 54, 97, 55, 45, 57, 53, 48, 54, 45, 48, 98, 99, 53, 99, 99, 101, 52, 98, 49, 50, 97, 0, 0, 0, 0, 0, 0, 1, 235, 0, 0, 0, 50, 43, 230, 70, 180, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 55, 53, 56, 48, 100, 56, 50, 45, 49, 52, 57, 52, 45, 52, 50, 97, 98, 45, 97, 57, 56, 97, 45, 98, 102, 54, 101, 57, 99, 54, 55, 53, 53, 50, 53, 0, 0, 0, 0, 0, 0, 1, 236, 0, 0, 0, 50, 29, 151, 52, 174, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 50, 52, 52, 56, 57, 48, 51, 45, 100, 48, 99, 50, 45, 52, 48, 98, 57, 45, 97, 100, 57, 100, 45, 50, 57, 49, 49, 51, 51, 57, 52, 55, 98, 99, 51, 0, 0, 0, 0, 0, 0, 1, 237, 0, 0, 0, 50, 192, 234, 202, 243, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 55, 100, 53, 51, 49, 102, 98, 45, 53, 102, 50, 56, 45, 52, 49, 57, 52, 45, 56, 51, 48, 101, 45, 100, 57, 57, 48, 49, 49, 54, 52, 51, 100, 100, 97, 0, 0, 0, 0, 0, 0, 1, 238, 0, 0, 0, 50, 58, 140, 210, 134, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 102, 51, 49, 55, 57, 53, 101, 102, 45, 101, 50, 53, 102, 45, 52, 53, 57, 48, 45, 57, 97, 56, 49, 45, 98, 55, 53, 54, 50, 57, 53, 98, 97, 55, 97, 102, 0, 0, 0, 0, 0, 0, 1, 239, 0, 0, 0, 50, 148, 82, 4, 208, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 99, 98, 49, 55, 50, 97, 53, 45, 56, 54, 50, 51, 45, 52, 53, 52, 99, 45, 97, 102, 97, 51, 45, 101, 102, 56, 97, 102, 53, 51, 57, 51, 54, 54, 53, 0, 0, 0, 0, 0, 0, 1, 240, 0, 0, 0, 50, 184, 226, 137, 107, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 54, 55, 50, 57, 102, 102, 49, 45, 101, 51, 48, 55, 45, 52, 101, 53, 49, 45, 57, 52, 55, 101, 45, 57, 57, 52, 55, 48, 54, 56, 98, 101, 53, 99, 54, 0, 0, 0, 0, 0, 0, 1, 241, 0, 0, 0, 50, 174, 3, 82, 100, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 98, 100, 51, 48, 51, 102, 99, 45, 49, 53, 51, 102, 45, 52, 48, 49, 49, 45, 57, 98, 102, 102, 45, 98, 52, 51, 49, 55, 56, 50, 48, 54, 102, 54, 100, 0, 0, 0, 0, 0, 0, 1, 242, 0, 0, 0, 50, 27, 252, 61, 240, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 101, 97, 52, 57, 57, 49, 52, 45, 102, 54, 49, 100, 45, 52, 55, 54, 99, 45, 57, 97, 48, 98, 45, 102, 102, 54, 100, 101, 100, 100, 52, 50, 50, 57, 53, 0, 0, 0, 0, 0, 0, 1, 243, 0, 0, 0, 50, 84, 7, 162, 91, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 98, 57, 97, 101, 57, 53, 45, 98, 101, 51, 57, 45, 52, 48, 51, 54, 45, 98, 48, 51, 102, 45, 99, 51, 49, 55, 102, 53, 48, 52, 97, 102, 101, 100, 0, 0, 0, 0, 0, 0, 1, 244, 0, 0, 0, 50, 108, 176, 244, 2, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 100, 56, 54, 51, 52, 54, 51, 99, 45, 53, 102, 54, 48, 45, 52, 98, 53, 54, 45, 97, 57, 51, 51, 45, 48, 56, 49, 56, 102, 51, 97, 100, 102, 54, 52, 98, 0, 0, 0, 0, 0, 0, 1, 245, 0, 0, 0, 50, 63, 153, 240, 96, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 53, 100, 52, 54, 102, 98, 50, 49, 45, 53, 55, 53, 57, 45, 52, 101, 56, 97, 45, 98, 54, 97, 99, 45, 57, 49, 49, 102, 50, 48, 57, 48, 54, 55, 102, 49, 0, 0, 0, 0, 0, 0, 1, 246, 0, 0, 0, 50, 250, 223, 142, 184, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 101, 57, 56, 100, 48, 55, 49, 45, 50, 52, 51, 53, 45, 52, 98, 99, 52, 45, 56, 51, 48, 53, 45, 101, 97, 97, 102, 57, 50, 53, 48, 97, 48, 99, 97, 0, 0, 0, 0, 0, 0, 1, 247, 0, 0, 0, 50, 183, 58, 56, 4, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 50, 100, 54, 48, 57, 100, 53, 45, 101, 51, 51, 101, 45, 52, 97, 50, 56, 45, 98, 57, 51, 102, 45, 57, 99, 100, 55, 54, 99, 51, 48, 54, 100, 99, 97, 0, 0, 0, 0, 0, 0, 1, 248, 0, 0, 0, 50, 112, 178, 4, 112, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 97, 53, 56, 50, 52, 57, 51, 45, 99, 98, 55, 50, 45, 52, 53, 55, 101, 45, 98, 52, 49, 48, 45, 53, 102, 49, 52, 51, 56, 50, 101, 52, 102, 54, 98, 0, 0, 0, 0, 0, 0, 1, 249, 0, 0, 0, 50, 39, 234, 209, 224, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 101, 99, 57, 53, 98, 101, 102, 45, 57, 102, 54, 51, 45, 52, 53, 98, 55, 45, 97, 52, 100, 50, 45, 57, 101, 97, 100, 57, 51, 50, 100, 54, 51, 55, 100, 0, 0, 0, 0, 0, 0, 1, 250, 0, 0, 0, 50, 142, 207, 26, 190, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 48, 57, 57, 50, 100, 57, 102, 52, 45, 98, 100, 50, 55, 45, 52, 51, 48, 57, 45, 56, 102, 57, 49, 45, 101, 54, 57, 56, 101, 51, 54, 51, 49, 50, 53, 56, 0, 0, 0, 0, 0, 0, 1, 251, 0, 0, 0, 50, 233, 103, 211, 190, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 48, 48, 98, 54, 56, 55, 55, 45, 102, 48, 99, 56, 45, 52, 97, 51, 100, 45, 57, 98, 51, 51, 45, 100, 50, 57, 99, 48, 99, 98, 101, 98, 50, 51, 49, 0, 0, 0, 0, 0, 0, 1, 252, 0, 0, 0, 50, 0, 179, 240, 222, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 100, 97, 100, 48, 56, 51, 102, 45, 100, 56, 54, 102, 45, 52, 56, 97, 98, 45, 97, 102, 100, 102, 45, 98, 102, 48, 49, 100, 102, 57, 56, 51, 53, 48, 101, 0, 0, 0, 0, 0, 0, 1, 253, 0, 0, 0, 50, 181, 121, 207, 88, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 53, 97, 49, 50, 48, 55, 102, 45, 50, 99, 54, 102, 45, 52, 97, 97, 97, 45, 57, 97, 50, 51, 45, 102, 50, 55, 53, 53, 99, 102, 100, 49, 55, 52, 97, 0, 0, 0, 0, 0, 0, 1, 254, 0, 0, 0, 50, 241, 104, 84, 35, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 50, 50, 51, 48, 54, 102, 101, 45, 100, 55, 54, 100, 45, 52, 52, 57, 49, 45, 56, 57, 50, 50, 45, 99, 50, 48, 52, 53, 55, 52, 48, 50, 48, 98, 97, 0, 0, 0, 0, 0, 0, 1, 255, 0, 0, 0, 50, 119, 13, 46, 31, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 55, 54, 51, 52, 48, 55, 54, 45, 99, 53, 56, 52, 45, 52, 52, 52, 56, 45, 56, 99, 98, 54, 45, 57, 97, 51, 57, 53, 100, 50, 97, 54, 57, 56, 51, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 0, 50, 204, 8, 159, 249, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 55, 48, 100, 49, 57, 54, 54, 97, 45, 52, 98, 48, 53, 45, 52, 98, 57, 56, 45, 97, 48, 98, 97, 45, 100, 54, 52, 99, 97, 100, 48, 101, 98, 48, 98, 48, 0, 0, 0, 0, 0, 0, 2, 1, 0, 0, 0, 50, 42, 94, 86, 117, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 49, 49, 53, 98, 100, 100, 51, 45, 48, 49, 97, 57, 45, 52, 52, 52, 56, 45, 97, 51, 56, 100, 45, 101, 55, 57, 55, 97, 98, 97, 101, 98, 51, 49, 97, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 50, 200, 1, 142, 72, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 98, 99, 100, 54, 102, 55, 51, 98, 45, 51, 54, 50, 97, 45, 52, 53, 98, 51, 45, 56, 100, 57, 102, 45, 55, 48, 51, 100, 102, 100, 53, 54, 102, 48, 97, 98, 0, 0, 0, 0, 0, 0, 2, 3, 0, 0, 0, 50, 21, 213, 165, 101, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 49, 50, 53, 102, 48, 49, 100, 45, 97, 51, 54, 57, 45, 52, 54, 49, 97, 45, 97, 56, 49, 57, 45, 56, 49, 55, 55, 99, 102, 97, 55, 52, 102, 50, 53, 0, 0, 0, 0, 0, 0, 2, 4, 0, 0, 0, 50, 211, 194, 203, 237, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 101, 99, 98, 57, 54, 97, 97, 45, 100, 50, 50, 52, 45, 52, 53, 55, 55, 45, 57, 50, 99, 53, 45, 102, 49, 102, 50, 48, 101, 52, 50, 51, 53, 54, 48, 0, 0, 0, 0, 0, 0, 2, 5, 0, 0, 0, 50, 3, 117, 203, 4, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 57, 56, 101, 57, 97, 49, 57, 45, 48, 97, 100, 51, 45, 52, 53, 97, 51, 45, 56, 97, 98, 97, 45, 50, 50, 51, 55, 50, 50, 49, 55, 98, 101, 100, 55, 0, 0, 0, 0, 0, 0, 2, 6, 0, 0, 0, 50, 30, 184, 66, 120, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 52, 97, 50, 50, 56, 52, 52, 45, 97, 56, 53, 101, 45, 52, 102, 54, 100, 45, 57, 98, 97, 100, 45, 101, 100, 48, 54, 49, 101, 54, 53, 101, 49, 102, 52, 0, 0, 0, 0, 0, 0, 2, 7, 0, 0, 0, 50, 55, 101, 116, 100, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 101, 55, 97, 48, 54, 98, 57, 45, 55, 101, 98, 53, 45, 52, 99, 53, 49, 45, 57, 98, 100, 99, 45, 99, 57, 57, 49, 50, 55, 98, 98, 99, 97, 49, 101, 0, 0, 0, 0, 0, 0, 2, 8, 0, 0, 0, 50, 229, 9, 161, 65, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 54, 55, 48, 101, 50, 102, 50, 49, 45, 99, 55, 57, 97, 45, 52, 50, 102, 55, 45, 56, 51, 49, 97, 45, 52, 99, 97, 50, 55, 52, 53, 100, 51, 49, 57, 102, 0, 0, 0, 0, 0, 0, 2, 9, 0, 0, 0, 50, 237, 205, 140, 26, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 57, 99, 49, 56, 55, 97, 101, 55, 45, 101, 54, 99, 56, 45, 52, 55, 98, 54, 45, 56, 52, 100, 55, 45, 97, 57, 98, 48, 99, 55, 102, 53, 102, 49, 102, 97, 0, 0, 0, 0, 0, 0, 2, 10, 0, 0, 0, 50, 106, 44, 157, 223, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 52, 49, 100, 52, 55, 98, 99, 50, 45, 57, 57, 55, 53, 45, 52, 51, 50, 52, 45, 97, 54, 55, 98, 45, 54, 57, 49, 49, 50, 48, 49, 49, 48, 50, 56, 101, 0, 0, 0, 0, 0, 0, 2, 11, 0, 0, 0, 50, 63, 186, 198, 205, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 51, 51, 55, 102, 53, 56, 49, 53, 45, 98, 55, 57, 49, 45, 52, 99, 49, 51, 45, 57, 98, 97, 51, 45, 97, 55, 54, 49, 97, 102, 56, 50, 49, 97, 53, 54, 0, 0, 0, 0, 0, 0, 2, 12, 0, 0, 0, 50, 163, 150, 130, 212, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 101, 100, 49, 99, 54, 98, 49, 97, 45, 55, 56, 49, 48, 45, 52, 54, 51, 99, 45, 98, 98, 98, 48, 45, 100, 102, 100, 52, 55, 97, 100, 54, 97, 102, 57, 55, 0, 0, 0, 0, 0, 0, 2, 13, 0, 0, 0, 50, 10, 211, 165, 178, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 50, 51, 97, 51, 52, 51, 56, 100, 45, 101, 55, 53, 51, 45, 52, 100, 48, 53, 45, 57, 98, 100, 102, 45, 101, 54, 57, 50, 51, 49, 48, 99, 51, 98, 101, 101, 0, 0, 0, 0, 0, 0, 2, 14, 0, 0, 0, 50, 32, 37, 35, 71, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 56, 54, 57, 53, 51, 50, 56, 98, 45, 49, 97, 53, 100, 45, 52, 49, 52, 48, 45, 97, 100, 48, 100, 45, 102, 54, 98, 54, 101, 48, 53, 98, 97, 52, 51, 48, 0, 0, 0, 0, 0, 0, 2, 15, 0, 0, 0, 50, 205, 196, 63, 153, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 97, 57, 101, 48, 102, 102, 51, 97, 45, 52, 48, 97, 53, 45, 52, 52, 50, 52, 45, 98, 56, 97, 56, 45, 99, 49, 56, 50, 49, 48, 101, 99, 57, 52, 102, 48, 0, 0, 0, 0, 0, 0, 2, 16, 0, 0, 0, 50, 163, 164, 151, 197, 0, 0, 255, 255, 255, 255, 0, 0, 0, 36, 49, 53, 51, 52, 97, 48, 49, 48, 45, 98, 102, 100, 100, 45, 52, 56, 49, 98, 45, 97, 101, 49, 57, 45, 102, 48, 102, 100, 57, 56, 102, 100, 54, 50, 99, 55, 0, 0 }; + + public static byte[] CreateMessage(long offset, byte[] key, byte[] payload, byte magicByte = 0, byte attributes = 0) + { + var message = Message.EncodeMessage(new Message + { + Attribute = attributes, + MagicNumber = magicByte, + Key = key, + Value = payload + }); + + return new KafkaMessagePacker() + .Pack(offset) + .Pack(message) + .PayloadNoLength(); + } + } +} diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs new file mode 100644 index 00000000..3e27ed1f --- /dev/null +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -0,0 +1,475 @@ +using NUnit.Framework; +using SimpleKafka; +using SimpleKafka.Common; +using SimpleKafka.Protocol; +using SimpleKafkaTests.Helpers; +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Integration +{ + [TestFixture] + [Category("Unit")] + class SimpleTests + { + [SetUp] + public void Setup() + { + + } + + [Test] + public async Task TestProducingWorksOk() + { + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + { + var request = new ProduceRequest + { + Acks = 1, + TimeoutMS = 10000, + Payload = new List + { + new Payload + { + Topic = IntegrationConfig.IntegrationTopic, + Partition = 0, + Codec = MessageCodec.CodecNone, + Messages = new List + { + new Message(Guid.NewGuid().ToString()) + } + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Console.WriteLine(response); + } + } + + [Test] + public async Task TestFetchingWorksOk() + { + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + { + var request = new FetchRequest + { + MaxWaitTime = 1000, + MinBytes = 1000, + Fetches = new List + { + new Fetch + { + Topic = IntegrationConfig.IntegrationTopic, + PartitionId = 0, + MaxBytes = 1024, + Offset = 0 + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Console.WriteLine(response); + } + } + + [Test] + public async Task TestListingAllTopicsWorksOk() + { + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + { + var request = new MetadataRequest { }; + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + { + foreach (var broker in first.Brokers) + { + Console.WriteLine("{0},{1},{2},{3}", broker.Address, broker.BrokerId, broker.Host, broker.Port); + } + foreach (var topic in first.Topics) + { + Console.WriteLine("{0},{1}", topic.ErrorCode, topic.Name); + foreach (var partition in topic.Partitions) + { + Console.WriteLine("{0},{1},{2},{3},{4}", partition.ErrorCode, partition.Isrs.Count, partition.LeaderId, partition.PartitionId, partition.Replicas.Count); + } + } + } + } + + } + + [Test] + public async Task TestOffsetWorksOk() + { + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + { + var request = new OffsetRequest + { + Offsets = new List + { + new Offset + { + Topic = IntegrationConfig.IntegrationTopic, + MaxOffsets = 1, + PartitionId = 0, + Time = -1 + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Console.WriteLine(response); + } + } + + [Test] + public async Task TestNewTopicProductionWorksOk() + { + var topic = Guid.NewGuid().ToString(); + RunKafkaTopicsCommand("--topic", topic, "--create", "--partitions", "1", "--replication-factor", "1"); + try + { + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + { + { + var request = new MetadataRequest + { + Topics = new List + { + topic + } + }; + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + Assert.That(first.Topics, Has.Count.EqualTo(1)); + + var firstTopic = first.Topics.First(); + Assert.That(firstTopic.ErrorCode, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(firstTopic.Name, Is.EqualTo(topic)); + Assert.That(firstTopic.Partitions, Has.Count.EqualTo(1)); + + var firstPartition = firstTopic.Partitions.First(); + Assert.That(firstPartition.PartitionId, Is.EqualTo(0)); + } + + { + var request = new ProduceRequest + { + Acks = 1, + TimeoutMS = 10000, + Payload = new List + { + new Payload + { + Topic = topic, + Partition = 0, + Codec = MessageCodec.CodecNone, + Messages = new List + { + new Message("Message 1"), + new Message("Message 2"), + new Message("Message 3"), + new Message("Message 4"), + } + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + + var first = response.First(); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Offset, Is.EqualTo(0)); + } + + { + var request = new FetchRequest + { + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List + { + new Fetch + { + MaxBytes = 40, + Offset = 0, + PartitionId = 0, + Topic = topic, + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(1)); + + var firstMessage = first.Messages.First(); + Assert.That(firstMessage.Meta.Offset, Is.EqualTo(0)); + Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(firstMessage.Attribute, Is.EqualTo(0)); + Assert.That(firstMessage.Key, Is.Null); + Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(firstMessage.Value, Is.Not.Null); + + var firstString = firstMessage.Value.ToUtf8String(); + Assert.That(firstString, Is.EqualTo("Message 1")); + } + + { + var request = new OffsetRequest + { + Offsets = new List + { + new Offset + { + MaxOffsets = 2, + PartitionId = 0, + Time = -1, + Topic = topic + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Offsets, Has.Count.EqualTo(2)); + + Assert.That(first.Offsets[0], Is.EqualTo(4)); + Assert.That(first.Offsets[1], Is.EqualTo(0)); + } + + { + var request = new OffsetFetchRequest + { + ConsumerGroup = topic, + Topics = new List + { + new OffsetFetch + { + PartitionId = 0, + Topic = topic + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.MetaData, Is.Empty); + Assert.That(first.Offset, Is.EqualTo(-1)); + } + + { + var request = new OffsetCommitRequest + { + ConsumerGroup = topic, + ConsumerGroupGenerationId = 1, + ConsumerId = "0", + OffsetCommits = new List + { + new OffsetCommit + { + Metadata = "Metadata 1", + Offset = 0, + PartitionId = 0, + TimeStamp = -1, + Topic = topic, + } + } + }; + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + } + + { + var request = new OffsetFetchRequest + { + ConsumerGroup = topic, + Topics = new List + { + new OffsetFetch + { + PartitionId = 0, + Topic = topic + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.MetaData, Is.EqualTo("Metadata 1")); + Assert.That(first.Offset, Is.EqualTo(0)); + } + + { + var request = new FetchRequest + { + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List + { + new Fetch + { + MaxBytes = 1024, + Offset = 0 + 1, + PartitionId = 0, + Topic = topic, + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(3)); + + var firstMessage = first.Messages.First(); + Assert.That(firstMessage.Meta.Offset, Is.EqualTo(1)); + Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(firstMessage.Attribute, Is.EqualTo(0)); + Assert.That(firstMessage.Key, Is.Null); + Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(firstMessage.Value, Is.Not.Null); + + var firstString = firstMessage.Value.ToUtf8String(); + Assert.That(firstString, Is.EqualTo("Message 2")); + + var lastMessage = first.Messages.Last(); + Assert.That(lastMessage.Meta.Offset, Is.EqualTo(3)); + Assert.That(lastMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(lastMessage.Attribute, Is.EqualTo(0)); + Assert.That(lastMessage.Key, Is.Null); + Assert.That(lastMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(lastMessage.Value, Is.Not.Null); + + var lastString = lastMessage.Value.ToUtf8String(); + Assert.That(lastString, Is.EqualTo("Message 4")); + + + } + + { + var request = new FetchRequest + { + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List + { + new Fetch + { + MaxBytes = 1024, + Offset = 3 + 1, + PartitionId = 0, + Topic = topic, + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); + var first = response; + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(0)); + } + } + Console.WriteLine("Test completed"); + } + finally + { + RunKafkaTopicsCommand("--topic", topic, "--delete"); + } + + } + + [Test] + public async Task TestSimpleKafkaBrokerWorksOk() + { + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUri)) + { + await brokers.RefreshAsync(CancellationToken.None); + Console.WriteLine(brokers); + + } + } + + [Test] + public async Task TestSimpleProducerWorksOk() + { + var keySerializer = new NullSerializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new LoadBalancedPartitioner(1); + + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUri)) + { + var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + + await producer.SendAsync(new KafkaMessage(IntegrationConfig.IntegrationTopic, null, "Message"), CancellationToken.None).ConfigureAwait(true); + + + } + } + + private static void RunKafkaTopicsCommand(params string[] args) + { + var cmd = "--host=tcp://server.home:2375 run --rm sceneskope/kafka:0.8.2.1 bin/kafka-topics.sh --zookeeper server.home "; + var arguments = cmd + String.Join(" ", args); + + var info = new ProcessStartInfo + { + Arguments = arguments, + CreateNoWindow = true, + FileName = @"c:\users\nick\bin\docker.exe", + UseShellExecute = false, + RedirectStandardOutput = true, + }; + var process = Process.Start(info); + var stdout = process.StandardOutput.ReadToEnd(); + process.WaitForExit(); + Console.WriteLine(stdout); + } + } +} diff --git a/src/SimpleKafkaTests/Properties/AssemblyInfo.cs b/src/SimpleKafkaTests/Properties/AssemblyInfo.cs new file mode 100644 index 00000000..ea93df57 --- /dev/null +++ b/src/SimpleKafkaTests/Properties/AssemblyInfo.cs @@ -0,0 +1,36 @@ +using System.Reflection; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; + +// General Information about an assembly is controlled through the following +// set of attributes. Change these attribute values to modify the information +// associated with an assembly. +[assembly: AssemblyTitle("SimpleKafkaTests")] +[assembly: AssemblyDescription("")] +[assembly: AssemblyConfiguration("")] +[assembly: AssemblyCompany("")] +[assembly: AssemblyProduct("SimpleKafkaTests")] +[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyTrademark("")] +[assembly: AssemblyCulture("")] + +// Setting ComVisible to false makes the types in this assembly not visible +// to COM components. If you need to access a type in this assembly from +// COM, set the ComVisible attribute to true on that type. +[assembly: ComVisible(false)] + +// The following GUID is for the ID of the typelib if this project is exposed to COM +[assembly: Guid("9b7b9e32-45e6-40c0-b6fa-9f0a99681b16")] + +// Version information for an assembly consists of the following four values: +// +// Major Version +// Minor Version +// Build Number +// Revision +// +// You can specify all the values or you can default the Build and Revision Numbers +// by using the '*' as shown below: +// [assembly: AssemblyVersion("1.0.*")] +[assembly: AssemblyVersion("1.0.0.0")] +[assembly: AssemblyFileVersion("1.0.0.0")] diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj new file mode 100644 index 00000000..600ef26e --- /dev/null +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -0,0 +1,85 @@ + + + + + Debug + AnyCPU + {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16} + Library + Properties + SimpleKafkaTests + SimpleKafkaTests + v4.5 + 512 + ..\ + true + + + true + full + false + bin\Debug\ + DEBUG;TRACE + prompt + 4 + + + pdbonly + true + bin\Release\ + TRACE + prompt + 4 + + + + ..\packages\NUnit.3.0.0-beta-1\lib\net45\nunit.framework.dll + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {db719bec-843a-4a5e-a151-ec47fec2ee48} + SimpleKafka + + + + + + + This project references NuGet package(s) that are missing on this computer. Use NuGet Package Restore to download them. For more information, see http://go.microsoft.com/fwlink/?LinkID=322105. The missing file is {0}. + + + + + \ No newline at end of file diff --git a/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs new file mode 100644 index 00000000..9fc6f6a4 --- /dev/null +++ b/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs @@ -0,0 +1,160 @@ +using System; +using System.IO; +using SimpleKafka.Common; +using NUnit.Framework; + +namespace SimpleKafkaTests.Unit +{ + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// + /// Modified to work with nunit from xunit. + [TestFixture] + [Category("Unit")] + public class BigEndianBinaryReaderTests + { + // validates my assumptions about the default implementation doing the opposite of this implementation + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x01, 0x00, 0x00, 0x00 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x80 })] + [TestCase(Int32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0x7F })] + public void NativeBinaryWriterTests(Int32 expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BinaryReader(new MemoryStream(givenBytes)); + + // act + var actualValue = binaryReader.ReadInt32(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] + [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] + public void Int32Tests(Int32 expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadInt32(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((UInt32)123456789, new Byte[] { 0x07, 0x5B, 0xCD, 0x15 })] + [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + public void UInt32Tests(UInt32 expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadUInt32(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase((Single)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Single)(1), new Byte[] { 0x3F, 0x80, 0x00, 0x00 })] + [TestCase((Single)(-1), new Byte[] { 0xBF, 0x80, 0x00, 0x00 })] + [TestCase(Single.MinValue, new Byte[] { 0xFF, 0x7F, 0xFF, 0xFF })] + [TestCase(Single.MaxValue, new Byte[] { 0x7F, 0x7F, 0xFF, 0xFF })] + [TestCase(Single.PositiveInfinity, new Byte[] { 0x7F, 0x80, 0x00, 0x00 })] + [TestCase(Single.NegativeInfinity, new Byte[] { 0xFF, 0x80, 0x00, 0x00 })] + [TestCase(Single.NaN, new Byte[] { 0xFF, 0xC0, 0x00, 0x00 })] + public void SingleTests(Single expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadSingle(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase((Double)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Double)(1), new Byte[] { 0x3F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Double)(-1), new Byte[] { 0xBF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.MinValue, new Byte[] { 0xFF, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Double.MaxValue, new Byte[] { 0x7F, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Double.PositiveInfinity, new Byte[] { 0x7F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.NegativeInfinity, new Byte[] { 0xFF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.NaN, new Byte[] { 0xFF, 0xF8, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + public void DoubleTests(Double expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadDouble(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase("0000", new Byte[] { 0x04, 0x30, 0x30, 0x30, 0x30 })] + [TestCase("€€€€", new Byte[] { 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] + public void StringTests(String expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadString(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase('0', new Byte[] { 0x30 })] + [TestCase('€', new Byte[] { 0xE2, 0x82, 0xAC })] + public void CharTests(Char expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadChar(); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + [Theory] + [TestCase(new Char[] { '0', '0', '0', '0' }, new Byte[] { 0x30, 0x30, 0x30, 0x30 })] + [TestCase(new Char[] { '€', '€', '€', '€' }, new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] + public void CharArrayTests(Char[] expectedValue, Byte[] givenBytes) + { + // arrange + var binaryReader = new BigEndianBinaryReader(givenBytes); + + // act + var actualValue = binaryReader.ReadChars(givenBytes.Length); + + // assert + Assert.That(expectedValue, Is.EqualTo(actualValue)); + } + + } +} diff --git a/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs b/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs new file mode 100644 index 00000000..95c4890d --- /dev/null +++ b/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs @@ -0,0 +1,236 @@ +using System; +using System.IO; +using SimpleKafka.Common; +using NUnit.Framework; + +namespace SimpleKafkaTests.Unit +{ + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// + /// Modified to work with nunit from xunit. + [TestFixture] + [Category("Unit")] + public class BigEndianBinaryWriterTests + { + // validates my assumptions about the default implementation doing the opposite of this implementation + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x01, 0x00, 0x00, 0x00 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x80 })] + [TestCase(Int32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0x7F })] + public void NativeBinaryWriterTests(Int32 number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] + [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] + public void Int32Tests(Int32 number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase((Int16)0, new Byte[] { 0x00, 0x00 })] + [TestCase((Int16)1, new Byte[] { 0x00, 0x01 })] + [TestCase((Int16)(-1), new Byte[] { 0xFF, 0xFF })] + [TestCase(Int16.MinValue, new Byte[] { 0x80, 0x00 })] + [TestCase(Int16.MaxValue, new Byte[] { 0x7F, 0xFF })] + public void Int16Tests(Int16 number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((UInt32)123456789, new Byte[] { 0x07, 0x5B, 0xCD, 0x15 })] + [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + public void UInt32Tests(UInt32 number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase((Single)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Single)(1), new Byte[] { 0x3F, 0x80, 0x00, 0x00 })] + [TestCase((Single)(-1), new Byte[] { 0xBF, 0x80, 0x00, 0x00 })] + [TestCase(Single.MinValue, new Byte[] { 0xFF, 0x7F, 0xFF, 0xFF })] + [TestCase(Single.MaxValue, new Byte[] { 0x7F, 0x7F, 0xFF, 0xFF })] + [TestCase(Single.PositiveInfinity, new Byte[] { 0x7F, 0x80, 0x00, 0x00 })] + [TestCase(Single.NegativeInfinity, new Byte[] { 0xFF, 0x80, 0x00, 0x00 })] + [TestCase(Single.NaN, new Byte[] { 0xFF, 0xC0, 0x00, 0x00 })] + public void SingleTests(Single number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase((Double)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Double)(1), new Byte[] { 0x3F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Double)(-1), new Byte[] { 0xBF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.MinValue, new Byte[] { 0xFF, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Double.MaxValue, new Byte[] { 0x7F, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Double.PositiveInfinity, new Byte[] { 0x7F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.NegativeInfinity, new Byte[] { 0xFF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Double.NaN, new Byte[] { 0xFF, 0xF8, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + public void DoubleTests(Double number, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(number); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Test] + public void StringNotSupportedTest() + { + Assert.Throws(Is.TypeOf(), () => + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + binaryWriter.Write("test"); + }); + } + + [Theory] + [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] + [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] + [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] + [TestCase("€€€€", new Byte[] { 0x00, 0x04, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] + [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] + [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] + [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] + [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.None)] + [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] + [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] + public void StringTests(String value, Byte[] expectedBytes, StringPrefixEncoding encoding) + { + + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(value, encoding); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase('0', new Byte[] { 0x30 })] + [TestCase('€', new Byte[] { 0xE2, 0x82, 0xAC })] + public void CharTests(Char value, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(value); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase(new Char[] { '0', '0', '0', '0' }, new Byte[] { 0x30, 0x30, 0x30, 0x30 })] + [TestCase(new Char[] { '€', '€', '€', '€' }, new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] + public void CharArrayTests(Char[] value, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(value); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + [Theory] + [TestCase(new Char[] { '0', '1', '2', '3' }, 1, 2, new Byte[] { 0x31, 0x32 })] + [TestCase(new Char[] { '€', '2', '€', '€' }, 1, 2, new Byte[] { 0x32, 0xE2, 0x82, 0xAC })] + public void CharSubArrayTests(Char[] value, Int32 index, Int32 count, Byte[] expectedBytes) + { + // arrange + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + + // act + binaryWriter.Write(value, index, count); + + // assert + var actualBytes = memoryStream.ToArray(); + Assert.That(expectedBytes, Is.EqualTo(actualBytes)); + } + + } +} diff --git a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs new file mode 100644 index 00000000..eb7f091e --- /dev/null +++ b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs @@ -0,0 +1,23 @@ +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Text; +using NUnit.Framework; +using SimpleKafka.Protocol; + +namespace SimpleKafkaTests.Unit +{ + [TestFixture] + [Category("Unit")] + public class ProtocolBaseRequestTests + { + [Test] + public void EnsureHeaderShouldPackCorrectByteLengths() + { + var result = BaseRequest.EncodeHeader(new FetchRequest { ClientId = "test", CorrelationId = 123456789 }).PayloadNoLength(); + + Assert.That(result.Length, Is.EqualTo(14)); + Assert.That(result, Is.EqualTo(new byte[] { 0, 1, 0, 0, 7, 91, 205, 21, 0, 4, 116, 101, 115, 116 })); + } + } +} diff --git a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs new file mode 100644 index 00000000..0020f542 --- /dev/null +++ b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs @@ -0,0 +1,115 @@ +using System; +using System.IO; +using System.Linq; +using System.Text; +using NUnit.Framework; +using SimpleKafka.Protocol; +using SimpleKafkaTests.Helpers; +using SimpleKafka.Common; + +namespace SimpleKafkaTests.Unit +{ + [TestFixture] + [Category("Unit")] + public class ProtocolMessageTests + { + [Test] + public void DecodeMessageShouldThrowWhenCrcFails() + { + Assert.Throws(Is.TypeOf(), () => + { + var testMessage = new Message(value: "kafka test message.", key: "test"); + + var encoded = Message.EncodeMessage(testMessage); + encoded[0] += 1; + + var result = Message.DecodeMessage(0, encoded).First(); + }); + } + + [Test] + [TestCase("test key", "test message")] + [TestCase(null, "test message")] + [TestCase("test key", null)] + [TestCase(null, null)] + public void EnsureMessageEncodeAndDecodeAreCompatible(string key, string value) + { + var testMessage = new Message(key: key, value: value); + + var encoded = Message.EncodeMessage(testMessage); + var result = Message.DecodeMessage(0, encoded).First(); + + Assert.That(testMessage.Key, Is.EqualTo(result.Key)); + Assert.That(testMessage.Value, Is.EqualTo(result.Value)); + } + + [Test] + public void EncodeMessageSetEncodesMultipleMessages() + { + //expected generated from python library + var expected = new byte[] + { + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 16, 45, 70, 24, 62, 0, 0, 0, 0, 0, 1, 49, 0, 0, 0, 1, 48, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 16, 90, 65, 40, 168, 0, 0, 0, 0, 0, 1, 49, 0, 0, 0, 1, 49, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 16, 195, 72, 121, 18, 0, 0, 0, 0, 0, 1, 49, 0, 0, 0, 1, 50 + }; + + var messages = new[] + { + new Message("0", "1"), + new Message("1", "1"), + new Message("2", "1") + }; + + var result = Message.EncodeMessageSet(messages); + + Assert.That(expected, Is.EqualTo(result)); + } + + [Test] + public void DecodeMessageSetShouldHandleResponseWithMaxBufferSizeHit() + { + //This message set has a truncated message bytes at the end of it + var result = Message.DecodeMessageSet(MessageHelper.FetchResponseMaxBytesOverflow).ToList(); + + var message = Encoding.UTF8.GetString(result.First().Value); + + Assert.That(message, Is.EqualTo("test")); + Assert.That(result.Count, Is.EqualTo(529)); + } + + [Test] + public void WhenMessageIsTruncatedThenBufferUnderRunExceptionIsThrown() + { + // arrange + var offset = (Int64)0; + var message = new Byte[] { }; + var messageSize = message.Length + 1; + var memoryStream = new MemoryStream(); + var binaryWriter = new BigEndianBinaryWriter(memoryStream); + binaryWriter.Write(offset); + binaryWriter.Write(messageSize); + binaryWriter.Write(message); + var payloadBytes = memoryStream.ToArray(); + + // act/assert + Assert.Throws(() => Message.DecodeMessageSet(payloadBytes).ToList()); + } + + [Test] + public void WhenMessageIsExactlyTheSizeOfBufferThenMessageIsDecoded() + { + // arrange + var expectedPayloadBytes = new Byte[] { 1, 2, 3, 4 }; + var payload = MessageHelper.CreateMessage(0, new Byte[] { 0 }, expectedPayloadBytes); + + // act/assert + var messages = Message.DecodeMessageSet(payload).ToList(); + var actualPayload = messages.First().Value; + + // assert + var expectedPayload = new Byte[] { 1, 2, 3, 4 }; + CollectionAssert.AreEqual(expectedPayload, actualPayload); + } + } +} diff --git a/src/SimpleKafkaTests/packages.config b/src/SimpleKafkaTests/packages.config new file mode 100644 index 00000000..1c6fbad1 --- /dev/null +++ b/src/SimpleKafkaTests/packages.config @@ -0,0 +1,4 @@ + + + + \ No newline at end of file From 062095205aaf1d04e35c2eae9294c88652877449 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Thu, 9 Apr 2015 18:10:45 +0100 Subject: [PATCH 04/14] First pass at changing to a more efficient close to zero copy encode/decode --- src/SimpleKafka/BigEndianDecoder.cs | 141 +++++++++++ src/SimpleKafka/BigEndianEncoder.cs | 171 +++++++++++++ src/SimpleKafka/BigEndianReader.cs | 8 +- .../Common/BigEndianBinaryReader.cs | 218 ---------------- .../Common/BigEndianBinaryWriter.cs | 174 ------------- src/SimpleKafka/Common/Crc32Provider.cs | 23 +- src/SimpleKafka/Common/KafkaMessagePacker.cs | 106 -------- .../Common/StringPrefixEncoding.cs | 15 ++ src/SimpleKafka/Extensions.cs | 16 ++ src/SimpleKafka/Interfaces/IKafkaRequest.cs | 12 +- src/SimpleKafka/KafkaConnection.cs | 60 ++--- src/SimpleKafka/Protocol/BaseRequest.cs | 16 +- src/SimpleKafka/Protocol/Broker.cs | 12 +- .../Protocol/ConsumerMetadataRequest.cs | 50 ++-- src/SimpleKafka/Protocol/FetchRequest.cs | 99 ++++---- src/SimpleKafka/Protocol/Message.cs | 130 +++++----- src/SimpleKafka/Protocol/MetadataRequest.cs | 65 +++-- .../Protocol/OffsetCommitRequest.cs | 98 ++++---- .../Protocol/OffsetFetchRequest.cs | 80 +++--- src/SimpleKafka/Protocol/OffsetRequest.cs | 92 ++++--- src/SimpleKafka/Protocol/ProduceRequest.cs | 88 ++++--- src/SimpleKafka/Protocol/Topic.cs | 39 +-- src/SimpleKafka/SimpleKafka.csproj | 6 +- src/SimpleKafkaTests/Helpers/MessageHelper.cs | 29 ++- .../Integration/SimpleTests.cs | 31 +-- src/SimpleKafkaTests/SimpleKafkaTests.csproj | 4 +- .../Unit/BigEndianBinaryReaderTests.cs | 160 ------------ .../Unit/BigEndianBinaryWriterTests.cs | 236 ------------------ .../Unit/BigEndianDecoderTests.cs | 87 +++++++ .../Unit/BigEndianEncoderTests.cs | 96 +++++++ .../Unit/ProtocolBaseRequestTests.cs | 11 +- .../Unit/ProtocolMessageTests.cs | 59 +++-- src/kafka-net.nuspec | 2 +- src/kafka-net.sln | 16 +- 34 files changed, 1026 insertions(+), 1424 deletions(-) create mode 100644 src/SimpleKafka/BigEndianDecoder.cs create mode 100644 src/SimpleKafka/BigEndianEncoder.cs delete mode 100644 src/SimpleKafka/Common/BigEndianBinaryReader.cs delete mode 100644 src/SimpleKafka/Common/BigEndianBinaryWriter.cs delete mode 100644 src/SimpleKafka/Common/KafkaMessagePacker.cs create mode 100644 src/SimpleKafka/Common/StringPrefixEncoding.cs delete mode 100644 src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs delete mode 100644 src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs create mode 100644 src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs create mode 100644 src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs diff --git a/src/SimpleKafka/BigEndianDecoder.cs b/src/SimpleKafka/BigEndianDecoder.cs new file mode 100644 index 00000000..d161746b --- /dev/null +++ b/src/SimpleKafka/BigEndianDecoder.cs @@ -0,0 +1,141 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public struct BigEndianDecoder + { + private int offset; + public int Offset { get { return offset; } } + + public void SetOffset(int offset) + { + this.offset = offset; + } + + private int length; + public int Length { get { return length; } } + + private readonly byte[] buffer; + public byte[] Buffer { get { return buffer; } } + + + public BigEndianDecoder(byte[] buffer) : this(buffer, 0, buffer.Length) { } + public BigEndianDecoder(byte[] buffer, int offset, int length) + { + this.buffer = buffer; + this.length = length; + this.offset = offset; + } + + public void Reset(int length) + { + offset = 0; + this.length = length; + } + + public int Available { get { return length - offset; } } + + public long ReadInt64() + { + unchecked + { + return + ((long)buffer[offset++] << 56) | + ((long)buffer[offset++] << 48) | + ((long)buffer[offset++] << 40) | + ((long)buffer[offset++] << 32) | + ((long)buffer[offset++] << 24) | + ((long)buffer[offset++] << 16) | + ((long)buffer[offset++] << 8) | + ((long)buffer[offset++]); + } + } + + public int ReadInt32() + { + unchecked + { + return (buffer[offset++] << 24) | + (buffer[offset++] << 16) | + (buffer[offset++] << 8) | + (buffer[offset++]); + } + } + + public uint ReadUInt32() + { + unchecked + { + return + ((uint)buffer[offset++] << 24) | + ((uint)buffer[offset++] << 16) | + ((uint)buffer[offset++] << 8) | + ((uint)buffer[offset++]); + } + } + + public short ReadInt16() + { + unchecked + { + return (short)( + (buffer[offset++] << 8) | + (buffer[offset++]) + ); + } + } + + public string ReadString() + { + var result = Encoding.UTF8.GetString(buffer); + offset += buffer.Length; + return result; + } + + public string ReadInt16String() + { + var length = ReadInt16(); + if (length == -1) + { + return null; + } + var result = Encoding.UTF8.GetString(buffer, offset, length); + offset += length; + return result; + } + + public string ReadInt32String() + { + var length = ReadInt32(); + if (length == -1) + { + return null; + } + var result = Encoding.UTF8.GetString(buffer, offset, length); + offset += length; + return result; + } + + public byte ReadByte() + { + return buffer[offset++]; + } + + internal byte[] ReadIntPrefixedBytes() + { + var length = ReadInt32(); + if (length == -1) + { + return null; + } + var result = new byte[length]; + Array.Copy(buffer, offset, result, 0, length); + offset += length; + return result; + } + } +} diff --git a/src/SimpleKafka/BigEndianEncoder.cs b/src/SimpleKafka/BigEndianEncoder.cs new file mode 100644 index 00000000..f5a10f12 --- /dev/null +++ b/src/SimpleKafka/BigEndianEncoder.cs @@ -0,0 +1,171 @@ +using SimpleKafka.Common; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public struct BigEndianEncoder + { + private int offset; + public int Offset { get { return offset; } } + + public void SetOffset(int offset) + { + this.offset = offset; + } + + private readonly byte[] buffer; + public byte[] Buffer { get { return buffer; } } + + public BigEndianEncoder(byte[] buffer, int offset = 0) + { + this.offset = offset; + this.buffer = buffer; + } + + public void Reset() + { + offset = 0; + } + + public void Write(long value) + { + unchecked + { + buffer[offset++] = (byte)((value >> 56)); + buffer[offset++] = (byte)((value >> 48)); + buffer[offset++] = (byte)((value >> 40)); + buffer[offset++] = (byte)(value >> 32); + buffer[offset++] = (byte)((value >> 24)); + buffer[offset++] = (byte)((value >> 16)); + buffer[offset++] = (byte)((value >> 8)); + buffer[offset++] = (byte)(value); + } + } + public void Write(int value) + { + unchecked + { + buffer[offset++] = (byte)((value >> 24)); + buffer[offset++] = (byte)((value >> 16)); + buffer[offset++] = (byte)((value >> 8)); + buffer[offset++] = (byte)(value); + } + } + + public void Write(uint value) + { + unchecked + { + buffer[offset++] = (byte)((value >> 24)); + buffer[offset++] = (byte)((value >> 16)); + buffer[offset++] = (byte)((value >> 8)); + buffer[offset++] = (byte)(value); + } + } + + public void Write(short value) + { + unchecked + { + buffer[offset++] = (byte)((value >> 8)); + buffer[offset++] = (byte)(value); + } + } + + public void Write(byte value) + { + buffer[offset++] = value; + } + + public void Write(string data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + { + if (data == null) + { + switch (encoding) + { + case StringPrefixEncoding.None: break; + case StringPrefixEncoding.Int16: Write((short)-1); break; + case StringPrefixEncoding.Int32: Write(-1); break; + default: throw new InvalidOperationException("Unknown encoding: " + encoding); + } + + } else + { + int adjust; + switch (encoding) + { + case StringPrefixEncoding.None: adjust = 0; break; + case StringPrefixEncoding.Int16: adjust = 2; break; + case StringPrefixEncoding.Int32: adjust = 4; break; + default: throw new InvalidOperationException("Unknown encoding: " + encoding); + } + var bytesWritten = Encoding.UTF8.GetBytes(data, 0, data.Length, buffer, offset + adjust); + switch (encoding) + { + case StringPrefixEncoding.None: break; + case StringPrefixEncoding.Int16: Write((short)bytesWritten); break; + case StringPrefixEncoding.Int32: Write(bytesWritten); break; + } + offset += bytesWritten; + } + } + + public void Write(byte[] data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + { + if (data == null) + { + switch (encoding) + { + case StringPrefixEncoding.Int16: Write((short)-1); break; + default: Write(-1); break; + } + + } + else + { + switch (encoding) + { + case StringPrefixEncoding.Int16: Write((short)data.Length); break; + default: Write(data.Length); break; + } + Array.Copy(data, 0, buffer, offset, data.Length); + offset += data.Length; + } + } + + internal int PrepareForCrc() + { + offset += 4; + return offset; + } + + internal void CalculateCrc(int crcMarker) + { + var crc = Crc32Provider.Compute(buffer, crcMarker, offset - crcMarker); + var current = offset; + offset = crcMarker - 4; + Write(crc); + offset = current; + } + + internal int PrepareForLength() + { + offset += 4; + return offset; + } + + internal void WriteLength(int lengthMarker) + { + var current = offset; + var length = offset - lengthMarker; + offset = lengthMarker - 4; + Write(length); + offset = current; + } + + } +} diff --git a/src/SimpleKafka/BigEndianReader.cs b/src/SimpleKafka/BigEndianReader.cs index be61a424..4358c7f3 100644 --- a/src/SimpleKafka/BigEndianReader.cs +++ b/src/SimpleKafka/BigEndianReader.cs @@ -37,13 +37,7 @@ public async Task ReadInt32Async(CancellationToken token) { var buffer = commonBuffer; await ReadFullyAsync(buffer, 0, 4, token).ConfigureAwait(false); - unchecked - { - return (buffer[0] << 24) | - (buffer[1] << 16) | - (buffer[2] << 8) | - (buffer[3]); - } + return new BigEndianDecoder(buffer).ReadInt32(); } public async Task ReadBytesAsync(int numberOfBytes, CancellationToken token) diff --git a/src/SimpleKafka/Common/BigEndianBinaryReader.cs b/src/SimpleKafka/Common/BigEndianBinaryReader.cs deleted file mode 100644 index 668a3224..00000000 --- a/src/SimpleKafka/Common/BigEndianBinaryReader.cs +++ /dev/null @@ -1,218 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Diagnostics.Contracts; -using System.IO; -using System.Linq; -using System.Text; - -namespace SimpleKafka.Common -{ - /// - /// A BinaryReader that is BigEndian aware binary reader. - /// - /// - /// Booleans, bytes and byte arrays will be written directly. - /// All other values will be converted to a byte array in BigEndian byte order and written. - /// Characters and Strings will all be encoded in UTF-8 (which is byte order independent). - /// - /// - /// BigEndianBinaryWriter code provided by Zoltu - /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter - /// - /// The code was modified to provide Kafka specific logic and helper functions. - /// - public class BigEndianBinaryReader : BinaryReader - { - private const int KafkaNullSize = -1; - - public BigEndianBinaryReader(IEnumerable payload) : base(new MemoryStream(payload.ToArray()), Encoding.UTF8) - { - - } - - public long Length{get{return base.BaseStream.Length;}} - public long Position { get { return base.BaseStream.Position; } set { base.BaseStream.Position = 0; } } - public bool HasData { get { return base.BaseStream.Position < base.BaseStream.Length; } } - - public bool Available(int dataSize) - { - return (base.BaseStream.Length - base.BaseStream.Position) >= dataSize; - } - - public override Decimal ReadDecimal() - { - var bytes = GetNextBytesNativeEndian(16); - - var ints = new Int32[4]; - ints[0] = (Int32)bytes[0] << 0 - | (Int32)bytes[1] << 8 - | (Int32)bytes[2] << 16 - | (Int32)bytes[3] << 24; - ints[1] = (Int32)bytes[4] << 0 - | (Int32)bytes[5] << 8 - | (Int32)bytes[6] << 16 - | (Int32)bytes[7] << 24; - ints[2] = (Int32)bytes[8] << 0 - | (Int32)bytes[9] << 8 - | (Int32)bytes[10] << 16 - | (Int32)bytes[11] << 24; - ints[3] = (Int32)bytes[12] << 0 - | (Int32)bytes[13] << 8 - | (Int32)bytes[14] << 16 - | (Int32)bytes[15] << 24; - - return new Decimal(ints); - } - - public override Single ReadSingle() - { - return EndianAwareRead(4, BitConverter.ToSingle); - } - - public override Double ReadDouble() - { - return EndianAwareRead(8, BitConverter.ToDouble); - } - - public override Int16 ReadInt16() - { - return EndianAwareRead(2, BitConverter.ToInt16); - } - - public override Int32 ReadInt32() - { - return EndianAwareRead(4, BitConverter.ToInt32); - } - - public override Int64 ReadInt64() - { - return EndianAwareRead(8, BitConverter.ToInt64); - } - - public override UInt16 ReadUInt16() - { - return EndianAwareRead(2, BitConverter.ToUInt16); - } - - public override UInt32 ReadUInt32() - { - return EndianAwareRead(4, BitConverter.ToUInt32); - } - - public override UInt64 ReadUInt64() - { - return EndianAwareRead(8, BitConverter.ToUInt64); - } - - public string ReadInt16String() - { - var size = ReadInt16(); - if (size == KafkaNullSize) return null; - return Encoding.UTF8.GetString(RawRead(size), 0, size); - } - - public string ReadIntString() - { - var size = ReadInt32(); - if (size == KafkaNullSize) return null; - return Encoding.UTF8.GetString(RawRead(size), 0, size); - } - - public byte[] ReadInt16PrefixedBytes() - { - var size = ReadInt16(); - if (size == KafkaNullSize) { return null; } - return RawRead(size); - } - - public byte[] ReadIntPrefixedBytes() - { - var size = ReadInt32(); - if (size == KafkaNullSize) { return null; } - return RawRead(size); - } - - public byte[] ReadToEnd() - { - var size = (int)(base.BaseStream.Length - base.BaseStream.Position); - var buffer = new byte[size]; - base.BaseStream.Read(buffer, 0, size); - return buffer; - } - - public byte[] CrcHash() - { - var currentPosition = base.BaseStream.Position; - try - { - base.BaseStream.Position = 0; - return Crc32Provider.ComputeHash(ReadToEnd()); - } - finally - { - base.BaseStream.Position = currentPosition; - } - } - - public uint Crc() - { - var currentPosition = base.BaseStream.Position; - try - { - base.BaseStream.Position = 0; - return Crc32Provider.Compute(ReadToEnd()); - } - finally - { - base.BaseStream.Position = currentPosition; - } - } - - public byte[] RawRead(int size) - { - if (size <= 0) { return new byte[0]; } - - var buffer = new byte[size]; - - base.Read(buffer, 0, size); - - return buffer; - } - - private T EndianAwareRead(Int32 size, Func converter) where T : struct - { - Contract.Requires(size >= 0); - Contract.Requires(converter != null); - - var bytes = GetNextBytesNativeEndian(size); - return converter(bytes, 0); - } - - private Byte[] GetNextBytesNativeEndian(Int32 count) - { - Contract.Requires(count >= 0); - Contract.Ensures(Contract.Result() != null); - Contract.Ensures(Contract.Result().Length == count); - - var bytes = GetNextBytes(count); - if (BitConverter.IsLittleEndian) - Array.Reverse(bytes); - return bytes; - } - - private Byte[] GetNextBytes(Int32 count) - { - Contract.Requires(count >= 0); - Contract.Ensures(Contract.Result() != null); - Contract.Ensures(Contract.Result().Length == count); - - var buffer = new Byte[count]; - var bytesRead = BaseStream.Read(buffer, 0, count); - - if (bytesRead != count) - throw new EndOfStreamException(); - - return buffer; - } - } -} diff --git a/src/SimpleKafka/Common/BigEndianBinaryWriter.cs b/src/SimpleKafka/Common/BigEndianBinaryWriter.cs deleted file mode 100644 index ebab24b8..00000000 --- a/src/SimpleKafka/Common/BigEndianBinaryWriter.cs +++ /dev/null @@ -1,174 +0,0 @@ -using System; -using System.Diagnostics.Contracts; -using System.IO; -using System.Text; - -namespace SimpleKafka.Common -{ - /// - /// A BinaryWriter that stores values in BigEndian format. - /// - /// - /// Booleans, bytes and byte arrays will be written directly. - /// All other values will be converted to a byte array in BigEndian byte order and written. - /// Characters and Strings will all be encoded in UTF-8 (which is byte order independent). - /// - /// - /// BigEndianBinaryWriter code provided by Zoltu - /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter - /// The code was modified to implement Kafka specific byte handling. - /// - public class BigEndianBinaryWriter : BinaryWriter - { - public BigEndianBinaryWriter(Stream stream) - : base(stream, Encoding.UTF8) - { - Contract.Requires(stream != null); - } - - public BigEndianBinaryWriter(Stream stream, Boolean leaveOpen) - : base(stream, Encoding.UTF8, leaveOpen) - { - Contract.Requires(stream != null); - } - - public override void Write(Decimal value) - { - var ints = Decimal.GetBits(value); - Contract.Assume(ints != null); - Contract.Assume(ints.Length == 4); - - if (BitConverter.IsLittleEndian) - Array.Reverse(ints); - - for (var i = 0; i < 4; ++i) - { - var bytes = BitConverter.GetBytes(ints[i]); - if (BitConverter.IsLittleEndian) - Array.Reverse(bytes); - - Write(bytes); - } - } - - public override void Write(Single value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(Double value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(Int16 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(Int32 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(Int64 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(UInt16 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(UInt32 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(UInt64 value) - { - var bytes = BitConverter.GetBytes(value); - WriteBigEndian(bytes); - } - - public override void Write(string value) - { - throw new NotSupportedException("Kafka requires specific string length prefix encoding."); - } - - public void Write(byte[] value, StringPrefixEncoding encoding) - { - if (value == null) - { - Write(-1); - return; - } - - switch (encoding) - { - case StringPrefixEncoding.Int16: - Write((Int16)value.Length); - break; - case StringPrefixEncoding.Int32: - Write(value.Length); - break; - } - - Write(value); - } - - public void Write(string value, StringPrefixEncoding encoding) - { - if (value == null) - { - switch (encoding) - { - case StringPrefixEncoding.Int16: - Write((Int16)(-1)); - return; - default: - Write(-1); - return; - } - } - - switch (encoding) - { - case StringPrefixEncoding.Int16: - Write((Int16)value.Length); - break; - case StringPrefixEncoding.Int32: - Write(value.Length); - break; - } - - Write(Encoding.UTF8.GetBytes(value)); - } - - - private void WriteBigEndian(Byte[] bytes) - { - Contract.Requires(bytes != null); - - if (BitConverter.IsLittleEndian) - Array.Reverse(bytes); - - Write(bytes); - } - } - - public enum StringPrefixEncoding - { - Int16, - Int32, - None - }; -} diff --git a/src/SimpleKafka/Common/Crc32Provider.cs b/src/SimpleKafka/Common/Crc32Provider.cs index 5cb19339..53e31187 100644 --- a/src/SimpleKafka/Common/Crc32Provider.cs +++ b/src/SimpleKafka/Common/Crc32Provider.cs @@ -33,16 +33,6 @@ public static UInt32 Compute(byte[] buffer, int offset, int length) return ~CalculateHash(buffer, offset, length); } - public static byte[] ComputeHash(byte[] buffer) - { - return UInt32ToBigEndianBytes(Compute(buffer)); - } - - public static byte[] ComputeHash(byte[] buffer, int offset, int length) - { - return UInt32ToBigEndianBytes(Compute(buffer, offset, length)); - } - private static UInt32[] InitializeTable(UInt32 polynomial) { var createTable = new UInt32[256]; @@ -63,21 +53,12 @@ private static UInt32[] InitializeTable(UInt32 polynomial) private static UInt32 CalculateHash(byte[] buffer, int offset, int length) { var crc = DefaultSeed; - for (var i = offset; i < length; i++) + while (length-- > 0) { - crc = (crc >> 8) ^ PolynomialTable[buffer[i] ^ crc & 0xff]; + crc = (crc >> 8) ^ PolynomialTable[buffer[offset++] ^ crc & 0xff]; } return crc; } - private static byte[] UInt32ToBigEndianBytes(UInt32 uint32) - { - var result = BitConverter.GetBytes(uint32); - - if (BitConverter.IsLittleEndian) - Array.Reverse(result); - - return result; - } } } \ No newline at end of file diff --git a/src/SimpleKafka/Common/KafkaMessagePacker.cs b/src/SimpleKafka/Common/KafkaMessagePacker.cs deleted file mode 100644 index 67ef5819..00000000 --- a/src/SimpleKafka/Common/KafkaMessagePacker.cs +++ /dev/null @@ -1,106 +0,0 @@ -using System; -using System.Collections.Generic; -using System.IO; - -namespace SimpleKafka.Common -{ - public class KafkaMessagePacker : IDisposable - { - private const int IntegerByteSize = 4; - private readonly BigEndianBinaryWriter _stream; - - public KafkaMessagePacker() - { - _stream = new BigEndianBinaryWriter(new MemoryStream()); - Pack(IntegerByteSize); //pre-allocate space for buffer length - } - - public KafkaMessagePacker Pack(byte value) - { - _stream.Write(value); - return this; - } - - public KafkaMessagePacker Pack(Int32 ints) - { - _stream.Write(ints); - return this; - } - - public KafkaMessagePacker Pack(Int16 ints) - { - _stream.Write(ints); - return this; - } - - public KafkaMessagePacker Pack(Int64 ints) - { - _stream.Write(ints); - return this; - } - - public KafkaMessagePacker Pack(byte[] buffer, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) - { - _stream.Write(buffer, encoding); - return this; - } - - public KafkaMessagePacker Pack(string data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) - { - _stream.Write(data, encoding); - return this; - } - - public KafkaMessagePacker Pack(IEnumerable data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) - { - foreach (var item in data) - { - _stream.Write(item, encoding); - } - - return this; - } - - public byte[] Payload() - { - var buffer = new byte[_stream.BaseStream.Length]; - _stream.BaseStream.Position = 0; - Pack((Int32)(_stream.BaseStream.Length - IntegerByteSize)); - _stream.BaseStream.Position = 0; - _stream.BaseStream.Read(buffer, 0, (int)_stream.BaseStream.Length); - return buffer; - } - - public byte[] PayloadNoLength() - { - var payloadLength = _stream.BaseStream.Length - IntegerByteSize; - var buffer = new byte[payloadLength]; - _stream.BaseStream.Position = IntegerByteSize; - _stream.BaseStream.Read(buffer, 0, (int)payloadLength); - return buffer; - } - - public byte[] CrcPayload() - { - var buffer = new byte[_stream.BaseStream.Length]; - - //copy the payload over - _stream.BaseStream.Position = 0; - _stream.BaseStream.Read(buffer, 0, (int)_stream.BaseStream.Length); - - //calculate the crc - var crc = Crc32Provider.ComputeHash(buffer, IntegerByteSize, buffer.Length); - buffer[0] = crc[0]; - buffer[1] = crc[1]; - buffer[2] = crc[2]; - buffer[3] = crc[3]; - - return buffer; - } - - public void Dispose() - { - using (_stream) { } - } - } -} \ No newline at end of file diff --git a/src/SimpleKafka/Common/StringPrefixEncoding.cs b/src/SimpleKafka/Common/StringPrefixEncoding.cs new file mode 100644 index 00000000..c881e920 --- /dev/null +++ b/src/SimpleKafka/Common/StringPrefixEncoding.cs @@ -0,0 +1,15 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka.Common +{ + public enum StringPrefixEncoding + { + Int16, + Int32, + None + } +} diff --git a/src/SimpleKafka/Extensions.cs b/src/SimpleKafka/Extensions.cs index a72cf88e..83c9d588 100644 --- a/src/SimpleKafka/Extensions.cs +++ b/src/SimpleKafka/Extensions.cs @@ -1,13 +1,29 @@ using System; using System.Collections.Generic; +using System.IO; using System.Linq; using System.Text; +using System.Threading; using System.Threading.Tasks; namespace SimpleKafka { internal static class Extensions { + public static async Task ReadFullyAsync(this Stream stream, byte[] buffer, int offset, int numberOfBytes, CancellationToken token) + { + while (numberOfBytes > 0) + { + var bytesRead = await stream.ReadAsync(buffer, offset, numberOfBytes, token).ConfigureAwait(false); + if (bytesRead <= 0) + { + throw new EndOfStreamException(); + } + numberOfBytes -= bytesRead; + offset += bytesRead; + } + } + public static TValue FindOrCreate(this Dictionary map, TKey key) where TValue : new() { diff --git a/src/SimpleKafka/Interfaces/IKafkaRequest.cs b/src/SimpleKafka/Interfaces/IKafkaRequest.cs index fd50e1ad..632ab2db 100644 --- a/src/SimpleKafka/Interfaces/IKafkaRequest.cs +++ b/src/SimpleKafka/Interfaces/IKafkaRequest.cs @@ -33,13 +33,13 @@ public interface IKafkaRequest /// /// Encode this request into the Kafka wire protocol. /// - /// Byte[] representing the binary wire protocol of this request. - byte[] Encode(); + /// Encoder to use + void Encode(ref BigEndianEncoder encoder); /// - /// Decode a response payload from Kafka into an enumerable of T responses. + /// Decode a response payload from Kafka into T. /// - /// Payload data returned by Kafka servers. - /// - IEnumerable Decode(byte[] payload); + /// Decoder to use + /// Response + T Decode(ref BigEndianDecoder decoder); } } \ No newline at end of file diff --git a/src/SimpleKafka/KafkaConnection.cs b/src/SimpleKafka/KafkaConnection.cs index 937977c3..2bd3fb8e 100644 --- a/src/SimpleKafka/KafkaConnection.cs +++ b/src/SimpleKafka/KafkaConnection.cs @@ -21,64 +21,58 @@ internal static async Task CreateAsync(IPEndPoint serverEndpoin private readonly IPEndPoint serverEndpoint; public IPEndPoint ServerEndpoint { get { return serverEndpoint; } } private readonly TcpClient client; - private readonly BigEndianReader reader; - private readonly BigEndianWriter writer; + private readonly byte[] buffer; + private readonly NetworkStream stream; + private BigEndianDecoder decoder; + private BigEndianEncoder encoder; - private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client) + private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferSize = 65536) { this.serverEndpoint = serverEndpoint; this.client = client; - var stream = client.GetStream(); - this.reader = new BigEndianReader(stream); - this.writer = new BigEndianWriter(stream); + this.stream = client.GetStream(); + this.buffer = new byte[bufferSize]; + decoder = new BigEndianDecoder(buffer); + encoder = new BigEndianEncoder(buffer); } - private async Task ReceiveResponseAsync(CancellationToken token) + private async Task ReceiveResponseAsync(CancellationToken token) { - var length = await reader.ReadInt32Async(token).ConfigureAwait(false); - var buffer = await reader.ReadBytesAsync(length, token).ConfigureAwait(false); - return buffer; + await stream.ReadFullyAsync(buffer, 0, 4, token).ConfigureAwait(false); + decoder.Reset(4); + var length = decoder.ReadInt32(); + await stream.ReadFullyAsync(buffer, 0, length, token).ConfigureAwait(false); + decoder.Reset(length); + return length; } - private async Task CommunicateWithClientAsync(byte[] buffer, int offset, int length, bool expectResponse, CancellationToken token) + public async Task SendRequestAsync(IKafkaRequest request, CancellationToken token) { await clientLock.WaitAsync(token).ConfigureAwait(false); try { + encoder.Reset(); + var marker = encoder.PrepareForLength(); + request.Encode(ref encoder); + encoder.WriteLength(marker); - await writer.WriteAsync(buffer, offset, length, token).ConfigureAwait(false); - if (expectResponse) + await stream.WriteAsync(buffer, 0, encoder.Offset, token).ConfigureAwait(false); + if (request.ExpectResponse) { - var resultBuffer = await ReceiveResponseAsync(token).ConfigureAwait(false); - return resultBuffer; + var length = await ReceiveResponseAsync(token).ConfigureAwait(false); + var result = request.Decode(ref decoder); + return result; } else { - return null; + return default(T); } } finally { clientLock.Release(); } - - } - - - public async Task SendRequestAsync(IKafkaRequest request, CancellationToken token) - { - var encoded = request.Encode(); - var resultBuffer = await CommunicateWithClientAsync(encoded, 0, encoded.Length, request.ExpectResponse, token).ConfigureAwait(false); - if (request.ExpectResponse) - { - var result = request.Decode(resultBuffer).Single(); - return result; - } - else - { - return default(T); - } } #region IDisposable Support diff --git a/src/SimpleKafka/Protocol/BaseRequest.cs b/src/SimpleKafka/Protocol/BaseRequest.cs index 35e0081a..94df9233 100644 --- a/src/SimpleKafka/Protocol/BaseRequest.cs +++ b/src/SimpleKafka/Protocol/BaseRequest.cs @@ -44,18 +44,12 @@ protected BaseRequest(short apiVersion = 0) /// public virtual bool ExpectResponse { get { return true; } } - /// - /// Encode the common head for kafka request. - /// - /// KafkaMessagePacker with header populated - /// Format: (hhihs) - public static KafkaMessagePacker EncodeHeader(IKafkaRequest request) + internal static void EncodeHeader(IKafkaRequest request, ref BigEndianEncoder encoder) { - return new KafkaMessagePacker() - .Pack(((Int16)request.ApiKey)) - .Pack(request.ApiVersion) - .Pack(request.CorrelationId) - .Pack(request.ClientId, StringPrefixEncoding.Int16); + encoder.Write((Int16)request.ApiKey); + encoder.Write(request.ApiVersion); + encoder.Write(request.CorrelationId); + encoder.Write(request.ClientId, StringPrefixEncoding.Int16); } } } \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Broker.cs b/src/SimpleKafka/Protocol/Broker.cs index 85fef582..44059534 100644 --- a/src/SimpleKafka/Protocol/Broker.cs +++ b/src/SimpleKafka/Protocol/Broker.cs @@ -10,14 +10,14 @@ public class Broker public int Port { get; set; } public Uri Address { get { return new Uri(string.Format("http://{0}:{1}", Host, Port));} } - public static Broker FromStream(BigEndianBinaryReader stream) + public static Broker Decode(ref BigEndianDecoder decoder) { return new Broker - { - BrokerId = stream.ReadInt32(), - Host = stream.ReadInt16String(), - Port = stream.ReadInt32() - }; + { + BrokerId = decoder.ReadInt32(), + Host = decoder.ReadInt16String(), + Port = decoder.ReadInt32() + }; } } } diff --git a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs index 5de3c34b..002c1a57 100644 --- a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs +++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs @@ -14,41 +14,27 @@ public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest Decode(byte[] payload) + public ConsumerMetadataResponse Decode(ref BigEndianDecoder decoder) { - return DecodeConsumerMetadataResponse(payload); + return DecodeConsumerMetadataResponse(ref decoder); } - private byte[] EncodeConsumerMetadataRequest(ConsumerMetadataRequest request) + private static void EncodeConsumerMetadataRequest(ConsumerMetadataRequest request, ref BigEndianEncoder encoder) { - using (var message = EncodeHeader(request).Pack(request.ConsumerGroup, StringPrefixEncoding.Int16)) - { - return message.Payload(); - } + EncodeHeader(request, ref encoder); + encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); } - private IEnumerable DecodeConsumerMetadataResponse(byte[] data) + private static ConsumerMetadataResponse DecodeConsumerMetadataResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) - { - var correlationId = stream.ReadInt32(); - - var response = new ConsumerMetadataResponse - { - Error = stream.ReadInt16(), - CoordinatorId = stream.ReadInt32(), - CoordinatorHost = stream.ReadInt16String(), - CoordinatorPort = stream.ReadInt32() - }; - - yield return response; - } + var correlationId = decoder.ReadInt32(); + return new ConsumerMetadataResponse(ref decoder); } } @@ -57,10 +43,18 @@ public class ConsumerMetadataResponse /// /// Error code of exception that occured during the request. Zero if no error. /// - public Int16 Error; + public readonly ErrorResponseCode Error; + + public readonly int CoordinatorId; + public readonly string CoordinatorHost; + public readonly int CoordinatorPort; - public int CoordinatorId; - public string CoordinatorHost; - public int CoordinatorPort; + internal ConsumerMetadataResponse(ref BigEndianDecoder decoder) + { + Error = (ErrorResponseCode)decoder.ReadInt16(); + CoordinatorId = decoder.ReadInt32(); + CoordinatorHost = decoder.ReadInt16String(); + CoordinatorPort = decoder.ReadInt32(); + } } } diff --git a/src/SimpleKafka/Protocol/FetchRequest.cs b/src/SimpleKafka/Protocol/FetchRequest.cs index 9bc350de..e994798a 100644 --- a/src/SimpleKafka/Protocol/FetchRequest.cs +++ b/src/SimpleKafka/Protocol/FetchRequest.cs @@ -5,7 +5,7 @@ namespace SimpleKafka.Protocol { - public class FetchRequest : BaseRequest, IKafkaRequest + public class FetchRequest : BaseRequest, IKafkaRequest> { internal const int DefaultMinBlockingByteBufferSize = 4096; internal const int DefaultBufferSize = DefaultMinBlockingByteBufferSize * 8; @@ -30,80 +30,79 @@ public class FetchRequest : BaseRequest, IKafkaRequest public List Fetches { get; set; } - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeFetchRequest(this); + EncodeFetchRequest(this, ref encoder); } - public IEnumerable Decode(byte[] payload) + public List Decode(ref BigEndianDecoder decoder) { - return DecodeFetchResponse(payload); + return DecodeFetchResponses(ref decoder); } - private byte[] EncodeFetchRequest(FetchRequest request) - { + private static void EncodeFetchRequest(FetchRequest request, ref BigEndianEncoder encoder) + { if (request.Fetches == null) request.Fetches = new List(); + EncodeHeader(request, ref encoder); + + var topicGroups = request.Fetches.GroupBy(x => x.Topic).ToList(); + encoder.Write(ReplicaId); + encoder.Write(request.MaxWaitTime); + encoder.Write(request.MinBytes); + encoder.Write(topicGroups.Count); - using (var message = EncodeHeader(request)) + foreach (var topicGroup in topicGroups) { - var topicGroups = request.Fetches.GroupBy(x => x.Topic).ToList(); - message.Pack(ReplicaId) - .Pack(request.MaxWaitTime) - .Pack(request.MinBytes) - .Pack(topicGroups.Count); + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); + encoder.Write(partitions.Count); - foreach (var topicGroup in topicGroups) + foreach (var partition in partitions) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) - .Pack(partitions.Count); - - foreach (var partition in partitions) + foreach (var fetch in partition) { - foreach (var fetch in partition) - { - message.Pack(partition.Key) - .Pack(fetch.Offset) - .Pack(fetch.MaxBytes); - } + encoder.Write(partition.Key); + encoder.Write(fetch.Offset); + encoder.Write(fetch.MaxBytes); } } - - return message.Payload(); } } - private IEnumerable DecodeFetchResponse(byte[] data) + private List DecodeFetchResponses(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var correlationId = decoder.ReadInt32(); + + var result = new List(); + + var topicCount = decoder.ReadInt32(); + for (int i = 0; i < topicCount; i++) { - var correlationId = stream.ReadInt32(); + var topic = decoder.ReadInt16String(); - var topicCount = stream.ReadInt32(); - for (int i = 0; i < topicCount; i++) + var partitionCount = decoder.ReadInt32(); + for (int j = 0; j < partitionCount; j++) { - var topic = stream.ReadInt16String(); - - var partitionCount = stream.ReadInt32(); - for (int j = 0; j < partitionCount; j++) + var partitionId = decoder.ReadInt32(); + var response = new FetchResponse { - var partitionId = stream.ReadInt32(); - var response = new FetchResponse - { - Topic = topic, - PartitionId = partitionId, - Error = stream.ReadInt16(), - HighWaterMark = stream.ReadInt64() - }; - //note: dont use initializer here as it breaks stream position. - response.Messages = Message.DecodeMessageSet(stream.ReadIntPrefixedBytes()) - .Select(x => { x.Meta.PartitionId = partitionId; return x; }) - .ToList(); - yield return response; - } + Topic = topic, + PartitionId = partitionId, + Error = decoder.ReadInt16(), + HighWaterMark = decoder.ReadInt64(), + }; + var messageSetSize = decoder.ReadInt32(); + var current = decoder.Offset; + response.Messages = Message.DecodeMessageSet(partitionId, ref decoder, messageSetSize); + result.Add(response); + + // In case any truncated messages + decoder.SetOffset(current + messageSetSize); } } + return result; } + } public class Fetch diff --git a/src/SimpleKafka/Protocol/Message.cs b/src/SimpleKafka/Protocol/Message.cs index 66d4528d..86f5b685 100644 --- a/src/SimpleKafka/Protocol/Message.cs +++ b/src/SimpleKafka/Protocol/Message.cs @@ -72,52 +72,53 @@ public Message(string value, string key = null) /// /// The collection of messages to encode together. /// Encoded byte[] representing the collection of messages. - public static byte[] EncodeMessageSet(IEnumerable messages) + public static void EncodeMessageSet(ref BigEndianEncoder encoder, IEnumerable messages) { - using (var stream = new KafkaMessagePacker()) + foreach (var message in messages) { - foreach (var message in messages) - { - stream.Pack(InitialMessageOffset) - .Pack(EncodeMessage(message)); - } - - return stream.PayloadNoLength(); + encoder.Write(InitialMessageOffset); + var marker = encoder.PrepareForLength(); + EncodeMessage(message, ref encoder); + encoder.WriteLength(marker); } } /// /// Decode a byte[] that represents a collection of messages. /// - /// The byte[] encode as a message set from kafka. - /// Enumerable representing stream of messages decoded from byte[] - public static IEnumerable DecodeMessageSet(byte[] messageSet) + /// The decoder positioned at the start of the buffer + /// The messages + public static List DecodeMessageSet(int partitionId, ref BigEndianDecoder decoder, int messageSetSize) { - using (var stream = new BigEndianBinaryReader(messageSet)) - { - while (stream.HasData) - { - //this checks that we have at least the minimum amount of data to retrieve a header - if (stream.Available(MessageHeaderSize) == false) - yield break; - - var offset = stream.ReadInt64(); - var messageSize = stream.ReadInt32(); + var numberOfBytes = messageSetSize; - //if messagessize is greater than the total payload, our max buffer is insufficient. - if ((stream.Length - MessageHeaderSize) < messageSize) - throw new BufferUnderRunException(MessageHeaderSize, messageSize); + var messages = new List(); + while (numberOfBytes > 0) + { - //if the stream does not have enough left in the payload, we got only a partial message - if (stream.Available(messageSize) == false) - yield break; + if (numberOfBytes < MessageHeaderSize) + { + break; + } + var offset = decoder.ReadInt64(); + var messageSize = decoder.ReadInt32(); + if (messageSetSize - MessageHeaderSize < messageSize) + { + // This message is too big to fit in the buffer so we will never get it + throw new BufferUnderRunException(numberOfBytes, messageSize); + } - foreach (var message in DecodeMessage(offset, stream.RawRead(messageSize))) - { - yield return message; - } + numberOfBytes -= MessageHeaderSize; + if (numberOfBytes < messageSize) + { + break; } + + var message = DecodeMessage(offset, partitionId, ref decoder, messageSize); + messages.Add(message); + numberOfBytes -= messageSize; } + return messages; } /// @@ -129,16 +130,14 @@ public static IEnumerable DecodeMessageSet(byte[] messageSet) /// Format: /// Crc (Int32), MagicByte (Byte), Attribute (Byte), Key (Byte[]), Value (Byte[]) /// - public static byte[] EncodeMessage(Message message) + public static void EncodeMessage(Message message, ref BigEndianEncoder encoder) { - using(var stream = new KafkaMessagePacker()) - { - return stream.Pack(message.MagicNumber) - .Pack(message.Attribute) - .Pack(message.Key) - .Pack(message.Value) - .CrcPayload(); - } + var marker = encoder.PrepareForCrc(); + encoder.Write(message.MagicNumber); + encoder.Write(message.Attribute); + encoder.Write(message.Key); + encoder.Write(message.Value); + encoder.CalculateCrc(marker); } /// @@ -146,35 +145,36 @@ public static byte[] EncodeMessage(Message message) /// /// The offset represting the log entry from kafka of this message. /// The byte[] encode as a message from kafka. - /// Enumerable representing stream of messages decoded from byte[]. + /// The message /// The return type is an Enumerable as the message could be a compressed message set. - public static IEnumerable DecodeMessage(long offset, byte[] payload) + public static Message DecodeMessage(long offset, int partitionId, ref BigEndianDecoder decoder, int messageSize) { - var crc = payload.Take(4).ToArray(); - using (var stream = new BigEndianBinaryReader(payload.Skip(4))) + var crc = decoder.ReadUInt32(); + var calculatedCrc = Crc32Provider.Compute(decoder.Buffer, decoder.Offset, messageSize - 4); + if (calculatedCrc != crc) { - if (crc.SequenceEqual(stream.CrcHash()) == false) - throw new FailCrcCheckException("Payload did not match CRC validation."); + throw new FailCrcCheckException("Payload did not match CRC validation."); + } - var message = new Message - { - Meta = new MessageMetadata { Offset = offset }, - MagicNumber = stream.ReadByte(), - Attribute = stream.ReadByte(), - Key = stream.ReadIntPrefixedBytes() - }; - - var codec = (MessageCodec)(ProtocolConstants.AttributeCodeMask & message.Attribute); - switch (codec) - { - case MessageCodec.CodecNone: - message.Value = stream.ReadIntPrefixedBytes(); - yield return message; - break; - default: - throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", codec)); - } + var message = new Message + { + Meta = new MessageMetadata { Offset = offset, PartitionId = partitionId }, + MagicNumber = decoder.ReadByte(), + Attribute = decoder.ReadByte(), + Key = decoder.ReadIntPrefixedBytes(), + + }; + var codec = (MessageCodec)(ProtocolConstants.AttributeCodeMask & message.Attribute); + switch (codec) + { + case MessageCodec.CodecNone: + message.Value = decoder.ReadIntPrefixedBytes(); + break; + + default: + throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", codec)); } + return message; } } diff --git a/src/SimpleKafka/Protocol/MetadataRequest.cs b/src/SimpleKafka/Protocol/MetadataRequest.cs index 22e1b64e..0a4fef2b 100644 --- a/src/SimpleKafka/Protocol/MetadataRequest.cs +++ b/src/SimpleKafka/Protocol/MetadataRequest.cs @@ -15,60 +15,51 @@ public class MetadataRequest : BaseRequest, IKafkaRequest /// public List Topics { get; set; } - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeMetadataRequest(this); + EncodeMetadataRequest(this, ref encoder); } - public IEnumerable Decode(byte[] payload) + public MetadataResponse Decode(ref BigEndianDecoder decoder) { - return new[] { DecodeMetadataResponse(payload) }; + return DecodeMetadataResponse(ref decoder); } - /// - /// Encode a request for metadata about topic and broker information. - /// - /// The MetaDataRequest to encode. - /// Encoded byte[] representing the request. - /// Format: (MessageSize), Header, ix(hs) - private byte[] EncodeMetadataRequest(MetadataRequest request) + private static void EncodeMetadataRequest(MetadataRequest request, ref BigEndianEncoder encoder) { if (request.Topics == null) request.Topics = new List(); - - using (var message = EncodeHeader(request) - .Pack(request.Topics.Count) - .Pack(request.Topics, StringPrefixEncoding.Int16)) + EncodeHeader(request, ref encoder); + encoder.Write(request.Topics.Count); + foreach (var topic in request.Topics) { - return message.Payload(); + encoder.Write(topic, StringPrefixEncoding.Int16); } } - /// - /// Decode the metadata response from kafka server. - /// - /// - /// - private MetadataResponse DecodeMetadataResponse(byte[] data) + private static MetadataResponse DecodeMetadataResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var response = new MetadataResponse { - var response = new MetadataResponse(); - response.CorrelationId = stream.ReadInt32(); + CorrelationId = decoder.ReadInt32(), + }; - var brokerCount = stream.ReadInt32(); - for (var i = 0; i < brokerCount; i++) - { - response.Brokers.Add(Broker.FromStream(stream)); - } - - var topicCount = stream.ReadInt32(); - for (var i = 0; i < topicCount; i++) - { - response.Topics.Add(Topic.FromStream(stream)); - } + var brokerCount = decoder.ReadInt32(); + var brokers = new List(brokerCount); + for (var i = 0; i < brokerCount; i++) + { + brokers.Add(Broker.Decode(ref decoder)); + } + response.Brokers = brokers; - return response; + var topicCount = decoder.ReadInt32(); + var topics = new List(topicCount); + for (var i = 0; i < topicCount; i++) + { + topics.Add(Topic.Decode(ref decoder)); } + response.Topics = topics; + + return response; } } diff --git a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs index 16469206..8c0d3ba8 100644 --- a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs @@ -11,7 +11,7 @@ namespace SimpleKafka.Protocol /// arbitrary ConsumerGroup name provided by the call. /// This now supports version 0 and 1 of the protocol /// - public class OffsetCommitRequest : BaseRequest, IKafkaRequest + public class OffsetCommitRequest : BaseRequest, IKafkaRequest> { public OffsetCommitRequest(Int16 version = 1) : base(version) { @@ -22,85 +22,77 @@ public OffsetCommitRequest(Int16 version = 1) : base(version) public string ConsumerId { get; set; } public List OffsetCommits { get; set; } - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeOffsetCommitRequest(this); + EncodeOffsetCommitRequest(this, ref encoder); } - public IEnumerable Decode(byte[] payload) + public List Decode(ref BigEndianDecoder decoder) { - return DecodeOffsetCommitResponse(payload); + return DecodeOffsetCommitResponse(ref decoder); } - private byte[] EncodeOffsetCommitRequest(OffsetCommitRequest request) + private static void EncodeOffsetCommitRequest(OffsetCommitRequest request, ref BigEndianEncoder encoder) { if (request.OffsetCommits == null) request.OffsetCommits = new List(); + EncodeHeader(request, ref encoder); + encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); + if (request.ApiVersion == 1) + { + encoder.Write(request.ConsumerGroupGenerationId); + encoder.Write(request.ConsumerId, StringPrefixEncoding.Int16); + } - using (var message = EncodeHeader(request).Pack(request.ConsumerGroup, StringPrefixEncoding.Int16)) + var topicGroups = request.OffsetCommits.GroupBy(x => x.Topic).ToList(); + encoder.Write(topicGroups.Count); + + foreach (var topicGroup in topicGroups) { - if (ApiVersion == 1) - { - message - .Pack(ConsumerGroupGenerationId) - .Pack(ConsumerId, StringPrefixEncoding.Int16); - } - var topicGroups = request.OffsetCommits.GroupBy(x => x.Topic).ToList(); - message.Pack(topicGroups.Count); + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); + encoder.Write(partitions.Count); - foreach (var topicGroup in topicGroups) + foreach (var partition in partitions) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) - .Pack(partitions.Count); - - foreach (var partition in partitions) + foreach (var commit in partition) { - foreach (var commit in partition) - { - message - .Pack(partition.Key) - .Pack(commit.Offset); - - if (ApiVersion == 1) - { - message.Pack(commit.TimeStamp); - } + encoder.Write(partition.Key); + encoder.Write(commit.Offset); - message - .Pack(commit.Metadata, StringPrefixEncoding.Int16); + if (request.ApiVersion == 1) + { + encoder.Write(commit.TimeStamp); } + + encoder.Write(commit.Metadata, StringPrefixEncoding.Int16); } } - - return message.Payload(); } } - private IEnumerable DecodeOffsetCommitResponse(byte[] data) + private static List DecodeOffsetCommitResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var correlationId = decoder.ReadInt32(); + + var responses = new List(); + var topicCount = decoder.ReadInt32(); + for (int i = 0; i < topicCount; i++) { - var correlationId = stream.ReadInt32(); + var topic = decoder.ReadInt16String(); - var topicCount = stream.ReadInt32(); - for (int i = 0; i < topicCount; i++) + var partitionCount = decoder.ReadInt32(); + for (int j = 0; j < partitionCount; j++) { - var topic = stream.ReadInt16String(); - - var partitionCount = stream.ReadInt32(); - for (int j = 0; j < partitionCount; j++) + var response = new OffsetCommitResponse() { - var response = new OffsetCommitResponse() - { - Topic = topic, - PartitionId = stream.ReadInt32(), - Error = stream.ReadInt16() - }; - - yield return response; - } + Topic = topic, + PartitionId = decoder.ReadInt32(), + Error = decoder.ReadInt16() + }; + responses.Add(response); } } + return responses; } } diff --git a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs index 88b8b354..daa7f152 100644 --- a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs @@ -12,7 +12,7 @@ namespace SimpleKafka.Protocol /// under any abritrary name. /// This now supports version 1 of the protocol /// - public class OffsetFetchRequest : BaseRequest, IKafkaRequest + public class OffsetFetchRequest : BaseRequest, IKafkaRequest> { public OffsetFetchRequest(short version = 1) : base(version) { @@ -22,73 +22,69 @@ public OffsetFetchRequest(short version = 1) : base(version) public string ConsumerGroup { get; set; } public List Topics { get; set; } - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeOffsetFetchRequest(this); + EncodeOffsetFetchRequest(this, ref encoder); } - protected byte[] EncodeOffsetFetchRequest(OffsetFetchRequest request) + private static void EncodeOffsetFetchRequest(OffsetFetchRequest request, ref BigEndianEncoder encoder) { if (request.Topics == null) request.Topics = new List(); + EncodeHeader(request, ref encoder); - using (var message = EncodeHeader(request)) - { - var topicGroups = request.Topics.GroupBy(x => x.Topic).ToList(); + encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); - message.Pack(ConsumerGroup, StringPrefixEncoding.Int16) - .Pack(topicGroups.Count); + var topicGroups = request.Topics.GroupBy(x => x.Topic).ToList(); + encoder.Write(topicGroups.Count); - foreach (var topicGroup in topicGroups) - { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) - .Pack(partitions.Count); + foreach (var topicGroup in topicGroups) + { + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); + encoder.Write(partitions.Count); - foreach (var partition in partitions) + foreach (var partition in partitions) + { + foreach (var offset in partition) { - foreach (var offset in partition) - { - message.Pack(offset.PartitionId); - } + encoder.Write(offset.PartitionId); } } - - return message.Payload(); } + } - public IEnumerable Decode(byte[] payload) + public List Decode(ref BigEndianDecoder decoder) { - return DecodeOffsetFetchResponse(payload); + return DecodeOffsetFetchResponse(ref decoder); } - protected IEnumerable DecodeOffsetFetchResponse(byte[] data) + private static List DecodeOffsetFetchResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var correlationId = decoder.ReadInt32(); + + var responses = new List(); + var topicCount = decoder.ReadInt32(); + for (int i = 0; i < topicCount; i++) { - var correlationId = stream.ReadInt32(); + var topic = decoder.ReadInt16String(); - var topicCount = stream.ReadInt32(); - for (int i = 0; i < topicCount; i++) + var partitionCount = decoder.ReadInt32(); + for (int j = 0; j < partitionCount; j++) { - var topic = stream.ReadInt16String(); - - var partitionCount = stream.ReadInt32(); - for (int j = 0; j < partitionCount; j++) + var response = new OffsetFetchResponse() { - var response = new OffsetFetchResponse() - { - Topic = topic, - PartitionId = stream.ReadInt32(), - Offset = stream.ReadInt64(), - MetaData = stream.ReadInt16String(), - Error = stream.ReadInt16() - }; - yield return response; - } + Topic = topic, + PartitionId = decoder.ReadInt32(), + Offset = decoder.ReadInt64(), + MetaData = decoder.ReadInt16String(), + Error = decoder.ReadInt16() + }; + responses.Add(response); } } + return responses; } } diff --git a/src/SimpleKafka/Protocol/OffsetRequest.cs b/src/SimpleKafka/Protocol/OffsetRequest.cs index 926ddc8b..c214b0eb 100644 --- a/src/SimpleKafka/Protocol/OffsetRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetRequest.cs @@ -8,84 +8,82 @@ namespace SimpleKafka.Protocol /// /// A funky Protocol for requesting the starting offset of each segment for the requested partition /// - public class OffsetRequest : BaseRequest, IKafkaRequest + public class OffsetRequest : BaseRequest, IKafkaRequest> { public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Offset; } } public List Offsets { get; set; } - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeOffsetRequest(this); + EncodeOffsetRequest(this, ref encoder); } - public IEnumerable Decode(byte[] payload) + public List Decode(ref BigEndianDecoder decoder) { - return DecodeOffsetResponse(payload); + return DecodeOffsetResponse(ref decoder); } - private byte[] EncodeOffsetRequest(OffsetRequest request) + private static void EncodeOffsetRequest(OffsetRequest request, ref BigEndianEncoder encoder) { if (request.Offsets == null) request.Offsets = new List(); - using (var message = EncodeHeader(request)) + EncodeHeader(request, ref encoder); + encoder.Write(ReplicaId); + + var topicGroups = request.Offsets.GroupBy(x => x.Topic).ToList(); + encoder.Write(topicGroups.Count); + + foreach (var topicGroup in topicGroups) { - var topicGroups = request.Offsets.GroupBy(x => x.Topic).ToList(); - message.Pack(ReplicaId) - .Pack(topicGroups.Count); + var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); + encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); + encoder.Write(partitions.Count); - foreach (var topicGroup in topicGroups) + foreach (var partition in partitions) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - message.Pack(topicGroup.Key, StringPrefixEncoding.Int16) - .Pack(partitions.Count); - - foreach (var partition in partitions) + foreach (var offset in partition) { - foreach (var offset in partition) - { - message.Pack(partition.Key) - .Pack(offset.Time) - .Pack(offset.MaxOffsets); - } + encoder.Write(partition.Key); + encoder.Write(offset.Time); + encoder.Write(offset.MaxOffsets); } } - - return message.Payload(); } + } - private IEnumerable DecodeOffsetResponse(byte[] data) + private static List DecodeOffsetResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var correlationId = decoder.ReadInt32(); + + var responses = new List(); + var topicCount = decoder.ReadInt32(); + for (int i = 0; i < topicCount; i++) { - var correlationId = stream.ReadInt32(); + var topic = decoder.ReadInt16String(); - var topicCount = stream.ReadInt32(); - for (int i = 0; i < topicCount; i++) + var partitionCount = decoder.ReadInt32(); + for (int j = 0; j < partitionCount; j++) { - var topic = stream.ReadInt16String(); - - var partitionCount = stream.ReadInt32(); - for (int j = 0; j < partitionCount; j++) + var response = new OffsetResponse() { - var response = new OffsetResponse() - { - Topic = topic, - PartitionId = stream.ReadInt32(), - Error = stream.ReadInt16(), - Offsets = new List() - }; - var offsetCount = stream.ReadInt32(); - for (int k = 0; k < offsetCount; k++) - { - response.Offsets.Add(stream.ReadInt64()); - } - - yield return response; + Topic = topic, + PartitionId = decoder.ReadInt32(), + Error = decoder.ReadInt16(), + Offsets = new List() + }; + var offsetCount = decoder.ReadInt32(); + for (int k = 0; k < offsetCount; k++) + { + response.Offsets.Add(decoder.ReadInt64()); } + + responses.Add(response); } } + return responses; } + } public class Offset diff --git a/src/SimpleKafka/Protocol/ProduceRequest.cs b/src/SimpleKafka/Protocol/ProduceRequest.cs index d7ed4e96..361bd01a 100644 --- a/src/SimpleKafka/Protocol/ProduceRequest.cs +++ b/src/SimpleKafka/Protocol/ProduceRequest.cs @@ -5,7 +5,7 @@ namespace SimpleKafka.Protocol { - public class ProduceRequest : BaseRequest, IKafkaRequest> + public class ProduceRequest : BaseRequest, IKafkaRequest> { /// /// Provide a hint to the broker call not to expect a response for requests without Acks. @@ -29,18 +29,18 @@ public class ProduceRequest : BaseRequest, IKafkaRequest Payload = new List(); - public byte[] Encode() + public void Encode(ref BigEndianEncoder encoder) { - return EncodeProduceRequest(this); + EncodeProduceRequest(this, ref encoder); } - public IEnumerable> Decode(byte[] payload) + public List Decode(ref BigEndianDecoder decoder) { - yield return DecodeProduceResponse(payload); + return DecodeProduceResponse(ref decoder); } #region Protocol... - private byte[] EncodeProduceRequest(ProduceRequest request) + private static void EncodeProduceRequest(ProduceRequest request, ref BigEndianEncoder encoder) { if (request.Payload == null) request.Payload = new List(); @@ -53,57 +53,55 @@ private byte[] EncodeProduceRequest(ProduceRequest request) } into tpc select tpc).ToList(); - using (var message = EncodeHeader(request) - .Pack(request.Acks) - .Pack(request.TimeoutMS) - .Pack(groupedPayloads.Count)) + EncodeHeader(request, ref encoder); + encoder.Write(request.Acks); + encoder.Write(request.TimeoutMS); + encoder.Write(groupedPayloads.Count); + foreach (var groupedPayload in groupedPayloads) { - foreach (var groupedPayload in groupedPayloads) - { - var payloads = groupedPayload.ToList(); - message.Pack(groupedPayload.Key.Topic, StringPrefixEncoding.Int16) - .Pack(payloads.Count) - .Pack(groupedPayload.Key.Partition); + var payloads = groupedPayload.ToList(); + encoder.Write(groupedPayload.Key.Topic, StringPrefixEncoding.Int16); + encoder.Write(payloads.Count); + encoder.Write(groupedPayload.Key.Partition); - switch (groupedPayload.Key.Codec) - { - case MessageCodec.CodecNone: - message.Pack(Message.EncodeMessageSet(payloads.SelectMany(x => x.Messages))); - break; - default: - throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", groupedPayload.Key.Codec)); - } + var marker = encoder.PrepareForLength(); + switch (groupedPayload.Key.Codec) + { + case MessageCodec.CodecNone: + Message.EncodeMessageSet(ref encoder, (payloads.SelectMany(x => x.Messages))); + break; + default: + throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", groupedPayload.Key.Codec)); } - - return message.Payload(); + encoder.WriteLength(marker); } } - private IEnumerable DecodeProduceResponse(byte[] data) + + private List DecodeProduceResponse(ref BigEndianDecoder decoder) { - using (var stream = new BigEndianBinaryReader(data)) + var correlationId = decoder.ReadInt32(); + + var responses = new List(); + var topicCount = decoder.ReadInt32(); + for (int i = 0; i < topicCount; i++) { - var correlationId = stream.ReadInt32(); + var topic = decoder.ReadInt16String(); - var topicCount = stream.ReadInt32(); - for (int i = 0; i < topicCount; i++) + var partitionCount = decoder.ReadInt32(); + for (int j = 0; j < partitionCount; j++) { - var topic = stream.ReadInt16String(); - - var partitionCount = stream.ReadInt32(); - for (int j = 0; j < partitionCount; j++) + var response = new ProduceResponse() { - var response = new ProduceResponse() - { - Topic = topic, - PartitionId = stream.ReadInt32(), - Error = stream.ReadInt16(), - Offset = stream.ReadInt64() - }; - - yield return response; - } + Topic = topic, + PartitionId = decoder.ReadInt32(), + Error = decoder.ReadInt16(), + Offset = decoder.ReadInt64() + }; + + responses.Add(response); } } + return responses; } #endregion } diff --git a/src/SimpleKafka/Protocol/Topic.cs b/src/SimpleKafka/Protocol/Topic.cs index 8abc1167..0fde0dab 100644 --- a/src/SimpleKafka/Protocol/Topic.cs +++ b/src/SimpleKafka/Protocol/Topic.cs @@ -10,20 +10,21 @@ public class Topic public string Name { get; set; } public List Partitions { get; set; } - public static Topic FromStream(BigEndianBinaryReader stream) + internal static Topic Decode(ref BigEndianDecoder decoder) { var topic = new Topic - { - ErrorCode = stream.ReadInt16(), - Name = stream.ReadInt16String(), - Partitions = new List() - }; + { + ErrorCode = decoder.ReadInt16(), + Name = decoder.ReadInt16String(), + }; - var numPartitions = stream.ReadInt32(); + var numPartitions = decoder.ReadInt32(); + var partitions = new List(numPartitions); for (int i = 0; i < numPartitions; i++) { - topic.Partitions.Add(Partition.FromStream(stream)); + partitions.Add(Partition.Decode(ref decoder)); } + topic.Partitions = partitions; return topic; } @@ -52,27 +53,29 @@ public class Partition /// public List Isrs { get; set; } - public static Partition FromStream(BigEndianBinaryReader stream) + public static Partition Decode(ref BigEndianDecoder decoder) { var partition = new Partition { - ErrorCode = stream.ReadInt16(), - PartitionId = stream.ReadInt32(), - LeaderId = stream.ReadInt32(), - Replicas = new List(), - Isrs = new List() + ErrorCode = decoder.ReadInt16(), + PartitionId = decoder.ReadInt32(), + LeaderId = decoder.ReadInt32(), }; - var numReplicas = stream.ReadInt32(); + var numReplicas = decoder.ReadInt32(); + var replicas = new List(numReplicas); for (int i = 0; i < numReplicas; i++) { - partition.Replicas.Add(stream.ReadInt32()); + replicas.Add(decoder.ReadInt32()); } + partition.Replicas = replicas; - var numIsr = stream.ReadInt32(); + var numIsr = decoder.ReadInt32(); + var isrs = new List(numIsr); for (int i = 0; i < numIsr; i++) { - partition.Isrs.Add(stream.ReadInt32()); + isrs.Add(decoder.ReadInt32()); } + partition.Isrs = isrs; return partition; } diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index 960f922e..5ea82259 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -47,13 +47,13 @@ + + - - - + diff --git a/src/SimpleKafkaTests/Helpers/MessageHelper.cs b/src/SimpleKafkaTests/Helpers/MessageHelper.cs index 24b1f89e..8292cac4 100644 --- a/src/SimpleKafkaTests/Helpers/MessageHelper.cs +++ b/src/SimpleKafkaTests/Helpers/MessageHelper.cs @@ -1,6 +1,7 @@ using System; using SimpleKafka.Common; using SimpleKafka.Protocol; +using SimpleKafka; namespace SimpleKafkaTests.Helpers { @@ -10,18 +11,22 @@ public static class MessageHelper public static byte[] CreateMessage(long offset, byte[] key, byte[] payload, byte magicByte = 0, byte attributes = 0) { - var message = Message.EncodeMessage(new Message - { - Attribute = attributes, - MagicNumber = magicByte, - Key = key, - Value = payload - }); - - return new KafkaMessagePacker() - .Pack(offset) - .Pack(message) - .PayloadNoLength(); + var message = new Message + { + Attribute = attributes, + MagicNumber = magicByte, + Key = key, + Value = payload + }; + + var buffer = new byte[1024]; + var encoder = new BigEndianEncoder(buffer); + Message.EncodeMessageSet(ref encoder, new[] { message }); + + var result = new byte[encoder.Offset]; + Array.Copy(encoder.Buffer, result, encoder.Offset); + + return result; } } } diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 3e27ed1f..5ed4cc34 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -14,7 +14,7 @@ namespace SimpleKafkaTests.Integration { [TestFixture] - [Category("Unit")] + [Category("Integration")] class SimpleTests { [SetUp] @@ -211,8 +211,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); @@ -248,8 +248,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); @@ -275,8 +275,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); @@ -304,8 +304,9 @@ public async Task TestNewTopicProductionWorksOk() } }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); @@ -326,8 +327,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); @@ -354,8 +355,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); @@ -406,8 +407,8 @@ public async Task TestNewTopicProductionWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index 600ef26e..42f74bb6 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -49,8 +49,8 @@ - - + + diff --git a/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs deleted file mode 100644 index 9fc6f6a4..00000000 --- a/src/SimpleKafkaTests/Unit/BigEndianBinaryReaderTests.cs +++ /dev/null @@ -1,160 +0,0 @@ -using System; -using System.IO; -using SimpleKafka.Common; -using NUnit.Framework; - -namespace SimpleKafkaTests.Unit -{ - /// - /// BigEndianBinaryWriter code provided by Zoltu - /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter - /// - /// Modified to work with nunit from xunit. - [TestFixture] - [Category("Unit")] - public class BigEndianBinaryReaderTests - { - // validates my assumptions about the default implementation doing the opposite of this implementation - [Theory] - [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Int32)1, new Byte[] { 0x01, 0x00, 0x00, 0x00 })] - [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Int32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x80 })] - [TestCase(Int32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0x7F })] - public void NativeBinaryWriterTests(Int32 expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BinaryReader(new MemoryStream(givenBytes)); - - // act - var actualValue = binaryReader.ReadInt32(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] - [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] - [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] - public void Int32Tests(Int32 expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadInt32(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] - [TestCase((UInt32)123456789, new Byte[] { 0x07, 0x5B, 0xCD, 0x15 })] - [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - public void UInt32Tests(UInt32 expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadUInt32(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase((Single)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Single)(1), new Byte[] { 0x3F, 0x80, 0x00, 0x00 })] - [TestCase((Single)(-1), new Byte[] { 0xBF, 0x80, 0x00, 0x00 })] - [TestCase(Single.MinValue, new Byte[] { 0xFF, 0x7F, 0xFF, 0xFF })] - [TestCase(Single.MaxValue, new Byte[] { 0x7F, 0x7F, 0xFF, 0xFF })] - [TestCase(Single.PositiveInfinity, new Byte[] { 0x7F, 0x80, 0x00, 0x00 })] - [TestCase(Single.NegativeInfinity, new Byte[] { 0xFF, 0x80, 0x00, 0x00 })] - [TestCase(Single.NaN, new Byte[] { 0xFF, 0xC0, 0x00, 0x00 })] - public void SingleTests(Single expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadSingle(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase((Double)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Double)(1), new Byte[] { 0x3F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Double)(-1), new Byte[] { 0xBF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.MinValue, new Byte[] { 0xFF, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Double.MaxValue, new Byte[] { 0x7F, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Double.PositiveInfinity, new Byte[] { 0x7F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.NegativeInfinity, new Byte[] { 0xFF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.NaN, new Byte[] { 0xFF, 0xF8, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - public void DoubleTests(Double expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadDouble(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase("0000", new Byte[] { 0x04, 0x30, 0x30, 0x30, 0x30 })] - [TestCase("€€€€", new Byte[] { 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] - public void StringTests(String expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadString(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase('0', new Byte[] { 0x30 })] - [TestCase('€', new Byte[] { 0xE2, 0x82, 0xAC })] - public void CharTests(Char expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadChar(); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - [Theory] - [TestCase(new Char[] { '0', '0', '0', '0' }, new Byte[] { 0x30, 0x30, 0x30, 0x30 })] - [TestCase(new Char[] { '€', '€', '€', '€' }, new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] - public void CharArrayTests(Char[] expectedValue, Byte[] givenBytes) - { - // arrange - var binaryReader = new BigEndianBinaryReader(givenBytes); - - // act - var actualValue = binaryReader.ReadChars(givenBytes.Length); - - // assert - Assert.That(expectedValue, Is.EqualTo(actualValue)); - } - - } -} diff --git a/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs b/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs deleted file mode 100644 index 95c4890d..00000000 --- a/src/SimpleKafkaTests/Unit/BigEndianBinaryWriterTests.cs +++ /dev/null @@ -1,236 +0,0 @@ -using System; -using System.IO; -using SimpleKafka.Common; -using NUnit.Framework; - -namespace SimpleKafkaTests.Unit -{ - /// - /// BigEndianBinaryWriter code provided by Zoltu - /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter - /// - /// Modified to work with nunit from xunit. - [TestFixture] - [Category("Unit")] - public class BigEndianBinaryWriterTests - { - // validates my assumptions about the default implementation doing the opposite of this implementation - [Theory] - [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Int32)1, new Byte[] { 0x01, 0x00, 0x00, 0x00 })] - [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Int32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x80 })] - [TestCase(Int32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0x7F })] - public void NativeBinaryWriterTests(Int32 number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] - [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] - [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] - public void Int32Tests(Int32 number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase((Int16)0, new Byte[] { 0x00, 0x00 })] - [TestCase((Int16)1, new Byte[] { 0x00, 0x01 })] - [TestCase((Int16)(-1), new Byte[] { 0xFF, 0xFF })] - [TestCase(Int16.MinValue, new Byte[] { 0x80, 0x00 })] - [TestCase(Int16.MaxValue, new Byte[] { 0x7F, 0xFF })] - public void Int16Tests(Int16 number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] - [TestCase((UInt32)123456789, new Byte[] { 0x07, 0x5B, 0xCD, 0x15 })] - [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] - public void UInt32Tests(UInt32 number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase((Single)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Single)(1), new Byte[] { 0x3F, 0x80, 0x00, 0x00 })] - [TestCase((Single)(-1), new Byte[] { 0xBF, 0x80, 0x00, 0x00 })] - [TestCase(Single.MinValue, new Byte[] { 0xFF, 0x7F, 0xFF, 0xFF })] - [TestCase(Single.MaxValue, new Byte[] { 0x7F, 0x7F, 0xFF, 0xFF })] - [TestCase(Single.PositiveInfinity, new Byte[] { 0x7F, 0x80, 0x00, 0x00 })] - [TestCase(Single.NegativeInfinity, new Byte[] { 0xFF, 0x80, 0x00, 0x00 })] - [TestCase(Single.NaN, new Byte[] { 0xFF, 0xC0, 0x00, 0x00 })] - public void SingleTests(Single number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase((Double)(0), new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Double)(1), new Byte[] { 0x3F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase((Double)(-1), new Byte[] { 0xBF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.MinValue, new Byte[] { 0xFF, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Double.MaxValue, new Byte[] { 0x7F, 0xEF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] - [TestCase(Double.PositiveInfinity, new Byte[] { 0x7F, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.NegativeInfinity, new Byte[] { 0xFF, 0xF0, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - [TestCase(Double.NaN, new Byte[] { 0xFF, 0xF8, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] - public void DoubleTests(Double number, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(number); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Test] - public void StringNotSupportedTest() - { - Assert.Throws(Is.TypeOf(), () => - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - binaryWriter.Write("test"); - }); - } - - [Theory] - [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] - [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] - [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] - [TestCase("€€€€", new Byte[] { 0x00, 0x04, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] - [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] - [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] - [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] - [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.None)] - [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] - [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] - public void StringTests(String value, Byte[] expectedBytes, StringPrefixEncoding encoding) - { - - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(value, encoding); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase('0', new Byte[] { 0x30 })] - [TestCase('€', new Byte[] { 0xE2, 0x82, 0xAC })] - public void CharTests(Char value, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(value); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase(new Char[] { '0', '0', '0', '0' }, new Byte[] { 0x30, 0x30, 0x30, 0x30 })] - [TestCase(new Char[] { '€', '€', '€', '€' }, new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] - public void CharArrayTests(Char[] value, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(value); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - [Theory] - [TestCase(new Char[] { '0', '1', '2', '3' }, 1, 2, new Byte[] { 0x31, 0x32 })] - [TestCase(new Char[] { '€', '2', '€', '€' }, 1, 2, new Byte[] { 0x32, 0xE2, 0x82, 0xAC })] - public void CharSubArrayTests(Char[] value, Int32 index, Int32 count, Byte[] expectedBytes) - { - // arrange - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - - // act - binaryWriter.Write(value, index, count); - - // assert - var actualBytes = memoryStream.ToArray(); - Assert.That(expectedBytes, Is.EqualTo(actualBytes)); - } - - } -} diff --git a/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs new file mode 100644 index 00000000..89af9f61 --- /dev/null +++ b/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs @@ -0,0 +1,87 @@ +using System; +using System.IO; +using SimpleKafka; +using SimpleKafka.Common; +using NUnit.Framework; + +namespace SimpleKafkaTests.Unit +{ + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// + /// Modified to work with nunit from xunit. + [TestFixture] + [Category("Unit")] + public class BigEndianDecoderTests + { + [Theory] + [TestCase((Int64)0, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int64)1, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int64)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int64.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Int64.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + public void Int64Tests(Int64 expectedValue, Byte[] givenBytes) + { + var decoder = new BigEndianDecoder(givenBytes); + var actualValue = decoder.ReadInt64(); + Assert.That(actualValue, Is.EqualTo(expectedValue)); + } + + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] + [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] + public void Int32Tests(Int32 expectedValue, Byte[] givenBytes) + { + var decoder = new BigEndianDecoder(givenBytes); + var actualValue = decoder.ReadInt32(); + Assert.That(actualValue, Is.EqualTo(expectedValue)); + } + + + [Theory] + [TestCase((Int16)0, new Byte[] { 0x00, 0x00 })] + [TestCase((Int16)1, new Byte[] { 0x00, 0x01 })] + [TestCase((Int16)(-1), new Byte[] { 0xFF, 0xFF })] + [TestCase(Int16.MinValue, new Byte[] { 0x80, 0x00 })] + [TestCase(Int16.MaxValue, new Byte[] { 0x7F, 0xFF })] + public void Int16Tests(Int16 expectedValue, Byte[] givenBytes) + { + var decoder = new BigEndianDecoder(givenBytes); + var actualValue = decoder.ReadInt16(); + Assert.That(actualValue, Is.EqualTo(expectedValue)); + } + + + [Theory] + [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] + [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] + [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] + [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] + [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] + [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] + [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] + [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] + [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] + public void StringTests(String expectedValue, Byte[] givenBytes, StringPrefixEncoding encoding) + { + var decoder = new BigEndianDecoder(givenBytes); + string actualValue = null; + switch (encoding) + { + case StringPrefixEncoding.None: actualValue = decoder.ReadString(); break; + case StringPrefixEncoding.Int16: actualValue = decoder.ReadInt16String(); break; + case StringPrefixEncoding.Int32: actualValue = decoder.ReadInt32String(); break; + default: Assert.Fail("Unknown encoding", encoding); break; + } + Assert.That(decoder.Offset, Is.EqualTo(givenBytes.Length)); + Assert.That(actualValue, Is.EqualTo(expectedValue)); + } + + } +} diff --git a/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs new file mode 100644 index 00000000..201d747a --- /dev/null +++ b/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs @@ -0,0 +1,96 @@ +using System; +using System.IO; +using SimpleKafka.Common; +using NUnit.Framework; +using SimpleKafka; + +namespace SimpleKafkaTests.Unit +{ + /// + /// BigEndianBinaryWriter code provided by Zoltu + /// https://github.com/Zoltu/Zoltu.EndianAwareBinaryReaderWriter + /// + /// Modified to work with nunit from xunit. + [TestFixture] + [Category("Unit")] + public class BigEndianEncoderTests + { + [Theory] + [TestCase((Int64)0, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int64)1, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int64)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int64.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] + [TestCase(Int64.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] + public void Int64Tests(Int64 number, Byte[] expectedBytes) + { + var buffer = new byte[8]; + var encoder = new BigEndianEncoder(buffer); + encoder.Write(number); + Assert.That(buffer, Is.EqualTo(expectedBytes)); + } + [Theory] + [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + public void UInt32Tests(UInt32 number, Byte[] expectedBytes) + { + var buffer = new byte[4]; + var encoder = new BigEndianEncoder(buffer); + encoder.Write(number); + Assert.That(buffer, Is.EqualTo(expectedBytes)); + } + + [Theory] + [TestCase((Int32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] + [TestCase((Int32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] + [TestCase((Int32)(-1), new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] + [TestCase(Int32.MinValue, new Byte[] { 0x80, 0x00, 0x00, 0x00 })] + [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] + public void Int32Tests(Int32 number, Byte[] expectedBytes) + { + var buffer = new byte[4]; + var encoder = new BigEndianEncoder(buffer); + encoder.Write(number); + Assert.That(buffer, Is.EqualTo(expectedBytes)); + } + + [Theory] + [TestCase((Int16)0, new Byte[] { 0x00, 0x00 })] + [TestCase((Int16)1, new Byte[] { 0x00, 0x01 })] + [TestCase((Int16)(-1), new Byte[] { 0xFF, 0xFF })] + [TestCase(Int16.MinValue, new Byte[] { 0x80, 0x00 })] + [TestCase(Int16.MaxValue, new Byte[] { 0x7F, 0xFF })] + public void Int16Tests(Int16 number, Byte[] expectedBytes) + { + var buffer = new byte[2]; + var encoder = new BigEndianEncoder(buffer); + encoder.Write(number); + Assert.That(buffer, Is.EqualTo(expectedBytes)); + } + + + [Theory] + [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] + [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] + [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] + [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] + [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] + [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] + [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] + [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] + [TestCase(null, new Byte[] { }, StringPrefixEncoding.None)] + [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] + [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] + public void StringTests(String value, Byte[] expectedBytes, StringPrefixEncoding encoding) + { + var buffer = new byte[expectedBytes.Length]; + var encoder = new BigEndianEncoder(buffer); + encoder.Write(value, encoding); + Assert.That(encoder.Offset, Is.EqualTo(expectedBytes.Length)); + Assert.That(buffer, Is.EqualTo(expectedBytes)); + } + + } +} diff --git a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs index eb7f091e..e246723d 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs @@ -4,6 +4,7 @@ using System.Text; using NUnit.Framework; using SimpleKafka.Protocol; +using SimpleKafka; namespace SimpleKafkaTests.Unit { @@ -14,10 +15,12 @@ public class ProtocolBaseRequestTests [Test] public void EnsureHeaderShouldPackCorrectByteLengths() { - var result = BaseRequest.EncodeHeader(new FetchRequest { ClientId = "test", CorrelationId = 123456789 }).PayloadNoLength(); - - Assert.That(result.Length, Is.EqualTo(14)); - Assert.That(result, Is.EqualTo(new byte[] { 0, 1, 0, 0, 7, 91, 205, 21, 0, 4, 116, 101, 115, 116 })); + var encoder = new BigEndianEncoder(new byte[14]); + var request = new FetchRequest { ClientId = "test", CorrelationId = 123456789 }; + + BaseRequest.EncodeHeader(request, ref encoder); + Assert.That(encoder.Offset, Is.EqualTo(14)); + Assert.That(encoder.Buffer, Is.EqualTo(new byte[] { 0, 1, 0, 0, 7, 91, 205, 21, 0, 4, 116, 101, 115, 116 })); } } } diff --git a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs index 0020f542..157d27dc 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs @@ -6,6 +6,7 @@ using SimpleKafka.Protocol; using SimpleKafkaTests.Helpers; using SimpleKafka.Common; +using SimpleKafka; namespace SimpleKafkaTests.Unit { @@ -19,11 +20,14 @@ public void DecodeMessageShouldThrowWhenCrcFails() Assert.Throws(Is.TypeOf(), () => { var testMessage = new Message(value: "kafka test message.", key: "test"); + var buffer = new byte[1024]; + var encoder = new BigEndianEncoder(buffer); - var encoded = Message.EncodeMessage(testMessage); - encoded[0] += 1; + Message.EncodeMessage(testMessage, ref encoder); + buffer[0] += 1; - var result = Message.DecodeMessage(0, encoded).First(); + var decoder = new BigEndianDecoder(buffer, 0, encoder.Offset); + var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); }); } @@ -36,8 +40,12 @@ public void EnsureMessageEncodeAndDecodeAreCompatible(string key, string value) { var testMessage = new Message(key: key, value: value); - var encoded = Message.EncodeMessage(testMessage); - var result = Message.DecodeMessage(0, encoded).First(); + var buffer = new byte[1024]; + var encoder = new BigEndianEncoder(buffer); + Message.EncodeMessage(testMessage, ref encoder); + + var decoder = new BigEndianDecoder(buffer); + var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); Assert.That(testMessage.Key, Is.EqualTo(result.Key)); Assert.That(testMessage.Value, Is.EqualTo(result.Value)); @@ -61,16 +69,19 @@ public void EncodeMessageSetEncodesMultipleMessages() new Message("2", "1") }; - var result = Message.EncodeMessageSet(messages); + var buffer = new byte[expected.Length]; + var encoder = new BigEndianEncoder(buffer); + Message.EncodeMessageSet(ref encoder, messages); - Assert.That(expected, Is.EqualTo(result)); + Assert.That(buffer, Is.EqualTo(expected)); } [Test] public void DecodeMessageSetShouldHandleResponseWithMaxBufferSizeHit() { //This message set has a truncated message bytes at the end of it - var result = Message.DecodeMessageSet(MessageHelper.FetchResponseMaxBytesOverflow).ToList(); + var decoder = new BigEndianDecoder(MessageHelper.FetchResponseMaxBytesOverflow); + var result = Message.DecodeMessageSet(0, ref decoder, decoder.Length); var message = Encoding.UTF8.GetString(result.First().Value); @@ -81,19 +92,22 @@ public void DecodeMessageSetShouldHandleResponseWithMaxBufferSizeHit() [Test] public void WhenMessageIsTruncatedThenBufferUnderRunExceptionIsThrown() { - // arrange - var offset = (Int64)0; - var message = new Byte[] { }; - var messageSize = message.Length + 1; - var memoryStream = new MemoryStream(); - var binaryWriter = new BigEndianBinaryWriter(memoryStream); - binaryWriter.Write(offset); - binaryWriter.Write(messageSize); - binaryWriter.Write(message); - var payloadBytes = memoryStream.ToArray(); - - // act/assert - Assert.Throws(() => Message.DecodeMessageSet(payloadBytes).ToList()); + Assert.Throws(() => + { + // arrange + var offset = (Int64)0; + var message = new Byte[] { }; + var messageSize = 5; + var payloadBytes = new byte[16]; + var encoder = new BigEndianEncoder(payloadBytes); + encoder.Write(offset); + encoder.Write(messageSize); + encoder.Write(message); + + var decoder = new BigEndianDecoder(payloadBytes); + + Message.DecodeMessageSet(0, ref decoder, payloadBytes.Length); + }); } [Test] @@ -104,7 +118,8 @@ public void WhenMessageIsExactlyTheSizeOfBufferThenMessageIsDecoded() var payload = MessageHelper.CreateMessage(0, new Byte[] { 0 }, expectedPayloadBytes); // act/assert - var messages = Message.DecodeMessageSet(payload).ToList(); + var decoder = new BigEndianDecoder(payload, 0, payload.Length); + var messages = Message.DecodeMessageSet(0, ref decoder, payload.Length); var actualPayload = messages.First().Value; // assert diff --git a/src/kafka-net.nuspec b/src/kafka-net.nuspec index 292fe471..0e3f686b 100644 --- a/src/kafka-net.nuspec +++ b/src/kafka-net.nuspec @@ -2,7 +2,7 @@ kafka-net - @build.number@-alpha + 0.9.0.2 James Roland James Roland https://github.com/Jroland/kafka-net/blob/master/LICENSE diff --git a/src/kafka-net.sln b/src/kafka-net.sln index a1c88085..25641f9a 100644 --- a/src/kafka-net.sln +++ b/src/kafka-net.sln @@ -1,7 +1,7 @@  Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 2013 -VisualStudioVersion = 12.0.31101.0 +# Visual Studio 14 +VisualStudioVersion = 14.0.22609.0 MinimumVisualStudioVersion = 10.0.40219.1 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "kafka-net", "kafka-net\kafka-net.csproj", "{1343EB68-55CB-4452-8386-24A9989DE1C0}" EndProject @@ -24,6 +24,10 @@ Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution ..\RELEASE_NOTES.md = ..\RELEASE_NOTES.md EndProjectSection EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "SimpleKafkaTests", "SimpleKafkaTests\SimpleKafkaTests.csproj", "{9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}" +EndProject +Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "SimpleKafka", "SimpleKafka\SimpleKafka.csproj", "{DB719BEC-843A-4A5E-A151-EC47FEC2EE48}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Any CPU = Debug|Any CPU @@ -42,6 +46,14 @@ Global {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Debug|Any CPU.Build.0 = Debug|Any CPU {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Release|Any CPU.ActiveCfg = Release|Any CPU {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Release|Any CPU.Build.0 = Release|Any CPU + {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Debug|Any CPU.Build.0 = Debug|Any CPU + {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Release|Any CPU.ActiveCfg = Release|Any CPU + {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Release|Any CPU.Build.0 = Release|Any CPU + {DB719BEC-843A-4A5E-A151-EC47FEC2EE48}.Debug|Any CPU.ActiveCfg = Debug|Any CPU + {DB719BEC-843A-4A5E-A151-EC47FEC2EE48}.Debug|Any CPU.Build.0 = Debug|Any CPU + {DB719BEC-843A-4A5E-A151-EC47FEC2EE48}.Release|Any CPU.ActiveCfg = Release|Any CPU + {DB719BEC-843A-4A5E-A151-EC47FEC2EE48}.Release|Any CPU.Build.0 = Release|Any CPU EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE From b74124d32cb8ceda1c8e1df970e18b248656724f Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Sun, 12 Apr 2015 05:29:57 +0100 Subject: [PATCH 05/14] Fleshing out integration tests to produce more complete functionality --- src/SimpleKafka/Interfaces/IKafkaRequest.cs | 4 +- src/SimpleKafka/KafkaBrokers.cs | 125 ++++-- src/SimpleKafka/KafkaConnection.cs | 10 +- src/SimpleKafka/KafkaConsumer.cs | 134 ++++++ .../{BigEndianDecoder.cs => KafkaDecoder.cs} | 6 +- .../{BigEndianEncoder.cs => KafkaEncoder.cs} | 4 +- src/SimpleKafka/KafkaProducer.cs | 41 +- src/SimpleKafka/Protocol/BaseRequest.cs | 2 +- src/SimpleKafka/Protocol/Broker.cs | 2 +- .../Protocol/ConsumerMetadataRequest.cs | 10 +- src/SimpleKafka/Protocol/FetchRequest.cs | 8 +- src/SimpleKafka/Protocol/Message.cs | 8 +- src/SimpleKafka/Protocol/MetadataRequest.cs | 8 +- .../Protocol/OffsetCommitRequest.cs | 8 +- .../Protocol/OffsetFetchRequest.cs | 8 +- src/SimpleKafka/Protocol/OffsetRequest.cs | 8 +- src/SimpleKafka/Protocol/ProduceRequest.cs | 8 +- src/SimpleKafka/Protocol/Topic.cs | 4 +- src/SimpleKafka/ReceivedKafkaMessage.cs | 23 + src/SimpleKafka/SimpleKafka.csproj | 9 +- src/SimpleKafka/TopicSelector.cs | 33 ++ .../Helpers/IntegrationConfig.cs | 10 +- src/SimpleKafkaTests/Helpers/MessageHelper.cs | 2 +- .../Integration/ConsumerTests.cs | 53 +++ .../Integration/IntegrationHelpers.cs | 72 ++++ .../Integration/SimpleTests.cs | 392 +++++++++--------- src/SimpleKafkaTests/SimpleKafkaTests.csproj | 2 + .../Unit/BigEndianDecoderTests.cs | 8 +- .../Unit/BigEndianEncoderTests.cs | 10 +- .../Unit/ProtocolBaseRequestTests.cs | 2 +- .../Unit/ProtocolMessageTests.cs | 18 +- .../Simple/SimpleBigEndianReader.cs} | 14 +- .../Simple/SimpleBigEndianWriter.cs} | 6 +- 33 files changed, 706 insertions(+), 346 deletions(-) create mode 100644 src/SimpleKafka/KafkaConsumer.cs rename src/SimpleKafka/{BigEndianDecoder.cs => KafkaDecoder.cs} (94%) rename src/SimpleKafka/{BigEndianEncoder.cs => KafkaEncoder.cs} (98%) create mode 100644 src/SimpleKafka/ReceivedKafkaMessage.cs create mode 100644 src/SimpleKafka/TopicSelector.cs create mode 100644 src/SimpleKafkaTests/Integration/ConsumerTests.cs create mode 100644 src/SimpleKafkaTests/Integration/IntegrationHelpers.cs rename src/{SimpleKafka/BigEndianReader.cs => kafka-net/Simple/SimpleBigEndianReader.cs} (80%) rename src/{SimpleKafka/BigEndianWriter.cs => kafka-net/Simple/SimpleBigEndianWriter.cs} (90%) diff --git a/src/SimpleKafka/Interfaces/IKafkaRequest.cs b/src/SimpleKafka/Interfaces/IKafkaRequest.cs index 632ab2db..ad608f34 100644 --- a/src/SimpleKafka/Interfaces/IKafkaRequest.cs +++ b/src/SimpleKafka/Interfaces/IKafkaRequest.cs @@ -34,12 +34,12 @@ public interface IKafkaRequest /// Encode this request into the Kafka wire protocol. /// /// Encoder to use - void Encode(ref BigEndianEncoder encoder); + void Encode(ref KafkaEncoder encoder); /// /// Decode a response payload from Kafka into T. /// /// Decoder to use /// Response - T Decode(ref BigEndianDecoder decoder); + T Decode(ref KafkaDecoder decoder); } } \ No newline at end of file diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs index 309cd908..5c0d0925 100644 --- a/src/SimpleKafka/KafkaBrokers.cs +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -37,43 +37,22 @@ public KafkaBrokers(params Uri[] addresses) } } - - public async Task RefreshAsync(CancellationToken token) - { - while (await TryToRefreshAsync(token).ConfigureAwait(false)) - { - - if (!IsLeaderElectionTakingPlace) - { - return true; - } - Log.Verbose("Leader election taking place"); - await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); - } - return false; - - } - private bool IsLeaderElectionTakingPlace + private bool IsLeaderElectionTakingPlaceForTopicAndPartition(string topic, int partition) { - get + var partitionsMap = topicToPartitions.TryGetValue(topic); + if (partitionsMap == null) { - foreach (var topicKvp in topicToPartitions) - { - foreach (var partition in topicKvp.Value) - { - if (partition.LeaderId == -1) - { - return true; - } - } - } - return false; } + else + { + var partitionInfo = partitionsMap[partition]; + return partitionInfo.LeaderId == -1; + } } - private async Task TryToRefreshAsync(CancellationToken token) + public async Task RefreshAsync(CancellationToken token) { if (brokers.Count == 0) { @@ -124,6 +103,87 @@ internal void AddTopic(string topic) } } + public async Task, T>>> BuildBrokerMapAsync(CancellationToken token, Dictionary> topicMap) + { + if (connections.Count == 0) + { + await RefreshAsync(token).ConfigureAwait(false); + } + + var ready = false; + while (!ready) + { + ready = true; + foreach (var topicKvp in topicMap) + { + var topic = topicKvp.Key; + var partitions = GetPartitionsForTopic(topic); + if (partitions == null) + { + AddTopic(topic); + var refreshed = await RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new KeyNotFoundException("Failed to refresh brokers"); + } + partitions = GetPartitionsForTopic(topic); + if (partitions == null) + { + throw new KeyNotFoundException("Failed to find topic: " + topic); + } + } + + foreach (var partitionKvp in topicKvp.Value) + { + var partitionNumber = partitionKvp.Key; + if (partitionNumber >= partitions.Length) + { + throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionNumber + " is too big. Only have " + partitions.Length + " partitions"); + } + + var partition = partitions[partitionNumber]; + if (partition.LeaderId == -1) + { + Log.Information("Topic {topic}, partition {partition} has no leader, waiting", topic, partitionNumber); + ready = false; + break; + } + } + if (!ready) + { + break; + } + } + + if (!ready) + { + Log.Verbose("Waiting before trying again"); + await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); + var refreshed = await RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new KeyNotFoundException("Failed to refresh brokers"); + } + } + } + + var brokerMap = new Dictionary, T>>(); + foreach (var topicKvp in topicMap) + { + var topic = topicKvp.Key; + var partitions = GetPartitionsForTopic(topic); + foreach (var partitionKvp in topicKvp.Value) + { + var partitionNumber = partitionKvp.Key; + var partition = partitions[partitionNumber]; + var brokerTopics = brokerMap.FindOrCreate(partition.LeaderId); + brokerTopics.Add(Tuple.Create(topic, partitionNumber), partitionKvp.Value); + } + } + + return brokerMap; + } + internal Partition[] GetPartitionsForTopic(string topic) { return topicToPartitions.TryGetValue(topic); @@ -137,10 +197,9 @@ private async Task TryToRefreshFromCurrentConnectionsAsync(CancellationToken tok var success = await TryToRefreshFromConnectionAsync(connection, token).ConfigureAwait(false); if (success) { - return; + break; } - else - { + else { connection.Dispose(); connections.Remove(connectionKvp.Key); } diff --git a/src/SimpleKafka/KafkaConnection.cs b/src/SimpleKafka/KafkaConnection.cs index 2bd3fb8e..ca086ddf 100644 --- a/src/SimpleKafka/KafkaConnection.cs +++ b/src/SimpleKafka/KafkaConnection.cs @@ -23,8 +23,8 @@ internal static async Task CreateAsync(IPEndPoint serverEndpoin private readonly TcpClient client; private readonly byte[] buffer; private readonly NetworkStream stream; - private BigEndianDecoder decoder; - private BigEndianEncoder encoder; + private KafkaDecoder decoder; + private KafkaEncoder encoder; private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferSize = 65536) { @@ -32,8 +32,8 @@ private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferS this.client = client; this.stream = client.GetStream(); this.buffer = new byte[bufferSize]; - decoder = new BigEndianDecoder(buffer); - encoder = new BigEndianEncoder(buffer); + decoder = new KafkaDecoder(buffer); + encoder = new KafkaEncoder(buffer); } @@ -84,7 +84,9 @@ protected virtual void Dispose(bool disposing) { if (disposing) { + stream.Dispose(); client.Close(); + clientLock.Dispose(); } disposedValue = true; } diff --git a/src/SimpleKafka/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs new file mode 100644 index 00000000..3504a370 --- /dev/null +++ b/src/SimpleKafka/KafkaConsumer.cs @@ -0,0 +1,134 @@ +using Serilog; +using SimpleKafka.Protocol; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class KafkaConsumer + { + private class TopicTracker + { + public readonly OffsetSelectionStrategy failureOffsetSelection; + public long nextOffset; + + public TopicTracker(TopicSelector selector) + { + switch (selector.DefaultOffsetSelection) + { + case OffsetSelectionStrategy.Earliest: nextOffset = -2; break; + case OffsetSelectionStrategy.Latest: nextOffset = -1; break; + case OffsetSelectionStrategy.Specified: nextOffset = selector.Offset; break; + default: throw new InvalidOperationException("Unknown default offset selection: " + selector.DefaultOffsetSelection); + } + failureOffsetSelection = selector.FailureOffsetSelection; + } + } + + private readonly KafkaBrokers brokers; + private readonly IKafkaSerializer keySerializer; + private readonly IKafkaSerializer valueSerializer; + private readonly Dictionary> topicMap; + private readonly int maxWaitTimeMs = 1000; + private readonly int minBytes = 1024; + private readonly int maxBytes = 65536; + + public KafkaConsumer(KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, params TopicSelector[] topics) + { + this.brokers = brokers; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + topicMap = new Dictionary>(); + foreach (var topic in topics) + { + var partitionMap = topicMap.FindOrCreate(topic.Topic); + if (partitionMap.ContainsKey(topic.Partition)) + { + throw new InvalidOperationException("Topic " + topic.Topic + ", partition " + topic.Partition + " duplicated"); + } + partitionMap.Add(topic.Partition, new TopicTracker(topic)); + } + } + + public async Task>> ReceiveAsync(CancellationToken token) + { + var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap); + var tasks = CreateFetchTasks(token, brokerMap); + var taskResults = await Task.WhenAll(tasks).ConfigureAwait(false); + + var messages = new List>(); + foreach (var taskResult in taskResults) + { + foreach (var fetchResponse in taskResult) + { + if (fetchResponse.Error != (int)ErrorResponseCode.NoError) + { + Log.Error("Error in fetch response {error} for {topic}/{partition}", (ErrorResponseCode)fetchResponse.Error, fetchResponse.Topic, fetchResponse.PartitionId); + } else + { + var tracker = topicMap[fetchResponse.Topic][fetchResponse.PartitionId]; + foreach (var message in fetchResponse.Messages) + { + var result = new ReceivedKafkaMessage( + fetchResponse.Topic, + keySerializer.Deserialize(message.Key), + valueSerializer.Deserialize(message.Value), + fetchResponse.PartitionId, + message.Meta.Offset + ); + tracker.nextOffset = message.Meta.Offset + 1; + messages.Add(result); + } + } + } + } + return messages; + } + + private List>> CreateFetchTasks(CancellationToken token, Dictionary, TopicTracker>> brokerMap) + { + var tasks = new List>>(brokerMap.Count); + + foreach (var brokerKvp in brokerMap) + { + var brokerId = brokerKvp.Key; + var trackerMap = brokerKvp.Value; + var request = CreateRequest(trackerMap); + + tasks.Add(brokers[brokerId].SendRequestAsync(request, token)); + } + + return tasks; + } + + private FetchRequest CreateRequest(Dictionary, TopicTracker> trackerMap) + { + var fetches = new List(trackerMap.Count); + foreach (var kvp in trackerMap) + { + var topic = kvp.Key.Item1; + var partition = kvp.Key.Item2; + var tracker = kvp.Value; + var fetch = new Fetch + { + MaxBytes = maxBytes, + Offset = tracker.nextOffset, + PartitionId = partition, + Topic = topic, + }; + fetches.Add(fetch); + } + var request = new FetchRequest + { + MaxWaitTime = maxWaitTimeMs, + MinBytes = 1024, + Fetches = fetches, + }; + return request; + } + } +} diff --git a/src/SimpleKafka/BigEndianDecoder.cs b/src/SimpleKafka/KafkaDecoder.cs similarity index 94% rename from src/SimpleKafka/BigEndianDecoder.cs rename to src/SimpleKafka/KafkaDecoder.cs index d161746b..6daff1a1 100644 --- a/src/SimpleKafka/BigEndianDecoder.cs +++ b/src/SimpleKafka/KafkaDecoder.cs @@ -6,7 +6,7 @@ namespace SimpleKafka { - public struct BigEndianDecoder + public struct KafkaDecoder { private int offset; public int Offset { get { return offset; } } @@ -23,8 +23,8 @@ public void SetOffset(int offset) public byte[] Buffer { get { return buffer; } } - public BigEndianDecoder(byte[] buffer) : this(buffer, 0, buffer.Length) { } - public BigEndianDecoder(byte[] buffer, int offset, int length) + public KafkaDecoder(byte[] buffer) : this(buffer, 0, buffer.Length) { } + public KafkaDecoder(byte[] buffer, int offset, int length) { this.buffer = buffer; this.length = length; diff --git a/src/SimpleKafka/BigEndianEncoder.cs b/src/SimpleKafka/KafkaEncoder.cs similarity index 98% rename from src/SimpleKafka/BigEndianEncoder.cs rename to src/SimpleKafka/KafkaEncoder.cs index f5a10f12..6edbe3fd 100644 --- a/src/SimpleKafka/BigEndianEncoder.cs +++ b/src/SimpleKafka/KafkaEncoder.cs @@ -7,7 +7,7 @@ namespace SimpleKafka { - public struct BigEndianEncoder + public struct KafkaEncoder { private int offset; public int Offset { get { return offset; } } @@ -20,7 +20,7 @@ public void SetOffset(int offset) private readonly byte[] buffer; public byte[] Buffer { get { return buffer; } } - public BigEndianEncoder(byte[] buffer, int offset = 0) + public KafkaEncoder(byte[] buffer, int offset = 0) { this.offset = offset; this.buffer = buffer; diff --git a/src/SimpleKafka/KafkaProducer.cs b/src/SimpleKafka/KafkaProducer.cs index 765a4e20..05d5400a 100644 --- a/src/SimpleKafka/KafkaProducer.cs +++ b/src/SimpleKafka/KafkaProducer.cs @@ -51,7 +51,7 @@ public async Task SendAsync(IEnumerable> messages, Can while (topicMap.Count > 0) { - var brokerMap = await BuildBrokerMap(token, topicMap); + var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap).ConfigureAwait(false); var completed = await SendMessagesToBrokersAsync(token, topicMap, brokerMap).ConfigureAwait(false); if (!completed) @@ -98,45 +98,6 @@ private async Task SendMessagesToBrokersAsync(CancellationToken token, Dic return brokerMap.Count > 0; } - private async Task, List>>> BuildBrokerMap(CancellationToken token, Dictionary>> topicMap) - { - var brokerMap = new Dictionary, List>>(); - foreach (var topicKvp in topicMap) - { - var topic = topicKvp.Key; - var partitions = brokers.GetPartitionsForTopic(topic); - if (partitions == null) - { - brokers.AddTopic(topic); - var refreshed = await brokers.RefreshAsync(token).ConfigureAwait(false); - if (!refreshed) - { - throw new KeyNotFoundException("Failed to refresh brokers"); - } - partitions = brokers.GetPartitionsForTopic(topic); - if (partitions == null) - { - throw new KeyNotFoundException("Failed to find topic: " + topic); - } - } - - foreach (var partitionKvp in topicKvp.Value) - { - var partitionNumber = partitionKvp.Key; - if (partitionNumber >= partitions.Length) - { - throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionNumber + " is too big. Only have " + partitions.Length + " partitions"); - } - - var partition = partitions[partitionNumber]; - var brokerTopics = brokerMap.FindOrCreate(partition.LeaderId); - var topicMessages = brokerTopics.FindOrCreate(Tuple.Create(topic, partitionNumber)); - topicMessages.AddRange(partitionKvp.Value); - } - } - - return brokerMap; - } private async Task> ProduceMessagesToBroker(int brokerId, Dictionary,List> topicMessages, CancellationToken token) { diff --git a/src/SimpleKafka/Protocol/BaseRequest.cs b/src/SimpleKafka/Protocol/BaseRequest.cs index 94df9233..5eb913ef 100644 --- a/src/SimpleKafka/Protocol/BaseRequest.cs +++ b/src/SimpleKafka/Protocol/BaseRequest.cs @@ -44,7 +44,7 @@ protected BaseRequest(short apiVersion = 0) /// public virtual bool ExpectResponse { get { return true; } } - internal static void EncodeHeader(IKafkaRequest request, ref BigEndianEncoder encoder) + internal static void EncodeHeader(IKafkaRequest request, ref KafkaEncoder encoder) { encoder.Write((Int16)request.ApiKey); encoder.Write(request.ApiVersion); diff --git a/src/SimpleKafka/Protocol/Broker.cs b/src/SimpleKafka/Protocol/Broker.cs index 44059534..08bdd73d 100644 --- a/src/SimpleKafka/Protocol/Broker.cs +++ b/src/SimpleKafka/Protocol/Broker.cs @@ -10,7 +10,7 @@ public class Broker public int Port { get; set; } public Uri Address { get { return new Uri(string.Format("http://{0}:{1}", Host, Port));} } - public static Broker Decode(ref BigEndianDecoder decoder) + public static Broker Decode(ref KafkaDecoder decoder) { return new Broker { diff --git a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs index 002c1a57..2152ba1c 100644 --- a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs +++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs @@ -14,24 +14,24 @@ public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest> public List Fetches { get; set; } - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeFetchRequest(this, ref encoder); } - public List Decode(ref BigEndianDecoder decoder) + public List Decode(ref KafkaDecoder decoder) { return DecodeFetchResponses(ref decoder); } - private static void EncodeFetchRequest(FetchRequest request, ref BigEndianEncoder encoder) + private static void EncodeFetchRequest(FetchRequest request, ref KafkaEncoder encoder) { if (request.Fetches == null) request.Fetches = new List(); EncodeHeader(request, ref encoder); @@ -69,7 +69,7 @@ private static void EncodeFetchRequest(FetchRequest request, ref BigEndianEncode } } - private List DecodeFetchResponses(ref BigEndianDecoder decoder) + private List DecodeFetchResponses(ref KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); diff --git a/src/SimpleKafka/Protocol/Message.cs b/src/SimpleKafka/Protocol/Message.cs index 86f5b685..b3121d88 100644 --- a/src/SimpleKafka/Protocol/Message.cs +++ b/src/SimpleKafka/Protocol/Message.cs @@ -72,7 +72,7 @@ public Message(string value, string key = null) /// /// The collection of messages to encode together. /// Encoded byte[] representing the collection of messages. - public static void EncodeMessageSet(ref BigEndianEncoder encoder, IEnumerable messages) + public static void EncodeMessageSet(ref KafkaEncoder encoder, IEnumerable messages) { foreach (var message in messages) { @@ -88,7 +88,7 @@ public static void EncodeMessageSet(ref BigEndianEncoder encoder, IEnumerable /// The decoder positioned at the start of the buffer /// The messages - public static List DecodeMessageSet(int partitionId, ref BigEndianDecoder decoder, int messageSetSize) + public static List DecodeMessageSet(int partitionId, ref KafkaDecoder decoder, int messageSetSize) { var numberOfBytes = messageSetSize; @@ -130,7 +130,7 @@ public static List DecodeMessageSet(int partitionId, ref BigEndianDecod /// Format: /// Crc (Int32), MagicByte (Byte), Attribute (Byte), Key (Byte[]), Value (Byte[]) /// - public static void EncodeMessage(Message message, ref BigEndianEncoder encoder) + public static void EncodeMessage(Message message, ref KafkaEncoder encoder) { var marker = encoder.PrepareForCrc(); encoder.Write(message.MagicNumber); @@ -147,7 +147,7 @@ public static void EncodeMessage(Message message, ref BigEndianEncoder encoder) /// The byte[] encode as a message from kafka. /// The message /// The return type is an Enumerable as the message could be a compressed message set. - public static Message DecodeMessage(long offset, int partitionId, ref BigEndianDecoder decoder, int messageSize) + public static Message DecodeMessage(long offset, int partitionId, ref KafkaDecoder decoder, int messageSize) { var crc = decoder.ReadUInt32(); var calculatedCrc = Crc32Provider.Compute(decoder.Buffer, decoder.Offset, messageSize - 4); diff --git a/src/SimpleKafka/Protocol/MetadataRequest.cs b/src/SimpleKafka/Protocol/MetadataRequest.cs index 0a4fef2b..99bf3bcd 100644 --- a/src/SimpleKafka/Protocol/MetadataRequest.cs +++ b/src/SimpleKafka/Protocol/MetadataRequest.cs @@ -15,17 +15,17 @@ public class MetadataRequest : BaseRequest, IKafkaRequest /// public List Topics { get; set; } - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeMetadataRequest(this, ref encoder); } - public MetadataResponse Decode(ref BigEndianDecoder decoder) + public MetadataResponse Decode(ref KafkaDecoder decoder) { return DecodeMetadataResponse(ref decoder); } - private static void EncodeMetadataRequest(MetadataRequest request, ref BigEndianEncoder encoder) + private static void EncodeMetadataRequest(MetadataRequest request, ref KafkaEncoder encoder) { if (request.Topics == null) request.Topics = new List(); EncodeHeader(request, ref encoder); @@ -36,7 +36,7 @@ private static void EncodeMetadataRequest(MetadataRequest request, ref BigEndian } } - private static MetadataResponse DecodeMetadataResponse(ref BigEndianDecoder decoder) + private static MetadataResponse DecodeMetadataResponse(ref KafkaDecoder decoder) { var response = new MetadataResponse { diff --git a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs index 8c0d3ba8..de67e599 100644 --- a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs @@ -22,17 +22,17 @@ public OffsetCommitRequest(Int16 version = 1) : base(version) public string ConsumerId { get; set; } public List OffsetCommits { get; set; } - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeOffsetCommitRequest(this, ref encoder); } - public List Decode(ref BigEndianDecoder decoder) + public List Decode(ref KafkaDecoder decoder) { return DecodeOffsetCommitResponse(ref decoder); } - private static void EncodeOffsetCommitRequest(OffsetCommitRequest request, ref BigEndianEncoder encoder) + private static void EncodeOffsetCommitRequest(OffsetCommitRequest request, ref KafkaEncoder encoder) { if (request.OffsetCommits == null) request.OffsetCommits = new List(); EncodeHeader(request, ref encoder); @@ -70,7 +70,7 @@ private static void EncodeOffsetCommitRequest(OffsetCommitRequest request, ref B } } - private static List DecodeOffsetCommitResponse(ref BigEndianDecoder decoder) + private static List DecodeOffsetCommitResponse(ref KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); diff --git a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs index daa7f152..e724d2bf 100644 --- a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs @@ -22,12 +22,12 @@ public OffsetFetchRequest(short version = 1) : base(version) public string ConsumerGroup { get; set; } public List Topics { get; set; } - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeOffsetFetchRequest(this, ref encoder); } - private static void EncodeOffsetFetchRequest(OffsetFetchRequest request, ref BigEndianEncoder encoder) + private static void EncodeOffsetFetchRequest(OffsetFetchRequest request, ref KafkaEncoder encoder) { if (request.Topics == null) request.Topics = new List(); EncodeHeader(request, ref encoder); @@ -54,13 +54,13 @@ private static void EncodeOffsetFetchRequest(OffsetFetchRequest request, ref Big } - public List Decode(ref BigEndianDecoder decoder) + public List Decode(ref KafkaDecoder decoder) { return DecodeOffsetFetchResponse(ref decoder); } - private static List DecodeOffsetFetchResponse(ref BigEndianDecoder decoder) + private static List DecodeOffsetFetchResponse(ref KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); diff --git a/src/SimpleKafka/Protocol/OffsetRequest.cs b/src/SimpleKafka/Protocol/OffsetRequest.cs index c214b0eb..bbbae57c 100644 --- a/src/SimpleKafka/Protocol/OffsetRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetRequest.cs @@ -13,17 +13,17 @@ public class OffsetRequest : BaseRequest, IKafkaRequest> public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Offset; } } public List Offsets { get; set; } - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeOffsetRequest(this, ref encoder); } - public List Decode(ref BigEndianDecoder decoder) + public List Decode(ref KafkaDecoder decoder) { return DecodeOffsetResponse(ref decoder); } - private static void EncodeOffsetRequest(OffsetRequest request, ref BigEndianEncoder encoder) + private static void EncodeOffsetRequest(OffsetRequest request, ref KafkaEncoder encoder) { if (request.Offsets == null) request.Offsets = new List(); EncodeHeader(request, ref encoder); @@ -52,7 +52,7 @@ private static void EncodeOffsetRequest(OffsetRequest request, ref BigEndianEnco } - private static List DecodeOffsetResponse(ref BigEndianDecoder decoder) + private static List DecodeOffsetResponse(ref KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); diff --git a/src/SimpleKafka/Protocol/ProduceRequest.cs b/src/SimpleKafka/Protocol/ProduceRequest.cs index 361bd01a..84d9813c 100644 --- a/src/SimpleKafka/Protocol/ProduceRequest.cs +++ b/src/SimpleKafka/Protocol/ProduceRequest.cs @@ -29,18 +29,18 @@ public class ProduceRequest : BaseRequest, IKafkaRequest> public List Payload = new List(); - public void Encode(ref BigEndianEncoder encoder) + public void Encode(ref KafkaEncoder encoder) { EncodeProduceRequest(this, ref encoder); } - public List Decode(ref BigEndianDecoder decoder) + public List Decode(ref KafkaDecoder decoder) { return DecodeProduceResponse(ref decoder); } #region Protocol... - private static void EncodeProduceRequest(ProduceRequest request, ref BigEndianEncoder encoder) + private static void EncodeProduceRequest(ProduceRequest request, ref KafkaEncoder encoder) { if (request.Payload == null) request.Payload = new List(); @@ -77,7 +77,7 @@ private static void EncodeProduceRequest(ProduceRequest request, ref BigEndianEn } } - private List DecodeProduceResponse(ref BigEndianDecoder decoder) + private List DecodeProduceResponse(ref KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); diff --git a/src/SimpleKafka/Protocol/Topic.cs b/src/SimpleKafka/Protocol/Topic.cs index 0fde0dab..a09ad19a 100644 --- a/src/SimpleKafka/Protocol/Topic.cs +++ b/src/SimpleKafka/Protocol/Topic.cs @@ -10,7 +10,7 @@ public class Topic public string Name { get; set; } public List Partitions { get; set; } - internal static Topic Decode(ref BigEndianDecoder decoder) + internal static Topic Decode(ref KafkaDecoder decoder) { var topic = new Topic { @@ -53,7 +53,7 @@ public class Partition /// public List Isrs { get; set; } - public static Partition Decode(ref BigEndianDecoder decoder) + public static Partition Decode(ref KafkaDecoder decoder) { var partition = new Partition { ErrorCode = decoder.ReadInt16(), diff --git a/src/SimpleKafka/ReceivedKafkaMessage.cs b/src/SimpleKafka/ReceivedKafkaMessage.cs new file mode 100644 index 00000000..b49a82fa --- /dev/null +++ b/src/SimpleKafka/ReceivedKafkaMessage.cs @@ -0,0 +1,23 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class ReceivedKafkaMessage : KafkaMessage + { + private readonly int partition; + public int Partition { get { return partition; } } + private readonly long offset; + public long Offset { get { return offset; } } + + public ReceivedKafkaMessage(string topic, TKey key, TValue value, int partition, long offset) : base(topic, key, value) + { + this.partition = partition; + this.offset = offset; + } + + } +} diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index 5ea82259..2ce4f172 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -47,10 +47,9 @@ - - - - + + + @@ -78,7 +77,9 @@ + + diff --git a/src/SimpleKafka/TopicSelector.cs b/src/SimpleKafka/TopicSelector.cs new file mode 100644 index 00000000..723b5805 --- /dev/null +++ b/src/SimpleKafka/TopicSelector.cs @@ -0,0 +1,33 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public enum OffsetSelectionStrategy + { + Earliest, + Latest, + Specified + }; + + public class TopicSelector + { + public string Topic { get; set; } + public int Partition { get; set; } + + public long Offset { get; set; } + + public OffsetSelectionStrategy DefaultOffsetSelection { get; set; } + public OffsetSelectionStrategy FailureOffsetSelection { get; set; } + + public TopicSelector() + { + DefaultOffsetSelection = OffsetSelectionStrategy.Specified; + FailureOffsetSelection = OffsetSelectionStrategy.Latest; + } + + } +} diff --git a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs index ab7c214d..b7e0ad23 100644 --- a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs +++ b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs @@ -12,7 +12,15 @@ public static class IntegrationConfig public static string IntegrationCompressionTopic = "IntegrationCompressionTopic"; public static string IntegrationTopic = "IntegrationTopic"; public static string IntegrationConsumer = "IntegrationConsumer"; - public static Uri IntegrationUri = new Uri("http://server.home:9092"); +// public static Uri IntegrationUri = new Uri("http://server.home:9092"); + public static Uri IntegrationUri = new Uri("http://server.home:39092"); + public static Uri[] IntegrationUriArray = new[] + { + new Uri("http://server.home:39092"), + //new Uri("http://server.home:39093"), + //new Uri("http://server.home:39094"), + }; + public static Uri IntegrationUriOld { get diff --git a/src/SimpleKafkaTests/Helpers/MessageHelper.cs b/src/SimpleKafkaTests/Helpers/MessageHelper.cs index 8292cac4..49700987 100644 --- a/src/SimpleKafkaTests/Helpers/MessageHelper.cs +++ b/src/SimpleKafkaTests/Helpers/MessageHelper.cs @@ -20,7 +20,7 @@ public static byte[] CreateMessage(long offset, byte[] key, byte[] payload, byte }; var buffer = new byte[1024]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); Message.EncodeMessageSet(ref encoder, new[] { message }); var result = new byte[encoder.Offset]; diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs new file mode 100644 index 00000000..1512e63a --- /dev/null +++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs @@ -0,0 +1,53 @@ +using NUnit.Framework; +using SimpleKafka; +using SimpleKafkaTests.Helpers; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Integration +{ + [TestFixture] + [Category("Integration")] + class ConsumerTests + { + [SetUp] + public void Setup() + { + IntegrationHelpers.zookeeperHost = "zookeeper:2181"; + IntegrationHelpers.dockerOptions = "--link compose_zookeeper_1:zookeeper"; + } + [Test] + public async Task TestSimpleConsumerWorksOk() + { + var keySerializer = new NullSerializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new LoadBalancedPartitioner(1); + + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + { + var topic = temporaryTopic.Topic; + var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + var consumer = new KafkaConsumer(brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); + + await producer.SendAsync(new KafkaMessage(topic, null, "Message"), CancellationToken.None).ConfigureAwait(true); + + var responses = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(1)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(0)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("Message")); + } + } + + } +} diff --git a/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs b/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs new file mode 100644 index 00000000..34c48de1 --- /dev/null +++ b/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs @@ -0,0 +1,72 @@ +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Globalization; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Integration +{ + internal static class IntegrationHelpers + { + + public static string dockerHost = "tcp://server.home:2375"; + public static string zookeeperHost = "server.home"; + public static string kafkaImage = "sceneskope/kafka:0.8.2.1"; + public static string dockerOptions = ""; + + public static void RunKafkaTopicsCommand(params object[] args) + { + var cmd = string.Format(CultureInfo.InvariantCulture, "--host={0} run --rm {1} {2} bin/kafka-topics.sh --zookeeper {3} ", + dockerHost, dockerOptions, kafkaImage, zookeeperHost); + + var arguments = cmd + String.Join(" ", args); + + var info = new ProcessStartInfo + { + Arguments = arguments, + CreateNoWindow = true, +// FileName = @"c:\users\nick\bin\docker.exe", + FileName = @"docker.exe", + UseShellExecute = false, + RedirectStandardOutput = true, + }; + var process = Process.Start(info); + var stdout = process.StandardOutput.ReadToEnd(); + process.WaitForExit(); + Console.WriteLine(stdout); + } + + public static void DeleteTopic(string topic) + { + RunKafkaTopicsCommand("--topic", topic, "--delete"); + } + + public static void CreateTopic(string topic, int partitions = 1, int replicationFactor = 1) + { + RunKafkaTopicsCommand("--topic", topic, "--create", "--partitions", partitions, "--replication-factor", replicationFactor); + } + + public static TemporaryTopic CreateTemporaryTopic(int partitions = 1, int replicationFactor = 1) + { + return new TemporaryTopic(partitions, replicationFactor); + } + + public class TemporaryTopic : IDisposable + { + private readonly string topic = Guid.NewGuid().ToString(); + public string Topic { get { return topic; } } + public TemporaryTopic(int partitions = 1, int replicationFactor = 1) + { + CreateTopic(topic, partitions, replicationFactor); + } + + public void Dispose() + { + DeleteTopic(topic); + } + } + + } +} diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 5ed4cc34..74a2a90c 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -20,10 +20,12 @@ class SimpleTests [SetUp] public void Setup() { - + IntegrationHelpers.dockerHost = "tcp://server.home:2375"; + IntegrationHelpers.zookeeperHost = "server.home"; + IntegrationHelpers.dockerOptions = ""; } - [Test] + [Test] public async Task TestProducingWorksOk() { using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) @@ -132,40 +134,48 @@ public async Task TestOffsetWorksOk() [Test] public async Task TestNewTopicProductionWorksOk() { - var topic = Guid.NewGuid().ToString(); - RunKafkaTopicsCommand("--topic", topic, "--create", "--partitions", "1", "--replication-factor", "1"); - try + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) { - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + var topic = temporaryTopic.Topic; { + var request = new MetadataRequest { - var request = new MetadataRequest - { - Topics = new List + Topics = new List { topic } - }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); - var first = response; - Assert.That(first.Topics, Has.Count.EqualTo(1)); - - var firstTopic = first.Topics.First(); - Assert.That(firstTopic.ErrorCode, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(firstTopic.Name, Is.EqualTo(topic)); - Assert.That(firstTopic.Partitions, Has.Count.EqualTo(1)); - - var firstPartition = firstTopic.Partitions.First(); - Assert.That(firstPartition.PartitionId, Is.EqualTo(0)); + }; + MetadataResponse response = null; + while (response == null) + { + response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + if (response.Topics[0].ErrorCode == (short)ErrorResponseCode.LeaderNotAvailable) + { + response = null; + await Task.Delay(1000); + } + } + Assert.That(response, Is.Not.Null); + var first = response; + Assert.That(first.Topics, Has.Count.EqualTo(1)); + var firstTopic = first.Topics.First(); + Assert.That(firstTopic.ErrorCode, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(firstTopic.Name, Is.EqualTo(topic)); + Assert.That(firstTopic.Partitions, Has.Count.EqualTo(1)); + + var firstPartition = firstTopic.Partitions.First(); + Assert.That(firstPartition.PartitionId, Is.EqualTo(0)); + } + + { + var request = new ProduceRequest { - var request = new ProduceRequest - { - Acks = 1, - TimeoutMS = 10000, - Payload = new List + Acks = 1, + TimeoutMS = 10000, + Payload = new List { new Payload { @@ -181,24 +191,24 @@ public async Task TestNewTopicProductionWorksOk() } } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Is.Not.Null); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Is.Not.Null); - var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Offset, Is.EqualTo(0)); - } + var first = response.First(); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Offset, Is.EqualTo(0)); + } + { + var request = new FetchRequest { - var request = new FetchRequest - { - MinBytes = 0, - MaxWaitTime = 0, - Fetches = new List + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List { new Fetch { @@ -208,34 +218,34 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic, } } - }; - - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); - - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.HighWaterMark, Is.EqualTo(4)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.Messages, Has.Count.EqualTo(1)); - - var firstMessage = first.Messages.First(); - Assert.That(firstMessage.Meta.Offset, Is.EqualTo(0)); - Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); - Assert.That(firstMessage.Attribute, Is.EqualTo(0)); - Assert.That(firstMessage.Key, Is.Null); - Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); - Assert.That(firstMessage.Value, Is.Not.Null); - - var firstString = firstMessage.Value.ToUtf8String(); - Assert.That(firstString, Is.EqualTo("Message 1")); - } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(1)); + + var firstMessage = first.Messages.First(); + Assert.That(firstMessage.Meta.Offset, Is.EqualTo(0)); + Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(firstMessage.Attribute, Is.EqualTo(0)); + Assert.That(firstMessage.Key, Is.Null); + Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(firstMessage.Value, Is.Not.Null); + + var firstString = firstMessage.Value.ToUtf8String(); + Assert.That(firstString, Is.EqualTo("Message 1")); + } + { + var request = new OffsetRequest { - var request = new OffsetRequest - { - Offsets = new List + Offsets = new List { new Offset { @@ -245,26 +255,46 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Offsets, Has.Count.EqualTo(2)); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Offsets, Has.Count.EqualTo(2)); - Assert.That(first.Offsets[0], Is.EqualTo(4)); - Assert.That(first.Offsets[1], Is.EqualTo(0)); - } + Assert.That(first.Offsets[0], Is.EqualTo(4)); + Assert.That(first.Offsets[1], Is.EqualTo(0)); + } + { + var request = new ConsumerMetadataRequest { - var request = new OffsetFetchRequest + ConsumerGroup = topic + }; + ConsumerMetadataResponse response = null; + while (response == null) + { + response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + if (response.Error == ErrorResponseCode.ConsumerCoordinatorNotAvailableCode) { - ConsumerGroup = topic, - Topics = new List + response = null; + await Task.Delay(1000); + } + } + Assert.That(response.Error, Is.EqualTo(ErrorResponseCode.NoError)); + Console.WriteLine("Id = {0}, Host = {1}, Port = {2}", response.CoordinatorId, response.CoordinatorHost, response.CoordinatorPort); + + } + + { + var request = new OffsetFetchRequest + { + ConsumerGroup = topic, + Topics = new List { new OffsetFetch { @@ -272,26 +302,26 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.MetaData, Is.Empty); - Assert.That(first.Offset, Is.EqualTo(-1)); - } + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.MetaData, Is.Empty); + Assert.That(first.Offset, Is.EqualTo(-1)); + } + { + var request = new OffsetCommitRequest { - var request = new OffsetCommitRequest - { - ConsumerGroup = topic, - ConsumerGroupGenerationId = 1, - ConsumerId = "0", - OffsetCommits = new List + ConsumerGroup = topic, + ConsumerGroupGenerationId = 1, + ConsumerId = "0", + OffsetCommits = new List { new OffsetCommit { @@ -302,21 +332,21 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic, } } - }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); - - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - } + }; + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + } + { + var request = new OffsetFetchRequest { - var request = new OffsetFetchRequest - { - ConsumerGroup = topic, - Topics = new List + ConsumerGroup = topic, + Topics = new List { new OffsetFetch { @@ -324,25 +354,25 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.MetaData, Is.EqualTo("Metadata 1")); - Assert.That(first.Offset, Is.EqualTo(0)); - } + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.MetaData, Is.EqualTo("Metadata 1")); + Assert.That(first.Offset, Is.EqualTo(0)); + } + { + var request = new FetchRequest { - var request = new FetchRequest - { - MinBytes = 0, - MaxWaitTime = 0, - Fetches = new List + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List { new Fetch { @@ -352,49 +382,49 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic, } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.HighWaterMark, Is.EqualTo(4)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.Messages, Has.Count.EqualTo(3)); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(3)); - var firstMessage = first.Messages.First(); - Assert.That(firstMessage.Meta.Offset, Is.EqualTo(1)); - Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); - Assert.That(firstMessage.Attribute, Is.EqualTo(0)); - Assert.That(firstMessage.Key, Is.Null); - Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); - Assert.That(firstMessage.Value, Is.Not.Null); + var firstMessage = first.Messages.First(); + Assert.That(firstMessage.Meta.Offset, Is.EqualTo(1)); + Assert.That(firstMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(firstMessage.Attribute, Is.EqualTo(0)); + Assert.That(firstMessage.Key, Is.Null); + Assert.That(firstMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(firstMessage.Value, Is.Not.Null); - var firstString = firstMessage.Value.ToUtf8String(); - Assert.That(firstString, Is.EqualTo("Message 2")); + var firstString = firstMessage.Value.ToUtf8String(); + Assert.That(firstString, Is.EqualTo("Message 2")); - var lastMessage = first.Messages.Last(); - Assert.That(lastMessage.Meta.Offset, Is.EqualTo(3)); - Assert.That(lastMessage.Meta.PartitionId, Is.EqualTo(0)); - Assert.That(lastMessage.Attribute, Is.EqualTo(0)); - Assert.That(lastMessage.Key, Is.Null); - Assert.That(lastMessage.MagicNumber, Is.EqualTo(0)); - Assert.That(lastMessage.Value, Is.Not.Null); + var lastMessage = first.Messages.Last(); + Assert.That(lastMessage.Meta.Offset, Is.EqualTo(3)); + Assert.That(lastMessage.Meta.PartitionId, Is.EqualTo(0)); + Assert.That(lastMessage.Attribute, Is.EqualTo(0)); + Assert.That(lastMessage.Key, Is.Null); + Assert.That(lastMessage.MagicNumber, Is.EqualTo(0)); + Assert.That(lastMessage.Value, Is.Not.Null); - var lastString = lastMessage.Value.ToUtf8String(); - Assert.That(lastString, Is.EqualTo("Message 4")); + var lastString = lastMessage.Value.ToUtf8String(); + Assert.That(lastString, Is.EqualTo("Message 4")); - } + } + { + var request = new FetchRequest { - var request = new FetchRequest - { - MinBytes = 0, - MaxWaitTime = 0, - Fetches = new List + MinBytes = 0, + MaxWaitTime = 0, + Fetches = new List { new Fetch { @@ -404,26 +434,20 @@ public async Task TestNewTopicProductionWorksOk() Topic = topic, } } - }; + }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Assert.That(response, Has.Count.EqualTo(1)); - var first = response.First(); + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); - Assert.That(first.HighWaterMark, Is.EqualTo(4)); - Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Topic, Is.EqualTo(topic)); - Assert.That(first.Messages, Has.Count.EqualTo(0)); - } - } - Console.WriteLine("Test completed"); - } - finally - { - RunKafkaTopicsCommand("--topic", topic, "--delete"); + Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.HighWaterMark, Is.EqualTo(4)); + Assert.That(first.PartitionId, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Messages, Has.Count.EqualTo(0)); + } } - + Console.WriteLine("Test completed"); } [Test] @@ -454,23 +478,5 @@ public async Task TestSimpleProducerWorksOk() } } - private static void RunKafkaTopicsCommand(params string[] args) - { - var cmd = "--host=tcp://server.home:2375 run --rm sceneskope/kafka:0.8.2.1 bin/kafka-topics.sh --zookeeper server.home "; - var arguments = cmd + String.Join(" ", args); - - var info = new ProcessStartInfo - { - Arguments = arguments, - CreateNoWindow = true, - FileName = @"c:\users\nick\bin\docker.exe", - UseShellExecute = false, - RedirectStandardOutput = true, - }; - var process = Process.Start(info); - var stdout = process.StandardOutput.ReadToEnd(); - process.WaitForExit(); - Console.WriteLine(stdout); - } } } diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index 42f74bb6..856d02b9 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -47,6 +47,8 @@ + + diff --git a/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs index 89af9f61..ccabe1bc 100644 --- a/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs +++ b/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs @@ -23,7 +23,7 @@ public class BigEndianDecoderTests [TestCase(Int64.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF })] public void Int64Tests(Int64 expectedValue, Byte[] givenBytes) { - var decoder = new BigEndianDecoder(givenBytes); + var decoder = new KafkaDecoder(givenBytes); var actualValue = decoder.ReadInt64(); Assert.That(actualValue, Is.EqualTo(expectedValue)); } @@ -36,7 +36,7 @@ public void Int64Tests(Int64 expectedValue, Byte[] givenBytes) [TestCase(Int32.MaxValue, new Byte[] { 0x7F, 0xFF, 0xFF, 0xFF })] public void Int32Tests(Int32 expectedValue, Byte[] givenBytes) { - var decoder = new BigEndianDecoder(givenBytes); + var decoder = new KafkaDecoder(givenBytes); var actualValue = decoder.ReadInt32(); Assert.That(actualValue, Is.EqualTo(expectedValue)); } @@ -50,7 +50,7 @@ public void Int32Tests(Int32 expectedValue, Byte[] givenBytes) [TestCase(Int16.MaxValue, new Byte[] { 0x7F, 0xFF })] public void Int16Tests(Int16 expectedValue, Byte[] givenBytes) { - var decoder = new BigEndianDecoder(givenBytes); + var decoder = new KafkaDecoder(givenBytes); var actualValue = decoder.ReadInt16(); Assert.That(actualValue, Is.EqualTo(expectedValue)); } @@ -70,7 +70,7 @@ public void Int16Tests(Int16 expectedValue, Byte[] givenBytes) [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] public void StringTests(String expectedValue, Byte[] givenBytes, StringPrefixEncoding encoding) { - var decoder = new BigEndianDecoder(givenBytes); + var decoder = new KafkaDecoder(givenBytes); string actualValue = null; switch (encoding) { diff --git a/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs b/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs index 201d747a..349ecfa6 100644 --- a/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs +++ b/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs @@ -24,7 +24,7 @@ public class BigEndianEncoderTests public void Int64Tests(Int64 number, Byte[] expectedBytes) { var buffer = new byte[8]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); encoder.Write(number); Assert.That(buffer, Is.EqualTo(expectedBytes)); } @@ -36,7 +36,7 @@ public void Int64Tests(Int64 number, Byte[] expectedBytes) public void UInt32Tests(UInt32 number, Byte[] expectedBytes) { var buffer = new byte[4]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); encoder.Write(number); Assert.That(buffer, Is.EqualTo(expectedBytes)); } @@ -50,7 +50,7 @@ public void UInt32Tests(UInt32 number, Byte[] expectedBytes) public void Int32Tests(Int32 number, Byte[] expectedBytes) { var buffer = new byte[4]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); encoder.Write(number); Assert.That(buffer, Is.EqualTo(expectedBytes)); } @@ -64,7 +64,7 @@ public void Int32Tests(Int32 number, Byte[] expectedBytes) public void Int16Tests(Int16 number, Byte[] expectedBytes) { var buffer = new byte[2]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); encoder.Write(number); Assert.That(buffer, Is.EqualTo(expectedBytes)); } @@ -86,7 +86,7 @@ public void Int16Tests(Int16 number, Byte[] expectedBytes) public void StringTests(String value, Byte[] expectedBytes, StringPrefixEncoding encoding) { var buffer = new byte[expectedBytes.Length]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); encoder.Write(value, encoding); Assert.That(encoder.Offset, Is.EqualTo(expectedBytes.Length)); Assert.That(buffer, Is.EqualTo(expectedBytes)); diff --git a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs index e246723d..c8dc792d 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs @@ -15,7 +15,7 @@ public class ProtocolBaseRequestTests [Test] public void EnsureHeaderShouldPackCorrectByteLengths() { - var encoder = new BigEndianEncoder(new byte[14]); + var encoder = new KafkaEncoder(new byte[14]); var request = new FetchRequest { ClientId = "test", CorrelationId = 123456789 }; BaseRequest.EncodeHeader(request, ref encoder); diff --git a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs index 157d27dc..22295acb 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs @@ -21,12 +21,12 @@ public void DecodeMessageShouldThrowWhenCrcFails() { var testMessage = new Message(value: "kafka test message.", key: "test"); var buffer = new byte[1024]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); Message.EncodeMessage(testMessage, ref encoder); buffer[0] += 1; - var decoder = new BigEndianDecoder(buffer, 0, encoder.Offset); + var decoder = new KafkaDecoder(buffer, 0, encoder.Offset); var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); }); } @@ -41,10 +41,10 @@ public void EnsureMessageEncodeAndDecodeAreCompatible(string key, string value) var testMessage = new Message(key: key, value: value); var buffer = new byte[1024]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); Message.EncodeMessage(testMessage, ref encoder); - var decoder = new BigEndianDecoder(buffer); + var decoder = new KafkaDecoder(buffer); var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); Assert.That(testMessage.Key, Is.EqualTo(result.Key)); @@ -70,7 +70,7 @@ public void EncodeMessageSetEncodesMultipleMessages() }; var buffer = new byte[expected.Length]; - var encoder = new BigEndianEncoder(buffer); + var encoder = new KafkaEncoder(buffer); Message.EncodeMessageSet(ref encoder, messages); Assert.That(buffer, Is.EqualTo(expected)); @@ -80,7 +80,7 @@ public void EncodeMessageSetEncodesMultipleMessages() public void DecodeMessageSetShouldHandleResponseWithMaxBufferSizeHit() { //This message set has a truncated message bytes at the end of it - var decoder = new BigEndianDecoder(MessageHelper.FetchResponseMaxBytesOverflow); + var decoder = new KafkaDecoder(MessageHelper.FetchResponseMaxBytesOverflow); var result = Message.DecodeMessageSet(0, ref decoder, decoder.Length); var message = Encoding.UTF8.GetString(result.First().Value); @@ -99,12 +99,12 @@ public void WhenMessageIsTruncatedThenBufferUnderRunExceptionIsThrown() var message = new Byte[] { }; var messageSize = 5; var payloadBytes = new byte[16]; - var encoder = new BigEndianEncoder(payloadBytes); + var encoder = new KafkaEncoder(payloadBytes); encoder.Write(offset); encoder.Write(messageSize); encoder.Write(message); - var decoder = new BigEndianDecoder(payloadBytes); + var decoder = new KafkaDecoder(payloadBytes); Message.DecodeMessageSet(0, ref decoder, payloadBytes.Length); }); @@ -118,7 +118,7 @@ public void WhenMessageIsExactlyTheSizeOfBufferThenMessageIsDecoded() var payload = MessageHelper.CreateMessage(0, new Byte[] { 0 }, expectedPayloadBytes); // act/assert - var decoder = new BigEndianDecoder(payload, 0, payload.Length); + var decoder = new KafkaDecoder(payload, 0, payload.Length); var messages = Message.DecodeMessageSet(0, ref decoder, payload.Length); var actualPayload = messages.First().Value; diff --git a/src/SimpleKafka/BigEndianReader.cs b/src/kafka-net/Simple/SimpleBigEndianReader.cs similarity index 80% rename from src/SimpleKafka/BigEndianReader.cs rename to src/kafka-net/Simple/SimpleBigEndianReader.cs index 4358c7f3..19fbf1d2 100644 --- a/src/SimpleKafka/BigEndianReader.cs +++ b/src/kafka-net/Simple/SimpleBigEndianReader.cs @@ -6,15 +6,15 @@ using System.Threading; using System.Threading.Tasks; -namespace SimpleKafka +namespace KafkaNet.Simple { - public class BigEndianReader + public class SimpleBigEndianReader { private readonly Stream stream; private readonly byte[] commonBuffer = new byte[8]; - public BigEndianReader(Stream stream) + public SimpleBigEndianReader(Stream stream) { this.stream = stream; } @@ -37,7 +37,13 @@ public async Task ReadInt32Async(CancellationToken token) { var buffer = commonBuffer; await ReadFullyAsync(buffer, 0, 4, token).ConfigureAwait(false); - return new BigEndianDecoder(buffer).ReadInt32(); + unchecked + { + return (buffer[0] << 24) | + (buffer[1] << 16) | + (buffer[2] << 8) | + (buffer[3]); + } } public async Task ReadBytesAsync(int numberOfBytes, CancellationToken token) diff --git a/src/SimpleKafka/BigEndianWriter.cs b/src/kafka-net/Simple/SimpleBigEndianWriter.cs similarity index 90% rename from src/SimpleKafka/BigEndianWriter.cs rename to src/kafka-net/Simple/SimpleBigEndianWriter.cs index ba3969e9..4a178afa 100644 --- a/src/SimpleKafka/BigEndianWriter.cs +++ b/src/kafka-net/Simple/SimpleBigEndianWriter.cs @@ -6,14 +6,14 @@ using System.Threading; using System.Threading.Tasks; -namespace SimpleKafka +namespace KafkaNet.Simple { - public class BigEndianWriter + public class SimpleBigEndianWriter { private readonly Stream stream; private readonly byte[] commonBuffer = new byte[8]; - public BigEndianWriter(Stream stream) + public SimpleBigEndianWriter(Stream stream) { this.stream = stream; } From 959a1dab0bd297f0ff08baf2e1faceb133b1a0c4 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Sun, 12 Apr 2015 05:35:00 +0100 Subject: [PATCH 06/14] Tidying up repository --- src/SimpleKafka/Properties/AssemblyInfo.cs | 38 +++++++++++++ src/kafka-net/Simple/SimpleBigEndianReader.cs | 57 ------------------- src/kafka-net/Simple/SimpleBigEndianWriter.cs | 42 -------------- 3 files changed, 38 insertions(+), 99 deletions(-) create mode 100644 src/SimpleKafka/Properties/AssemblyInfo.cs delete mode 100644 src/kafka-net/Simple/SimpleBigEndianReader.cs delete mode 100644 src/kafka-net/Simple/SimpleBigEndianWriter.cs diff --git a/src/SimpleKafka/Properties/AssemblyInfo.cs b/src/SimpleKafka/Properties/AssemblyInfo.cs new file mode 100644 index 00000000..40666612 --- /dev/null +++ b/src/SimpleKafka/Properties/AssemblyInfo.cs @@ -0,0 +1,38 @@ +using System.Reflection; +using System.Runtime.CompilerServices; +using System.Runtime.InteropServices; + +// General Information about an assembly is controlled through the following +// set of attributes. Change these attribute values to modify the information +// associated with an assembly. +[assembly: AssemblyTitle("SimpleKafka")] +[assembly: AssemblyDescription("")] +[assembly: AssemblyConfiguration("")] +[assembly: AssemblyCompany("")] +[assembly: AssemblyProduct("SimpleKafka")] +[assembly: AssemblyCopyright("Copyright © 2015")] +[assembly: AssemblyTrademark("")] +[assembly: AssemblyCulture("")] + +// Setting ComVisible to false makes the types in this assembly not visible +// to COM components. If you need to access a type in this assembly from +// COM, set the ComVisible attribute to true on that type. +[assembly: ComVisible(false)] + +// The following GUID is for the ID of the typelib if this project is exposed to COM +[assembly: Guid("db719bec-843a-4a5e-a151-ec47fec2ee48")] + +// Version information for an assembly consists of the following four values: +// +// Major Version +// Minor Version +// Build Number +// Revision +// +// You can specify all the values or you can default the Build and Revision Numbers +// by using the '*' as shown below: +// [assembly: AssemblyVersion("1.0.*")] +[assembly: AssemblyVersion("1.0.0.0")] +[assembly: AssemblyFileVersion("1.0.0.0")] + +[assembly: InternalsVisibleTo("SimpleKafkaTests")] diff --git a/src/kafka-net/Simple/SimpleBigEndianReader.cs b/src/kafka-net/Simple/SimpleBigEndianReader.cs deleted file mode 100644 index 19fbf1d2..00000000 --- a/src/kafka-net/Simple/SimpleBigEndianReader.cs +++ /dev/null @@ -1,57 +0,0 @@ -using System; -using System.Collections.Generic; -using System.IO; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - -namespace KafkaNet.Simple -{ - public class SimpleBigEndianReader - { - private readonly Stream stream; - private readonly byte[] commonBuffer = new byte[8]; - - - public SimpleBigEndianReader(Stream stream) - { - this.stream = stream; - } - - private async Task ReadFullyAsync(byte[] buffer, int offset, int numberOfBytes, CancellationToken token) - { - while (numberOfBytes > 0) - { - var bytesRead = await stream.ReadAsync(buffer, offset, numberOfBytes, token).ConfigureAwait(false); - if (bytesRead <= 0) - { - throw new EndOfStreamException(); - } - numberOfBytes -= bytesRead; - offset += bytesRead; - } - } - - public async Task ReadInt32Async(CancellationToken token) - { - var buffer = commonBuffer; - await ReadFullyAsync(buffer, 0, 4, token).ConfigureAwait(false); - unchecked - { - return (buffer[0] << 24) | - (buffer[1] << 16) | - (buffer[2] << 8) | - (buffer[3]); - } - } - - public async Task ReadBytesAsync(int numberOfBytes, CancellationToken token) - { - var buffer = new byte[numberOfBytes]; - await ReadFullyAsync(buffer, 0, numberOfBytes, token).ConfigureAwait(false); - return buffer; - } - - } -} diff --git a/src/kafka-net/Simple/SimpleBigEndianWriter.cs b/src/kafka-net/Simple/SimpleBigEndianWriter.cs deleted file mode 100644 index 4a178afa..00000000 --- a/src/kafka-net/Simple/SimpleBigEndianWriter.cs +++ /dev/null @@ -1,42 +0,0 @@ -using System; -using System.Collections.Generic; -using System.IO; -using System.Linq; -using System.Text; -using System.Threading; -using System.Threading.Tasks; - -namespace KafkaNet.Simple -{ - public class SimpleBigEndianWriter - { - private readonly Stream stream; - private readonly byte[] commonBuffer = new byte[8]; - - public SimpleBigEndianWriter(Stream stream) - { - this.stream = stream; - } - - public async Task WriteAsync(byte[] buffer, int offset, int count, CancellationToken token) - { - await stream.WriteAsync(buffer, offset, count, token).ConfigureAwait(false); - } - - public async Task WriteAsync(int value, CancellationToken token) - { - var buffer = commonBuffer; - unchecked - { - buffer[3] = (byte)(value & 0x0ff); - value = value >> 8; - buffer[2] = (byte)(value & 0x0ff); - value = value >> 8; - buffer[1] = (byte)(value & 0x0ff); - value = value >> 8; - buffer[0] = (byte)(value & 0x0ff); - } - await stream.WriteAsync(buffer, 0, 4, token).ConfigureAwait(false); - } - } -} From 9856153f9f9120e3f152f54eee20803c2321d3bf Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 13 Apr 2015 08:52:57 +0100 Subject: [PATCH 07/14] All functionality complete with zero error handling --- src/SimpleKafka/Extensions.cs | 15 ++ src/SimpleKafka/KafkaBrokers.cs | 26 ++ src/SimpleKafka/KafkaConsumer.cs | 162 ++++++++++++- src/SimpleKafka/KafkaMessage.cs | 9 + src/SimpleKafka/SimpleKafka.csproj | 1 + src/SimpleKafka/TopicPartitionOffset.cs | 15 ++ src/SimpleKafka/TopicSelector.cs | 12 +- .../Helpers/IntegrationConfig.cs | 4 +- .../Integration/ConsumerTests.cs | 226 +++++++++++++++++- .../Integration/IntegrationHelpers.cs | 2 +- 10 files changed, 456 insertions(+), 16 deletions(-) create mode 100644 src/SimpleKafka/TopicPartitionOffset.cs diff --git a/src/SimpleKafka/Extensions.cs b/src/SimpleKafka/Extensions.cs index 83c9d588..52fc4724 100644 --- a/src/SimpleKafka/Extensions.cs +++ b/src/SimpleKafka/Extensions.cs @@ -49,5 +49,20 @@ public static TValue TryGetValue(this Dictionary map return null; } } + + public static TValue GetOrCreate(this IDictionary map, TKey key, Func creator) + { + TValue result; + if (map.TryGetValue(key, out result)) + { + return result; + } + else + { + result = creator(); + map.Add(key, result); + return result; + } + } } } diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs index 5c0d0925..06ce7622 100644 --- a/src/SimpleKafka/KafkaBrokers.cs +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -103,6 +103,32 @@ internal void AddTopic(string topic) } } + + private readonly Dictionary offsetCoordinatorMap = new Dictionary(); + public async Task> BuildOffsetCoordinatorMapAsync(CancellationToken token, params string[] consumerGroups) + { + if (connections.Count == 0) + { + await RefreshAsync(token).ConfigureAwait(false); + } + + foreach (var consumerGroup in consumerGroups) + { + var currentCoordinator = offsetCoordinatorMap.GetOrCreate(consumerGroup, () => -1); + if (currentCoordinator == -1) + { + var request = new ConsumerMetadataRequest { ConsumerGroup = consumerGroup }; + var response = await connections.Values.First().SendRequestAsync(request, token).ConfigureAwait(false); + if (response.Error != ErrorResponseCode.NoError) + { + throw new InvalidOperationException("Failed to retrieve consumer offsets " + response.Error); + } + offsetCoordinatorMap[consumerGroup] = response.CoordinatorId; + } + } + return offsetCoordinatorMap; + } + public async Task, T>>> BuildBrokerMapAsync(CancellationToken token, Dictionary> topicMap) { if (connections.Count == 0) diff --git a/src/SimpleKafka/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs index 3504a370..8b641dea 100644 --- a/src/SimpleKafka/KafkaConsumer.cs +++ b/src/SimpleKafka/KafkaConsumer.cs @@ -20,8 +20,13 @@ public TopicTracker(TopicSelector selector) { switch (selector.DefaultOffsetSelection) { - case OffsetSelectionStrategy.Earliest: nextOffset = -2; break; - case OffsetSelectionStrategy.Latest: nextOffset = -1; break; + + case OffsetSelectionStrategy.Earliest: + case OffsetSelectionStrategy.Last: + case OffsetSelectionStrategy.Next: + case OffsetSelectionStrategy.NextUncommitted: + nextOffset = (long)selector.DefaultOffsetSelection; + break; case OffsetSelectionStrategy.Specified: nextOffset = selector.Offset; break; default: throw new InvalidOperationException("Unknown default offset selection: " + selector.DefaultOffsetSelection); } @@ -36,9 +41,11 @@ public TopicTracker(TopicSelector selector) private readonly int maxWaitTimeMs = 1000; private readonly int minBytes = 1024; private readonly int maxBytes = 65536; + private readonly string consumerGroup; - public KafkaConsumer(KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, params TopicSelector[] topics) + public KafkaConsumer(string consumerGroup, KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, params TopicSelector[] topics) { + this.consumerGroup = consumerGroup; this.brokers = brokers; this.keySerializer = keySerializer; this.valueSerializer = valueSerializer; @@ -54,9 +61,48 @@ public KafkaConsumer(KafkaBrokers brokers, IKafkaSerializer keySerializer, } } + public async Task CommitAsync(IEnumerable offsets, CancellationToken token) + { + var coordinator = await GetOffsetCoordinatorConnectionAsync(token).ConfigureAwait(false); + var offsetCommits = new List(); + foreach (var offset in offsets) { + var offsetCommit = new OffsetCommit { + Offset = offset.Offset, + PartitionId = offset.Partition, + Topic = offset.Topic + }; + offsetCommits.Add(offsetCommit); + } + var request = new OffsetCommitRequest + { + ConsumerGroup = consumerGroup, + ConsumerGroupGenerationId = 0, + ConsumerId = "test", + OffsetCommits = offsetCommits + }; + + var responses = await coordinator.SendRequestAsync(request, token).ConfigureAwait(false); + foreach (var response in responses) + { + if (response.Error != (short)ErrorResponseCode.NoError) + { + throw new InvalidOperationException("Failed to commit: " + (ErrorResponseCode)response.Error); + } + } + } + + private async Task GetOffsetCoordinatorConnectionAsync(CancellationToken token) + { + var map = await brokers.BuildOffsetCoordinatorMapAsync(token, consumerGroup).ConfigureAwait(false); + var coordinator = map[consumerGroup]; + return brokers[coordinator]; + } + public async Task>> ReceiveAsync(CancellationToken token) { - var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap); + var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap).ConfigureAwait(false); + await RetrieveAnyTopicOffsets(token, brokerMap).ConfigureAwait(false); + await RetrieveAnyConsumerOffsets(token, brokerMap).ConfigureAwait(false); var tasks = CreateFetchTasks(token, brokerMap); var taskResults = await Task.WhenAll(tasks).ConfigureAwait(false); @@ -125,10 +171,116 @@ private FetchRequest CreateRequest(Dictionary, TopicTracker> var request = new FetchRequest { MaxWaitTime = maxWaitTimeMs, - MinBytes = 1024, + MinBytes = minBytes, Fetches = fetches, }; return request; } + + private async Task RetrieveAnyTopicOffsets(CancellationToken token, Dictionary, TopicTracker>> brokerMap) + { + foreach (var brokerKvp in brokerMap) + { + List offsets = null; + var trackerMap = brokerKvp.Value; + foreach (var trackerKvp in trackerMap) + { + var tracker = trackerKvp.Value; + if ((tracker.nextOffset < 0) && (tracker.nextOffset != (long)OffsetSelectionStrategy.NextUncommitted)) + { + if (offsets == null) + { + offsets = new List(); + } + var offset = new Offset + { + MaxOffsets = 1, + PartitionId = trackerKvp.Key.Item2, + Topic = trackerKvp.Key.Item1 + }; + switch (tracker.nextOffset) + { + case (long)OffsetSelectionStrategy.Earliest: offset.Time = -2; break; + case (long)OffsetSelectionStrategy.Next: offset.Time = -1; break; + case (long)OffsetSelectionStrategy.Last: offset.Time = -1; break; + default: throw new InvalidOperationException("Unknown offset: " + tracker.nextOffset); + } + offsets.Add(offset); + } + } + + if (offsets != null) + { + var request = new OffsetRequest { Offsets = offsets }; + var responses = await brokers[brokerKvp.Key].SendRequestAsync(request, token).ConfigureAwait(false); + foreach (var response in responses) + { + if (response.Error != (short)ErrorResponseCode.NoError) + { + throw new InvalidOperationException("Unknown error fetching offsets: " + (ErrorResponseCode)response.Error); + } + var tracker = trackerMap[Tuple.Create(response.Topic, response.PartitionId)]; + switch (tracker.nextOffset) + { + case (long)OffsetSelectionStrategy.Earliest: + case (long)OffsetSelectionStrategy.Next: + tracker.nextOffset = response.Offsets[0]; + break; + + case (long)OffsetSelectionStrategy.Last: + tracker.nextOffset = response.Offsets[0] - 1; + break; + } + } + } + } + + } + + private async Task RetrieveAnyConsumerOffsets(CancellationToken token, Dictionary, TopicTracker>> brokerMap) + { + foreach (var brokerKvp in brokerMap) + { + List fetches = null; + var trackerMap = brokerKvp.Value; + foreach (var trackerKvp in trackerMap) + { + var tracker = trackerKvp.Value; + if (tracker.nextOffset == (long)OffsetSelectionStrategy.NextUncommitted) + { + if (fetches == null) + { + fetches = new List(); + } + var fetch = new OffsetFetch + { + Topic = trackerKvp.Key.Item1, + PartitionId = trackerKvp.Key.Item2, + }; + fetches.Add(fetch); + } + } + + if (fetches != null) + { + var request = new OffsetFetchRequest { ConsumerGroup = consumerGroup, Topics = fetches }; + var coordinator = await GetOffsetCoordinatorConnectionAsync(token).ConfigureAwait(false); + var responses = await coordinator.SendRequestAsync(request, token).ConfigureAwait(false); + foreach (var response in responses) + { + if (response.Error != (short)ErrorResponseCode.NoError) + { + throw new InvalidOperationException("Unknown error fetching offsets: " + (ErrorResponseCode)response.Error); + } + var tracker = trackerMap[Tuple.Create(response.Topic, response.PartitionId)]; + tracker.nextOffset = response.Offset + 1; + } + } + } + + + } + + } } diff --git a/src/SimpleKafka/KafkaMessage.cs b/src/SimpleKafka/KafkaMessage.cs index 8fa1d352..31960ebb 100644 --- a/src/SimpleKafka/KafkaMessage.cs +++ b/src/SimpleKafka/KafkaMessage.cs @@ -6,6 +6,14 @@ namespace SimpleKafka { + public static class KafkaMessage + { + public static KafkaMessage Create(string topic, TKey key, TValue value) + { + return new KafkaMessage(topic, key, value); + } + } + public class KafkaMessage { private readonly string topic; @@ -22,5 +30,6 @@ public KafkaMessage(string topic, TKey key, TValue value) this.value = value; } + } } diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index 2ce4f172..1d082430 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -79,6 +79,7 @@ + diff --git a/src/SimpleKafka/TopicPartitionOffset.cs b/src/SimpleKafka/TopicPartitionOffset.cs new file mode 100644 index 00000000..f42eb906 --- /dev/null +++ b/src/SimpleKafka/TopicPartitionOffset.cs @@ -0,0 +1,15 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class TopicPartitionOffset + { + public string Topic { get; set; } + public int Partition { get; set; } + public long Offset { get; set; } + } +} diff --git a/src/SimpleKafka/TopicSelector.cs b/src/SimpleKafka/TopicSelector.cs index 723b5805..ad41be8b 100644 --- a/src/SimpleKafka/TopicSelector.cs +++ b/src/SimpleKafka/TopicSelector.cs @@ -8,9 +8,11 @@ namespace SimpleKafka { public enum OffsetSelectionStrategy { - Earliest, - Latest, - Specified + Earliest = -2, + Last = -3, + Next = -1, + Specified = 0, + NextUncommitted = -4, }; public class TopicSelector @@ -19,14 +21,14 @@ public class TopicSelector public int Partition { get; set; } public long Offset { get; set; } - + public OffsetSelectionStrategy DefaultOffsetSelection { get; set; } public OffsetSelectionStrategy FailureOffsetSelection { get; set; } public TopicSelector() { DefaultOffsetSelection = OffsetSelectionStrategy.Specified; - FailureOffsetSelection = OffsetSelectionStrategy.Latest; + FailureOffsetSelection = OffsetSelectionStrategy.Next; } } diff --git a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs index b7e0ad23..5074f09c 100644 --- a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs +++ b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs @@ -13,10 +13,10 @@ public static class IntegrationConfig public static string IntegrationTopic = "IntegrationTopic"; public static string IntegrationConsumer = "IntegrationConsumer"; // public static Uri IntegrationUri = new Uri("http://server.home:9092"); - public static Uri IntegrationUri = new Uri("http://server.home:39092"); + public static Uri IntegrationUri = new Uri("http://server.home:39093"); public static Uri[] IntegrationUriArray = new[] { - new Uri("http://server.home:39092"), + new Uri("http://server.home:39093"), //new Uri("http://server.home:39093"), //new Uri("http://server.home:39094"), }; diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs index 1512e63a..33751d6a 100644 --- a/src/SimpleKafkaTests/Integration/ConsumerTests.cs +++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs @@ -14,11 +14,12 @@ namespace SimpleKafkaTests.Integration [Category("Integration")] class ConsumerTests { + private readonly string defaultConsumerGroup = "unit-tests"; + [SetUp] public void Setup() { - IntegrationHelpers.zookeeperHost = "zookeeper:2181"; - IntegrationHelpers.dockerOptions = "--link compose_zookeeper_1:zookeeper"; + IntegrationHelpers.zookeeperHost = "server.home:32181"; } [Test] public async Task TestSimpleConsumerWorksOk() @@ -32,7 +33,7 @@ public async Task TestSimpleConsumerWorksOk() { var topic = temporaryTopic.Topic; var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); - var consumer = new KafkaConsumer(brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); + var consumer = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); await producer.SendAsync(new KafkaMessage(topic, null, "Message"), CancellationToken.None).ConfigureAwait(true); @@ -49,5 +50,224 @@ public async Task TestSimpleConsumerWorksOk() } } + [Test] + public async Task TestProducing3MessagesAllowsTheConsumerToChooseTheCorrectMessage() + { + var keySerializer = new NullSerializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new LoadBalancedPartitioner(1); + + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + { + var topic = temporaryTopic.Topic; + { + var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + + await producer.SendAsync(new[] { + KafkaMessage.Create(topic, (string)null, "1"), + KafkaMessage.Create(topic, (string)null, "2"), + KafkaMessage.Create(topic, (string)null, "3"), + }, CancellationToken.None).ConfigureAwait(true); + } + + { + var earliest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Earliest }); + + var responses = await earliest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(3)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(0)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("1")); + } + + { + var latest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Last }); + + var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(1)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(2)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("3")); + } + + { + var latest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Next }); + + var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(0)); + + } + + { + var specified = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Specified, Offset = 1 }); + + var responses = await specified.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(2)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(1)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("2")); + } + + } + + } + + [Test] + public async Task TestProducing3MessagesAllowsTheConsumerToCommitAndRestart() + { + var keySerializer = new NullSerializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new LoadBalancedPartitioner(1); + + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + { + var topic = temporaryTopic.Topic; + { + var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + + await producer.SendAsync(new[] { + KafkaMessage.Create(topic, (string)null, "1"), + KafkaMessage.Create(topic, (string)null, "2"), + KafkaMessage.Create(topic, (string)null, "3"), + }, CancellationToken.None).ConfigureAwait(true); + } + + { + var noPreviousCommits = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic, + DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, + FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); + + var responses = await noPreviousCommits.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(3)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(0)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("1")); + + await noPreviousCommits.CommitAsync(new[] { + new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 0 } + }, CancellationToken.None).ConfigureAwait(true); ; + } + + { + var previousCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector + { + Partition = 0, + Topic = topic, + DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, + FailureOffsetSelection = OffsetSelectionStrategy.Earliest + }); + + var responses = await previousCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(2)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(1)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("2")); + + } + + { + var previousCommitAgain = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector + { + Partition = 0, + Topic = topic, + DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, + FailureOffsetSelection = OffsetSelectionStrategy.Earliest + }); + + var responses = await previousCommitAgain.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(2)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(1)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("2")); + + await previousCommitAgain.CommitAsync(new[] { + new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 1 } + }, CancellationToken.None).ConfigureAwait(true); ; + } + + { + var secondCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector + { + Partition = 0, + Topic = topic, + DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, + FailureOffsetSelection = OffsetSelectionStrategy.Earliest + }); + + var responses = await secondCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(1)); + + var first = responses.First(); + Assert.That(first.Key, Is.Null); + Assert.That(first.Offset, Is.EqualTo(2)); + Assert.That(first.Partition, Is.EqualTo(0)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("3")); + + await secondCommit.CommitAsync(new[] { + new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 2 } + }, CancellationToken.None).ConfigureAwait(true); ; + } + + { + var thirdCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector + { + Partition = 0, + Topic = topic, + DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, + FailureOffsetSelection = OffsetSelectionStrategy.Earliest + }); + + var responses = await thirdCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(0)); + + } + + } + } } } diff --git a/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs b/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs index 34c48de1..b7a22e12 100644 --- a/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs +++ b/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs @@ -12,7 +12,7 @@ internal static class IntegrationHelpers { public static string dockerHost = "tcp://server.home:2375"; - public static string zookeeperHost = "server.home"; + public static string zookeeperHost = "server.home:32181"; public static string kafkaImage = "sceneskope/kafka:0.8.2.1"; public static string dockerOptions = ""; From 69dafc5da2b26ceefb17996b9ab649e36a11795f Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 13 Apr 2015 11:55:45 +0100 Subject: [PATCH 08/14] Reducing allocations to decrease load on client systems --- .../Common/StringPrefixEncoding.cs | 15 -- src/SimpleKafka/Extensions.cs | 4 +- src/SimpleKafka/Interfaces/IKafkaRequest.cs | 18 +-- src/SimpleKafka/KafkaBrokers.cs | 18 +-- src/SimpleKafka/KafkaConnection.cs | 11 +- src/SimpleKafka/KafkaConsumer.cs | 16 +- src/SimpleKafka/KafkaDecoder.cs | 27 +--- src/SimpleKafka/KafkaEncoder.cs | 81 ++++------ src/SimpleKafka/KafkaProducer.cs | 8 +- src/SimpleKafka/Protocol/BaseRequest.cs | 30 +++- src/SimpleKafka/Protocol/Broker.cs | 22 +-- .../Protocol/ConsumerMetadataRequest.cs | 45 ++++-- src/SimpleKafka/Protocol/FetchRequest.cs | 145 ++++++++++------- src/SimpleKafka/Protocol/Message.cs | 44 ++++-- src/SimpleKafka/Protocol/MetadataRequest.cs | 87 ++++++----- .../Protocol/OffsetCommitRequest.cs | 126 ++++++++++----- .../Protocol/OffsetFetchRequest.cs | 108 ++++++++----- src/SimpleKafka/Protocol/OffsetRequest.cs | 146 +++++++++--------- src/SimpleKafka/Protocol/Partition.cs | 83 ++++++++++ src/SimpleKafka/Protocol/ProduceRequest.cs | 144 ++++++++++------- src/SimpleKafka/Protocol/Topic.cs | 96 ++---------- src/SimpleKafka/SimpleKafka.csproj | 2 +- src/SimpleKafkaTests/Helpers/MessageHelper.cs | 2 +- .../Integration/SimpleTests.cs | 28 ++-- src/SimpleKafkaTests/SimpleKafkaTests.csproj | 4 +- ...anDecoderTests.cs => KafkaDecoderTests.cs} | 28 +--- ...anEncoderTests.cs => KafkaEncoderTests.cs} | 23 +-- .../Unit/ProtocolBaseRequestTests.cs | 2 +- .../Unit/ProtocolMessageTests.cs | 16 +- 29 files changed, 753 insertions(+), 626 deletions(-) delete mode 100644 src/SimpleKafka/Common/StringPrefixEncoding.cs create mode 100644 src/SimpleKafka/Protocol/Partition.cs rename src/SimpleKafkaTests/Unit/{BigEndianDecoderTests.cs => KafkaDecoderTests.cs} (64%) rename src/SimpleKafkaTests/Unit/{BigEndianEncoderTests.cs => KafkaEncoderTests.cs} (73%) diff --git a/src/SimpleKafka/Common/StringPrefixEncoding.cs b/src/SimpleKafka/Common/StringPrefixEncoding.cs deleted file mode 100644 index c881e920..00000000 --- a/src/SimpleKafka/Common/StringPrefixEncoding.cs +++ /dev/null @@ -1,15 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Linq; -using System.Text; -using System.Threading.Tasks; - -namespace SimpleKafka.Common -{ - public enum StringPrefixEncoding - { - Int16, - Int32, - None - } -} diff --git a/src/SimpleKafka/Extensions.cs b/src/SimpleKafka/Extensions.cs index 52fc4724..b8aaea2d 100644 --- a/src/SimpleKafka/Extensions.cs +++ b/src/SimpleKafka/Extensions.cs @@ -24,7 +24,7 @@ public static async Task ReadFullyAsync(this Stream stream, byte[] buffer, int o } } - public static TValue FindOrCreate(this Dictionary map, TKey key) + public static TValue GetOrCreate(this IDictionary map, TKey key) where TValue : new() { TValue result; @@ -36,7 +36,7 @@ public static TValue FindOrCreate(this Dictionary map, return result; } - public static TValue TryGetValue(this Dictionary map, TKey key) + public static TValue TryGetValue(this IDictionary map, TKey key) where TValue : class { TValue result; diff --git a/src/SimpleKafka/Interfaces/IKafkaRequest.cs b/src/SimpleKafka/Interfaces/IKafkaRequest.cs index ad608f34..cd0c9318 100644 --- a/src/SimpleKafka/Interfaces/IKafkaRequest.cs +++ b/src/SimpleKafka/Interfaces/IKafkaRequest.cs @@ -7,8 +7,7 @@ namespace SimpleKafka /// KafkaRequest represents a Kafka request messages as an object which can Encode itself into the appropriate /// binary request and Decode any responses to that request. /// - /// The type of the KafkaResponse expected back from the request. - public interface IKafkaRequest + public interface IKafkaRequest { /// /// Indicates this request should wait for a response from the broker @@ -26,20 +25,5 @@ public interface IKafkaRequest /// Id which will be echoed back by Kafka to correlate responses to this request. Usually automatically assigned by driver. /// int CorrelationId { get; set; } - /// - /// Enum identifying the specific type of request message being represented. - /// - ApiKeyRequestType ApiKey { get; } - /// - /// Encode this request into the Kafka wire protocol. - /// - /// Encoder to use - void Encode(ref KafkaEncoder encoder); - /// - /// Decode a response payload from Kafka into T. - /// - /// Decoder to use - /// Response - T Decode(ref KafkaDecoder decoder); } } \ No newline at end of file diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs index 06ce7622..dcc4f592 100644 --- a/src/SimpleKafka/KafkaBrokers.cs +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -202,7 +202,7 @@ public async Task, T>>> BuildBroke { var partitionNumber = partitionKvp.Key; var partition = partitions[partitionNumber]; - var brokerTopics = brokerMap.FindOrCreate(partition.LeaderId); + var brokerTopics = brokerMap.GetOrCreate(partition.LeaderId); brokerTopics.Add(Tuple.Create(topic, partitionNumber), partitionKvp.Value); } } @@ -254,31 +254,31 @@ private async Task TryToRefreshFromConnectionAsync(KafkaConnection connect } - private void RefreshTopics(List topics) + private void RefreshTopics(Topic[] topics) { var previousTopics = new HashSet(topicToPartitions.Keys); foreach (var topic in topics) { - if (topic.ErrorCode != (short)ErrorResponseCode.NoError) + if (topic.ErrorCode != ErrorResponseCode.NoError) { - Log.Information("Topic {topic} has error {error}", topic.Name, (ErrorResponseCode)topic.ErrorCode); + Log.Information("Topic {topic} has error {error}", topic.Name, topic.ErrorCode); } else { var currentPartitions = topicToPartitions.TryGetValue(topic.Name); - if ((currentPartitions == null) || (currentPartitions.Length != topic.Partitions.Count)) + if ((currentPartitions == null) || (currentPartitions.Length != topic.Partitions.Length)) { - currentPartitions = new Partition[topic.Partitions.Count]; + currentPartitions = new Partition[topic.Partitions.Length]; topicToPartitions[topic.Name] = currentPartitions; } foreach (var partition in topic.Partitions) { - if (partition.ErrorCode != (short)ErrorResponseCode.NoError) + if (partition.ErrorCode != ErrorResponseCode.NoError) { - Log.Verbose("Topic {topic} partition {partition} has error {error}", topic.Name, partition.PartitionId, (ErrorResponseCode)partition.ErrorCode); + Log.Verbose("Topic {topic} partition {partition} has error {error}", topic.Name, partition.PartitionId, partition.ErrorCode); } currentPartitions[partition.PartitionId] = partition; } @@ -293,7 +293,7 @@ private void RefreshTopics(List topics) } } - private async Task RefreshBrokersAsync(List latestBrokers, CancellationToken token) + private async Task RefreshBrokersAsync(Broker[] latestBrokers, CancellationToken token) { var previousBrokers = new HashSet(brokers); var previousConnections = connections.ToDictionary(kvp => kvp.Key, kvp => kvp.Value); diff --git a/src/SimpleKafka/KafkaConnection.cs b/src/SimpleKafka/KafkaConnection.cs index ca086ddf..f9347615 100644 --- a/src/SimpleKafka/KafkaConnection.cs +++ b/src/SimpleKafka/KafkaConnection.cs @@ -1,4 +1,5 @@ -using System; +using SimpleKafka.Protocol; +using System; using System.Linq; using System.Net; using System.Net.Sockets; @@ -47,21 +48,21 @@ private async Task ReceiveResponseAsync(CancellationToken token) return length; } - public async Task SendRequestAsync(IKafkaRequest request, CancellationToken token) + public async Task SendRequestAsync(BaseRequest request, CancellationToken token) { await clientLock.WaitAsync(token).ConfigureAwait(false); try { encoder.Reset(); var marker = encoder.PrepareForLength(); - request.Encode(ref encoder); - encoder.WriteLength(marker); + request.Encode(encoder) + .WriteLength(marker); await stream.WriteAsync(buffer, 0, encoder.Offset, token).ConfigureAwait(false); if (request.ExpectResponse) { var length = await ReceiveResponseAsync(token).ConfigureAwait(false); - var result = request.Decode(ref decoder); + var result = request.Decode(decoder); return result; } else diff --git a/src/SimpleKafka/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs index 8b641dea..e06c2a0c 100644 --- a/src/SimpleKafka/KafkaConsumer.cs +++ b/src/SimpleKafka/KafkaConsumer.cs @@ -52,7 +52,7 @@ public KafkaConsumer(string consumerGroup, KafkaBrokers brokers, IKafkaSerialize topicMap = new Dictionary>(); foreach (var topic in topics) { - var partitionMap = topicMap.FindOrCreate(topic.Topic); + var partitionMap = topicMap.GetOrCreate(topic.Topic); if (partitionMap.ContainsKey(topic.Partition)) { throw new InvalidOperationException("Topic " + topic.Topic + ", partition " + topic.Partition + " duplicated"); @@ -84,9 +84,9 @@ public async Task CommitAsync(IEnumerable offsets, Cancell var responses = await coordinator.SendRequestAsync(request, token).ConfigureAwait(false); foreach (var response in responses) { - if (response.Error != (short)ErrorResponseCode.NoError) + if (response.Error != ErrorResponseCode.NoError) { - throw new InvalidOperationException("Failed to commit: " + (ErrorResponseCode)response.Error); + throw new InvalidOperationException("Failed to commit: " + response.Error); } } } @@ -113,7 +113,7 @@ public async Task>> ReceiveAsync(Cancella { if (fetchResponse.Error != (int)ErrorResponseCode.NoError) { - Log.Error("Error in fetch response {error} for {topic}/{partition}", (ErrorResponseCode)fetchResponse.Error, fetchResponse.Topic, fetchResponse.PartitionId); + Log.Error("Error in fetch response {error} for {topic}/{partition}", fetchResponse.Error, fetchResponse.Topic, fetchResponse.PartitionId); } else { var tracker = topicMap[fetchResponse.Topic][fetchResponse.PartitionId]; @@ -215,9 +215,9 @@ private async Task RetrieveAnyTopicOffsets(CancellationToken token, Dictionary> 8)); buffer[offset++] = (byte)(value); } + return this; } - public void Write(int value) + public KafkaEncoder Write(int value) { unchecked { @@ -54,9 +57,10 @@ public void Write(int value) buffer[offset++] = (byte)((value >> 8)); buffer[offset++] = (byte)(value); } + return this; } - public void Write(uint value) + public KafkaEncoder Write(uint value) { unchecked { @@ -65,106 +69,85 @@ public void Write(uint value) buffer[offset++] = (byte)((value >> 8)); buffer[offset++] = (byte)(value); } + return this; } - public void Write(short value) + public KafkaEncoder Write(short value) { unchecked { buffer[offset++] = (byte)((value >> 8)); buffer[offset++] = (byte)(value); } + return this; } - public void Write(byte value) + public KafkaEncoder Write(byte value) { buffer[offset++] = value; + return this; } - public void Write(string data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + public KafkaEncoder Write(string data) { if (data == null) { - switch (encoding) - { - case StringPrefixEncoding.None: break; - case StringPrefixEncoding.Int16: Write((short)-1); break; - case StringPrefixEncoding.Int32: Write(-1); break; - default: throw new InvalidOperationException("Unknown encoding: " + encoding); - } - - } else + Write((short)-1); + } + else { - int adjust; - switch (encoding) - { - case StringPrefixEncoding.None: adjust = 0; break; - case StringPrefixEncoding.Int16: adjust = 2; break; - case StringPrefixEncoding.Int32: adjust = 4; break; - default: throw new InvalidOperationException("Unknown encoding: " + encoding); - } - var bytesWritten = Encoding.UTF8.GetBytes(data, 0, data.Length, buffer, offset + adjust); - switch (encoding) - { - case StringPrefixEncoding.None: break; - case StringPrefixEncoding.Int16: Write((short)bytesWritten); break; - case StringPrefixEncoding.Int32: Write(bytesWritten); break; - } + var bytesWritten = Encoding.UTF8.GetBytes(data, 0, data.Length, buffer, offset + 2); + Write((short)bytesWritten); offset += bytesWritten; } + return this; } - public void Write(byte[] data, StringPrefixEncoding encoding = StringPrefixEncoding.Int32) + public KafkaEncoder Write(byte[] data) { if (data == null) { - switch (encoding) - { - case StringPrefixEncoding.Int16: Write((short)-1); break; - default: Write(-1); break; - } - + Write(-1); } else { - switch (encoding) - { - case StringPrefixEncoding.Int16: Write((short)data.Length); break; - default: Write(data.Length); break; - } + Write(data.Length); Array.Copy(data, 0, buffer, offset, data.Length); offset += data.Length; } + return this; } - internal int PrepareForCrc() + public int PrepareForCrc() { offset += 4; return offset; } - internal void CalculateCrc(int crcMarker) + public KafkaEncoder CalculateCrc(int crcMarker) { var crc = Crc32Provider.Compute(buffer, crcMarker, offset - crcMarker); var current = offset; offset = crcMarker - 4; Write(crc); offset = current; + return this; } - internal int PrepareForLength() + public int PrepareForLength() { offset += 4; return offset; } - internal void WriteLength(int lengthMarker) + public KafkaEncoder WriteLength(int lengthMarker) { var current = offset; var length = offset - lengthMarker; offset = lengthMarker - 4; Write(length); offset = current; + return this; } } diff --git a/src/SimpleKafka/KafkaProducer.cs b/src/SimpleKafka/KafkaProducer.cs index 05d5400a..8678fb72 100644 --- a/src/SimpleKafka/KafkaProducer.cs +++ b/src/SimpleKafka/KafkaProducer.cs @@ -73,7 +73,7 @@ private async Task SendMessagesToBrokersAsync(CancellationToken token, Dic var responses = await ProduceMessagesToBroker(brokerKvp.Key, brokerKvp.Value, token).ConfigureAwait(false); foreach (var response in responses) { - switch ((ErrorResponseCode)response.Error) + switch (response.Error) { case ErrorResponseCode.NoError: var partitions = topicMap[response.Topic]; @@ -90,7 +90,7 @@ private async Task SendMessagesToBrokersAsync(CancellationToken token, Dic break; default: - throw new InvalidOperationException("Unhandled error " + (ErrorResponseCode)response.Error + ", " + response.Topic + ":" + response.PartitionId); + throw new InvalidOperationException("Unhandled error " + response.Error + ", " + response.Topic + ":" + response.PartitionId); } } } @@ -127,9 +127,9 @@ private Dictionary>> BuildTopicMap(IEnumer var topicMap = new Dictionary>>(); foreach (var message in messages) { - var partitionMap = topicMap.FindOrCreate(message.Topic); + var partitionMap = topicMap.GetOrCreate(message.Topic); var partition = messagePartitioner.CalculatePartition(message); - var messageList = partitionMap.FindOrCreate(partition); + var messageList = partitionMap.GetOrCreate(partition); var encodedMessage = new Message { Key = keySerializer.Serialize(message.Key), diff --git a/src/SimpleKafka/Protocol/BaseRequest.cs b/src/SimpleKafka/Protocol/BaseRequest.cs index 5eb913ef..47643764 100644 --- a/src/SimpleKafka/Protocol/BaseRequest.cs +++ b/src/SimpleKafka/Protocol/BaseRequest.cs @@ -3,7 +3,7 @@ namespace SimpleKafka.Protocol { - public abstract class BaseRequest + public abstract class BaseRequest { /// /// From Documentation: @@ -17,9 +17,11 @@ public abstract class BaseRequest private readonly short _apiVersion; private string _clientId = "Kafka-Net"; private int _correlationId = 1; + private readonly ApiKeyRequestType apiKey; - protected BaseRequest(short apiVersion = 0) + protected BaseRequest(ApiKeyRequestType apiKey, short apiVersion = 0) { + this.apiKey = apiKey; _apiVersion = apiVersion; } @@ -44,12 +46,26 @@ protected BaseRequest(short apiVersion = 0) /// public virtual bool ExpectResponse { get { return true; } } - internal static void EncodeHeader(IKafkaRequest request, ref KafkaEncoder encoder) + /// + /// Encode this request into the Kafka wire protocol. + /// + /// Encoder to use + internal abstract KafkaEncoder Encode(KafkaEncoder encoder); + + /// + /// Decode a response payload from Kafka into T. + /// + /// Decoder to use + /// Response + internal abstract T Decode(KafkaDecoder decoder); + + internal KafkaEncoder EncodeHeader(KafkaEncoder encoder) { - encoder.Write((Int16)request.ApiKey); - encoder.Write(request.ApiVersion); - encoder.Write(request.CorrelationId); - encoder.Write(request.ClientId, StringPrefixEncoding.Int16); + return encoder + .Write((Int16)apiKey) + .Write(ApiVersion) + .Write(CorrelationId) + .Write(ClientId); } } } \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Broker.cs b/src/SimpleKafka/Protocol/Broker.cs index 08bdd73d..fb9d00b7 100644 --- a/src/SimpleKafka/Protocol/Broker.cs +++ b/src/SimpleKafka/Protocol/Broker.cs @@ -5,19 +5,21 @@ namespace SimpleKafka.Protocol { public class Broker { - public int BrokerId { get; set; } - public string Host { get; set; } - public int Port { get; set; } + public readonly int BrokerId; + public readonly string Host; + public readonly int Port; public Uri Address { get { return new Uri(string.Format("http://{0}:{1}", Host, Port));} } - public static Broker Decode(ref KafkaDecoder decoder) + private Broker(int brokerId, string host, int port) { - return new Broker - { - BrokerId = decoder.ReadInt32(), - Host = decoder.ReadInt16String(), - Port = decoder.ReadInt32() - }; + this.BrokerId = brokerId; + this.Host = host; + this.Port = port; + } + + internal static Broker Decode(KafkaDecoder decoder) + { + return new Broker(decoder.ReadInt32(), decoder.ReadString(), decoder.ReadInt32()); } } } diff --git a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs index 2152ba1c..eca5f4de 100644 --- a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs +++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs @@ -9,32 +9,35 @@ namespace SimpleKafka.Protocol /// The offsets for a given consumer group is maintained by a specific broker called the offset coordinator. i.e., a consumer needs /// to issue its offset commit and fetch requests to this specific broker. It can discover the current offset coordinator by issuing a consumer metadata request. /// - public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest + public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest { - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.ConsumerMetadataRequest; } } public string ConsumerGroup { get; set; } - public void Encode(ref KafkaEncoder encoder) + public ConsumerMetadataRequest() : base(ApiKeyRequestType.ConsumerMetadataRequest) { } + + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeConsumerMetadataRequest(this, ref encoder); + return EncodeConsumerMetadataRequest(this, encoder); } - public ConsumerMetadataResponse Decode(ref KafkaDecoder decoder) + internal override ConsumerMetadataResponse Decode(KafkaDecoder decoder) { - return DecodeConsumerMetadataResponse(ref decoder); + return DecodeConsumerMetadataResponse(decoder); } - private static void EncodeConsumerMetadataRequest(ConsumerMetadataRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeConsumerMetadataRequest(ConsumerMetadataRequest request, KafkaEncoder encoder) { - EncodeHeader(request, ref encoder); - encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); + return + request + .EncodeHeader(encoder) + .Write(request.ConsumerGroup); } - private static ConsumerMetadataResponse DecodeConsumerMetadataResponse(ref KafkaDecoder decoder) + private static ConsumerMetadataResponse DecodeConsumerMetadataResponse(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); - return new ConsumerMetadataResponse(ref decoder); + return ConsumerMetadataResponse.Decode(decoder); } } @@ -49,12 +52,22 @@ public class ConsumerMetadataResponse public readonly string CoordinatorHost; public readonly int CoordinatorPort; - internal ConsumerMetadataResponse(ref KafkaDecoder decoder) + private ConsumerMetadataResponse(ErrorResponseCode error, int coordinatorId, string coordinatorHost, int coordinatorPort) { - Error = (ErrorResponseCode)decoder.ReadInt16(); - CoordinatorId = decoder.ReadInt32(); - CoordinatorHost = decoder.ReadInt16String(); - CoordinatorPort = decoder.ReadInt32(); + this.Error = error; + this.CoordinatorId = coordinatorId; + this.CoordinatorHost = coordinatorHost; + this.CoordinatorPort = coordinatorPort; + } + + internal static ConsumerMetadataResponse Decode(KafkaDecoder decoder) + { + var error = decoder.ReadErrorResponseCode(); + var coordinatorId = decoder.ReadInt32(); + var coordinatorHost = decoder.ReadString(); + var coordinatorPort = decoder.ReadInt32(); + + return new ConsumerMetadataResponse(error, coordinatorId, coordinatorHost, coordinatorPort); } } } diff --git a/src/SimpleKafka/Protocol/FetchRequest.cs b/src/SimpleKafka/Protocol/FetchRequest.cs index 10704d90..47509684 100644 --- a/src/SimpleKafka/Protocol/FetchRequest.cs +++ b/src/SimpleKafka/Protocol/FetchRequest.cs @@ -5,7 +5,7 @@ namespace SimpleKafka.Protocol { - public class FetchRequest : BaseRequest, IKafkaRequest> + public class FetchRequest : BaseRequest>, IKafkaRequest { internal const int DefaultMinBlockingByteBufferSize = 4096; internal const int DefaultBufferSize = DefaultMinBlockingByteBufferSize * 8; @@ -30,46 +30,76 @@ public class FetchRequest : BaseRequest, IKafkaRequest> public List Fetches { get; set; } - public void Encode(ref KafkaEncoder encoder) + public FetchRequest() : base(ApiKeyRequestType.Fetch) { } + + + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeFetchRequest(this, ref encoder); + return EncodeFetchRequest(this, encoder); } - public List Decode(ref KafkaDecoder decoder) + internal override List Decode(KafkaDecoder decoder) { - return DecodeFetchResponses(ref decoder); + return DecodeFetchResponses(decoder); } - private static void EncodeFetchRequest(FetchRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeFetchRequest(FetchRequest request, KafkaEncoder encoder) { - if (request.Fetches == null) request.Fetches = new List(); - EncodeHeader(request, ref encoder); + request + .EncodeHeader(encoder) + .Write(ReplicaId) + .Write(request.MaxWaitTime) + .Write(request.MinBytes); - var topicGroups = request.Fetches.GroupBy(x => x.Topic).ToList(); - encoder.Write(ReplicaId); - encoder.Write(request.MaxWaitTime); - encoder.Write(request.MinBytes); - encoder.Write(topicGroups.Count); - - foreach (var topicGroup in topicGroups) + if (request.Fetches == null) + { + // no topics + encoder.Write(0); + } + else if (request.Fetches.Count == 1) + { + // single topic/partition - quick mode + var fetch = request.Fetches[0]; + encoder + .Write(1) + .Write(fetch.Topic) + .Write(1); + + EncodeFetch(encoder, fetch); + } + else { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); - encoder.Write(partitions.Count); + // Multiple topics/partitions - slower mode + var topicGroups = new Dictionary>(); + foreach (var fetch in request.Fetches) { + var fetchList = topicGroups.GetOrCreate(fetch.Topic, () => new List(request.Fetches.Count)); + fetchList.Add(fetch); + } - foreach (var partition in partitions) - { - foreach (var fetch in partition) - { - encoder.Write(partition.Key); - encoder.Write(fetch.Offset); - encoder.Write(fetch.MaxBytes); + encoder.Write(topicGroups.Count); + foreach (var topicGroupKvp in topicGroups) { + var topicGroup = topicGroupKvp.Key; + var fetches = topicGroupKvp.Value; + encoder + .Write(topicGroup) + .Write(fetches.Count); + foreach (var fetch in fetches) { + EncodeFetch(encoder, fetch); } } } + return encoder; + } + + private static void EncodeFetch(KafkaEncoder encoder, Fetch fetch) + { + encoder + .Write(fetch.PartitionId) + .Write(fetch.Offset) + .Write(fetch.MaxBytes); } - private List DecodeFetchResponses(ref KafkaDecoder decoder) + private List DecodeFetchResponses(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); @@ -78,26 +108,13 @@ private List DecodeFetchResponses(ref KafkaDecoder decoder) var topicCount = decoder.ReadInt32(); for (int i = 0; i < topicCount; i++) { - var topic = decoder.ReadInt16String(); + var topic = decoder.ReadString(); var partitionCount = decoder.ReadInt32(); for (int j = 0; j < partitionCount; j++) { - var partitionId = decoder.ReadInt32(); - var response = new FetchResponse - { - Topic = topic, - PartitionId = partitionId, - Error = decoder.ReadInt16(), - HighWaterMark = decoder.ReadInt64(), - }; - var messageSetSize = decoder.ReadInt32(); - var current = decoder.Offset; - response.Messages = Message.DecodeMessageSet(partitionId, ref decoder, messageSetSize); + var response = FetchResponse.Decode(decoder, topic); result.Add(response); - - // In case any truncated messages - decoder.SetOffset(current + messageSetSize); } } return result; @@ -132,28 +149,36 @@ public Fetch() public class FetchResponse { - /// - /// The name of the topic this response entry is for. - /// - public string Topic { get; set; } - /// - /// The id of the partition this response is for. - /// - public int PartitionId { get; set; } - /// - /// Error code of exception that occured during the request. Zero if no error. - /// - public Int16 Error { get; set; } - /// - /// The offset at the end of the log for this partition. This can be used by the client to determine how many messages behind the end of the log they are. - /// - public long HighWaterMark { get; set; } + public readonly string Topic; + public readonly int PartitionId; + public readonly ErrorResponseCode Error; + public readonly long HighWaterMark; + public readonly IList Messages; - public List Messages { get; set; } + private FetchResponse(string topic, int partitionId, ErrorResponseCode error, long highWaterMark, IList messages) + { + this.Topic = topic; + this.PartitionId = partitionId; + this.Error = error; + this.HighWaterMark = highWaterMark; + this.Messages = messages; + } - public FetchResponse() + internal static FetchResponse Decode(KafkaDecoder decoder, string topic) { - Messages = new List(); + var partitionId = decoder.ReadInt32(); + var error = decoder.ReadErrorResponseCode(); + var highWaterMark = decoder.ReadInt64(); + + var messageSetSize = decoder.ReadInt32(); + var current = decoder.Offset; + var messages = Message.DecodeMessageSet(partitionId, decoder, messageSetSize); + var response = new FetchResponse(topic, partitionId, error, highWaterMark, messages); + + // In case any truncated messages + decoder.SetOffset(current + messageSetSize); + + return response; } } } \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/Message.cs b/src/SimpleKafka/Protocol/Message.cs index b3121d88..1a99572e 100644 --- a/src/SimpleKafka/Protocol/Message.cs +++ b/src/SimpleKafka/Protocol/Message.cs @@ -72,15 +72,16 @@ public Message(string value, string key = null) /// /// The collection of messages to encode together. /// Encoded byte[] representing the collection of messages. - public static void EncodeMessageSet(ref KafkaEncoder encoder, IEnumerable messages) + internal static KafkaEncoder EncodeMessageSet(KafkaEncoder encoder, IEnumerable messages) { foreach (var message in messages) { encoder.Write(InitialMessageOffset); var marker = encoder.PrepareForLength(); - EncodeMessage(message, ref encoder); - encoder.WriteLength(marker); + EncodeMessage(message, encoder) + .WriteLength(marker); } + return encoder; } /// @@ -88,7 +89,7 @@ public static void EncodeMessageSet(ref KafkaEncoder encoder, IEnumerable /// The decoder positioned at the start of the buffer /// The messages - public static List DecodeMessageSet(int partitionId, ref KafkaDecoder decoder, int messageSetSize) + internal static List DecodeMessageSet(int partitionId, KafkaDecoder decoder, int messageSetSize) { var numberOfBytes = messageSetSize; @@ -114,7 +115,7 @@ public static List DecodeMessageSet(int partitionId, ref KafkaDecoder d break; } - var message = DecodeMessage(offset, partitionId, ref decoder, messageSize); + var message = DecodeMessage(offset, partitionId, decoder, messageSize); messages.Add(message); numberOfBytes -= messageSize; } @@ -130,14 +131,17 @@ public static List DecodeMessageSet(int partitionId, ref KafkaDecoder d /// Format: /// Crc (Int32), MagicByte (Byte), Attribute (Byte), Key (Byte[]), Value (Byte[]) /// - public static void EncodeMessage(Message message, ref KafkaEncoder encoder) + internal static KafkaEncoder EncodeMessage(Message message, KafkaEncoder encoder) { var marker = encoder.PrepareForCrc(); - encoder.Write(message.MagicNumber); - encoder.Write(message.Attribute); - encoder.Write(message.Key); - encoder.Write(message.Value); - encoder.CalculateCrc(marker); + encoder + .Write(message.MagicNumber) + .Write(message.Attribute) + .Write(message.Key) + .Write(message.Value) + .CalculateCrc(marker); + + return encoder; } /// @@ -147,7 +151,7 @@ public static void EncodeMessage(Message message, ref KafkaEncoder encoder) /// The byte[] encode as a message from kafka. /// The message /// The return type is an Enumerable as the message could be a compressed message set. - public static Message DecodeMessage(long offset, int partitionId, ref KafkaDecoder decoder, int messageSize) + internal static Message DecodeMessage(long offset, int partitionId, KafkaDecoder decoder, int messageSize) { var crc = decoder.ReadUInt32(); var calculatedCrc = Crc32Provider.Compute(decoder.Buffer, decoder.Offset, messageSize - 4); @@ -158,17 +162,17 @@ public static Message DecodeMessage(long offset, int partitionId, ref KafkaDecod var message = new Message { - Meta = new MessageMetadata { Offset = offset, PartitionId = partitionId }, + Meta = new MessageMetadata(offset, partitionId), MagicNumber = decoder.ReadByte(), Attribute = decoder.ReadByte(), - Key = decoder.ReadIntPrefixedBytes(), + Key = decoder.ReadBytes(), }; var codec = (MessageCodec)(ProtocolConstants.AttributeCodeMask & message.Attribute); switch (codec) { case MessageCodec.CodecNone: - message.Value = decoder.ReadIntPrefixedBytes(); + message.Value = decoder.ReadBytes(); break; default: @@ -190,10 +194,16 @@ public class MessageMetadata /// /// The log offset of this message as stored by the Kafka server. /// - public long Offset { get; set; } + public readonly long Offset; /// /// The partition id this offset is from. /// - public int PartitionId { get; set; } + public readonly int PartitionId; + + public MessageMetadata(long offset, int partitionId) + { + this.PartitionId = partitionId; + this.Offset = offset; + } } } diff --git a/src/SimpleKafka/Protocol/MetadataRequest.cs b/src/SimpleKafka/Protocol/MetadataRequest.cs index 99bf3bcd..fa8a201d 100644 --- a/src/SimpleKafka/Protocol/MetadataRequest.cs +++ b/src/SimpleKafka/Protocol/MetadataRequest.cs @@ -3,77 +3,84 @@ namespace SimpleKafka.Protocol { - public class MetadataRequest : BaseRequest, IKafkaRequest + public class MetadataRequest : BaseRequest, IKafkaRequest { - /// - /// Indicates the type of kafka encoding this request is - /// - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.MetaData; } } - /// /// The list of topics to get metadata for. /// public List Topics { get; set; } - public void Encode(ref KafkaEncoder encoder) + public MetadataRequest() : base(ApiKeyRequestType.MetaData) { } + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeMetadataRequest(this, ref encoder); + return EncodeMetadataRequest(this, encoder); } - public MetadataResponse Decode(ref KafkaDecoder decoder) + internal override MetadataResponse Decode(KafkaDecoder decoder) { - return DecodeMetadataResponse(ref decoder); + return DecodeMetadataResponse(decoder); } - private static void EncodeMetadataRequest(MetadataRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeMetadataRequest(MetadataRequest request, KafkaEncoder encoder) { - if (request.Topics == null) request.Topics = new List(); - EncodeHeader(request, ref encoder); - encoder.Write(request.Topics.Count); - foreach (var topic in request.Topics) + request + .EncodeHeader(encoder); + + if (request.Topics == null) { - encoder.Write(topic, StringPrefixEncoding.Int16); + encoder.Write(0); } + else + { + encoder.Write(request.Topics.Count); + foreach (var topic in request.Topics) + { + encoder.Write(topic); + } + } + + return encoder; } - private static MetadataResponse DecodeMetadataResponse(ref KafkaDecoder decoder) + private static MetadataResponse DecodeMetadataResponse(KafkaDecoder decoder) { - var response = new MetadataResponse - { - CorrelationId = decoder.ReadInt32(), - }; + var correlationId = decoder.ReadInt32(); + var response = MetadataResponse.Decode(decoder); + return response; + } + + } + + public class MetadataResponse + { + public readonly Broker[] Brokers; + public readonly Topic[] Topics; + private MetadataResponse(Broker[] brokers, Topic[] topics) + { + this.Brokers = brokers; + this.Topics = topics; + } + internal static MetadataResponse Decode(KafkaDecoder decoder) + { var brokerCount = decoder.ReadInt32(); - var brokers = new List(brokerCount); + var brokers = new Broker[brokerCount]; for (var i = 0; i < brokerCount; i++) { - brokers.Add(Broker.Decode(ref decoder)); + brokers[i] = Broker.Decode(decoder); } - response.Brokers = brokers; var topicCount = decoder.ReadInt32(); - var topics = new List(topicCount); + var topics = new Topic[topicCount]; for (var i = 0; i < topicCount; i++) { - topics.Add(Topic.Decode(ref decoder)); + topics[i] = Topic.Decode(decoder); } - response.Topics = topics; - return response; - } + var response = new MetadataResponse(brokers, topics); - } + return response; - public class MetadataResponse - { - public int CorrelationId { get; set; } - public MetadataResponse() - { - Brokers = new List(); - Topics = new List(); } - - public List Brokers { get; set; } - public List Topics { get; set; } } } \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs index de67e599..f8b6bd1c 100644 --- a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs @@ -11,66 +11,98 @@ namespace SimpleKafka.Protocol /// arbitrary ConsumerGroup name provided by the call. /// This now supports version 0 and 1 of the protocol /// - public class OffsetCommitRequest : BaseRequest, IKafkaRequest> + public class OffsetCommitRequest : BaseRequest>, IKafkaRequest { - public OffsetCommitRequest(Int16 version = 1) : base(version) + public OffsetCommitRequest(Int16 version = 1) + : base(ApiKeyRequestType.OffsetCommit, version) { } - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetCommit; } } public string ConsumerGroup { get; set; } public int ConsumerGroupGenerationId { get; set; } public string ConsumerId { get; set; } public List OffsetCommits { get; set; } - public void Encode(ref KafkaEncoder encoder) + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeOffsetCommitRequest(this, ref encoder); + return EncodeOffsetCommitRequest(this, encoder); } - public List Decode(ref KafkaDecoder decoder) + internal override List Decode(KafkaDecoder decoder) { - return DecodeOffsetCommitResponse(ref decoder); + return DecodeOffsetCommitResponse(decoder); } - private static void EncodeOffsetCommitRequest(OffsetCommitRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeOffsetCommitRequest(OffsetCommitRequest request, KafkaEncoder encoder) { - if (request.OffsetCommits == null) request.OffsetCommits = new List(); - EncodeHeader(request, ref encoder); - encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); + request + .EncodeHeader(encoder) + .Write(request.ConsumerGroup); + if (request.ApiVersion == 1) { - encoder.Write(request.ConsumerGroupGenerationId); - encoder.Write(request.ConsumerId, StringPrefixEncoding.Int16); + encoder + .Write(request.ConsumerGroupGenerationId) + .Write(request.ConsumerId); } - var topicGroups = request.OffsetCommits.GroupBy(x => x.Topic).ToList(); - encoder.Write(topicGroups.Count); - - foreach (var topicGroup in topicGroups) + if (request.OffsetCommits == null) + { + // Nothing to commit + encoder.Write(0); + } + else if (request.OffsetCommits.Count == 1) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); - encoder.Write(partitions.Count); + var commit = request.OffsetCommits[0]; + // Shortcut the single version + encoder + .Write(1) + .Write(commit.Topic) + .Write(1); - foreach (var partition in partitions) + EncodeCommit(encoder, request.ApiVersion, commit); + } + else + { + // Complete complex request + var topicGroups = new Dictionary>(); + foreach (var commit in request.OffsetCommits) { - foreach (var commit in partition) - { - encoder.Write(partition.Key); - encoder.Write(commit.Offset); + var topicGroup = topicGroups.GetOrCreate(commit.Topic, () => new List(request.OffsetCommits.Count)); + topicGroup.Add(commit); + } - if (request.ApiVersion == 1) - { - encoder.Write(commit.TimeStamp); - } + encoder.Write(topicGroups.Count); + foreach (var topicGroupKvp in topicGroups) + { + var topic = topicGroupKvp.Key; + var commits = topicGroupKvp.Value; + encoder + .Write(topic) + .Write(commits.Count); - encoder.Write(commit.Metadata, StringPrefixEncoding.Int16); + foreach (var commit in commits) + { + EncodeCommit(encoder, request.ApiVersion, commit); } } } + return encoder; + } + + private static void EncodeCommit(KafkaEncoder encoder, int apiVersion, OffsetCommit commit) + { + encoder + .Write(commit.PartitionId) + .Write(commit.Offset); + + if (apiVersion == 1) + { + encoder.Write(commit.TimeStamp); + } + encoder.Write(commit.Metadata); } - private static List DecodeOffsetCommitResponse(ref KafkaDecoder decoder) + private static List DecodeOffsetCommitResponse(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); @@ -78,17 +110,12 @@ private static List DecodeOffsetCommitResponse(ref KafkaDe var topicCount = decoder.ReadInt32(); for (int i = 0; i < topicCount; i++) { - var topic = decoder.ReadInt16String(); + var topic = decoder.ReadString(); var partitionCount = decoder.ReadInt32(); for (int j = 0; j < partitionCount; j++) { - var response = new OffsetCommitResponse() - { - Topic = topic, - PartitionId = decoder.ReadInt32(), - Error = decoder.ReadInt16() - }; + var response = OffsetCommitResponse.Decode(decoder, topic); responses.Add(response); } } @@ -123,7 +150,7 @@ public OffsetCommit() { TimeStamp = -1; } - + } public class OffsetCommitResponse @@ -131,14 +158,29 @@ public class OffsetCommitResponse /// /// The name of the topic this response entry is for. /// - public string Topic; + public readonly string Topic; /// /// The id of the partition this response is for. /// - public Int32 PartitionId; + public readonly int PartitionId; /// /// Error code of exception that occured during the request. Zero if no error. /// - public Int16 Error; + public readonly ErrorResponseCode Error; + + private OffsetCommitResponse(string topic, int partitionId, ErrorResponseCode error) + { + this.Topic = topic; + this.PartitionId = partitionId; + this.Error = error; + } + + internal static OffsetCommitResponse Decode(KafkaDecoder decoder, string topic) + { + var partitionId = decoder.ReadInt32(); + var error = decoder.ReadErrorResponseCode(); + var response = new OffsetCommitResponse(topic, partitionId, error); + return response; + } } } \ No newline at end of file diff --git a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs index e724d2bf..563debf8 100644 --- a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs @@ -12,55 +12,77 @@ namespace SimpleKafka.Protocol /// under any abritrary name. /// This now supports version 1 of the protocol /// - public class OffsetFetchRequest : BaseRequest, IKafkaRequest> + public class OffsetFetchRequest : BaseRequest>, IKafkaRequest { - public OffsetFetchRequest(short version = 1) : base(version) + public OffsetFetchRequest(short version = 1) : base(ApiKeyRequestType.OffsetFetch, version) { } - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.OffsetFetch; } } public string ConsumerGroup { get; set; } public List Topics { get; set; } - public void Encode(ref KafkaEncoder encoder) + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeOffsetFetchRequest(this, ref encoder); + return EncodeOffsetFetchRequest(this, encoder); } - private static void EncodeOffsetFetchRequest(OffsetFetchRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeOffsetFetchRequest(OffsetFetchRequest request, KafkaEncoder encoder) { - if (request.Topics == null) request.Topics = new List(); - EncodeHeader(request, ref encoder); + request + .EncodeHeader(encoder) + .Write(request.ConsumerGroup); - encoder.Write(request.ConsumerGroup, StringPrefixEncoding.Int16); - - var topicGroups = request.Topics.GroupBy(x => x.Topic).ToList(); - encoder.Write(topicGroups.Count); - - foreach (var topicGroup in topicGroups) + if (request.Topics == null) + { + // nothing to fetch + encoder.Write(0); + } + else if (request.Topics.Count == 1) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); - encoder.Write(partitions.Count); + // Short cut single instance request + var fetch = request.Topics[0]; + encoder + .Write(1) + .Write(fetch.Topic) + .Write(1) + .Write(fetch.PartitionId); + } + else + { + // more complex + var topicGroups = new Dictionary>(); + foreach (var fetch in request.Topics) + { + var partitions = topicGroups.GetOrCreate(fetch.Topic, () => new List(request.Topics.Count)); + partitions.Add(fetch.PartitionId); + } - foreach (var partition in partitions) + encoder.Write(topicGroups.Count); + foreach (var kvp in topicGroups) { - foreach (var offset in partition) + var topic = kvp.Key; + var partitions = kvp.Value; + encoder + .Write(topic) + .Write(partitions.Count); + foreach (var fetch in partitions) { - encoder.Write(offset.PartitionId); + encoder.Write(fetch); } + } } + return encoder; } - public List Decode(ref KafkaDecoder decoder) + internal override List Decode(KafkaDecoder decoder) { - return DecodeOffsetFetchResponse(ref decoder); + return DecodeOffsetFetchResponse(decoder); } - private static List DecodeOffsetFetchResponse(ref KafkaDecoder decoder) + private static List DecodeOffsetFetchResponse(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); @@ -68,19 +90,12 @@ private static List DecodeOffsetFetchResponse(ref KafkaDeco var topicCount = decoder.ReadInt32(); for (int i = 0; i < topicCount; i++) { - var topic = decoder.ReadInt16String(); + var topic = decoder.ReadString(); var partitionCount = decoder.ReadInt32(); for (int j = 0; j < partitionCount; j++) { - var response = new OffsetFetchResponse() - { - Topic = topic, - PartitionId = decoder.ReadInt32(), - Offset = decoder.ReadInt64(), - MetaData = decoder.ReadInt16String(), - Error = decoder.ReadInt16() - }; + var response = OffsetFetchResponse.Decode(decoder, topic); responses.Add(response); } } @@ -106,23 +121,42 @@ public class OffsetFetchResponse /// /// The name of the topic this response entry is for. /// - public string Topic; + public readonly string Topic; /// /// The id of the partition this response is for. /// - public Int32 PartitionId; + public readonly int PartitionId; /// /// The offset position saved to the server. /// - public Int64 Offset; + public readonly long Offset; /// /// Any arbitrary metadata stored during a CommitRequest. /// - public string MetaData; + public readonly string MetaData; /// /// Error code of exception that occured during the request. Zero if no error. /// - public Int16 Error; + public readonly ErrorResponseCode Error; + + private OffsetFetchResponse(string topic, int partitionId, long offset, string metaData, ErrorResponseCode error) + { + this.Topic = topic; + this.PartitionId = partitionId; + this.Offset = offset; + this.MetaData = metaData; + this.Error = error; + } + + internal static OffsetFetchResponse Decode(KafkaDecoder decoder, string topic) + { + var partitionId = decoder.ReadInt32(); + var offset = decoder.ReadInt64(); + var metaData = decoder.ReadString(); + var error = decoder.ReadErrorResponseCode(); + var response = new OffsetFetchResponse(topic, partitionId, offset, metaData, error); + return response; + } public override string ToString() { diff --git a/src/SimpleKafka/Protocol/OffsetRequest.cs b/src/SimpleKafka/Protocol/OffsetRequest.cs index bbbae57c..39e87258 100644 --- a/src/SimpleKafka/Protocol/OffsetRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetRequest.cs @@ -8,51 +8,79 @@ namespace SimpleKafka.Protocol /// /// A funky Protocol for requesting the starting offset of each segment for the requested partition /// - public class OffsetRequest : BaseRequest, IKafkaRequest> + public class OffsetRequest : BaseRequest>, IKafkaRequest { - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Offset; } } public List Offsets { get; set; } - public void Encode(ref KafkaEncoder encoder) + public OffsetRequest() : base(ApiKeyRequestType.Offset) { } + + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeOffsetRequest(this, ref encoder); + return EncodeOffsetRequest(this, encoder); } - public List Decode(ref KafkaDecoder decoder) + internal override List Decode(KafkaDecoder decoder) { - return DecodeOffsetResponse(ref decoder); + return DecodeOffsetResponse(decoder); } - private static void EncodeOffsetRequest(OffsetRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeOffsetRequest(OffsetRequest request, KafkaEncoder encoder) { - if (request.Offsets == null) request.Offsets = new List(); - EncodeHeader(request, ref encoder); - encoder.Write(ReplicaId); - - var topicGroups = request.Offsets.GroupBy(x => x.Topic).ToList(); - encoder.Write(topicGroups.Count); + request + .EncodeHeader(encoder) + .Write(ReplicaId); - foreach (var topicGroup in topicGroups) + if (request.Offsets == null) + { + encoder.Write(0); + } + else if (request.Offsets.Count == 1) { - var partitions = topicGroup.GroupBy(x => x.PartitionId).ToList(); - encoder.Write(topicGroup.Key, StringPrefixEncoding.Int16); - encoder.Write(partitions.Count); + // shortcut the single request + var offset = request.Offsets[0]; + encoder + .Write(1) + .Write(offset.Topic) + .Write(1) + .Write(offset.PartitionId) + .Write(offset.Time) + .Write(offset.MaxOffsets); + } + else + { + // Full request + var topicGroups = new Dictionary>(); + foreach (var offset in request.Offsets) + { + var offsets = topicGroups.GetOrCreate(offset.Topic, () => new List(request.Offsets.Count)); + offsets.Add(offset); + } - foreach (var partition in partitions) + encoder.Write(topicGroups.Count); + foreach (var kvp in topicGroups) { - foreach (var offset in partition) + var topic = kvp.Key; + var offsets = kvp.Value; + + encoder + .Write(topic) + .Write(offsets.Count); + + foreach (var offset in offsets) { - encoder.Write(partition.Key); - encoder.Write(offset.Time); - encoder.Write(offset.MaxOffsets); + encoder + .Write(offset.PartitionId) + .Write(offset.Time) + .Write(offset.MaxOffsets); } } } - + return encoder; } - private static List DecodeOffsetResponse(ref KafkaDecoder decoder) + + private static List DecodeOffsetResponse(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); @@ -60,30 +88,19 @@ private static List DecodeOffsetResponse(ref KafkaDecoder decode var topicCount = decoder.ReadInt32(); for (int i = 0; i < topicCount; i++) { - var topic = decoder.ReadInt16String(); + var topic = decoder.ReadString(); var partitionCount = decoder.ReadInt32(); for (int j = 0; j < partitionCount; j++) { - var response = new OffsetResponse() - { - Topic = topic, - PartitionId = decoder.ReadInt32(), - Error = decoder.ReadInt16(), - Offsets = new List() - }; - var offsetCount = decoder.ReadInt32(); - for (int k = 0; k < offsetCount; k++) - { - response.Offsets.Add(decoder.ReadInt64()); - } + var response = OffsetResponse.Decode(decoder, topic); responses.Add(response); } } return responses; } - + } public class Offset @@ -106,47 +123,32 @@ public Offset() public class OffsetResponse { - public string Topic { get; set; } - public int PartitionId { get; set; } - public Int16 Error { get; set; } - public List Offsets { get; set; } - } - - public class OffsetPosition - { - public OffsetPosition() { } - public OffsetPosition(int partitionId, long offset) - { - PartitionId = partitionId; - Offset = offset; - } - public int PartitionId { get; set; } - public long Offset { get; set; } + public readonly string Topic; + public readonly int PartitionId; + public readonly ErrorResponseCode Error; + public readonly long[] Offsets; - public override string ToString() + private OffsetResponse(string topic, int partitionId, ErrorResponseCode error, long[] offsets) { - return string.Format("PartitionId:{0}, Offset:{1}", PartitionId, Offset); - } + this.Topic = topic; + this.PartitionId = partitionId; + this.Error = error; + this.Offsets = offsets; - public override bool Equals(object obj) - { - if (ReferenceEquals(null, obj)) return false; - if (ReferenceEquals(this, obj)) return true; - if (obj.GetType() != this.GetType()) return false; - return Equals((OffsetPosition)obj); - } - - protected bool Equals(OffsetPosition other) - { - return PartitionId == other.PartitionId && Offset == other.Offset; } - public override int GetHashCode() + internal static OffsetResponse Decode(KafkaDecoder decoder, string topic) { - unchecked + var partitionId = decoder.ReadInt32(); + var error = decoder.ReadErrorResponseCode(); + var offsetCount = decoder.ReadInt32(); + var offsets = new long[offsetCount]; + for (int k = 0; k < offsetCount; k++) { - return (PartitionId * 397) ^ Offset.GetHashCode(); + offsets[k] = decoder.ReadInt64(); } + var response = new OffsetResponse(topic, partitionId, error, offsets); + return response; } } -} \ No newline at end of file +} diff --git a/src/SimpleKafka/Protocol/Partition.cs b/src/SimpleKafka/Protocol/Partition.cs new file mode 100644 index 00000000..7ca9f68e --- /dev/null +++ b/src/SimpleKafka/Protocol/Partition.cs @@ -0,0 +1,83 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka.Protocol +{ + public class Partition + { + /// + /// Error code. 0 indicates no error occured. + /// + public readonly ErrorResponseCode ErrorCode; + /// + /// The Id of the partition that this metadata describes. + /// + public readonly int PartitionId; + /// + /// The node id for the kafka broker currently acting as leader for this partition. If no leader exists because we are in the middle of a leader election this id will be -1. + /// + public readonly int LeaderId; + /// + /// The set of alive nodes that currently acts as slaves for the leader for this partition. + /// + public readonly int[] Replicas; + /// + /// The set subset of the replicas that are "caught up" to the leader + /// + public readonly int[] Isrs; + + private Partition(ErrorResponseCode errorCode, int partitionId, int leaderId, int[] replicas, int[] isrs) + { + this.ErrorCode = errorCode; + this.PartitionId = partitionId; + this.LeaderId = leaderId; + this.Replicas = replicas; + this.Isrs = isrs; + } + + internal static Partition Decode(KafkaDecoder decoder) + { + var errorCode = decoder.ReadErrorResponseCode(); + var partitionId = decoder.ReadInt32(); + var leaderId = decoder.ReadInt32(); + + var numReplicas = decoder.ReadInt32(); + var replicas = new int[numReplicas]; + for (int i = 0; i < numReplicas; i++) + { + replicas[i] = decoder.ReadInt32(); + } + + var numIsr = decoder.ReadInt32(); + var isrs = new int[numIsr]; + for (int i = 0; i < numIsr; i++) + { + isrs[i] = decoder.ReadInt32(); + } + var partition = new Partition(errorCode, partitionId, leaderId, replicas, isrs); + + return partition; + } + + protected bool Equals(Partition other) + { + return PartitionId == other.PartitionId; + } + + public override int GetHashCode() + { + return PartitionId; + } + + public override bool Equals(object obj) + { + if (ReferenceEquals(null, obj)) return false; + if (ReferenceEquals(this, obj)) return true; + if (obj.GetType() != this.GetType()) return false; + return Equals((Partition)obj); + } + } +} diff --git a/src/SimpleKafka/Protocol/ProduceRequest.cs b/src/SimpleKafka/Protocol/ProduceRequest.cs index 84d9813c..79494fd2 100644 --- a/src/SimpleKafka/Protocol/ProduceRequest.cs +++ b/src/SimpleKafka/Protocol/ProduceRequest.cs @@ -5,17 +5,13 @@ namespace SimpleKafka.Protocol { - public class ProduceRequest : BaseRequest, IKafkaRequest> + public class ProduceRequest : BaseRequest>, IKafkaRequest { /// /// Provide a hint to the broker call not to expect a response for requests without Acks. /// public override bool ExpectResponse { get { return Acks > 0; } } /// - /// Indicates the type of kafka encoding this request is. - /// - public ApiKeyRequestType ApiKey { get { return ApiKeyRequestType.Produce; } } - /// /// Time kafka will wait for requested ack level before returning. /// public int TimeoutMS = 1000; @@ -28,56 +24,89 @@ public class ProduceRequest : BaseRequest, IKafkaRequest> /// public List Payload = new List(); + public ProduceRequest() : base(ApiKeyRequestType.Produce) { } - public void Encode(ref KafkaEncoder encoder) + internal override KafkaEncoder Encode(KafkaEncoder encoder) { - EncodeProduceRequest(this, ref encoder); + return EncodeProduceRequest(this, encoder); } - public List Decode(ref KafkaDecoder decoder) + internal override List Decode(KafkaDecoder decoder) { - return DecodeProduceResponse(ref decoder); + return DecodeProduceResponse(decoder); } #region Protocol... - private static void EncodeProduceRequest(ProduceRequest request, ref KafkaEncoder encoder) + private static KafkaEncoder EncodeProduceRequest(ProduceRequest request, KafkaEncoder encoder) { - if (request.Payload == null) request.Payload = new List(); - - var groupedPayloads = (from p in request.Payload - group p by new - { - p.Topic, - p.Partition, - p.Codec - } into tpc - select tpc).ToList(); - - EncodeHeader(request, ref encoder); - encoder.Write(request.Acks); - encoder.Write(request.TimeoutMS); - encoder.Write(groupedPayloads.Count); - foreach (var groupedPayload in groupedPayloads) + request.EncodeHeader(encoder) + .Write(request.Acks) + .Write(request.TimeoutMS); + + if (request.Payload == null) + { + encoder.Write(0); + } + else if (request.Payload.Count == 1) + { + // Short cut single request + var payload = request.Payload[0]; + encoder + .Write(1) + .Write(payload.Topic) + .Write(1); + + WritePayload(encoder, payload); + } + else { - var payloads = groupedPayload.ToList(); - encoder.Write(groupedPayload.Key.Topic, StringPrefixEncoding.Int16); - encoder.Write(payloads.Count); - encoder.Write(groupedPayload.Key.Partition); + // More complex + var topicGroups = new Dictionary>(); + foreach (var payload in request.Payload) + { + var payloads = topicGroups.GetOrCreate(payload.Topic, () => new List(request.Payload.Count)); + payloads.Add(payload); + } - var marker = encoder.PrepareForLength(); - switch (groupedPayload.Key.Codec) + encoder.Write(topicGroups.Count); + foreach (var kvp in topicGroups) { - case MessageCodec.CodecNone: - Message.EncodeMessageSet(ref encoder, (payloads.SelectMany(x => x.Messages))); - break; - default: - throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", groupedPayload.Key.Codec)); + var topic = kvp.Key; + var payloads = kvp.Value; + + encoder + .Write(topic) + .Write(payloads.Count); + + foreach (var payload in payloads) + { + WritePayload(encoder, payload); + } } - encoder.WriteLength(marker); } + return encoder; } - private List DecodeProduceResponse(ref KafkaDecoder decoder) + private static void WritePayload(KafkaEncoder encoder, Payload payload) + { + encoder + .Write(payload.Partition); + + var marker = encoder.PrepareForLength(); + switch (payload.Codec) + { + case MessageCodec.CodecNone: + Message.EncodeMessageSet(encoder, payload.Messages); + break; + + default: + throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", payload.Codec)); + + } + encoder.WriteLength(marker); + } + + private List DecodeProduceResponse(KafkaDecoder decoder) { var correlationId = decoder.ReadInt32(); @@ -85,19 +114,12 @@ private List DecodeProduceResponse(ref KafkaDecoder decoder) var topicCount = decoder.ReadInt32(); for (int i = 0; i < topicCount; i++) { - var topic = decoder.ReadInt16String(); + var topic = decoder.ReadString(); var partitionCount = decoder.ReadInt32(); for (int j = 0; j < partitionCount; j++) { - var response = new ProduceResponse() - { - Topic = topic, - PartitionId = decoder.ReadInt32(), - Error = decoder.ReadInt16(), - Offset = decoder.ReadInt64() - }; - + var response = ProduceResponse.Decode(decoder, topic); responses.Add(response); } } @@ -111,19 +133,37 @@ public class ProduceResponse /// /// The topic the offset came from. /// - public string Topic { get; set; } + public readonly string Topic; /// /// The partition the offset came from. /// - public int PartitionId { get; set; } + public readonly int PartitionId; /// /// Error response code. 0 is success. /// - public Int16 Error { get; set; } + public readonly ErrorResponseCode Error; /// /// The offset number to commit as completed. /// - public long Offset { get; set; } + public readonly long Offset; + + private ProduceResponse(string topic, int partitionId, ErrorResponseCode error, long offset) + { + this.Topic = topic; + this.PartitionId = partitionId; + this.Error = error; + this.Offset = offset; + } + + internal static ProduceResponse Decode(KafkaDecoder decoder, string topic) + { + var partitionId = decoder.ReadInt32(); + var error = decoder.ReadErrorResponseCode(); + var offset = decoder.ReadInt64(); + + var response = new ProduceResponse(topic, partitionId, error, offset); + return response; + } public override bool Equals(object obj) { diff --git a/src/SimpleKafka/Protocol/Topic.cs b/src/SimpleKafka/Protocol/Topic.cs index a09ad19a..884ef704 100644 --- a/src/SimpleKafka/Protocol/Topic.cs +++ b/src/SimpleKafka/Protocol/Topic.cs @@ -6,97 +6,33 @@ namespace SimpleKafka.Protocol { public class Topic { - public Int16 ErrorCode { get; set; } - public string Name { get; set; } - public List Partitions { get; set; } + public readonly ErrorResponseCode ErrorCode; + public readonly string Name; + public readonly Partition[] Partitions; - internal static Topic Decode(ref KafkaDecoder decoder) + private Topic(ErrorResponseCode errorCode, string name, Partition[] partitions) { - var topic = new Topic - { - ErrorCode = decoder.ReadInt16(), - Name = decoder.ReadInt16String(), - }; + this.ErrorCode = errorCode; + this.Name = name; + this.Partitions = partitions; + } + + internal static Topic Decode(KafkaDecoder decoder) + { + var errorCode = decoder.ReadErrorResponseCode(); + var name = decoder.ReadString(); var numPartitions = decoder.ReadInt32(); - var partitions = new List(numPartitions); + var partitions = new Partition[numPartitions]; for (int i = 0; i < numPartitions; i++) { - partitions.Add(Partition.Decode(ref decoder)); + partitions[i] = Partition.Decode(decoder); } - topic.Partitions = partitions; + var topic = new Topic(errorCode, name, partitions); return topic; } } - public class Partition - { - /// - /// Error code. 0 indicates no error occured. - /// - public Int16 ErrorCode { get; set; } - /// - /// The Id of the partition that this metadata describes. - /// - public int PartitionId { get; set; } - /// - /// The node id for the kafka broker currently acting as leader for this partition. If no leader exists because we are in the middle of a leader election this id will be -1. - /// - public int LeaderId { get; set; } - /// - /// The set of alive nodes that currently acts as slaves for the leader for this partition. - /// - public List Replicas { get; set; } - /// - /// The set subset of the replicas that are "caught up" to the leader - /// - public List Isrs { get; set; } - - public static Partition Decode(ref KafkaDecoder decoder) - { - var partition = new Partition { - ErrorCode = decoder.ReadInt16(), - PartitionId = decoder.ReadInt32(), - LeaderId = decoder.ReadInt32(), - }; - - var numReplicas = decoder.ReadInt32(); - var replicas = new List(numReplicas); - for (int i = 0; i < numReplicas; i++) - { - replicas.Add(decoder.ReadInt32()); - } - partition.Replicas = replicas; - - var numIsr = decoder.ReadInt32(); - var isrs = new List(numIsr); - for (int i = 0; i < numIsr; i++) - { - isrs.Add(decoder.ReadInt32()); - } - partition.Isrs = isrs; - - return partition; - } - - protected bool Equals(Partition other) - { - return PartitionId == other.PartitionId; - } - - public override int GetHashCode() - { - return PartitionId; - } - - public override bool Equals(object obj) - { - if (ReferenceEquals(null, obj)) return false; - if (ReferenceEquals(this, obj)) return true; - if (obj.GetType() != this.GetType()) return false; - return Equals((Partition) obj); - } - } } diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index 1d082430..e60467d4 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -52,7 +52,6 @@ - @@ -74,6 +73,7 @@ + diff --git a/src/SimpleKafkaTests/Helpers/MessageHelper.cs b/src/SimpleKafkaTests/Helpers/MessageHelper.cs index 49700987..74b65849 100644 --- a/src/SimpleKafkaTests/Helpers/MessageHelper.cs +++ b/src/SimpleKafkaTests/Helpers/MessageHelper.cs @@ -21,7 +21,7 @@ public static byte[] CreateMessage(long offset, byte[] key, byte[] payload, byte var buffer = new byte[1024]; var encoder = new KafkaEncoder(buffer); - Message.EncodeMessageSet(ref encoder, new[] { message }); + Message.EncodeMessageSet(encoder, new[] { message }); var result = new byte[encoder.Offset]; Array.Copy(encoder.Buffer, result, encoder.Offset); diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 74a2a90c..967feb32 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -99,7 +99,7 @@ public async Task TestListingAllTopicsWorksOk() Console.WriteLine("{0},{1}", topic.ErrorCode, topic.Name); foreach (var partition in topic.Partitions) { - Console.WriteLine("{0},{1},{2},{3},{4}", partition.ErrorCode, partition.Isrs.Count, partition.LeaderId, partition.PartitionId, partition.Replicas.Count); + Console.WriteLine("{0},{1},{2},{3},{4}", partition.ErrorCode, partition.Isrs.Length, partition.LeaderId, partition.PartitionId, partition.Replicas.Length); } } } @@ -150,7 +150,7 @@ public async Task TestNewTopicProductionWorksOk() while (response == null) { response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - if (response.Topics[0].ErrorCode == (short)ErrorResponseCode.LeaderNotAvailable) + if (response.Topics[0].ErrorCode == ErrorResponseCode.LeaderNotAvailable) { response = null; await Task.Delay(1000); @@ -159,12 +159,12 @@ public async Task TestNewTopicProductionWorksOk() } Assert.That(response, Is.Not.Null); var first = response; - Assert.That(first.Topics, Has.Count.EqualTo(1)); + Assert.That(first.Topics, Has.Length.EqualTo(1)); var firstTopic = first.Topics.First(); - Assert.That(firstTopic.ErrorCode, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(firstTopic.ErrorCode, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(firstTopic.Name, Is.EqualTo(topic)); - Assert.That(firstTopic.Partitions, Has.Count.EqualTo(1)); + Assert.That(firstTopic.Partitions, Has.Length.EqualTo(1)); var firstPartition = firstTopic.Partitions.First(); Assert.That(firstPartition.PartitionId, Is.EqualTo(0)); @@ -197,7 +197,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Is.Not.Null); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.Offset, Is.EqualTo(0)); @@ -224,7 +224,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.Topic, Is.EqualTo(topic)); @@ -261,10 +261,10 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); - Assert.That(first.Offsets, Has.Count.EqualTo(2)); + Assert.That(first.Offsets, Has.Length.EqualTo(2)); Assert.That(first.Offsets[0], Is.EqualTo(4)); Assert.That(first.Offsets[1], Is.EqualTo(0)); @@ -308,7 +308,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.MetaData, Is.Empty); @@ -337,7 +337,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); } @@ -360,7 +360,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.Topic, Is.EqualTo(topic)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.MetaData, Is.EqualTo("Metadata 1")); @@ -388,7 +388,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.Topic, Is.EqualTo(topic)); @@ -440,7 +440,7 @@ public async Task TestNewTopicProductionWorksOk() Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); - Assert.That(first.Error, Is.EqualTo((short)ErrorResponseCode.NoError)); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(first.HighWaterMark, Is.EqualTo(4)); Assert.That(first.PartitionId, Is.EqualTo(0)); Assert.That(first.Topic, Is.EqualTo(topic)); diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index 856d02b9..da400b95 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -51,8 +51,8 @@ - - + + diff --git a/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs b/src/SimpleKafkaTests/Unit/KafkaDecoderTests.cs similarity index 64% rename from src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs rename to src/SimpleKafkaTests/Unit/KafkaDecoderTests.cs index ccabe1bc..61ce0421 100644 --- a/src/SimpleKafkaTests/Unit/BigEndianDecoderTests.cs +++ b/src/SimpleKafkaTests/Unit/KafkaDecoderTests.cs @@ -13,7 +13,7 @@ namespace SimpleKafkaTests.Unit /// Modified to work with nunit from xunit. [TestFixture] [Category("Unit")] - public class BigEndianDecoderTests + public class KafkaDecoderTests { [Theory] [TestCase((Int64)0, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] @@ -57,28 +57,14 @@ public void Int16Tests(Int16 expectedValue, Byte[] givenBytes) [Theory] - [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] - [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] - [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] - [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] - [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] - [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] - [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] - [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] - [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] - public void StringTests(String expectedValue, Byte[] givenBytes, StringPrefixEncoding encoding) + [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 })] + [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] + [TestCase("", new Byte[] { 0x00, 0x00 })] + [TestCase(null, new Byte[] { 0xFF, 0xFF })] + public void StringTests(String expectedValue, Byte[] givenBytes) { var decoder = new KafkaDecoder(givenBytes); - string actualValue = null; - switch (encoding) - { - case StringPrefixEncoding.None: actualValue = decoder.ReadString(); break; - case StringPrefixEncoding.Int16: actualValue = decoder.ReadInt16String(); break; - case StringPrefixEncoding.Int32: actualValue = decoder.ReadInt32String(); break; - default: Assert.Fail("Unknown encoding", encoding); break; - } + var actualValue = decoder.ReadString(); Assert.That(decoder.Offset, Is.EqualTo(givenBytes.Length)); Assert.That(actualValue, Is.EqualTo(expectedValue)); } diff --git a/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs b/src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs similarity index 73% rename from src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs rename to src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs index 349ecfa6..a7dc0bd2 100644 --- a/src/SimpleKafkaTests/Unit/BigEndianEncoderTests.cs +++ b/src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs @@ -13,7 +13,7 @@ namespace SimpleKafkaTests.Unit /// Modified to work with nunit from xunit. [TestFixture] [Category("Unit")] - public class BigEndianEncoderTests + public class KafkaEncoderTests { [Theory] [TestCase((Int64)0, new Byte[] { 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00 })] @@ -31,7 +31,6 @@ public void Int64Tests(Int64 number, Byte[] expectedBytes) [Theory] [TestCase((UInt32)0, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] [TestCase((UInt32)1, new Byte[] { 0x00, 0x00, 0x00, 0x01 })] - [TestCase(UInt32.MinValue, new Byte[] { 0x00, 0x00, 0x00, 0x00 })] [TestCase(UInt32.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })] public void UInt32Tests(UInt32 number, Byte[] expectedBytes) { @@ -71,23 +70,15 @@ public void Int16Tests(Int16 number, Byte[] expectedBytes) [Theory] - [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int16)] - [TestCase("0000", new Byte[] { 0x00, 0x00, 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.Int32)] - [TestCase("0000", new Byte[] { 0x30, 0x30, 0x30, 0x30 }, StringPrefixEncoding.None)] - [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int16)] - [TestCase("€€€€", new Byte[] { 0x00, 0x00, 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.Int32)] - [TestCase("€€€€", new Byte[] { 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { }, StringPrefixEncoding.None)] - [TestCase("", new Byte[] { 0x00, 0x00 }, StringPrefixEncoding.Int16)] - [TestCase("", new Byte[] { 0x00, 0x00, 0x00, 0x00 }, StringPrefixEncoding.Int32)] - [TestCase(null, new Byte[] { }, StringPrefixEncoding.None)] - [TestCase(null, new Byte[] { 0xFF, 0xFF }, StringPrefixEncoding.Int16)] - [TestCase(null, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF }, StringPrefixEncoding.Int32)] - public void StringTests(String value, Byte[] expectedBytes, StringPrefixEncoding encoding) + [TestCase("0000", new Byte[] { 0x00, 0x04, 0x30, 0x30, 0x30, 0x30 })] + [TestCase("€€€€", new Byte[] { 0x00, 0x0C, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC, 0xE2, 0x82, 0xAC })] + [TestCase("", new Byte[] { 0x00, 0x00 })] + [TestCase(null, new Byte[] { 0xFF, 0xFF })] + public void StringTests(String value, Byte[] expectedBytes) { var buffer = new byte[expectedBytes.Length]; var encoder = new KafkaEncoder(buffer); - encoder.Write(value, encoding); + encoder.Write(value); Assert.That(encoder.Offset, Is.EqualTo(expectedBytes.Length)); Assert.That(buffer, Is.EqualTo(expectedBytes)); } diff --git a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs index c8dc792d..54fd30f5 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs @@ -18,7 +18,7 @@ public void EnsureHeaderShouldPackCorrectByteLengths() var encoder = new KafkaEncoder(new byte[14]); var request = new FetchRequest { ClientId = "test", CorrelationId = 123456789 }; - BaseRequest.EncodeHeader(request, ref encoder); + request.EncodeHeader(encoder); Assert.That(encoder.Offset, Is.EqualTo(14)); Assert.That(encoder.Buffer, Is.EqualTo(new byte[] { 0, 1, 0, 0, 7, 91, 205, 21, 0, 4, 116, 101, 115, 116 })); } diff --git a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs index 22295acb..f42ac5ca 100644 --- a/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs +++ b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs @@ -23,11 +23,11 @@ public void DecodeMessageShouldThrowWhenCrcFails() var buffer = new byte[1024]; var encoder = new KafkaEncoder(buffer); - Message.EncodeMessage(testMessage, ref encoder); + Message.EncodeMessage(testMessage, encoder); buffer[0] += 1; var decoder = new KafkaDecoder(buffer, 0, encoder.Offset); - var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); + var result = Message.DecodeMessage(0, 0, decoder, encoder.Offset); }); } @@ -42,10 +42,10 @@ public void EnsureMessageEncodeAndDecodeAreCompatible(string key, string value) var buffer = new byte[1024]; var encoder = new KafkaEncoder(buffer); - Message.EncodeMessage(testMessage, ref encoder); + Message.EncodeMessage(testMessage, encoder); var decoder = new KafkaDecoder(buffer); - var result = Message.DecodeMessage(0, 0, ref decoder, encoder.Offset); + var result = Message.DecodeMessage(0, 0, decoder, encoder.Offset); Assert.That(testMessage.Key, Is.EqualTo(result.Key)); Assert.That(testMessage.Value, Is.EqualTo(result.Value)); @@ -71,7 +71,7 @@ public void EncodeMessageSetEncodesMultipleMessages() var buffer = new byte[expected.Length]; var encoder = new KafkaEncoder(buffer); - Message.EncodeMessageSet(ref encoder, messages); + Message.EncodeMessageSet(encoder, messages); Assert.That(buffer, Is.EqualTo(expected)); } @@ -81,7 +81,7 @@ public void DecodeMessageSetShouldHandleResponseWithMaxBufferSizeHit() { //This message set has a truncated message bytes at the end of it var decoder = new KafkaDecoder(MessageHelper.FetchResponseMaxBytesOverflow); - var result = Message.DecodeMessageSet(0, ref decoder, decoder.Length); + var result = Message.DecodeMessageSet(0, decoder, decoder.Length); var message = Encoding.UTF8.GetString(result.First().Value); @@ -106,7 +106,7 @@ public void WhenMessageIsTruncatedThenBufferUnderRunExceptionIsThrown() var decoder = new KafkaDecoder(payloadBytes); - Message.DecodeMessageSet(0, ref decoder, payloadBytes.Length); + Message.DecodeMessageSet(0, decoder, payloadBytes.Length); }); } @@ -119,7 +119,7 @@ public void WhenMessageIsExactlyTheSizeOfBufferThenMessageIsDecoded() // act/assert var decoder = new KafkaDecoder(payload, 0, payload.Length); - var messages = Message.DecodeMessageSet(0, ref decoder, payload.Length); + var messages = Message.DecodeMessageSet(0, decoder, payload.Length); var actualPayload = messages.First().Value; // assert From 834c54dff5213f4bb6db2f31ebd69b3685c15dc9 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Tue, 14 Apr 2015 15:02:22 +0100 Subject: [PATCH 09/14] Refactor to support different partitions and keys better --- src/SimpleKafka/FixedPartitioner.cs | 31 +++ src/SimpleKafka/IKafkaMessagePartitioner.cs | 4 +- src/SimpleKafka/Interfaces/IKafkaRequest.cs | 29 --- src/SimpleKafka/KafkaBrokers.cs | 131 ++++++++--- src/SimpleKafka/KafkaConnection.cs | 2 +- src/SimpleKafka/KafkaConsumer.cs | 21 +- src/SimpleKafka/KafkaMessage.cs | 35 --- src/SimpleKafka/KafkaProducer.cs | 203 +++++++++++------- src/SimpleKafka/KeyedMessage.cs | 51 +++++ src/SimpleKafka/LoadBalancedPartitioner.cs | 14 +- ...NullSerializer.cs => NullKeySerializer.cs} | 0 .../Protocol/ConsumerMetadataRequest.cs | 2 +- src/SimpleKafka/Protocol/FetchRequest.cs | 2 +- src/SimpleKafka/Protocol/Message.cs | 2 +- src/SimpleKafka/Protocol/MetadataRequest.cs | 2 +- .../Protocol/OffsetCommitRequest.cs | 2 +- .../Protocol/OffsetFetchRequest.cs | 2 +- src/SimpleKafka/Protocol/OffsetRequest.cs | 2 +- src/SimpleKafka/Protocol/ProduceRequest.cs | 2 +- src/SimpleKafka/ReceivedKafkaMessage.cs | 13 +- src/SimpleKafka/SimpleKafka.csproj | 9 +- .../Integration/ConsumerTests.cs | 49 ++--- .../Integration/SimpleTests.cs | 6 +- src/SimpleKafkaTests/SimpleKafkaTests.csproj | 3 + src/kafka-net.sln | 7 +- 25 files changed, 390 insertions(+), 234 deletions(-) create mode 100644 src/SimpleKafka/FixedPartitioner.cs delete mode 100644 src/SimpleKafka/Interfaces/IKafkaRequest.cs delete mode 100644 src/SimpleKafka/KafkaMessage.cs create mode 100644 src/SimpleKafka/KeyedMessage.cs rename src/SimpleKafka/{NullSerializer.cs => NullKeySerializer.cs} (100%) diff --git a/src/SimpleKafka/FixedPartitioner.cs b/src/SimpleKafka/FixedPartitioner.cs new file mode 100644 index 00000000..843a1d0c --- /dev/null +++ b/src/SimpleKafka/FixedPartitioner.cs @@ -0,0 +1,31 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + class FixedPartitioner : IKafkaMessagePartitioner + { + private readonly int partitionNumber; + public FixedPartitioner(int partitionNumber) + { + this.partitionNumber = partitionNumber; + } + + public int CalculatePartition(TPartitionKey key, int numberOfPartitions) + { + if (partitionNumber >= numberOfPartitions) + { + throw new InvalidOperationException( + string.Format( + "Fixed partition number ({0}) is more than the number of partitions ({1})", + partitionNumber, numberOfPartitions)); + + } + return partitionNumber; + } + + } +} diff --git a/src/SimpleKafka/IKafkaMessagePartitioner.cs b/src/SimpleKafka/IKafkaMessagePartitioner.cs index 5ee67623..71738910 100644 --- a/src/SimpleKafka/IKafkaMessagePartitioner.cs +++ b/src/SimpleKafka/IKafkaMessagePartitioner.cs @@ -6,8 +6,8 @@ namespace SimpleKafka { - public interface IKafkaMessagePartitioner + public interface IKafkaMessagePartitioner { - int CalculatePartition(KafkaMessage message); + int CalculatePartition(TPartitionKey partitionKey, int numberOfPartitions); } } diff --git a/src/SimpleKafka/Interfaces/IKafkaRequest.cs b/src/SimpleKafka/Interfaces/IKafkaRequest.cs deleted file mode 100644 index cd0c9318..00000000 --- a/src/SimpleKafka/Interfaces/IKafkaRequest.cs +++ /dev/null @@ -1,29 +0,0 @@ -using System.Collections.Generic; -using SimpleKafka.Protocol; - -namespace SimpleKafka -{ - /// - /// KafkaRequest represents a Kafka request messages as an object which can Encode itself into the appropriate - /// binary request and Decode any responses to that request. - /// - public interface IKafkaRequest - { - /// - /// Indicates this request should wait for a response from the broker - /// - bool ExpectResponse { get; } - /// - /// Descriptive name used to identify the source of this request. - /// - string ClientId { get; set; } - /// - /// The API Version used for this request - /// - short ApiVersion { get; } - /// - /// Id which will be echoed back by Kafka to correlate responses to this request. Usually automatically assigned by driver. - /// - int CorrelationId { get; set; } - } -} \ No newline at end of file diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs index dcc4f592..9bafac89 100644 --- a/src/SimpleKafka/KafkaBrokers.cs +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -129,7 +129,91 @@ public async Task> BuildOffsetCoordinatorMapAsync(Cancel return offsetCoordinatorMap; } - public async Task, T>>> BuildBrokerMapAsync(CancellationToken token, Dictionary> topicMap) + public async Task> GetValidPartitionsForTopicsAsync(IEnumerable topics, CancellationToken token) + { + while (true) + { + var result = await GetPartitionsForTopicsAsync(topics, token).ConfigureAwait(false); + var anyWithElection = result.Values.Any(partitions => partitions.Any(partition => partition.LeaderId == -1)); + if (!anyWithElection) + { + return result; + } + + await BackoffAndRefresh(token).ConfigureAwait(false); + } + } + + public async Task> GetPartitionsForTopicsAsync(IEnumerable topics, CancellationToken token) + { + var result = new Dictionary(); + foreach (var topic in topics) + { + var partitions = await GetPartitionsForTopicAsync(topic, token).ConfigureAwait(false); + result[topic] = partitions; + } + return result; + } + + public async Task GetPartitionsForTopicAsync(string topic, CancellationToken token) + { + var partitions = GetPartitionsForTopic(topic); + if (partitions == null) + { + AddTopic(topic); + var refreshed = await RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new KeyNotFoundException("Failed to refresh brokers"); + } + partitions = GetPartitionsForTopic(topic); + if (partitions == null) + { + throw new KeyNotFoundException("Failed to find topic: " + topic); + } + } + return partitions; + } + + public async Task GetLeaderForTopicAndPartitionAsync(string topic, int partitionId, CancellationToken token) + { + while (true) + { + var partitions = await GetPartitionsForTopicAsync(topic, token).ConfigureAwait(false); + var partition = GetPartitionIfReady(topic, partitionId, partitions); + + if (partition != null) + { + return partition.LeaderId; + } + else + { + token.ThrowIfCancellationRequested(); + await BackoffAndRefresh(token).ConfigureAwait(false); + } + } + } + + private static Partition GetPartitionIfReady(string topic, int partitionId, Partition[] partitions) + { + if (partitionId >= partitions.Length) + { + throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionId + " is too big. Only have " + partitions.Length + " partitions"); + } + + var partition = partitions[partitionId]; + if (partition.LeaderId == -1) + { + return null; + } + else + { + return partition; + } + } + + public async Task, T>>> BuildBrokerMapAsync( + CancellationToken token, Dictionary> topicMap) { if (connections.Count == 0) { @@ -143,34 +227,14 @@ public async Task, T>>> BuildBroke foreach (var topicKvp in topicMap) { var topic = topicKvp.Key; - var partitions = GetPartitionsForTopic(topic); - if (partitions == null) - { - AddTopic(topic); - var refreshed = await RefreshAsync(token).ConfigureAwait(false); - if (!refreshed) - { - throw new KeyNotFoundException("Failed to refresh brokers"); - } - partitions = GetPartitionsForTopic(topic); - if (partitions == null) - { - throw new KeyNotFoundException("Failed to find topic: " + topic); - } - } + var partitions = await GetPartitionsForTopicAsync(topic, token).ConfigureAwait(false); foreach (var partitionKvp in topicKvp.Value) { var partitionNumber = partitionKvp.Key; - if (partitionNumber >= partitions.Length) - { - throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionNumber + " is too big. Only have " + partitions.Length + " partitions"); - } - - var partition = partitions[partitionNumber]; - if (partition.LeaderId == -1) + var partition = GetPartitionIfReady(topic, partitionNumber, partitions); + if (partition == null) { - Log.Information("Topic {topic}, partition {partition} has no leader, waiting", topic, partitionNumber); ready = false; break; } @@ -183,13 +247,7 @@ public async Task, T>>> BuildBroke if (!ready) { - Log.Verbose("Waiting before trying again"); - await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); - var refreshed = await RefreshAsync(token).ConfigureAwait(false); - if (!refreshed) - { - throw new KeyNotFoundException("Failed to refresh brokers"); - } + await BackoffAndRefresh(token).ConfigureAwait(false); } } @@ -210,6 +268,17 @@ public async Task, T>>> BuildBroke return brokerMap; } + public async Task BackoffAndRefresh(CancellationToken token) + { + Log.Verbose("Waiting before trying again"); + await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); + var refreshed = await RefreshAsync(token).ConfigureAwait(false); + if (!refreshed) + { + throw new KeyNotFoundException("Failed to refresh brokers"); + } + } + internal Partition[] GetPartitionsForTopic(string topic) { return topicToPartitions.TryGetValue(topic); diff --git a/src/SimpleKafka/KafkaConnection.cs b/src/SimpleKafka/KafkaConnection.cs index f9347615..fedce6d0 100644 --- a/src/SimpleKafka/KafkaConnection.cs +++ b/src/SimpleKafka/KafkaConnection.cs @@ -27,7 +27,7 @@ internal static async Task CreateAsync(IPEndPoint serverEndpoin private KafkaDecoder decoder; private KafkaEncoder encoder; - private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferSize = 65536) + private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferSize = 1048576) { this.serverEndpoint = serverEndpoint; this.client = client; diff --git a/src/SimpleKafka/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs index e06c2a0c..9a7c7df0 100644 --- a/src/SimpleKafka/KafkaConsumer.cs +++ b/src/SimpleKafka/KafkaConsumer.cs @@ -9,6 +9,21 @@ namespace SimpleKafka { + public static class KafkaConsumer + { + public static KafkaConsumer Create(string consumerGroup, KafkaBrokers brokers, + IKafkaSerializer valueSerializer, params TopicSelector[] topics) + { + return new KafkaConsumer(consumerGroup, brokers, new NullSerializer(), valueSerializer, topics); + } + public static KafkaConsumer Create(string consumerGroup, KafkaBrokers brokers, + IKafkaSerializer keySerializer, + IKafkaSerializer valueSerializer, params TopicSelector[] topics) + { + return new KafkaConsumer(consumerGroup, brokers, keySerializer, valueSerializer, topics); + } + } + public class KafkaConsumer { private class TopicTracker @@ -41,6 +56,8 @@ public TopicTracker(TopicSelector selector) private readonly int maxWaitTimeMs = 1000; private readonly int minBytes = 1024; private readonly int maxBytes = 65536; + private readonly string consumerId = "test"; + private readonly int consumerGroupGenerationId = 0; private readonly string consumerGroup; public KafkaConsumer(string consumerGroup, KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, params TopicSelector[] topics) @@ -76,8 +93,8 @@ public async Task CommitAsync(IEnumerable offsets, Cancell var request = new OffsetCommitRequest { ConsumerGroup = consumerGroup, - ConsumerGroupGenerationId = 0, - ConsumerId = "test", + ConsumerGroupGenerationId = consumerGroupGenerationId, + ConsumerId = consumerId, OffsetCommits = offsetCommits }; diff --git a/src/SimpleKafka/KafkaMessage.cs b/src/SimpleKafka/KafkaMessage.cs deleted file mode 100644 index 31960ebb..00000000 --- a/src/SimpleKafka/KafkaMessage.cs +++ /dev/null @@ -1,35 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Linq; -using System.Text; -using System.Threading.Tasks; - -namespace SimpleKafka -{ - public static class KafkaMessage - { - public static KafkaMessage Create(string topic, TKey key, TValue value) - { - return new KafkaMessage(topic, key, value); - } - } - - public class KafkaMessage - { - private readonly string topic; - public string Topic { get { return topic; } } - private readonly TKey key; - public TKey Key { get { return key; } } - private readonly TValue value; - public TValue Value { get { return value; } } - - public KafkaMessage(string topic, TKey key, TValue value) - { - this.topic = topic; - this.key = key; - this.value = value; - } - - - } -} diff --git a/src/SimpleKafka/KafkaProducer.cs b/src/SimpleKafka/KafkaProducer.cs index 8678fb72..5996ae07 100644 --- a/src/SimpleKafka/KafkaProducer.cs +++ b/src/SimpleKafka/KafkaProducer.cs @@ -8,31 +8,58 @@ namespace SimpleKafka { - public class KafkaProducer - { - private class MessageAndPartition - { - readonly KafkaMessage message; - readonly int partition; + public static class KafkaProducer { + public static KafkaProducer Create( + KafkaBrokers brokers, + IKafkaSerializer valueSerializer) { + return new KafkaProducer(brokers, + new NullSerializer(), + valueSerializer, + new LoadBalancedPartitioner()); + } - public MessageAndPartition(KafkaMessage message, int partition) - { + public static KafkaProducer Create( + KafkaBrokers brokers, + IKafkaSerializer keySerializer, + IKafkaSerializer valueSerializer + ) { + return new KafkaProducer( + brokers, + keySerializer, + valueSerializer, + new LoadBalancedPartitioner()); + } - this.message = message; - this.partition = partition; - } + public static KafkaProducer Create( + KafkaBrokers brokers, + IKafkaSerializer keySerializer, + IKafkaSerializer valueSerializer, + IKafkaMessagePartitioner partitioner) + { + return new KafkaProducer( + brokers, + keySerializer, + valueSerializer, + partitioner); } + } + + public class KafkaProducer + { + private readonly KafkaBrokers brokers; private readonly IKafkaSerializer keySerializer; private readonly IKafkaSerializer valueSerializer; - private readonly IKafkaMessagePartitioner messagePartitioner; + private readonly IKafkaMessagePartitioner messagePartitioner; private readonly int acks = 1; private readonly int timeoutMs = 10000; private readonly MessageCodec codec = MessageCodec.CodecNone; - public KafkaProducer(KafkaBrokers brokers, IKafkaSerializer keySerializer, IKafkaSerializer valueSerializer, - IKafkaMessagePartitioner messagePartitioner) + public KafkaProducer(KafkaBrokers brokers, + IKafkaSerializer keySerializer, + IKafkaSerializer valueSerializer, + IKafkaMessagePartitioner messagePartitioner) { this.brokers = brokers; this.keySerializer = keySerializer; @@ -40,104 +67,124 @@ public KafkaProducer(KafkaBrokers brokers, IKafkaSerializer keySerializer, this.messagePartitioner = messagePartitioner; } - public async Task SendAsync(KafkaMessage message, CancellationToken token) + public async Task SendAsync(KeyedMessage message, CancellationToken token) { - await SendAsync(new KafkaMessage[] { message }, token); + await SendAsync(new[] { message}, token).ConfigureAwait(false); } - public async Task SendAsync(IEnumerable> messages, CancellationToken token) + public async Task SendAsync(IEnumerable> messages, CancellationToken token) { var topicMap = BuildTopicMap(messages); - - while (topicMap.Count > 0) + while (topicMap != null) { - var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap).ConfigureAwait(false); + var partitionsMap = await brokers.GetValidPartitionsForTopicsAsync(topicMap.Keys, token).ConfigureAwait(false); + var brokerMap = BuildBrokerMap(topicMap, partitionsMap); + var results = await SendMessagesAsync(brokerMap, token).ConfigureAwait(false); - var completed = await SendMessagesToBrokersAsync(token, topicMap, brokerMap).ConfigureAwait(false); - if (!completed) + topicMap = ProcessResults(topicMap, brokerMap, results); + if (topicMap != null) { - var refreshed = await brokers.RefreshAsync(token).ConfigureAwait(false); - if (!refreshed) - { - throw new InvalidOperationException("Failed to refresh"); - } + await brokers.BackoffAndRefresh(token).ConfigureAwait(false); } - } } - private async Task SendMessagesToBrokersAsync(CancellationToken token, Dictionary>> topicMap, Dictionary, List>> brokerMap) + private Dictionary, List>>> BuildBrokerMap(Dictionary>> topicMap, Dictionary partitionsMap) { - foreach (var brokerKvp in brokerMap) + var brokerMap = new Dictionary, List>>>(); + foreach (var messageSet in topicMap.Values) { - var responses = await ProduceMessagesToBroker(brokerKvp.Key, brokerKvp.Value, token).ConfigureAwait(false); - foreach (var response in responses) + foreach (var message in messageSet) { - switch (response.Error) - { - case ErrorResponseCode.NoError: - var partitions = topicMap[response.Topic]; - partitions.Remove(response.PartitionId); - if (partitions.Count == 0) - { - topicMap.Remove(response.Topic); - } - break; - - - case ErrorResponseCode.LeaderNotAvailable: - case ErrorResponseCode.NotLeaderForPartition: - break; - - default: - throw new InvalidOperationException("Unhandled error " + response.Error + ", " + response.Topic + ":" + response.PartitionId); - } + var partitions = partitionsMap[message.Topic]; + var partitionId = messagePartitioner.CalculatePartition(message.PartitionKey, partitions.Length); + var partition = partitions[partitionId]; + + brokerMap.GetOrCreate(partition.LeaderId) + .GetOrCreate(Tuple.Create(message.Topic, partitionId)) + .Add(message); } } - return brokerMap.Count > 0; + return brokerMap; } - - private async Task> ProduceMessagesToBroker(int brokerId, Dictionary,List> topicMessages, CancellationToken token) + private async Task>> SendMessagesAsync(Dictionary, List>>> brokerMap, CancellationToken token) { - var payload = new List(topicMessages.Count); - foreach (var kvp in topicMessages) + var tasks = new List(brokerMap.Count); + var results = new Dictionary>(); + foreach (var brokerKvp in brokerMap) { - payload.Add(new Payload + var request = new ProduceRequest { - Topic = kvp.Key.Item1, - Partition = kvp.Key.Item2, - Codec = codec, - Messages = kvp.Value - }); + Acks = (short)acks, + TimeoutMS = timeoutMs, + Payload = brokerKvp.Value.Select(kvp => new Payload + { + Codec = codec, + Messages = kvp.Value.Select(m => m.Value).ToList(), + Partition = kvp.Key.Item2, + Topic = kvp.Key.Item1 + }).ToList() + }; + var brokerId = brokerKvp.Key; + tasks.Add( + brokers[brokerId] + .SendRequestAsync(request, token) + .ContinueWith(task => results.Add(brokerId, task.Result), token) + ); } - var request = new ProduceRequest + await Task.WhenAll(tasks).ConfigureAwait(false); + return results; + } + + private static Dictionary>> ProcessResults(Dictionary>> topicMap, Dictionary, List>>> brokerMap, Dictionary> results) + { + Dictionary>> remainingTopicMap = null; + + foreach (var kvp in results) { - Acks = (short)acks, - TimeoutMS = timeoutMs, - Payload = payload, - }; - var response = await brokers[brokerId].SendRequestAsync(request, token).ConfigureAwait(false); - return response; + var brokerId = kvp.Key; + var brokerMessages = brokerMap[brokerId]; + + foreach (var response in kvp.Value) + { + if (response.Error == ErrorResponseCode.NoError) + { + // nothing to do - success!! + } + else + { + if (remainingTopicMap == null) + { + remainingTopicMap = new Dictionary>>(); + } + remainingTopicMap + .GetOrCreate(response.Topic) + .AddRange(brokerMessages[Tuple.Create(response.Topic, response.PartitionId)]); + } + } + } + + topicMap = remainingTopicMap; + return topicMap; } - private Dictionary>> BuildTopicMap(IEnumerable> messages) + private Dictionary>> BuildTopicMap(IEnumerable> messages) { - var topicMap = new Dictionary>>(); + var result = new Dictionary>>(); + foreach (var message in messages) { - var partitionMap = topicMap.GetOrCreate(message.Topic); - var partition = messagePartitioner.CalculatePartition(message); - var messageList = partitionMap.GetOrCreate(partition); - var encodedMessage = new Message + var encoded = new Message { Key = keySerializer.Serialize(message.Key), - Value = valueSerializer.Serialize(message.Value), + Value = valueSerializer.Serialize(message.Value) }; - messageList.Add(encodedMessage); + var prepared = KeyedMessage.Create(message.Topic, (object)null, message.PartitionKey, encoded); + result.GetOrCreate(message.Topic).Add(prepared); } - return topicMap; + return result; } } } diff --git a/src/SimpleKafka/KeyedMessage.cs b/src/SimpleKafka/KeyedMessage.cs new file mode 100644 index 00000000..91595f59 --- /dev/null +++ b/src/SimpleKafka/KeyedMessage.cs @@ -0,0 +1,51 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public static class KeyedMessage + { + public static KeyedMessage Create(string topic, TValue value) + { + return new KeyedMessage(topic, value); + } + + public static KeyedMessage Create(string topic, TKey key, TValue value) + { + return new KeyedMessage(topic, key, key, value); + } + + public static KeyedMessage Create(string topic, TKey key, TPartitionKey partitionKey, TValue value) + { + return new KeyedMessage(topic, key, partitionKey, value); + } + } + + + public class KeyedMessage + { + public readonly string Topic; + public readonly TKey Key; + public readonly TPartitionKey PartitionKey; + public readonly TValue Value; + public readonly bool HasKey; + + internal KeyedMessage(string topic, TValue value) + { + this.Topic = topic; + this.Value = value; + } + + internal KeyedMessage(string topic, TKey key, TPartitionKey partitionKey, TValue value) + { + this.Topic = topic; + this.Value = value; + this.Key = key; + this.PartitionKey = partitionKey; + this.HasKey = true; + } + } +} diff --git a/src/SimpleKafka/LoadBalancedPartitioner.cs b/src/SimpleKafka/LoadBalancedPartitioner.cs index 630af3ae..cc8dda33 100644 --- a/src/SimpleKafka/LoadBalancedPartitioner.cs +++ b/src/SimpleKafka/LoadBalancedPartitioner.cs @@ -6,18 +6,16 @@ namespace SimpleKafka { - public class LoadBalancedPartitioner : IKafkaMessagePartitioner + public class LoadBalancedPartitioner : IKafkaMessagePartitioner { - private readonly int numberOfPartitions; - public LoadBalancedPartitioner(int numberOfPartitions) - { - this.numberOfPartitions = numberOfPartitions; - } - private int current; - public int CalculatePartition(KafkaMessage message) + public int CalculatePartition(TPartitionKey partitionKey, int numberOfPartitions) { + if (current >= numberOfPartitions) + { + current = 0; + } var partition = current; current = (current + 1) % numberOfPartitions; return partition; diff --git a/src/SimpleKafka/NullSerializer.cs b/src/SimpleKafka/NullKeySerializer.cs similarity index 100% rename from src/SimpleKafka/NullSerializer.cs rename to src/SimpleKafka/NullKeySerializer.cs diff --git a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs index eca5f4de..fdaa6101 100644 --- a/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs +++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs @@ -9,7 +9,7 @@ namespace SimpleKafka.Protocol /// The offsets for a given consumer group is maintained by a specific broker called the offset coordinator. i.e., a consumer needs /// to issue its offset commit and fetch requests to this specific broker. It can discover the current offset coordinator by issuing a consumer metadata request. /// - public class ConsumerMetadataRequest : BaseRequest, IKafkaRequest + public class ConsumerMetadataRequest : BaseRequest { public string ConsumerGroup { get; set; } diff --git a/src/SimpleKafka/Protocol/FetchRequest.cs b/src/SimpleKafka/Protocol/FetchRequest.cs index 47509684..9fd0e47c 100644 --- a/src/SimpleKafka/Protocol/FetchRequest.cs +++ b/src/SimpleKafka/Protocol/FetchRequest.cs @@ -5,7 +5,7 @@ namespace SimpleKafka.Protocol { - public class FetchRequest : BaseRequest>, IKafkaRequest + public class FetchRequest : BaseRequest> { internal const int DefaultMinBlockingByteBufferSize = 4096; internal const int DefaultBufferSize = DefaultMinBlockingByteBufferSize * 8; diff --git a/src/SimpleKafka/Protocol/Message.cs b/src/SimpleKafka/Protocol/Message.cs index 1a99572e..a14c6816 100644 --- a/src/SimpleKafka/Protocol/Message.cs +++ b/src/SimpleKafka/Protocol/Message.cs @@ -18,7 +18,7 @@ public Payload() public string Topic { get; set; } public int Partition { get; set; } public MessageCodec Codec { get; set; } - public List Messages { get; set; } + public IList Messages { get; set; } } /// diff --git a/src/SimpleKafka/Protocol/MetadataRequest.cs b/src/SimpleKafka/Protocol/MetadataRequest.cs index fa8a201d..91617df6 100644 --- a/src/SimpleKafka/Protocol/MetadataRequest.cs +++ b/src/SimpleKafka/Protocol/MetadataRequest.cs @@ -3,7 +3,7 @@ namespace SimpleKafka.Protocol { - public class MetadataRequest : BaseRequest, IKafkaRequest + public class MetadataRequest : BaseRequest { /// /// The list of topics to get metadata for. diff --git a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs index f8b6bd1c..0c04e9a6 100644 --- a/src/SimpleKafka/Protocol/OffsetCommitRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs @@ -11,7 +11,7 @@ namespace SimpleKafka.Protocol /// arbitrary ConsumerGroup name provided by the call. /// This now supports version 0 and 1 of the protocol /// - public class OffsetCommitRequest : BaseRequest>, IKafkaRequest + public class OffsetCommitRequest : BaseRequest> { public OffsetCommitRequest(Int16 version = 1) : base(ApiKeyRequestType.OffsetCommit, version) diff --git a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs index 563debf8..aca77fb8 100644 --- a/src/SimpleKafka/Protocol/OffsetFetchRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs @@ -12,7 +12,7 @@ namespace SimpleKafka.Protocol /// under any abritrary name. /// This now supports version 1 of the protocol /// - public class OffsetFetchRequest : BaseRequest>, IKafkaRequest + public class OffsetFetchRequest : BaseRequest> { public OffsetFetchRequest(short version = 1) : base(ApiKeyRequestType.OffsetFetch, version) { diff --git a/src/SimpleKafka/Protocol/OffsetRequest.cs b/src/SimpleKafka/Protocol/OffsetRequest.cs index 39e87258..19e1ba35 100644 --- a/src/SimpleKafka/Protocol/OffsetRequest.cs +++ b/src/SimpleKafka/Protocol/OffsetRequest.cs @@ -8,7 +8,7 @@ namespace SimpleKafka.Protocol /// /// A funky Protocol for requesting the starting offset of each segment for the requested partition /// - public class OffsetRequest : BaseRequest>, IKafkaRequest + public class OffsetRequest : BaseRequest> { public List Offsets { get; set; } diff --git a/src/SimpleKafka/Protocol/ProduceRequest.cs b/src/SimpleKafka/Protocol/ProduceRequest.cs index 79494fd2..6dfa7932 100644 --- a/src/SimpleKafka/Protocol/ProduceRequest.cs +++ b/src/SimpleKafka/Protocol/ProduceRequest.cs @@ -5,7 +5,7 @@ namespace SimpleKafka.Protocol { - public class ProduceRequest : BaseRequest>, IKafkaRequest + public class ProduceRequest : BaseRequest> { /// /// Provide a hint to the broker call not to expect a response for requests without Acks. diff --git a/src/SimpleKafka/ReceivedKafkaMessage.cs b/src/SimpleKafka/ReceivedKafkaMessage.cs index b49a82fa..dcc08747 100644 --- a/src/SimpleKafka/ReceivedKafkaMessage.cs +++ b/src/SimpleKafka/ReceivedKafkaMessage.cs @@ -6,17 +6,26 @@ namespace SimpleKafka { - public class ReceivedKafkaMessage : KafkaMessage + public class ReceivedKafkaMessage { private readonly int partition; public int Partition { get { return partition; } } private readonly long offset; public long Offset { get { return offset; } } + private readonly TKey key; + public TKey Key { get { return key; } } + private readonly string topic; + public string Topic { get { return topic; } } + private readonly TValue value; + public TValue Value { get { return value; } } - public ReceivedKafkaMessage(string topic, TKey key, TValue value, int partition, long offset) : base(topic, key, value) + public ReceivedKafkaMessage(string topic, TKey key, TValue value, int partition, long offset) { this.partition = partition; this.offset = offset; + this.key = key; + this.topic = topic; + this.value = value; } } diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index e60467d4..f0be7de0 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -47,6 +47,7 @@ + @@ -55,14 +56,13 @@ - - + - + @@ -85,6 +85,9 @@ + + + diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs index 33751d6a..55c63e75 100644 --- a/src/SimpleKafkaTests/Integration/ConsumerTests.cs +++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs @@ -24,18 +24,19 @@ public void Setup() [Test] public async Task TestSimpleConsumerWorksOk() { - var keySerializer = new NullSerializer(); + var keySerializer = new NullSerializer(); var valueSerializer = new StringSerializer(); - var messagePartitioner = new LoadBalancedPartitioner(1); + var messagePartitioner = new LoadBalancedPartitioner(); using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) { var topic = temporaryTopic.Topic; - var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); - var consumer = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); + var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); + var consumer = KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic }); - await producer.SendAsync(new KafkaMessage(topic, null, "Message"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(topic, "Message"), CancellationToken.None).ConfigureAwait(true); var responses = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); Assert.That(responses, Is.Not.Null); @@ -53,26 +54,24 @@ public async Task TestSimpleConsumerWorksOk() [Test] public async Task TestProducing3MessagesAllowsTheConsumerToChooseTheCorrectMessage() { - var keySerializer = new NullSerializer(); var valueSerializer = new StringSerializer(); - var messagePartitioner = new LoadBalancedPartitioner(1); using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) { var topic = temporaryTopic.Topic; { - var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + var producer = KafkaProducer.Create(brokers, valueSerializer); await producer.SendAsync(new[] { - KafkaMessage.Create(topic, (string)null, "1"), - KafkaMessage.Create(topic, (string)null, "2"), - KafkaMessage.Create(topic, (string)null, "3"), + KeyedMessage.Create(topic, "1"), + KeyedMessage.Create(topic, "2"), + KeyedMessage.Create(topic, "3"), }, CancellationToken.None).ConfigureAwait(true); } { - var earliest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var earliest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Earliest }); var responses = await earliest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); @@ -88,7 +87,7 @@ await producer.SendAsync(new[] { } { - var latest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var latest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Last }); var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); @@ -104,7 +103,7 @@ await producer.SendAsync(new[] { } { - var latest = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var latest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Next }); var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); @@ -114,7 +113,7 @@ await producer.SendAsync(new[] { } { - var specified = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var specified = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Specified, Offset = 1 }); var responses = await specified.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); @@ -136,26 +135,24 @@ await producer.SendAsync(new[] { [Test] public async Task TestProducing3MessagesAllowsTheConsumerToCommitAndRestart() { - var keySerializer = new NullSerializer(); var valueSerializer = new StringSerializer(); - var messagePartitioner = new LoadBalancedPartitioner(1); using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) { var topic = temporaryTopic.Topic; { - var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + var producer = KafkaProducer.Create(brokers, valueSerializer); await producer.SendAsync(new[] { - KafkaMessage.Create(topic, (string)null, "1"), - KafkaMessage.Create(topic, (string)null, "2"), - KafkaMessage.Create(topic, (string)null, "3"), + KeyedMessage.Create(topic, "1"), + KeyedMessage.Create(topic, "2"), + KeyedMessage.Create(topic, "3"), }, CancellationToken.None).ConfigureAwait(true); } { - var noPreviousCommits = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var noPreviousCommits = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); @@ -177,7 +174,7 @@ await noPreviousCommits.CommitAsync(new[] { } { - var previousCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var previousCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, @@ -200,7 +197,7 @@ await noPreviousCommits.CommitAsync(new[] { } { - var previousCommitAgain = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var previousCommitAgain = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, @@ -226,7 +223,7 @@ await previousCommitAgain.CommitAsync(new[] { } { - var secondCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var secondCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, @@ -252,7 +249,7 @@ await secondCommit.CommitAsync(new[] { } { - var thirdCommit = new KafkaConsumer(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + var thirdCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 967feb32..48ebda7b 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -464,15 +464,13 @@ public async Task TestSimpleKafkaBrokerWorksOk() [Test] public async Task TestSimpleProducerWorksOk() { - var keySerializer = new NullSerializer(); var valueSerializer = new StringSerializer(); - var messagePartitioner = new LoadBalancedPartitioner(1); using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUri)) { - var producer = new KafkaProducer(brokers, keySerializer, valueSerializer, messagePartitioner); + var producer = KafkaProducer.Create(brokers, valueSerializer); - await producer.SendAsync(new KafkaMessage(IntegrationConfig.IntegrationTopic, null, "Message"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(IntegrationConfig.IntegrationTopic, "Message"), CancellationToken.None).ConfigureAwait(true); } diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index da400b95..1a38c7d0 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -69,6 +69,9 @@ SimpleKafka + + + diff --git a/src/kafka-net.sln b/src/kafka-net.sln index 25641f9a..44196427 100644 --- a/src/kafka-net.sln +++ b/src/kafka-net.sln @@ -1,7 +1,7 @@  Microsoft Visual Studio Solution File, Format Version 12.00 -# Visual Studio 14 -VisualStudioVersion = 14.0.22609.0 +# Visual Studio 2013 +VisualStudioVersion = 12.0.31101.0 MinimumVisualStudioVersion = 10.0.40219.1 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "kafka-net", "kafka-net\kafka-net.csproj", "{1343EB68-55CB-4452-8386-24A9989DE1C0}" EndProject @@ -37,15 +37,12 @@ Global {1343EB68-55CB-4452-8386-24A9989DE1C0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {1343EB68-55CB-4452-8386-24A9989DE1C0}.Debug|Any CPU.Build.0 = Debug|Any CPU {1343EB68-55CB-4452-8386-24A9989DE1C0}.Release|Any CPU.ActiveCfg = Release|Any CPU - {1343EB68-55CB-4452-8386-24A9989DE1C0}.Release|Any CPU.Build.0 = Release|Any CPU {D80AE407-BB81-4C11-BFDC-5DD463F8B1BF}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {D80AE407-BB81-4C11-BFDC-5DD463F8B1BF}.Debug|Any CPU.Build.0 = Debug|Any CPU {D80AE407-BB81-4C11-BFDC-5DD463F8B1BF}.Release|Any CPU.ActiveCfg = Release|Any CPU - {D80AE407-BB81-4C11-BFDC-5DD463F8B1BF}.Release|Any CPU.Build.0 = Release|Any CPU {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Debug|Any CPU.Build.0 = Debug|Any CPU {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Release|Any CPU.ActiveCfg = Release|Any CPU - {53E0B3CE-6C41-4C8A-8B66-9BD03667B1E0}.Release|Any CPU.Build.0 = Release|Any CPU {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Debug|Any CPU.ActiveCfg = Debug|Any CPU {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Debug|Any CPU.Build.0 = Debug|Any CPU {9B7B9E32-45E6-40C0-B6FA-9F0A99681B16}.Release|Any CPU.ActiveCfg = Release|Any CPU From 37a736b59fb64f21f82608e2316ee7f4abfc05f4 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Tue, 14 Apr 2015 16:02:07 +0100 Subject: [PATCH 10/14] Tests for multiple partitions --- src/SimpleKafka/Int32Partitioner.cs | 16 +++ src/SimpleKafka/Int32Serializer.cs | 24 ++++ src/SimpleKafka/SimpleKafka.csproj | 2 + .../Integration/MultiplePartitionTests.cs | 132 ++++++++++++++++++ src/SimpleKafkaTests/SimpleKafkaTests.csproj | 1 + 5 files changed, 175 insertions(+) create mode 100644 src/SimpleKafka/Int32Partitioner.cs create mode 100644 src/SimpleKafka/Int32Serializer.cs create mode 100644 src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs diff --git a/src/SimpleKafka/Int32Partitioner.cs b/src/SimpleKafka/Int32Partitioner.cs new file mode 100644 index 00000000..6d49f3cb --- /dev/null +++ b/src/SimpleKafka/Int32Partitioner.cs @@ -0,0 +1,16 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class Int32Partitioner : IKafkaMessagePartitioner + { + public int CalculatePartition(int partitionKey, int numberOfPartitions) + { + return partitionKey % numberOfPartitions; + } + } +} diff --git a/src/SimpleKafka/Int32Serializer.cs b/src/SimpleKafka/Int32Serializer.cs new file mode 100644 index 00000000..e0fa3fad --- /dev/null +++ b/src/SimpleKafka/Int32Serializer.cs @@ -0,0 +1,24 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class Int32Serializer : IKafkaSerializer + { + public byte[] Serialize(int value) + { + var buffer = new byte[4]; + var encoder = new KafkaEncoder(buffer); + encoder.Write(value); + return buffer; + } + + public int Deserialize(byte[] serialized) + { + return new KafkaDecoder(serialized).ReadInt32(); + } + } +} diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index f0be7de0..4f2f7595 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -48,6 +48,8 @@ + + diff --git a/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs new file mode 100644 index 00000000..8f75cc2f --- /dev/null +++ b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs @@ -0,0 +1,132 @@ +using NUnit.Framework; +using SimpleKafka; +using SimpleKafkaTests.Helpers; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Integration +{ + [TestFixture] + [Category("Integration")] + class MultiplePartitionTests + { + private readonly string defaultConsumerGroup = "unit-tests"; + + [SetUp] + public void Setup() + { + IntegrationHelpers.zookeeperHost = "server.home:32181"; + } + + [Test] + public async Task ProduceToTwoPartitions() + { + var keySerializer = new Int32Serializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new Int32Partitioner(); + + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic(partitions:2)) + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + { + var topic = temporaryTopic.Topic; + var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); + var consumers = new[] { + KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 0, Topic = topic }), + KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, + new TopicSelector { Partition = 1, Topic = topic }) + }; + + + await producer.SendAsync(new[] { + KeyedMessage.Create(topic, 0, "Message to partition 0"), + KeyedMessage.Create(topic, 1, "Message to partition 1") + }, CancellationToken.None).ConfigureAwait(true); + + for (var i = 0; i < consumers.Length; i++) + { + var responses = await consumers[i].ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Is.Not.Null); + Assert.That(responses, Has.Count.EqualTo(1)); + + var first = responses.First(); + Assert.That(first.Offset, Is.EqualTo(0)); + Assert.That(first.Partition, Is.EqualTo(i)); + Assert.That(first.Key, Is.EqualTo(i)); + Assert.That(first.Topic, Is.EqualTo(topic)); + Assert.That(first.Value, Is.EqualTo("Message to partition " + i)); + } + } + } + + [Theory] + [TestCase(1, 1, 1)] + [TestCase(1, 2, 2)] + [TestCase(1, 2, 4)] + [TestCase(2, 1, 1)] + [TestCase(2, 2, 2)] + [TestCase(2, 2, 4)] + public async Task ProduceToMultiplePartitions(int numberOfPartitions, int numberOfKeys, int numberOfMessages) + { + var keySerializer = new Int32Serializer(); + var valueSerializer = new StringSerializer(); + var messagePartitioner = new Int32Partitioner(); + + using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic(partitions: numberOfPartitions)) + using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + { + var topic = temporaryTopic.Topic; + { + var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); + var messages = + Enumerable + .Range(0, numberOfMessages) + .Select(i => KeyedMessage.Create(topic, i % numberOfKeys, i % numberOfPartitions, "Message " + i)); + await producer.SendAsync(messages, CancellationToken.None).ConfigureAwait(true); + } + + { + var selectors = + Enumerable + .Range(0, numberOfPartitions) + .Select(partition => new TopicSelector { Partition = partition, Topic = topic }) + .ToArray(); + var consumer = KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, selectors); + + var responses = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + Assert.That(responses, Has.Count.EqualTo(numberOfMessages)); + var received = new bool[numberOfMessages]; + var offsets = new long[numberOfPartitions]; + foreach (var response in responses) + { + var split = response.Value.Split(' '); + Assert.That(split, Has.Length.EqualTo(2)); + Assert.That(split[0], Is.EqualTo("Message")); + int messageNumber; + var parsed = Int32.TryParse(split[1], out messageNumber); + Assert.That(parsed, Is.True); + Assert.That(messageNumber, Is.InRange(0, numberOfMessages - 1)); + var key = messageNumber % numberOfKeys; + Assert.That(response.Key, Is.EqualTo(key)); + + var partition = messageNumber % numberOfPartitions; + Assert.That(response.Partition, Is.EqualTo(partition)); + + Assert.That(received[messageNumber], Is.False); + received[messageNumber] = true; + + Assert.That(response.Offset, Is.EqualTo(offsets[response.Partition])); + offsets[response.Partition] += 1; + + Assert.That(response.Topic, Is.EqualTo(topic)); + + } + } + } + } + } +} diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index 1a38c7d0..00605938 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -49,6 +49,7 @@ + From 04fb56ae3fa2f3b7c0f7ef22d59482e07161f964 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 20 Apr 2015 08:29:11 +0100 Subject: [PATCH 11/14] Added more tests, used docker to create clusters --- src/SimpleKafka/BackoffHandler.cs | 43 +++ src/SimpleKafka/Common/Crc32Provider.cs | 5 - src/SimpleKafka/KafkaBrokers.cs | 108 +++---- src/SimpleKafka/KafkaConsumer.cs | 71 ++++- src/SimpleKafka/KafkaProducer.cs | 45 ++- src/SimpleKafka/SimpleKafka.csproj | 2 + src/SimpleKafka/StringPartitioner.cs | 16 ++ .../Helpers/IntegrationConfig.cs | 12 +- .../Helpers/KafkaTestCluster.cs | 263 ++++++++++++++++++ .../Integration/ConsumerTests.cs | 36 ++- .../Integration/FailureTests.cs | 144 ++++++++++ .../Integration/IntegrationHelpers.cs | 72 ----- .../Integration/MultiplePartitionTests.cs | 27 +- .../Integration/SimpleTests.cs | 223 ++++++++++++--- src/SimpleKafkaTests/SimpleKafkaTests.csproj | 8 +- src/SimpleKafkaTests/Unit/PartitionerTests.cs | 52 ++++ src/kafka-net.sln | 1 + 17 files changed, 877 insertions(+), 251 deletions(-) create mode 100644 src/SimpleKafka/BackoffHandler.cs create mode 100644 src/SimpleKafka/StringPartitioner.cs create mode 100644 src/SimpleKafkaTests/Helpers/KafkaTestCluster.cs create mode 100644 src/SimpleKafkaTests/Integration/FailureTests.cs delete mode 100644 src/SimpleKafkaTests/Integration/IntegrationHelpers.cs create mode 100644 src/SimpleKafkaTests/Unit/PartitionerTests.cs diff --git a/src/SimpleKafka/BackoffHandler.cs b/src/SimpleKafka/BackoffHandler.cs new file mode 100644 index 00000000..75a973a7 --- /dev/null +++ b/src/SimpleKafka/BackoffHandler.cs @@ -0,0 +1,43 @@ +using Serilog; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + internal class BackoffHandler + { + private static readonly Random generator = new Random(); + + private readonly int maxRetries; + private readonly int basePeriodMs; + private readonly int jitterMs; + private int backoffCount; + + public BackoffHandler(int maxRetries = 10, int basePeriodMs = 300, int jitterMs = 50) + { + this.maxRetries = maxRetries; + this.basePeriodMs = basePeriodMs; + this.jitterMs = jitterMs; + } + + public async Task BackoffIfAllowedAsync(CancellationToken token) + { + if (++backoffCount >= maxRetries) + { + return false; + } + else + { + Log.Verbose("Backoff {attempt} out of {maxRetries}", backoffCount, maxRetries); + var delay = generator.Next(basePeriodMs - jitterMs, basePeriodMs + jitterMs); + await Task.Delay(delay, token).ConfigureAwait(false); + return true; + } + + } + } +} diff --git a/src/SimpleKafka/Common/Crc32Provider.cs b/src/SimpleKafka/Common/Crc32Provider.cs index 53e31187..3e2ef77d 100644 --- a/src/SimpleKafka/Common/Crc32Provider.cs +++ b/src/SimpleKafka/Common/Crc32Provider.cs @@ -23,11 +23,6 @@ static Crc32Provider() PolynomialTable = InitializeTable(DefaultPolynomial); } - public static UInt32 Compute(byte[] buffer) - { - return ~CalculateHash(buffer, 0, buffer.Length); - } - public static UInt32 Compute(byte[] buffer, int offset, int length) { return ~CalculateHash(buffer, offset, length); diff --git a/src/SimpleKafka/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs index 9bafac89..b51f467f 100644 --- a/src/SimpleKafka/KafkaBrokers.cs +++ b/src/SimpleKafka/KafkaBrokers.cs @@ -11,12 +11,27 @@ namespace SimpleKafka { public class KafkaBrokers : IDisposable { - private readonly Random backoffGenerator = new Random(); private readonly HashSet brokers = new HashSet(); private readonly Dictionary topicToPartitions = new Dictionary(StringComparer.CurrentCultureIgnoreCase); private readonly Dictionary connections = new Dictionary(); - public KafkaConnection this[int brokerId] + + public async Task RunBrokerCommand(int brokerId, Func> operation) + { + var connection = this[brokerId]; + try + { + return await operation(connection).ConfigureAwait(false); + } + catch (Exception) + { + connection.Dispose(); + connections.Remove(brokerId); + throw; + } + } + + private KafkaConnection this[int brokerId] { get { @@ -38,20 +53,6 @@ public KafkaBrokers(params Uri[] addresses) } - private bool IsLeaderElectionTakingPlaceForTopicAndPartition(string topic, int partition) - { - var partitionsMap = topicToPartitions.TryGetValue(topic); - if (partitionsMap == null) - { - return false; - } - else - { - var partitionInfo = partitionsMap[partition]; - return partitionInfo.LeaderId == -1; - } - } - public async Task RefreshAsync(CancellationToken token) { if (brokers.Count == 0) @@ -131,16 +132,15 @@ public async Task> BuildOffsetCoordinatorMapAsync(Cancel public async Task> GetValidPartitionsForTopicsAsync(IEnumerable topics, CancellationToken token) { - while (true) + var result = await GetPartitionsForTopicsAsync(topics, token).ConfigureAwait(false); + var anyWithElection = result.Values.Any(partitions => partitions.Any(partition => partition.LeaderId == -1)); + if (!anyWithElection) { - var result = await GetPartitionsForTopicsAsync(topics, token).ConfigureAwait(false); - var anyWithElection = result.Values.Any(partitions => partitions.Any(partition => partition.LeaderId == -1)); - if (!anyWithElection) - { - return result; - } - - await BackoffAndRefresh(token).ConfigureAwait(false); + return result; + } + else + { + throw new SimpleKafka.Protocol.LeaderNotFoundException("Not all leaders found"); } } @@ -175,30 +175,11 @@ public async Task GetPartitionsForTopicAsync(string topic, Cancella return partitions; } - public async Task GetLeaderForTopicAndPartitionAsync(string topic, int partitionId, CancellationToken token) - { - while (true) - { - var partitions = await GetPartitionsForTopicAsync(topic, token).ConfigureAwait(false); - var partition = GetPartitionIfReady(topic, partitionId, partitions); - - if (partition != null) - { - return partition.LeaderId; - } - else - { - token.ThrowIfCancellationRequested(); - await BackoffAndRefresh(token).ConfigureAwait(false); - } - } - } - private static Partition GetPartitionIfReady(string topic, int partitionId, Partition[] partitions) { if (partitionId >= partitions.Length) { - throw new IndexOutOfRangeException("Topic " + topic + ", partition " + partitionId + " is too big. Only have " + partitions.Length + " partitions"); + throw new InvalidPartitionException("Topic {0} partition {1} is too big. Only have {2} partitions", topic, partitionId, partitions.Length); } var partition = partitions[partitionId]; @@ -220,6 +201,8 @@ public async Task, T>>> BuildBroke await RefreshAsync(token).ConfigureAwait(false); } + var backoffs = new BackoffHandler(); + var ready = false; while (!ready) { @@ -247,7 +230,11 @@ public async Task, T>>> BuildBroke if (!ready) { - await BackoffAndRefresh(token).ConfigureAwait(false); + if (!await backoffs.BackoffIfAllowedAsync(token).ConfigureAwait(false)) + { + throw new InvalidPartitionException("Cannot build broker map"); + } + await RefreshAsync(token); } } @@ -268,16 +255,6 @@ public async Task, T>>> BuildBroke return brokerMap; } - public async Task BackoffAndRefresh(CancellationToken token) - { - Log.Verbose("Waiting before trying again"); - await Task.Delay(backoffGenerator.Next(1000, 10000)).ConfigureAwait(false); - var refreshed = await RefreshAsync(token).ConfigureAwait(false); - if (!refreshed) - { - throw new KeyNotFoundException("Failed to refresh brokers"); - } - } internal Partition[] GetPartitionsForTopic(string topic) { @@ -364,17 +341,12 @@ private void RefreshTopics(Topic[] topics) private async Task RefreshBrokersAsync(Broker[] latestBrokers, CancellationToken token) { - var previousBrokers = new HashSet(brokers); - var previousConnections = connections.ToDictionary(kvp => kvp.Key, kvp => kvp.Value); foreach (var broker in latestBrokers) { var uri = broker.Address; if (!brokers.Contains(uri)) { brokers.Add(uri); - } else - { - previousBrokers.Remove(uri); } var currentConnection = connections.TryGetValue(broker.BrokerId); @@ -382,22 +354,8 @@ private async Task RefreshBrokersAsync(Broker[] latestBrokers, CancellationToken { var newConnection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(uri, token).ConfigureAwait(false); connections.Add(broker.BrokerId, newConnection); - } else - { - previousConnections.Remove(broker.BrokerId); } } - - foreach (var oldBroker in previousBrokers) - { - brokers.Remove(oldBroker); - } - - foreach (var oldConnectionKvp in previousConnections) - { - connections.Remove(oldConnectionKvp.Key); - oldConnectionKvp.Value.Dispose(); - } } public override string ToString() diff --git a/src/SimpleKafka/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs index 9a7c7df0..9d2f85e9 100644 --- a/src/SimpleKafka/KafkaConsumer.cs +++ b/src/SimpleKafka/KafkaConsumer.cs @@ -80,7 +80,7 @@ public KafkaConsumer(string consumerGroup, KafkaBrokers brokers, IKafkaSerialize public async Task CommitAsync(IEnumerable offsets, CancellationToken token) { - var coordinator = await GetOffsetCoordinatorConnectionAsync(token).ConfigureAwait(false); + var coordinatorId = await GetOffsetCoordinatorIdAsync(token).ConfigureAwait(false); var offsetCommits = new List(); foreach (var offset in offsets) { var offsetCommit = new OffsetCommit { @@ -98,7 +98,8 @@ public async Task CommitAsync(IEnumerable offsets, Cancell OffsetCommits = offsetCommits }; - var responses = await coordinator.SendRequestAsync(request, token).ConfigureAwait(false); + var responses = await brokers.RunBrokerCommand(coordinatorId, c => + c.SendRequestAsync(request, token)).ConfigureAwait(false); foreach (var response in responses) { if (response.Error != ErrorResponseCode.NoError) @@ -108,21 +109,59 @@ public async Task CommitAsync(IEnumerable offsets, Cancell } } - private async Task GetOffsetCoordinatorConnectionAsync(CancellationToken token) + private async Task GetOffsetCoordinatorIdAsync(CancellationToken token) { var map = await brokers.BuildOffsetCoordinatorMapAsync(token, consumerGroup).ConfigureAwait(false); var coordinator = map[consumerGroup]; - return brokers[coordinator]; + return coordinator; } public async Task>> ReceiveAsync(CancellationToken token) { - var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap).ConfigureAwait(false); - await RetrieveAnyTopicOffsets(token, brokerMap).ConfigureAwait(false); - await RetrieveAnyConsumerOffsets(token, brokerMap).ConfigureAwait(false); - var tasks = CreateFetchTasks(token, brokerMap); - var taskResults = await Task.WhenAll(tasks).ConfigureAwait(false); + var backoffs = new BackoffHandler(); + while (true) + { + Exception caughtException = null; + + try + { + var brokerMap = await brokers.BuildBrokerMapAsync(token, topicMap).ConfigureAwait(false); + await RetrieveAnyTopicOffsets(token, brokerMap).ConfigureAwait(false); + await RetrieveAnyConsumerOffsets(token, brokerMap).ConfigureAwait(false); + var tasks = CreateFetchTasks(token, brokerMap); + var taskResults = await Task.WhenAll(tasks).ConfigureAwait(false); + + var messages = DecodeResults(taskResults); + return messages; + } + catch (OperationCanceledException) + { + throw; + } + catch (Exception ex) + { + Log.Information("Error receiving: {ex}", ex); + caughtException = ex; + } + token.ThrowIfCancellationRequested(); + + if (!await backoffs.BackoffIfAllowedAsync(token).ConfigureAwait(false)) + { + if (caughtException != null) + { + throw caughtException; + } + else + { + throw new ResponseTimeoutException("Error receiving"); + } + } + await brokers.RefreshAsync(token).ConfigureAwait(false); + } + } + private List> DecodeResults(List[] taskResults) + { var messages = new List>(); foreach (var taskResult in taskResults) { @@ -131,12 +170,13 @@ public async Task>> ReceiveAsync(Cancella if (fetchResponse.Error != (int)ErrorResponseCode.NoError) { Log.Error("Error in fetch response {error} for {topic}/{partition}", fetchResponse.Error, fetchResponse.Topic, fetchResponse.PartitionId); - } else + } + else { var tracker = topicMap[fetchResponse.Topic][fetchResponse.PartitionId]; foreach (var message in fetchResponse.Messages) { - var result = new ReceivedKafkaMessage( + var result = new ReceivedKafkaMessage( fetchResponse.Topic, keySerializer.Deserialize(message.Key), valueSerializer.Deserialize(message.Value), @@ -162,7 +202,8 @@ private List>> CreateFetchTasks(CancellationToken token var trackerMap = brokerKvp.Value; var request = CreateRequest(trackerMap); - tasks.Add(brokers[brokerId].SendRequestAsync(request, token)); + tasks.Add(brokers.RunBrokerCommand(brokerId, c => + c.SendRequestAsync(request, token))); } return tasks; @@ -229,7 +270,7 @@ private async Task RetrieveAnyTopicOffsets(CancellationToken token, Dictionary c.SendRequestAsync(request, token)).ConfigureAwait(false); foreach (var response in responses) { if (response.Error != ErrorResponseCode.NoError) @@ -281,8 +322,8 @@ private async Task RetrieveAnyConsumerOffsets(CancellationToken token, Dictionar if (fetches != null) { var request = new OffsetFetchRequest { ConsumerGroup = consumerGroup, Topics = fetches }; - var coordinator = await GetOffsetCoordinatorConnectionAsync(token).ConfigureAwait(false); - var responses = await coordinator.SendRequestAsync(request, token).ConfigureAwait(false); + var coordinatorId = await GetOffsetCoordinatorIdAsync(token).ConfigureAwait(false); + var responses = await brokers.RunBrokerCommand(coordinatorId, c => c.SendRequestAsync(request, token)).ConfigureAwait(false); foreach (var response in responses) { if (response.Error != ErrorResponseCode.NoError) diff --git a/src/SimpleKafka/KafkaProducer.cs b/src/SimpleKafka/KafkaProducer.cs index 5996ae07..050d99cd 100644 --- a/src/SimpleKafka/KafkaProducer.cs +++ b/src/SimpleKafka/KafkaProducer.cs @@ -1,4 +1,5 @@ -using SimpleKafka.Protocol; +using Serilog; +using SimpleKafka.Protocol; using System; using System.Collections.Generic; using System.Linq; @@ -74,17 +75,44 @@ public async Task SendAsync(KeyedMessage message, C public async Task SendAsync(IEnumerable> messages, CancellationToken token) { + var backoffs = new BackoffHandler(); var topicMap = BuildTopicMap(messages); while (topicMap != null) { - var partitionsMap = await brokers.GetValidPartitionsForTopicsAsync(topicMap.Keys, token).ConfigureAwait(false); - var brokerMap = BuildBrokerMap(topicMap, partitionsMap); - var results = await SendMessagesAsync(brokerMap, token).ConfigureAwait(false); + Exception caughtException = null; + try + { + var partitionsMap = await brokers.GetValidPartitionsForTopicsAsync(topicMap.Keys, token).ConfigureAwait(false); + var brokerMap = BuildBrokerMap(topicMap, partitionsMap); + var results = await SendMessagesAsync(brokerMap, token).ConfigureAwait(false); - topicMap = ProcessResults(topicMap, brokerMap, results); + topicMap = ProcessResults(topicMap, brokerMap, results); + } + catch (OperationCanceledException) + { + throw; + } + catch (Exception ex) + { + Log.Information("Error sending: {ex}", ex); + caughtException = ex; + token.ThrowIfCancellationRequested(); + } if (topicMap != null) { - await brokers.BackoffAndRefresh(token).ConfigureAwait(false); + var canContinue = await backoffs.BackoffIfAllowedAsync(token).ConfigureAwait(false); + if (!canContinue) + { + if (caughtException != null) + { + throw caughtException; + } + else + { + throw new ResponseTimeoutException("Timeout sending"); + } + } + await brokers.RefreshAsync(token).ConfigureAwait(false); } } } @@ -129,8 +157,9 @@ private async Task>> SendMessagesAsync(Dic }; var brokerId = brokerKvp.Key; tasks.Add( - brokers[brokerId] - .SendRequestAsync(request, token) + brokers + .RunBrokerCommand(brokerId, c => + c.SendRequestAsync(request, token)) .ContinueWith(task => results.Add(brokerId, task.Result), token) ); } diff --git a/src/SimpleKafka/SimpleKafka.csproj b/src/SimpleKafka/SimpleKafka.csproj index 4f2f7595..7efd21aa 100644 --- a/src/SimpleKafka/SimpleKafka.csproj +++ b/src/SimpleKafka/SimpleKafka.csproj @@ -47,6 +47,7 @@ + @@ -80,6 +81,7 @@ + diff --git a/src/SimpleKafka/StringPartitioner.cs b/src/SimpleKafka/StringPartitioner.cs new file mode 100644 index 00000000..11b760dd --- /dev/null +++ b/src/SimpleKafka/StringPartitioner.cs @@ -0,0 +1,16 @@ +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafka +{ + public class StringPartitioner : IKafkaMessagePartitioner + { + public int CalculatePartition(string partitionKey, int numberOfPartitions) + { + return Math.Abs(partitionKey.GetHashCode() % numberOfPartitions); + } + } +} diff --git a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs index 5074f09c..bff1a69a 100644 --- a/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs +++ b/src/SimpleKafkaTests/Helpers/IntegrationConfig.cs @@ -9,17 +9,9 @@ namespace SimpleKafkaTests.Helpers { public static class IntegrationConfig { - public static string IntegrationCompressionTopic = "IntegrationCompressionTopic"; - public static string IntegrationTopic = "IntegrationTopic"; - public static string IntegrationConsumer = "IntegrationConsumer"; +// public static string IntegrationConsumer = "IntegrationConsumer"; // public static Uri IntegrationUri = new Uri("http://server.home:9092"); - public static Uri IntegrationUri = new Uri("http://server.home:39093"); - public static Uri[] IntegrationUriArray = new[] - { - new Uri("http://server.home:39093"), - //new Uri("http://server.home:39093"), - //new Uri("http://server.home:39094"), - }; +// public static Uri IntegrationUri = new Uri("http://server.home:39093"); public static Uri IntegrationUriOld { diff --git a/src/SimpleKafkaTests/Helpers/KafkaTestCluster.cs b/src/SimpleKafkaTests/Helpers/KafkaTestCluster.cs new file mode 100644 index 00000000..f3c175ae --- /dev/null +++ b/src/SimpleKafkaTests/Helpers/KafkaTestCluster.cs @@ -0,0 +1,263 @@ +using System; +using System.Collections.Generic; +using System.Diagnostics; +using System.Globalization; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Helpers +{ + internal class KafkaTestCluster : IDisposable + { + private class ProcessMonitor + { + private readonly Process process; + private readonly StringBuilder stdout = new StringBuilder(); + private readonly StringBuilder stderr = new StringBuilder(); + + public string Stdout { get { return stdout.ToString(); } } + public string Stderr { get { return stderr.ToString(); } } + + public ProcessMonitor(Process process) + { + this.process = process; + process.OutputDataReceived += (_, e) => stdout.AppendLine(e.Data); + process.ErrorDataReceived += (_, e) => stderr.AppendLine(e.Data); + process.BeginErrorReadLine(); + process.BeginOutputReadLine(); + } + + public void WaitForExit() + { + process.WaitForExit(); + } + + public bool WaitForExit(int milliseconds) + { + return process.WaitForExit(milliseconds); + } + + public bool HasExited { get { return process.HasExited; } } + } + private readonly ProcessMonitor zookeeperProcess; + private readonly string dockerHost; + private readonly int dockerPort; + private readonly int portBase; + private readonly ProcessMonitor[] kafkaProcesses; + + public Uri[] CreateBrokerUris() + { + return Enumerable + .Range(0, kafkaProcesses.Length) + .Select(brokerId => new Uri("http://" + dockerHost + ":" + (portBase + 1 + brokerId))) + .ToArray(); + } + + public KafkaTestCluster(string dockerHost, int brokerCount = 1, int portBase = 45678, int dockerPort = 2375) + { + this.dockerHost = dockerHost; + this.portBase = portBase; + this.dockerPort = dockerPort; + DestroyContainers(brokerCount); + this.zookeeperProcess = StartZookeeper(); + this.kafkaProcesses = StartKafkaBrokers(brokerCount); + } + + public void StopKafkaBroker(int brokerId) + { + StopKafka(brokerId); + kafkaProcesses[brokerId] = null; + } + + public void RestartKafkaBroker(int brokerId) + { + var process = RunAndCheckDocker("start", "-a", GetKafkaName(brokerId)); + kafkaProcesses[brokerId] = process; + } + + private void RunTopicCommand(params object[] args) + { + var process = RunDocker("run", "--rm", KafkaImage, "bin/kafka-topics.sh", "--zookeeper", dockerHost + ":" + portBase, + String.Join(" ", args)); + process.WaitForExit(); + if (!string.IsNullOrWhiteSpace(process.Stdout)) + { + Console.WriteLine("Stdout: {0}", process.Stdout); + } + if (!string.IsNullOrWhiteSpace(process.Stderr)) + { + Console.WriteLine("Stderr: {0}", process.Stderr); + } + } + + public class DisposableTopic : IDisposable + { + private readonly KafkaTestCluster cluster; + private readonly string name; + public string Name { get { return name; } } + public DisposableTopic(KafkaTestCluster cluster, + string name, int partitions = 1, int replicationFactor = 1) + { + this.cluster = cluster; + this.name = name; + cluster.CreateTopic(name, partitions, replicationFactor); + } + + public void Dispose() + { + cluster.DeleteTopic(name); + + } + } + + public DisposableTopic CreateTemporaryTopic(int partitions = 1, int replicationFactor = 1) { + return new DisposableTopic(this, Guid.NewGuid().ToString(), partitions, replicationFactor); + } + + + public void CreateTopic(string topicName, int partitions = 1, int replicationFactor = 1) { + RunTopicCommand("--topic", topicName, "--create", "--partitions", partitions, "--replication-factor", replicationFactor); + } + + public void DeleteTopic(string topicName) + { + RunTopicCommand("--topic", topicName, "--delete"); + } + + private ProcessMonitor RunAndCheckDocker(params object[] args) + { + var process = RunDocker(args); + var exited = process.WaitForExit(1000); + if (exited) + { + var stdout = process.Stdout; + var stderr = process.Stderr; + throw new InvalidOperationException("Failed to run\nStdout: " + stdout + "\nStderr: " + stderr); + } + else + { + return process; + } + } + private ProcessMonitor RunDocker(params object[] args) + { + var arguments = string.Format(CultureInfo.InvariantCulture, "--host=tcp://{0}:{1} {2}", + dockerHost, + dockerPort, + String.Join(" ", args) + ); + + var info = new ProcessStartInfo + { + Arguments = arguments, + CreateNoWindow = true, + FileName = "docker.exe", + UseShellExecute = false, + RedirectStandardOutput = true, + RedirectStandardError = true, + }; + var process = Process.Start(info); + return new ProcessMonitor(process); + } + + private ProcessMonitor[] StartKafkaBrokers(int brokerCount) + { + return + Enumerable + .Range(0, brokerCount) + .Select(StartKafka) + .ToArray(); + } + + private const string KafkaImage = "sceneskope/kafka:0.8.2.1-1"; + + private ProcessMonitor StartKafka(int brokerId) + { + var port = portBase + 1 + brokerId; + return RunAndCheckDocker("run", + "--name", GetKafkaName(brokerId), + "--publish", port + ":9092", + "--env", "KAFKA_BROKER_ID=" + brokerId, + "--env", "KAFKA_ADVERTISED_HOST_NAME=" + dockerHost, + "--env", "KAFKA_ADVERTISED_PORT=" + port, + "--env", "KAFKA_ZOOKEEPER_CONNECT=" + dockerHost + ":" + portBase, + "--env", "KAFKA_AUTO_CREATE_TOPICS_ENABLE=false", + "--env", "KAFKA_DELETE_TOPIC_ENABLE=true", + KafkaImage + ); + } + + private void StopKafka(int brokerId) + { + StopAndWaitForContainer(GetKafkaName(brokerId), kafkaProcesses[brokerId]); + Console.WriteLine("KOut[{0}] = {1}\n, KErr[{0}] = {2}", brokerId, kafkaProcesses[brokerId].Stdout, kafkaProcesses[brokerId].Stderr); + } + + private void StopKafkaBrokers() + { + for (var i = 0; i < kafkaProcesses.Length; i++) + { + StopKafka(i); + } + } + + private string GetKafkaName(int brokerId) { return "kTest_" + portBase + "_" + brokerId; } + private string GetZookeeperName() { return "zkTest_" + portBase; } + private ProcessMonitor StartZookeeper() + { + return RunAndCheckDocker("run", "--rm", + "--name", GetZookeeperName(), + "--publish", portBase + ":2181", + "--env", "JMXDISABLE=true", + "sceneskope/zookeeper:3.4.6"); + } + + private void StopZookeeper() + { + StopAndWaitForContainer(GetZookeeperName(), zookeeperProcess); + Console.WriteLine("ZKOut = {0}\nZkErr = {1}", zookeeperProcess.Stdout, zookeeperProcess.Stderr); + } + + private void StopAndWaitForContainer(string containerName, ProcessMonitor process) + { + if (process.HasExited) + { + return; + } + + var stopCommand = RunDocker("stop", containerName); + stopCommand.WaitForExit(); + Console.WriteLine("stop {0} = {1},{2}", containerName, stopCommand.Stdout, stopCommand.Stderr); + var exited = process.WaitForExit(2000); + if (exited) + { + return; + } + + RunDocker("kill", containerName); + process.WaitForExit(); + } + + + + public void Dispose() + { + StopKafkaBrokers(); + StopZookeeper(); + DestroyContainers(kafkaProcesses.Length); + } + + public void DestroyContainers(int brokerCount) + { + var args = new List { "rm", "-f", GetZookeeperName() }; + for (var i = 0; i < brokerCount; i++) + { + args.Add(GetKafkaName(i)); + } + var process = RunDocker(args.ToArray()); + process.WaitForExit(); + Console.WriteLine("Destroyed. Out = {0}, Err = {1}", process.Stdout, process.Stderr); + } + } +} diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs index 55c63e75..083c44db 100644 --- a/src/SimpleKafkaTests/Integration/ConsumerTests.cs +++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs @@ -16,22 +16,32 @@ class ConsumerTests { private readonly string defaultConsumerGroup = "unit-tests"; - [SetUp] - public void Setup() + private KafkaTestCluster testCluster; + + [OneTimeSetUp] + public void BuildTestCluster() + { + testCluster = new KafkaTestCluster("server.home", 1); + } + + [OneTimeTearDown] + public void DestroyTestCluster() { - IntegrationHelpers.zookeeperHost = "server.home:32181"; + testCluster.Dispose(); + testCluster = null; } + + [Test] public async Task TestSimpleConsumerWorksOk() { var keySerializer = new NullSerializer(); var valueSerializer = new StringSerializer(); var messagePartitioner = new LoadBalancedPartitioner(); - - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); var consumer = KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); @@ -56,10 +66,10 @@ public async Task TestProducing3MessagesAllowsTheConsumerToChooseTheCorrectMessa { var valueSerializer = new StringSerializer(); - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; { var producer = KafkaProducer.Create(brokers, valueSerializer); @@ -137,10 +147,10 @@ public async Task TestProducing3MessagesAllowsTheConsumerToCommitAndRestart() { var valueSerializer = new StringSerializer(); - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; { var producer = KafkaProducer.Create(brokers, valueSerializer); diff --git a/src/SimpleKafkaTests/Integration/FailureTests.cs b/src/SimpleKafkaTests/Integration/FailureTests.cs new file mode 100644 index 00000000..7710d90a --- /dev/null +++ b/src/SimpleKafkaTests/Integration/FailureTests.cs @@ -0,0 +1,144 @@ +using NUnit.Framework; +using SimpleKafka; +using SimpleKafkaTests.Helpers; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Integration +{ + [TestFixture] + [Category("Integration")] + class FailureTests + { + [Test] + public void TestClusterCanBeManaged() + { + using (var cluster = new KafkaTestCluster("server.home", 3)) + { + cluster.CreateTopic("test", 2, 2); + cluster.DeleteTopic("test"); + } + } + [Test] + public async Task TestManagedClusterWorks() + { + using (var cluster = new KafkaTestCluster("server.home", 1)) + { + var topic = "test"; + cluster.CreateTopic(topic); + using (var brokers = new KafkaBrokers(cluster.CreateBrokerUris())) + { + var producer = KafkaProducer.Create(brokers, new StringSerializer()); + await producer.SendAsync(KeyedMessage.Create(topic, "Test"), CancellationToken.None).ConfigureAwait(true); + + var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), + new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); + var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + + Assert.That(result, Has.Count.EqualTo(1)); + var first = result[0]; + + Assert.That(first.Value, Is.EqualTo("Test")); + Assert.That(first.Offset, Is.EqualTo(0)); + + } + cluster.DeleteTopic(topic); + } + } + + + [Test] + public async Task VerifyABrokerStoppingAndRestartingCanBeHandledByTheConsumer() + { + using (var cluster = new KafkaTestCluster("server.home", 1)) + { + var topic = "test"; + cluster.CreateTopic(topic); + using (var brokers = new KafkaBrokers(cluster.CreateBrokerUris())) + { + var producer = KafkaProducer.Create(brokers, new StringSerializer()); + await producer.SendAsync(KeyedMessage.Create(topic, "Test"), CancellationToken.None); + + await Task.Delay(1000); + cluster.StopKafkaBroker(0); + cluster.RestartKafkaBroker(0); + + var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), + new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); + var result = await consumer.ReceiveAsync(CancellationToken.None); + + Assert.That(result, Has.Count.EqualTo(1)); + var first = result[0]; + + Assert.That(first.Value, Is.EqualTo("Test")); + Assert.That(first.Offset, Is.EqualTo(0)); + + } + cluster.DeleteTopic(topic); + } + } + + [Test] + public async Task VerifyABrokerStoppingAndRestartingCanBeHandledByTheProducer() + { + using (var cluster = new KafkaTestCluster("server.home", 1)) + { + var topic = "test"; + cluster.CreateTopic(topic); + using (var brokers = new KafkaBrokers(cluster.CreateBrokerUris())) + { + { + var producer = KafkaProducer.Create(brokers, new StringSerializer()); + await producer.SendAsync(KeyedMessage.Create(topic, "Test 0"), CancellationToken.None).ConfigureAwait(true); + } + + + { + var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), + new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); + var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + + Assert.That(result, Has.Count.EqualTo(1)); + var first = result[0]; + + Assert.That(first.Value, Is.EqualTo("Test 0")); + Assert.That(first.Offset, Is.EqualTo(0)); + } + + cluster.StopKafkaBroker(0); + cluster.RestartKafkaBroker(0); + + { + var producer = KafkaProducer.Create(brokers, new StringSerializer()); + await producer.SendAsync(KeyedMessage.Create(topic, "Test 1"), CancellationToken.None).ConfigureAwait(true); + } + + + { + var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), + new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); + var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + + Assert.That(result, Has.Count.EqualTo(2)); + var first = result[0]; + + Assert.That(first.Value, Is.EqualTo("Test 0")); + Assert.That(first.Offset, Is.EqualTo(0)); + + var second = result[1]; + Assert.That(second.Value, Is.EqualTo("Test 1")); + Assert.That(second.Offset, Is.EqualTo(1)); + + } + + + } + cluster.DeleteTopic(topic); + } + } + } +} diff --git a/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs b/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs deleted file mode 100644 index b7a22e12..00000000 --- a/src/SimpleKafkaTests/Integration/IntegrationHelpers.cs +++ /dev/null @@ -1,72 +0,0 @@ -using System; -using System.Collections.Generic; -using System.Diagnostics; -using System.Globalization; -using System.Linq; -using System.Text; -using System.Threading.Tasks; - -namespace SimpleKafkaTests.Integration -{ - internal static class IntegrationHelpers - { - - public static string dockerHost = "tcp://server.home:2375"; - public static string zookeeperHost = "server.home:32181"; - public static string kafkaImage = "sceneskope/kafka:0.8.2.1"; - public static string dockerOptions = ""; - - public static void RunKafkaTopicsCommand(params object[] args) - { - var cmd = string.Format(CultureInfo.InvariantCulture, "--host={0} run --rm {1} {2} bin/kafka-topics.sh --zookeeper {3} ", - dockerHost, dockerOptions, kafkaImage, zookeeperHost); - - var arguments = cmd + String.Join(" ", args); - - var info = new ProcessStartInfo - { - Arguments = arguments, - CreateNoWindow = true, -// FileName = @"c:\users\nick\bin\docker.exe", - FileName = @"docker.exe", - UseShellExecute = false, - RedirectStandardOutput = true, - }; - var process = Process.Start(info); - var stdout = process.StandardOutput.ReadToEnd(); - process.WaitForExit(); - Console.WriteLine(stdout); - } - - public static void DeleteTopic(string topic) - { - RunKafkaTopicsCommand("--topic", topic, "--delete"); - } - - public static void CreateTopic(string topic, int partitions = 1, int replicationFactor = 1) - { - RunKafkaTopicsCommand("--topic", topic, "--create", "--partitions", partitions, "--replication-factor", replicationFactor); - } - - public static TemporaryTopic CreateTemporaryTopic(int partitions = 1, int replicationFactor = 1) - { - return new TemporaryTopic(partitions, replicationFactor); - } - - public class TemporaryTopic : IDisposable - { - private readonly string topic = Guid.NewGuid().ToString(); - public string Topic { get { return topic; } } - public TemporaryTopic(int partitions = 1, int replicationFactor = 1) - { - CreateTopic(topic, partitions, replicationFactor); - } - - public void Dispose() - { - DeleteTopic(topic); - } - } - - } -} diff --git a/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs index 8f75cc2f..d633d65b 100644 --- a/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs +++ b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs @@ -16,10 +16,19 @@ class MultiplePartitionTests { private readonly string defaultConsumerGroup = "unit-tests"; - [SetUp] - public void Setup() + private KafkaTestCluster testCluster; + + [OneTimeSetUp] + public void BuildTestCluster() + { + testCluster = new KafkaTestCluster("server.home", 1); + } + + [OneTimeTearDown] + public void DestroyTestCluster() { - IntegrationHelpers.zookeeperHost = "server.home:32181"; + testCluster.Dispose(); + testCluster = null; } [Test] @@ -29,10 +38,10 @@ public async Task ProduceToTwoPartitions() var valueSerializer = new StringSerializer(); var messagePartitioner = new Int32Partitioner(); - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic(partitions:2)) - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); var consumers = new[] { KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, @@ -76,10 +85,10 @@ public async Task ProduceToMultiplePartitions(int numberOfPartitions, int number var valueSerializer = new StringSerializer(); var messagePartitioner = new Int32Partitioner(); - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic(partitions: numberOfPartitions)) - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUriArray)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions: 2)) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; { var producer = KafkaProducer.Create(brokers, keySerializer, valueSerializer, messagePartitioner); var messages = diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 48ebda7b..0edc3618 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -17,18 +17,26 @@ namespace SimpleKafkaTests.Integration [Category("Integration")] class SimpleTests { - [SetUp] - public void Setup() + private KafkaTestCluster testCluster; + + [OneTimeSetUp] + public void BuildTestCluster() + { + testCluster = new KafkaTestCluster("server.home", 1); + } + + [OneTimeTearDown] + public void DestroyTestCluster() { - IntegrationHelpers.dockerHost = "tcp://server.home:2375"; - IntegrationHelpers.zookeeperHost = "server.home"; - IntegrationHelpers.dockerOptions = ""; + testCluster.Dispose(); + testCluster = null; } - [Test] + [Test] public async Task TestProducingWorksOk() { - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) { var request = new ProduceRequest { @@ -38,7 +46,7 @@ public async Task TestProducingWorksOk() { new Payload { - Topic = IntegrationConfig.IntegrationTopic, + Topic = temporaryTopic.Name, Partition = 0, Codec = MessageCodec.CodecNone, Messages = new List @@ -50,24 +58,27 @@ public async Task TestProducingWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Console.WriteLine(response); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); } } [Test] public async Task TestFetchingWorksOk() { - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) { var request = new FetchRequest { - MaxWaitTime = 1000, + MaxWaitTime = 0, MinBytes = 1000, Fetches = new List { new Fetch { - Topic = IntegrationConfig.IntegrationTopic, + Topic = temporaryTopic.Name, PartitionId = 0, MaxBytes = 1024, Offset = 0 @@ -76,33 +87,27 @@ public async Task TestFetchingWorksOk() }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Console.WriteLine(response); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); + Assert.That(first.Messages, Has.Count.EqualTo(0)); } } [Test] public async Task TestListingAllTopicsWorksOk() { - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) { var request = new MetadataRequest { }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); Assert.That(response, Is.Not.Null); - var first = response; - { - foreach (var broker in first.Brokers) - { - Console.WriteLine("{0},{1},{2},{3}", broker.Address, broker.BrokerId, broker.Host, broker.Port); - } - foreach (var topic in first.Topics) - { - Console.WriteLine("{0},{1}", topic.ErrorCode, topic.Name); - foreach (var partition in topic.Partitions) - { - Console.WriteLine("{0},{1},{2},{3},{4}", partition.ErrorCode, partition.Isrs.Length, partition.LeaderId, partition.PartitionId, partition.Replicas.Length); - } - } - } + Assert.That(response.Brokers, Has.Length.EqualTo(1)); + Assert.That(response.Topics, Has.Length.EqualTo(1)); + Assert.That(response.Topics[0].Name, Is.EqualTo(temporaryTopic.Name)); + Assert.That(response.Topics[0].ErrorCode, Is.EqualTo(ErrorResponseCode.NoError)); + Assert.That(response.Topics[0].Partitions, Has.Length.EqualTo(1)); } } @@ -110,34 +115,173 @@ public async Task TestListingAllTopicsWorksOk() [Test] public async Task TestOffsetWorksOk() { - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + { + var request = new OffsetRequest + { + Offsets = new List + { + new Offset + { + Topic = temporaryTopic.Name, + MaxOffsets = 1, + PartitionId = 0, + Time = -1 + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + var first = response.First(); + Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); + } + } + + [Test] + public async Task TestMultipleOffsetWorksOk() + { + using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) { + var topic = temporaryTopic.Name; var request = new OffsetRequest { Offsets = new List { new Offset { - Topic = IntegrationConfig.IntegrationTopic, + Topic = topic, MaxOffsets = 1, PartitionId = 0, Time = -1 + }, + new Offset + { + Topic = topic, + MaxOffsets = 1, + PartitionId = 1, + Time = -1 } + } }; var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); - Console.WriteLine(response); + Assert.That(response, Has.Count.EqualTo(2)); + Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.NoError)); + Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.NoError)); + } + } + + [Test] + public async Task TestOffsetCommitWorksOk() + { + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + { + var request = new OffsetCommitRequest + { + OffsetCommits = new List + { + new OffsetCommit + { + Topic = temporaryTopic.Name, + Offset = 0 + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); + } + } + + [Test] + public async Task TestMultipleOffsetCommitsWorksOk() + { + using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + { + var request = new OffsetCommitRequest + { + OffsetCommits = + Enumerable + .Range(0, 2) + .Select(i => new OffsetCommit { + Topic = temporaryTopic.Name, + PartitionId = i, + Offset = 0 + }) + .ToList() + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(2)); + Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); + Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.Unknown)); + } + } + + [Test] + public async Task TestOffsetFetchWorksOk() + { + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + { + var request = new OffsetFetchRequest + { + Topics = new List + { + new OffsetFetch { + Topic = temporaryTopic.Name, + PartitionId = 0 + } + } + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(1)); + Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); + } + } + + [Test] + public async Task TestMultipleOffsetFetchesWorkOk() + { + using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + { + var topic = temporaryTopic.Name; + var request = new OffsetFetchRequest + { + Topics = + Enumerable + .Range(0, 2) + .Select(i => new OffsetFetch + { + Topic = topic, + PartitionId = i + }) + .ToList() + }; + + var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + Assert.That(response, Has.Count.EqualTo(2)); + Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); + Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.Unknown)); } } [Test] public async Task TestNewTopicProductionWorksOk() { - using (var temporaryTopic = IntegrationHelpers.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(IntegrationConfig.IntegrationUri).ConfigureAwait(true)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) { - var topic = temporaryTopic.Topic; + var topic = temporaryTopic.Name; { var request = new MetadataRequest { @@ -453,11 +597,9 @@ public async Task TestNewTopicProductionWorksOk() [Test] public async Task TestSimpleKafkaBrokerWorksOk() { - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUri)) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { await brokers.RefreshAsync(CancellationToken.None); - Console.WriteLine(brokers); - } } @@ -466,11 +608,12 @@ public async Task TestSimpleProducerWorksOk() { var valueSerializer = new StringSerializer(); - using (var brokers = new KafkaBrokers(IntegrationConfig.IntegrationUri)) + using (var temporaryTopic = testCluster.CreateTemporaryTopic()) + using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris())) { var producer = KafkaProducer.Create(brokers, valueSerializer); - await producer.SendAsync(KeyedMessage.Create(IntegrationConfig.IntegrationTopic, "Message"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(temporaryTopic.Name, "Message"), CancellationToken.None).ConfigureAwait(true); } diff --git a/src/SimpleKafkaTests/SimpleKafkaTests.csproj b/src/SimpleKafkaTests/SimpleKafkaTests.csproj index 00605938..d4df0474 100644 --- a/src/SimpleKafkaTests/SimpleKafkaTests.csproj +++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj @@ -46,14 +46,16 @@ + - + + @@ -70,9 +72,7 @@ SimpleKafka - - - + diff --git a/src/SimpleKafkaTests/Unit/PartitionerTests.cs b/src/SimpleKafkaTests/Unit/PartitionerTests.cs new file mode 100644 index 00000000..329cae06 --- /dev/null +++ b/src/SimpleKafkaTests/Unit/PartitionerTests.cs @@ -0,0 +1,52 @@ +using NUnit.Framework; +using SimpleKafka; +using System; +using System.Collections.Generic; +using System.Linq; +using System.Text; +using System.Threading.Tasks; + +namespace SimpleKafkaTests.Unit +{ + [TestFixture] + [Category("Unit")] + class PartitionerTests + { + [Theory] + [TestCase("", 1, 0)] + [TestCase("Hello1", 1, 0)] + [TestCase("Hello2", 2, 1)] + public void TestStringPartitioner(string stringToTest, int numberOfPartitions, int partitionExpected) + { + var partitioner = new StringPartitioner(); + var result = partitioner.CalculatePartition(stringToTest, numberOfPartitions); + Assert.That(result, Is.EqualTo(partitionExpected)); + } + + [Theory] + [TestCase("", 1, 0)] + [TestCase("Hello1", 1, 0)] + [TestCase("Hello4", 2, 1)] + public void TestFixedPartitioner(object objectToTest, int numberOfPartitions, int partitionExpected) + { + var partitioner = new FixedPartitioner(partitionExpected); + var result = partitioner.CalculatePartition(objectToTest, numberOfPartitions); + Assert.That(result, Is.EqualTo(partitionExpected)); + } + + [Theory] + [TestCase("Test", 1)] + [TestCase("Test", 2)] + public void TestLoadBalancedPartitioner(object objectToTest, int numberOfPartitions) + { + var partitioner = new LoadBalancedPartitioner(); + var expected = 0; + for (var i = 0; i < numberOfPartitions * 2; i++ ) + { + var result = partitioner.CalculatePartition(objectToTest, numberOfPartitions); + Assert.That(result, Is.EqualTo(expected)); + expected = (expected + 1) % numberOfPartitions; + } + } + } +} diff --git a/src/kafka-net.sln b/src/kafka-net.sln index 44196427..ff8ed88e 100644 --- a/src/kafka-net.sln +++ b/src/kafka-net.sln @@ -19,6 +19,7 @@ EndProject Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution Items", "{F4C67622-F19D-4D3D-9C60-C2A9E717A197}" ProjectSection(SolutionItems) = preProject ..\.gitignore = ..\.gitignore + coverage.bat = coverage.bat ..\LICENSE = ..\LICENSE ..\README.md = ..\README.md ..\RELEASE_NOTES.md = ..\RELEASE_NOTES.md From 533911e0b3aa9384534afd5eb1f812efa926e91f Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 20 Apr 2015 09:05:47 +0100 Subject: [PATCH 12/14] Updated readme and tidies some tests --- README.md | 97 +++++++++---------- .../Integration/ConsumerTests.cs | 32 +++--- .../Integration/FailureTests.cs | 12 +-- .../Integration/MultiplePartitionTests.cs | 8 +- .../Integration/SimpleTests.cs | 60 ++++++------ 5 files changed, 101 insertions(+), 108 deletions(-) diff --git a/README.md b/README.md index 1d68f020..debd3f15 100644 --- a/README.md +++ b/README.md @@ -1,88 +1,80 @@ -kafka-net +simple-kafka-net ========= -Native C# client for Apache Kafka. +Native C# client for Apache Kafka derived from [jroland/kafka-net]. License ----------- -Copyright 2014, James Roland under Apache License, V2.0. See LICENSE file. +Original kafka-net Copyright 2014, James Roland +Modified version Copyright 2015, Nick Randell + +Apache License, V2.0. See LICENSE file. Summary ----------- -This project is a .NET implementation of the [Apache Kafka] protocol. The wire protocol portion is based on the [kafka-python] library writen by [David Arthur] and the general class layout attempts to follow a similar pattern as his project. To that end, this project builds up from the low level KafkaConnection object for handling async requests to/from the kafka server, all the way up to a higher level Producer/Consumer classes. +This project is a .NET implementation of the [Apache Kafka] protocol. The wire protocol portion is based on the [kafka-python] library writen by [David Arthur] and the general class layout attempts to follow a similar pattern as his project. + +It is very much work in progress but is designed to be asynchronous from the ground up, providing thin wrapper over the protocol and building up a robust broker manager with simple producer and consumer classes. Even though it is asynchronous, no threads are involved unless the runtime uses them. This does mean that it is not yet possible to have multiple messages in flight at the same time, but that will change. + +The protocol encoding and decoding has also been modified to work as efficiently as possible without copying data around. + +One of the aims of this fork is to allow the client to have much more control over the partitions to consume as this allows larger solutions to scale by having consumers running on different servers. + +Testing makes use of docker to spin up test clusters to give control over different scenarios. + +The current version 0.1 is not very robust, but generally works for single brokers ok. + Examples ----------- ##### Producer -```sh -var options = new KafkaOptions(new Uri("http://SERVER1:9092"), new Uri("http://SERVER2:9092")); -var router = new BrokerRouter(options); -var client = new Producer(router); +```using (var broker = new KafkaBroker(new Uri("http://SERVER1:9092"))) +{ + var producer = KafkaProducer.Create(brokers, new StringSerializer()); + await producer.SendAsync(KeyedMessage.Create("Test Topic", "Test"), CancellationToken.None); +} +``` -client.SendMessageAsync("TestHarness", new[] { new Message("hello world")}).Wait(); -using (client) { } -``` ##### Consumer -```sh -var options = new KafkaOptions(new Uri("http://SERVER1:9092"), new Uri("http://SERVER2:9092")); -var router = new BrokerRouter(options); -var consumer = new Consumer(new ConsumerOptions("TestHarness", new BrokerRouter(options))); - -//Consume returns a blocking IEnumerable (ie: never ending stream) -foreach (var message in consumer.Consume()) +```using (var broker = new KafkaBroker(new Uri("http://SERVER1:9092"))) { - Console.WriteLine("Response: P{0},O{1} : {2}", - message.Meta.PartitionId, message.Meta.Offset, message.Value); + var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), + new TopicSelector { Topic = "Test Topic", Partition = 0, Offset = 0 }); + var result = await consumer.ReceiveAsync(CancellationToken.None); + foreach (var message in result) + { + Console.WriteLine("Received {0}", message.Value); + } } ``` -##### TestHarness -The TestHarness project it a simple example console application that will read message from a kafka server and write them to the screen. It will also take anything typed in the console and send this as a message to the kafka servers. +The topic selector is used to determine which topics and offsets to use. -Simply modify the kafka server Uri in the code to point to a functioning test server. +Things left to do +------------ +- [ ] Make use of correlation id to allow multiple messages in flight at the same time +- [ ] Compression Pieces of the Puzzle ----------- ##### Protocol The protocol has been divided up into concrete classes for each request/response pair. Each class knows how to encode and decode itself into/from their appropriate Kafka protocol byte array. One benefit of this is that it allows for a nice generic send method on the KafkaConnection. -##### KafkaConnection -Provides async methods on a persistent connection to a kafka broker (server). The send method uses the TcpClient send async function and the read stream has a dedicated thread which uses the correlation Id to match send responses to the correct request. +This has been kept almost identical to the original version, but the encoding/decoding is now done with a preallocated buffer. -##### BrokerRouter -Provides metadata based routing of messages to the correct Kafka partition. This class also manages the multiple KafkaConnections for each Kafka server returned by the broker section in the metadata response. Routing logic is provided by the IPartitionSelector. +##### KafkaConnection +Provides async methods on a persistent connection to a kafka broker (server). Sending a message and receiving a response is carried out within a lock as only one message is in flight at any time. This will change!! -##### IPartitionSelector -Provides the logic for routing which partition the BrokerRouter should choose. The default selector is the DefaultPartitionSelector which will use round robin partition selection if the key property on the message is null and a mod/hash of the key value if present. +##### KafkaBrokers +Provides management of a group of brokers, maintaining a connection to each of the valid brokers ##### Producer -Provides a higher level class which uses the combination of the BrokerRouter and KafkaConnection to send batches of messages to a Kafka broker. +Provides a higher level class which uses the combination of the KafkaBrokers and KafkaConnection to send messages. There is no queuing or batching of messages internally. That would be the work of a higher level producer. ##### Consumer -Provides a higher level class which will consumer messages from a whitelist of partitions from a single topic. The consumption mechanism is a blocking IEnumerable of messages. If no whitelist is provided then all partitions will be consumed creating one KafkaConnection for each partition leader. - - - -Status ------------ -[![Build status](https://ci.appveyor.com/api/projects/status/3tg02biqn5q8uijy)](https://ci.appveyor.com/project/Jroland/kafka-net) - - -The current version of this project is a functioning "work in progress" as it was only started in early February. The wire protocol is complete except for Offset Commits to the servers (as there is a bug in 0.8.0 which prevents testing of this feature). The library is functioning in that there is a complete Producer and Consumer class thus messages can pass to and from a Kafka server. - -##### The major items that needs work are: -* Better handling of options for providing customization of internal behaviour of the base API. (right now the classes pass around option parameters) -* General structure of the classes is not finalized and breaking changes will occur. -* Only Gzip compression is implemented, snappy on the todo. -* Currently only works with .NET Framework 4.5 as it uses the await command. -* Test coverage. -* Documentation. - -Comment ----------- -This is a pet project for me and is not currently backed by a need for a Kafka server client. Which means the client is only currently being tested against a small set of Kafka test servers and not against any server that has any real data load. +Provides the ability to receive messages from brokers. @@ -92,3 +84,4 @@ This is a pet project for me and is not currently backed by a need for a Kafka s [kafka-python]:https://github.com/mumrah/kafka-python [Apache Kafka]:http://kafka.apache.org [David Arthur]:https://github.com/mumrah +[jroland/kafka-net]:https://github.com/jroland/kafka-net \ No newline at end of file diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs index 083c44db..760eb895 100644 --- a/src/SimpleKafkaTests/Integration/ConsumerTests.cs +++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs @@ -46,9 +46,9 @@ public async Task TestSimpleConsumerWorksOk() var consumer = KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, new TopicSelector { Partition = 0, Topic = topic }); - await producer.SendAsync(KeyedMessage.Create(topic, "Message"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(topic, "Message"), CancellationToken.None); - var responses = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await consumer.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(1)); @@ -77,14 +77,14 @@ await producer.SendAsync(new[] { KeyedMessage.Create(topic, "1"), KeyedMessage.Create(topic, "2"), KeyedMessage.Create(topic, "3"), - }, CancellationToken.None).ConfigureAwait(true); + }, CancellationToken.None); } { var earliest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await earliest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await earliest.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(3)); @@ -100,7 +100,7 @@ await producer.SendAsync(new[] { var latest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Last }); - var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await latest.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(1)); @@ -116,7 +116,7 @@ await producer.SendAsync(new[] { var latest = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Next }); - var responses = await latest.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await latest.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(0)); @@ -126,7 +126,7 @@ await producer.SendAsync(new[] { var specified = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer, new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Specified, Offset = 1 }); - var responses = await specified.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await specified.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(2)); @@ -158,7 +158,7 @@ await producer.SendAsync(new[] { KeyedMessage.Create(topic, "1"), KeyedMessage.Create(topic, "2"), KeyedMessage.Create(topic, "3"), - }, CancellationToken.None).ConfigureAwait(true); + }, CancellationToken.None); } { @@ -167,7 +167,7 @@ await producer.SendAsync(new[] { DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted, FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await noPreviousCommits.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await noPreviousCommits.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(3)); @@ -180,7 +180,7 @@ await producer.SendAsync(new[] { await noPreviousCommits.CommitAsync(new[] { new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 0 } - }, CancellationToken.None).ConfigureAwait(true); ; + }, CancellationToken.None); ; } { @@ -193,7 +193,7 @@ await noPreviousCommits.CommitAsync(new[] { FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await previousCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await previousCommit.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(2)); @@ -216,7 +216,7 @@ await noPreviousCommits.CommitAsync(new[] { FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await previousCommitAgain.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await previousCommitAgain.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(2)); @@ -229,7 +229,7 @@ await noPreviousCommits.CommitAsync(new[] { await previousCommitAgain.CommitAsync(new[] { new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 1 } - }, CancellationToken.None).ConfigureAwait(true); ; + }, CancellationToken.None); ; } { @@ -242,7 +242,7 @@ await previousCommitAgain.CommitAsync(new[] { FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await secondCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await secondCommit.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(1)); @@ -255,7 +255,7 @@ await previousCommitAgain.CommitAsync(new[] { await secondCommit.CommitAsync(new[] { new TopicPartitionOffset { Topic = topic, Partition = 0, Offset = 2 } - }, CancellationToken.None).ConfigureAwait(true); ; + }, CancellationToken.None); ; } { @@ -268,7 +268,7 @@ await secondCommit.CommitAsync(new[] { FailureOffsetSelection = OffsetSelectionStrategy.Earliest }); - var responses = await thirdCommit.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await thirdCommit.ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(0)); diff --git a/src/SimpleKafkaTests/Integration/FailureTests.cs b/src/SimpleKafkaTests/Integration/FailureTests.cs index 7710d90a..d5ef60f3 100644 --- a/src/SimpleKafkaTests/Integration/FailureTests.cs +++ b/src/SimpleKafkaTests/Integration/FailureTests.cs @@ -33,11 +33,11 @@ public async Task TestManagedClusterWorks() using (var brokers = new KafkaBrokers(cluster.CreateBrokerUris())) { var producer = KafkaProducer.Create(brokers, new StringSerializer()); - await producer.SendAsync(KeyedMessage.Create(topic, "Test"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(topic, "Test"), CancellationToken.None); var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); - var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var result = await consumer.ReceiveAsync(CancellationToken.None); Assert.That(result, Has.Count.EqualTo(1)); var first = result[0]; @@ -93,14 +93,14 @@ public async Task VerifyABrokerStoppingAndRestartingCanBeHandledByTheProducer() { { var producer = KafkaProducer.Create(brokers, new StringSerializer()); - await producer.SendAsync(KeyedMessage.Create(topic, "Test 0"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(topic, "Test 0"), CancellationToken.None); } { var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); - var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var result = await consumer.ReceiveAsync(CancellationToken.None); Assert.That(result, Has.Count.EqualTo(1)); var first = result[0]; @@ -114,14 +114,14 @@ public async Task VerifyABrokerStoppingAndRestartingCanBeHandledByTheProducer() { var producer = KafkaProducer.Create(brokers, new StringSerializer()); - await producer.SendAsync(KeyedMessage.Create(topic, "Test 1"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(topic, "Test 1"), CancellationToken.None); } { var consumer = KafkaConsumer.Create(topic, brokers, new StringSerializer(), new TopicSelector { Topic = topic, Partition = 0, Offset = 0 }); - var result = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var result = await consumer.ReceiveAsync(CancellationToken.None); Assert.That(result, Has.Count.EqualTo(2)); var first = result[0]; diff --git a/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs index d633d65b..805542cb 100644 --- a/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs +++ b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs @@ -54,11 +54,11 @@ public async Task ProduceToTwoPartitions() await producer.SendAsync(new[] { KeyedMessage.Create(topic, 0, "Message to partition 0"), KeyedMessage.Create(topic, 1, "Message to partition 1") - }, CancellationToken.None).ConfigureAwait(true); + }, CancellationToken.None); for (var i = 0; i < consumers.Length; i++) { - var responses = await consumers[i].ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await consumers[i].ReceiveAsync(CancellationToken.None); Assert.That(responses, Is.Not.Null); Assert.That(responses, Has.Count.EqualTo(1)); @@ -95,7 +95,7 @@ public async Task ProduceToMultiplePartitions(int numberOfPartitions, int number Enumerable .Range(0, numberOfMessages) .Select(i => KeyedMessage.Create(topic, i % numberOfKeys, i % numberOfPartitions, "Message " + i)); - await producer.SendAsync(messages, CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(messages, CancellationToken.None); } { @@ -106,7 +106,7 @@ public async Task ProduceToMultiplePartitions(int numberOfPartitions, int number .ToArray(); var consumer = KafkaConsumer.Create(defaultConsumerGroup, brokers, keySerializer, valueSerializer, selectors); - var responses = await consumer.ReceiveAsync(CancellationToken.None).ConfigureAwait(true); + var responses = await consumer.ReceiveAsync(CancellationToken.None); Assert.That(responses, Has.Count.EqualTo(numberOfMessages)); var received = new bool[numberOfMessages]; var offsets = new long[numberOfPartitions]; diff --git a/src/SimpleKafkaTests/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs index 0edc3618..c8b64f8f 100644 --- a/src/SimpleKafkaTests/Integration/SimpleTests.cs +++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs @@ -36,7 +36,7 @@ public void DestroyTestCluster() public async Task TestProducingWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new ProduceRequest { @@ -57,7 +57,7 @@ public async Task TestProducingWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); @@ -68,7 +68,7 @@ public async Task TestProducingWorksOk() public async Task TestFetchingWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new FetchRequest { @@ -86,7 +86,7 @@ public async Task TestFetchingWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); @@ -98,10 +98,10 @@ public async Task TestFetchingWorksOk() public async Task TestListingAllTopicsWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new MetadataRequest { }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Is.Not.Null); Assert.That(response.Brokers, Has.Length.EqualTo(1)); Assert.That(response.Topics, Has.Length.EqualTo(1)); @@ -116,7 +116,7 @@ public async Task TestListingAllTopicsWorksOk() public async Task TestOffsetWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new OffsetRequest { @@ -132,7 +132,7 @@ public async Task TestOffsetWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); Assert.That(first.Error, Is.EqualTo(ErrorResponseCode.NoError)); @@ -143,7 +143,7 @@ public async Task TestOffsetWorksOk() public async Task TestMultipleOffsetWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var topic = temporaryTopic.Name; var request = new OffsetRequest @@ -168,7 +168,7 @@ public async Task TestMultipleOffsetWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(2)); Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.NoError)); Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.NoError)); @@ -179,7 +179,7 @@ public async Task TestMultipleOffsetWorksOk() public async Task TestOffsetCommitWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new OffsetCommitRequest { @@ -193,7 +193,7 @@ public async Task TestOffsetCommitWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); } @@ -203,7 +203,7 @@ public async Task TestOffsetCommitWorksOk() public async Task TestMultipleOffsetCommitsWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new OffsetCommitRequest { @@ -218,7 +218,7 @@ public async Task TestMultipleOffsetCommitsWorksOk() .ToList() }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(2)); Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.Unknown)); @@ -229,7 +229,7 @@ public async Task TestMultipleOffsetCommitsWorksOk() public async Task TestOffsetFetchWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var request = new OffsetFetchRequest { @@ -242,7 +242,7 @@ public async Task TestOffsetFetchWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); } @@ -252,7 +252,7 @@ public async Task TestOffsetFetchWorksOk() public async Task TestMultipleOffsetFetchesWorkOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2)) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var topic = temporaryTopic.Name; var request = new OffsetFetchRequest @@ -268,7 +268,7 @@ public async Task TestMultipleOffsetFetchesWorkOk() .ToList() }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(2)); Assert.That(response[0].Error, Is.EqualTo(ErrorResponseCode.Unknown)); Assert.That(response[1].Error, Is.EqualTo(ErrorResponseCode.Unknown)); @@ -279,7 +279,7 @@ public async Task TestMultipleOffsetFetchesWorkOk() public async Task TestNewTopicProductionWorksOk() { using (var temporaryTopic = testCluster.CreateTemporaryTopic()) - using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]).ConfigureAwait(true)) + using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0])) { var topic = temporaryTopic.Name; { @@ -293,7 +293,7 @@ public async Task TestNewTopicProductionWorksOk() MetadataResponse response = null; while (response == null) { - response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + response = await connection.SendRequestAsync(request, CancellationToken.None); if (response.Topics[0].ErrorCode == ErrorResponseCode.LeaderNotAvailable) { response = null; @@ -337,7 +337,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Is.Not.Null); var first = response.First(); @@ -364,7 +364,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -401,7 +401,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -422,7 +422,7 @@ public async Task TestNewTopicProductionWorksOk() ConsumerMetadataResponse response = null; while (response == null) { - response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + response = await connection.SendRequestAsync(request, CancellationToken.None); if (response.Error == ErrorResponseCode.ConsumerCoordinatorNotAvailableCode) { response = null; @@ -448,7 +448,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -477,7 +477,7 @@ public async Task TestNewTopicProductionWorksOk() } } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -500,7 +500,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -528,7 +528,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -580,7 +580,7 @@ public async Task TestNewTopicProductionWorksOk() } }; - var response = await connection.SendRequestAsync(request, CancellationToken.None).ConfigureAwait(true); + var response = await connection.SendRequestAsync(request, CancellationToken.None); Assert.That(response, Has.Count.EqualTo(1)); var first = response.First(); @@ -613,7 +613,7 @@ public async Task TestSimpleProducerWorksOk() { var producer = KafkaProducer.Create(brokers, valueSerializer); - await producer.SendAsync(KeyedMessage.Create(temporaryTopic.Name, "Message"), CancellationToken.None).ConfigureAwait(true); + await producer.SendAsync(KeyedMessage.Create(temporaryTopic.Name, "Message"), CancellationToken.None); } From f3ffb2505c51e582ac29d7f54cff0f4f0dffc394 Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 20 Apr 2015 09:10:08 +0100 Subject: [PATCH 13/14] Added some missing files --- .gitignore | 5 +++++ src/.nuget/packages.config | 6 ++++++ src/coverage.bat | 5 +++++ 3 files changed, 16 insertions(+) create mode 100644 src/.nuget/packages.config create mode 100644 src/coverage.bat diff --git a/.gitignore b/.gitignore index 53f43cd4..7df37034 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,8 @@ +src/coverage +src/results.xml +src/TestResult.xml + + ################# ## Eclipse ################# diff --git a/src/.nuget/packages.config b/src/.nuget/packages.config new file mode 100644 index 00000000..fc4c9492 --- /dev/null +++ b/src/.nuget/packages.config @@ -0,0 +1,6 @@ + + + + + + \ No newline at end of file diff --git a/src/coverage.bat b/src/coverage.bat new file mode 100644 index 00000000..fc96e1bf --- /dev/null +++ b/src/coverage.bat @@ -0,0 +1,5 @@ +.\packages\OpenCover.4.5.3723\OpenCover.Console.exe -register:user "-filter:+[SimpleKafka]* -[*Test]*" "-target:.\packages\NUnit.Console.3.0.0-beta-1\tools\nunit-console.exe" "-targetargs:.\SimpleKafkaTests\bin\Debug\SimpleKafkaTests.dll" + +.\packages\ReportGenerator.2.1.4.0\ReportGenerator.exe "-reports:results.xml" "-targetdir:.\coverage" + +pause From e506b16244c3950c1a306bd484d71be40c52535a Mon Sep 17 00:00:00 2001 From: Nick Randell Date: Mon, 20 Apr 2015 09:42:14 +0100 Subject: [PATCH 14/14] Getting a nuget package built --- RELEASE_NOTES.md | 74 ++------------------- build.bat | 23 +++++++ src/simple-kafka-net.nuspec | 22 ++++++ src/{kafka-net.sln => simple-kafka-net.sln} | 0 4 files changed, 50 insertions(+), 69 deletions(-) create mode 100644 build.bat create mode 100644 src/simple-kafka-net.nuspec rename src/{kafka-net.sln => simple-kafka-net.sln} (100%) diff --git a/RELEASE_NOTES.md b/RELEASE_NOTES.md index 56336ace..8a9da620 100644 --- a/RELEASE_NOTES.md +++ b/RELEASE_NOTES.md @@ -1,75 +1,11 @@ -kafka-net Release Notes +simple-kafka-net Release Notes ========= -Version 0.9.0.14 +Version 0.1 ------- -Fix memory leak in NagleBlockingCollection. -Timeout does not reset when new data is added. -Fix thread contention when producer has many threads loading data into it's buffer. -Fix many deadlock senarios on cancelling and disposing. -More unit tests around threading. - - -Version 0.9.0.1 -------- - -#### Feature: Nagle Producer -The producer class has been significantly updated to use a message batching technique similar to the [nagle algorithm]. - - -The producer accepts messages and groups them together into a single batched transmission. The total number of messages to batch before sending and the maximum amount of time to wait for the max batch size, is configurable. Tunning these two parameters, along with gzip compression can increase the driver throughput by orders of magnitude. - -```sh -var producer = new Producer(new BrokerRouter(options)) - { - BatchSize = 100, - BatchDelayTime = TimeSpan.FromMilliseconds(100) - }; - -// BatchSize - The producer will wait until it receives 100 messages, group them together into one request and send. -// BatchDelayTime - If the producer has not received 100 messages within 100 milliseconds, the producer will send what it has received. - -``` - - -#### Feature: Memory management -The producer now has better options for managing how much memory it consumes when it starts to get backed up. - -There are now two parameters on the producer constructor: -MaximumAsyncRequests -MaximumMessageBuffer - -These two parameters prevents the producer from going over a maximum of resources used in terms of network and memory. - -##### MaximumMessageBuffer -This parameter represents the maximum number of messages the producer will hold in its buffer at any one time. This includes all in flight messages and those buffered by the batching mechanism. This maximum will be hit if more messages arrive to the producer than it can send to Kafka. When the maximum is reached, the producer will block on any new messages until space is available. - -##### MaximumAsyncRequests -This parameter represents the maximum number of queued up async TCP commands allowed in flight at any one time. This can occur when the batch size is too low and the producer creates a high number of transmission requests out to the Kafka brokers. Having thousands of queued up async messages can adversly affect memory and increase timeout errors. - -```sh -var producer = new Producer(new BrokerRouter(options), maximumAsyncRequests: 30, maximumMessageBuffer:1000); - -//maximum outbound async requests will be limited to 30 -//maximum amount of messages in the producer at any one time will be limited to 1000 -``` - -#### Issues/Features Summary -* Fix some integration tests to run on any Kafka configuration. More need conversion. -* Redesign of TcpKafkaSockets - * Performance improvements - * Remove several deadlock senarios - * Remove several race conditions -* Nagle producer - * Memory management - * Significant performance improvement -* Add MaximumReconnectionTimeout - * Put a maximum amount of time to wait when backing off -* Update documentation in code -* Update/extend unit tests - - +Initial version taken from the original jroland/kafka-net +Very basic functionality, but appears to be good enough to work with a simple broker. -[nagle algorithm]:http://en.wikipedia.org/wiki/Nagle%27s_algorithm \ No newline at end of file +[jroland/kafka-net]:https://github.com/jroland/kafka-net \ No newline at end of file diff --git a/build.bat b/build.bat new file mode 100644 index 00000000..ece57b04 --- /dev/null +++ b/build.bat @@ -0,0 +1,23 @@ +@echo Off +set config=%1 +if "%config%" == "" ( + set config=Release +) + +set version=0.1.0 +if not "%PackageVersion%" == "" ( + set version=%PackageVersion% +) + +set nuget= +if "%nuget%" == "" ( + set nuget=nuget +) + +%WINDIR%\Microsoft.NET\Framework\v4.0.30319\msbuild src\simple-kafka-net.sln /p:Configuration="%config%" /m /v:M /fl /flp:LogFile=msbuild.log;Verbosity=diag /nr:false + +mkdir Build +mkdir Build\lib +mkdir Build\lib\net45 + +%nuget% pack "src\simple-kafka-net.nuspec" -NoPackageAnalysis -verbosity detailed -o Build -Version %version% -p Configuration="%config%" \ No newline at end of file diff --git a/src/simple-kafka-net.nuspec b/src/simple-kafka-net.nuspec new file mode 100644 index 00000000..5a2e4268 --- /dev/null +++ b/src/simple-kafka-net.nuspec @@ -0,0 +1,22 @@ + + + + simple-kafka-net + $version$ + Nick Randell + Nick Randell + https://github.com/nrandell/simple-kafka-net/blob/master/LICENSE + https://github.com/nrandell/simple-kafka-net + http://kafka.apache.org/images/kafka_logo.png + simple-kafka-net + false + Provides high and low level classes for communicating with an Apache Kafka cluster. Based off jroland/kafka-net but simplified and providing more control for clients + Native C# client for Apache Kafka. + Copyright Nick Randell 2015 + C# Apache Kafka + https://github.com/nrandell/simple-kafka-net/blob/master/RELEASE_NOTES.md + + + + + \ No newline at end of file diff --git a/src/kafka-net.sln b/src/simple-kafka-net.sln similarity index 100% rename from src/kafka-net.sln rename to src/simple-kafka-net.sln