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/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
------------
-[](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/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/.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/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
new file mode 100644
index 00000000..3e2ef77d
--- /dev/null
+++ b/src/SimpleKafka/Common/Crc32Provider.cs
@@ -0,0 +1,59 @@
+// 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, int offset, int length)
+ {
+ return ~CalculateHash(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;
+ while (length-- > 0)
+ {
+ crc = (crc >> 8) ^ PolynomialTable[buffer[offset++] ^ crc & 0xff];
+ }
+ return crc;
+ }
+
+ }
+}
\ 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/Extensions.cs b/src/SimpleKafka/Extensions.cs
new file mode 100644
index 00000000..b8aaea2d
--- /dev/null
+++ b/src/SimpleKafka/Extensions.cs
@@ -0,0 +1,68 @@
+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 GetOrCreate(this IDictionary 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 IDictionary map, TKey key)
+ where TValue : class
+ {
+ TValue result;
+ if (map.TryGetValue(key, out result))
+ {
+ return result;
+ }
+ else
+ {
+ 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/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
new file mode 100644
index 00000000..71738910
--- /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(TPartitionKey partitionKey, int numberOfPartitions);
+ }
+}
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/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/KafkaBrokers.cs b/src/SimpleKafka/KafkaBrokers.cs
new file mode 100644
index 00000000..b51f467f
--- /dev/null
+++ b/src/SimpleKafka/KafkaBrokers.cs
@@ -0,0 +1,410 @@
+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 HashSet brokers = new HashSet();
+ private readonly Dictionary topicToPartitions = new Dictionary(StringComparer.CurrentCultureIgnoreCase);
+
+ private readonly Dictionary connections = new Dictionary();
+
+ 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
+ {
+ 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)
+ {
+ 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);
+ }
+ }
+
+
+ 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> GetValidPartitionsForTopicsAsync(IEnumerable topics, CancellationToken token)
+ {
+ var result = await GetPartitionsForTopicsAsync(topics, token).ConfigureAwait(false);
+ var anyWithElection = result.Values.Any(partitions => partitions.Any(partition => partition.LeaderId == -1));
+ if (!anyWithElection)
+ {
+ return result;
+ }
+ else
+ {
+ throw new SimpleKafka.Protocol.LeaderNotFoundException("Not all leaders found");
+ }
+ }
+
+ 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;
+ }
+
+ private static Partition GetPartitionIfReady(string topic, int partitionId, Partition[] partitions)
+ {
+ if (partitionId >= partitions.Length)
+ {
+ throw new InvalidPartitionException("Topic {0} partition {1} is too big. Only have {2} partitions", topic, partitionId, partitions.Length);
+ }
+
+ 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)
+ {
+ await RefreshAsync(token).ConfigureAwait(false);
+ }
+
+ var backoffs = new BackoffHandler();
+
+ var ready = false;
+ while (!ready)
+ {
+ ready = true;
+ foreach (var topicKvp in topicMap)
+ {
+ var topic = topicKvp.Key;
+ var partitions = await GetPartitionsForTopicAsync(topic, token).ConfigureAwait(false);
+
+ foreach (var partitionKvp in topicKvp.Value)
+ {
+ var partitionNumber = partitionKvp.Key;
+ var partition = GetPartitionIfReady(topic, partitionNumber, partitions);
+ if (partition == null)
+ {
+ ready = false;
+ break;
+ }
+ }
+ if (!ready)
+ {
+ break;
+ }
+ }
+
+ if (!ready)
+ {
+ if (!await backoffs.BackoffIfAllowedAsync(token).ConfigureAwait(false))
+ {
+ throw new InvalidPartitionException("Cannot build broker map");
+ }
+ await RefreshAsync(token);
+ }
+ }
+
+ 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.GetOrCreate(partition.LeaderId);
+ brokerTopics.Add(Tuple.Create(topic, partitionNumber), partitionKvp.Value);
+ }
+ }
+
+ return brokerMap;
+ }
+
+
+ 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)
+ {
+ break;
+ }
+ 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(Topic[] topics)
+ {
+ var previousTopics = new HashSet(topicToPartitions.Keys);
+
+
+ foreach (var topic in topics)
+ {
+ if (topic.ErrorCode != ErrorResponseCode.NoError)
+ {
+ 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.Length))
+ {
+ currentPartitions = new Partition[topic.Partitions.Length];
+ topicToPartitions[topic.Name] = currentPartitions;
+ }
+
+ foreach (var partition in topic.Partitions)
+ {
+ if (partition.ErrorCode != ErrorResponseCode.NoError)
+ {
+ Log.Verbose("Topic {topic} partition {partition} has error {error}", topic.Name, partition.PartitionId, partition.ErrorCode);
+ }
+ currentPartitions[partition.PartitionId] = partition;
+ }
+
+ previousTopics.Remove(topic.Name);
+ }
+ }
+
+ foreach (var oldTopic in previousTopics)
+ {
+ topicToPartitions.Remove(oldTopic);
+ }
+ }
+
+ private async Task RefreshBrokersAsync(Broker[] latestBrokers, CancellationToken token)
+ {
+ foreach (var broker in latestBrokers)
+ {
+ var uri = broker.Address;
+
+ if (!brokers.Contains(uri)) {
+ brokers.Add(uri);
+ }
+
+ var currentConnection = connections.TryGetValue(broker.BrokerId);
+ if (currentConnection == null)
+ {
+ var newConnection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(uri, token).ConfigureAwait(false);
+ connections.Add(broker.BrokerId, newConnection);
+ }
+ }
+ }
+
+ 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..fedce6d0
--- /dev/null
+++ b/src/SimpleKafka/KafkaConnection.cs
@@ -0,0 +1,105 @@
+using SimpleKafka.Protocol;
+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 byte[] buffer;
+ private readonly NetworkStream stream;
+ private KafkaDecoder decoder;
+ private KafkaEncoder encoder;
+
+ private KafkaConnection(IPEndPoint serverEndpoint, TcpClient client, int bufferSize = 1048576)
+ {
+ this.serverEndpoint = serverEndpoint;
+ this.client = client;
+ this.stream = client.GetStream();
+ this.buffer = new byte[bufferSize];
+ decoder = new KafkaDecoder(buffer);
+ encoder = new KafkaEncoder(buffer);
+ }
+
+
+ private async Task ReceiveResponseAsync(CancellationToken token)
+ {
+ 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;
+ }
+
+ public async Task SendRequestAsync(BaseRequest request, CancellationToken token)
+ {
+ await clientLock.WaitAsync(token).ConfigureAwait(false);
+ try
+ {
+ encoder.Reset();
+ var marker = encoder.PrepareForLength();
+ 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(decoder);
+ return result;
+ }
+ else
+ {
+ return default(T);
+ }
+ }
+ finally
+ {
+ clientLock.Release();
+ }
+ }
+
+ #region IDisposable Support
+ private bool disposedValue = false; // To detect redundant calls
+
+ protected virtual void Dispose(bool disposing)
+ {
+ if (!disposedValue)
+ {
+ if (disposing)
+ {
+ stream.Dispose();
+ client.Close();
+ clientLock.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/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/KafkaConsumer.cs b/src/SimpleKafka/KafkaConsumer.cs
new file mode 100644
index 00000000..9d2f85e9
--- /dev/null
+++ b/src/SimpleKafka/KafkaConsumer.cs
@@ -0,0 +1,344 @@
+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 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
+ {
+ public readonly OffsetSelectionStrategy failureOffsetSelection;
+ public long nextOffset;
+
+ public TopicTracker(TopicSelector selector)
+ {
+ switch (selector.DefaultOffsetSelection)
+ {
+
+ 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);
+ }
+ 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;
+ 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)
+ {
+ this.consumerGroup = consumerGroup;
+ this.brokers = brokers;
+ this.keySerializer = keySerializer;
+ this.valueSerializer = valueSerializer;
+ topicMap = new Dictionary>();
+ foreach (var topic in topics)
+ {
+ var partitionMap = topicMap.GetOrCreate(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 CommitAsync(IEnumerable offsets, CancellationToken token)
+ {
+ var coordinatorId = await GetOffsetCoordinatorIdAsync(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 = consumerGroupGenerationId,
+ ConsumerId = consumerId,
+ OffsetCommits = offsetCommits
+ };
+
+ var responses = await brokers.RunBrokerCommand(coordinatorId, c =>
+ c.SendRequestAsync(request, token)).ConfigureAwait(false);
+ foreach (var response in responses)
+ {
+ if (response.Error != ErrorResponseCode.NoError)
+ {
+ throw new InvalidOperationException("Failed to commit: " + response.Error);
+ }
+ }
+ }
+
+ private async Task GetOffsetCoordinatorIdAsync(CancellationToken token)
+ {
+ var map = await brokers.BuildOffsetCoordinatorMapAsync(token, consumerGroup).ConfigureAwait(false);
+ var coordinator = map[consumerGroup];
+ return coordinator;
+ }
+
+ public async Task>> ReceiveAsync(CancellationToken token)
+ {
+ 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)
+ {
+ foreach (var fetchResponse in taskResult)
+ {
+ if (fetchResponse.Error != (int)ErrorResponseCode.NoError)
+ {
+ Log.Error("Error in fetch response {error} for {topic}/{partition}", 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.RunBrokerCommand(brokerId, c =>
+ c.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 = 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.RunBrokerCommand(brokerKvp.Key, c => c.SendRequestAsync(request, token)).ConfigureAwait(false);
+ foreach (var response in responses)
+ {
+ if (response.Error != ErrorResponseCode.NoError)
+ {
+ throw new InvalidOperationException("Unknown error fetching offsets: " + 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 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)
+ {
+ throw new InvalidOperationException("Unknown error fetching offsets: " + response.Error);
+ }
+ var tracker = trackerMap[Tuple.Create(response.Topic, response.PartitionId)];
+ tracker.nextOffset = response.Offset + 1;
+ }
+ }
+ }
+
+
+ }
+
+
+ }
+}
diff --git a/src/SimpleKafka/KafkaDecoder.cs b/src/SimpleKafka/KafkaDecoder.cs
new file mode 100644
index 00000000..90c3b508
--- /dev/null
+++ b/src/SimpleKafka/KafkaDecoder.cs
@@ -0,0 +1,128 @@
+using SimpleKafka.Protocol;
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace SimpleKafka
+{
+ internal class KafkaDecoder
+ {
+ 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 KafkaDecoder(byte[] buffer) : this(buffer, 0, buffer.Length) { }
+ public KafkaDecoder(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 ErrorResponseCode ReadErrorResponseCode()
+ {
+ return (ErrorResponseCode)ReadInt16();
+ }
+
+ public string ReadString()
+ {
+ var length = ReadInt16();
+ 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[] ReadBytes()
+ {
+ 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/KafkaEncoder.cs b/src/SimpleKafka/KafkaEncoder.cs
new file mode 100644
index 00000000..aa4f816d
--- /dev/null
+++ b/src/SimpleKafka/KafkaEncoder.cs
@@ -0,0 +1,154 @@
+using SimpleKafka.Common;
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace SimpleKafka
+{
+ internal class KafkaEncoder
+ {
+ private int offset;
+ public int Offset { get { return offset; } }
+
+ public KafkaEncoder SetOffset(int offset)
+ {
+ this.offset = offset;
+ return this;
+ }
+
+ private readonly byte[] buffer;
+ public byte[] Buffer { get { return buffer; } }
+
+ public KafkaEncoder(byte[] buffer, int offset = 0)
+ {
+ this.offset = offset;
+ this.buffer = buffer;
+ }
+
+ public KafkaEncoder Reset()
+ {
+ offset = 0;
+ return this;
+ }
+
+ public KafkaEncoder 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);
+ }
+ return this;
+ }
+ public KafkaEncoder Write(int value)
+ {
+ unchecked
+ {
+ buffer[offset++] = (byte)((value >> 24));
+ buffer[offset++] = (byte)((value >> 16));
+ buffer[offset++] = (byte)((value >> 8));
+ buffer[offset++] = (byte)(value);
+ }
+ return this;
+ }
+
+ public KafkaEncoder Write(uint value)
+ {
+ unchecked
+ {
+ buffer[offset++] = (byte)((value >> 24));
+ buffer[offset++] = (byte)((value >> 16));
+ buffer[offset++] = (byte)((value >> 8));
+ buffer[offset++] = (byte)(value);
+ }
+ return this;
+ }
+
+ public KafkaEncoder Write(short value)
+ {
+ unchecked
+ {
+ buffer[offset++] = (byte)((value >> 8));
+ buffer[offset++] = (byte)(value);
+ }
+ return this;
+ }
+
+ public KafkaEncoder Write(byte value)
+ {
+ buffer[offset++] = value;
+ return this;
+ }
+
+ public KafkaEncoder Write(string data)
+ {
+ if (data == null)
+ {
+ Write((short)-1);
+ }
+ else
+ {
+ var bytesWritten = Encoding.UTF8.GetBytes(data, 0, data.Length, buffer, offset + 2);
+ Write((short)bytesWritten);
+ offset += bytesWritten;
+ }
+ return this;
+ }
+
+ public KafkaEncoder Write(byte[] data)
+ {
+ if (data == null)
+ {
+ Write(-1);
+ }
+ else
+ {
+ Write(data.Length);
+ Array.Copy(data, 0, buffer, offset, data.Length);
+ offset += data.Length;
+ }
+ return this;
+ }
+
+ public int PrepareForCrc()
+ {
+ offset += 4;
+ return offset;
+ }
+
+ 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;
+ }
+
+ public int PrepareForLength()
+ {
+ offset += 4;
+ return offset;
+ }
+
+ 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
new file mode 100644
index 00000000..050d99cd
--- /dev/null
+++ b/src/SimpleKafka/KafkaProducer.cs
@@ -0,0 +1,219 @@
+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 static class KafkaProducer {
+ public static KafkaProducer Create(
+ KafkaBrokers brokers,
+ IKafkaSerializer valueSerializer) {
+ return new KafkaProducer(brokers,
+ new NullSerializer(),
+ valueSerializer,
+ new LoadBalancedPartitioner());
+ }
+
+ public static KafkaProducer Create(
+ KafkaBrokers brokers,
+ IKafkaSerializer keySerializer,
+ IKafkaSerializer valueSerializer
+ ) {
+ return new KafkaProducer(
+ brokers,
+ keySerializer,
+ valueSerializer,
+ new LoadBalancedPartitioner());
+ }
+
+ 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 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(KeyedMessage message, CancellationToken token)
+ {
+ await SendAsync(new[] { message}, token).ConfigureAwait(false);
+ }
+
+ public async Task SendAsync(IEnumerable> messages, CancellationToken token)
+ {
+ var backoffs = new BackoffHandler();
+ var topicMap = BuildTopicMap(messages);
+ while (topicMap != null)
+ {
+ 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);
+ }
+ catch (OperationCanceledException)
+ {
+ throw;
+ }
+ catch (Exception ex)
+ {
+ Log.Information("Error sending: {ex}", ex);
+ caughtException = ex;
+ token.ThrowIfCancellationRequested();
+ }
+ if (topicMap != null)
+ {
+ 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);
+ }
+ }
+ }
+
+ private Dictionary, List>>> BuildBrokerMap(Dictionary>> topicMap, Dictionary partitionsMap)
+ {
+ var brokerMap = new Dictionary, List>>>();
+ foreach (var messageSet in topicMap.Values)
+ {
+ foreach (var message in messageSet)
+ {
+ 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;
+ }
+
+ private async Task>> SendMessagesAsync(Dictionary, List>>> brokerMap, CancellationToken token)
+ {
+ var tasks = new List(brokerMap.Count);
+ var results = new Dictionary>();
+ foreach (var brokerKvp in brokerMap)
+ {
+ var request = new ProduceRequest
+ {
+ 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
+ .RunBrokerCommand(brokerId, c =>
+ c.SendRequestAsync(request, token))
+ .ContinueWith(task => results.Add(brokerId, task.Result), token)
+ );
+ }
+ 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)
+ {
+ 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)
+ {
+ var result = new Dictionary>>();
+
+ foreach (var message in messages)
+ {
+ var encoded = new Message
+ {
+ Key = keySerializer.Serialize(message.Key),
+ Value = valueSerializer.Serialize(message.Value)
+ };
+ var prepared = KeyedMessage.Create(message.Topic, (object)null, message.PartitionKey, encoded);
+ result.GetOrCreate(message.Topic).Add(prepared);
+ }
+ 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
new file mode 100644
index 00000000..cc8dda33
--- /dev/null
+++ b/src/SimpleKafka/LoadBalancedPartitioner.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 LoadBalancedPartitioner : IKafkaMessagePartitioner
+ {
+ private int current;
+
+ 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/NullKeySerializer.cs b/src/SimpleKafka/NullKeySerializer.cs
new file mode 100644
index 00000000..2cc5a94d
--- /dev/null
+++ b/src/SimpleKafka/NullKeySerializer.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/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/SimpleKafka/Protocol/BaseRequest.cs b/src/SimpleKafka/Protocol/BaseRequest.cs
new file mode 100644
index 00000000..47643764
--- /dev/null
+++ b/src/SimpleKafka/Protocol/BaseRequest.cs
@@ -0,0 +1,71 @@
+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;
+ private readonly ApiKeyRequestType apiKey;
+
+ protected BaseRequest(ApiKeyRequestType apiKey, short apiVersion = 0)
+ {
+ this.apiKey = apiKey;
+ _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 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)
+ {
+ 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
new file mode 100644
index 00000000..fb9d00b7
--- /dev/null
+++ b/src/SimpleKafka/Protocol/Broker.cs
@@ -0,0 +1,25 @@
+using System;
+using SimpleKafka.Common;
+
+namespace SimpleKafka.Protocol
+{
+ public class Broker
+ {
+ 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));} }
+
+ private Broker(int brokerId, string host, int port)
+ {
+ 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
new file mode 100644
index 00000000..fdaa6101
--- /dev/null
+++ b/src/SimpleKafka/Protocol/ConsumerMetadataRequest.cs
@@ -0,0 +1,73 @@
+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
+ {
+ public string ConsumerGroup { get; set; }
+
+ public ConsumerMetadataRequest() : base(ApiKeyRequestType.ConsumerMetadataRequest) { }
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeConsumerMetadataRequest(this, encoder);
+ }
+
+
+ internal override ConsumerMetadataResponse Decode(KafkaDecoder decoder)
+ {
+ return DecodeConsumerMetadataResponse(decoder);
+ }
+
+ private static KafkaEncoder EncodeConsumerMetadataRequest(ConsumerMetadataRequest request, KafkaEncoder encoder)
+ {
+ return
+ request
+ .EncodeHeader(encoder)
+ .Write(request.ConsumerGroup);
+ }
+
+ private static ConsumerMetadataResponse DecodeConsumerMetadataResponse(KafkaDecoder decoder)
+ {
+ var correlationId = decoder.ReadInt32();
+ return ConsumerMetadataResponse.Decode(decoder);
+ }
+ }
+
+ public class ConsumerMetadataResponse
+ {
+ ///
+ /// Error code of exception that occured during the request. Zero if no error.
+ ///
+ public readonly ErrorResponseCode Error;
+
+ public readonly int CoordinatorId;
+ public readonly string CoordinatorHost;
+ public readonly int CoordinatorPort;
+
+ private ConsumerMetadataResponse(ErrorResponseCode error, int coordinatorId, string coordinatorHost, int coordinatorPort)
+ {
+ 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
new file mode 100644
index 00000000..9fd0e47c
--- /dev/null
+++ b/src/SimpleKafka/Protocol/FetchRequest.cs
@@ -0,0 +1,184 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using SimpleKafka.Common;
+
+namespace SimpleKafka.Protocol
+{
+ public class FetchRequest : BaseRequest>
+ {
+ 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 FetchRequest() : base(ApiKeyRequestType.Fetch) { }
+
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeFetchRequest(this, encoder);
+ }
+
+ internal override List Decode(KafkaDecoder decoder)
+ {
+ return DecodeFetchResponses(decoder);
+ }
+
+ private static KafkaEncoder EncodeFetchRequest(FetchRequest request, KafkaEncoder encoder)
+ {
+ request
+ .EncodeHeader(encoder)
+ .Write(ReplicaId)
+ .Write(request.MaxWaitTime)
+ .Write(request.MinBytes);
+
+ 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
+ {
+ // 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);
+ }
+
+ 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(KafkaDecoder decoder)
+ {
+ var correlationId = decoder.ReadInt32();
+
+ var result = new List();
+
+ var topicCount = decoder.ReadInt32();
+ for (int i = 0; i < topicCount; i++)
+ {
+ var topic = decoder.ReadString();
+
+ var partitionCount = decoder.ReadInt32();
+ for (int j = 0; j < partitionCount; j++)
+ {
+ var response = FetchResponse.Decode(decoder, topic);
+ result.Add(response);
+ }
+ }
+ return result;
+ }
+
+ }
+
+ 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
+ {
+ public readonly string Topic;
+ public readonly int PartitionId;
+ public readonly ErrorResponseCode Error;
+ public readonly long HighWaterMark;
+ public readonly IList Messages;
+
+ 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;
+ }
+
+ internal static FetchResponse Decode(KafkaDecoder decoder, string topic)
+ {
+ 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
new file mode 100644
index 00000000..a14c6816
--- /dev/null
+++ b/src/SimpleKafka/Protocol/Message.cs
@@ -0,0 +1,209 @@
+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 IList 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.
+ internal static KafkaEncoder EncodeMessageSet(KafkaEncoder encoder, IEnumerable messages)
+ {
+ foreach (var message in messages)
+ {
+ encoder.Write(InitialMessageOffset);
+ var marker = encoder.PrepareForLength();
+ EncodeMessage(message, encoder)
+ .WriteLength(marker);
+ }
+ return encoder;
+ }
+
+ ///
+ /// Decode a byte[] that represents a collection of messages.
+ ///
+ /// The decoder positioned at the start of the buffer
+ /// The messages
+ internal static List DecodeMessageSet(int partitionId, KafkaDecoder decoder, int messageSetSize)
+ {
+ var numberOfBytes = messageSetSize;
+
+ var messages = new List();
+ while (numberOfBytes > 0)
+ {
+
+ 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);
+ }
+
+ numberOfBytes -= MessageHeaderSize;
+ if (numberOfBytes < messageSize)
+ {
+ break;
+ }
+
+ var message = DecodeMessage(offset, partitionId, decoder, messageSize);
+ messages.Add(message);
+ numberOfBytes -= messageSize;
+ }
+ return messages;
+ }
+
+ ///
+ /// 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[])
+ ///
+ internal static KafkaEncoder EncodeMessage(Message message, KafkaEncoder encoder)
+ {
+ var marker = encoder.PrepareForCrc();
+ encoder
+ .Write(message.MagicNumber)
+ .Write(message.Attribute)
+ .Write(message.Key)
+ .Write(message.Value)
+ .CalculateCrc(marker);
+
+ return encoder;
+ }
+
+ ///
+ /// 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.
+ /// The message
+ /// The return type is an Enumerable as the message could be a compressed message set.
+ 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);
+ if (calculatedCrc != crc)
+ {
+ throw new FailCrcCheckException("Payload did not match CRC validation.");
+ }
+
+ var message = new Message
+ {
+ Meta = new MessageMetadata(offset, partitionId),
+ MagicNumber = decoder.ReadByte(),
+ Attribute = decoder.ReadByte(),
+ Key = decoder.ReadBytes(),
+
+ };
+ var codec = (MessageCodec)(ProtocolConstants.AttributeCodeMask & message.Attribute);
+ switch (codec)
+ {
+ case MessageCodec.CodecNone:
+ message.Value = decoder.ReadBytes();
+ break;
+
+ default:
+ throw new NotSupportedException(string.Format("Codec type of {0} is not supported.", codec));
+ }
+ return message;
+ }
+ }
+
+ ///
+ /// 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 readonly long Offset;
+ ///
+ /// The partition id this offset is from.
+ ///
+ 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
new file mode 100644
index 00000000..91617df6
--- /dev/null
+++ b/src/SimpleKafka/Protocol/MetadataRequest.cs
@@ -0,0 +1,86 @@
+using System.Collections.Generic;
+using SimpleKafka.Common;
+
+namespace SimpleKafka.Protocol
+{
+ public class MetadataRequest : BaseRequest
+ {
+ ///
+ /// The list of topics to get metadata for.
+ ///
+ public List Topics { get; set; }
+
+ public MetadataRequest() : base(ApiKeyRequestType.MetaData) { }
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeMetadataRequest(this, encoder);
+ }
+
+ internal override MetadataResponse Decode(KafkaDecoder decoder)
+ {
+ return DecodeMetadataResponse(decoder);
+ }
+
+ private static KafkaEncoder EncodeMetadataRequest(MetadataRequest request, KafkaEncoder encoder)
+ {
+ request
+ .EncodeHeader(encoder);
+
+ if (request.Topics == null)
+ {
+ encoder.Write(0);
+ }
+ else
+ {
+ encoder.Write(request.Topics.Count);
+ foreach (var topic in request.Topics)
+ {
+ encoder.Write(topic);
+ }
+ }
+
+ return encoder;
+ }
+
+ private static MetadataResponse DecodeMetadataResponse(KafkaDecoder decoder)
+ {
+ 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 Broker[brokerCount];
+ for (var i = 0; i < brokerCount; i++)
+ {
+ brokers[i] = Broker.Decode(decoder);
+ }
+
+ var topicCount = decoder.ReadInt32();
+ var topics = new Topic[topicCount];
+ for (var i = 0; i < topicCount; i++)
+ {
+ topics[i] = Topic.Decode(decoder);
+ }
+
+ var response = new MetadataResponse(brokers, topics);
+
+ return response;
+
+ }
+ }
+}
\ 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..0c04e9a6
--- /dev/null
+++ b/src/SimpleKafka/Protocol/OffsetCommitRequest.cs
@@ -0,0 +1,186 @@
+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>
+ {
+ public OffsetCommitRequest(Int16 version = 1)
+ : base(ApiKeyRequestType.OffsetCommit, version)
+ {
+ }
+ public string ConsumerGroup { get; set; }
+ public int ConsumerGroupGenerationId { get; set; }
+ public string ConsumerId { get; set; }
+ public List OffsetCommits { get; set; }
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeOffsetCommitRequest(this, encoder);
+ }
+
+ internal override List Decode(KafkaDecoder decoder)
+ {
+ return DecodeOffsetCommitResponse(decoder);
+ }
+
+ private static KafkaEncoder EncodeOffsetCommitRequest(OffsetCommitRequest request, KafkaEncoder encoder)
+ {
+ request
+ .EncodeHeader(encoder)
+ .Write(request.ConsumerGroup);
+
+ if (request.ApiVersion == 1)
+ {
+ encoder
+ .Write(request.ConsumerGroupGenerationId)
+ .Write(request.ConsumerId);
+ }
+
+ if (request.OffsetCommits == null)
+ {
+ // Nothing to commit
+ encoder.Write(0);
+ }
+ else if (request.OffsetCommits.Count == 1)
+ {
+ var commit = request.OffsetCommits[0];
+ // Shortcut the single version
+ encoder
+ .Write(1)
+ .Write(commit.Topic)
+ .Write(1);
+
+ EncodeCommit(encoder, request.ApiVersion, commit);
+ }
+ else
+ {
+ // Complete complex request
+ var topicGroups = new Dictionary>();
+ foreach (var commit in request.OffsetCommits)
+ {
+ var topicGroup = topicGroups.GetOrCreate(commit.Topic, () => new List(request.OffsetCommits.Count));
+ topicGroup.Add(commit);
+ }
+
+ encoder.Write(topicGroups.Count);
+ foreach (var topicGroupKvp in topicGroups)
+ {
+ var topic = topicGroupKvp.Key;
+ var commits = topicGroupKvp.Value;
+ encoder
+ .Write(topic)
+ .Write(commits.Count);
+
+ 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(KafkaDecoder decoder)
+ {
+ var correlationId = decoder.ReadInt32();
+
+ var responses = new List();
+ var topicCount = decoder.ReadInt32();
+ for (int i = 0; i < topicCount; i++)
+ {
+ var topic = decoder.ReadString();
+
+ var partitionCount = decoder.ReadInt32();
+ for (int j = 0; j < partitionCount; j++)
+ {
+ var response = OffsetCommitResponse.Decode(decoder, topic);
+ responses.Add(response);
+ }
+ }
+ return responses;
+ }
+ }
+
+ 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 readonly string Topic;
+ ///
+ /// The id of the partition this response is for.
+ ///
+ public readonly int PartitionId;
+ ///
+ /// Error code of exception that occured during the request. Zero if no 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
new file mode 100644
index 00000000..aca77fb8
--- /dev/null
+++ b/src/SimpleKafka/Protocol/OffsetFetchRequest.cs
@@ -0,0 +1,167 @@
+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>
+ {
+ public OffsetFetchRequest(short version = 1) : base(ApiKeyRequestType.OffsetFetch, version)
+ {
+
+ }
+ public string ConsumerGroup { get; set; }
+ public List Topics { get; set; }
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeOffsetFetchRequest(this, encoder);
+ }
+
+ private static KafkaEncoder EncodeOffsetFetchRequest(OffsetFetchRequest request, KafkaEncoder encoder)
+ {
+ request
+ .EncodeHeader(encoder)
+ .Write(request.ConsumerGroup);
+
+ if (request.Topics == null)
+ {
+ // nothing to fetch
+ encoder.Write(0);
+ }
+ else if (request.Topics.Count == 1)
+ {
+ // 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);
+ }
+
+ encoder.Write(topicGroups.Count);
+ foreach (var kvp in topicGroups)
+ {
+ var topic = kvp.Key;
+ var partitions = kvp.Value;
+ encoder
+ .Write(topic)
+ .Write(partitions.Count);
+ foreach (var fetch in partitions)
+ {
+ encoder.Write(fetch);
+ }
+
+ }
+ }
+
+ return encoder;
+ }
+
+ internal override List Decode(KafkaDecoder decoder)
+ {
+ return DecodeOffsetFetchResponse(decoder);
+ }
+
+
+ private static List DecodeOffsetFetchResponse(KafkaDecoder decoder)
+ {
+ var correlationId = decoder.ReadInt32();
+
+ var responses = new List();
+ var topicCount = decoder.ReadInt32();
+ for (int i = 0; i < topicCount; i++)
+ {
+ var topic = decoder.ReadString();
+
+ var partitionCount = decoder.ReadInt32();
+ for (int j = 0; j < partitionCount; j++)
+ {
+ var response = OffsetFetchResponse.Decode(decoder, topic);
+ responses.Add(response);
+ }
+ }
+ return responses;
+ }
+
+ }
+
+ 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 readonly string Topic;
+ ///
+ /// The id of the partition this response is for.
+ ///
+ public readonly int PartitionId;
+ ///
+ /// The offset position saved to the server.
+ ///
+ public readonly long Offset;
+ ///
+ /// Any arbitrary metadata stored during a CommitRequest.
+ ///
+ public readonly string MetaData;
+ ///
+ /// Error code of exception that occured during the request. Zero if no 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()
+ {
+ 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..19e1ba35
--- /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>
+ {
+ public List Offsets { get; set; }
+
+ public OffsetRequest() : base(ApiKeyRequestType.Offset) { }
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeOffsetRequest(this, encoder);
+ }
+
+ internal override List Decode(KafkaDecoder decoder)
+ {
+ return DecodeOffsetResponse(decoder);
+ }
+
+ private static KafkaEncoder EncodeOffsetRequest(OffsetRequest request, KafkaEncoder encoder)
+ {
+ request
+ .EncodeHeader(encoder)
+ .Write(ReplicaId);
+
+ if (request.Offsets == null)
+ {
+ encoder.Write(0);
+ }
+ else if (request.Offsets.Count == 1)
+ {
+ // 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);
+ }
+
+ encoder.Write(topicGroups.Count);
+ foreach (var kvp in topicGroups)
+ {
+ var topic = kvp.Key;
+ var offsets = kvp.Value;
+
+ encoder
+ .Write(topic)
+ .Write(offsets.Count);
+
+ foreach (var offset in offsets)
+ {
+ encoder
+ .Write(offset.PartitionId)
+ .Write(offset.Time)
+ .Write(offset.MaxOffsets);
+ }
+ }
+ }
+ return encoder;
+ }
+
+
+
+ private static List DecodeOffsetResponse(KafkaDecoder decoder)
+ {
+ var correlationId = decoder.ReadInt32();
+
+ var responses = new List();
+ var topicCount = decoder.ReadInt32();
+ for (int i = 0; i < topicCount; i++)
+ {
+ var topic = decoder.ReadString();
+
+ var partitionCount = decoder.ReadInt32();
+ for (int j = 0; j < partitionCount; j++)
+ {
+ var response = OffsetResponse.Decode(decoder, topic);
+
+ responses.Add(response);
+ }
+ }
+ return responses;
+ }
+
+ }
+
+ 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 readonly string Topic;
+ public readonly int PartitionId;
+ public readonly ErrorResponseCode Error;
+ public readonly long[] Offsets;
+
+ private OffsetResponse(string topic, int partitionId, ErrorResponseCode error, long[] offsets)
+ {
+ this.Topic = topic;
+ this.PartitionId = partitionId;
+ this.Error = error;
+ this.Offsets = offsets;
+
+ }
+
+ internal static OffsetResponse Decode(KafkaDecoder decoder, string topic)
+ {
+ var partitionId = decoder.ReadInt32();
+ var error = decoder.ReadErrorResponseCode();
+ var offsetCount = decoder.ReadInt32();
+ var offsets = new long[offsetCount];
+ for (int k = 0; k < offsetCount; k++)
+ {
+ offsets[k] = decoder.ReadInt64();
+ }
+ var response = new OffsetResponse(topic, partitionId, error, offsets);
+ return response;
+ }
+ }
+}
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
new file mode 100644
index 00000000..6dfa7932
--- /dev/null
+++ b/src/SimpleKafka/Protocol/ProduceRequest.cs
@@ -0,0 +1,193 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using SimpleKafka.Common;
+
+namespace SimpleKafka.Protocol
+{
+ public class ProduceRequest : BaseRequest>
+ {
+ ///
+ /// Provide a hint to the broker call not to expect a response for requests without Acks.
+ ///
+ public override bool ExpectResponse { get { return Acks > 0; } }
+ ///
+ /// 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 ProduceRequest() : base(ApiKeyRequestType.Produce) { }
+
+ internal override KafkaEncoder Encode(KafkaEncoder encoder)
+ {
+ return EncodeProduceRequest(this, encoder);
+ }
+
+ internal override List Decode(KafkaDecoder decoder)
+ {
+ return DecodeProduceResponse(decoder);
+ }
+
+ #region Protocol...
+ private static KafkaEncoder EncodeProduceRequest(ProduceRequest request, KafkaEncoder encoder)
+ {
+ 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
+ {
+ // 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);
+ }
+
+ encoder.Write(topicGroups.Count);
+ foreach (var kvp in topicGroups)
+ {
+ var topic = kvp.Key;
+ var payloads = kvp.Value;
+
+ encoder
+ .Write(topic)
+ .Write(payloads.Count);
+
+ foreach (var payload in payloads)
+ {
+ WritePayload(encoder, payload);
+ }
+ }
+ }
+ return encoder;
+ }
+
+ 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();
+
+ var responses = new List();
+ var topicCount = decoder.ReadInt32();
+ for (int i = 0; i < topicCount; i++)
+ {
+ var topic = decoder.ReadString();
+
+ var partitionCount = decoder.ReadInt32();
+ for (int j = 0; j < partitionCount; j++)
+ {
+ var response = ProduceResponse.Decode(decoder, topic);
+ responses.Add(response);
+ }
+ }
+ return responses;
+ }
+ #endregion
+ }
+
+ public class ProduceResponse
+ {
+ ///
+ /// The topic the offset came from.
+ ///
+ public readonly string Topic;
+ ///
+ /// The partition the offset came from.
+ ///
+ public readonly int PartitionId;
+ ///
+ /// Error response code. 0 is success.
+ ///
+ public readonly ErrorResponseCode Error;
+ ///
+ /// The offset number to commit as completed.
+ ///
+ 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)
+ {
+ 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..884ef704
--- /dev/null
+++ b/src/SimpleKafka/Protocol/Topic.cs
@@ -0,0 +1,38 @@
+using System;
+using System.Collections.Generic;
+using SimpleKafka.Common;
+
+namespace SimpleKafka.Protocol
+{
+ public class Topic
+ {
+ public readonly ErrorResponseCode ErrorCode;
+ public readonly string Name;
+ public readonly Partition[] Partitions;
+
+ private Topic(ErrorResponseCode errorCode, string name, Partition[] partitions)
+ {
+ 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 Partition[numPartitions];
+ for (int i = 0; i < numPartitions; i++)
+ {
+ partitions[i] = Partition.Decode(decoder);
+ }
+ var topic = new Topic(errorCode, name, partitions);
+
+ return topic;
+ }
+ }
+
+
+}
diff --git a/src/SimpleKafka/ReceivedKafkaMessage.cs b/src/SimpleKafka/ReceivedKafkaMessage.cs
new file mode 100644
index 00000000..dcc08747
--- /dev/null
+++ b/src/SimpleKafka/ReceivedKafkaMessage.cs
@@ -0,0 +1,32 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace SimpleKafka
+{
+ 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)
+ {
+ 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
new file mode 100644
index 00000000..7efd21aa
--- /dev/null
+++ b/src/SimpleKafka/SimpleKafka.csproj
@@ -0,0 +1,110 @@
+
+
+
+
+ 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/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/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/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
new file mode 100644
index 00000000..ad41be8b
--- /dev/null
+++ b/src/SimpleKafka/TopicSelector.cs
@@ -0,0 +1,35 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Text;
+using System.Threading.Tasks;
+
+namespace SimpleKafka
+{
+ public enum OffsetSelectionStrategy
+ {
+ Earliest = -2,
+ Last = -3,
+ Next = -1,
+ Specified = 0,
+ NextUncommitted = -4,
+ };
+
+ 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.Next;
+ }
+
+ }
+}
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/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..bff1a69a
--- /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 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 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/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/Helpers/MessageHelper.cs b/src/SimpleKafkaTests/Helpers/MessageHelper.cs
new file mode 100644
index 00000000..74b65849
--- /dev/null
+++ b/src/SimpleKafkaTests/Helpers/MessageHelper.cs
@@ -0,0 +1,32 @@
+using System;
+using SimpleKafka.Common;
+using SimpleKafka.Protocol;
+using SimpleKafka;
+
+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 = new Message
+ {
+ Attribute = attributes,
+ MagicNumber = magicByte,
+ Key = key,
+ Value = payload
+ };
+
+ var buffer = new byte[1024];
+ var encoder = new KafkaEncoder(buffer);
+ Message.EncodeMessageSet(encoder, new[] { message });
+
+ var result = new byte[encoder.Offset];
+ Array.Copy(encoder.Buffer, result, encoder.Offset);
+
+ return result;
+ }
+ }
+}
diff --git a/src/SimpleKafkaTests/Integration/ConsumerTests.cs b/src/SimpleKafkaTests/Integration/ConsumerTests.cs
new file mode 100644
index 00000000..760eb895
--- /dev/null
+++ b/src/SimpleKafkaTests/Integration/ConsumerTests.cs
@@ -0,0 +1,280 @@
+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
+ {
+ private readonly string defaultConsumerGroup = "unit-tests";
+
+ private KafkaTestCluster testCluster;
+
+ [OneTimeSetUp]
+ public void BuildTestCluster()
+ {
+ testCluster = new KafkaTestCluster("server.home", 1);
+ }
+
+ [OneTimeTearDown]
+ public void DestroyTestCluster()
+ {
+ testCluster.Dispose();
+ testCluster = null;
+ }
+
+
+ [Test]
+ public async Task TestSimpleConsumerWorksOk()
+ {
+ var keySerializer = new NullSerializer();
+ var valueSerializer = new StringSerializer();
+ var messagePartitioner = new LoadBalancedPartitioner();
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ 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 });
+
+ await producer.SendAsync(KeyedMessage.Create(topic, "Message"), CancellationToken.None);
+
+ var responses = await consumer.ReceiveAsync(CancellationToken.None);
+ 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"));
+ }
+ }
+
+ [Test]
+ public async Task TestProducing3MessagesAllowsTheConsumerToChooseTheCorrectMessage()
+ {
+ var valueSerializer = new StringSerializer();
+
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ var topic = temporaryTopic.Name;
+ {
+ var producer = KafkaProducer.Create(brokers, valueSerializer);
+
+ await producer.SendAsync(new[] {
+ KeyedMessage.Create(topic, "1"),
+ KeyedMessage.Create(topic, "2"),
+ KeyedMessage.Create(topic, "3"),
+ }, 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);
+ 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 = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Last });
+
+ var responses = await latest.ReceiveAsync(CancellationToken.None);
+ 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 = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector { Partition = 0, Topic = topic, DefaultOffsetSelection = OffsetSelectionStrategy.Next });
+
+ var responses = await latest.ReceiveAsync(CancellationToken.None);
+ Assert.That(responses, Is.Not.Null);
+ Assert.That(responses, Has.Count.EqualTo(0));
+
+ }
+
+ {
+ 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);
+ 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 valueSerializer = new StringSerializer();
+
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ var topic = temporaryTopic.Name;
+ {
+ var producer = KafkaProducer.Create(brokers, valueSerializer);
+
+ await producer.SendAsync(new[] {
+ KeyedMessage.Create(topic, "1"),
+ KeyedMessage.Create(topic, "2"),
+ KeyedMessage.Create(topic, "3"),
+ }, CancellationToken.None);
+ }
+
+ {
+ var noPreviousCommits = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector { Partition = 0, Topic = topic,
+ DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted,
+ FailureOffsetSelection = OffsetSelectionStrategy.Earliest });
+
+ var responses = await noPreviousCommits.ReceiveAsync(CancellationToken.None);
+ 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); ;
+ }
+
+ {
+ var previousCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector
+ {
+ Partition = 0,
+ Topic = topic,
+ DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted,
+ FailureOffsetSelection = OffsetSelectionStrategy.Earliest
+ });
+
+ var responses = await previousCommit.ReceiveAsync(CancellationToken.None);
+ 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 = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector
+ {
+ Partition = 0,
+ Topic = topic,
+ DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted,
+ FailureOffsetSelection = OffsetSelectionStrategy.Earliest
+ });
+
+ var responses = await previousCommitAgain.ReceiveAsync(CancellationToken.None);
+ 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); ;
+ }
+
+ {
+ var secondCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector
+ {
+ Partition = 0,
+ Topic = topic,
+ DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted,
+ FailureOffsetSelection = OffsetSelectionStrategy.Earliest
+ });
+
+ var responses = await secondCommit.ReceiveAsync(CancellationToken.None);
+ 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); ;
+ }
+
+ {
+ var thirdCommit = KafkaConsumer.Create(defaultConsumerGroup, brokers, valueSerializer,
+ new TopicSelector
+ {
+ Partition = 0,
+ Topic = topic,
+ DefaultOffsetSelection = OffsetSelectionStrategy.NextUncommitted,
+ FailureOffsetSelection = OffsetSelectionStrategy.Earliest
+ });
+
+ 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
new file mode 100644
index 00000000..d5ef60f3
--- /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);
+
+ 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 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);
+ }
+
+
+ {
+ 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 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);
+ }
+
+
+ {
+ 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(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/MultiplePartitionTests.cs b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs
new file mode 100644
index 00000000..805542cb
--- /dev/null
+++ b/src/SimpleKafkaTests/Integration/MultiplePartitionTests.cs
@@ -0,0 +1,141 @@
+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";
+
+ private KafkaTestCluster testCluster;
+
+ [OneTimeSetUp]
+ public void BuildTestCluster()
+ {
+ testCluster = new KafkaTestCluster("server.home", 1);
+ }
+
+ [OneTimeTearDown]
+ public void DestroyTestCluster()
+ {
+ testCluster.Dispose();
+ testCluster = null;
+ }
+
+ [Test]
+ public async Task ProduceToTwoPartitions()
+ {
+ var keySerializer = new Int32Serializer();
+ var valueSerializer = new StringSerializer();
+ var messagePartitioner = new Int32Partitioner();
+
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic(partitions:2))
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ var topic = temporaryTopic.Name;
+ 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);
+
+ for (var i = 0; i < consumers.Length; i++)
+ {
+ var responses = await consumers[i].ReceiveAsync(CancellationToken.None);
+ 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 = testCluster.CreateTemporaryTopic(partitions: 2))
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ var topic = temporaryTopic.Name;
+ {
+ 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);
+ }
+
+ {
+ 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);
+ 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/Integration/SimpleTests.cs b/src/SimpleKafkaTests/Integration/SimpleTests.cs
new file mode 100644
index 00000000..c8b64f8f
--- /dev/null
+++ b/src/SimpleKafkaTests/Integration/SimpleTests.cs
@@ -0,0 +1,623 @@
+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("Integration")]
+ class SimpleTests
+ {
+ private KafkaTestCluster testCluster;
+
+ [OneTimeSetUp]
+ public void BuildTestCluster()
+ {
+ testCluster = new KafkaTestCluster("server.home", 1);
+ }
+
+ [OneTimeTearDown]
+ public void DestroyTestCluster()
+ {
+ testCluster.Dispose();
+ testCluster = null;
+ }
+
+ [Test]
+ public async Task TestProducingWorksOk()
+ {
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]))
+ {
+ var request = new ProduceRequest
+ {
+ Acks = 1,
+ TimeoutMS = 10000,
+ Payload = new List
+ {
+ new Payload
+ {
+ Topic = temporaryTopic.Name,
+ Partition = 0,
+ Codec = MessageCodec.CodecNone,
+ Messages = new List
+ {
+ new Message(Guid.NewGuid().ToString())
+ }
+ }
+ }
+ };
+
+ 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));
+ }
+ }
+
+ [Test]
+ public async Task TestFetchingWorksOk()
+ {
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]))
+ {
+ var request = new FetchRequest
+ {
+ MaxWaitTime = 0,
+ MinBytes = 1000,
+ Fetches = new List
+ {
+ new Fetch
+ {
+ Topic = temporaryTopic.Name,
+ PartitionId = 0,
+ MaxBytes = 1024,
+ Offset = 0
+ }
+ }
+ };
+
+ 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));
+ Assert.That(first.Messages, Has.Count.EqualTo(0));
+ }
+ }
+
+ [Test]
+ public async Task TestListingAllTopicsWorksOk()
+ {
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]))
+ {
+ var request = new MetadataRequest { };
+ 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));
+ 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));
+ }
+
+ }
+
+ [Test]
+ public async Task TestOffsetWorksOk()
+ {
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]))
+ {
+ 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);
+ 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]))
+ {
+ var topic = temporaryTopic.Name;
+ var request = new OffsetRequest
+ {
+ Offsets = new List
+ {
+ new Offset
+ {
+ 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);
+ 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]))
+ {
+ var request = new OffsetCommitRequest
+ {
+ OffsetCommits = new List
+ {
+ new OffsetCommit
+ {
+ Topic = temporaryTopic.Name,
+ Offset = 0
+ }
+ }
+ };
+
+ var response = await connection.SendRequestAsync(request, CancellationToken.None);
+ 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]))
+ {
+ 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);
+ 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]))
+ {
+ var request = new OffsetFetchRequest
+ {
+ Topics = new List
+ {
+ new OffsetFetch {
+ Topic = temporaryTopic.Name,
+ PartitionId = 0
+ }
+ }
+ };
+
+ var response = await connection.SendRequestAsync(request, CancellationToken.None);
+ 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]))
+ {
+ 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);
+ 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 = testCluster.CreateTemporaryTopic())
+ using (var connection = await KafkaConnectionFactory.CreateSimpleKafkaConnectionAsync(testCluster.CreateBrokerUris()[0]))
+ {
+ var topic = temporaryTopic.Name;
+ {
+ var request = new MetadataRequest
+ {
+ Topics = new List
+ {
+ topic
+ }
+ };
+ MetadataResponse response = null;
+ while (response == null)
+ {
+ response = await connection.SendRequestAsync(request, CancellationToken.None);
+ if (response.Topics[0].ErrorCode == ErrorResponseCode.LeaderNotAvailable)
+ {
+ response = null;
+ await Task.Delay(1000);
+ }
+
+ }
+ Assert.That(response, Is.Not.Null);
+ var first = response;
+ Assert.That(first.Topics, Has.Length.EqualTo(1));
+
+ var firstTopic = first.Topics.First();
+ Assert.That(firstTopic.ErrorCode, Is.EqualTo(ErrorResponseCode.NoError));
+ Assert.That(firstTopic.Name, Is.EqualTo(topic));
+ Assert.That(firstTopic.Partitions, Has.Length.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);
+ Assert.That(response, Is.Not.Null);
+
+ var first = response.First();
+ 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));
+ }
+
+ {
+ 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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ 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));
+ 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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ 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.Length.EqualTo(2));
+
+ Assert.That(first.Offsets[0], Is.EqualTo(4));
+ Assert.That(first.Offsets[1], Is.EqualTo(0));
+ }
+
+ {
+ var request = new ConsumerMetadataRequest
+ {
+ ConsumerGroup = topic
+ };
+ ConsumerMetadataResponse response = null;
+ while (response == null)
+ {
+ response = await connection.SendRequestAsync(request, CancellationToken.None);
+ if (response.Error == ErrorResponseCode.ConsumerCoordinatorNotAvailableCode)
+ {
+ 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
+ {
+ PartitionId = 0,
+ Topic = topic
+ }
+ }
+ };
+
+ 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));
+ 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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ Assert.That(first.Error, Is.EqualTo(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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ 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"));
+ 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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ 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));
+ 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);
+ Assert.That(response, Has.Count.EqualTo(1));
+ var first = response.First();
+
+ 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));
+ Assert.That(first.Messages, Has.Count.EqualTo(0));
+ }
+ }
+ Console.WriteLine("Test completed");
+ }
+
+ [Test]
+ public async Task TestSimpleKafkaBrokerWorksOk()
+ {
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ await brokers.RefreshAsync(CancellationToken.None);
+ }
+ }
+
+ [Test]
+ public async Task TestSimpleProducerWorksOk()
+ {
+ var valueSerializer = new StringSerializer();
+
+ using (var temporaryTopic = testCluster.CreateTemporaryTopic())
+ using (var brokers = new KafkaBrokers(testCluster.CreateBrokerUris()))
+ {
+ var producer = KafkaProducer.Create(brokers, valueSerializer);
+
+ await producer.SendAsync(KeyedMessage.Create(temporaryTopic.Name, "Message"), CancellationToken.None);
+
+
+ }
+ }
+
+ }
+}
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..d4df0474
--- /dev/null
+++ b/src/SimpleKafkaTests/SimpleKafkaTests.csproj
@@ -0,0 +1,91 @@
+
+
+
+
+ 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/KafkaDecoderTests.cs b/src/SimpleKafkaTests/Unit/KafkaDecoderTests.cs
new file mode 100644
index 00000000..61ce0421
--- /dev/null
+++ b/src/SimpleKafkaTests/Unit/KafkaDecoderTests.cs
@@ -0,0 +1,73 @@
+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 KafkaDecoderTests
+ {
+ [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 KafkaDecoder(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 KafkaDecoder(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 KafkaDecoder(givenBytes);
+ var actualValue = decoder.ReadInt16();
+ Assert.That(actualValue, Is.EqualTo(expectedValue));
+ }
+
+
+ [Theory]
+ [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);
+ var actualValue = decoder.ReadString();
+ Assert.That(decoder.Offset, Is.EqualTo(givenBytes.Length));
+ Assert.That(actualValue, Is.EqualTo(expectedValue));
+ }
+
+ }
+}
diff --git a/src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs b/src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs
new file mode 100644
index 00000000..a7dc0bd2
--- /dev/null
+++ b/src/SimpleKafkaTests/Unit/KafkaEncoderTests.cs
@@ -0,0 +1,87 @@
+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 KafkaEncoderTests
+ {
+ [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 KafkaEncoder(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.MaxValue, new Byte[] { 0xFF, 0xFF, 0xFF, 0xFF })]
+ public void UInt32Tests(UInt32 number, Byte[] expectedBytes)
+ {
+ var buffer = new byte[4];
+ var encoder = new KafkaEncoder(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 KafkaEncoder(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 KafkaEncoder(buffer);
+ encoder.Write(number);
+ Assert.That(buffer, Is.EqualTo(expectedBytes));
+ }
+
+
+ [Theory]
+ [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);
+ Assert.That(encoder.Offset, Is.EqualTo(expectedBytes.Length));
+ Assert.That(buffer, Is.EqualTo(expectedBytes));
+ }
+
+ }
+}
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/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs
new file mode 100644
index 00000000..54fd30f5
--- /dev/null
+++ b/src/SimpleKafkaTests/Unit/ProtocolBaseRequestTests.cs
@@ -0,0 +1,26 @@
+using System;
+using System.Collections.Generic;
+using System.Diagnostics;
+using System.Text;
+using NUnit.Framework;
+using SimpleKafka.Protocol;
+using SimpleKafka;
+
+namespace SimpleKafkaTests.Unit
+{
+ [TestFixture]
+ [Category("Unit")]
+ public class ProtocolBaseRequestTests
+ {
+ [Test]
+ public void EnsureHeaderShouldPackCorrectByteLengths()
+ {
+ var encoder = new KafkaEncoder(new byte[14]);
+ var request = new FetchRequest { ClientId = "test", CorrelationId = 123456789 };
+
+ 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
new file mode 100644
index 00000000..f42ac5ca
--- /dev/null
+++ b/src/SimpleKafkaTests/Unit/ProtocolMessageTests.cs
@@ -0,0 +1,130 @@
+using System;
+using System.IO;
+using System.Linq;
+using System.Text;
+using NUnit.Framework;
+using SimpleKafka.Protocol;
+using SimpleKafkaTests.Helpers;
+using SimpleKafka.Common;
+using SimpleKafka;
+
+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 buffer = new byte[1024];
+ var encoder = new KafkaEncoder(buffer);
+
+ Message.EncodeMessage(testMessage, encoder);
+ buffer[0] += 1;
+
+ var decoder = new KafkaDecoder(buffer, 0, encoder.Offset);
+ var result = Message.DecodeMessage(0, 0, decoder, encoder.Offset);
+ });
+ }
+
+ [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 buffer = new byte[1024];
+ var encoder = new KafkaEncoder(buffer);
+ Message.EncodeMessage(testMessage, encoder);
+
+ var decoder = new KafkaDecoder(buffer);
+ 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));
+ }
+
+ [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 buffer = new byte[expected.Length];
+ var encoder = new KafkaEncoder(buffer);
+ Message.EncodeMessageSet(encoder, messages);
+
+ Assert.That(buffer, Is.EqualTo(expected));
+ }
+
+ [Test]
+ 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, decoder, decoder.Length);
+
+ 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()
+ {
+ Assert.Throws(() =>
+ {
+ // arrange
+ var offset = (Int64)0;
+ var message = new Byte[] { };
+ var messageSize = 5;
+ var payloadBytes = new byte[16];
+ var encoder = new KafkaEncoder(payloadBytes);
+ encoder.Write(offset);
+ encoder.Write(messageSize);
+ encoder.Write(message);
+
+ var decoder = new KafkaDecoder(payloadBytes);
+
+ Message.DecodeMessageSet(0, decoder, payloadBytes.Length);
+ });
+ }
+
+ [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 decoder = new KafkaDecoder(payload, 0, payload.Length);
+ var messages = Message.DecodeMessageSet(0, decoder, payload.Length);
+ 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
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
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/Consumer.cs b/src/kafka-net/Consumer.cs
old mode 100644
new mode 100755
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
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 71%
rename from src/kafka-net.sln
rename to src/simple-kafka-net.sln
index a1c88085..ff8ed88e 100644
--- a/src/kafka-net.sln
+++ b/src/simple-kafka-net.sln
@@ -19,11 +19,16 @@ 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
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
@@ -33,15 +38,20 @@ 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
+ {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