Skip to content

KAFKA-3452: Support session windows#2166

Closed
dguy wants to merge 18 commits intoapache:trunkfrom
dguy:kafka-3452-session-merge
Closed

KAFKA-3452: Support session windows#2166
dguy wants to merge 18 commits intoapache:trunkfrom
dguy:kafka-3452-session-merge

Conversation

@dguy
Copy link
Copy Markdown
Contributor

@dguy dguy commented Nov 24, 2016

Add support for SessionWindows based on design detailed in https://cwiki.apache.org/confluence/display/KAFKA/KIP-94+Session+Windows.
This includes refactoring of the RocksDBWindowStore such that functionality common with the RocksDBSessionStore isn't duplicated.

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Nov 24, 2016

@enothereska @mjsax @guozhangwang.

This follows the design as proposed in the KIP.
Some key things:
I've extracted all of the Segment functionality out of RocksDBWindowStore and into Segments. I've introduced a new class SegmentedBytesStore that both the RocksDBWindowStore and RocksDBSessionStore delegate to. They are wrappers on top of the SegmentedBytesStore that know how to deal with the specifics to those stores, i.e., iterators, key layout etc.
The new class SessionWindows doesn't derive from Windows as it doesn't make sense to pass a SessionWindows instance to the non-sesssion-window aggregate methods on KGroupedStream

Thanks

Copy link
Copy Markdown
Contributor

@enothereska enothereska left a comment

Choose a reason for hiding this comment

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

Initial pass on just high level APIs and high-level implementation only.

/**
* Combine values of this stream by key into {@link SessionWindows}
* The resulting {@link KTable} will be materialized in a local state
* store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog"
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.

StoreName is not defined in this API.


/**
* Aggregate values of this stream by key into {@link SessionWindows}.
* The resulting {@link KTable} will be materialized in a local state
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.

Same here with storename.

/**
* Count number of records of this stream by key into {@link SessionWindows}.
* The resulting {@link KTable} will be materialized in a local state
* store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog"
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.

Same here with storename.

* +-----------+-------------+------------+
*
* The previous 2 sessions would be merged into a single session with start time 10 and end time 20.
* The aggregate value for this session would be the result of aggregating all 4 values.
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.

Nice explanation!

import org.apache.kafka.streams.kstream.Windowed;

import java.nio.ByteBuffer;

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.

Consider adding a line or two on what this class is meant for.

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.

This is a general thing we lack for many internal classes... It's hard to get started if it's not clear what a class does and even more important why.

The comment added is actually quite good!

@@ -25,21 +25,24 @@
class TupleForwarder<K, V> {
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.

Can we add a line to describe what this class is for?

8 + // offset
4 + // partition
topic.length();
(topic == null ? 0 : topic.length());
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!

}
entries.add(new ThreadCache.DirtyEntry(key, node.entry.value, node.entry));
node.entry.markClean();
if (node.entry.value == null) {
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.

So if value==null we automatically delete?

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.

Yes - automatically delete from the cache.

* @param aggTwo the second aggregate
* @return the new aggregate value
*/
V apply(K aggKey, V aggOne, V aggTwo);
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.

Do we need the key?

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.

It is consistent with Aggregator - also, why not? Maybe a developer wants to do something with the key.

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.

It's a matter of taste I guess. For example ValueJoiner has no key argument either (there was a discussion about adding the key though...). The API is nor really consistent right now. Should we try to make it consisted adding key parameter everywhere?
\cc @enothereska @miguno @guozhangwang



