You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/12/21 22:32:05 UTC

[GitHub] [kafka] vvcephei commented on a change in pull request #11567: WindowKeyQuery and WindowRangeQuery

vvcephei commented on a change in pull request #11567:
URL: https://github.com/apache/kafka/pull/11567#discussion_r773472551



##########
File path: streams/src/main/java/org/apache/kafka/streams/query/WindowKeyQuery.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.query;
+
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.time.Instant;
+import java.util.Optional;
+
+public class WindowKeyQuery<K, V> implements Query<WindowStoreIterator<V>> {

Review comment:
       ```suggestion
   @Evolving
   public class WindowKeyQuery<K, V> implements Query<WindowStoreIterator<V>> {
   ```

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java
##########
@@ -497,6 +530,23 @@ public void verifyStore() {
                     shouldHandleRangeQueries(valueExtractor);
                 }
             }
+
+            if (storeToTest.isWindowed()){

Review comment:
       Looks like we still need the session store test, right?

##########
File path: streams/src/main/java/org/apache/kafka/streams/query/WindowKeyQuery.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.query;
+
+import org.apache.kafka.streams.state.WindowStoreIterator;
+
+import java.time.Instant;
+import java.util.Optional;
+
+public class WindowKeyQuery<K, V> implements Query<WindowStoreIterator<V>> {
+    private final K key;
+    private final Optional<Instant> timeFrom;
+    private final Optional<Instant> timeTo;
+
+    private WindowKeyQuery(final K key, final Optional<Instant> timeTo, final Optional<Instant> timeFrom) {
+        this.key = key;
+        this.timeFrom = timeFrom;
+        this.timeTo = timeTo;
+    }
+
+    public static <K, V> WindowKeyQuery<K, V> withKeyAndWindowBounds(final K key, final Instant timeFrom, final Instant timeTo) {

Review comment:
       ```suggestion
       public static <K, V> WindowKeyQuery<K, V> withKeyAndWindowStartRange(final K key, final Instant timeFrom, final Instant timeTo) {
   ```

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java
##########
@@ -244,13 +249,23 @@ public boolean keyValue() {
                 return Stores.inMemoryWindowStore(STORE_NAME, Duration.ofDays(1), WINDOW_SIZE,
                     false);
             }
+
+            @Override
+            public boolean isWindowed() {
+                return true;
+            }
         },
         ROCKS_WINDOW {
             @Override
             public StoreSupplier<?> supplier() {
                 return Stores.persistentWindowStore(STORE_NAME, Duration.ofDays(1), WINDOW_SIZE,
                     false);
             }
+
+            @Override
+            public boolean isWindowed(){
+                return true;
+            }
         },
         TIME_ROCKS_WINDOW {

Review comment:
       I think you missed marking this one as "windowed"

##########
File path: streams/src/main/java/org/apache/kafka/streams/query/WindowRangeQuery.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.query;
+
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.state.KeyValueIterator;
+
+import java.time.Instant;
+import java.util.Optional;
+
+public class WindowRangeQuery<K, V> implements Query<KeyValueIterator<Windowed<K>, V>> {
+
+    private final Optional<K> key;
+    private final Optional<Instant> timeFrom;
+    private final Optional<Instant> timeTo;
+
+    private WindowRangeQuery(final Optional<K> key, final Optional<Instant> timeFrom, final Optional<Instant> timeTo) {
+        this.key = key;
+        this.timeFrom = timeFrom;
+        this.timeTo = timeTo;
+    }
+
+    public static <K, V> WindowRangeQuery<K, V> withKey(final K key) {
+        return new WindowRangeQuery<>(Optional.of(key), Optional.empty(), Optional.empty());
+    }
+
+    public static <K, V> WindowRangeQuery<K, V> withWindowStartRange(final Instant timeFrom, final Instant timeTo) {
+        return new WindowRangeQuery<>(Optional.empty(), Optional.of(timeFrom), Optional.of(timeTo));
+    }
+
+    public Optional<K> getKey() {

Review comment:
       This isn't in the KIP, but I think it was just a minor oversight. Can you edit the KIP to add it?

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
##########
@@ -350,9 +353,35 @@ public boolean isOpen() {
         return open;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound,
         final boolean collectExecutionInfo) {
+
+        if (query instanceof WindowKeyQuery) {
+            final WindowKeyQuery<Bytes, byte[]> windowKeyQuery = (WindowKeyQuery<Bytes, byte[]>) query;
+            if (windowKeyQuery.getTimeFrom().isPresent() && windowKeyQuery.getTimeTo().isPresent()) {
+                final Bytes key = windowKeyQuery.getKey();
+                final Instant lower = windowKeyQuery.getTimeFrom().get();
+                final Instant upper = windowKeyQuery.getTimeTo().get();
+                final WindowStoreIterator<byte[]> iterator = this.fetch(key, lower, upper);
+                final R result = (R) iterator;
+                final QueryResult<R> queryResult = QueryResult.forResult(result);
+                return queryResult;
+            }
+        } else if (query instanceof WindowRangeQuery) {
+            final WindowRangeQuery<Bytes, byte[]> windowRangeQuery = (WindowRangeQuery<Bytes, byte[]>) query;
+            if (windowRangeQuery.getTimeFrom().isPresent() &&
+                    windowRangeQuery.getTimeTo().isPresent()) {

Review comment:
       It looks like this method will return "unknown query" if the window bounds aren't specified. Can we make sure there's a test case for `WindowRangeQuery#withKey`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java
##########
@@ -317,14 +318,31 @@ public boolean isOpen() {
     @Override
     public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound,
         final boolean collectExecutionInfo) {
+
+        if (query instanceof WindowRangeQuery) {
+            @SuppressWarnings("unchecked") final WindowRangeQuery<Bytes, byte[]> windowRangeQuery = (WindowRangeQuery<Bytes, byte[]>) query;
+            if (windowRangeQuery.getKey().isPresent()) {
+                final Bytes key = windowRangeQuery.getKey().get();
+                final KeyValueIterator<Windowed<Bytes>, byte[]> keyValueIterator = this.fetch(key);
+                @SuppressWarnings("unchecked") final R result = (R) keyValueIterator;
+                final QueryResult<R> queryResult = QueryResult.forResult(result);
+                return queryResult;
+            }
+        }
+
+        return null;

Review comment:
       We shouldn't return `null`, but instead return a "unknown query" result.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java
##########
@@ -532,6 +582,48 @@ public void verifyStore() {
         );
     }
 
+    private <T> void shouldHandleWindowKeyQueries(final Integer key, final Instant timeFrom, final Instant timeTo, final Function<T, Integer> extractor) {
+        shouldHandleWindowKeyQuery(
+                key,
+                timeFrom,
+                timeTo,
+                extractor,
+                mkSet(1, 2, 3)
+
+        );
+    }
+
+    private <T> void shouldHandleWindowRangeQueries(final Function<T, Integer> extractor) {
+        shouldHandleRangeQuery(

Review comment:
       Are these calling the right method?
   
   ```suggestion
           shouldHandleWindowRangeQuery(
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java
##########
@@ -317,14 +318,31 @@ public boolean isOpen() {
     @Override
     public <R> QueryResult<R> query(final Query<R> query, final PositionBound positionBound,
         final boolean collectExecutionInfo) {
+
+        if (query instanceof WindowRangeQuery) {
+            @SuppressWarnings("unchecked") final WindowRangeQuery<Bytes, byte[]> windowRangeQuery = (WindowRangeQuery<Bytes, byte[]>) query;
+            if (windowRangeQuery.getKey().isPresent()) {
+                final Bytes key = windowRangeQuery.getKey().get();
+                final KeyValueIterator<Windowed<Bytes>, byte[]> keyValueIterator = this.fetch(key);
+                @SuppressWarnings("unchecked") final R result = (R) keyValueIterator;
+                final QueryResult<R> queryResult = QueryResult.forResult(result);
+                return queryResult;
+            }
+        }
+
+        return null;
+
+        /*
         return StoreQueryUtils.handleBasicQueries(
-            query,
-            positionBound,
-            collectExecutionInfo,
-            this,
-            position,
-            context.taskId().partition()
+                query,
+                positionBound,
+                collectExecutionInfo,
+                this,
+                position,
+                context.taskId().partition()
         );
+
+         */

Review comment:
       I assume that the commented code is just a temporary state while you're finalizing the PR.
   
   Since executing these queries doesn't require any specialization to the concrete store type, just the WindowStore and SessionStore interfaces, I think you might want to move the logic into StoreQueryUtils, but I've been thinking that, rather than having a universal method to `handleBasicQueries` on any state store, the code will be cleaner with less type-checks and casts if you instead add a new `handleWindowQueries` and `handleSessionQueries`.
   
   What do you think?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org