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 2020/09/16 18:17:05 UTC

[GitHub] [kafka] vvcephei opened a new pull request #9293: POC: Allow bypassing the cache layer in IQ

vvcephei opened a new pull request #9293:
URL: https://github.com/apache/kafka/pull/9293


   Just a quick POC of how we can optionally bypass the caching layer in IQ. Some queries that are common in IQ but rare in the PAPI perform really poorly with the caching layer involved. We should have a comprehensive solution to this in the future, but for now it seems worth considering just giving a way to bypass it.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] guozhangwang commented on a change in pull request #9293: POC: Allow bypassing the cache layer in IQ

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on a change in pull request #9293:
URL: https://github.com/apache/kafka/pull/9293#discussion_r728524198



##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -46,43 +46,90 @@
  * @param <K>
  * @param <V>
  */
-public class MeteredKeyValueStore<K, V>
-    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
+public class MeteredKeyValueStore<K, V, OwnType extends MeteredKeyValueStore<K, V, OwnType>>

Review comment:
       I had a slight (? well, maybe not so slight) different idea, which assumes that we only want to have the control at the whole app level, rather than a per-store level at IQ APIs only: suppose we just have, e.g. a config on Streams that enables bypass or not for IQ, then upon calling the get/range/etc APIs at the cached layer:
   
   1) Whether the current thread is a stream thread, if yes, never bypass the cache.
   2) If not, depend on the config to decide whether bypass the cache.
   
   Doing so we then would not need the unwrap/rewrap and the template types as well. The question then goes back to: whether we would need bypassing at the per-store or global level? Personally I think it's not necessary for a finer-grained control, mainly because that in the long run with the new IQ API and the caller likely to have a totally different way to handle consistency / staleness, i.e. we would deprecate `staleStore` per store or global configs in the long run anyways, and the API calls itself is associated with a token to indicate whether allow staleness or bypass cache.




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



[GitHub] [kafka] vvcephei commented on a change in pull request #9293: POC: Allow bypassing the cache layer in IQ

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9293:
URL: https://github.com/apache/kafka/pull/9293#discussion_r489632127



##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -46,43 +46,90 @@
  * @param <K>
  * @param <V>
  */
-public class MeteredKeyValueStore<K, V>
-    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
+public class MeteredKeyValueStore<K, V, OwnType extends MeteredKeyValueStore<K, V, OwnType>>

Review comment:
       The self-type reference is pretty mind-bending, but basically what it's for is to let `reWrap` return the same type you call it on. So, `MeteredKeyValueStore#reWrap` returns a `MeteredKeyValueStore` with the same generic types, and `MeteredTimestampedKeyValueStore#reWrap` returns a `MeteredTimestampedKeyValueStore` with the same generic parameters.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -46,43 +46,90 @@
  * @param <K>
  * @param <V>
  */