/**
* A session based window specification uses for aggregating events into sessions.
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.

nit: uses -> used

* then a new session will be created.
*
* For example, If we have a session gap of 5 and the following data arrives:
* +--------------------------------------+
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.

nit: this will not render nicely as compiled JavaDoc. Can you add some markup?

* @return a new SessionWindows with the provided inactivity gap
* and default maintain duration
*/
public static SessionWindows inactivityGap(final long gapMs) {
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.

What about with(final long inactivityGap) instead of inactivityGap(final long gapMs)? This would be closer to TimeWindows#of(long size) ?

final SessionMerger<K, V> sessionMerger = new SessionMerger<K, V>() {
@Override
public V apply(final K aggKey, final V aggOne, final V aggTwo) {
return aggregator.apply(aggKey, aggTwo, aggOne);
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 have to document that the aggregator is used a SessionMerger.

* @see org.apache.kafka.streams.state.Stores#create(String)
*/
public class RocksDBStore<K, V> implements KeyValueStore<K, V> {
class RocksDBStore<K, V> implements KeyValueStore<K, V> {
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 not allow developers to use it?

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.

It is in the internals package and i believe in making internal code as private as possible. If they want to use the RocksDBStore they can already create one via the various suppliers etc.

All that said, though, this could break anyone that has sub-classed this. So i'll change it back.

new Properties()),
t1);

final long t2 = t1 + (sessionGap / 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.

final long t2 = t1 + sessionGap;
to test the corner case

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.

nah - that is what i wanted. There are further tests in KStreamSessionWindowAggregateProcessorTest


}


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.

Could you add a test, that does merge two session into a single one, because a late record arrives (ie late record is in between both session) ?

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.

Is already covered in KStreamSessionWindowAggregateProcessorTest

StringSerializer.class,
new Properties()),
t2);
final long t3 = t1 + sessionGap;
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.

final long t3 = t1 + sessionGap + 1; -- otherwise I would assume that the sessions get merged.

@Override
public void schedule(long interval) {
throw new UnsupportedOperationException("schedule() not supported.");
// throw new UnsupportedOperationException("schedule() not supported.");
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.

???

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.

Good question!

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 12, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/79/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 12, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/78/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 12, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/77/
Test FAILed (JDK 7 and Scala 2.10).

final String storeName);

/**
* Combine values of this stream by the grouped key into {@link SessionWindows}.
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.

Do we need repeated Javadoc everywhere? Not sure what's best practices here. Just there is a lot of repetition and perhaps one aggregate can have the Javadoc and others can point to it?

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.

I know - it is consistent with the rest of the class.

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.

Agree that it is a problem, but if you use an IDE and hoover over a class, it should display it -- and not just a link to another method... I would be awesome if JavaDoc would allow to factor common stuff out for overloaded methods. But I don't know a good way to do it either -- there is only c&p... :(

*
* @return itself
*/
public SessionWindows until(long durationMs) {
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.

Should duration be final?

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 one yes.

* Fetch any session aggregates with the matching key and the sessions end is >= earliestEndTime and the sessions
* start is <= latestStartTime
*/
KeyValueIterator<Windowed<K>, AGG> findSessionsToMerge(K key, long earliestSessionEndTime, long latestSessionStartTime);
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.

Should parameters be final for consistency with your other code?

* where each table contains records with unmodified keys and values
* that represent the latest (rolling) count (i.e., number of records) for each key within that window
*/
KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows, final String storeName);
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.

This probably doesn't belong to this PR, but I wonder why we need count at all, it's just an aggregate.

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 guess, it's THE aggregate everyone wants. Syntactic sugar.

while (iterator.hasNext()) {
final KeyValue<Windowed<K>, T> next = iterator.next();
merged.add(next);
agg = sessionMerger.apply(key, agg, next.value);
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.

This would have probably better been part of the KIP discussion, but there will be some aggregates, like Average for which this kind of merging won't work because they will need more than just the previous aggregate values. How do we catch such cases and warn the user?

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 the same for non SessionWindows based aggregates. I.e, if you aggregate an int into an Average you cant just apply the new int to the currently aggregated Average.

if (!mergedWindow.equals(newTimeWindow)) {
for (final KeyValue<Windowed<K>, T> session : merged) {
store.remove(session.key);
tupleForwarder.maybeForward(session.key, null, session.value);
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.

Don't we want to first invalidate previous values, then send new values, i.e., flip the order of this forwarding with the previous one? I can't immediately find the Javadoc where we tell the user the order.

};
}

static class SessionStoreIterator<K, AGG> implements KeyValueIterator<Windowed<K>, AGG> {
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.

Why is this part of RocksDbSessionStore? Isn't it useful for stores other than RocksDb-based?

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.

It doesn't strictly need to be part of RocksDbSessionStore, but it is only used here as of now. If it is needed elsewhere later, then we should move it later.

class CachingSessionStore<K, AGG> implements SessionStore<K, AGG>, CachedStateStore<Windowed<K>, AGG> {

private final SegmentedBytesStore bytesStore;
private final RocksDBSessionStore.SessionKeySchema keySchema;
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.

If we use a different store, not RocksDb, will this variable still be good?

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.

Yes - it applies to a SegmentedBytesStore - which is not RocksDB specific. The class could be moved out of RocksDBSessionStore, but I don't really see the need to at the moment.

public void apply(final List<ThreadCache.DirtyEntry> entries) {
for (ThreadCache.DirtyEntry entry : entries) {
final Bytes binaryKey = entry.key();
final RecordContext current = context.recordContext();
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.

In previous stores, like CachingKeyValueStore, you put the code below in a separate method called putAndMaybeForward that I thought was a neat way to break the code into chunks.

}

public void close() {
bytesStore.close();
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.

Do we want to flush and close the cache too?

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.

Yeah. The closing of the cache was dependent on another PR that has now been merged. flush doesn't matter so much as we always flush and then close, but i'll add it anyway

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/142/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/143/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/141/
Test PASSed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/145/
Test FAILed (JDK 8 and Scala 2.11).

Objects.requireNonNull(sessionMerger, "sessionMerger can't be null");
Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");

return (KTable<Windowed<K>, T>) doAggregate(
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.

Is the casting needed?

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.

It is consistent with what is already there. Also, i can't think of another way of doing it without duplicating doAggregate, which i'd rather not do

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/144/
Test PASSed (JDK 8 and Scala 2.12).

@enothereska
Copy link
Copy Markdown
Contributor

@dguy I had a look over:

  • APIs
  • main algo for merging sessions
  • session store and IQ read only session store
  • kgroupedstreamimpl
  • basic skimming over new tests (they look great!).

Apart from the above comments I don't have anything else. Thanks.

@asfbot
Copy link
Copy Markdown

asfbot commented Dec 14, 2016

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/143/
Test FAILed (JDK 7 and Scala 2.10).

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Jan 5, 2017

@guozhangwang i've addressed your comments.
w.r.t your suggested refactoring: though i agree the hierarchy could do with some tidying up, i'm not 100% sure what you are suggesting will work and a lot of it is un-related to this PR. I think it needs some more thought and this PR has been outstanding for nearly 2 months already. Also, the changes you are suggesting are internal (i.e., not affecting public APIs), so they could be done once this PR is merged. I think we might already have a JIRA for creating Logged**Stores ? Will have to look.

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/525/
Test FAILed (JDK 7 and Scala 2.10).

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/526/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/527/
Test FAILed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/534/
Test FAILed (JDK 8 and Scala 2.12).

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/535/
Test PASSed (JDK 8 and Scala 2.11).

@asfbot
Copy link
Copy Markdown

asfbot commented Jan 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.10/533/
Test PASSed (JDK 7 and Scala 2.10).

@guozhangwang
Copy link
Copy Markdown
Contributor

I think it needs some more thought and this PR has been outstanding for nearly 2 months already.

I understand... Just afraid such tech debt will keep growing and eventually come back and bite us as unstable and hard-to-debug code. Well how about merging this PR as is and see if we can still push it as a separate PR before the release deadline?

@dguy
Copy link
Copy Markdown
Contributor Author

dguy commented Jan 6, 2017

@guozhangwang - that is fine with me. I agree with the tech-debt and i think we should really focus the next release on bug fixing, tech-debt, usability etc.
If we get this in i'm more than happy to tackle your suggestions above

@guozhangwang
Copy link
Copy Markdown
Contributor

@dguy Cool, will merge it as is. Note that I have a couple follow-up comments in the previous round, but I think it would be easier to address them as a new PR than this 5000+ LOC one.

@asfgit asfgit closed this in e0de3a4 Jan 6, 2017
@dguy dguy deleted the kafka-3452-session-merge branch January 13, 2017 08:39
asfgit pushed a commit that referenced this pull request Jan 17, 2017
This is a follow up of #2166 - refactoring the store hierarchies as requested

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #2360 from dguy/state-store-refactor

(cherry picked from commit 73b7ae0)
Signed-off-by: Guozhang Wang <wangguoz@gmail.com>
asfgit pushed a commit that referenced this pull request Jan 17, 2017
This is a follow up of #2166 - refactoring the store hierarchies as requested

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #2360 from dguy/state-store-refactor
asfgit pushed a commit that referenced this pull request Feb 3, 2017
This is a refactoring follow-up of #2166. Main refactoring changes:

1. Extract `InMemoryKeyValueStore` out of `InMemoryKeyValueStoreSupplier` and remove its duplicates in test package.

2. Add two abstract classes `AbstractKeyValueIterator` and `AbstractKeyValueStore` to collapse common functional logics.

3. Added specialized `BytesXXStore` to accommodate cases where key value types are Bytes / byte[] so that we can save calling the dummy serdes.

4. Make the key type in `ThreadCache` from byte[] to Bytes, as SessionStore / WindowStore's result serialized bytes are in the form of Bytes anyways, so that we can save unnecessary `Bytes.get()` and `Bytes.wrap(bytes)`.

Each of these should arguably be a separate PR and I apologize for the mess, this is because this branch was extracted from a rather large diff that has multiple refactoring mingled together and dguy and myself have already put lots of efforts to break it down to a few separate PRs, and this is the only left-over work. Such PR won't happen in the future.

Ping dguy enothereska mjsax for reviews

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy, Matthias J. Sax, Jun Rao

Closes #2333 from guozhangwang/K3452-followup-state-store-refactor
soenkeliebau pushed a commit to soenkeliebau/kafka that referenced this pull request Feb 7, 2017
Add support for SessionWindows based on design detailed in https://cwiki.apache.org/confluence/display/KAFKA/KIP-94+Session+Windows.
This includes refactoring of the RocksDBWindowStore such that functionality common with the RocksDBSessionStore isn't duplicated.

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Eno Thereska <eno.thereska@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>

Closes apache#2166 from dguy/kafka-3452-session-merge
soenkeliebau pushed a commit to soenkeliebau/kafka that referenced this pull request Feb 7, 2017
This is a follow up of apache#2166 - refactoring the store hierarchies as requested

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes apache#2360 from dguy/state-store-refactor
soenkeliebau pushed a commit to soenkeliebau/kafka that referenced this pull request Feb 7, 2017
This is a refactoring follow-up of apache#2166. Main refactoring changes:

1. Extract `InMemoryKeyValueStore` out of `InMemoryKeyValueStoreSupplier` and remove its duplicates in test package.

2. Add two abstract classes `AbstractKeyValueIterator` and `AbstractKeyValueStore` to collapse common functional logics.

3. Added specialized `BytesXXStore` to accommodate cases where key value types are Bytes / byte[] so that we can save calling the dummy serdes.

4. Make the key type in `ThreadCache` from byte[] to Bytes, as SessionStore / WindowStore's result serialized bytes are in the form of Bytes anyways, so that we can save unnecessary `Bytes.get()` and `Bytes.wrap(bytes)`.

Each of these should arguably be a separate PR and I apologize for the mess, this is because this branch was extracted from a rather large diff that has multiple refactoring mingled together and dguy and myself have already put lots of efforts to break it down to a few separate PRs, and this is the only left-over work. Such PR won't happen in the future.

Ping dguy enothereska mjsax for reviews

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy, Matthias J. Sax, Jun Rao

Closes apache#2333 from guozhangwang/K3452-followup-state-store-refactor
@MarcoAbi
Copy link
Copy Markdown

Hello,

i'm having an issue trying to use SessionWindows

This exception is thrown:

Exception in thread "StreamThread-1" org.apache.kafka.streams.errors.StreamsException: stream-thread [StreamThread-1] Failed to rebalance at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:612) at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:368) Caused by: java.lang.IndexOutOfBoundsException at java.nio.Buffer.checkIndex(Buffer.java:546) at java.nio.HeapByteBuffer.getLong(HeapByteBuffer.java:416) at org.apache.kafka.streams.kstream.internals.SessionKeySerde.extractEnd(SessionKeySerde.java:117) at org.apache.kafka.streams.state.internals.SessionKeySchema.segmentTimestamp(SessionKeySchema.java:45) at org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.put(RocksDBSegmentedBytesStore.java:71) at org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore$1.restore(RocksDBSegmentedBytesStore.java:104) at org.apache.kafka.streams.processor.internals.ProcessorStateManager.restoreActiveState(ProcessorStateManager.java:230) at org.apache.kafka.streams.processor.internals.ProcessorStateManager.register(ProcessorStateManager.java:193) at org.apache.kafka.streams.processor.internals.AbstractProcessorContext.register(AbstractProcessorContext.java:99) at org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.init(RocksDBSegmentedBytesStore.java:101) at org.apache.kafka.streams.state.internals.ChangeLoggingSegmentedBytesStore.init(ChangeLoggingSegmentedBytesStore.java:68) at org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore.init(MeteredSegmentedBytesStore.java:66) at org.apache.kafka.streams.state.internals.RocksDBSessionStore.init(RocksDBSessionStore.java:78) at org.apache.kafka.streams.state.internals.CachingSessionStore.init(CachingSessionStore.java:97) at org.apache.kafka.streams.processor.internals.AbstractTask.initializeStateStores(AbstractTask.java:86) at org.apache.kafka.streams.processor.internals.StreamTask.<init>(StreamTask.java:141) at org.apache.kafka.streams.processor.internals.StreamThread.createStreamTask(StreamThread.java:834) at org.apache.kafka.streams.processor.internals.StreamThread$TaskCreator.createTask(StreamThread.java:1207) at org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:1180) at org.apache.kafka.streams.processor.internals.StreamThread.addStreamTasks(StreamThread.java:937) at org.apache.kafka.streams.processor.internals.StreamThread.access$500(StreamThread.java:69) at org.apache.kafka.streams.processor.internals.StreamThread$1.onPartitionsAssigned(StreamThread.java:236) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:255) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:339) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:303) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:286) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1030) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:995) at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:582) ... 1 more

