public void TestBasic()
        {
            // test consumer timeout logic
            var consumerConfig0 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer0, 200);
            var zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0);
            var topicMessageSterams0 =
                zkConsumerConnector0.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            // no messages to consume, we should hit timeout;
            // also the iterator should support re-entrant, so loop it twice
            for (var i = 0; i < 2; i++)
            {
                Assert.Throws<ConsumerTimeoutException>(
                    () => this.GetMessages(nMessages * 2, topicMessageSterams0));
            }

            zkConsumerConnector0.Shutdown();

            // send some messages to each broker
            var sentMessages1 =
                this.SendMessagesToBrokerPartition(Configs.First(), Topic, 0, nMessages)
                .Union(this.SendMessagesToBrokerPartition(Configs.Last(), Topic, 0, nMessages)).ToList();

            // wait to make sure the topic and partition have a leader for the successful case
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 0, 500); 
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 1, 500);

            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 0, 1000);
            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 1, 1000);

            // create a consuemr
            var consumerConfig1 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer1);
            var zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1);
            var topicMessageStreams1 =
                zkConsumerConnector1.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            var receivedMessages1 = this.GetMessages(nMessages * 2, topicMessageStreams1);
            Assert.Equal(sentMessages1.OrderBy(x => x).ToArray(), receivedMessages1.OrderBy(x => x).ToArray());

            // also check partition ownership
            var actual_1 = this.GetZKChildrenValues(this.dirs.ConsumerOwnerDir);
            var expected_1 = new List<Tuple<string, string>>
                                 {
                                     Tuple.Create("0", "group1_consumer1-0"),
                                     Tuple.Create("1", "group1_consumer1-0")
                                 };
            Assert.Equal(expected_1, actual_1);

            // commit consumer offsets
            zkConsumerConnector1.CommitOffsets();

            // create a consumer
            var consumerConfig2 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer2);
            consumerConfig2.RebalanceBackoffMs = RebalanceBackoutMs;

            var zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2);
            var topicMessageStreams2 =
                zkConsumerConnector2.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            // send some messages to each broker
            var sentMessages2 =
                this.SendMessagesToBrokerPartition(Configs.First(), Topic, 0, nMessages)
                .Union(this.SendMessagesToBrokerPartition(Configs.Last(), Topic, 1, nMessages)).ToList();

            // wait to make sure the topic and partition have a leader for the successful case
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 0, 500);
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 1, 500);

            var receivedMessages2 =
                this.GetMessages(nMessages, topicMessageStreams1)
                    .Union(this.GetMessages(nMessages, topicMessageStreams2))
                    .ToList();
            Assert.Equal(sentMessages2.OrderBy(x => x).ToList(), receivedMessages2.OrderBy(x => x).ToList());

            // also check partition ownership
            var actual_2 = this.GetZKChildrenValues(this.dirs.ConsumerOwnerDir);
            var expected_2 = new List<Tuple<string, string>>
                                 {
                                     Tuple.Create("0", "group1_consumer1-0"),
                                     Tuple.Create("1", "group1_consumer2-0")
                                 };
            Assert.Equal(expected_2, actual_2);

            // create a consumer with empty map
            var consumerConfig3 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer3);
            var zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3);
            zkConsumerConnector3.CreateMessageStreams(new Dictionary<string, int>());

            // send some messages to each broker
            var sentMessages3 =
                this.SendMessagesToBrokerPartition(Configs.First(), Topic, 0, nMessages)
                    .Union(this.SendMessagesToBrokerPartition(Configs.Last(), Topic, 1, nMessages))
                    .ToList();

            // wait to make sure the topic and partition have a leader for the successful case
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 0, 500);
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 1, 500);

            var receivedMessages3 =
                this.GetMessages(nMessages, topicMessageStreams1)
                    .Union(this.GetMessages(nMessages, topicMessageStreams2))
                    .ToList();
            Assert.Equal(sentMessages3.OrderBy(x => x).ToList(), receivedMessages3.OrderBy(x => x).ToList());

            // also check partition ownership
            var actual_3 = this.GetZKChildrenValues(this.dirs.ConsumerOwnerDir);
            Assert.Equal(expected_2, actual_3);

            zkConsumerConnector1.Shutdown();
            zkConsumerConnector2.Shutdown();
            zkConsumerConnector3.Shutdown();

            Logger.Info("all consumer connectors stopped");
        }
        public void TestConsumerDecoder()
        {
            // send some messages to each broker
            var sentMessages = this.SendMessagesToBrokerPartition(
                Configs.First(), Topic, 0, nMessages)
                .Union(
                    this.SendMessagesToBrokerPartition(
                        Configs.First(), Topic, 1, nMessages))
                .ToList();

            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 0, 1000);
            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 1, 1000);

            // create a consuemr
            var consumerConfig = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer1);

            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 0, 500);
            TestUtils.WaitUntilLeaderIsElectedOrChanged(this.ZkClient, Topic, 1, 500);

            var zkConsumerConnector = new ZookeeperConsumerConnector(consumerConfig);
            var topicMessageStreams =
                zkConsumerConnector.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            var receivedMessages = this.GetMessages(nMessages * 2, topicMessageStreams);
            Assert.Equal(sentMessages.OrderBy(x => x).ToArray(), receivedMessages.OrderBy(x => x).ToArray());

            zkConsumerConnector.Shutdown();
        }
        public void TestLeaderSelectionForPartition()
        {
            var zkClient = new ZkClient(this.zookeeperConnect, 6000, 30000, new ZkStringSerializer());

            // create topic topic1 with 1 partition on broker 0
            AdminUtils.CreateTopic(zkClient, Topic, 1, 1, new Dictionary<string, string>());
            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 0, 3000);

            var sentMessages1 = this.SendMessages(
                Configs.First(), nMessages, "batch1", CompressionCodecs.NoCompressionCodec, 1);

            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 0, 1000);

            // create a consuemr
            var consumerConfig1 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer1);
            var zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1);
            var topicMessageStreams1 =
                zkConsumerConnector1.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            var topicRegistry = zkConsumerConnector1.TopicRegistry;
            Assert.Equal(1, topicRegistry.Select(x => x.Key).Count());
            Assert.Equal(Topic, topicRegistry.Select(x => x.Key).First());

            var topicsAndPartitionsInRegistry =
                topicRegistry.Select(x => Tuple.Create(x.Key, x.Value.Select(p => p.Value))).ToList();

            var brokerPartition = topicsAndPartitionsInRegistry.First().Item2.First();
            Assert.Equal(0, brokerPartition.PartitionId);

            // also check partition ownership
            var actual_1 = this.GetZKChildrenValues(this.dirs.ConsumerOwnerDir);
            var expected_1 = new List<Tuple<string, string>>
                                 {
                                     Tuple.Create("0", "group1_consumer1-0"),
                                 };
            Assert.Equal(expected_1, actual_1);

            var receivedMessages1 = this.GetMessages(nMessages, topicMessageStreams1);
            Assert.Equal(sentMessages1, receivedMessages1);
            zkConsumerConnector1.Shutdown();
            zkClient.Dispose();
        }
        public void TestCompressionSetConsumption()
        {
            // send some messages to each broker
            var sentMessages = this.SendMessagesToBrokerPartition(
                Configs.First(), Topic, 0, 200, CompressionCodecs.DefaultCompressionCodec)
                .Union(
                    this.SendMessagesToBrokerPartition(
                        Configs.First(), Topic, 1, 200, CompressionCodecs.DefaultCompressionCodec))
                .ToList();

            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 0, 1000);
            TestUtils.WaitUntilMetadataIsPropagated(this.Servers, Topic, 1, 1000);

            // create a consuemr
            var consumerConfig1 = TestUtils.CreateConsumerProperties(ZkConnect, Group, Consumer0);
            var zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1);
            var topicMessageStreams1 =
                zkConsumerConnector1.CreateMessageStreams(
                    new Dictionary<string, int> { { Topic, 1 } }, new StringDecoder(), new StringDecoder());

            var receivedMessages1 = this.GetMessages(400, topicMessageStreams1);
            Assert.Equal(sentMessages.OrderBy(x => x).ToArray(), receivedMessages1.OrderBy(x => x).ToArray());

            // also check partition ownership
            var actual_2 = this.GetZKChildrenValues(this.dirs.ConsumerOwnerDir);
            var expected_2 = new List<Tuple<string, string>>
                                 {
                                     Tuple.Create("0", "group1_consumer0-0"),
                                     Tuple.Create("1", "group1_consumer0-0")
                                 };
            Assert.Equal(expected_2, actual_2);
            zkConsumerConnector1.Shutdown();
        }