Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.utils.Bytes;
import org.apache.kafka.streams.query.PositionBound;
import org.apache.kafka.streams.query.Query;
import org.apache.kafka.streams.query.QueryConfig;
import org.apache.kafka.streams.query.QueryResult;
import org.apache.kafka.streams.state.KeyValueStore;

/**
* A caching key-value store with headers is a caching key-value store that only forwards the query to the
* wrapped store.
*/
public class CachingKeyValueStoreWithHeaders extends CachingKeyValueStore {

CachingKeyValueStoreWithHeaders(final KeyValueStore<Bytes, byte[]> underlying) {
super(underlying, CacheType.TIMESTAMPED_KEY_VALUE_STORE_WITH_HEADERS);
}

@Override
public <R> QueryResult<R> query(final Query<R> query,
final PositionBound positionBound,
final QueryConfig config) {
return wrapped().query(query, positionBound, config);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ private KeyValueStore<Bytes, byte[]> maybeWrapCaching(final KeyValueStore<Bytes,
if (!enableCaching) {
return inner;
}
return new CachingKeyValueStore(inner, CachingKeyValueStore.CacheType.TIMESTAMPED_KEY_VALUE_STORE_WITH_HEADERS);
return new CachingKeyValueStoreWithHeaders(inner);
}

private KeyValueStore<Bytes, byte[]> maybeWrapLogging(final KeyValueStore<Bytes, byte[]> inner) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,12 +44,15 @@

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;

import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Properties;

Expand All @@ -59,6 +62,9 @@
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.lenient;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

@ExtendWith(MockitoExtension.class)
Expand Down Expand Up @@ -506,33 +512,56 @@ public void shouldMaintainPositionAcrossOperationsOnHeadersStore() {
}
}

@Test
public void shouldReturnUnknownQueryTypeForKeyQueryOnHeadersStore() {
private static ThreadCache mockCacheHit() {
final ThreadCache cache = mock(ThreadCache.class);
final LRUCacheEntry entry = mock(LRUCacheEntry.class);
final byte[] entryValue = "mockEntryValue".getBytes(StandardCharsets.UTF_8);
lenient().when(entry.value()).thenReturn(entryValue);
lenient().when(cache.get(any(String.class), any(Bytes.class))).thenReturn(entry);
return cache;
}

private TimestampedKeyValueStoreWithHeaders<String, String> headersStoreMaybeWithCache(final boolean cachingEnabled) {
when(supplier.name()).thenReturn("test-store");
when(supplier.metricsScope()).thenReturn("metricScope");
when(supplier.get()).thenReturn(new RocksDBTimestampedStoreWithHeaders("test-store", "metrics-scope"));

builder = new TimestampedKeyValueStoreBuilderWithHeaders<>(
supplier,
final File dir = TestUtils.tempDirectory();
final ThreadCache cache = mockCacheHit();
final InternalMockProcessorContext<String, String> context = new InternalMockProcessorContext<>(
dir,
Serdes.String(),
Serdes.String(),
new MockTime()
null,
cache
);

final TimestampedKeyValueStoreWithHeaders<String, String> store = builder
.withLoggingDisabled()
.withCachingDisabled()
.build();

final File dir = TestUtils.tempDirectory();
final Properties props = StreamsTestUtils.getStreamsConfig();
final InternalMockProcessorContext<String, String> context = new InternalMockProcessorContext<>(
dir,
builder = new TimestampedKeyValueStoreBuilderWithHeaders<>(
supplier,
Serdes.String(),
Serdes.String(),
new StreamsConfig(props)
new MockTime()
);

final TimestampedKeyValueStoreWithHeaders<String, String> store;
if (cachingEnabled) {
store = builder.withLoggingDisabled()
.withCachingEnabled()
.build();
} else {
store = builder.withLoggingDisabled()
.withCachingDisabled()
.build();
}

store.init(context, store);
return store;
}

@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldReturnUnknownQueryTypeForKeyQueryOnHeadersStore(final boolean cachingEnabled) {
final TimestampedKeyValueStoreWithHeaders<String, String> store = headersStoreMaybeWithCache(cachingEnabled);
Comment on lines +561 to +564
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.

missing a test for IQv2 that verifies that we indeed get an error-result back for kv-header-store case with caching enanbled

This is covered by this test when cachingEnabled = true... right?

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.

Ah. Sounds correct.

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.

Actually, maybe you meant IQv2StoreIntegrationTest... let me look into adding an integration test there


try {
final KeyQuery<Bytes, byte[]> query = KeyQuery.withKey(new Bytes("test-key".getBytes()));
Expand All @@ -555,33 +584,10 @@ public void shouldReturnUnknownQueryTypeForKeyQueryOnHeadersStore() {
}
}

@Test
public void shouldReturnUnknownQueryTypeForRangeQueryOnHeadersStore() {
when(supplier.name()).thenReturn("test-store");
when(supplier.metricsScope()).thenReturn("metricScope");
when(supplier.get()).thenReturn(new RocksDBTimestampedStoreWithHeaders("test-store", "metrics-scope"));

builder = new TimestampedKeyValueStoreBuilderWithHeaders<>(
supplier,
Serdes.String(),
Serdes.String(),
new MockTime()
);

final TimestampedKeyValueStoreWithHeaders<String, String> store = builder
.withLoggingDisabled()
.withCachingDisabled()
.build();

final File dir = TestUtils.tempDirectory();
final Properties props = StreamsTestUtils.getStreamsConfig();
final InternalMockProcessorContext<String, String> context = new InternalMockProcessorContext<>(
dir,
Serdes.String(),
Serdes.String(),
new StreamsConfig(props)
);
store.init(context, store);
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void shouldReturnUnknownQueryTypeForRangeQueryOnHeadersStore(final boolean cachingEnabled) {
final TimestampedKeyValueStoreWithHeaders<String, String> store = headersStoreMaybeWithCache(cachingEnabled);

try {
final RangeQuery<Bytes, byte[]> query = RangeQuery.withRange(
Expand Down
Loading