diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 3606c570fd834..1472d74ed60ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -32,7 +32,26 @@ public interface ReadOnlyWindowStore { /** * Get all the key-value pairs with the given key and the time range from all * the existing windows. - * + *

+ * The time range is inclusive and applies to the starting timestamp of the window. + * For example, if we have the following windows: + *

+ *

+     * +-------------------------------+
+     * |  key  | start time | end time |
+     * +-------+------------+----------+
+     * |   A   |     10     |    20    |
+     * +-------+------------+----------+
+     * |   A   |     15     |    25    |
+     * +-------+------------+----------+
+     * |   A   |     20     |    30    |
+     * +-------+------------+----------+
+     * |   A   |     25     |    35    |
+     * +--------------------------------
+     * 
+ * And we call {@code store.fetch("A", 10, 20)} then the results will contain the first + * three windows from the table above, i.e., all those where 10 <= start time <= 20. + * * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized */