Skip to content

Kafka-4351: Topic regex behavioral change with MirrorMaker new consumer#2072

Closed
huxihx wants to merge 21 commits intoapache:trunkfrom
huxihx:kafka-4351_Regex_behavior_change_for_new_consumer
Closed

Kafka-4351: Topic regex behavioral change with MirrorMaker new consumer#2072
huxihx wants to merge 21 commits intoapache:trunkfrom
huxihx:kafka-4351_Regex_behavior_change_for_new_consumer

Conversation

@huxihx
Copy link
Contributor

@huxihx huxihx commented Oct 28, 2016

Support CSV values in the regexp for MirrorMaker new consumer as OldConsumer does.

…nsumer

Author: huxi_2b@hotmail.com

Support CSV values in the regexp for MirrorMaker new consumer as OldConsumer does.
@huxihx
Copy link
Contributor Author

huxihx commented Oct 28, 2016

@joestein please help review this. Thanks.

@ijuma
Copy link
Member

ijuma commented Oct 28, 2016

Thanks for the PR. A couple of things:

  1. This could break existing users, so we need to decide if we're OK with that. Probably worth starting a mailing list thread to get wider input. If we go ahead, we should add a note to the upgrade notes so that users are aware of the change.
  2. There should be tests to verify the change.

@huxihx
Copy link
Contributor Author

huxihx commented Oct 28, 2016

@ijuma, totally agreed on that we need to test the change. As for the doc update thing, I think we might not need to do this since right now the document already claims:

For convenience we allow the use of ',' instead of '|' to specify a list of topics.

And we know this is exactly what this fix is about. Without the change, only old consumer supports CSV-typed regular expression because of the fact that TopicFilter will convert them internally. But the new consumer is not able to do this. Does it make sense?

@ijuma
Copy link
Member

ijuma commented Oct 28, 2016

That's a fair point, if we already document it like that, then it's just a bug fix. Let's add a note to the upgrade notes still though (upgrade.html).

@huxihx
Copy link
Contributor Author

huxihx commented Oct 28, 2016

@ijuma Please guide what I should do next. I am a freshman to the process. Thanks.

.replace(" ", "")
.replaceAll("""^["']+""","")
.replaceAll("""["']+$""","") // property files may bring quotes
consumer.subscribe(Pattern.compile(revisedWhitelist), consumerRebalanceListener)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What if we use something like this

          val whitelist = Whitelist(whitelistOpt.get)
          consumer.subscribe(Pattern.compile(whitelist.regex), consumerRebalanceListener)

so that regex manipulation is hardcoded in one place?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vahidhashemian Agreed on your suggestion. And if we employs Whitelist here, we need to additionally do a minor change for the exception capturing. Here is the analysis:

Before initializing Whitelist, the construction method for TopicFilter will be executed, which invokes Pattern.compile(regex). If an 'invalid' regex is offered, TopicFilter captures it and throws RuntimeException(not PatternSyntaxException), which means, later, the init method for MirrorMakerNewConsumer fails to capture this exception since it stares at PatternSyntaxException (although the impact is ignorable since the enclosing run method captures throwable)

Does it make sense? If you agree with the analysis above, I will make another commit following your suggestion and have this PR ready to be reviewed again. Is it okay to you?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for explaining. It makes sense to me.
On a separate note, do you think it makes sense to add some unit test for this change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vahidhashemian Testing is definitely important but as for this case, I am afriad what we need to test is whether Whitelist behaves as expected, which has already been tested in TopicFilterTest. Besides, I made a new commit following your suggestion, please have a review. Thanks.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like you committed more than needed in the last try?
On the unit test note, I'm wondering if it makes sense to add (at least) one unit test that verifies the issue raised in the JIRA would not occur with this patch?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@vahidhashemian already committed a new one. And as for the unit test thing, which file do you think should be better for the test case to be added in? TopicFilterTest or MirrorMakerTest?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the unit test I'd add it to MirrorMakerTest.

amethystic and others added 6 commits October 29, 2016 11:30
author: huxi_2b@hotmail.com

Support CSV values in the regexp for MirrorMaker new consumer as
OldConsumer does.
Author: huxi_2b@hotmail.com

