[
https://issues.apache.org/jira/browse/KAFKA-19983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18044926#comment-18044926
]
sanghyeok An edited comment on KAFKA-19983 at 12/14/25 8:55 AM:
----------------------------------------------------------------
HI, [~mjsax] .
I’ve looked into it and would like to discuss it with you.
# In this PR([https://github.com/apache/kafka/pull/16906)], it was decided to
decommission ProcessorContext and use {{StateStoreContext}} instead of
{{{}ProcessorContext{}}}. As part of this change, the internal field was
changed from {{ProcessorContext}} to {{{}InternalProcessorContext<?, ?>{}}}.
# {{MockProcessorContext.getStateStore()}} returns an anonymous
{{{}StateStoreContext{}}}. However, this anonymous {{StateStoreContext}} is not
related to {{InternalProcessorContext}} (note that {{InternalProcessorContext}}
implements {{{}StateStoreContext{}}}).
# In {{{}InMemoryWindowStore.init(){}}},
{{ProcessorContextUtils.asInternalProcessorContext()}} is called. At this
point, since the anonymous {{StateStoreContext}} provided by
{{MockProcessorContext.getStateStore()}} is not an
{{{}InternalProcessorContext{}}}, an {{IllegalArgumentException}} is thrown.
There is a message saying, “This component requires internal features of Kafka
Streams and must be disabled for unit tests.”{_}.{_}
However, I believe that the {{StateStoreContext}} provided by
{{MockProcessorContext.getStateStore()}} is intended to be used specifically
for testing Kafka Streams internal features.
Therefore, my opinion is that {{MockProcessorContext.getStateStore()}} should
internally return an implementation that also implements
{{InternalProcessorContext rather than StateStoreContext.}}
What do you think?
Please let me know your opinion.
was (Author: JIRAUSER303328):
HI, [~mjsax] .
I’ve looked into it and would like to discuss it with you.
# In this PR, it was decided to decommission ProcessorContext and use
{{StateStoreContext}} instead of {{{}ProcessorContext{}}}. As part of this
change, the internal field was changed from {{ProcessorContext}} to
{{{}InternalProcessorContext<?, ?>{}}}.
# {{MockProcessorContext.getStateStore()}} returns an anonymous
{{{}StateStoreContext{}}}. However, this anonymous {{StateStoreContext}} is not
related to {{InternalProcessorContext}} (note that {{InternalProcessorContext}}
implements {{{}StateStoreContext{}}}).
# In {{{}InMemoryWindowStore.init(){}}},
{{ProcessorContextUtils.asInternalProcessorContext()}} is called. At this
point, since the anonymous {{StateStoreContext}} provided by
{{MockProcessorContext.getStateStore()}} is not an
{{{}InternalProcessorContext{}}}, an {{IllegalArgumentException}} is thrown.
There is a message saying, “This component requires internal features of Kafka
Streams and must be disabled for unit tests.”{_}.{_} However, I believe that
the {{StateStoreContext}} provided by {{MockProcessorContext.getStateStore()}}
is intended to be used specifically for testing Kafka Streams internal features.
Therefore, my opinion is that {{MockProcessorContext.getStateStore()}} should
internally return an implementation that also implements
{{{}InternalProcessorContext{}}}.
What do you think about this analysis?
> MockProcessorContext doesn't work with WindowStores (again)
> -----------------------------------------------------------
>
> Key: KAFKA-19983
> URL: https://issues.apache.org/jira/browse/KAFKA-19983
> Project: Kafka
> Issue Type: Improvement
> Components: streams-test-utils
> Affects Versions: 4.0.0
> Reporter: Matthias J. Sax
> Assignee: sanghyeok An
> Priority: Major
>
> We had a similar issue in the past:
> https://issues.apache.org/jira/browse/KAFKA-10200
> A user reported that that the issue is there again, and looking into the
> code, I could confirm that we re-introduce the issue with 4.0.0 release.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)