Skip to content

KAFKA-13722: Refactor SerdeGetter#18242

Merged
chia7712 merged 2 commits intoapache:trunkfrom
mjsax:kafka-13722-processorcontext-storeserde
Dec 18, 2024
Merged

KAFKA-13722: Refactor SerdeGetter#18242
chia7712 merged 2 commits intoapache:trunkfrom
mjsax:kafka-13722-processorcontext-storeserde

Conversation

@mjsax
Copy link
Copy Markdown
Member

@mjsax mjsax commented Dec 18, 2024

Refactor SerdeGetter to not use old ProcessorContext any longer.

Refactor SerdeGetter to not use old ProcessorContext any longer.
@mjsax mjsax added the streams label Dec 18, 2024
@github-actions github-actions bot added the small Small PRs label Dec 18, 2024
Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

LGTM and one minor comment

newProcessorContext != null ? newProcessorContext.valueSerde() : stateStorecontext.valueSerde();

public Serde<?> valueSerde() {
return processorContext != null ? processorContext.valueSerde() : stateStorecontext.valueSerde();
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 IDE incorrectly assumes that stateStoreContext.valueSerde() might throw a NPE. To eliminate this false alert, perhaps we can retain the Supplier<Serde<?>> instead.

    private final Supplier<Serde<?>> keySerdeSupplier;
    private final Supplier<Serde<?>> valueSerdeSupplier;

    public SerdeGetter(final ProcessorContext<?, ?> context) {
        keySerdeSupplier = context::keySerde;
        valueSerdeSupplier = context::valueSerde;
    }

    public SerdeGetter(final StateStoreContext context) {
        keySerdeSupplier = context::keySerde;
        valueSerdeSupplier = context::valueSerde;
    }

    public Serde<?> keySerde() {
        return keySerdeSupplier.get();
    }

    public Serde<?> valueSerde() {
        return valueSerdeSupplier.get();
    }

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.

@chia7712 I like your proposal!

Copy link
Copy Markdown
Member

@cadonna cadonna 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, @mjsax !

LGTM!

newProcessorContext != null ? newProcessorContext.valueSerde() : stateStorecontext.valueSerde();

public Serde<?> valueSerde() {
return processorContext != null ? processorContext.valueSerde() : stateStorecontext.valueSerde();
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.

@chia7712 I like your proposal!

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Dec 18, 2024

Java 23:

Found 5 test failures:
FAILED ❌ UserQuotaTest > testThrottledProducerConsumer(String, String).quorum=kraft.groupProtocol=consumer
FAILED ❌ UserQuotaTest > testQuotaOverrideDelete(String, String).quorum=kraft.groupProtocol=classic
FAILED ❌ UserQuotaTest > testQuotaOverrideDelete(String, String).quorum=kraft.groupProtocol=consumer
FAILED ❌ PlaintextAdminIntegrationTest > testConsumerGroupsDeprecatedConsumerGroupState(String, String).quorum=kraft.groupProtocol=consumer
FAILED ❌ PlaintextAdminIntegrationTest > testConsumerGroups(String, String).quorum=kraft.groupProtocol=consumer

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Dec 18, 2024

Java 17:

Found 1 test failures:
FAILED ❌ CooperativeConsumerCoordinatorTest > testOutdatedCoordinatorAssignment()

@chia7712
Copy link
Copy Markdown
Member

testThrottledProducerConsumer -> https://issues.apache.org/jira/browse/KAFKA-8073
testQuotaOverrideDelete -> https://issues.apache.org/jira/browse/KAFKA-8032
testConsumerGroupsDeprecatedConsumerGroupState -> https://issues.apache.org/jira/browse/KAFKA-18298
testConsumerGroups -> https://issues.apache.org/jira/browse/KAFKA-18297
testOutdatedCoordinatorAssignment -> https://issues.apache.org/jira/browse/KAFKA-15900

@chia7712 chia7712 merged commit e551cb7 into apache:trunk Dec 18, 2024
@mjsax mjsax deleted the kafka-13722-processorcontext-storeserde branch December 19, 2024 00:30
mjsax added a commit that referenced this pull request Dec 19, 2024
Reviewers: Bruno Cadonna <cadonna@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Dec 19, 2024

Cherry-picked to 4.0 branch.

ijuma added a commit to ijuma/kafka that referenced this pull request Dec 19, 2024
…e-old-protocol-versions

* apache-github/trunk: (25 commits)
  KAFKA-18270: FindCoordinator v0 incorrectly tagged as deprecated (apache#18262)
  KAFKA-18284: Add group coordinator records for Streams rebalance protocol (apache#18228)
  MINOR: Fix flaky state updater test (apache#18253)
  MINOR: improve StreamsResetter logging (apache#18237)
  KAFKA-18227: Ensure v2 partitions are not added to last transaction during upgrade (apache#18176)
  Add IT for share consumer with duration base offet auto reset (apache#18251)
  KAFKA-18283: Add StreamsGroupDescribe RPC definitions (apache#18230)
  KAFKA-18241: add docs check to CI (apache#18183)
  KAFKA-18223 Improve flaky test report (apache#18212)
  MINOR Remove triage label in nightly job (apache#18147)
  KAFKA-18294 Remove deprecated SourceTask#commitRecord (apache#18260)
  KAFKA-18264 Remove NotLeaderForPartitionException (apache#18211)
  KAFKA-13722: Refactor SerdeGetter (apache#18242)
  KAFKA-18094 Remove deprecated TopicListing(String, Boolean) (apache#18248)
  KAFKA-18282: Add StreamsGroupHeartbeat RPC definitions (apache#18227)
  KAFKA-18026: KIP-1112 migrate KTableSuppressProcessorSupplier (apache#18150)
  KAFKA-18026: transition KTable#filter impl to use processor wrapper (apache#18205)
  KAFKA-18293 Remove `org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler` and `org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHandler` (apache#18244)
  MINOR: add assertion about groupEpoch and targetAssignmentEpoch to testConsumerGroups (apache#18203)
  KAFKA-17960; PlaintextAdminIntegrationTest.testConsumerGroups fails with CONSUMER group protocol (apache#18234)
  ...
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
Reviewers: Bruno Cadonna <cadonna@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
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