Skip to content

Comments

KAFKA-5588: Remove deprecated new-consumer option for tools#5097

Merged
ijuma merged 8 commits intoapache:trunkfrom
ppatierno:kafka-5588
Jun 6, 2018
Merged

KAFKA-5588: Remove deprecated new-consumer option for tools#5097
ijuma merged 8 commits intoapache:trunkfrom
ppatierno:kafka-5588

Conversation

@ppatierno
Copy link
Contributor

@ppatierno ppatierno commented May 30, 2018

Removed the deprecated "--new-consumer" option for all consumer based tools.
This option was already deprecated in the previous version so it can be removed in the coming new 2.0.0 major version.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Contributor

@viktorsomogyi viktorsomogyi left a comment

Choose a reason for hiding this comment

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

Reviewed where do we use the new-consumer option. It seems that inside the ducktapes we still use them. Do you think those usages should be removed from there too?

cmd += " --new-consumer"

cmd = "%s --new-consumer --bootstrap-server %s %s --list" % \

cmd = "%s --new-consumer --bootstrap-server %s %s --group %s --describe" % \

"new-consumer", "Use the new consumer implementation."

@ppatierno
Copy link
Contributor Author

@viktorsomogyi I was thinking the same thing but first waiting for a review by some committer.

Copy link
Contributor

@vahidhashemian vahidhashemian left a comment

Choose a reason for hiding this comment

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

Thank you for the patch.

Would it be possible to also add a test case for both ConsumerGroupCommand and ConsoleConsumer that verifies using the --new-consumer option causes an error?

I think the upgrade document also needs to be updated to mention this change.

@ppatierno
Copy link
Contributor Author

ppatierno commented May 31, 2018

@vahidhashemian thanks for the feedback.
Regarding ConsoleConsumer, the ConsumerConfig class uses the tryParse method which just prints the help and exits if a wrong option is specified. It's different from the ConsumerPerformace where an exception is raised in this case (so I use that as "expected" for the unit test). Any opinion on that?
Regarding the ConsumerGroupCommand, we have the ConsumerGroupCommandTest class which is just base class for other tests but it doesn't have unit tests aboud configu itself.
I was thinking to not modify the current ConsumerGroupCommandTest in the kafka.admin package but adding a ConsumerGroupCommandTest in the kafka.tools package alongside ConsoleConsumerTest and ConsumerPerformanceTest. Wdyt ?
Regarding the upgrade doc I'll update that, thanks!

@vahidhashemian
Copy link
Contributor

@ppatierno, do you think for ConsoleConsumerTest we can do something like in shouldExitOnInvalidConfigWithAutoOffsetResetAndConflictingFromBeginningOldConsumer? Also, for consumer groups, maybe something similar to testDescribeWithMultipleSubActions in DescribeConsumerGroupTest?

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 PR. Please search the *.py code too as it has a few mentions of --new-consumer.

@ijuma ijuma added this to the 2.0.0 milestone Jun 1, 2018
@ppatierno
Copy link
Contributor Author

@viktorsomogyi @ijuma related to the Python files as far as I can see the --new-consumer option is needed for Kafka version <= 0.10.0.0 and there is a check for that in the console_consumer.py and consumer_performance.py files.
The same doesn't happen in the kafka.py.
Instead of removing the option from there I think that such a check should be added to kafka.py as well.
Wdyt ?

@ijuma
Copy link
Member

ijuma commented Jun 1, 2018

@ppatierno Sounds good.

@ppatierno
Copy link
Contributor Author

@vahidhashemian thanks for the suggestions, I didn't notice that Exit class provided a way for changing the exit behaviour. I added the unit tests which make sense about unrecognized --new-consumer option; I added the test for all the possible consumer group classes (describe, list, delete and reset) for consistency because they are different mutual options for the same tool.

@viktorsomogyi @ijuma I updated the Python tests as proposed.

Copy link
Contributor

@vahidhashemian vahidhashemian 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 addressing my comment. LGTM.

Copy link
Contributor

@viktorsomogyi viktorsomogyi 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 addressing my comments too.

@ppatierno
Copy link
Contributor Author

@vahidhashemian @viktorsomogyi thanks for your review and for accepting the changes.
@ijuma have you time to review and maybe merged by tomorrow ? Tomorrow is the deadline as far as I understood from Rajini emal :

Please ensure that all KIPs for 2.0.0 have been merged by Tuesday the 5th of June. Any remaining KIPs will be moved to the next release

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.

Just some minor nits.

def testDeleteWithUnrecognizedNewConsumerOption() {
val cgcArgs = Array("--new-consumer", "--bootstrap-server", brokerList, "--delete", "--group", group)
getConsumerGroupService(cgcArgs)
fail("Expected an error due to presence of unrecognized --new-consumer option")
Copy link
Member

Choose a reason for hiding this comment

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

This is not needed given the expected exception annotation. Same for other similar cases.

Copy link
Contributor

Choose a reason for hiding this comment

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

Do we really need tests for checking unrecognized option "--new-consumer" ?
maybe one round of manual testing should be sufficient. maintaining these tests in repo may not be worth.

result.keySet.contains(group) && result.get(group).contains(Errors.COORDINATOR_NOT_AVAILABLE))
}