Support CSV values in the regexp for MirrorMaker new consumer as
OldConsumer does.
@huxihx
Copy link
Contributor Author

huxihx commented Nov 3, 2016

@vahidhashemian @ijuma any feedback for this PR? Thanks.

@huxihx
Copy link
Contributor Author

huxihx commented Nov 4, 2016

@joestein Could you help review this PR? Thanks.

@vahidhashemian
Copy link
Contributor

@amethystic The change you made to fix the issue looks good to me. But @ijuma's concerns above should still be addressed in my opinion.

@huxihx
Copy link
Contributor Author

huxihx commented Nov 4, 2016

@vahidhashemian Thanks for your feedback. As for @ijuma 's suggestion, seems that he mentions of upgrade.html, so do you know how to add a update not for this PR? How should I do then ?

@ijuma
Copy link
Member

ijuma commented Nov 15, 2016

upgrade.html is in docs folder. Not sure what you mean by "a update not for this PR".

@huxihx
Copy link
Contributor Author

huxihx commented Nov 16, 2016

@ijuma Thanks. So can this PR be closed now?

@ijuma
Copy link
Member

ijuma commented Nov 17, 2016

@amethystic no, as it has not been merged yet. To clarify: before we merge it, we need:

  1. Tests
  2. Add a note to docs/upgrade.html mentioning this change

…pression is supported for the --whitelist option of the new consumer in MirrorMaker'
@huxihx
Copy link
Contributor Author

huxihx commented Nov 18, 2016

@ijuma Thanks for the advice. Already updated the upgrade note. Besides, I did a test in my local environment, which was passed. Both single topic and regular expressions are supported now for the new consumer of MirrorMaker. Do you think we need to add some unit test cases although it is not an easy work to do this?

@huxihx huxihx closed this Nov 18, 2016
@huxihx huxihx reopened this Nov 18, 2016
@huxihx
Copy link
Contributor Author

huxihx commented Dec 1, 2016

@ijuma could you please address the comments above? Thanks.

@ijuma
Copy link
Member

ijuma commented Dec 1, 2016

@amethystic, yes it would be good to add at least one test.

@huxihx
Copy link
Contributor Author

huxihx commented Dec 5, 2016

@ijuma Sorry, but do you mean the unit test?

@asfbot
Copy link

asfbot commented Dec 13, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/95/
Test PASSed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented Dec 13, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/96/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link

asfbot commented Dec 13, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/94/
Test PASSed (JDK 7 and Scala 2.10).

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the delay. Thanks for adding the test, very useful. I left a few suggestions and questions.

import org.junit.Test
import org.junit.{After, Before, Test}

class MirrorMakerTest extends ZooKeeperTestHarness {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suggest creating a new test under integration/kafka/tools/MirrorMaker.scala instead of reusing the existing unit test to avoid making the existing unit tests slower due to initialisation that they don't need.

Also, it seems like it may be better to inherit from KafkaServerTestHarness than ZooKeeperTestHarness?

val props = new Properties()
props.put("bootstrap.servers", brokerList)
props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer")
props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of setting the serializers like this, you can just pass them to the KafkaProducer constructor.

props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer")
val producer1 = new KafkaProducer[Array[Byte], Array[Byte]](props)
producer1.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, msg.getBytes()))
producer1.flush() // Explicitly invoke flush method to make effect immediately
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

close should have the same effect as flush. Does the test fail if you remove flush?

props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer")
props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer")
val producer1 = new KafkaProducer[Array[Byte], Array[Byte]](props)
producer1.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, msg.getBytes()))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should probably call get on the returned Future to ensure that we get a useful error if there is a failure. Also, you can omit [Array[Byte], Array[Byte]] in both cases above.

val producer1 = new KafkaProducer[Array[Byte], Array[Byte]](props)
producer1.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, msg.getBytes()))
producer1.flush() // Explicitly invoke flush method to make effect immediately
producer1.close() // Close the producer
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This comment is redundant.


// Create a MirrorMaker consumer
val config = new util.HashMap[String, AnyRef]
config.put(ConsumerConfig.GROUP_ID_CONFIG, "test-gropu")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Typo in group name.

