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
Expand Up @@ -17,6 +17,8 @@
package org.apache.kafka.streams.kstream.internals.foreignkeyjoin;

import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serializer;
Expand Down Expand Up @@ -64,13 +66,18 @@ public void setIfUnset(final SerdeGetter getter) {

@Override
public byte[] serialize(final String topic, final SubscriptionResponseWrapper<V> data) {
return serialize(topic, new RecordHeaders(), data);
}

@Override
public byte[] serialize(final String topic, final Headers headers, final SubscriptionResponseWrapper<V> data) {
//{1-bit-isHashNull}{7-bits-version}{Optional-16-byte-Hash}{n-bytes serialized data}

if (data.version() < 0) {
throw new UnsupportedVersionException("SubscriptionResponseWrapper version cannot be negative");
}

final byte[] serializedData = data.foreignValue() == null ? null : serializer.serialize(topic, data.foreignValue());
final byte[] serializedData = data.foreignValue() == null ? null : serializer.serialize(topic, headers, data.foreignValue());
final int serializedDataLength = serializedData == null ? 0 : serializedData.length;
final long[] originalHash = data.originalValueHash();
final int hashLength = originalHash == null ? 0 : 2 * Long.BYTES;
Expand Down Expand Up @@ -111,6 +118,11 @@ public void setIfUnset(final SerdeGetter getter) {

@Override
public SubscriptionResponseWrapper<V> deserialize(final String topic, final byte[] data) {
return deserialize(topic, new RecordHeaders(), data);
}

@Override
public SubscriptionResponseWrapper<V> deserialize(final String topic, final Headers headers, final byte[] data) {
//{1-bit-isHashNull}{7-bits-version}{Optional-16-byte-Hash}{n-bytes serialized data}

final ByteBuffer buf = ByteBuffer.wrap(data);
Expand All @@ -134,7 +146,7 @@ public SubscriptionResponseWrapper<V> deserialize(final String topic, final byte
final byte[] serializedValue;
serializedValue = new byte[data.length - lengthSum];
buf.get(serializedValue, 0, serializedValue.length);
value = deserializer.deserialize(topic, serializedValue);
value = deserializer.deserialize(topic, headers, serializedValue);
} else {
value = null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
import org.apache.kafka.streams.processor.internals.SerdeGetter;
Expand Down Expand Up @@ -57,13 +59,18 @@ public void configure(final Map<String, ?> configs,

@Override
public LeftOrRightValue<V1, V2> deserialize(final String topic, final byte[] data) {
return deserialize(topic, new RecordHeaders(), data);
}

@Override
public LeftOrRightValue<V1, V2> deserialize(final String topic, final Headers headers, final byte[] data) {
if (data == null || data.length == 0) {
return null;
}

return (data[0] == 1)
? LeftOrRightValue.makeLeftValue(leftDeserializer.deserialize(topic, rawValue(data)))
: LeftOrRightValue.makeRightValue(rightDeserializer.deserialize(topic, rawValue(data)));
? LeftOrRightValue.makeLeftValue(leftDeserializer.deserialize(topic, headers, rawValue(data)))
: LeftOrRightValue.makeRightValue(rightDeserializer.deserialize(topic, headers, rawValue(data)));
}

private byte[] rawValue(final byte[] data) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
import org.apache.kafka.streams.processor.internals.SerdeGetter;
Expand Down Expand Up @@ -61,13 +63,18 @@ public void configure(final Map<String, ?> configs, final boolean isKey) {

@Override
public byte[] serialize(final String topic, final LeftOrRightValue<V1, V2> data) {
return serialize(topic, new RecordHeaders(), data);
}

@Override
public byte[] serialize(final String topic, final Headers headers, final LeftOrRightValue<V1, V2> data) {
if (data == null) {
return null;
}

final byte[] rawValue = (data.leftValue() != null)
? leftSerializer.serialize(topic, data.leftValue())
: rightSerializer.serialize(topic, data.rightValue());
? leftSerializer.serialize(topic, headers, data.leftValue())
: rightSerializer.serialize(topic, headers, data.rightValue());

if (rawValue == null) {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer;
Expand Down Expand Up @@ -56,9 +58,14 @@ public void configure(final Map<String, ?> configs, final boolean isKey) {

@Override
public TimestampedKeyAndJoinSide<K> deserialize(final String topic, final byte[] data) {
return deserialize(topic, new RecordHeaders(), data);
}

@Override
public TimestampedKeyAndJoinSide<K> deserialize(final String topic, final Headers headers, final byte[] data) {
final boolean isLeft = data[StateSerdes.TIMESTAMP_SIZE] == 1;
final K key = keyDeserializer.deserialize(topic, rawKey(data));
final long timestamp = timestampDeserializer.deserialize(topic, rawTimestamp(data));
final K key = keyDeserializer.deserialize(topic, headers, rawKey(data));
final long timestamp = timestampDeserializer.deserialize(topic, headers, rawTimestamp(data));

return isLeft ? TimestampedKeyAndJoinSide.makeLeft(key, timestamp) :
TimestampedKeyAndJoinSide.makeRight(key, timestamp);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
*/
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.LongSerializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer;
Expand Down Expand Up @@ -56,9 +58,14 @@ public void configure(final Map<String, ?> configs, final boolean isKey) {

@Override
public byte[] serialize(final String topic, final TimestampedKeyAndJoinSide<K> data) {
return serialize(topic, new RecordHeaders(), data);
}

@Override
public byte[] serialize(final String topic, final Headers headers, final TimestampedKeyAndJoinSide<K> data) {
final byte boolByte = (byte) (data.isLeftSide() ? 1 : 0);
final byte[] keyBytes = keySerializer.serialize(topic, data.key());
final byte[] timestampBytes = timestampSerializer.serialize(topic, data.timestamp());
final byte[] keyBytes = keySerializer.serialize(topic, headers, data.key());
final byte[] timestampBytes = timestampSerializer.serialize(topic, headers, data.timestamp());

return ByteBuffer
.allocate(timestampBytes.length + 1 + keyBytes.length)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,14 @@
package org.apache.kafka.streams.kstream.internals.foreignkeyjoin;

import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.state.internals.Murmur3;

import org.junit.jupiter.api.Test;
Expand All @@ -32,6 +36,10 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

public class SubscriptionResponseWrapperSerdeTest {
private static final class NonNullableSerde<T> implements Serde<T>, Serializer<T>, Deserializer<T> {
Expand Down Expand Up @@ -147,6 +155,47 @@ public void shouldThrowExceptionOnSerializeWhenDataVersionUnknown() {
}
}

@Test
public void shouldPassHeadersToUnderlyingSerializer() {
final Serializer<String> mockSerializer = mock(StringSerializer.class);
final Serde<String> mockSerde = mock(Serdes.StringSerde.class);
when(mockSerde.serializer()).thenReturn(mockSerializer);

final String topic = "dummy";
final String foreignValue = "foreignValue";
final Headers headers = new RecordHeaders().add("key", "value".getBytes());
final SubscriptionResponseWrapper<String> data = new SubscriptionResponseWrapper<>(null, foreignValue, 1);

final SubscriptionResponseWrapperSerde<String> testSerde = new SubscriptionResponseWrapperSerde<>(mockSerde);

testSerde.serializer().serialize(topic, headers, data);

verify(mockSerializer).serialize(topic, headers, foreignValue);
verify(mockSerializer, never()).serialize(topic, foreignValue);
}

@Test
public void shouldPassHeadersToUnderlyingDeserializer() {
final Deserializer<String> mockDeserializer = mock(StringDeserializer.class);
final Serde<String> mockSerde = mock(Serdes.StringSerde.class);
when(mockSerde.deserializer()).thenReturn(mockDeserializer);
when(mockSerde.serializer()).thenReturn(Serdes.String().serializer());

final String topic = "dummy";
final String foreignValue = "foreignValue";
final Headers headers = new RecordHeaders().add("key", "value".getBytes());
final SubscriptionResponseWrapper<String> data = new SubscriptionResponseWrapper<>(null, foreignValue, 1);

final SubscriptionResponseWrapperSerde<String> testSerde = new SubscriptionResponseWrapperSerde<>(mockSerde);

final byte[] serializedData = testSerde.serializer().serialize(topic, headers, data);

testSerde.deserializer().deserialize(topic, headers, serializedData);

verify(mockDeserializer).deserialize(topic, headers, foreignValue.getBytes());
verify(mockDeserializer, never()).deserialize(topic, foreignValue.getBytes());
}

public static class InvalidSubscriptionResponseWrapper extends SubscriptionResponseWrapper<String> {

public InvalidSubscriptionResponseWrapper(final long[] originalValueHash,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringDeserializer;

import org.junit.jupiter.api.Test;

import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

public class LeftOrRightValueDeserializerTest {

@Test
public void shouldPassHeadersToUnderlyingDeserializer() {
final Deserializer<String> mockDeserializer = mock(StringDeserializer.class);

final String topic = "dummy";
final String value = "some-string";
final Headers headers = new RecordHeaders().add("key", "value".getBytes());
final LeftOrRightValue<String, Object> data = LeftOrRightValue.makeLeftValue(value);
final byte[] serializedBytes = new LeftOrRightValueSerializer<>(Serdes.String().serializer(), null).serialize(topic, headers, data);

when(mockDeserializer.deserialize(topic, headers, value.getBytes())).thenReturn("dummy-value");

final LeftOrRightValueDeserializer<String, String> testDeserializer = new LeftOrRightValueDeserializer<>(mockDeserializer, null);

testDeserializer.deserialize(topic, headers, serializedBytes);

verify(mockDeserializer).deserialize(topic, headers, value.getBytes());
verify(mockDeserializer, never()).deserialize(topic, value.getBytes());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,34 +16,38 @@
*/
package org.apache.kafka.streams.state.internals;

import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.serialization.StringSerializer;

import org.junit.jupiter.api.Test;

import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
Comment on lines 27 to 29
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.

@mjsax Do we use org.hamcrest? I thought AssertJ was chosen for KS.... Well never asked that, but had an assumption :)

import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;

public class LeftOrRightValueSerializerTest {
private static final String TOPIC = "some-topic";

private static final LeftOrRightValueSerde<String, Integer> STRING_OR_INTEGER_SERDE =
new LeftOrRightValueSerde<>(Serdes.String(), Serdes.Integer());
private static final LeftOrRightValueSerde<String, Integer> STRING_OR_INTEGER_SERDE = new LeftOrRightValueSerde<>(Serdes.String(), Serdes.Integer());

@Test
public void shouldSerializeStringValue() {
final String value = "some-string";

final LeftOrRightValue<String, Integer> leftOrRightValue = LeftOrRightValue.makeLeftValue(value);

final byte[] serialized =
STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue);
final byte[] serialized = STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue);

assertThat(serialized, is(notNullValue()));

final LeftOrRightValue<String, Integer> deserialized =
STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized);
final LeftOrRightValue<String, Integer> deserialized = STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized);

assertThat(deserialized, is(leftOrRightValue));
}
Expand All @@ -54,13 +58,11 @@ public void shouldSerializeIntegerValue() {

final LeftOrRightValue<String, Integer> leftOrRightValue = LeftOrRightValue.makeRightValue(value);

final byte[] serialized =
STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue);
final byte[] serialized = STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue);

assertThat(serialized, is(notNullValue()));

final LeftOrRightValue<String, Integer> deserialized =
STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized);
final LeftOrRightValue<String, Integer> deserialized = STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized);

assertThat(deserialized, is(leftOrRightValue));
}
Expand All @@ -76,4 +78,21 @@ public void shouldThrowIfSerializeOtherValueAsNull() {
assertThrows(NullPointerException.class,
() -> STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, LeftOrRightValue.makeRightValue(null)));
}

@Test
public void shouldPassHeadersToUnderlyingSerializer() {
final Serializer<String> mockSerializer = mock(StringSerializer.class);

final String topic = "dummy";
final String value = "some-string";
final Headers headers = new RecordHeaders().add("key", "value".getBytes());
final LeftOrRightValue<String, String> data = LeftOrRightValue.makeLeftValue(value);

final LeftOrRightValueSerializer<String, String> testSerializer = new LeftOrRightValueSerializer<>(mockSerializer, null);

testSerializer.serialize(topic, headers, data);

verify(mockSerializer).serialize(topic, headers, value);
verify(mockSerializer, never()).serialize(topic, value);
}
}
Loading