@Test(expected = classOf[joptsimple.OptionException])
Copy link
Member

Choose a reason for hiding this comment

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

Nit: why are we not importing OptionException?


// Override exit procedure to throw an exception instead of exiting, so we can catch the exit
// properly for this test case
Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull))
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 seems redundant given the code.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, I am going to remove it from all the other places in the same test scala class

<li><a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-176%3A+Remove+deprecated+new-consumer+option+for+tools">KIP-176</a> removes
the <code>--new-consumer</code> option for all consumer based tools as <code>kafka-console-consumer</code>, <code>kafka-consumer-perf-test</code>
and <code>kafka-consumer-groups</code>. The new consumer is automatically used if the bootstrap servers list is provided on the command line
otherwise, providing the zookeeper connection instead, the old one is used.
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 make it clear that we haven't changed the behaviour, -new-consumer was already a no-op.

@ppatierno
Copy link
Contributor Author

@ijuma thanks for the review. I addressed the change requests.

val hideHeader = options.has(hideHeaderOpt)
val recordFetchTimeoutMs = options.valueOf(recordFetchTimeoutOpt).longValue()
val bootstrapServers = options.valueOf(bootstrapServersOpt)
val zkConnectionStr = options.valueOf(zkConnectOpt)
Copy link
Member

Choose a reason for hiding this comment

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

These seem to be unused?

Copy link
Contributor

Choose a reason for hiding this comment

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

They are used in the added unit tests in ConsumerPerformanceTest.scala.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes exactly. They are there just for testing that without the - - new-consumer option all depends on the bootstrap servers list and zookeeper connection string and that the provided value to the configuration are what we expect.

Copy link
Member

Choose a reason for hiding this comment

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

I see. This is a bit odd and I'd rather not do that. I will push a commit with minor fixes.

*/
package unit.kafka.admin

import joptsimple.OptionException
Copy link
Contributor

Choose a reason for hiding this comment

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

With this import, joptsimple. can be dropped from the expected exception class of the first method below.
Also, no need for the fail statement there (similar to @ijuma's earlier comment). Not your code, but it would be great to clean them 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.

Yes it makes sense. I have done the same for the testConfigUsingNewConsumerUnrecognizedOption in the ConsumerPerformanceTest class.

Copy link
Contributor

Choose a reason for hiding this comment

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

Perfect. 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.

LGTM. I agree that we probably don't want to keep the tests for --new-consumer for long, but we can include them for now and remove them in one or two release cycles.

@ijuma
Copy link
Member

ijuma commented Jun 6, 2018

Test failure is unrelated, merging to trunk.

@ijuma ijuma merged commit be8808d into apache:trunk Jun 6, 2018
@ppatierno ppatierno deleted the kafka-5588 branch June 6, 2018 06:45
ijuma added a commit to edoardocomar/kafka that referenced this pull request Jun 6, 2018
…grained-acl-create-topics

* apache-github/trunk:
  KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097)
  MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040)
  KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE
  MINOR: docs should point to latest version (apache#5132)
  KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298)
  [KAFKA-6730] Simplify State Store Recovery (apache#5013)
  MINOR: Rename package `internal` to `internals` for consistency (apache#5137)
  KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801)
  MINOR: Add missing configs for resilience settings
  MINOR: Add regression tests for KTable mapValues and filter (apache#5134)
  KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829)
  KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956)
  KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128)
  KAFKA-6813: return to double-counting for count topology names (apache#5075)
  KAFKA-5919; Adding checks on "version" field for tools using it
  MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
ijuma added a commit to big-andy-coates/kafka that referenced this pull request Jun 6, 2018
…refix

* apache-github/trunk:
  KAFKA-6726: Fine Grained ACL for CreateTopics (KIP-277) (apache#4795)
  KAFKA-5588: Remove deprecated --new-consumer tools option (apache#5097)
  MINOR: Fix for the location of the trogdor.sh executable file in the documentation. (apache#5040)
  KAFKA-6997: Exclude test-sources.jar when $INCLUDE_TEST_JARS is FALSE
  MINOR: docs should point to latest version (apache#5132)
  KAFKA-6981: Move the error handling configuration properties into the ConnectorConfig and SinkConnectorConfig classes (KIP-298)
  [KAFKA-6730] Simplify State Store Recovery (apache#5013)
  MINOR: Rename package `internal` to `internals` for consistency (apache#5137)
  KAFKA-6704: InvalidStateStoreException from IQ when StreamThread closes store (apache#4801)
  MINOR: Add missing configs for resilience settings
  MINOR: Add regression tests for KTable mapValues and filter (apache#5134)
  KAFKA-6750: Add listener name to authentication context (KIP-282) (apache#4829)
  KAFKA-3665: Enable TLS hostname verification by default (KIP-294) (apache#4956)
  KAFKA-6938: Add documentation for accessing Headers on Kafka Streams Processor API (apache#5128)
  KAFKA-6813: return to double-counting for count topology names (apache#5075)
  KAFKA-5919; Adding checks on "version" field for tools using it
  MINOR: Remove deprecated KafkaStreams constructors in docs (apache#5118)
ying-zheng pushed a commit to ying-zheng/kafka that referenced this pull request Jul 6, 2018
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Vahid Hashemian <vahidhashemian@us.ibm.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
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.

5 participants