Skip to content

KAFKA-6813: Remove deprecated APIs in KIP-182, Part II#4976

Merged
guozhangwang merged 27 commits intoapache:trunkfrom
guozhangwang:K6813-part2-store-supplier
May 10, 2018
Merged

KAFKA-6813: Remove deprecated APIs in KIP-182, Part II#4976
guozhangwang merged 27 commits intoapache:trunkfrom
guozhangwang:K6813-part2-store-supplier

Conversation

@guozhangwang
Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang commented May 8, 2018

  1. Remove the deprecated StateStoreSuppliers, and the corresponding Stores.create() functions and factories: only the base StateStoreSupplier and MockStoreSupplier were still preserved as they are needed by the deprecated TopologyBuilder and KStreamBuilder. Will remove them in a follow-up PR.

  2. Add TopologyWrapper.java as the original InternalTopologyBuilderAccessor was removed, but I realized it is still needed as of now.

  3. Minor: removed StateStoreTestUtils.java and inline its logic in its callers since now with StoreBuilder it is just a one-liner.

Committer Checklist (excluded from commit message)

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

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. Just a couple of minor comments otherwise LGTM.

private final String name;
private final long retentionPeriod;

private static final int NUM_SEGMENTS = 3;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Why the increase from 2 -> 3 ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

RocksDBSessionStoreSupplier.NUM_SEGMENTS is 3 -- it's not an increase here, is it?

private final boolean retainDuplicates;

@SuppressWarnings("deprecation")
private static final int MIN_SEGMENTS = 2;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Should the MIN_SEGMENTS here be 3 as well or should the one above change?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

RocksDBWindowStoreSupplier.MIN_SEMENTS is 2 -- note that MIN_SEGMENTS and NUM_SEGMENTS are two different things.

@bbejeck
Copy link
Copy Markdown
Member

bbejeck commented May 8, 2018

test failures in Java 8 unrelated.

retest this please.

@mjsax mjsax added the streams label May 9, 2018
private final String name;
private final long retentionPeriod;

private static final int NUM_SEGMENTS = 3;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

RocksDBSessionStoreSupplier.NUM_SEGMENTS is 3 -- it's not an increase here, is it?

private final boolean retainDuplicates;

@SuppressWarnings("deprecation")
private static final int MIN_SEGMENTS = 2;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

RocksDBWindowStoreSupplier.MIN_SEMENTS is 2 -- note that MIN_SEGMENTS and NUM_SEGMENTS are two different things.

builder.addSource(null, "source", null, null, null, "topic");
builder.addProcessor("processor", new MockProcessorSupplier(), "source");
builder.addStateStore(new MockStateStoreSupplier("store", true), "processor");
builder.addStateStore(storeBuilder, "processor");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

the original store is persistent -- but storeBuilder is in-memory -- change intended?

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.

For this test it does not matter. I was just reusing the store instance.

final long thirtySecondTimeout = 30 * 1000;

final TopologyBuilder builder = new TopologyBuilder()
final Topology topology = new TopologyWrapper()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

use TopologyWrapper on left hand side and avoid cast below?

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.

Unfortunately I cannot since once the addXX is called the returned type is Topology. But I refactored the code a bit to not use chained operators:

        final TopologyWrapper topology = new TopologyWrapper();
        topology.addSource("ingest", Pattern.compile("topic-\\d+"));
        topology.addProcessor("my-processor", processorSupplier, "ingest");
        topology.addStateStore(storeBuilder, "my-processor");

@guozhangwang
Copy link
Copy Markdown
Contributor Author

@mjsax @bbejeck updated based on comments

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 9, 2018

@guozhangwang Build failing with checkstyle error.

@guozhangwang
Copy link
Copy Markdown
Contributor Author

@guozhangwang Build failing with checkstyle error.

ah right, thanks.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented May 9, 2018

Retest this please

Copy link
Copy Markdown
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

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

LGTM. Feel free to merge after Jenkins passed.

@guozhangwang guozhangwang merged commit 0b1a118 into apache:trunk May 10, 2018
ijuma added a commit to ijuma/kafka that referenced this pull request May 11, 2018
…-record-version

* apache-github/trunk:
  KAFKA-6894: Improve err msg when connecting processor with global store (apache#5000)
  KAFKA-6893; Create processors before starting acceptor in SocketServer (apache#4999)
  MINOR: Fix typo in ConsumerRebalanceListener JavaDoc (apache#4996)
  MINOR: Remove deprecated valueTransformer.punctuate (apache#4993)
  MINOR: Update dynamic broker configuration doc for truststore update (apache#4954)
  KAFKA-6870 Concurrency conflicts in SampledStat (apache#4985)
  KAFKA-6361: Fix log divergence between leader and follower after fast leader fail over (apache#4882)
  KAFKA-6813: Remove deprecated APIs in KIP-182, Part II (apache#4976)
  KAFKA-6878 Switch the order of underlying.init and initInternal (apache#4988)
  KAFKA-6299; Fix AdminClient error handling when metadata changes (apache#4295)
  KAFKA-6878: NPE when querying global state store not in READY state (apache#4978)
  KAFKA 6673: Implemented missing override equals method (apache#4745)
  KAFKA-6834: Handle compaction with batches bigger than max.message.bytes (apache#4953)
ying-zheng pushed a commit to ying-zheng/kafka that referenced this pull request Jul 6, 2018
1. Remove the deprecated StateStoreSuppliers, and the corresponding Stores.create() functions and factories: only the base StateStoreSupplier and MockStoreSupplier were still preserved as they are needed by the deprecated TopologyBuilder and KStreamBuilder. Will remove them in a follow-up PR.

2. Add TopologyWrapper.java as the original InternalTopologyBuilderAccessor was removed, but I realized it is still needed as of now.

3. Minor: removed StateStoreTestUtils.java and inline its logic in its callers since now with StoreBuilder it is just a one-liner.

Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
@guozhangwang guozhangwang deleted the K6813-part2-store-supplier branch April 24, 2020 23:57
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