-public class MeteredKeyValueStore<K, V>
-    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
+public class MeteredKeyValueStore<K, V, OwnType extends MeteredKeyValueStore<K, V, OwnType>>
+    extends MeteredStore<K, V, MeteredKeyValueStore<K, V, OwnType>, KeyValueStore<Bytes, byte[]>>
     implements KeyValueStore<K, V> {
 
     final Serde<K> keySerde;
     final Serde<V> valueSerde;
     StateSerdes<K, V> serdes;
 
-    private final String metricsScope;
+    protected final String metricsScope;
     protected final Time time;
     protected Sensor putSensor;
-    private Sensor putIfAbsentSensor;
+    protected Sensor putIfAbsentSensor;
     protected Sensor getSensor;
-    private Sensor deleteSensor;
-    private Sensor putAllSensor;
-    private Sensor allSensor;
-    private Sensor rangeSensor;
-    private Sensor flushSensor;
-    private Sensor e2eLatencySensor;
-    private ProcessorContext context;
-    private StreamsMetricsImpl streamsMetrics;
-    private final String threadId;
-    private String taskId;
+    protected Sensor deleteSensor;
+    protected Sensor putAllSensor;
+    protected Sensor allSensor;
+    protected Sensor rangeSensor;
+    protected Sensor flushSensor;
+    protected Sensor e2eLatencySensor;
+    protected ProcessorContext context;
+    protected StreamsMetricsImpl streamsMetrics;
+    protected final String threadId;
+    protected String taskId;
 
     MeteredKeyValueStore(final KeyValueStore<Bytes, byte[]> inner,
                          final String metricsScope,
                          final Time time,
                          final Serde<K> keySerde,
                          final Serde<V> valueSerde) {
+        this(
+            inner,
+            metricsScope,
+            Thread.currentThread().getName(),
+            time != null ? time : Time.SYSTEM,
+            keySerde,
+            valueSerde
+        );
+    }
+
+    protected MeteredKeyValueStore(final KeyValueStore<Bytes, byte[]> inner,
+                         final String metricsScope,
+                         final String threadId,
+                         final Time time,
+                         final Serde<K> keySerde,
+                         final Serde<V> valueSerde) {
         super(inner);
         this.metricsScope = metricsScope;
-        threadId = Thread.currentThread().getName();
-        this.time = time != null ? time : Time.SYSTEM;
+        this.threadId = threadId;
+        this.time = time;
         this.keySerde = keySerde;
         this.valueSerde = valueSerde;
     }
 
+    public MeteredKeyValueStore<K, V, OwnType> reWrap(final KeyValueStore<Bytes, byte[]> inner) {
+        final MeteredKeyValueStore<K, V, OwnType> reWrapped = new MeteredKeyValueStore<>(
+            inner,
+            metricsScope,
+            threadId,
+            time,
+            keySerde,
+            valueSerde
+        );
+
+        copyInit(reWrapped);
+
+        return reWrapped;
+    }
+
+    protected void copyInit(final MeteredKeyValueStore<K, V, OwnType> reWrapped) {

Review comment:
       We're basically cloning an already initialized Metered store, so we need to copy over everything that gets set in init.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
##########
@@ -84,26 +96,53 @@ public StreamThreadStateStoreProvider(final StreamThread streamThread,
     }
 
     @SuppressWarnings("unchecked")
-    private <T> T validateAndListStores(final StateStore store, final QueryableStoreType<T> queryableStoreType, final String storeName, final TaskId taskId) {
+    private static <T> T validateAndListStores(final StateStore store,
+                                               final QueryableStoreType<T> queryableStoreType,
+                                               final String storeName,
+                                               final TaskId taskId,
+                                               final boolean bypassCache) {
         if (store != null && queryableStoreType.accepts(store)) {
             if (!store.isOpen()) {
                 throw new InvalidStateStoreException(
                         "Cannot get state store " + storeName + " for task " + taskId +
                             " because the store is not open. " +
                             "The state store may have migrated to another instances.");
             }
+            final StateStore maybeUncached = maybeBypassCache(store, bypassCache);
             if (store instanceof TimestampedKeyValueStore && queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) {
-                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) store);
+                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) maybeUncached);
             } else if (store instanceof TimestampedWindowStore && queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) {
-                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) store);
+                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) maybeUncached);
             } else {
-                return (T) store;
+                return (T) maybeUncached;
             }
         } else {
             return null;
         }
     }
 
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    private static StateStore maybeBypassCache(final StateStore store, final boolean bypassCache) {

Review comment:
       Here's where we actually bypass the cache. I've just realized that `reWrap` should be in `WrappedStateStore`, in which case, this logic should change a little, but the basic idea is there.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
##########
@@ -84,26 +96,53 @@ public StreamThreadStateStoreProvider(final StreamThread streamThread,
     }
 
     @SuppressWarnings("unchecked")
-    private <T> T validateAndListStores(final StateStore store, final QueryableStoreType<T> queryableStoreType, final String storeName, final TaskId taskId) {
+    private static <T> T validateAndListStores(final StateStore store,
+                                               final QueryableStoreType<T> queryableStoreType,
+                                               final String storeName,
+                                               final TaskId taskId,
+                                               final boolean bypassCache) {
         if (store != null && queryableStoreType.accepts(store)) {
             if (!store.isOpen()) {
                 throw new InvalidStateStoreException(
                         "Cannot get state store " + storeName + " for task " + taskId +
                             " because the store is not open. " +
                             "The state store may have migrated to another instances.");
             }
+            final StateStore maybeUncached = maybeBypassCache(store, bypassCache);
             if (store instanceof TimestampedKeyValueStore && queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) {
-                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) store);
+                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) maybeUncached);
             } else if (store instanceof TimestampedWindowStore && queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) {
-                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) store);
+                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) maybeUncached);
             } else {
-                return (T) store;
+                return (T) maybeUncached;
             }
         } else {
             return null;
         }
     }
 
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    private static StateStore maybeBypassCache(final StateStore store, final boolean bypassCache) {
+        if (!bypassCache) {
+            return store;
+        } else if (store instanceof MeteredStore) {
+            final MeteredStore meteredStore = (MeteredStore) store;
+            final StateStore wrapped = meteredStore.wrapped();
+            final StateStore uncached = maybeBypassCache(wrapped, bypassCache);
+            if (uncached == wrapped) {
+                return store;
+            } else {
+                return meteredStore.reWrap(uncached);

Review comment:
       And this is where, if we did actually strip out the caching layer, we restore the layers "above" it.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -46,43 +46,90 @@
  * @param <K>
  * @param <V>
  */
-public class MeteredKeyValueStore<K, V>
-    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
+public class MeteredKeyValueStore<K, V, OwnType extends MeteredKeyValueStore<K, V, OwnType>>
+    extends MeteredStore<K, V, MeteredKeyValueStore<K, V, OwnType>, KeyValueStore<Bytes, byte[]>>
     implements KeyValueStore<K, V> {
 
     final Serde<K> keySerde;
     final Serde<V> valueSerde;
     StateSerdes<K, V> serdes;
 
-    private final String metricsScope;
+    protected final String metricsScope;

Review comment:
       need to be protected so that we can set them in `copyInit`

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java
##########
@@ -50,6 +50,31 @@
         super(inner, metricScope, time, keySerde, valueSerde);
     }
 
+    private MeteredTimestampedKeyValueStore(final KeyValueStore<Bytes, byte[]> inner,
+                                            final String metricsScope,
+                                            final String threadId,
+                                            final Time time,
+                                            final Serde<K> keySerde,
+                                            final Serde<ValueAndTimestamp<V>> valueSerde) {
+        super(inner, metricsScope, threadId, time, keySerde, valueSerde);
+    }
+
+    @Override
+    public MeteredTimestampedKeyValueStore<K, V> reWrap(final KeyValueStore<Bytes, byte[]> inner) {

Review comment:
       Overriding this here lets us preserve the correct type of the store when re-wrapping.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredStore.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.Serde;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.state.KeyValueStore;
+
+/**
+ * A Metered {@link KeyValueStore} wrapper that is used for recording operation metrics, and hence its
+ * inner KeyValueStore implementation do not need to provide its own metrics collecting functionality.
+ * The inner {@link KeyValueStore} of this class is of type &lt;Bytes,byte[]&gt;, hence we use {@link Serde}s
+ * to convert from &lt;K,V&gt; to &lt;Bytes,byte[]&gt;
+ *
+ * @param <K>
+ * @param <V>
+ */
+public abstract class MeteredStore<K, V, OwnType extends MeteredStore<K, V, OwnType, InnerType>, InnerType extends StateStore>
+    extends WrappedStateStore<InnerType, K, V> {
+
+    public MeteredStore(final InnerType wrapped) {
+        super(wrapped);
+    }
+
+    abstract OwnType reWrap(final InnerType inner);

Review comment:
       In retrospect, this actually belongs in `WrappedStateStore`. I'd fix it if we choose to go this way.

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java
##########
@@ -46,43 +46,90 @@
  * @param <K>
  * @param <V>
  */
-public class MeteredKeyValueStore<K, V>
-    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, K, V>
+public class MeteredKeyValueStore<K, V, OwnType extends MeteredKeyValueStore<K, V, OwnType>>
+    extends MeteredStore<K, V, MeteredKeyValueStore<K, V, OwnType>, KeyValueStore<Bytes, byte[]>>
     implements KeyValueStore<K, V> {
 
     final Serde<K> keySerde;
     final Serde<V> valueSerde;
     StateSerdes<K, V> serdes;
 
-    private final String metricsScope;
+    protected final String metricsScope;
     protected final Time time;
     protected Sensor putSensor;
-    private Sensor putIfAbsentSensor;
+    protected Sensor putIfAbsentSensor;
     protected Sensor getSensor;
-    private Sensor deleteSensor;
-    private Sensor putAllSensor;
-    private Sensor allSensor;
-    private Sensor rangeSensor;
-    private Sensor flushSensor;
-    private Sensor e2eLatencySensor;
-    private ProcessorContext context;
-    private StreamsMetricsImpl streamsMetrics;
-    private final String threadId;
-    private String taskId;
+    protected Sensor deleteSensor;
+    protected Sensor putAllSensor;
+    protected Sensor allSensor;
+    protected Sensor rangeSensor;
+    protected Sensor flushSensor;
+    protected Sensor e2eLatencySensor;
+    protected ProcessorContext context;
+    protected StreamsMetricsImpl streamsMetrics;
+    protected final String threadId;
+    protected String taskId;
 
     MeteredKeyValueStore(final KeyValueStore<Bytes, byte[]> inner,
                          final String metricsScope,
                          final Time time,
                          final Serde<K> keySerde,
                          final Serde<V> valueSerde) {
+        this(
+            inner,
+            metricsScope,
+            Thread.currentThread().getName(),
+            time != null ? time : Time.SYSTEM,
+            keySerde,
+            valueSerde
+        );
+    }
+
+    protected MeteredKeyValueStore(final KeyValueStore<Bytes, byte[]> inner,
+                         final String metricsScope,
+                         final String threadId,
+                         final Time time,
+                         final Serde<K> keySerde,
+                         final Serde<V> valueSerde) {
         super(inner);
         this.metricsScope = metricsScope;
-        threadId = Thread.currentThread().getName();
-        this.time = time != null ? time : Time.SYSTEM;
+        this.threadId = threadId;
+        this.time = time;
         this.keySerde = keySerde;
         this.valueSerde = valueSerde;
     }
 
+    public MeteredKeyValueStore<K, V, OwnType> reWrap(final KeyValueStore<Bytes, byte[]> inner) {

Review comment:
       This is the basic way that the cache bypass is implemented. You descend the stack of wrapped stores until you find the cache, then you pop it off, then you re-wrap the remaining store layers. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java
##########
@@ -84,26 +96,53 @@ public StreamThreadStateStoreProvider(final StreamThread streamThread,
     }
 
     @SuppressWarnings("unchecked")
-    private <T> T validateAndListStores(final StateStore store, final QueryableStoreType<T> queryableStoreType, final String storeName, final TaskId taskId) {
+    private static <T> T validateAndListStores(final StateStore store,
+                                               final QueryableStoreType<T> queryableStoreType,
+                                               final String storeName,
+                                               final TaskId taskId,
+                                               final boolean bypassCache) {
         if (store != null && queryableStoreType.accepts(store)) {
             if (!store.isOpen()) {
                 throw new InvalidStateStoreException(
                         "Cannot get state store " + storeName + " for task " + taskId +
                             " because the store is not open. " +
                             "The state store may have migrated to another instances.");
             }
+            final StateStore maybeUncached = maybeBypassCache(store, bypassCache);
             if (store instanceof TimestampedKeyValueStore && queryableStoreType instanceof QueryableStoreTypes.KeyValueStoreType) {
-                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) store);
+                return (T) new ReadOnlyKeyValueStoreFacade<>((TimestampedKeyValueStore<Object, Object>) maybeUncached);
             } else if (store instanceof TimestampedWindowStore && queryableStoreType instanceof QueryableStoreTypes.WindowStoreType) {
-                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) store);
+                return (T) new ReadOnlyWindowStoreFacade<>((TimestampedWindowStore<Object, Object>) maybeUncached);
             } else {
-                return (T) store;
+                return (T) maybeUncached;
             }
         } else {
             return null;
         }
     }
 
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    private static StateStore maybeBypassCache(final StateStore store, final boolean bypassCache) {
+        if (!bypassCache) {
+            return store;
+        } else if (store instanceof MeteredStore) {
+            final MeteredStore meteredStore = (MeteredStore) store;
+            final StateStore wrapped = meteredStore.wrapped();
+            final StateStore uncached = maybeBypassCache(wrapped, bypassCache);
+            if (uncached == wrapped) {
+                return store;
+            } else {
+                return meteredStore.reWrap(uncached);
+            }
+        } else if (WrappedStateStore.isCachingLayer(store)) {
+            return ((WrappedStateStore) store).wrapped();

Review comment:
       Specifically, this is where we drop the caching layer from the returned store hierarchy.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
##########
@@ -800,6 +811,76 @@ public void shouldBeAbleToQueryMapValuesAfterFilterState() throws Exception {
         }
     }
 
+    @Test
+    public void shouldQueryKeyValueWithCacheBypassed() throws Exception {

Review comment:
       This is the test I used to verify the behavior. Note: this test does not verify that the cache is skipped, just that the query works with the cache skipped. I set breakpoints in the code to additionally verify that the cache is actually skipped.
   
   If we go this route, I'd add unit tests so we can mechanically verify that we're skipping the cache.




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

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