Skip to content

KAFKA-7190: KIP-443; Remove streams overrides on repartition topics#6511

Merged
guozhangwang merged 3 commits intoapache:trunkfrom
guozhangwang:K7190-remove-repartition-overrides
Apr 3, 2019
Merged

KAFKA-7190: KIP-443; Remove streams overrides on repartition topics#6511
guozhangwang merged 3 commits intoapache:trunkfrom
guozhangwang:K7190-remove-repartition-overrides

Conversation

@guozhangwang
Copy link
Copy Markdown
Contributor

As described in KIP-443 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-443%3A+Return+to+default+segment.ms+and+segment.index.bytes+in+Streams+repartition+topics). We want to remove the aggressive overrides of segment.ms and segment.index.bytes for repartition topics. The remaining segment.bytes should still be effective in bounding its footprint.

Committer Checklist (excluded from commit message)

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

final Map<String, Object> topicProps = originalsWithPrefix(TOPIC_PREFIX, false);
final Map<String, Object> producerProps = getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames());

if (topicProps.containsKey(topicPrefix(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG))) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is a piggy-backed fix to not use SEGMENT_INDEX_BYTES_CONFIG but SEGMENT_BYTES_CONFIG

@guozhangwang
Copy link
Copy Markdown
Contributor Author

Copy link
Copy Markdown
Member

@bbejeck bbejeck left a comment

Choose a reason for hiding this comment

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

Thanks @guozhangwang LGTM

@guozhangwang guozhangwang merged commit 213466b into apache:trunk Apr 3, 2019
guozhangwang added a commit to guozhangwang/kafka that referenced this pull request Apr 3, 2019
…pache#6511)

As described in KIP-443 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-443%3A+Return+to+default+segment.ms+and+segment.index.bytes+in+Streams+repartition+topics). We want to remove the aggressive overrides of segment.ms and segment.index.bytes for repartition topics. The remaining segment.bytes should still be effective in bounding its footprint.

Reviewers: Bill Bejeck <bbejeck@gmail.com>
@guozhangwang guozhangwang deleted the K7190-remove-repartition-overrides branch April 3, 2019 00:18
final Map<String, String> properties = topicConfig.getProperties(Collections.<String, String>emptyMap(), 10000);
assertEquals(5, properties.size());
assertEquals(3, properties.size());
assertEquals(String.valueOf(Long.MAX_VALUE), properties.get(TopicConfig.RETENTION_MS_CONFIG));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think this needs to be changed to -1 to fit with what is being set in the properties. At the moment this test fails consistently.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@soenkeliebau That was my bad, I ran than unit test while this change was stashed. Thanks!

jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* apache/trunk:
  MINOR: Add security considerations for remote JMX in Kafka docs (apache#6544)
  MINOR: Remove redundant access specifiers from metrics interfaces (apache#6527)
  MINOR: Correct KStream documentation (apache#6552)
  KAFKA-8013; Avoid underflow when reading a Struct from a partially correct buffer (apache#6340)
  KAFKA-8058: Fix ConnectClusterStateImpl.connectors() method (apache#6384)
  MINOR: Move common consumer tests out of abstract consumer class (apache#6548)
  KAFKA-8168; Add a generated ApiMessageType class
  KAFKA-7893; Refactor ConsumerBounceTest to reuse functionality from BaseConsumerTest (apache#6238)
  MINOR: Tighten up metadata upgrade test (apache#6531)
  KAFKA-8190; Don't update keystore modification time during validation (apache#6539)
  MINOR: Fixed a few warning in core and connects (apache#6545)
  KAFKA-7904; Add AtMinIsr partition metric and TopicCommand option (KIP-427)
  MINOR: fix throttling and status in ConnectionStressWorker
  KAFKA-8090: Use automatic RPC generation in ControlledShutdown
  KAFKA-6399: Remove Streams max.poll.interval override (apache#6509)
  KAFKA-8126: Flaky Test org.apache.kafka.connect.runtime.WorkerTest.testAddRemoveTask (apache#6475)
  HOTFIX: Update unit test for KIP-443
  KAFKA-7190: KIP-443; Remove streams overrides on repartition topics (apache#6511)
  KAFKA-8183: Add retries to WorkerUtils#verifyTopics (apache#6532)
  KAFKA-8181: Removed Avro topic from TOC on kafka (apache#6529)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
…pache#6511)

* remove streams overrides on segment.ms and segment.index.bytes

* kip comments
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants