public static string StopBrokerLeaderForPartition(Cluster cluster, string topic, int partition) { var brokerMeta = cluster.FindBrokerMetaForPartitionId(topic, partition); var brokerName = GetBrokerNameFromIp(brokerMeta.Host); StopBroker(brokerName); return brokerName; }
public Producer(Cluster cluster, ProducerConfiguration producerConfiguration) { Configuration = producerConfiguration; _cluster = cluster; _cluster.OnThreadHang += e => { if (OnPermError != null) OnPermError(e, _allPartitionQueues.SelectMany(p => p.Value.Queue.ToArray()).ToArray()); }; }
public Producer(Cluster cluster, ProducerConfiguration producerConfiguration) { Configuration = producerConfiguration; _cluster = cluster; _queueSizeEvents = new Subject<QueueResizeInfo>(); QueueSizeEvents = _queueSizeEvents.ObserveOn(Scheduler.Default); _cluster.OnThreadHang += e => { if (OnPermError != null) OnPermError(e, _allPartitionQueues.SelectMany(p => p.Value.Queue.ToArray()).ToArray()); }; }
public static void ReassignPartitions(Cluster cluster, string topic, int partition) { var brokerMeta = cluster.FindBrokerMetaForPartitionId(topic, partition); var brokerToMoveTo = brokerMeta.NodeId == 1 ? 2 : 1; var partitionsJson = string.Format("{{\"partitions\":[{{\"topic\":\"{0}\",\"partition\":{1},\"replicas\":[{2}]}}], \"version\":1}}", topic, partition, brokerToMoveTo); _log.Info(string.Format("Reassigning Partitions (topic {0}, partition {1}, from node {2} to node {3})", topic, partition, brokerMeta.NodeId, brokerToMoveTo)); var generateJson = "ssh -c \"printf '" + partitionsJson.Replace("\"", @"\\\""") + "' >partitions-to-move.json\" broker1"; Vagrant(generateJson); var reassignScript = "ssh -c '/opt/kafka_2.10-" + _kafkaVersion + "/bin/kafka-reassign-partitions.sh --zookeeper 192.168.56.2 --reassignment-json-file partitions-to-move.json --execute' broker1"; Vagrant(reassignScript); _log.Info("Reassigned Partitions"); }
public async void ExplicitOffset() { kafka4net.Tracing.EtwTrace.Marker("ExplicitOffset"); // create new topic with 3 partitions var topic = "part33." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic,3,3); // fill it out with 10K messages const int count = 10*1000; var producer = new Producer(_seed2Addresses, new ProducerConfiguration(topic)); await producer.ConnectAsync(); var sentMessagesObservable = Observable.FromEvent<Message[]>(evtHandler => producer.OnSuccess += evtHandler, evtHandler => { }) .SelectMany(msgs=>msgs) .Take(count) .TakeUntil(DateTime.Now.AddSeconds(10)) .ToList(); _log.Info("Sending data"); Enumerable.Range(1, count). Select(i => new Message { Value = BitConverter.GetBytes(i) }). ForEach(producer.Send); var sentMsgs = await sentMessagesObservable; _log.Info("Producer sent {0} messages.", sentMsgs.Count); _log.Debug("Closing producer"); await producer.CloseAsync(TimeSpan.FromSeconds(5)); var offsetFetchCluster = new Cluster(_seed2Addresses); await offsetFetchCluster.ConnectAsync(); // consume tail-300 for each partition await Task.Delay(TimeSpan.FromSeconds(1)); var offsets = new TopicPartitionOffsets( topic, (await offsetFetchCluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd)) .GetPartitionsOffset.Select(kv=>new KeyValuePair<int,long>(kv.Key,kv.Value-300))); _log.Info("Sum of offsets {0}. Raw: {1}",offsets.Partitions.Sum(p=>offsets.NextOffset(p)), offsets); var consumer = new Consumer(new ConsumerConfiguration(_seed2Addresses, topic, offsets)); var messages = consumer.OnMessageArrived. GroupBy(m => m.Partition).Replay(); messages.Connect(); await consumer.IsConnected; var consumerSubscription = messages.Subscribe(p => p.Take(10).Subscribe( m => _log.Debug("Got message {0}/{1}", m.Partition, BitConverter.ToInt32(m.Value, 0)), e => _log.Error("Error", e), () => _log.Debug("Complete part {0}", p.Key) )); // wait for 3 partitions to arrrive and every partition to read at least 100 messages await messages.Select(g => g.Take(100)).Take(3).ToTask(); consumerSubscription.Dispose(); consumer.Dispose(); kafka4net.Tracing.EtwTrace.Marker("/ExplicitOffset"); }
public async void ConsumerFollowsRebalancingPartitions() { kafka4net.Tracing.EtwTrace.Marker("ConsumerFollowsRebalancingPartitions"); // create a topic var topic = "topic33." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic,11,3); // Stop two brokers to let leadership shift to broker1. VagrantBrokerUtil.StopBroker("broker2"); VagrantBrokerUtil.StopBroker("broker3"); await Task.Delay(TimeSpan.FromSeconds(5)); // now start back up VagrantBrokerUtil.StartBroker("broker2"); VagrantBrokerUtil.StartBroker("broker3"); // wait a little for everything to start await Task.Delay(TimeSpan.FromSeconds(5)); // we should have all of them with leader 1 var cluster = new Cluster(_seed2Addresses); await cluster.ConnectAsync(); var partitionMeta = await cluster.GetOrFetchMetaForTopicAsync(topic); // make sure they're all on a single leader Assert.AreEqual(1, partitionMeta.GroupBy(p=>p.Leader).Count()); // now publish messages const int count = 25000; var producer = new Producer(cluster, new ProducerConfiguration(topic)); _log.Debug("Connecting"); await producer.ConnectAsync(); _log.Debug("Filling out {0} with {1} messages", topic, count); var sentList = await Enumerable.Range(0, count) .Select(i => new Message { Value = BitConverter.GetBytes(i) }) .ToObservable() .Do(producer.Send) .Select(msg => BitConverter.ToInt32(msg.Value, 0)) .ToList(); await Task.Delay(TimeSpan.FromSeconds(1)); _log.Info("Done sending messages. Closing producer."); await producer.CloseAsync(TimeSpan.FromSeconds(5)); _log.Info("Producer closed, starting consumer subscription."); await Task.Delay(TimeSpan.FromSeconds(1)); var heads = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicStart); var tails = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); var messagesInTopic = (int)tails.MessagesSince(heads); _log.Info("Topic offsets indicate producer sent {0} messages.", messagesInTopic); var consumer = new Consumer(new ConsumerConfiguration(_seed2Addresses, topic, new StartPositionTopicStart(), maxBytesPerFetch: 4 * 8)); var current = 0; var received = new ReplaySubject<ReceivedMessage>(); Task rebalanceTask = null; var consumerSubscription = consumer.OnMessageArrived. Subscribe(async msg => { current++; if (current == 18) { rebalanceTask = Task.Factory.StartNew(VagrantBrokerUtil.RebalanceLeadership, CancellationToken.None, TaskCreationOptions.None, TaskScheduler.Default); } received.OnNext(msg); //_log.Info("Got: {0}", BitConverter.ToInt32(msg.Value, 0)); }); await consumer.IsConnected; _log.Info("Waiting for receiver complete"); var receivedList = await received.Select(msg => BitConverter.ToInt32(msg.Value, 0)). Take(messagesInTopic). TakeUntil(DateTime.Now.AddMinutes(3)). ToList(). ToTask(); if (rebalanceTask != null) { _log.Info("Waiting for rebalance complete"); await rebalanceTask;//.TimeoutAfter(TimeSpan.FromSeconds(10)); _log.Info("Rebalance complete"); } _log.Info("Receiver complete. Disposing Subscription"); consumerSubscription.Dispose(); _log.Info("Consumer subscription disposed. Closing consumer."); consumer.Dispose(); _log.Info("Consumer closed."); tails = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); await cluster.CloseAsync(TimeSpan.FromSeconds(5)); _log.Info("Sum of offsets: {0}", tails.MessagesSince(heads)); _log.Info("Offsets: [{0}]", string.Join(",", tails.Partitions.Select(p => string.Format("{0}:{1}", p, tails.NextOffset(p))))); if (messagesInTopic != receivedList.Count) { // log some debug info. _log.Error("Did not receive all messages. Messages sent but NOT received: {0}", string.Join(",", sentList.Except(receivedList).OrderBy(i => i))); } Assert.AreEqual(messagesInTopic, receivedList.Count); kafka4net.Tracing.EtwTrace.Marker("/ConsumerFollowsRebalancingPartitions"); }
public async void ProducerRecoveryTest() { kafka4net.Tracing.EtwTrace.Marker("ProducerRecoveryTest"); const int count = 200; var topic = "part62." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic, 6, 2); var producer = new Producer(_seed2Addresses, new ProducerConfiguration(topic)); _log.Debug("Connecting"); await producer.ConnectAsync(); _log.Debug("Filling out {0}", topic); // when we get a confirm back, add to list actually sent. var actuallySentList = new List<int>(count); producer.OnSuccess += msgs => actuallySentList.AddRange(msgs.Select(msg => BitConverter.ToInt32(msg.Value, 0))); Task stopBrokerTask = null; var sentList = await Observable.Interval(TimeSpan.FromMilliseconds(100)) .Select(l => (int)l) .Do(l => { if (l == 20) stopBrokerTask = Task.Factory.StartNew(() => VagrantBrokerUtil.StopBroker("broker2"), CancellationToken.None, TaskCreationOptions.None, TaskScheduler.Default); }) .Select(i => new Message { Value = BitConverter.GetBytes(i) }) .Take(count) .Do(producer.Send) .Select(msg => BitConverter.ToInt32(msg.Value, 0)) .ToList(); _log.Info("Done waiting for sending. Closing producer."); await producer.CloseAsync(TimeSpan.FromSeconds(30)); _log.Info("Producer closed."); if (stopBrokerTask != null) await stopBrokerTask.TimeoutAfter(TimeSpan.FromSeconds(10)); // // Check length of result topic // var c2 = new Cluster(_seed2Addresses); await c2.ConnectAsync(); var heads = await c2.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicStart); var tails = await c2.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); _log.Info("Sum of offsets: {0}", tails.MessagesSince(heads)); _log.Info("Offsets: [{0}]", string.Join(",", tails.Partitions.Select(p => string.Format("{0}:{1}", p, tails.NextOffset(p))))); // if (sentList.Count != actuallySentList.Count) { // log some debug info. _log.Error("Did not send all messages. Messages sent but NOT acknowledged: {0}", string.Join(",", sentList.Except(actuallySentList).OrderBy(i => i))); } Assert.AreEqual(sentList.Count, actuallySentList.Count, "Actually sent"); Assert.AreEqual(sentList.Count, tails.MessagesSince(heads), "Offsets"); kafka4net.Tracing.EtwTrace.Marker("/ProducerRecoveryTest"); }
public async void IfFirstBrokerIsDownThenNextOneWillConnect() { var badSeed = "192.168.56.111," + _seed2Addresses; var cluster = new Cluster(badSeed); await cluster.ConnectAsync(); await cluster.GetAllTopicsAsync(); //var producer = new Producer(cluster, new ProducerConfiguration("notopic")); //await producer.ConnectAsync(); await cluster.CloseAsync(TimeSpan.FromSeconds(1)); }
public async void ProducerTestWhenPartitionReassignmentOccurs() { // Scenario: Broker gives away it's topic to another broker // See https://github.com/ntent-ad/kafka4net/issues/27 var topic = "topic11." + _rnd.Next(); // Create topic VagrantBrokerUtil.CreateTopic(topic, 1, 1); // Create cluster and producer var cluster = new Cluster(_seed2Addresses); //await cluster.ConnectAsync(); //await cluster.GetOrFetchMetaForTopicAsync(topic); VagrantBrokerUtil.DescribeTopic(topic); var producer = new Producer(cluster, new ProducerConfiguration(topic, batchFlushSize: 1)); var ctx = SynchronizationContext.Current; producer.OnPermError += (exception, messages) => ctx.Post(d => { throw exception; }, null); int successfullySent = 0; producer.OnSuccess += messages => successfullySent++; _log.Info("Connecting producer"); await producer.ConnectAsync(); _log.Info("Producer Send data before reassignment"); producer.Send(new Message { Value = new byte[] { 0, 0, 0, 0 } }); // Run the reassignment VagrantBrokerUtil.ReassignPartitions(cluster, topic, 0); _log.Info("Waiting for reassignment completion"); await Task.Delay(5 * 1000); VagrantBrokerUtil.DescribeTopic(topic); _log.Info("Producer Send data after reassignment"); producer.Send(new Message { Value = new byte[] { 1, 1, 1, 1 } }); _log.Info("Waiting for producer to complete"); await producer.CloseAsync(TimeSpan.FromSeconds(60)); Assert.That(successfullySent, Is.EqualTo(2)); _log.Info("Done"); }
public async void ProducerConnectWhenOneBrokerIsDownAndThanUp() { // Scenario: 1 broker is down, Producer connects. Broker is brought up and forced to become master. // See https://github.com/ntent-ad/kafka4net/issues/14 var topic = "topic11." + _rnd.Next(); // Create topic VagrantBrokerUtil.CreateTopic(topic, 1, 2); var cluster = new Cluster(_seed3Addresses); await cluster.ConnectAsync(); await cluster.GetOrFetchMetaForTopicAsync(topic); VagrantBrokerUtil.DescribeTopic(topic); // Stop the leader //var partitionDown = cluster.PartitionStateChanges.FirstAsync(_ => _.ErrorCode.IsFailure()); var preferredBroker = VagrantBrokerUtil.StopBrokerLeaderForPartition(cluster, topic, 0); //_log.Info("Waiting for partition to be down"); //await partitionDown; await cluster.CloseAsync(TimeSpan.FromSeconds(3)); await Task.Delay(30 * 1000); VagrantBrokerUtil.DescribeTopic(topic); // Create new cluster and publisher, while preferred leader is down cluster = new Cluster(_seed3Addresses); cluster.NewBrokers.Subscribe(b => _log.Info("Discovered new broker: {0}", b)); _log.Info("Connecting cluster"); await cluster.ConnectAsync(); var producer = new Producer(cluster, new ProducerConfiguration(topic)); _log.Info("Connecting producer"); await producer.ConnectAsync(); // Start preferred leader up _log.Info("Starting preferred broker"); VagrantBrokerUtil.StartBroker(preferredBroker); _log.Info("Waiting for preferred broker ({0}) to start up", preferredBroker); await Task.Delay(30 * 1000); //VagrantBrokerUtil.RebalanceLeadership(); _log.Info("Stopping 2nd leader broker"); VagrantBrokerUtil.StopBrokerLeaderForPartition(cluster, topic, 0); _log.Info("Producer Send data"); producer.Send(new Message() { Value = new byte[]{0,0,0,0}}); _log.Info("Waiting for producer to complete"); await producer.CloseAsync(TimeSpan.FromSeconds(60)); _log.Info("Done"); }
public async void SchedulerThreadIsIsolatedFromUserCode() { kafka4net.Tracing.EtwTrace.Marker("SchedulerThreadIsIsolatedFromUserCode"); const string threadName = "kafka-scheduler"; _log.Info("Test Runner is using thread {0}", Thread.CurrentThread.Name); var topic = "topic." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic,6,3); var cluster = new Cluster(_seed2Addresses); await cluster.ConnectAsync(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicStart); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); var topics = await cluster.GetAllTopicsAsync(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); // now create a producer var producer = new Producer(cluster, new ProducerConfiguration(topic)); await producer.ConnectAsync(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); // create a producer that also creates a cluster var producerWithCluster = new Producer(_seed2Addresses, new ProducerConfiguration(topic)); await producerWithCluster.ConnectAsync(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); // TODO: Subscribe and check thread on notification observables! // run them both for a little while (~5 seconds) var msgs = await Observable.Interval(TimeSpan.FromMilliseconds(100)) .Do(l => { producer.Send(new Message { Value = BitConverter.GetBytes(l) }); producerWithCluster.Send(new Message { Value = BitConverter.GetBytes(l) }); _log.Debug("After Producer Send using thread {0}", Thread.CurrentThread.Name); }).Take(50).ToArray(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); // now consumer(s) var consumer = new Consumer(new ConsumerConfiguration(_seed2Addresses, topic, new StartPositionTopicStart())); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); var msgsRcv = new List<long>(); var messageSubscription = consumer.OnMessageArrived .Do(msg => Assert.AreEqual(threadName, Thread.CurrentThread.Name), exception => Assert.AreEqual(threadName, Thread.CurrentThread.Name), () => Assert.AreEqual(threadName, Thread.CurrentThread.Name)) .Take(50) .TakeUntil(DateTime.Now.AddSeconds(500)) .ObserveOn(System.Reactive.Concurrency.DefaultScheduler.Instance) .Do(msg => Assert.AreNotEqual(threadName, Thread.CurrentThread.Name), exception => Assert.AreNotEqual(threadName, Thread.CurrentThread.Name), () => Assert.AreNotEqual(threadName, Thread.CurrentThread.Name)) .Subscribe( msg=> { msgsRcv.Add(BitConverter.ToInt64(msg.Value,0)); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); _log.Debug("In Consumer Subscribe OnNext using thread {0}", Thread.CurrentThread.Name); }, exception => { _log.Debug("In Consumer Subscribe OnError using thread {0} Error: {1}", Thread.CurrentThread.Name, exception.Message); throw exception; }, () => { Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); _log.Debug("In Consumer Subscribe OnComplete using thread {0}", Thread.CurrentThread.Name); }); await consumer.IsConnected; _log.Info("Waitng for consumer to read"); await Task.Delay(TimeSpan.FromSeconds(6)); _log.Debug("After Consumer Subscribe using thread {0}", Thread.CurrentThread.Name); consumer.Dispose(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); Assert.AreEqual(msgs.Length, msgsRcv.Count); messageSubscription.Dispose(); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); // now close down await producer.CloseAsync(TimeSpan.FromSeconds(5)); _log.Debug("After Consumer Close using thread {0}", Thread.CurrentThread.Name); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); await producerWithCluster.CloseAsync(TimeSpan.FromSeconds(5)); _log.Debug("After Producer Subscribe using thread {0}", Thread.CurrentThread.Name); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); await cluster.CloseAsync(TimeSpan.FromSeconds(5)); _log.Debug("After Cluster Close using thread {0}", Thread.CurrentThread.Name); Assert.AreNotEqual(threadName, Thread.CurrentThread.Name); kafka4net.Tracing.EtwTrace.Marker("/SchedulerThreadIsIsolatedFromUserCode"); }
public async void MultipleProducersOneCluster() { kafka4net.Tracing.EtwTrace.Marker("MultipleProducersOneCluster"); var cluster = new Cluster(_seed2Addresses); var topic1 = "topic." + _rnd.Next(); var topic2 = "topic." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic1, 6, 3); VagrantBrokerUtil.CreateTopic(topic2, 6, 3); // declare two producers var producer1 = new Producer(cluster, new ProducerConfiguration(topic1)); await producer1.ConnectAsync(); var producer2 = new Producer(cluster, new ProducerConfiguration(topic2)); await producer2.ConnectAsync(); // run them both for a little while (~10 seconds) var msgs = await Observable.Interval(TimeSpan.FromMilliseconds(100)) .Do(l => { producer1.Send(new Message {Value = BitConverter.GetBytes(l)}); producer2.Send(new Message {Value = BitConverter.GetBytes(l)}); }).Take(100); _log.Info("Done Sending, await on producer close."); // now stop them. await Task.WhenAll(new [] { producer1.CloseAsync(TimeSpan.FromSeconds(5)), producer2.CloseAsync(TimeSpan.FromSeconds(5)) }); await Task.Delay(TimeSpan.FromSeconds(2)); // check we got all 100 on each topic. _log.Info("Closed Producers. Checking Offsets"); var topic1Heads = await cluster.FetchPartitionOffsetsAsync(topic1, ConsumerLocation.TopicStart); var topic2Heads = await cluster.FetchPartitionOffsetsAsync(topic2, ConsumerLocation.TopicStart); var topic1Tails = await cluster.FetchPartitionOffsetsAsync(topic1, ConsumerLocation.TopicEnd); var topic2Tails = await cluster.FetchPartitionOffsetsAsync(topic2, ConsumerLocation.TopicEnd); Assert.AreEqual(100, topic1Tails.MessagesSince(topic1Heads)); Assert.AreEqual(100, topic2Tails.MessagesSince(topic2Heads)); kafka4net.Tracing.EtwTrace.Marker("/MultipleProducersOneCluster"); }
public async void ReadOffsets() { kafka4net.Tracing.EtwTrace.Marker("ReadOffsets"); var sentEvents = new Subject<Message>(); var topic = "part12." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic,1,1); var cluster = new Cluster(_seed2Addresses); await cluster.ConnectAsync(); var producer = new Producer(cluster, new ProducerConfiguration(topic, maxMessageSetSizeInBytes: 1024*1024)); producer.OnSuccess += e => e.ForEach(sentEvents.OnNext); await producer.ConnectAsync(); // read offsets of empty queue var heads = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicStart); var tails = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); Assert.AreEqual(1, heads.Partitions.Count(), "Expected just one head partition"); Assert.AreEqual(1, tails.Partitions.Count(), "Expected just one tail partition"); Assert.AreEqual(0L, heads.NextOffset(heads.Partitions.First()), "Expected start at 0"); Assert.AreEqual(0L, tails.NextOffset(tails.Partitions.First()), "Expected end at 0"); // log the broker selected as master var brokerMeta = cluster.FindBrokerMetaForPartitionId(topic, heads.Partitions.First()); _log.Info("Partition Leader is {0}", brokerMeta); // saw some inconsistency, so run this a few times. const int count = 1100; const int loops = 10; for (int i = 0; i < loops; i++) { // NOTE that the configuration for the test machines through vagrant are set to 1MB rolling file segments // so we need to generate large messages to force multiple segments to be created. // send count messages var t = sentEvents.Take(count).ToTask(); Enumerable.Range(1, count). Select(_ => new Message { Value = new byte[1024] }). ForEach(producer.Send); _log.Info("Waiting for {0} sent messages", count); await t; // re-read offsets after messages published await Task.Delay(TimeSpan.FromSeconds(2)); // NOTE: There seems to be a race condition on the Kafka broker that the offsets are not immediately available after getting a successful produce response tails = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); _log.Info("2:After loop {0} of {1} messages, Next Offset is {2}", i + 1, count, tails.NextOffset(tails.Partitions.First())); Assert.AreEqual(count * (i + 1), tails.NextOffset(tails.Partitions.First()), "Expected end at " + count * (i + 1)); } _log.Info("Closing producer"); await producer.CloseAsync(TimeSpan.FromSeconds(5)); await Task.Delay(TimeSpan.FromSeconds(1)); // re-read offsets after messages published heads = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicStart); tails = await cluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd); Assert.AreEqual(1, heads.Partitions.Count(), "Expected just one head partition"); Assert.AreEqual(1, tails.Partitions.Count(), "Expected just one tail partition"); Assert.AreEqual(0L, heads.NextOffset(heads.Partitions.First()), "Expected start at 0"); Assert.AreEqual(count*loops, tails.NextOffset(tails.Partitions.First()), "Expected end at " + count); kafka4net.Tracing.EtwTrace.Marker("/ReadOffsets"); }
public async void StopAtExplicitOffsetOnEmptyTopic() { kafka4net.Tracing.EtwTrace.Marker("StopAtExplicitOffsetOnEmptyTopic"); // create new topic with 3 partitions var topic = "part33." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic, 3, 3); var offsetFetchCluster = new Cluster(_seed2Addresses); await offsetFetchCluster.ConnectAsync(); await Task.Delay(TimeSpan.FromSeconds(1)); var offsets = (await offsetFetchCluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd)); _log.Info("Sum of offsets {0}. Raw: {1}", offsets.Partitions.Sum(p => offsets.NextOffset(p)), offsets); var startStopProvider = new StartAndStopAtExplicitOffsets(offsets, offsets); _log.Info("Attempting to consume {0} messages and stop at {1}", 0, offsets); var consumer = new Consumer(new ConsumerConfiguration(_seed2Addresses, topic, startStopProvider, stopPosition: startStopProvider)); var startTime = DateTime.Now; var timeout = startTime.AddSeconds(30); var messages = await consumer.OnMessageArrived.TakeUntil(timeout).ToList(); _log.Info("Finished"); Assert.IsTrue(DateTime.Now < timeout); Assert.AreEqual(0, messages.Count); consumer.Dispose(); kafka4net.Tracing.EtwTrace.Marker("/StopAtExplicitOffsetOnEmptyTopic"); }
public async void StartAndStopAtExplicitOffset() { kafka4net.Tracing.EtwTrace.Marker("StartAndStopAtExplicitOffset"); // create new topic with 3 partitions var topic = "part33." + _rnd.Next(); VagrantBrokerUtil.CreateTopic(topic, 3, 3); // fill it out with 10K messages const int count = 10 * 1000; var producer = new Producer(_seed2Addresses, new ProducerConfiguration(topic)); await producer.ConnectAsync(); var sentMessagesObservable = Observable.FromEvent<Message[]>(evtHandler => producer.OnSuccess += evtHandler, evtHandler => { }) .SelectMany(msgs => msgs) .Take(count) .TakeUntil(DateTime.Now.AddSeconds(10)) .ToList(); _log.Info("Sending data"); Enumerable.Range(1, count). Select(i => new Message { Value = BitConverter.GetBytes(i) }). ForEach(producer.Send); var sentMsgs = await sentMessagesObservable; _log.Info("Producer sent {0} messages.", sentMsgs.Count); _log.Debug("Closing producer"); await producer.CloseAsync(TimeSpan.FromSeconds(5)); var offsetFetchCluster = new Cluster(_seed2Addresses); await offsetFetchCluster.ConnectAsync(); await Task.Delay(TimeSpan.FromSeconds(1)); var offsets = (await offsetFetchCluster.FetchPartitionOffsetsAsync(topic, ConsumerLocation.TopicEnd)); _log.Info("Sum of offsets {0}. Raw: {1}", offsets.Partitions.Sum(p => offsets.NextOffset(p)), offsets); // consume first 300 for each partition var offsetStarts = new TopicPartitionOffsets(topic, offsets.GetPartitionsOffset.ToDictionary(pair => pair.Key, pair => pair.Value > 300 ? 300 : pair.Value)); var offsetStops = new TopicPartitionOffsets(topic, offsets.GetPartitionsOffset.ToDictionary(pair => pair.Key, pair => pair.Value > 600 ? 600 : pair.Value)); var numMessages = offsetStops.MessagesSince(offsetStarts); var startStopProvider = new StartAndStopAtExplicitOffsets(offsetStarts, offsetStops); _log.Info("Attempting to consume {0} messages and stop at {1}", numMessages, offsetStops); var consumer = new Consumer(new ConsumerConfiguration(_seed2Addresses, topic, startStopProvider, stopPosition: startStopProvider)); var messages = await consumer.OnMessageArrived.ToList(); consumer.Dispose(); Assert.AreEqual(numMessages, messages.Count); kafka4net.Tracing.EtwTrace.Marker("/StartAndStopAtExplicitOffset"); }
/// <summary> /// Create a new consumer using the specified configuration. See @ConsumerConfiguration /// </summary> /// <param name="consumerConfig"></param> public Consumer(ConsumerConfiguration consumerConfig) { Configuration = consumerConfig; _cluster = new Cluster(consumerConfig.SeedBrokers); _cluster.OnThreadHang += e => OnMessageArrivedInput.OnError(e); // Low/high watermark implementation FlowControl = _flowControlInput. Scan(1, (last, current) => { if (current < Configuration.LowWatermark) return 1; if (current > Configuration.HighWatermark) return -1; return last; // While in between watermarks, carry over previous on/off state }). DistinctUntilChanged(). Select(i => i > 0). Do(f =>FlowControlEnabled = f). Do(f => EtwTrace.Log.ConsumerFlowControl(f ? 1 : 0)); var onMessage = Observable.Create<ReceivedMessage>(observer => { if (Interlocked.CompareExchange(ref _haveSubscriber, 1, 0) == 1) throw new InvalidOperationException("Only one subscriber is allowed. Use OnMessageArrived.Publish().RefCount()"); // Relay messages from partition to consumer's output // Ensure that only single subscriber is allowed because it is important to count // outstanding messaged consumed by user OnMessageArrivedInput.Subscribe(observer); // // It is not possible to wait for completion of partition resolution process, so start it asynchronously. // This means that OnMessageArrived.Subscribe() will complete when consumer is not actually connected yet. // Task.Run(async () => { try { await _cluster.ConnectAsync(); await SubscribeClient(); EtwTrace.Log.ConsumerStarted(GetHashCode(), Topic); _connectionComplete.TrySetResult(true); // check that we actually got any partitions subscribed if (_topicPartitions.Count == 0) OnMessageArrivedInput.OnCompleted(); } catch (Exception e) { _connectionComplete.TrySetException(e); OnMessageArrivedInput.OnError(e); } }); // upon unsubscribe return Disposable.Create(() => _partitionsSubscription.Values.ForEach(s=>s.Dispose())); }); if (Configuration.UseFlowControl) { // Increment counter of messages sent for processing onMessage = onMessage.Do(msg => { var count = Interlocked.Increment(ref _outstandingMessageProcessingCount); _flowControlInput.OnNext(count); }); } // handle stop condition onMessage = onMessage.Do(message => { // check if this partition is done per the condition passed in configuration. If so, unsubscribe it. bool partitionDone = (Configuration.StopPosition.IsPartitionConsumingComplete(message)); IDisposable partitionSubscription; if (partitionDone && _partitionsSubscription.TryGetValue(message.Partition, out partitionSubscription)) { _partitionsSubscription.Remove(message.Partition); // calling Dispose here will cause the OnTopicPartitionComplete method to be called when it is completed. partitionSubscription.Dispose(); } }); OnMessageArrived = onMessage. // isolate driver from user code misbehave ObserveOn(Configuration.OutgoingScheduler); // If permanent error within any single partition, fail the whole consumer (intentionally). // Do not try to keep going (failfast principle). _cluster.PartitionStateChanges. Where(s => s.ErrorCode.IsPermanentFailure()). Subscribe(state => OnMessageArrivedInput.OnError(new PartitionFailedException(state.Topic, state.PartitionId, state.ErrorCode))); }