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 @@ -18,6 +18,7 @@

import org.apache.kafka.common.header.Headers;

import java.util.Arrays;
import java.util.Objects;

/**
Expand Down Expand Up @@ -96,12 +97,18 @@ public boolean equals(final Object o) {
}
final AggregationWithHeaders<?> that = (AggregationWithHeaders<?>) o;
return Objects.equals(aggregation, that.aggregation)
&& Objects.equals(this.headers, that.headers);
&& headersEqual(this.headers, that.headers);
}

private static boolean headersEqual(final Headers a, final Headers b) {
if (a == b) return true;
if (a == null || b == null) return false;
return Arrays.equals(a.toArray(), b.toArray());
Comment on lines +103 to +106
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

why do need it?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

RecordHeaders.equals have getClass() != o.getClass() because of that tests were failing for RecordHeaders.equals(LazyHeaders) so we need to handle that.

}

@Override
public int hashCode() {
return Objects.hash(aggregation, headers);
return Objects.hash(aggregation, Arrays.hashCode(headers.toArray()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;

import java.util.Arrays;
import java.util.Objects;

/**
Expand Down Expand Up @@ -108,12 +109,18 @@ public boolean equals(final Object o) {
final ValueTimestampHeaders<?> that = (ValueTimestampHeaders<?>) o;
return timestamp == that.timestamp
&& Objects.equals(value, that.value)
&& Objects.equals(this.headers, that.headers);
&& headersEqual(this.headers, that.headers);
}

private static boolean headersEqual(final Headers a, final Headers b) {
if (a == b) return true;
if (a == null || b == null) return false;
return Arrays.equals(a.toArray(), b.toArray());
}

@Override
public int hashCode() {
return Objects.hash(value, timestamp, headers);
return Objects.hash(value, timestamp, Arrays.hashCode(headers.toArray()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/*
* 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.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.header.internals.RecordHeaders;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;

/**
* A lazy implementation of {@link Headers} that defers deserialization of header bytes
* until first read access. This avoids unnecessary parsing when the downstream
* deserializer does not inspect headers.
*
* <p>Headers added via {@link #add(Header)} or {@link #add(String, byte[])} before
* materialization are accumulated in a side list and merged on first read access.
*
* <p>Instances are confined to a single {@code StreamThread} and are not shared
* across threads, so no synchronization is needed.
*/
class LazyHeaders implements Headers {

private final byte[] rawHeaders;
private RecordHeaders materialized;
private List<Header> pendingAdds;

/**
* Creates a new LazyHeaders wrapping the given raw header bytes.
*
* @param rawHeaders the serialized header bytes (without the varint size prefix),
* as expected by {@link HeadersDeserializer#deserialize(byte[])}.
* May be null or empty for empty headers.
*/
LazyHeaders(final byte[] rawHeaders) {
this.rawHeaders = rawHeaders;
}

private RecordHeaders materialize() {
if (materialized == null) {
final Headers deserialized = HeadersDeserializer.deserialize(rawHeaders);
materialized = (deserialized instanceof RecordHeaders)
? (RecordHeaders) deserialized
: new RecordHeaders(deserialized);
if (pendingAdds != null) {
for (final Header h : pendingAdds) {
materialized.add(h);
}
pendingAdds = null;
}
}
return materialized;
}

/**
* Returns true if the headers have been deserialized.
* Visible for testing.
*/
boolean isDeserialized() {
return materialized != null;
}

@Override
public Headers add(final Header header) throws IllegalStateException {
Objects.requireNonNull(header, "header cannot be null");
if (materialized != null) {
materialized.add(header);
} else {
if (pendingAdds == null) {
pendingAdds = new ArrayList<>();
}
pendingAdds.add(header);
}
return this;
}

@Override
public Headers add(final String key, final byte[] value) throws IllegalStateException {
return add(new RecordHeader(key, value));
}

@Override
public Headers remove(final String key) throws IllegalStateException {
materialize().remove(key);
return this;
}

@Override
public Header lastHeader(final String key) {
return materialize().lastHeader(key);
}

@Override
public Iterable<Header> headers(final String key) {
return materialize().headers(key);
}

@Override
public Header[] toArray() {
return materialize().toArray();
}

@Override
public Iterator<Header> iterator() {
return materialize().iterator();
}

@Override
public boolean equals(final Object o) {
if (this == o) return true;
if (!(o instanceof Headers)) return false;
final Headers other = (o instanceof LazyHeaders)
? ((LazyHeaders) o).materialize()
: (Headers) o;
return Arrays.equals(materialize().toArray(), other.toArray());
}

@Override
public int hashCode() {
return Arrays.hashCode(materialize().toArray());
}

@Override
public String toString() {
if (materialized != null) {
return materialized.toString();
}
return "LazyHeaders(not yet deserialized)";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ public static Headers headers(final byte[] valueWithHeaders) {
return null;
}

// If the header is empty, simply return it
// If the header is empty, return empty headers without lazy wrapping
if (hasEmptyHeaders(valueWithHeaders)) {
return new RecordHeaders();
}
Expand Down Expand Up @@ -173,7 +173,7 @@ public static byte[] readBytes(final ByteBuffer buffer, final int length) {
public static Headers readHeaders(final ByteBuffer buffer) {
final int headersSize = ByteUtils.readVarint(buffer);
final byte[] rawHeaders = readBytes(buffer, headersSize);
return HeadersDeserializer.deserialize(rawHeaders);
return (headersSize == 0) ? new RecordHeaders() : new LazyHeaders(rawHeaders);
}

/**
Expand Down
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.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.common.utils.internals.ByteUtils;
Expand Down Expand Up @@ -73,7 +74,7 @@ public ValueTimestampHeaders<V> deserialize(final String topic, final byte[] val
final int headersSize = ByteUtils.readVarint(buffer);

final byte[] rawHeaders = readBytes(buffer, headersSize);
final Headers headers = HeadersDeserializer.deserialize(rawHeaders);
final Headers headers = (headersSize == 0) ? new RecordHeaders() : new LazyHeaders(rawHeaders);
final byte[] rawTimestamp = readBytes(buffer, Long.BYTES);
final long timestamp = timestampDeserializer.deserialize(topic, rawTimestamp);
final byte[] rawValue = readBytes(buffer, buffer.remaining());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import org.apache.kafka.streams.state.AggregationWithHeaders;
import org.apache.kafka.streams.state.SessionStore;

import java.util.Arrays;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
Expand All @@ -38,6 +40,8 @@

import static org.apache.kafka.common.utils.Utils.mkEntry;
import static org.apache.kafka.common.utils.Utils.mkMap;
import static org.mockito.ArgumentMatchers.argThat;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
Expand Down Expand Up @@ -100,12 +104,12 @@ public void shouldLogPutWithHeaders() {

verify(inner).put(key1, serializedValue);
verify(context).logChange(
store.name(),
binaryKey,
value1,
0L,
headers,
Position.emptyPosition()
eq(store.name()),
eq(binaryKey),
eq(value1),
eq(0L),
argThat(actual -> Arrays.equals(actual.toArray(), headers.toArray())),
eq(Position.emptyPosition())
);
}

Expand All @@ -124,12 +128,12 @@ public void shouldLogPutWithPosition() {

verify(inner).put(key1, serializedValue);
verify(context).logChange(
store.name(),
binaryKey,
value1,
0L,
headers,
POSITION
eq(store.name()),
eq(binaryKey),
eq(value1),
eq(0L),
argThat(actual -> Arrays.equals(actual.toArray(), headers.toArray())),
eq(POSITION)
);
}

Expand Down Expand Up @@ -217,12 +221,12 @@ public void shouldHandleMultipleHeadersInSingleRecord() {

verify(inner).put(key1, serializedValue);
verify(context).logChange(
store.name(),
binaryKey,
value1,
0L,
headers,
Position.emptyPosition()
eq(store.name()),
eq(binaryKey),
eq(value1),
eq(0L),
argThat(actual -> Arrays.equals(actual.toArray(), headers.toArray())),
eq(Position.emptyPosition())
);
}
}
Loading