Skip to content
Open
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,27 @@
/*
* 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;

/**
* A key-(value/timestamp/headers) store that supports put/get/delete.
*
* @param <K> The key type
* @param <V> The value type
*/
public interface TimestampedKeyValueStoreWithHeaders<K, V>
extends KeyValueStore<K, ValueTimestampHeaders<V>> {
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serializer;
Expand Down Expand Up @@ -323,7 +324,7 @@ public void put(final K key,
final V value) {
Objects.requireNonNull(key, "key cannot be null");
try {
maybeMeasureLatency(() -> wrapped().put(keyBytes(key), serdes.rawValue(value)), time, putSensor);
maybeMeasureLatency(() -> wrapped().put(keyBytes(key), serdes.rawValue(value, new RecordHeaders())), time, putSensor);
maybeRecordE2ELatency();
} catch (final ProcessorStateException e) {
final String message = String.format(e.getMessage(), key, value);
Expand Down Expand Up @@ -420,11 +421,11 @@ public void close() {
}

protected V outerValue(final byte[] value) {
return value != null ? serdes.valueFrom(value) : null;
return value != null ? serdes.valueFrom(value, new RecordHeaders()) : null;
}

protected Bytes keyBytes(final K key) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking into the POC PR, we actually added Headers parameter here, and I think it would be the right thing to do, and just pass in new RecrodHeaders() when used inside MeteredKeyValueStore but pass in the extracted Headers object from the value in the MeteredTimestampedKeyValueWithHeadersStore case?

This implies we need to overload put() in MeteredTimestampedKeyValueWithHeadersStore.

return Bytes.wrap(serdes.rawKey(key));
return Bytes.wrap(serdes.rawKey(key, new RecordHeaders()));
}

private List<KeyValue<Bytes, byte[]>> innerEntries(final List<KeyValue<K, V>> from) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* 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.common.utils.Bytes;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.processor.internals.SerdeGetter;
import org.apache.kafka.streams.query.Position;
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;
import org.apache.kafka.streams.state.TimestampedKeyValueStoreWithHeaders;
import org.apache.kafka.streams.state.ValueTimestampHeaders;


/**
* A Metered {@link TimestampedKeyValueStoreWithHeaders} wrapper that is used for recording operation metrics, and hence
* its inner KeyValueStore implementation does 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, ValueTimestampHeaders&lt;V&gt;&gt; to &lt;Bytes, byte[]&gt;.
*
* @param <K> key type
* @param <V> value type (wrapped in {@link ValueTimestampHeaders})
*/
public class MeteredTimestampedKeyValueStoreWithHeaders<K, V>
extends MeteredKeyValueStore<K, ValueTimestampHeaders<V>>
implements TimestampedKeyValueStoreWithHeaders<K, V> {

MeteredTimestampedKeyValueStoreWithHeaders(final KeyValueStore<Bytes, byte[]> inner,
final String metricScope,
final Time time,
final Serde<K> keySerde,
final Serde<ValueTimestampHeaders<V>> valueSerde) {
super(inner, metricScope, time, keySerde, valueSerde);
}

@SuppressWarnings("unchecked")
@Override
protected Serde<ValueTimestampHeaders<V>> prepareValueSerdeForStore(final Serde<ValueTimestampHeaders<V>> valueSerde,
final SerdeGetter getter) {
if (valueSerde == null) {
return new ValueTimestampHeadersSerde<>((Serde<V>) getter.valueSerde());
} else {
return super.prepareValueSerdeForStore(valueSerde, getter);
}
}

@Override
public <R> QueryResult<R> query(final Query<R> query,
final PositionBound positionBound,
final QueryConfig config) {
throw new UnsupportedOperationException("Querying is not supported for " + getClass().getSimpleName());
}

@Override
public Position getPosition() {
throw new UnsupportedOperationException("Position is not supported for " + getClass().getSimpleName());
}

}
Loading