Can you please help me with this?

Thanks.
Marco

@mjsax
Copy link
Copy Markdown
Member

mjsax commented Feb 21, 2017

@MarcoAbi Could you please report this at the Kafka mailing list: http://kafka.apache.org/contact

Thanks.

valeraBr pushed a commit to valeraBr/kafka that referenced this pull request May 2, 2024
This is a follow up of apache/kafka#2166 - refactoring the store hierarchies as requested

Author: Damian Guy <damian.guy@gmail.com>

Reviewers: Guozhang Wang <wangguoz@gmail.com>

Closes #2360 from dguy/state-store-refactor
valeraBr pushed a commit to valeraBr/kafka that referenced this pull request May 2, 2024
This is a refactoring follow-up of apache/kafka#2166. Main refactoring changes:

1. Extract `InMemoryKeyValueStore` out of `InMemoryKeyValueStoreSupplier` and remove its duplicates in test package.

2. Add two abstract classes `AbstractKeyValueIterator` and `AbstractKeyValueStore` to collapse common functional logics.

3. Added specialized `BytesXXStore` to accommodate cases where key value types are Bytes / byte[] so that we can save calling the dummy serdes.

4. Make the key type in `ThreadCache` from byte[] to Bytes, as SessionStore / WindowStore's result serialized bytes are in the form of Bytes anyways, so that we can save unnecessary `Bytes.get()` and `Bytes.wrap(bytes)`.

Each of these should arguably be a separate PR and I apologize for the mess, this is because this branch was extracted from a rather large diff that has multiple refactoring mingled together and dguy and myself have already put lots of efforts to break it down to a few separate PRs, and this is the only left-over work. Such PR won't happen in the future.

Ping dguy enothereska mjsax for reviews

Author: Guozhang Wang <wangguoz@gmail.com>

Reviewers: Damian Guy, Matthias J. Sax, Jun Rao

Closes #2333 from guozhangwang/K3452-followup-state-store-refactor
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants