New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-4730: Streams does not have an in-memory windowed store #6239
Changes from all commits
f95b264
09cea4a
931c20e
ec45c5b
083785d
441ea10
88c315a
7707d99
201833c
eabe346
8284985
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ | |
import org.apache.kafka.common.utils.Time; | ||
import org.apache.kafka.streams.internals.ApiUtils; | ||
import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; | ||
import org.apache.kafka.streams.state.internals.InMemoryWindowBytesStoreSupplier; | ||
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; | ||
import org.apache.kafka.streams.state.internals.MemoryNavigableLRUCache; | ||
import org.apache.kafka.streams.state.internals.RocksDbKeyValueBytesStoreSupplier; | ||
|
@@ -143,6 +144,43 @@ public String metricsScope() { | |
}; | ||
} | ||
|
||
/** | ||
* Create an in-memory {@link WindowBytesStoreSupplier}. | ||
* @param name name of the store (cannot be {@code null}) | ||
* @param retentionPeriod length of time to retain data in the store (cannot be negative) | ||
* Note that the retention period must be at least long enough to contain the | ||
* windowed data's entire life cycle, from window-start through window-end, | ||
* and for the entire grace period. | ||
* @param windowSize size of the windows (cannot be negative) | ||
* @return an instance of {@link WindowBytesStoreSupplier} | ||
* @throws IllegalArgumentException if {@code retentionPeriod} or {@code windowSize} can't be represented as {@code long milliseconds} | ||
*/ | ||
public static WindowBytesStoreSupplier inMemoryWindowStore(final String name, | ||
final Duration retentionPeriod, | ||
final Duration windowSize, | ||
final boolean retainDuplicates) throws IllegalArgumentException { | ||
Objects.requireNonNull(name, "name cannot be null"); | ||
final String rpMsgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); | ||
final long retentionMs = ApiUtils.validateMillisecondDuration(retentionPeriod, rpMsgPrefix); | ||
final String wsMsgPrefix = prepareMillisCheckFailMsgPrefix(windowSize, "windowSize"); | ||
final long windowSizeMs = ApiUtils.validateMillisecondDuration(windowSize, wsMsgPrefix); | ||
|
||
Objects.requireNonNull(name, "name cannot be null"); | ||
if (retentionMs < 0L) { | ||
throw new IllegalArgumentException("retentionPeriod cannot be negative"); | ||
} | ||
if (windowSizeMs < 0L) { | ||
throw new IllegalArgumentException("windowSize cannot be negative"); | ||
} | ||
if (windowSizeMs > retentionMs) { | ||
throw new IllegalArgumentException("The retention period of the window store " | ||
+ name + " must be no smaller than its window size. Got size=[" | ||
+ windowSize + "], retention=[" + retentionPeriod + "]"); | ||
} | ||
|
||
return new InMemoryWindowBytesStoreSupplier(name, retentionMs, windowSizeMs, retainDuplicates); | ||
} | ||
|
||
/** | ||
* Create a persistent {@link WindowBytesStoreSupplier}. | ||
* @param name name of the store (cannot be {@code null}) | ||
|
@@ -166,7 +204,7 @@ public static WindowBytesStoreSupplier persistentWindowStore(final String name, | |
final long windowSize, | ||
final boolean retainDuplicates) { | ||
if (numSegments < 2) { | ||
throw new IllegalArgumentException("numSegments cannot must smaller than 2"); | ||
throw new IllegalArgumentException("numSegments cannot be smaller than 2"); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oof. Thanks. |
||
} | ||
|
||
final long legacySegmentInterval = Math.max(retentionPeriod / (numSegments - 1), 60_000L); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,88 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.streams.state.internals; | ||
|
||
import org.apache.kafka.common.serialization.Serdes; | ||
import org.apache.kafka.common.utils.Bytes; | ||
import org.apache.kafka.streams.state.WindowBytesStoreSupplier; | ||
import org.apache.kafka.streams.state.WindowStore; | ||
|
||
public class InMemoryWindowBytesStoreSupplier implements WindowBytesStoreSupplier { | ||
private final String name; | ||
private final long retentionPeriod; | ||
private final long windowSize; | ||
private final boolean retainDuplicates; | ||
|
||
public InMemoryWindowBytesStoreSupplier(final String name, | ||
final long retentionPeriod, | ||
final long windowSize, | ||
final boolean retainDuplicates) { | ||
this.name = name; | ||
this.retentionPeriod = retentionPeriod; | ||
this.windowSize = windowSize; | ||
this.retainDuplicates = retainDuplicates; | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return name; | ||
} | ||
|
||
@Override | ||
public WindowStore<Bytes, byte[]> get() { | ||
return new InMemoryWindowStore<>(name, | ||
Serdes.Bytes(), | ||
Serdes.ByteArray(), | ||
retentionPeriod, | ||
windowSize, | ||
retainDuplicates, | ||
metricsScope()); | ||
} | ||
|
||
@Override | ||
public String metricsScope() { | ||
return "in-memory-window-state"; | ||
} | ||
|
||
@Deprecated | ||
@Override | ||
public int segments() { | ||
throw new IllegalStateException("Segments is deprecated and should not be called"); | ||
} | ||
|
||
@Override | ||
public long retentionPeriod() { | ||
return retentionPeriod; | ||
} | ||
|
||
|
||
@Override | ||
public long windowSize() { | ||
return windowSize; | ||
} | ||
|
||
// In-memory window store is not *really* segmented, so just say size is 1 ms | ||
@Override | ||
public long segmentIntervalMs() { | ||
return 1; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm.. if users decides to have a caching layer on top of this in-memory window store (which would be a weird usage, but we cannot forbid it) then the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not sure I follow, SegmentedCacheFunction only seems to use this for comparison, why would that be a problem? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think you're right, it is only logical, not physically impacting the data layout. With a |
||
} | ||
|
||
@Override | ||
public boolean retainDuplicates() { | ||
return retainDuplicates; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This link is broken so I updated it to where I think it was intended to point, please check if it makes sense or not
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Another doc change we need to make is in
docs/ops.html
, underwe need to add the store-scope:
<li><code>in-memory-window-state</code></li>
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
And in
docs/streams/upgrade-guide.html
we can add a new sectionand add a one-line paragraph saying that we added a new interface in
Stores
that provides a built-in in-memory window store, etc etc.