Skip to content

KAFKA-6455: Extend CacheFlushListener to forward timestamp#6147

Merged
mjsax merged 3 commits intoapache:trunkfrom
mjsax:minor-extend-flushlistener
Jan 19, 2019
Merged

KAFKA-6455: Extend CacheFlushListener to forward timestamp#6147
mjsax merged 3 commits intoapache:trunkfrom
mjsax:minor-extend-flushlistener

Conversation

@mjsax
Copy link
Copy Markdown
Member

@mjsax mjsax commented Jan 15, 2019

This PR is of part of the overall KIP-258 story. It does not resolve KAFKA-6455, but prepares it. It's only internal changes so we can merge right away.

@mjsax mjsax added the streams label Jan 15, 2019
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 15, 2019

Call for review @guozhangwang @bbejeck @vvcephei

Copy link
Copy Markdown
Member Author

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

Marked the relevant changes.

All others are to keep the code working :)

* @param timestamp timestamp of new value
*/
void apply(final K key, final V newValue, final V oldValue);
void apply(final K key, final V newValue, final V oldValue, final long timestamp);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

This is the actual change.

package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.streams.state.internals.ThreadCache;
package org.apache.kafka.streams.state.internals;
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Minor fix: The interface should be in state package, not kstream.

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.

Good catch!

serdes.keyFrom(entry.key().get()),
serdes.valueFrom(entry.newValue()),
oldValue,
entry.entry().context().timestamp());
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

part of the fix: we now also forward the timestamp on eviction

key,
newValue,
oldValue,
entry.entry().context().timestamp());
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

part of the fix: we now also forward the timestamp on eviction

windowedKey,
serdes.valueFrom(entry.newValue()),
oldValue,
entry.entry().context().timestamp());
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

part of the fix: we now also forward the timestamp on eviction

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 16, 2019

Retest this please.

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 for PR @mjsax, left a couple of comments.

}
forward(child, key, value);
} else {
if (children.size() == 1) {
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.

I think this if/else block here can be removed and just go with

for (final ProcessorNode child : children) {
         forward(child, key, value);
 }

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I copied this from ProcessorContextImpl#forward() -- we added it there are "optimization" to avoid creating the iterator -- might be pre-mature optimization to be fair.

I don't feel strong about it, however, I would prefer to keep both classed "in sync". Thus, if we remove it here, I think we should also remove it from there?

Throughts?

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.

Yeah if it exists elsewhere let's just leave it as is for now.

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.

Let's remove on both side: I think in J8 it is not really a big difference.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Ok. Will simplify this.

for (final ProcessorNode child : (List<ProcessorNode<K, V>>) currentNode().children()) {
setCurrentNode(child);
child.process(key, value);
final List<ProcessorNode<K, V>> children = (List<ProcessorNode<K, V>>) currentNode().children();
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.

We should add a test or tests for this section

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Thinking about this once more, can a "global processor" have multiple children to begin with? Maybe we can simplify this method significantly? Thoughts?

Will add a test for GlobalProcessorContextImpl though -- there is not at all atm.

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.

Thinking about this once more, can a "global processor" have multiple children to begin with?

Yeah, you could be correct about, I don't think it can.

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.

+1 on assuming a single children, check-and-throw-otherwise

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 17, 2019

Updated this.

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 17, 2019

Retest this please.

Copy link
Copy Markdown
Contributor

@guozhangwang guozhangwang left a comment

Choose a reason for hiding this comment

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

Pleas feel free to merge after comments addressed.

for (final ProcessorNode child : (List<ProcessorNode<K, V>>) currentNode().children()) {
setCurrentNode(child);
child.process(key, value);
final List<ProcessorNode<K, V>> children = (List<ProcessorNode<K, V>>) currentNode().children();
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.

+1 on assuming a single children, check-and-throw-otherwise

}
forward(child, key, value);
} else {
if (children.size() == 1) {
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.

Let's remove on both side: I think in J8 it is not really a big difference.

package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.streams.state.internals.ThreadCache;
package org.apache.kafka.streams.state.internals;
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.

Good catch!

import org.junit.Before;
import org.junit.Test;

import java.util.ArrayList;
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.

Shouldn't we add new test case similar to the ones in GlobalProcessorContextImplTest.java in this class as well?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Why? ProcessorContextImpl was not changed.

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.

Ack, you right.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@bbejeck @guozhangwang I was thinking about this once more, and for global tasks, we know that there are exactly two processors: the source processor and the "state maintainer" processor. The first one has exactly one child and it calls forward() from above. The second processor only call this forward overload on flush and should have zero children.

Does this make sense?

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.

There is a related JIRA about that but whether we'd keep it as is still open questions, I think we can make this assumption still atm but just bring it up FYI.

https://issues.apache.org/jira/browse/KAFKA-7125

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

The changes in this class are just a code simplification as discussed with @bbejeck and @guozhangwang -- no change in behavior.

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 18, 2019

Updated this.

@mjsax mjsax force-pushed the minor-extend-flushlistener branch from 8e74612 to 30b8447 Compare January 18, 2019 03:06
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 18, 2019

Retest this please.

\cc For review @ableegoldman

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 18, 2019

Test failure in core. Retest this please.

@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 19, 2019

Retest this please.

@mjsax mjsax merged commit e19eb3e into apache:trunk Jan 19, 2019
@mjsax mjsax deleted the minor-extend-flushlistener branch January 19, 2019 17:40
abbccdda pushed a commit to abbccdda/kafka that referenced this pull request Jan 24, 2019
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* ak/trunk:
  MINOR: fix race condition in KafkaStreamsTest (apache#6185)
  KAFKA-4850: Enable bloomfilters (apache#6012)
  MINOR: ducker-ak: add down -f, avoid using a terminal in ducker test
  KAFKA-5117: Stop resolving externalized configs in Connect REST API
  MINOR: Cleanup handling of mixed transactional/idempotent records (apache#6172)
  KAFKA-7844: Use regular subproject for generator to fix *All targets (apache#6182)
  Fix Documentation for cleanup.policy is out of date (apache#6181)
  MINOR: increase timeouts for KafkaStreamsTest (apache#6178)
  MINOR: Rejoin split ssl principal mapping rules (apache#6099)
  MINOR: Handle case where connector status endpoints returns 404 (apache#6176)
  MINOR: Remove unused imports, exceptions, and values (apache#6117)
  KAFKA-3522: Add internal RecordConverter interface (apache#6150)
  Fix Javadoc of KafkaConsumer (apache#6155)
  KAFKA-6455: Extend CacheFlushListener to forward timestamp (apache#6147)
  MINOR: Log partition info when creating new request batch in controller (apache#6145)
  KAFKA-7652: Part I; Fix SessionStore's findSession(single-key) (apache#6134)
  MINOR: Remove the InvalidTopicException handling in InternalTopicManager (apache#6167)
  [KAFKA-7024] Rocksdb state directory should be created before opening the DB (apache#6138)
  MINOR:: Fix typos (apache#6079)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
@mjsax mjsax added the kip Requires or implements a KIP label Jun 12, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

kip Requires or implements a KIP streams

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants