|
|
|
@ -1375,6 +1375,54 @@ class PlaintextConsumerTest extends BaseConsumerTest {
@@ -1375,6 +1375,54 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
|
|
|
|
assertEquals(500, consumer0.committed(tp2).offset) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLeadMetricsCleanUpWithSubscribe() { |
|
|
|
|
val numMessages = 1000 |
|
|
|
|
val topic2 = "topic2" |
|
|
|
|
createTopic(topic2, 2, serverCount) |
|
|
|
|
// send some messages. |
|
|
|
|
sendRecords(numMessages, tp) |
|
|
|
|
// Test subscribe |
|
|
|
|
// Create a consumer and consumer some messages. |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLeadMetricsCleanUpWithSubscribe") |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLeadMetricsCleanUpWithSubscribe") |
|
|
|
|
val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) |
|
|
|
|
try { |
|
|
|
|
val listener0 = new TestConsumerReassignmentListener |
|
|
|
|
consumer.subscribe(List(topic, topic2).asJava, listener0) |
|
|
|
|
var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() |
|
|
|
|
TestUtils.waitUntilTrue(() => { |
|
|
|
|
records = consumer.poll(100) |
|
|
|
|
!records.records(tp).isEmpty |
|
|
|
|
}, "Consumer did not consume any message before timeout.") |
|
|
|
|
assertEquals("should be assigned once", 1, listener0.callsToAssigned) |
|
|
|
|
// Verify the metric exist. |
|
|
|
|
val tags1 = new util.HashMap[String, String]() |
|
|
|
|
tags1.put("client-id", "testPerPartitionLeadMetricsCleanUpWithSubscribe") |
|
|
|
|
tags1.put("topic", tp.topic()) |
|
|
|
|
tags1.put("partition", String.valueOf(tp.partition())) |
|
|
|
|
|
|
|
|
|
val tags2 = new util.HashMap[String, String]() |
|
|
|
|
tags2.put("client-id", "testPerPartitionLeadMetricsCleanUpWithSubscribe") |
|
|
|
|
tags2.put("topic", tp2.topic()) |
|
|
|
|
tags2.put("partition", String.valueOf(tp2.partition())) |
|
|
|
|
val fetchLead0 = consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags1)) |
|
|
|
|
assertNotNull(fetchLead0) |
|
|
|
|
assertTrue(s"The lead should be ${records.count}", fetchLead0.metricValue() == records.count) |
|
|
|
|
|
|
|
|
|
// Remove topic from subscription |
|
|
|
|
consumer.subscribe(List(topic2).asJava, listener0) |
|
|
|
|
TestUtils.waitUntilTrue(() => { |
|
|
|
|
consumer.poll(100) |
|
|
|
|
listener0.callsToAssigned >= 2 |
|
|
|
|
}, "Expected rebalance did not occur.") |
|
|
|
|
// Verify the metric has gone |
|
|
|
|
assertNull(consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags1))) |
|
|
|
|
assertNull(consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags2))) |
|
|
|
|
} finally { |
|
|
|
|
consumer.close() |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLagMetricsCleanUpWithSubscribe() { |
|
|
|
@ -1426,6 +1474,41 @@ class PlaintextConsumerTest extends BaseConsumerTest {
@@ -1426,6 +1474,41 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLeadMetricsCleanUpWithAssign() { |
|
|
|
|
val numMessages = 1000 |
|
|
|
|
// Test assign |
|
|
|
|
// send some messages. |
|
|
|
|
sendRecords(numMessages, tp) |
|
|
|
|
sendRecords(numMessages, tp2) |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLeadMetricsCleanUpWithAssign") |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLeadMetricsCleanUpWithAssign") |
|
|
|
|
val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) |
|
|
|
|
try { |
|
|
|
|
consumer.assign(List(tp).asJava) |
|
|
|
|
var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() |
|
|
|
|
TestUtils.waitUntilTrue(() => { |
|
|
|
|
records = consumer.poll(100) |
|
|
|
|
!records.records(tp).isEmpty |
|
|
|
|
}, "Consumer did not consume any message before timeout.") |
|
|
|
|
// Verify the metric exist. |
|
|
|
|
val tags = new util.HashMap[String, String]() |
|
|
|
|
tags.put("client-id", "testPerPartitionLeadMetricsCleanUpWithAssign") |
|
|
|
|
tags.put("topic", tp.topic()) |
|
|
|
|
tags.put("partition", String.valueOf(tp.partition())) |
|
|
|
|
val fetchLead = consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags)) |
|
|
|
|
assertNotNull(fetchLead) |
|
|
|
|
|
|
|
|
|
assertTrue(s"The lead should be ${records.count}", records.count == fetchLead.metricValue()) |
|
|
|
|
|
|
|
|
|
consumer.assign(List(tp2).asJava) |
|
|
|
|
TestUtils.waitUntilTrue(() => !consumer.poll(100).isEmpty, "Consumer did not consume any message before timeout.") |
|
|
|
|
assertNull(consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags))) |
|
|
|
|
} finally { |
|
|
|
|
consumer.close() |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLagMetricsCleanUpWithAssign() { |
|
|
|
|
val numMessages = 1000 |
|
|
|
@ -1467,6 +1550,34 @@ class PlaintextConsumerTest extends BaseConsumerTest {
@@ -1467,6 +1550,34 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLeadWithMaxPollRecords() { |
|
|
|
|
val numMessages = 1000 |
|
|
|
|
val maxPollRecords = 10 |
|
|
|
|
sendRecords(numMessages, tp) |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLeadWithMaxPollRecords") |
|
|
|
|
consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) |
|
|
|
|
val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) |
|
|
|
|
consumer.assign(List(tp).asJava) |
|
|
|
|
try { |
|
|
|
|
var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() |
|
|
|
|
TestUtils.waitUntilTrue(() => { |
|
|
|
|
records = consumer.poll(100) |
|
|
|
|
!records.isEmpty |
|
|
|
|
}, "Consumer did not consume any message before timeout.") |
|
|
|
|
|
|
|
|
|
val tags = new util.HashMap[String, String]() |
|
|
|
|
tags.put("client-id", "testPerPartitionLeadWithMaxPollRecords") |
|
|
|
|
tags.put("topic", tp.topic()) |
|
|
|
|
tags.put("partition", String.valueOf(tp.partition())) |
|
|
|
|
val lead = consumer.metrics.get(new MetricName("records-lead", "consumer-fetch-manager-metrics", "", tags)) |
|
|
|
|
assertTrue(s"The lead should be $maxPollRecords", lead.metricValue() == maxPollRecords) |
|
|
|
|
} finally { |
|
|
|
|
consumer.close() |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@Test |
|
|
|
|
def testPerPartitionLagWithMaxPollRecords() { |
|
|
|
|
val numMessages = 1000 |
|
|
|
|