config.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
config.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
config.put("key.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer")
config.put("value.deserializer", "org.apache.kafka.common.serialization.ByteArrayDeserializer")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Again, you can pass the deserializers via the constructor.

val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config)
MirrorMaker.createMirrorMakerProducer(brokerList)

val whitelist = Some("new.*|another_topic|foo" ) // Test a regular expression
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought we wanted to test multiple regular expressions separated by commas?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I replaced all "|" with commas. Is that what you expect?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that should fail without your fix, I think. If we wanted to make the test even better, we'd produce to another_topic and yet_another_topic and check that the consumer only consumes from the two that match.

But the test as it is verifies the fix, so I won't block the merge if you don't feel like extending the test. :)

}

// Only for testing
private[kafka] def createMirrorMakerProducer(brokerList: String): Unit = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this method here? Can it not be in the test?

@@ -561,9 +578,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
val consumerRebalanceListener = new InternalRebalanceListenerForNewConsumer(this, customRebalanceListener)
if (whitelistOpt.isDefined) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of whitelistOpt.isDefined, we can write it as:

whitelistOpt.foreach { whitelist =>
  try {
    consumer.subscribe(Pattern.compile(Whitelist(whitelist).regex), consumerRebalanceListener)
    ...
  }
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What are your thoughts on this comment?

1. Move unit test case under integration/kafka/tools
2. Remove test code in MirrorMaker
@asfbot
Copy link

asfbot commented Dec 22, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/361/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link

asfbot commented Dec 22, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/359/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link

asfbot commented Dec 22, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/360/
Test FAILed (JDK 8 and Scala 2.12).

@huxihx
Copy link
Contributor Author

huxihx commented Dec 22, 2016

@ijuma already addressed most of your comments. Please have a review. Thanks

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, I think we're close. Left a few more comments.


private class MirrorMakerNewConsumer(consumer: Consumer[Array[Byte], Array[Byte]],
// Only for testing
private[kafka] class MirrorMakerNewConsumer(consumer: Consumer[Array[Byte], Array[Byte]],
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be private[tools]?

}

private class MirrorMakerProducer(val producerProps: Properties) {
private[kafka] class MirrorMakerProducer(val producerProps: Properties) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be private[tools]?

@After
override def tearDown() {
super.tearDown()
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't need tearDown and setUp defined here if they just call super.

mirrorMakerConsumer.init()
try {
val data = mirrorMakerConsumer.receive()
println(new String(data.value))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably left unintentionally?

try {
val data = mirrorMakerConsumer.receive()
println(new String(data.value))
assertTrue(s"MirrorMaker consumer should get the correct topic: $topic", data.topic.equals(topic))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In Scala, data.topic == topic is more idiomatic. Same for the next line.

assertTrue(s"MirrorMaker consumer should get the correct topic: $topic", data.topic.equals(topic))
assertTrue("MirrorMaker consumer should read the correct message.", new String(data.value).equals(msg))
} catch {
case e: RuntimeException =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need to catch and rethrow, just let the original exception bubble up.

// Create a test producer to delivery a message
val producerProps = new Properties()
producerProps.put("bootstrap.servers", brokerList)
producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can set the class itself instead of the name, I think. e.g. classOf[ByteArraySerializer]. Same in the line below.

@@ -561,9 +578,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
val consumerRebalanceListener = new InternalRebalanceListenerForNewConsumer(this, customRebalanceListener)
if (whitelistOpt.isDefined) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What are your thoughts on this comment?

import org.junit.Assert.assertTrue
import org.junit.{After, Before, Test}

class TestMirrorMaker extends KafkaServerTestHarness {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought MirrrorMakerTest was fine, why did you rename it?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The compiler will complain "MirrorMakerTest is already defined as class MirrorMakerTest". That's why I changed the name. Do you think there is another way I could avoid this error?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, that makes sense because they both have the same package (it's just the source folder that differs). In that case, I suggest MirrorMakerIntegrationTest for this one so the difference is obvious.

producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
val producer = new MirrorMakerProducer(producerProps)
MirrorMaker.producer = producer
MirrorMaker.producer.send(new ProducerRecord(topic, msg.getBytes()))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As I said before, it would be good to call get on the result of send to ensure exceptions get bubbled up.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The producer here is not of type 'KafkaProducer' but 'MirrorMakerProducer'. It's send method is already KafkaProducer.send().get() by default if not explicitly specifying 'async'. Does it make sense?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, thanks for clarifying.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, the default seems to be async?

val sync = producerProps.getProperty("producer.type", "async").equals("sync")

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, my mistake. Do you think it's okay to add props.put("producer.type", "sync") before MirrorMakerProducer?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good to me.

amethystic added 3 commits December 22, 2016 21:02
@asfbot
Copy link

asfbot commented Dec 25, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/387/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link

asfbot commented Dec 25, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/386/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented Dec 25, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/385/
Test FAILed (JDK 7 and Scala 2.10).

@huxihx
Copy link
Contributor Author

huxihx commented Dec 25, 2016

@ijuma Thanks for your great patience. Please review it again after addressing your comments. Thanks.

Copy link
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the updates, a few more comments.

assertTrue("MirrorMaker consumer should read the correct message.", new String(data.value) == msg)
return
} catch {
case _: ConsumerTimeoutException => // swallow it
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, this doesn't seem right. Why do we swallow this exception?

Copy link
Contributor Author

@huxihx huxihx Dec 31, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

`val mirrorMakerConsumer = new MirrorMakerNewConsumer(consumer, None, whitelist)

mirrorMakerConsumer.init()

try {
TestUtils.waitUntilTrue(() => {
try {
val data = mirrorMakerConsumer.receive()
data.topic == topic && new String(data.value) == msg
} catch {
case _: ConsumerTimeoutException => false
}
}, "MirrorMaker consumer should be able to subscribe the correct topic and read the correct message.")
} finally {
consumer.close()
}`
Is that okay to you then? Thanks.

mirrorMakerConsumer.init()
try {
val maxTryCount = 3 // it might need to call multiple poll calls to retrieve the message
for (_ <- 0 until maxTryCount) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should use TestUtils.waitUntilTrue instead of this.

MirrorMaker.producer.close()

servers foreach { server =>
println(server.zkUtils.getAllTopics().mkString(","))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We shouldn't have this. Unintentional?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I always forget to remove them before committing. Sorry, my mistake.

producerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
producerProps.put("producer.type", "sync")
producerProps.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
producerProps.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did it not work to use classOf[ByteArraySerializer] instead of hardcoding the String?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems MirrorMakerProducer is not a parameterized class which cannot use type parameters during construction.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's not what I was suggesting. That's OK, I'll do it before merging.

fix MirrorMakerIntegrationTest.testRegularExpressionTopic code as per
Ijuma's comments.
@asfbot
Copy link

asfbot commented Dec 31, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/405/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link

asfbot commented Dec 31, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/406/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link

asfbot commented Dec 31, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/407/
Test FAILed (JDK 8 and Scala 2.11).

@ijuma
Copy link
Member

ijuma commented Dec 31, 2016

Thanks for the updates, LGTM with a few minor changes that I will do before merging to trunk: moved upgrade notes to the right place and reworded them a little, added license comment to test, minor style improvements.

@huxihx
Copy link
Contributor Author

huxihx commented Jan 1, 2017

Thanks for your great patience. I do learn a lot from this whole review cycle.

@asfgit asfgit closed this in 29d456c Jan 3, 2017
@huxihx huxihx deleted the kafka-4351_Regex_behavior_change_for_new_consumer branch January 4, 2017 00:59
soenkeliebau pushed a commit to soenkeliebau/kafka that referenced this pull request Feb 7, 2017
…ted regex

This makes it consistent with MirrorMaker with the old consumer.

Author: huxi <huxi@zhenrongbao.com>
Author: amethystic <huxi_2b@hotmail.com>

Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Ismael Juma <ismael@juma.me.uk>

Closes apache#2072 from amethystic/kafka-4351_Regex_behavior_change_for_new_consumer
efeg pushed a commit to efeg/kafka that referenced this pull request May 29, 2024
Add anomaly detect-to-fix-complete-timer metrics which measures the time from anomaly being detected to self-healing completed.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants