MINOR: Remove connection id from Send and consolidate request/message utils (#9714)

Connection id is now only present in `NetworkSend`, which is now
the class used by `Selector`/`NetworkClient`/`KafkaChannel` (which
works well since `NetworkReceive` is the class used for
received data).

The previous `NetworkSend` was also responsible for adding a size
prefix. This logic is already present in `SendBuilder`, but for the
minority of cases where `SendBuilder` is not used (including
a number of tests), we now have `ByteBufferSend.sizePrefixed()`.

With regards to the request/message utilities:
* Renamed `toByteBuffer`/`toBytes` in `MessageUtil` to
`toVersionPrefixedByteBuffer`/`toVersionPrefixedBytes` for clarity.
* Introduced new `MessageUtil.toByteBuffer` that does not include
the version as the prefix.
* Renamed `serializeBody` in `AbstractRequest/Response` to
`serialize` for symmetry with `parse`.
* Introduced `RequestTestUtils` and moved relevant methods from
`TestUtils`.
* Moved `serializeWithHeader` methods that were only used in
tests to `RequestTestUtils`.
* Deleted `MessageTestUtil`.

Finally, a couple of changes to simplify coding patterns:
* Added `flip()` and `buffer()` to `ByteBufferAccessor`.
* Added `MessageSizeAccumulator.sizeExcludingZeroCopy`.
* Used lambdas instead of `TestCondition`.
* Used `Arrays.copyOf` instead of `System.arraycopy` in `MessageUtil`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
This commit is contained in:
Ismael Juma 2020-12-09 11:15:58 -08:00 committed by GitHub
parent ff88874e0d
commit 1f98112e99
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
95 changed files with 772 additions and 775 deletions

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersionsResponseKey;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.network.ChannelState;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.network.Send;
@ -519,7 +520,7 @@ public class NetworkClient implements KafkaClient {
log.debug("Sending {} request with header {} and timeout {} to node {}: {}",
clientRequest.apiKey(), header, clientRequest.requestTimeoutMs(), destination, request);
}
Send send = request.toSend(destination, header);
Send send = request.toSend(header);
InFlightRequest inFlightRequest = new InFlightRequest(
clientRequest,
header,
@ -528,7 +529,7 @@ public class NetworkClient implements KafkaClient {
send,
now);
this.inFlightRequests.add(inFlightRequest);
selector.send(send);
selector.send(new NetworkSend(clientRequest.destination(), send));
}
/**
@ -831,10 +832,10 @@ public class NetworkClient implements KafkaClient {
*/
private void handleCompletedSends(List<ClientResponse> responses, long now) {
// if no response is expected then when the send is completed, return it
for (Send send : this.selector.completedSends()) {
InFlightRequest request = this.inFlightRequests.lastSent(send.destination());
for (NetworkSend send : this.selector.completedSends()) {
InFlightRequest request = this.inFlightRequests.lastSent(send.destinationId());
if (!request.expectResponse) {
this.inFlightRequests.completeLastSent(send.destination());
this.inFlightRequests.completeLastSent(send.destinationId());
responses.add(request.completed(null, now));
}
}

View File

@ -81,7 +81,7 @@ public class ConsumerProtocol {
.setPartitions(topicEntry.getValue()));
}
return MessageUtil.toByteBuffer(version, data);
return MessageUtil.toVersionPrefixedByteBuffer(version, data);
}
public static Subscription deserializeSubscription(final ByteBuffer buffer, short version) {
@ -127,7 +127,7 @@ public class ConsumerProtocol {
.setPartitions(topicEntry.getValue()));
}
return MessageUtil.toByteBuffer(version, data);
return MessageUtil.toVersionPrefixedByteBuffer(version, data);
}
public static Assignment deserializeAssignment(final ByteBuffer buffer, short version) {

View File

@ -26,32 +26,24 @@ import java.nio.channels.GatheringByteChannel;
*/
public class ByteBufferSend implements Send {
private final String destination;
private final long size;
protected final ByteBuffer[] buffers;
private long remaining;
private boolean pending = false;
public ByteBufferSend(String destination, ByteBuffer... buffers) {
this.destination = destination;
public ByteBufferSend(ByteBuffer... buffers) {
this.buffers = buffers;
for (ByteBuffer buffer : buffers)
remaining += buffer.remaining();
this.size = remaining;
}
public ByteBufferSend(String destination, ByteBuffer[] buffers, long size) {
this.destination = destination;
public ByteBufferSend(ByteBuffer[] buffers, long size) {
this.buffers = buffers;
this.size = size;
this.remaining = size;
}
@Override
public String destination() {
return destination;
}
@Override
public boolean completed() {
return remaining <= 0 && !pending;
@ -79,10 +71,15 @@ public class ByteBufferSend implements Send {
@Override
public String toString() {
return "ByteBufferSend(" +
"destination='" + destination + "'" +
", size=" + size +
", remaining=" + remaining +
", pending=" + pending +
')';
}
public static ByteBufferSend sizePrefixed(ByteBuffer buffer) {
ByteBuffer sizeBuffer = ByteBuffer.allocate(4);
sizeBuffer.putInt(0, buffer.remaining());
return new ByteBufferSend(sizeBuffer, buffer);
}
}

View File

@ -124,7 +124,7 @@ public class KafkaChannel implements AutoCloseable {
private final MemoryPool memoryPool;
private final ChannelMetadataRegistry metadataRegistry;
private NetworkReceive receive;
private Send send;
private NetworkSend send;
// Track connection and mute state of channels to enable outstanding requests on channels to be
// processed after the channel is disconnected.
private boolean disconnected;
@ -376,18 +376,18 @@ public class KafkaChannel implements AutoCloseable {
return socket.getInetAddress().toString();
}
public void setSend(Send send) {
public void setSend(NetworkSend send) {
if (this.send != null)
throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress, connection id is " + id);
this.send = send;
this.transportLayer.addInterestOps(SelectionKey.OP_WRITE);
}
public Send maybeCompleteSend() {
public NetworkSend maybeCompleteSend() {
if (send != null && send.completed()) {
midWrite = false;
transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
Send result = send;
NetworkSend result = send;
send = null;
return result;
}

View File

@ -16,22 +16,35 @@
*/
package org.apache.kafka.common.network;
import java.nio.ByteBuffer;
import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
/**
* A size delimited Send that consists of a 4 byte network-ordered size N followed by N bytes of content
*/
public class NetworkSend extends ByteBufferSend {
public class NetworkSend implements Send {
private final String destinationId;
private final Send send;
public NetworkSend(String destination, ByteBuffer buffer) {
super(destination, sizeBuffer(buffer.remaining()), buffer);
public NetworkSend(String destinationId, Send send) {
this.destinationId = destinationId;
this.send = send;
}
private static ByteBuffer sizeBuffer(int size) {
ByteBuffer sizeBuffer = ByteBuffer.allocate(4);
sizeBuffer.putInt(size);
sizeBuffer.rewind();
return sizeBuffer;
public String destinationId() {
return destinationId;
}
@Override
public boolean completed() {
return send.completed();
}
@Override
public long writeTo(GatheringByteChannel channel) throws IOException {
return send.writeTo(channel);
}
@Override
public long size() {
return send.size();
}
}

View File

@ -62,7 +62,7 @@ public interface Selectable {
* Queue the given request for sending in the subsequent {@link #poll(long) poll()} calls
* @param send The request to send
*/
void send(Send send);
void send(NetworkSend send);
/**
* Do I/O. Reads, writes, connection establishment, etc.
@ -74,7 +74,7 @@ public interface Selectable {
/**
* The list of sends that completed on the last {@link #poll(long) poll()} call.
*/
List<Send> completedSends();
List<NetworkSend> completedSends();
/**
* The collection of receives that completed on the last {@link #poll(long) poll()} call.

View File

@ -106,7 +106,7 @@ public class Selector implements Selectable, AutoCloseable {
private final Map<String, KafkaChannel> channels;
private final Set<KafkaChannel> explicitlyMutedChannels;
private boolean outOfMemory;
private final List<Send> completedSends;
private final List<NetworkSend> completedSends;
private final LinkedHashMap<String, NetworkReceive> completedReceives;
private final Set<SelectionKey> immediatelyConnectedKeys;
private final Map<String, KafkaChannel> closingChannels;
@ -383,8 +383,8 @@ public class Selector implements Selectable, AutoCloseable {
* Queue the given request for sending in the subsequent {@link #poll(long)} calls
* @param send The request to send
*/
public void send(Send send) {
String connectionId = send.destination();
public void send(NetworkSend send) {
String connectionId = send.destinationId();
KafkaChannel channel = openOrClosingChannelOrFail(connectionId);
if (closingChannels.containsKey(connectionId)) {
// ensure notification via `disconnected`, leave channel in the state in which closing was triggered
@ -642,7 +642,7 @@ public class Selector implements Selectable, AutoCloseable {
void write(KafkaChannel channel) throws IOException {
String nodeId = channel.id();
long bytesSent = channel.write();
Send send = channel.maybeCompleteSend();
NetworkSend send = channel.maybeCompleteSend();
// We may complete the send with bytesSent < 1 if `TransportLayer.hasPendingWrites` was true and `channel.write()`
// caused the pending writes to be written to the socket channel buffer
if (bytesSent > 0 || send != null) {
@ -714,7 +714,7 @@ public class Selector implements Selectable, AutoCloseable {
}
@Override
public List<Send> completedSends() {
public List<NetworkSend> completedSends() {
return this.completedSends;
}

View File

@ -20,15 +20,10 @@ import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
/**
* This interface models the in-progress sending of data to a specific destination
* This interface models the in-progress sending of data.
*/
public interface Send {
/**
* The id for the destination of this send
*/
String destination();
/**
* Is this send complete?
*/

View File

@ -133,4 +133,11 @@ public class ByteBufferAccessor implements Readable, Writable {
return ByteUtils.readVarlong(buf);
}
public void flip() {
buf.flip();
}
public ByteBuffer buffer() {
return buf;
}
}

View File

@ -32,6 +32,14 @@ public class MessageSizeAccumulator {
return totalSize;
}
/**
* Size excluding zero copy fields as specified by {@link #zeroCopySize}. This is typically the size of the byte
* buffer used to serialize messages.
*/
public int sizeExcludingZeroCopy() {
return totalSize - zeroCopySize;
}
/**
* Get the total "zero-copy" size of the message. This is the summed
* total of all fields which have either have a type of 'bytes' with

View File

@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Utils;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
@ -159,12 +160,9 @@ public final class MessageUtil {
}
public static byte[] duplicate(byte[] array) {
if (array == null) {
if (array == null)
return null;
}
byte[] newArray = new byte[array.length];
System.arraycopy(array, 0, newArray, 0, array.length);
return newArray;
return Arrays.copyOf(array, array.length);
}
/**
@ -182,19 +180,27 @@ public final class MessageUtil {
}
}
public static ByteBuffer toByteBuffer(final short version, final Message message) {
public static ByteBuffer toByteBuffer(final Message message, final short version) {
ObjectSerializationCache cache = new ObjectSerializationCache();
int size = message.size(cache, version);
ByteBuffer bytes = ByteBuffer.allocate(2 + size);
ByteBufferAccessor accessor = new ByteBufferAccessor(bytes);
accessor.writeShort(version);
message.write(accessor, cache, version);
int messageSize = message.size(cache, version);
ByteBufferAccessor bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize));
message.write(bytes, cache, version);
bytes.flip();
return bytes;
return bytes.buffer();
}
public static byte[] toBytes(final short version, final Message message) {
ByteBuffer buffer = toByteBuffer(version, message);
public static ByteBuffer toVersionPrefixedByteBuffer(final short version, final Message message) {
ObjectSerializationCache cache = new ObjectSerializationCache();
int messageSize = message.size(cache, version);
ByteBufferAccessor bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize + 2));
bytes.writeShort(version);
message.write(bytes, cache, version);
bytes.flip();
return bytes.buffer();
}
public static byte[] toVersionPrefixedBytes(final short version, final Message message) {
ByteBuffer buffer = toVersionPrefixedByteBuffer(version, message);
// take the inner array directly if it is full with data
if (buffer.hasArray() &&
buffer.arrayOffset() == 0 &&

View File

@ -22,7 +22,6 @@ import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MultiRecordsSend;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.requests.ResponseHeader;
import org.apache.kafka.common.utils.ByteUtils;
@ -38,12 +37,11 @@ import java.util.Queue;
* allocating new space for "zero-copy" fields (see {@link #writeByteBuffer(ByteBuffer)}
* and {@link #writeRecords(BaseRecords)}).
*
* See {@link org.apache.kafka.common.requests.EnvelopeRequest#toSend(String, RequestHeader)}
* See {@link org.apache.kafka.common.requests.EnvelopeRequest#toSend(RequestHeader)}
* for example usage.
*/
public class SendBuilder implements Writable {
private final ByteBuffer buffer;
private final String destinationId;
private final Queue<Send> sends = new ArrayDeque<>(1);
private long sizeOfSends = 0;
@ -51,8 +49,7 @@ public class SendBuilder implements Writable {
private final List<ByteBuffer> buffers = new ArrayList<>();
private long sizeOfBuffers = 0;
SendBuilder(String destinationId, int size) {
this.destinationId = destinationId;
SendBuilder(int size) {
this.buffer = ByteBuffer.allocate(size);
this.buffer.mark();
}
@ -131,7 +128,7 @@ public class SendBuilder implements Writable {
/**
* Write a record set. The underlying record data will be retained
* in the result of {@link #build()}. See {@link BaseRecords#toSend(String)}.
* in the result of {@link #build()}. See {@link BaseRecords#toSend()}.
*
* @param records the records to write
*/
@ -142,7 +139,7 @@ public class SendBuilder implements Writable {
addBuffer(((MemoryRecords) records).buffer());
} else {
flushPendingSend();
addSend(records.toSend(destinationId));
addSend(records.toSend());
}
}
@ -150,7 +147,7 @@ public class SendBuilder implements Writable {
flushPendingBuffer();
if (!buffers.isEmpty()) {
ByteBuffer[] byteBufferArray = buffers.toArray(new ByteBuffer[0]);
addSend(new ByteBufferSend(destinationId, byteBufferArray, sizeOfBuffers));
addSend(new ByteBufferSend(byteBufferArray, sizeOfBuffers));
clearBuffers();
}
}
@ -175,17 +172,15 @@ public class SendBuilder implements Writable {
if (sends.size() == 1) {
return sends.poll();
} else {
return new MultiRecordsSend(destinationId, sends, sizeOfSends);
return new MultiRecordsSend(sends, sizeOfSends);
}
}
public static Send buildRequestSend(
String destination,
RequestHeader header,
Message apiRequest
) {
return buildSend(
destination,
header.data(),
header.headerVersion(),
apiRequest,
@ -194,13 +189,11 @@ public class SendBuilder implements Writable {
}
public static Send buildResponseSend(
String destination,
ResponseHeader header,
Message apiResponse,
short apiVersion
) {
return buildSend(
destination,
header.data(),
header.headerVersion(),
apiResponse,
@ -209,22 +202,22 @@ public class SendBuilder implements Writable {
}
private static Send buildSend(
String destination,
Message header,
short headerVersion,
Message apiMessage,
short apiVersion
) {
ObjectSerializationCache serializationCache = new ObjectSerializationCache();
MessageSizeAccumulator messageSize = RequestUtils.size(serializationCache, header, headerVersion, apiMessage, apiVersion);
int totalSize = messageSize.totalSize();
int sizeExcludingZeroCopyFields = totalSize - messageSize.zeroCopySize();
MessageSizeAccumulator messageSize = new MessageSizeAccumulator();
header.addSize(messageSize, serializationCache, headerVersion);
apiMessage.addSize(messageSize, serializationCache, apiVersion);
SendBuilder builder = new SendBuilder(destination, sizeExcludingZeroCopyFields + 4);
builder.writeInt(totalSize);
SendBuilder builder = new SendBuilder(messageSize.sizeExcludingZeroCopy() + 4);
builder.writeInt(messageSize.totalSize());
header.write(builder, serializationCache, headerVersion);
apiMessage.write(builder, serializationCache, apiVersion);
return builder.build();
}

View File

@ -62,8 +62,8 @@ public abstract class AbstractRecords implements Records {
}
@Override
public DefaultRecordsSend toSend(String destination) {
return new DefaultRecordsSend(destination, this);
public DefaultRecordsSend toSend() {
return new DefaultRecordsSend(this);
}
private Iterator<Record> recordsIterator() {

View File

@ -30,5 +30,5 @@ public interface BaseRecords {
* Encapsulate this {@link BaseRecords} object into {@link RecordsSend}
* @return Initialized {@link RecordsSend} object
*/
RecordsSend<? extends BaseRecords> toSend(String destination);
RecordsSend<? extends BaseRecords> toSend();
}

View File

@ -20,12 +20,12 @@ import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
public class DefaultRecordsSend extends RecordsSend<Records> {
public DefaultRecordsSend(String destination, Records records) {
this(destination, records, records.sizeInBytes());
public DefaultRecordsSend(Records records) {
this(records, records.sizeInBytes());
}
public DefaultRecordsSend(String destination, Records records, int maxBytesToWrite) {
super(destination, records, maxBytesToWrite);
public DefaultRecordsSend(Records records, int maxBytesToWrite) {
super(records, maxBytesToWrite);
}
@Override

View File

@ -77,8 +77,8 @@ public class LazyDownConversionRecords implements BaseRecords {
}
@Override
public LazyDownConversionRecordsSend toSend(String destination) {
return new LazyDownConversionRecordsSend(destination, this);
public LazyDownConversionRecordsSend toSend() {
return new LazyDownConversionRecordsSend(this);
}
public TopicPartition topicPartition() {

View File

@ -39,8 +39,8 @@ public final class LazyDownConversionRecordsSend extends RecordsSend<LazyDownCon
private RecordsSend convertedRecordsWriter;
private Iterator<ConvertedRecords<?>> convertedRecordsIterator;
public LazyDownConversionRecordsSend(String destination, LazyDownConversionRecords records) {
super(destination, records, records.sizeInBytes());
public LazyDownConversionRecordsSend(LazyDownConversionRecords records) {
super(records, records.sizeInBytes());
convertedRecordsWriter = null;
recordConversionStats = new RecordConversionStats();
convertedRecordsIterator = records().iterator(MAX_READ_SIZE);
@ -90,7 +90,7 @@ public final class LazyDownConversionRecordsSend extends RecordsSend<LazyDownCon
convertedRecords = buildOverflowBatch(remaining);
}
convertedRecordsWriter = new DefaultRecordsSend(destination(), convertedRecords, Math.min(convertedRecords.sizeInBytes(), remaining));
convertedRecordsWriter = new DefaultRecordsSend(convertedRecords, Math.min(convertedRecords.sizeInBytes(), remaining));
}
return convertedRecordsWriter.writeTo(channel);
}

View File

@ -34,7 +34,6 @@ import java.util.Queue;
public class MultiRecordsSend implements Send {
private static final Logger log = LoggerFactory.getLogger(MultiRecordsSend.class);
private final String dest;
private final Queue<Send> sendQueue;
private final long size;
private Map<TopicPartition, RecordConversionStats> recordConversionStats;
@ -43,11 +42,10 @@ public class MultiRecordsSend implements Send {
private Send current;
/**
* Construct a MultiRecordsSend for the given destination from a queue of Send objects. The queue will be
* consumed as the MultiRecordsSend progresses (on completion, it will be empty).
* Construct a MultiRecordsSend from a queue of Send objects. The queue will be consumed as the MultiRecordsSend
* progresses (on completion, it will be empty).
*/
public MultiRecordsSend(String dest, Queue<Send> sends) {
this.dest = dest;
public MultiRecordsSend(Queue<Send> sends) {
this.sendQueue = sends;
long size = 0;
@ -58,8 +56,7 @@ public class MultiRecordsSend implements Send {
this.current = sendQueue.poll();
}
public MultiRecordsSend(String dest, Queue<Send> sends, long size) {
this.dest = dest;
public MultiRecordsSend(Queue<Send> sends, long size) {
this.sendQueue = sends;
this.size = size;
this.current = sendQueue.poll();
@ -70,11 +67,6 @@ public class MultiRecordsSend implements Send {
return size;
}
@Override
public String destination() {
return dest;
}
@Override
public boolean completed() {
return current == null;
@ -128,8 +120,7 @@ public class MultiRecordsSend implements Send {
@Override
public String toString() {
return "MultiRecordsSend(" +
"dest='" + dest + "'" +
", size=" + size +
"size=" + size +
", totalWritten=" + totalWritten +
')';
}

View File

@ -27,24 +27,17 @@ import java.nio.channels.GatheringByteChannel;
public abstract class RecordsSend<T extends BaseRecords> implements Send {
private static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.allocate(0);
private final String destination;
private final T records;
private final int maxBytesToWrite;
private int remaining;
private boolean pending = false;
protected RecordsSend(String destination, T records, int maxBytesToWrite) {
this.destination = destination;
protected RecordsSend(T records, int maxBytesToWrite) {
this.records = records;
this.maxBytesToWrite = maxBytesToWrite;
this.remaining = maxBytesToWrite;
}
@Override
public String destination() {
return destination;
}
@Override
public boolean completed() {
return remaining <= 0 && !pending;

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.protocol.SendBuilder;
@ -97,20 +98,15 @@ public abstract class AbstractRequest implements AbstractRequestResponse {
return apiKey;
}
public final Send toSend(String destination, RequestHeader header) {
return SendBuilder.buildRequestSend(destination, header, data());
}
// Visible for testing
public final ByteBuffer serializeWithHeader(RequestHeader header) {
return RequestUtils.serialize(header.data(), header.headerVersion(), data(), version);
public final Send toSend(RequestHeader header) {
return SendBuilder.buildRequestSend(header, data());
}
protected abstract Message data();
// Visible for testing
public final ByteBuffer serializeBody() {
return RequestUtils.serialize(null, (short) 0, data(), version);
public final ByteBuffer serialize() {
return MessageUtil.toByteBuffer(data(), version);
}
// Visible for testing

View File

@ -20,6 +20,7 @@ import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.protocol.SendBuilder;
import java.nio.ByteBuffer;
@ -27,7 +28,6 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@ -40,25 +40,20 @@ public abstract class AbstractResponse implements AbstractRequestResponse {
this.apiKey = apiKey;
}
public final Send toSend(String destination, ResponseHeader header, short version) {
return SendBuilder.buildResponseSend(destination, header, data(), version);
public final Send toSend(ResponseHeader header, short version) {
return SendBuilder.buildResponseSend(header, data(), version);
}
/**
* Visible for testing, typically {@link #toSend(String, ResponseHeader, short)} should be used instead.
* Serializes header and body without prefixing with size (unlike `toSend`, which does include a size prefix).
*/
public final ByteBuffer serializeWithHeader(short version, int correlationId) {
return serializeWithHeader(new ResponseHeader(correlationId, apiKey.responseHeaderVersion(version)), version);
}
final ByteBuffer serializeWithHeader(ResponseHeader header, short version) {
Objects.requireNonNull(header, "header should not be null");
return RequestUtils.serialize(header.data(), header.headerVersion(), data(), version);
}
// Visible for testing
final ByteBuffer serializeBody(short version) {
return RequestUtils.serialize(null, (short) 0, data(), version);
final ByteBuffer serialize(short version) {
return MessageUtil.toByteBuffer(data(), version);
}
/**

View File

@ -108,7 +108,7 @@ public class RequestContext implements AuthorizableRequestContext {
* over the network.
*/
public Send buildResponseSend(AbstractResponse body) {
return body.toSend(connectionId, header.toResponseHeader(), apiVersion());
return body.toSend(header.toResponseHeader(), apiVersion());
}
/**

View File

@ -19,7 +19,6 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.ProduceRequestData;
import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Message;
import org.apache.kafka.common.protocol.MessageSizeAccumulator;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.RecordBatch;
@ -76,36 +75,22 @@ public final class RequestUtils {
return new AbstractMap.SimpleEntry<>(hasIdempotentRecords, hasTransactionalRecords);
}
public static MessageSizeAccumulator size(
ObjectSerializationCache serializationCache,
Message header,
short headerVersion,
Message apiMessage,
short apiVersion
) {
MessageSizeAccumulator messageSize = new MessageSizeAccumulator();
if (header != null)
header.addSize(messageSize, serializationCache, headerVersion);
apiMessage.addSize(messageSize, serializationCache, apiVersion);
return messageSize;
}
public static ByteBuffer serialize(
Message header,
short headerVersion,
Message apiMessage,
short apiVersion
) {
ObjectSerializationCache serializationCache = new ObjectSerializationCache();
MessageSizeAccumulator messageSize = RequestUtils.size(serializationCache, header, headerVersion, apiMessage, apiVersion);
ObjectSerializationCache cache = new ObjectSerializationCache();
ByteBuffer buffer = ByteBuffer.allocate(messageSize.totalSize());
ByteBufferAccessor bufferWritable = new ByteBufferAccessor(buffer);
if (header != null)
header.write(bufferWritable, serializationCache, headerVersion);
apiMessage.write(bufferWritable, serializationCache, apiVersion);
int headerSize = header.size(cache, headerVersion);
int messageSize = apiMessage.size(cache, apiVersion);
ByteBufferAccessor writable = new ByteBufferAccessor(ByteBuffer.allocate(headerSize + messageSize));
buffer.rewind();
return buffer;
header.write(writable, cache, headerVersion);
apiMessage.write(writable, cache, apiVersion);
writable.flip();
return writable.buffer();
}
}

View File

@ -28,8 +28,8 @@ import org.apache.kafka.common.message.RequestHeaderData;
import org.apache.kafka.common.message.SaslAuthenticateRequestData;
import org.apache.kafka.common.message.SaslHandshakeRequestData;
import org.apache.kafka.common.network.Authenticator;
import org.apache.kafka.common.network.ByteBufferSend;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.ReauthenticationContext;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.network.TransportLayer;
@ -243,7 +243,7 @@ public class SaslClientAuthenticator implements Authenticator {
case SEND_APIVERSIONS_REQUEST:
// Always use version 0 request since brokers treat requests with schema exceptions as GSSAPI tokens
ApiVersionsRequest apiVersionsRequest = new ApiVersionsRequest.Builder().build((short) 0);
send(apiVersionsRequest.toSend(node, nextRequestHeader(ApiKeys.API_VERSIONS, apiVersionsRequest.version())));
send(apiVersionsRequest.toSend(nextRequestHeader(ApiKeys.API_VERSIONS, apiVersionsRequest.version())));
setSaslState(SaslState.RECEIVE_APIVERSIONS_RESPONSE);
break;
case RECEIVE_APIVERSIONS_RESPONSE:
@ -325,7 +325,7 @@ public class SaslClientAuthenticator implements Authenticator {
private void sendHandshakeRequest(short version) throws IOException {
SaslHandshakeRequest handshakeRequest = createSaslHandshakeRequest(version);
send(handshakeRequest.toSend(node, nextRequestHeader(ApiKeys.SASL_HANDSHAKE, handshakeRequest.version())));
send(handshakeRequest.toSend(nextRequestHeader(ApiKeys.SASL_HANDSHAKE, handshakeRequest.version())));
}
private void sendInitialToken() throws IOException {
@ -435,12 +435,12 @@ public class SaslClientAuthenticator implements Authenticator {
ByteBuffer tokenBuf = ByteBuffer.wrap(saslToken);
Send send;
if (saslAuthenticateVersion == DISABLE_KAFKA_SASL_AUTHENTICATE_HEADER) {
send = new NetworkSend(node, tokenBuf);
send = ByteBufferSend.sizePrefixed(tokenBuf);
} else {
SaslAuthenticateRequestData data = new SaslAuthenticateRequestData()
.setAuthBytes(tokenBuf.array());
SaslAuthenticateRequest request = new SaslAuthenticateRequest.Builder(data).build(saslAuthenticateVersion);
send = request.toSend(node, nextRequestHeader(ApiKeys.SASL_AUTHENTICATE, saslAuthenticateVersion));
send = request.toSend(nextRequestHeader(ApiKeys.SASL_AUTHENTICATE, saslAuthenticateVersion));
}
send(send);
return true;

View File

@ -28,12 +28,12 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.message.SaslAuthenticateResponseData;
import org.apache.kafka.common.message.SaslHandshakeResponseData;
import org.apache.kafka.common.network.Authenticator;
import org.apache.kafka.common.network.ByteBufferSend;
import org.apache.kafka.common.network.ChannelBuilders;
import org.apache.kafka.common.network.ChannelMetadataRegistry;
import org.apache.kafka.common.network.ClientInformation;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.ReauthenticationContext;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.network.TransportLayer;
@ -408,7 +408,7 @@ public class SaslServerAuthenticator implements Authenticator {
reauthInfo.ensurePrincipalUnchanged(principal());
}
if (response != null) {
netOutBuffer = new NetworkSend(connectionId, ByteBuffer.wrap(response));
netOutBuffer = ByteBufferSend.sizePrefixed(ByteBuffer.wrap(response));
flushNetOutBufferAndUpdateInterestOps();
}
} else {

View File

@ -30,14 +30,14 @@ import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopi
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopicCollection;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.MessageTestUtil;
import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.test.MockClusterResourceListener;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.net.InetSocketAddress;
@ -69,7 +69,7 @@ public class MetadataTest {
new ClusterResourceListeners());
private static MetadataResponse emptyMetadataResponse() {
return TestUtils.metadataResponse(
return RequestTestUtils.metadataResponse(
Collections.emptyList(),
null,
-1,
@ -197,7 +197,7 @@ public class MetadataTest {
.setBrokers(new MetadataResponseBrokerCollection());
for (short version = ApiKeys.METADATA.oldestVersion(); version < 9; version++) {
ByteBuffer buffer = MessageTestUtil.messageToByteBuffer(data, version);
ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
MetadataResponse response = MetadataResponse.parse(buffer, version);
assertFalse(response.hasReliableLeaderEpochs());
metadata.updateWithCurrentRequestVersion(response, false, 100);
@ -207,7 +207,7 @@ public class MetadataTest {
}
for (short version = 9; version <= ApiKeys.METADATA.latestVersion(); version++) {
ByteBuffer buffer = MessageTestUtil.messageToByteBuffer(data, version);
ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
MetadataResponse response = MetadataResponse.parse(buffer, version);
assertTrue(response.hasReliableLeaderEpochs());
metadata.updateWithCurrentRequestVersion(response, false, 100);
@ -297,7 +297,7 @@ public class MetadataTest {
Map<String, Integer> partitionCounts = new HashMap<>();
partitionCounts.put("topic", 1);
partitionCounts.put("topic1", 1);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, partitionCounts);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, partitionCounts);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 100);
assertEquals("MockClusterResourceListener did not get cluster metadata correctly",
@ -315,7 +315,7 @@ public class MetadataTest {
boolean[] updateResult = {true, false, false, false, false, true, false, false, false, true};
TopicPartition tp = new TopicPartition("topic", 0);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), Collections.singletonMap("topic", 1), _tp -> 0);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
@ -345,7 +345,7 @@ public class MetadataTest {
assertFalse(metadata.lastSeenLeaderEpoch(tp).isPresent());
// Metadata with newer epoch is handled
metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 10);
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 10);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 1L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 10));
@ -361,12 +361,12 @@ public class MetadataTest {
assertTrue(metadata.updateLastSeenEpochIfNewer(tp, 12));
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 12);
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 12);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 2L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
// Don't overwrite metadata with older epoch
metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 11);
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1), _tp -> 11);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 3L);
assertOptional(metadata.lastSeenLeaderEpoch(tp), leaderAndEpoch -> assertEquals(leaderAndEpoch.intValue(), 12));
}
@ -381,7 +381,7 @@ public class MetadataTest {
// First epoch seen, accept it
{
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNotNull(metadata.fetch().partition(tp));
assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent());
@ -390,7 +390,7 @@ public class MetadataTest {
// Fake an empty ISR, but with an older epoch, should reject it
{
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99,
(error, partition, leader, leaderEpoch, replicas, isr, offlineReplicas) ->
new MetadataResponse.PartitionMetadata(error, partition, leader,
leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion());
@ -401,7 +401,7 @@ public class MetadataTest {
// Fake an empty ISR, with same epoch, accept it
{
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100,
(error, partition, leader, leaderEpoch, replicas, isr, offlineReplicas) ->
new MetadataResponse.PartitionMetadata(error, partition, leader,
leaderEpoch, replicas, Collections.emptyList(), offlineReplicas), ApiKeys.METADATA.latestVersion());
@ -412,7 +412,7 @@ public class MetadataTest {
// Empty metadata response, should not keep old partition but should keep the last-seen epoch
{
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.emptyMap());
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 20L);
assertNull(metadata.fetch().partition(tp));
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
@ -420,7 +420,7 @@ public class MetadataTest {
// Back in the metadata, with old epoch, should not get added
{
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 99);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNull(metadata.fetch().partition(tp));
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 100);
@ -438,7 +438,7 @@ public class MetadataTest {
assertFalse(metadata.updateLastSeenEpochIfNewer(tp, 99));
// Update epoch to 100
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 100);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
assertNotNull(metadata.fetch().partition(tp));
assertTrue(metadata.lastSeenLeaderEpoch(tp).isPresent());
@ -461,7 +461,7 @@ public class MetadataTest {
assertEquals(metadata.lastSeenLeaderEpoch(tp).get().longValue(), 101);
// Metadata with equal or newer epoch is accepted
metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 101);
metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, _tp -> 101);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 30L);
assertNotNull(metadata.fetch().partition(tp));
assertEquals(Objects.requireNonNull(metadata.fetch().partitionCountForTopic("topic-1")).longValue(), 5);
@ -472,7 +472,7 @@ public class MetadataTest {
@Test
public void testNoEpoch() {
metadata.updateWithCurrentRequestVersion(emptyMetadataResponse(), false, 0L);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1));
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), Collections.singletonMap("topic-1", 1));
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 10L);
TopicPartition tp = new TopicPartition("topic-1", 0);
@ -501,7 +501,7 @@ public class MetadataTest {
errors.put("topic3", Errors.INVALID_TOPIC_EXCEPTION);
errors.put("topic4", Errors.TOPIC_AUTHORIZATION_FAILED);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 4, errors, counts);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 4, errors, counts);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L);
Cluster cluster = metadata.fetch();
@ -536,7 +536,7 @@ public class MetadataTest {
metadata.requestUpdate();
Metadata.MetadataRequestAndVersion versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
assertFalse(metadata.updateRequested());
// bump the request version for new topics added to the metadata
@ -546,7 +546,7 @@ public class MetadataTest {
versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
metadata.requestUpdateForNewTopics();
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
// metadata update is still needed
assertTrue(metadata.updateRequested());
@ -554,7 +554,7 @@ public class MetadataTest {
// the next update will resolve it
versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
assertFalse(metadata.updateRequested());
}
@ -576,7 +576,7 @@ public class MetadataTest {
Metadata.MetadataRequestAndVersion versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
assertFalse(versionAndBuilder.isPartialUpdate);
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
assertFalse(metadata.updateRequested());
// Request a metadata update for a new topic. This should perform a partial metadata update.
@ -584,7 +584,7 @@ public class MetadataTest {
versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
assertTrue(versionAndBuilder.isPartialUpdate);
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, time.milliseconds());
assertFalse(metadata.updateRequested());
// Request both types of metadata updates. This should always perform a full update.
@ -593,7 +593,7 @@ public class MetadataTest {
versionAndBuilder = metadata.newMetadataRequestAndVersion(time.milliseconds());
assertFalse(versionAndBuilder.isPartialUpdate);
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), false, time.milliseconds());
assertFalse(metadata.updateRequested());
// Request only a partial metadata update, but elapse enough time such that a full refresh is needed.
@ -602,7 +602,7 @@ public class MetadataTest {
versionAndBuilder = metadata.newMetadataRequestAndVersion(refreshTimeMs);
assertFalse(versionAndBuilder.isPartialUpdate);
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, refreshTimeMs);
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)), true, refreshTimeMs);
assertFalse(metadata.updateRequested());
// Request two partial metadata updates that are overlapping.
@ -614,10 +614,10 @@ public class MetadataTest {
assertTrue(overlappingVersionAndBuilder.isPartialUpdate);
assertTrue(metadata.updateRequested());
metadata.update(versionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic-1", 1)), true, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic-1", 1)), true, time.milliseconds());
assertTrue(metadata.updateRequested());
metadata.update(overlappingVersionAndBuilder.requestVersion,
TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic-2", 1)), true, time.milliseconds());
RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic-2", 1)), true, time.milliseconds());
assertFalse(metadata.updateRequested());
}
@ -626,7 +626,7 @@ public class MetadataTest {
Time time = new MockTime();
String invalidTopic = "topic dfsa";
MetadataResponse invalidTopicResponse = TestUtils.metadataUpdateWith("clusterId", 1,
MetadataResponse invalidTopicResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap(invalidTopic, Errors.INVALID_TOPIC_EXCEPTION), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(invalidTopicResponse, false, time.milliseconds());
@ -649,7 +649,7 @@ public class MetadataTest {
Time time = new MockTime();
String invalidTopic = "foo";
MetadataResponse unauthorizedTopicResponse = TestUtils.metadataUpdateWith("clusterId", 1,
MetadataResponse unauthorizedTopicResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap(invalidTopic, Errors.TOPIC_AUTHORIZATION_FAILED), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(unauthorizedTopicResponse, false, time.milliseconds());
@ -674,7 +674,7 @@ public class MetadataTest {
topicErrors.put("invalidTopic", Errors.INVALID_TOPIC_EXCEPTION);
topicErrors.put("sensitiveTopic1", Errors.TOPIC_AUTHORIZATION_FAILED);
topicErrors.put("sensitiveTopic2", Errors.TOPIC_AUTHORIZATION_FAILED);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("clusterId", 1, topicErrors, Collections.emptyMap());
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1, topicErrors, Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
TopicAuthorizationException e1 = assertThrows(TopicAuthorizationException.class,
@ -708,7 +708,7 @@ public class MetadataTest {
Node node0 = new Node(0, "localhost", 9092);
Node node1 = new Node(1, "localhost", 9093);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 2, Collections.emptyMap(), partitionCounts, _tp -> 99,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 2, Collections.emptyMap(), partitionCounts, _tp -> 99,
(error, partition, leader, leaderEpoch, replicas, isr, offlineReplicas) ->
new MetadataResponse.PartitionMetadata(error, partition, Optional.of(node0.id()), leaderEpoch,
Collections.singletonList(node0.id()), Collections.emptyList(),
@ -832,7 +832,7 @@ public class MetadataTest {
"keepValidTopic")));
MetadataResponse metadataResponse =
TestUtils.metadataUpdateWith(oldClusterId, oldNodes, oldTopicErrors, oldTopicPartitionCounts, _tp -> 100);
RequestTestUtils.metadataUpdateWith(oldClusterId, oldNodes, oldTopicErrors, oldTopicPartitionCounts, _tp -> 100);
metadata.updateWithCurrentRequestVersion(metadataResponse, true, time.milliseconds());
// Update the metadata to add a new topic variant, "new", which will be retained with "keep". Note this
@ -863,7 +863,7 @@ public class MetadataTest {
"keepValidTopic",
"newValidTopic")));
metadataResponse = TestUtils.metadataUpdateWith(newClusterId, newNodes, newTopicErrors, newTopicPartitionCounts, _tp -> 200);
metadataResponse = RequestTestUtils.metadataUpdateWith(newClusterId, newNodes, newTopicErrors, newTopicPartitionCounts, _tp -> 200);
metadata.updateWithCurrentRequestVersion(metadataResponse, true, time.milliseconds());
cluster = metadata.fetch();
@ -878,7 +878,7 @@ public class MetadataTest {
// Perform another metadata update, but this time all topic metadata should be cleared.
retainTopics.set(Collections.emptySet());
metadataResponse = TestUtils.metadataUpdateWith(newClusterId, newNodes, newTopicErrors, newTopicPartitionCounts, _tp -> 300);
metadataResponse = RequestTestUtils.metadataUpdateWith(newClusterId, newNodes, newTopicErrors, newTopicPartitionCounts, _tp -> 300);
metadata.updateWithCurrentRequestVersion(metadataResponse, true, time.milliseconds());
cluster = metadata.fetch();

View File

@ -37,6 +37,7 @@ import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.ProduceRequest;
import org.apache.kafka.common.requests.ProduceResponse;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
@ -199,8 +200,7 @@ public class NetworkClientTest {
networkClient.poll(1, time.milliseconds());
assertEquals(1, networkClient.inFlightRequestCount());
ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData());
ByteBuffer buffer = produceResponse.serializeWithHeader(requestVersion, request.correlationId());
buffer.rewind();
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId());
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
List<ClientResponse> responses = networkClient.poll(1, time.milliseconds());
assertEquals(1, responses.size());
@ -211,7 +211,7 @@ public class NetworkClientTest {
}
private void delayedApiVersionsResponse(int correlationId, short version, ApiVersionsResponse response) {
ByteBuffer buffer = response.serializeWithHeader(version, correlationId);
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(response, version, correlationId);
selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer)));
}
@ -508,8 +508,7 @@ public class NetworkClientTest {
client.poll(1, time.milliseconds());
int throttleTime = 100;
ProduceResponse produceResponse = new ProduceResponse(new ProduceResponseData().setThrottleTimeMs(throttleTime));
ByteBuffer buffer = produceResponse.serializeWithHeader(requestVersion, request.correlationId());
buffer.rewind();
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(produceResponse, requestVersion, request.correlationId());
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
client.poll(1, time.milliseconds());
@ -589,8 +588,7 @@ public class NetworkClientTest {
}
private void sendResponse(AbstractResponse response, short version, int correlationId) {
ByteBuffer buffer = response.serializeWithHeader(version, correlationId);
buffer.rewind();
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(response, version, correlationId);
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
}
@ -652,7 +650,7 @@ public class NetworkClientTest {
public void testAuthenticationFailureWithInFlightMetadataRequest() {
int refreshBackoffMs = 50;
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith(2, Collections.emptyMap());
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(2, Collections.emptyMap());
Metadata metadata = new Metadata(refreshBackoffMs, 5000, new LogContext(), new ClusterResourceListeners());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
@ -683,7 +681,7 @@ public class NetworkClientTest {
RequestHeader header = parseHeader(requestBuffer);
assertEquals(ApiKeys.METADATA, header.apiKey());
ByteBuffer responseBuffer = metadataResponse.serializeWithHeader(header.apiVersion(), header.correlationId());
ByteBuffer responseBuffer = RequestTestUtils.serializeResponseWithHeader(metadataResponse, header.apiVersion(), header.correlationId());
selector.delayedReceive(new DelayedReceive(node1.idString(), new NetworkReceive(node1.idString(), responseBuffer)));
int initialUpdateVersion = metadata.updateVersion();

View File

@ -159,6 +159,7 @@ import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetDeleteResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.UpdateFeaturesRequest;
import org.apache.kafka.common.requests.UpdateFeaturesResponse;
import org.apache.kafka.common.resource.PatternType;
@ -561,7 +562,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
env.kafkaClient().prepareResponse(request -> request instanceof MetadataRequest, null, true);
env.kafkaClient().prepareResponse(request -> request instanceof MetadataRequest,
TestUtils.metadataResponse(discoveredCluster.nodes(), discoveredCluster.clusterResource().clusterId(),
RequestTestUtils.metadataResponse(discoveredCluster.nodes(), discoveredCluster.clusterResource().clusterId(),
1, Collections.emptyList()));
env.kafkaClient().prepareResponse(body -> body instanceof CreateTopicsRequest,
prepareCreateTopicsResponse("myTopic", Errors.NONE));
@ -586,7 +587,7 @@ public class KafkaAdminClientTest {
Cluster discoveredCluster = mockCluster(3, 0);
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
env.kafkaClient().prepareResponse(body -> body instanceof MetadataRequest,
TestUtils.metadataResponse(discoveredCluster.nodes(), discoveredCluster.clusterResource().clusterId(),
RequestTestUtils.metadataResponse(discoveredCluster.nodes(), discoveredCluster.clusterResource().clusterId(),
1, Collections.emptyList()));
env.kafkaClient().prepareResponse(body -> body instanceof CreateTopicsRequest,
prepareCreateTopicsResponse("myTopic", Errors.NONE));
@ -703,7 +704,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponseFrom(
prepareCreateTopicsResponse("myTopic", Errors.NOT_CONTROLLER),
env.cluster().nodeById(0));
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(env.cluster().nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(),
1,
Collections.<MetadataResponse.TopicMetadata>emptyList()));
@ -1047,7 +1048,7 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(null, true);
// The next one succeeds and gives us the controller id
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(initializedCluster.nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(initializedCluster.nodes(),
initializedCluster.clusterResource().clusterId(),
initializedCluster.controller().id(),
Collections.emptyList()));
@ -1057,7 +1058,7 @@ public class KafkaAdminClientTest {
MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(
Errors.NONE, new TopicPartition(topic, 0), Optional.of(leader.id()), Optional.of(10),
singletonList(leader.id()), singletonList(leader.id()), singletonList(leader.id()));
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(initializedCluster.nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(initializedCluster.nodes(),
initializedCluster.clusterResource().clusterId(), 1,
singletonList(new MetadataResponse.TopicMetadata(Errors.NONE, topic, false,
singletonList(partitionMetadata), MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED))));
@ -1859,7 +1860,7 @@ public class KafkaAdminClientTest {
topics.add(new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, false,
Collections.emptyList()));
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(env.cluster().nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(), env.cluster().controller().id(), topics));
Map<TopicPartition, RecordsToDelete> recordsToDelete = new HashMap<>();
@ -1892,7 +1893,7 @@ public class KafkaAdminClientTest {
List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>();
topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, topic, false, partitionMetadata));
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(env.cluster().nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(), env.cluster().controller().id(), topicMetadata));
env.kafkaClient().prepareResponseFrom(new DeleteRecordsResponse(new DeleteRecordsResponseData().setTopics(
@ -1980,7 +1981,7 @@ public class KafkaAdminClientTest {
t.add(new MetadataResponse.TopicMetadata(Errors.NONE, "my_topic", false, p));
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(cluster.nodes(), cluster.clusterResource().clusterId(), cluster.controller().id(), t));
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(cluster.nodes(), cluster.clusterResource().clusterId(), cluster.controller().id(), t));
env.kafkaClient().prepareResponse(new DeleteRecordsResponse(m));
Map<TopicPartition, RecordsToDelete> recordsToDelete = new HashMap<>();
@ -2043,21 +2044,23 @@ public class KafkaAdminClientTest {
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
// Prepare the metadata response used for the first describe cluster
MetadataResponse response = TestUtils.metadataResponse(0,
MetadataResponse response = RequestTestUtils.metadataResponse(0,
env.cluster().nodes(),
env.cluster().clusterResource().clusterId(),
2,
Collections.emptyList(),
MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED);
MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED,
ApiKeys.METADATA.latestVersion());
env.kafkaClient().prepareResponse(response);
// Prepare the metadata response used for the second describe cluster
MetadataResponse response2 = TestUtils.metadataResponse(0,
MetadataResponse response2 = RequestTestUtils.metadataResponse(0,
env.cluster().nodes(),
env.cluster().clusterResource().clusterId(),
3,
Collections.emptyList(),
1 << AclOperation.DESCRIBE.code() | 1 << AclOperation.ALTER.code());
1 << AclOperation.DESCRIBE.code() | 1 << AclOperation.ALTER.code(),
ApiKeys.METADATA.latestVersion());
env.kafkaClient().prepareResponse(response2);
// Test DescribeCluster with the authorized operations omitted.
@ -2083,14 +2086,14 @@ public class KafkaAdminClientTest {
// Empty metadata response should be retried
env.kafkaClient().prepareResponse(
TestUtils.metadataResponse(
RequestTestUtils.metadataResponse(
Collections.emptyList(),
env.cluster().clusterResource().clusterId(),
-1,
Collections.emptyList()));
env.kafkaClient().prepareResponse(
TestUtils.metadataResponse(
RequestTestUtils.metadataResponse(
env.cluster().nodes(),
env.cluster().clusterResource().clusterId(),
env.cluster().controller().id(),
@ -2196,7 +2199,7 @@ public class KafkaAdminClientTest {
// Empty metadata causes the request to fail since we have no list of brokers
// to send the ListGroups requests to
env.kafkaClient().prepareResponse(
TestUtils.metadataResponse(
RequestTestUtils.metadataResponse(
Collections.emptyList(),
env.cluster().clusterResource().clusterId(),
-1,
@ -3457,7 +3460,7 @@ public class KafkaAdminClientTest {
.setName("B")
.setPartitions(Collections.singletonList(normalPartitionResponse)))
);
MetadataResponse controllerNodeResponse = TestUtils.metadataResponse(env.cluster().nodes(),
MetadataResponse controllerNodeResponse = RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(), 1, Collections.emptyList());
AlterPartitionReassignmentsResponseData normalResponse =
new AlterPartitionReassignmentsResponseData()
@ -3585,7 +3588,7 @@ public class KafkaAdminClientTest {
ListPartitionReassignmentsResponseData notControllerData = new ListPartitionReassignmentsResponseData()
.setErrorCode(Errors.NOT_CONTROLLER.code())
.setErrorMessage(Errors.NOT_CONTROLLER.message());
MetadataResponse controllerNodeResponse = TestUtils.metadataResponse(env.cluster().nodes(),
MetadataResponse controllerNodeResponse = RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(), 1, Collections.emptyList());
ListPartitionReassignmentsResponseData reassignmentsData = new ListPartitionReassignmentsResponseData()
.setTopics(Arrays.asList(tp1Reassignment, tp2Reassignment));
@ -4042,7 +4045,7 @@ public class KafkaAdminClientTest {
0),
env.cluster().nodeById(0));
final int controllerId = 1;
env.kafkaClient().prepareResponse(TestUtils.metadataResponse(env.cluster().nodes(),
env.kafkaClient().prepareResponse(RequestTestUtils.metadataResponse(env.cluster().nodes(),
env.cluster().clusterResource().clusterId(),
controllerId,
Collections.emptyList()));

View File

@ -79,6 +79,7 @@ import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.SyncGroupResponse;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.Deserializer;
@ -629,7 +630,7 @@ public class KafkaConsumerTest {
}
private void initMetadata(MockClient mockClient, Map<String, Integer> partitionCounts) {
MetadataResponse initialMetadata = TestUtils.metadataUpdateWith(1, partitionCounts);
MetadataResponse initialMetadata = RequestTestUtils.metadataUpdateWith(1, partitionCounts);
mockClient.updateMetadata(initialMetadata);
}
@ -900,7 +901,7 @@ public class KafkaConsumerTest {
consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer));
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith(1, partitionCounts));
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith(1, partitionCounts));
consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE));
@ -939,7 +940,7 @@ public class KafkaConsumerTest {
consumer.subscribe(Pattern.compile(otherTopic), getConsumerRebalanceListener(consumer));
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith(1, partitionCounts));
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith(1, partitionCounts));
prepareRebalance(client, node, singleton(otherTopic), assignor, singletonList(otherTopicPartition), coordinator);
consumer.poll(Duration.ZERO);
@ -1685,7 +1686,7 @@ public class KafkaConsumerTest {
consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer));
Node coordinator = prepareRebalance(client, node, assignor, singletonList(tp0), null);
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith(1, Collections.singletonMap(topic, 1)));
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap(topic, 1)));
consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE));
@ -2408,7 +2409,7 @@ public class KafkaConsumerTest {
List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>();
topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION,
invalidTopicName, false, Collections.emptyList()));
MetadataResponse updateResponse = TestUtils.metadataResponse(cluster.nodes(),
MetadataResponse updateResponse = RequestTestUtils.metadataResponse(cluster.nodes(),
cluster.clusterResource().clusterId(),
cluster.controller().id(),
topicMetadata);

View File

@ -44,6 +44,7 @@ import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.JoinGroupResponse;
import org.apache.kafka.common.requests.LeaveGroupRequest;
import org.apache.kafka.common.requests.LeaveGroupResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.SyncGroupRequest;
import org.apache.kafka.common.requests.SyncGroupResponse;
import org.apache.kafka.common.utils.LogContext;
@ -128,7 +129,7 @@ public class AbstractCoordinatorTest {
HEARTBEAT_INTERVAL_MS);
metrics = new Metrics(mockTime);
mockClient.updateMetadata(TestUtils.metadataUpdateWith(1, emptyMap()));
mockClient.updateMetadata(RequestTestUtils.metadataUpdateWith(1, emptyMap()));
this.node = metadata.fetch().nodes().get(0);
this.coordinatorNode = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port());

View File

@ -67,6 +67,7 @@ import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.OffsetCommitRequest;
import org.apache.kafka.common.requests.OffsetCommitResponse;
import org.apache.kafka.common.requests.OffsetFetchResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.SyncGroupRequest;
import org.apache.kafka.common.requests.SyncGroupResponse;
import org.apache.kafka.common.utils.LogContext;
@ -143,7 +144,7 @@ public class ConsumerCoordinatorTest {
private final String consumerId = "consumer";
private MockClient client;
private MetadataResponse metadataResponse = TestUtils.metadataUpdateWith(1, new HashMap<String, Integer>() {
private MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap<String, Integer>() {
{
put(topic1, 1);
put(topic2, 1);
@ -821,7 +822,7 @@ public class ConsumerCoordinatorTest {
// partially update the metadata with one topic first,
// let the leader to refresh metadata during assignment
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@ -859,7 +860,7 @@ public class ConsumerCoordinatorTest {
final List<TopicPartition> owned = Collections.emptyList();
final List<TopicPartition> oldAssigned = singletonList(t1p);
subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener);
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
coordinator.maybeUpdateSubscriptionMetadata();
assertEquals(singleton(topic1), subscriptions.subscription());
@ -878,7 +879,7 @@ public class ConsumerCoordinatorTest {
final Map<String, Integer> updatedPartitions = new HashMap<>();
for (String topic : updatedSubscription)
updatedPartitions.put(topic, 1);
client.updateMetadata(TestUtils.metadataUpdateWith(1, updatedPartitions));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, updatedPartitions));
return true;
}, syncGroupResponse(oldAssigned, Errors.NONE));
coordinator.poll(time.timer(Long.MAX_VALUE));
@ -913,7 +914,7 @@ public class ConsumerCoordinatorTest {
}, joinGroupLeaderResponse(2, consumerId, updatedSubscriptions, Errors.NONE));
// update the metadata again back to topic1
client.prepareResponse(body -> {
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
return true;
}, syncGroupResponse(newAssigned, Errors.NONE));
@ -969,7 +970,7 @@ public class ConsumerCoordinatorTest {
// Set up a non-leader consumer with pattern subscription and a cluster containing one topic matching the
// pattern.
subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener);
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
coordinator.maybeUpdateSubscriptionMetadata();
assertEquals(singleton(topic1), subscriptions.subscription());
@ -1013,7 +1014,7 @@ public class ConsumerCoordinatorTest {
final List<String> topics = Arrays.asList(topic1, topic2);
final List<TopicPartition> partitions = Arrays.asList(t1p, t2p);
subscriptions.subscribe(toSet(topics), rebalanceListener);
client.updateMetadata(TestUtils.metadataUpdateWith(1,
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1,
Utils.mkMap(Utils.mkEntry(topic1, 1), Utils.mkEntry(topic2, 1))));
coordinator.maybeUpdateSubscriptionMetadata();
@ -1036,12 +1037,12 @@ public class ConsumerCoordinatorTest {
assertEquals(1, rebalanceListener.assignedCount);
// Change metadata to trigger rebalance.
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
coordinator.poll(time.timer(0));
// Revert metadata to original value. Fail pending JoinGroup. Another
// JoinGroup should be sent, which will be completed successfully.
client.updateMetadata(TestUtils.metadataUpdateWith(1,
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1,
Utils.mkMap(Utils.mkEntry(topic1, 1), Utils.mkEntry(topic2, 1))));
client.respond(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NOT_COORDINATOR));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
@ -1170,7 +1171,7 @@ public class ConsumerCoordinatorTest {
// partially update the metadata with one topic first,
// let the leader to refresh metadata during assignment
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@ -1377,7 +1378,7 @@ public class ConsumerCoordinatorTest {
assertFalse(coordinator.rejoinNeededOrPending());
// a new partition is added to the topic
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 2)), false, time.milliseconds());
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 2)), false, time.milliseconds());
coordinator.maybeUpdateSubscriptionMetadata();
// we should detect the change and ask for reassignment
@ -1397,7 +1398,7 @@ public class ConsumerCoordinatorTest {
subscriptions.subscribe(new HashSet<>(topics), rebalanceListener);
// we only have metadata for one topic initially
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@ -1416,7 +1417,7 @@ public class ConsumerCoordinatorTest {
Map<String, Integer> topicPartitionCounts = new HashMap<>();
topicPartitionCounts.put(topic1, 1);
topicPartitionCounts.put(topic2, 1);
client.updateMetadata(TestUtils.metadataUpdateWith(1, topicPartitionCounts));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, topicPartitionCounts));
return true;
}
return false;
@ -1441,7 +1442,7 @@ public class ConsumerCoordinatorTest {
public void testSubscriptionChangeWithAuthorizationFailure() {
// Subscribe to two topics of which only one is authorized and verify that metadata failure is propagated.
subscriptions.subscribe(Utils.mkSet(topic1, topic2), rebalanceListener);
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith("kafka-cluster", 1,
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
Collections.singletonMap(topic2, Errors.TOPIC_AUTHORIZATION_FAILED), singletonMap(topic1, 1)));
assertThrows(TopicAuthorizationException.class, () -> coordinator.poll(time.timer(Long.MAX_VALUE)));
@ -1457,7 +1458,7 @@ public class ConsumerCoordinatorTest {
// references to topic2 have been removed from SubscriptionState.
subscriptions.subscribe(Utils.mkSet(topic1), rebalanceListener);
assertEquals(Collections.singleton(topic1), subscriptions.metadataTopics());
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith("kafka-cluster", 1,
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
Collections.emptyMap(), singletonMap(topic1, 1)));
Map<String, List<String>> memberSubscriptions = singletonMap(consumerId, singletonList(topic1));
@ -1490,7 +1491,7 @@ public class ConsumerCoordinatorTest {
subscriptions.subscribe(topics, rebalanceListener);
// we only have metadata for one topic initially
client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic1, 1)));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@ -1537,7 +1538,7 @@ public class ConsumerCoordinatorTest {
else
subscriptions.subscribe(singleton(topic1), rebalanceListener);
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith("kafka-cluster", 1,
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
Collections.singletonMap(topic1, Errors.UNKNOWN_TOPIC_OR_PARTITION), Collections.emptyMap()));
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
@ -1558,7 +1559,7 @@ public class ConsumerCoordinatorTest {
for (String topic : unavailableTopicsInLastMetadata)
topicErrors.put(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION);
client.prepareMetadataUpdate(TestUtils.metadataUpdateWith("kafka-cluster", 1,
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
topicErrors, singletonMap(topic1, 1)));
consumerClient.poll(time.timer(0));
@ -1595,7 +1596,7 @@ public class ConsumerCoordinatorTest {
MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE,
Topic.GROUP_METADATA_TOPIC_NAME, true, singletonList(partitionMetadata));
client.updateMetadata(TestUtils.metadataResponse(singletonList(node), "clusterId", node.id(),
client.updateMetadata(RequestTestUtils.metadataResponse(singletonList(node), "clusterId", node.id(),
singletonList(topicMetadata)));
coordinator.maybeUpdateSubscriptionMetadata();
@ -2355,7 +2356,7 @@ public class ConsumerCoordinatorTest {
subscriptions.assignFromUser(singleton(t1p));
// Initial leader epoch of 4
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("kafka-cluster", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
Collections.emptyMap(), singletonMap(topic1, 1), tp -> 4);
client.updateMetadata(metadataResponse);

View File

@ -23,11 +23,11 @@ import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.util.ArrayList;
@ -75,7 +75,7 @@ public class ConsumerMetadataTest {
topics.add(topicMetadata("__matching_topic", false));
topics.add(topicMetadata("non_matching_topic", false));
MetadataResponse response = TestUtils.metadataResponse(singletonList(node),
MetadataResponse response = RequestTestUtils.metadataResponse(singletonList(node),
"clusterId", node.id(), topics);
metadata.updateWithCurrentRequestVersion(response, false, time.milliseconds());
@ -113,7 +113,7 @@ public class ConsumerMetadataTest {
public void testTransientTopics() {
subscription.subscribe(singleton("foo"), new NoOpConsumerRebalanceListener());
ConsumerMetadata metadata = newConsumerMetadata(false);
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith(1, singletonMap("foo", 1)), false, time.milliseconds());
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith(1, singletonMap("foo", 1)), false, time.milliseconds());
assertFalse(metadata.updateRequested());
metadata.addTransientTopics(singleton("foo"));
@ -125,13 +125,13 @@ public class ConsumerMetadataTest {
Map<String, Integer> topicPartitionCounts = new HashMap<>();
topicPartitionCounts.put("foo", 1);
topicPartitionCounts.put("bar", 1);
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith(1, topicPartitionCounts), false, time.milliseconds());
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith(1, topicPartitionCounts), false, time.milliseconds());
assertFalse(metadata.updateRequested());
assertEquals(Utils.mkSet("foo", "bar"), new HashSet<>(metadata.fetch().topics()));
metadata.clearTransientTopics();
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith(1, topicPartitionCounts), false, time.milliseconds());
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith(1, topicPartitionCounts), false, time.milliseconds());
assertEquals(singleton("foo"), new HashSet<>(metadata.fetch().topics()));
}
@ -151,7 +151,7 @@ public class ConsumerMetadataTest {
for (String expectedInternalTopic : expectedInternalTopics)
topics.add(topicMetadata(expectedInternalTopic, true));
MetadataResponse response = TestUtils.metadataResponse(singletonList(node),
MetadataResponse response = RequestTestUtils.metadataResponse(singletonList(node),
"clusterId", node.id(), topics);
metadata.updateWithCurrentRequestVersion(response, false, time.milliseconds());

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.HeartbeatRequest;
import org.apache.kafka.common.requests.HeartbeatResponse;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.test.TestUtils;
@ -247,7 +248,7 @@ public class ConsumerNetworkClientTest {
@Test(expected = InvalidTopicException.class)
public void testInvalidTopicExceptionPropagatedFromMetadata() {
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("clusterId", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap("topic", Errors.INVALID_TOPIC_EXCEPTION), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
consumerClient.poll(time.timer(Duration.ZERO));
@ -255,7 +256,7 @@ public class ConsumerNetworkClientTest {
@Test(expected = TopicAuthorizationException.class)
public void testTopicAuthorizationExceptionPropagatedFromMetadata() {
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("clusterId", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("clusterId", 1,
Collections.singletonMap("topic", Errors.TOPIC_AUTHORIZATION_FAILED), Collections.emptyMap());
metadata.updateWithCurrentRequestVersion(metadataResponse, false, time.milliseconds());
consumerClient.poll(time.timer(Duration.ZERO));

View File

@ -87,6 +87,7 @@ import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.OffsetsForLeaderEpochRequest;
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.BytesDeserializer;
import org.apache.kafka.common.serialization.Deserializer;
@ -163,7 +164,7 @@ public class FetcherTest {
private TopicPartition tp3 = new TopicPartition(topicName, 3);
private int validLeaderEpoch = 0;
private MetadataResponse initialUpdateResponse =
TestUtils.metadataUpdateWith(1, singletonMap(topicName, 4));
RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 4));
private int minBytes = 1;
private int maxBytes = Integer.MAX_VALUE;
@ -201,7 +202,7 @@ public class FetcherTest {
client.updateMetadata(initialUpdateResponse);
// A dummy metadata update to ensure valid leader epoch.
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), singletonMap(topicName, 4),
tp -> validLeaderEpoch), false, 0L);
}
@ -873,7 +874,7 @@ public class FetcherTest {
subscriptions.assignFromSubscribed(singleton(tp0));
subscriptions.seek(tp0, 0);
client.updateMetadata(TestUtils.metadataUpdateWith(
client.updateMetadata(RequestTestUtils.metadataUpdateWith(
1, singletonMap(topicName, 4), tp -> validLeaderEpoch));
assertEquals(1, fetcher.sendFetches());
@ -897,7 +898,7 @@ public class FetcherTest {
subscriptions.assignFromSubscribed(singleton(tp0));
subscriptions.seek(tp0, 0);
client.updateMetadata(TestUtils.metadataUpdateWith(
client.updateMetadata(RequestTestUtils.metadataUpdateWith(
1, singletonMap(topicName, 4), tp -> validLeaderEpoch));
assertEquals(1, fetcher.sendFetches());
@ -1165,7 +1166,7 @@ public class FetcherTest {
public void testEpochSetInFetchRequest() {
buildFetcher();
subscriptions.assignFromUser(singleton(tp0));
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), Collections.singletonMap(topicName, 4), tp -> 99);
client.updateMetadata(metadataResponse);
@ -1619,7 +1620,7 @@ public class FetcherTest {
// Set up metadata with no leader epoch
subscriptions.assignFromUser(singleton(tp0));
MetadataResponse metadataWithNoLeaderEpochs = TestUtils.metadataUpdateWith(
MetadataResponse metadataWithNoLeaderEpochs = RequestTestUtils.metadataUpdateWith(
"kafka-cluster", 1, Collections.emptyMap(), singletonMap(topicName, 4), tp -> null);
client.updateMetadata(metadataWithNoLeaderEpochs);
@ -1642,7 +1643,7 @@ public class FetcherTest {
// Set up metadata with leaderEpoch=1
subscriptions.assignFromUser(singleton(tp0));
MetadataResponse metadataWithLeaderEpochs = TestUtils.metadataUpdateWith(
MetadataResponse metadataWithLeaderEpochs = RequestTestUtils.metadataUpdateWith(
"kafka-cluster", 1, Collections.emptyMap(), singletonMap(topicName, 4), tp -> 1);
client.updateMetadata(metadataWithLeaderEpochs);
@ -2031,7 +2032,7 @@ public class FetcherTest {
altTopics.add(alteredTopic);
}
Node controller = originalResponse.controller();
MetadataResponse altered = TestUtils.metadataResponse(
MetadataResponse altered = RequestTestUtils.metadataResponse(
originalResponse.brokers(),
originalResponse.clusterId(),
controller != null ? controller.id() : MetadataResponse.NO_CONTROLLER_ID,
@ -2064,8 +2065,8 @@ public class FetcherTest {
1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, ClientDnsLookup.USE_ALL_DNS_IPS,
time, true, new ApiVersions(), throttleTimeSensor, new LogContext());
ByteBuffer buffer = ApiVersionsResponse.createApiVersionsResponse(
400, RecordBatch.CURRENT_MAGIC_VALUE).serializeWithHeader(ApiKeys.API_VERSIONS.latestVersion(), 0);
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(ApiVersionsResponse.createApiVersionsResponse(
400, RecordBatch.CURRENT_MAGIC_VALUE), ApiKeys.API_VERSIONS.latestVersion(), 0);
selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer)));
while (!client.ready(node, time.milliseconds())) {
@ -2083,7 +2084,7 @@ public class FetcherTest {
client.send(request, time.milliseconds());
client.poll(1, time.milliseconds());
FetchResponse<MemoryRecords> response = fullFetchResponse(tp0, nextRecords, Errors.NONE, i, throttleTimeMs);
buffer = response.serializeWithHeader(ApiKeys.FETCH.latestVersion(), request.correlationId());
buffer = RequestTestUtils.serializeResponseWithHeader(response, ApiKeys.FETCH.latestVersion(), request.correlationId());
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
client.poll(1, time.milliseconds());
// If a throttled response is received, advance the time to ensure progress.
@ -2241,7 +2242,7 @@ public class FetcherTest {
Map<String, Integer> partitionCounts = new HashMap<>();
partitionCounts.put(topic1, 1);
partitionCounts.put(topic2, 1);
client.updateMetadata(TestUtils.metadataUpdateWith(1, partitionCounts, tp -> validLeaderEpoch));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, partitionCounts, tp -> validLeaderEpoch));
int expectedBytes = 0;
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> fetchPartitionData = new LinkedHashMap<>();
@ -2489,7 +2490,7 @@ public class FetcherTest {
Errors.LEADER_NOT_AVAILABLE, Errors.FENCED_LEADER_EPOCH, Errors.UNKNOWN_LEADER_EPOCH);
final int newLeaderEpoch = 3;
MetadataResponse updatedMetadata = TestUtils.metadataUpdateWith("dummy", 3,
MetadataResponse updatedMetadata = RequestTestUtils.metadataUpdateWith("dummy", 3,
singletonMap(topicName, Errors.NONE), singletonMap(topicName, 4), tp -> newLeaderEpoch);
Node originalLeader = initialUpdateResponse.cluster().leaderFor(tp1);
@ -2621,7 +2622,7 @@ public class FetcherTest {
client.updateMetadata(initialUpdateResponse);
// Metadata update with leader epochs
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1,
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), Collections.singletonMap(topicName, 4), tp -> 99);
client.updateMetadata(metadataResponse);
@ -2658,7 +2659,7 @@ public class FetcherTest {
client.reset();
// Metadata initially has one topic
MetadataResponse initialMetadata = TestUtils.metadataUpdateWith(3, singletonMap(topicName, 2));
MetadataResponse initialMetadata = RequestTestUtils.metadataUpdateWith(3, singletonMap(topicName, 2));
client.updateMetadata(initialMetadata);
// The first metadata refresh should contain one topic
@ -2672,7 +2673,7 @@ public class FetcherTest {
Map<String, Integer> partitionNumByTopic = new HashMap<>();
partitionNumByTopic.put(topicName, 2);
partitionNumByTopic.put(anotherTopic, 1);
MetadataResponse updatedMetadata = TestUtils.metadataUpdateWith(3, partitionNumByTopic);
MetadataResponse updatedMetadata = RequestTestUtils.metadataUpdateWith(3, partitionNumByTopic);
client.prepareMetadataUpdate(updatedMetadata);
client.prepareResponseFrom(listOffsetResponse(t2p0, Errors.NONE, 1000L, 54L),
metadata.fetch().leaderFor(t2p0));
@ -2704,13 +2705,13 @@ public class FetcherTest {
client.reset();
MetadataResponse initialMetadata = TestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
MetadataResponse initialMetadata = RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
client.updateMetadata(initialMetadata);
Map<String, Integer> partitionNumByTopic = new HashMap<>();
partitionNumByTopic.put(topicName, 1);
partitionNumByTopic.put(anotherTopic, 1);
MetadataResponse updatedMetadata = TestUtils.metadataUpdateWith(1, partitionNumByTopic);
MetadataResponse updatedMetadata = RequestTestUtils.metadataUpdateWith(1, partitionNumByTopic);
client.prepareMetadataUpdate(updatedMetadata);
client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP),
@ -3343,7 +3344,7 @@ public class FetcherTest {
}
};
MetadataResponse initialMetadataResponse = TestUtils.metadataUpdateWith(1,
MetadataResponse initialMetadataResponse = RequestTestUtils.metadataUpdateWith(1,
singletonMap(topicName, numPartitions), tp -> validLeaderEpoch);
client.updateMetadata(initialMetadataResponse);
fetchSize = 10000;
@ -3402,7 +3403,7 @@ public class FetcherTest {
public void testFetcherSessionEpochUpdate() throws Exception {
buildFetcher(2);
MetadataResponse initialMetadataResponse = TestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
MetadataResponse initialMetadataResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
client.updateMetadata(initialMetadataResponse);
assignFromUser(Collections.singleton(tp0));
subscriptions.seek(tp0, 0L);
@ -3551,7 +3552,7 @@ public class FetcherTest {
Map<String, Integer> partitionNumByTopic = new HashMap<>();
partitionNumByTopic.put(topicName, 2);
partitionNumByTopic.put(topicName2, 1);
MetadataResponse updateMetadataResponse = TestUtils.metadataUpdateWith(2, partitionNumByTopic);
MetadataResponse updateMetadataResponse = RequestTestUtils.metadataUpdateWith(2, partitionNumByTopic);
Cluster updatedCluster = updateMetadataResponse.cluster();
// The metadata refresh should contain all the topics.
@ -3592,7 +3593,7 @@ public class FetcherTest {
private void testGetOffsetsForTimesWithUnknownOffset() {
client.reset();
// Ensure metadata has both partitions.
MetadataResponse initialMetadataUpdate = TestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
MetadataResponse initialMetadataUpdate = RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
client.updateMetadata(initialMetadataUpdate);
ListOffsetResponseData data = new ListOffsetResponseData()
@ -3625,7 +3626,7 @@ public class FetcherTest {
// Unknown Offset
client.reset();
// Ensure metadata has both partition.
MetadataResponse initialMetadataUpdate = TestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
MetadataResponse initialMetadataUpdate = RequestTestUtils.metadataUpdateWith(1, singletonMap(topicName, 1));
client.updateMetadata(initialMetadataUpdate);
// Force LIST_OFFSETS version 0
Node node = metadata.fetch().nodes().get(0);
@ -3681,7 +3682,7 @@ public class FetcherTest {
// Initialize the epoch=1
Map<String, Integer> partitionCounts = new HashMap<>();
partitionCounts.put(tp0.topic(), 4);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, tp -> 1);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, tp -> 1);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L);
// Seek
@ -3707,7 +3708,7 @@ public class FetcherTest {
buildFetcher();
assignFromUser(Utils.mkSet(tp0, tp1, tp2, tp3));
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 3,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 3,
Collections.emptyMap(), singletonMap(topicName, 4),
tp -> 5), false, 0L);
@ -3772,7 +3773,7 @@ public class FetcherTest {
final int epochOne = 1;
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
Node node = metadata.fetch().nodes().get(0);
@ -3818,7 +3819,7 @@ public class FetcherTest {
final int epochTwo = 2;
// Start with metadata, epoch=1
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
// Offset validation requires OffsetForLeaderEpoch request v3 or higher
@ -3833,7 +3834,7 @@ public class FetcherTest {
subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(0, Optional.of(epochOne), leaderAndEpoch));
// Update metadata to epoch=2, enter validation
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochTwo), false, 0L);
fetcher.validateOffsetsIfNeeded();
@ -3848,7 +3849,7 @@ public class FetcherTest {
subscriptions.seekUnvalidated(tp0, new SubscriptionState.FetchPosition(0, Optional.of(epochOne), leaderAndEpoch));
// Update metadata to epoch=2, enter validation
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochTwo), false, 0L);
// Subscription should not stay in AWAITING_VALIDATION in prepareFetchRequest
@ -3869,7 +3870,7 @@ public class FetcherTest {
final int epochOne = 1;
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
Node node = metadata.fetch().nodes().get(0);
@ -3884,7 +3885,7 @@ public class FetcherTest {
// Inject an older version of the metadata response
final short responseVersion = 8;
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, responseVersion), false, 0L);
fetcher.validateOffsetsIfNeeded();
// Offset validation is skipped
@ -3933,7 +3934,7 @@ public class FetcherTest {
final int epochOne = 1;
final long initialOffset = 5;
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
// Offset validation requires OffsetForLeaderEpoch request v3 or higher
@ -3983,7 +3984,7 @@ public class FetcherTest {
final int epochOne = 1;
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
// Offset validation requires OffsetForLeaderEpoch request v3 or higher
@ -4024,7 +4025,7 @@ public class FetcherTest {
final int epochThree = 3;
// Start with metadata, epoch=1
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochOne), false, 0L);
// Offset validation requires OffsetForLeaderEpoch request v3 or higher
@ -4036,7 +4037,7 @@ public class FetcherTest {
subscriptions.seekValidated(tp0, new SubscriptionState.FetchPosition(0, Optional.of(epochOne), leaderAndEpoch));
// Update metadata to epoch=2, enter validation
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> epochTwo), false, 0L);
fetcher.validateOffsetsIfNeeded();
assertTrue(subscriptions.awaitingValidation(tp0));
@ -4073,7 +4074,7 @@ public class FetcherTest {
apiVersions.update("0", NodeApiVersions.create(ApiKeys.OFFSET_FOR_LEADER_EPOCH.id, (short) 0, (short) 2));
// Start with metadata, epoch=1
metadata.updateWithCurrentRequestVersion(TestUtils.metadataUpdateWith("dummy", 1,
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith("dummy", 1,
Collections.emptyMap(), partitionCounts, tp -> 1), false, 0L);
// Request offset reset
@ -4118,7 +4119,7 @@ public class FetcherTest {
// Initialize the epoch=2
Map<String, Integer> partitionCounts = new HashMap<>();
partitionCounts.put(tp0.topic(), 4);
MetadataResponse metadataResponse = TestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, tp -> 2);
MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith("dummy", 1, Collections.emptyMap(), partitionCounts, tp -> 2);
metadata.updateWithCurrentRequestVersion(metadataResponse, false, 0L);
// Offset validation requires OffsetForLeaderEpoch request v3 or higher
@ -4164,7 +4165,7 @@ public class FetcherTest {
Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis());
subscriptions.assignFromUser(singleton(tp0));
client.updateMetadata(TestUtils.metadataUpdateWith(2, singletonMap(topicName, 4), tp -> validLeaderEpoch));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(2, singletonMap(topicName, 4), tp -> validLeaderEpoch));
subscriptions.seek(tp0, 0);
// Node preferred replica before first fetch response
@ -4207,7 +4208,7 @@ public class FetcherTest {
Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED, Duration.ofMinutes(5).toMillis());
subscriptions.assignFromUser(singleton(tp0));
client.updateMetadata(TestUtils.metadataUpdateWith(2, singletonMap(topicName, 4), tp -> validLeaderEpoch));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(2, singletonMap(topicName, 4), tp -> validLeaderEpoch));
subscriptions.seek(tp0, 0);
@ -4506,7 +4507,7 @@ public class FetcherTest {
MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, false,
partitionsMetadata);
List<Node> brokers = new ArrayList<>(initialUpdateResponse.brokers());
return TestUtils.metadataResponse(brokers, initialUpdateResponse.clusterId(),
return RequestTestUtils.metadataResponse(brokers, initialUpdateResponse.clusterId(),
initialUpdateResponse.controller().id(), Collections.singletonList(topicMetadata));
}

View File

@ -54,6 +54,7 @@ import org.apache.kafka.common.requests.FindCoordinatorResponse;
import org.apache.kafka.common.requests.InitProducerIdResponse;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.TxnOffsetCommitRequest;
import org.apache.kafka.common.requests.TxnOffsetCommitResponse;
import org.apache.kafka.common.serialization.ByteArraySerializer;
@ -304,7 +305,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BATCH_SIZE_CONFIG, "1");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
client.updateMetadata(initialUpdateResponse);
@ -577,7 +578,7 @@ public class KafkaProducerTest {
while (running.get()) {
while (!metadata.updateRequested() && System.currentTimeMillis() - startTimeMs < 100)
Thread.yield();
MetadataResponse updateResponse = TestUtils.metadataUpdateWith("kafka-cluster", 1,
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
singletonMap(topic, Errors.UNKNOWN_TOPIC_OR_PARTITION), emptyMap());
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
time.sleep(60 * 1000L);
@ -612,13 +613,13 @@ public class KafkaProducerTest {
exchanger.exchange(null); // 1
while (!metadata.updateRequested())
Thread.sleep(100);
MetadataResponse updateResponse = TestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
MetadataResponse updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 2
time.sleep(120 * 1000L);
// Update the metadata again, but it should be expired at this point.
updateResponse = TestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
updateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(updateResponse, false, time.milliseconds());
exchanger.exchange(null); // 3
while (!metadata.updateRequested())
@ -662,7 +663,7 @@ public class KafkaProducerTest {
ProducerMetadata metadata = newMetadata(0, 90000);
metadata.add(topic, nowMs);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, nowMs);
KafkaProducer<String, String> producer = kafkaProducer(configs, keySerializer, valueSerializer, metadata,
@ -717,7 +718,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -761,7 +762,7 @@ public class KafkaProducerTest {
long nowMs = Time.SYSTEM.milliseconds();
ProducerMetadata metadata = newMetadata(0, 90000);
metadata.add(topic, nowMs);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, nowMs);
@SuppressWarnings("unchecked") // it is safe to suppress, since this is a mock class
@ -796,7 +797,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -831,7 +832,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -856,7 +857,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -882,7 +883,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -929,7 +930,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT, true);
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -993,7 +994,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
@ -1047,7 +1048,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1071,7 +1072,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, "15000");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, emptyMap());
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, emptyMap());
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1086,7 +1087,7 @@ public class KafkaProducerTest {
List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>();
topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION,
invalidTopicName, false, Collections.emptyList()));
MetadataResponse updateResponse = TestUtils.metadataResponse(
MetadataResponse updateResponse = RequestTestUtils.metadataResponse(
new ArrayList<>(initialUpdateResponse.brokers()),
initialUpdateResponse.clusterId(),
initialUpdateResponse.controller().id(),
@ -1113,7 +1114,7 @@ public class KafkaProducerTest {
// return with a KafkaException.
String topicName = "test";
Time time = Time.SYSTEM;
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, emptyMap());
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, emptyMap());
ProducerMetadata metadata = new ProducerMetadata(0, Long.MAX_VALUE, Long.MAX_VALUE,
new LogContext(), new ClusterResourceListeners(), time);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1155,7 +1156,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "this-is-a-transactional-id");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, emptyMap());
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, emptyMap());
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1174,7 +1175,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "this-is-a-transactional-id");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1202,7 +1203,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "this-is-a-transactional-id");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());
@ -1231,7 +1232,7 @@ public class KafkaProducerTest {
configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "this-is-a-transactional-id");
Time time = new MockTime();
MetadataResponse initialUpdateResponse = TestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("testTopic", 1));
ProducerMetadata metadata = newMetadata(0, Long.MAX_VALUE);
metadata.updateWithCurrentRequestVersion(initialUpdateResponse, false, time.milliseconds());

View File

@ -21,9 +21,9 @@ import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Test;
@ -287,7 +287,7 @@ public class ProducerMetadataTest {
Map<String, Integer> partitionCounts = new HashMap<>();
for (String topic : topics)
partitionCounts.put(topic, 1);
return TestUtils.metadataUpdateWith(1, partitionCounts);
return RequestTestUtils.metadataUpdateWith(1, partitionCounts);
}
private void clearBackgroundError() {

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.errors.TransactionAbortedException;
import org.apache.kafka.common.message.ProduceRequestData;
import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
import org.apache.kafka.common.requests.MetadataRequest;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.RequestUtils;
import org.apache.kafka.common.utils.ProducerIdAndEpoch;
import org.apache.kafka.clients.producer.RecordMetadata;
@ -285,8 +286,9 @@ public class SenderTest {
1000, 1000, 64 * 1024, 64 * 1024, 1000, 10 * 1000, 127 * 1000, ClientDnsLookup.USE_ALL_DNS_IPS,
time, true, new ApiVersions(), throttleTimeSensor, logContext);
ByteBuffer buffer = ApiVersionsResponse.createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE).
serializeWithHeader(ApiKeys.API_VERSIONS.latestVersion(), 0);
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(
ApiVersionsResponse.createApiVersionsResponse(400, RecordBatch.CURRENT_MAGIC_VALUE),
ApiKeys.API_VERSIONS.latestVersion(), 0);
selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer)));
while (!client.ready(node, time.milliseconds())) {
@ -306,7 +308,7 @@ public class SenderTest {
client.send(request, time.milliseconds());
client.poll(1, time.milliseconds());
ProduceResponse response = produceResponse(tp0, i, Errors.NONE, throttleTimeMs);
buffer = response.serializeWithHeader(ApiKeys.PRODUCE.latestVersion(), request.correlationId());
buffer = RequestTestUtils.serializeResponseWithHeader(response, ApiKeys.PRODUCE.latestVersion(), request.correlationId());
selector.completeReceive(new NetworkReceive(node.idString(), buffer));
client.poll(1, time.milliseconds());
// If a throttled response is received, advance the time to ensure progress.
@ -417,7 +419,7 @@ public class SenderTest {
Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, null, apiVersions);
// Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1
MetadataResponse metadataUpdate1 = TestUtils.metadataUpdateWith(2, Collections.singletonMap("test", 2));
MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap("test", 2));
client.prepareMetadataUpdate(metadataUpdate1);
// Send the first message.
@ -438,7 +440,7 @@ public class SenderTest {
appendToAccumulator(tp2, 0L, "key2", "value2");
// Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0
MetadataResponse metadataUpdate2 = TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2));
MetadataResponse metadataUpdate2 = RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2));
client.prepareMetadataUpdate(metadataUpdate2);
// Sender should not send the second message to node 0.
assertEquals(1, sender.inFlightBatches(tp2).size());
@ -505,12 +507,12 @@ public class SenderTest {
@Test
public void testMetadataTopicExpiry() throws Exception {
long offset = 0;
client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
Future<RecordMetadata> future = appendToAccumulator(tp0);
sender.runOnce();
assertTrue("Topic not added to metadata", metadata.containsTopic(tp0.topic()));
client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
sender.runOnce(); // send produce request
client.respond(produceResponse(tp0, offset, Errors.NONE, 0));
sender.runOnce();
@ -522,12 +524,12 @@ public class SenderTest {
assertTrue("Topic not retained in metadata list", metadata.containsTopic(tp0.topic()));
time.sleep(TOPIC_IDLE_MS);
client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
assertFalse("Unused topic has not been expired", metadata.containsTopic(tp0.topic()));
future = appendToAccumulator(tp0);
sender.runOnce();
assertTrue("Topic not added to metadata", metadata.containsTopic(tp0.topic()));
client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
sender.runOnce(); // send produce request
client.respond(produceResponse(tp0, offset + 1, Errors.NONE, 0));
sender.runOnce();
@ -570,7 +572,7 @@ public class SenderTest {
// Process metadata response, prepare FindCoordinator and InitProducerId responses.
// Verify producerId after the sender is run to process responses.
MetadataResponse metadataUpdate = TestUtils.metadataUpdateWith(1, Collections.emptyMap());
MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWith(1, Collections.emptyMap());
client.respond(metadataUpdate);
prepareFindCoordinatorResponse(Errors.NONE);
prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch);
@ -595,7 +597,7 @@ public class SenderTest {
// Process metadata and InitProducerId responses.
// Verify producerId after the sender is run to process responses.
MetadataResponse metadataUpdate = TestUtils.metadataUpdateWith(1, Collections.emptyMap());
MetadataResponse metadataUpdate = RequestTestUtils.metadataUpdateWith(1, Collections.emptyMap());
client.respond(metadataUpdate);
sender.runOnce();
sender.runOnce();
@ -2086,7 +2088,7 @@ public class SenderTest {
Sender sender = new Sender(logContext, client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
senderMetrics, time, REQUEST_TIMEOUT, 1000L, txnManager, new ApiVersions());
// Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1
MetadataResponse metadataUpdate1 = TestUtils.metadataUpdateWith(2, Collections.singletonMap(topic, 2));
MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap(topic, 2));
client.prepareMetadataUpdate(metadataUpdate1);
// Send the first message.
long nowMs = time.milliseconds();
@ -2701,7 +2703,7 @@ public class SenderTest {
metadata.add("test", time.milliseconds());
if (updateMetadata)
this.client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("test", 2)));
}
private void assertSendFailure(Class<? extends RuntimeException> expectedError) throws Exception {

View File

@ -25,6 +25,7 @@ import org.apache.kafka.clients.NodeApiVersions;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.ProducerIdAndEpoch;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Cluster;
@ -145,7 +146,7 @@ public class TransactionManagerTest {
@Before
public void setup() {
this.metadata.add("test", time.milliseconds());
this.client.updateMetadata(TestUtils.metadataUpdateWith(1, singletonMap("test", 2)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap("test", 2)));
this.brokerNode = new Node(0, "localhost", 2211);
initializeTransactionManager(Optional.of(transactionalId), false);

View File

@ -42,11 +42,12 @@ public class KafkaChannelTest {
KafkaChannel channel = new KafkaChannel("0", transport, () -> authenticator,
1024, pool, metadataRegistry);
NetworkSend send = new NetworkSend("0", ByteBuffer.wrap(TestUtils.randomBytes(128)));
ByteBufferSend send = ByteBufferSend.sizePrefixed(ByteBuffer.wrap(TestUtils.randomBytes(128)));
NetworkSend networkSend = new NetworkSend("0", send);
channel.setSend(send);
channel.setSend(networkSend);
assertTrue(channel.hasSend());
assertThrows(IllegalStateException.class, () -> channel.setSend(send));
assertThrows(IllegalStateException.class, () -> channel.setSend(networkSend));
Mockito.when(transport.write(Mockito.any(ByteBuffer[].class))).thenReturn(4L);
assertEquals(4L, channel.write());
@ -61,7 +62,7 @@ public class KafkaChannelTest {
Mockito.when(transport.write(Mockito.any(ByteBuffer[].class))).thenReturn(64L);
assertEquals(64, channel.write());
assertEquals(0, send.remaining());
assertEquals(send, channel.maybeCompleteSend());
assertEquals(networkSend, channel.maybeCompleteSend());
}
@Test

View File

@ -70,7 +70,7 @@ public class NetworkTestUtils {
String prefix = TestUtils.randomString(minMessageSize);
int requests = 0;
int responses = 0;
selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes(StandardCharsets.UTF_8))));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap((prefix + "-0").getBytes(StandardCharsets.UTF_8)))));
requests++;
while (responses < messageCount) {
selector.poll(0L);
@ -82,7 +82,7 @@ public class NetworkTestUtils {
}
for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) {
selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes())));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap((prefix + "-" + requests).getBytes()))));
}
}
}

View File

@ -35,6 +35,7 @@ import org.apache.kafka.test.TestUtils;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
import java.nio.channels.SelectionKey;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
@ -75,7 +76,7 @@ public class NioEchoServer extends Thread {
private final List<SocketChannel> socketChannels;
private final AcceptorThread acceptorThread;
private final Selector selector;
private volatile WritableByteChannel outputChannel;
private volatile GatheringByteChannel outputChannel;
private final CredentialCache credentialCache;
private final Metrics metrics;
private volatile int numSent = 0;
@ -225,18 +226,17 @@ public class NioEchoServer extends Thread {
if (!maybeBeginServerReauthentication(channel, rcv, time)) {
String channelId = channel.id();
selector.mute(channelId);
NetworkSend send = new NetworkSend(rcv.source(), rcv.payload());
NetworkSend send = new NetworkSend(rcv.source(), ByteBufferSend.sizePrefixed(rcv.payload()));
if (outputChannel == null)
selector.send(send);
else {
for (ByteBuffer buffer : send.buffers)
outputChannel.write(buffer);
send.writeTo(outputChannel);
selector.unmute(channelId);
}
}
}
for (Send send : selector.completedSends()) {
selector.unmute(send.destination());
for (NetworkSend send : selector.completedSends()) {
selector.unmute(send.destinationId());
numSent += 1;
}
}
@ -276,14 +276,46 @@ public class NioEchoServer extends Thread {
* the responses (eg. testing graceful close).
*/
public void outputChannel(WritableByteChannel channel) {
this.outputChannel = channel;
if (channel instanceof GatheringByteChannel)
this.outputChannel = (GatheringByteChannel) channel;
else {
this.outputChannel = new GatheringByteChannel() {
@Override
public boolean isOpen() {
return channel.isOpen();
}
@Override
public void close() throws IOException {
channel.close();
}
@Override
public int write(ByteBuffer src) throws IOException {
return channel.write(src);
}
@Override
public long write(ByteBuffer[] srcs, int offset, int length) throws IOException {
long result = 0;
for (int i = offset; i < offset + length; ++i)
result += write(srcs[i]);
return result;
}
@Override
public long write(ByteBuffer[] srcs) throws IOException {
return write(srcs, 0, srcs.length);
}
};
}
}
public Selector selector() {
return selector;
}
public void closeKafkaChannels() throws IOException {
public void closeKafkaChannels() {
closeKafkaChannels = true;
selector.wakeup();
try {

View File

@ -253,8 +253,8 @@ public class SelectorTest {
}
// prepare new sends for the next round
for (Send send : selector.completedSends()) {
String dest = send.destination();
for (NetworkSend send : selector.completedSends()) {
String dest = send.destinationId();
if (requests.containsKey(dest))
requests.put(dest, requests.get(dest) + 1);
else
@ -298,9 +298,10 @@ public class SelectorTest {
String payload = TestUtils.randomString(payloadSize);
String nodeId = "0";
blockingConnect(nodeId);
NetworkSend send = createSend(nodeId, payload);
ByteBufferSend send = ByteBufferSend.sizePrefixed(ByteBuffer.wrap(payload.getBytes()));
NetworkSend networkSend = new NetworkSend(nodeId, send);
selector.send(send);
selector.send(networkSend);
KafkaChannel channel = selector.channel(nodeId);
KafkaMetric outgoingByteTotal = findUntaggedMetricByName("outgoing-byte-total");
@ -958,7 +959,7 @@ public class SelectorTest {
KafkaChannel channel = mock(KafkaChannel.class);
when(channel.id()).thenReturn("1");
when(channel.write()).thenReturn(0L);
ByteBufferSend send = new ByteBufferSend("destination", ByteBuffer.allocate(0));
NetworkSend send = new NetworkSend("destination", new ByteBufferSend(ByteBuffer.allocate(0)));
when(channel.maybeCompleteSend()).thenReturn(send);
selector.write(channel);
assertEquals(asList(send), selector.completedSends());
@ -1046,8 +1047,8 @@ public class SelectorTest {
selector.poll(10000L);
}
protected NetworkSend createSend(String node, String payload) {
return new NetworkSend(node, ByteBuffer.wrap(payload.getBytes()));
protected final NetworkSend createSend(String node, String payload) {
return new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(payload.getBytes())));
}
protected String asString(NetworkReceive receive) {

View File

@ -31,7 +31,6 @@ import org.apache.kafka.common.utils.Java;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestCondition;
import org.apache.kafka.test.TestSslUtils;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
@ -642,7 +641,7 @@ public class SslTransportLayerTest {
NetworkTestUtils.waitForChannelReady(selector, node);
int messageSize = 1024 * 1024;
String message = TestUtils.randomString(messageSize);
selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(message.getBytes()))));
while (selector.completedReceives().isEmpty()) {
selector.poll(100L);
}
@ -668,28 +667,21 @@ public class SslTransportLayerTest {
// Send a message of 80K. This is 5X as large as the socket buffer. It should take at least three selector.poll()
// to read this message from socket if the SslTransportLayer.read() does not read all data from socket buffer.
String message = TestUtils.randomString(81920);
selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(message.getBytes()))));
// Send the message to echo server
TestUtils.waitForCondition(new TestCondition() {
@Override
public boolean conditionMet() {
try {
selector.poll(100L);
} catch (IOException e) {
return false;
}
return selector.completedSends().size() > 0;
TestUtils.waitForCondition(() -> {
try {
selector.poll(100L);
} catch (IOException e) {
return false;
}
return selector.completedSends().size() > 0;
}, "Timed out waiting for message to be sent");
// Wait for echo server to send the message back
TestUtils.waitForCondition(new TestCondition() {
@Override
public boolean conditionMet() {
return server.numSent() >= 2;
}
}, "Timed out waiting for echo server to send message");
TestUtils.waitForCondition(() ->
server.numSent() >= 2, "Timed out waiting for echo server to send message");
// Read the message from socket with only one poll()
selector.poll(1000L);
@ -764,7 +756,7 @@ public class SslTransportLayerTest {
assertEquals("Time not reset", 0, channel.getAndResetNetworkThreadTimeNanos());
selector.mute(node);
selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(message.getBytes()))));
while (selector.completedSends().isEmpty()) {
selector.poll(100L);
}
@ -776,16 +768,13 @@ public class SslTransportLayerTest {
selector.unmute(node);
// Wait for echo server to send the message back
TestUtils.waitForCondition(new TestCondition() {
@Override
public boolean conditionMet() {
try {
selector.poll(100L);
} catch (IOException e) {
return false;
}
return !selector.completedReceives().isEmpty();
TestUtils.waitForCondition(() -> {
try {
selector.poll(100L);
} catch (IOException e) {
return false;
}
return !selector.completedReceives().isEmpty();
}, "Timed out waiting for a message to receive from echo server");
long receiveTimeNanos = channel.getAndResetNetworkThreadTimeNanos();
@ -956,19 +945,15 @@ public class SslTransportLayerTest {
int count = 20;
final int totalSendSize = count * (message.length + 4);
for (int i = 0; i < count; i++) {
selector.send(new NetworkSend(node, ByteBuffer.wrap(message)));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(message))));
do {
selector.poll(0L);
} while (selector.completedSends().isEmpty());
}
server.selector().unmuteAll();
selector.close(node);
TestUtils.waitForCondition(new TestCondition() {
@Override
public boolean conditionMet() {
return bytesOut.toByteArray().length == totalSendSize;
}
}, 5000, "All requests sent were not processed");
TestUtils.waitForCondition(() ->
bytesOut.toByteArray().length == totalSendSize, 5000, "All requests sent were not processed");
}
/**

View File

@ -115,7 +115,7 @@ public class SslVersionsTransportLayerTest {
int msgSz = 1024 * 1024;
String message = TestUtils.randomString(msgSz);
selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(message.getBytes()))));
while (selector.completedReceives().isEmpty()) {
selector.poll(100L);
}

View File

@ -1,31 +0,0 @@
/*
* 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.common.protocol;
import java.nio.ByteBuffer;
public final class MessageTestUtil {
public static ByteBuffer messageToByteBuffer(Message message, short version) {
ObjectSerializationCache cache = new ObjectSerializationCache();
int size = message.size(cache, version);
ByteBuffer bytes = ByteBuffer.allocate(size);
message.write(new ByteBufferAccessor(bytes), cache, version);
bytes.rewind();
return bytes;
}
}

View File

@ -36,7 +36,7 @@ public class SendBuilderTest {
public void testZeroCopyByteBuffer() {
byte[] data = Utils.utf8("foo");
ByteBuffer zeroCopyBuffer = ByteBuffer.wrap(data);
SendBuilder builder = new SendBuilder("a", 8);
SendBuilder builder = new SendBuilder(8);
builder.writeInt(5);
builder.writeByteBuffer(zeroCopyBuffer);
@ -63,7 +63,7 @@ public class SendBuilderTest {
assertEquals(4, data.length);
ByteBuffer buffer = ByteBuffer.wrap(data);
SendBuilder builder = new SendBuilder("a", 0);
SendBuilder builder = new SendBuilder(0);
buffer.limit(2);
builder.writeByteBuffer(buffer);
@ -84,7 +84,7 @@ public class SendBuilderTest {
ByteBuffer buffer = ByteBuffer.allocate(128);
MemoryRecords records = createRecords(buffer, "foo");
SendBuilder builder = new SendBuilder("a", 8);
SendBuilder builder = new SendBuilder(8);
builder.writeInt(5);
builder.writeRecords(records);
builder.writeInt(15);

View File

@ -158,7 +158,7 @@ public class LazyDownConversionRecordsTest {
LazyDownConversionRecords lazyRecords = new LazyDownConversionRecords(new TopicPartition("test", 1),
inputRecords, toMagic, 0L, Time.SYSTEM);
LazyDownConversionRecordsSend lazySend = lazyRecords.toSend("foo");
LazyDownConversionRecordsSend lazySend = lazyRecords.toSend();
File outputFile = tempFile();
FileChannel channel = FileChannel.open(outputFile.toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE);

View File

@ -33,7 +33,6 @@ public class MultiRecordsSendTest {
@Test
public void testSendsFreedAfterWriting() throws IOException {
String dest = "1";
int numChunks = 4;
int chunkSize = 32;
int totalSize = numChunks * chunkSize;
@ -44,10 +43,10 @@ public class MultiRecordsSendTest {
for (int i = 0; i < numChunks; i++) {
ByteBuffer buffer = ByteBuffer.wrap(TestUtils.randomBytes(chunkSize));
chunks[i] = buffer;
sends.add(new ByteBufferSend(dest, buffer));
sends.add(new ByteBufferSend(buffer));
}
MultiRecordsSend send = new MultiRecordsSend(dest, sends);
MultiRecordsSend send = new MultiRecordsSend(sends);
assertEquals(totalSize, send.size());
for (int i = 0; i < numChunks; i++) {

View File

@ -90,7 +90,7 @@ public class AddPartitionsToTxnResponseTest {
AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
for (short version = 0; version <= ApiKeys.ADD_PARTITIONS_TO_TXN.latestVersion(); version++) {
AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serializeBody(version), version);
AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
assertEquals(expectedErrorCounts, parsedResponse.errorCounts());
assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
assertEquals(version >= 1, parsedResponse.shouldClientThrottle(version));

View File

@ -64,7 +64,7 @@ public class CreateAclsRequestTest {
@Test
public void shouldRoundTripV0() {
final CreateAclsRequest original = new CreateAclsRequest(data(LITERAL_ACL1, LITERAL_ACL2), V0);
final ByteBuffer buffer = original.serializeBody();
final ByteBuffer buffer = original.serialize();
final CreateAclsRequest result = CreateAclsRequest.parse(buffer, V0);
@ -74,7 +74,7 @@ public class CreateAclsRequestTest {
@Test
public void shouldRoundTripV1() {
final CreateAclsRequest original = new CreateAclsRequest(data(LITERAL_ACL1, PREFIXED_ACL1), V1);
final ByteBuffer buffer = original.serializeBody();
final ByteBuffer buffer = original.serialize();
final CreateAclsRequest result = CreateAclsRequest.parse(buffer, V1);

View File

@ -64,7 +64,7 @@ public class DeleteAclsRequestTest {
@Test
public void shouldRoundTripLiteralV0() {
final DeleteAclsRequest original = new DeleteAclsRequest.Builder(requestData(LITERAL_FILTER)).build(V0);
final ByteBuffer buffer = original.serializeBody();
final ByteBuffer buffer = original.serialize();
final DeleteAclsRequest result = DeleteAclsRequest.parse(buffer, V0);
@ -82,7 +82,7 @@ public class DeleteAclsRequestTest {
ANY_FILTER.entryFilter()))
).build(V0);
final DeleteAclsRequest result = DeleteAclsRequest.parse(original.serializeBody(), V0);
final DeleteAclsRequest result = DeleteAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(expected, result);
}
@ -92,7 +92,7 @@ public class DeleteAclsRequestTest {
final DeleteAclsRequest original = new DeleteAclsRequest.Builder(
requestData(LITERAL_FILTER, PREFIXED_FILTER, ANY_FILTER)
).build(V1);
final ByteBuffer buffer = original.serializeBody();
final ByteBuffer buffer = original.serialize();
final DeleteAclsRequest result = DeleteAclsRequest.parse(buffer, V1);

View File

@ -108,7 +108,7 @@ public class DeleteAclsResponseTest {
.setThrottleTimeMs(10)
.setFilterResults(singletonList(LITERAL_RESPONSE)),
V0);
final ByteBuffer buffer = original.serializeBody(V0);
final ByteBuffer buffer = original.serialize(V0);
final DeleteAclsResponse result = DeleteAclsResponse.parse(buffer, V0);
assertEquals(original.filterResults(), result.filterResults());
@ -121,7 +121,7 @@ public class DeleteAclsResponseTest {
.setThrottleTimeMs(10)
.setFilterResults(asList(LITERAL_RESPONSE, PREFIXED_RESPONSE)),
V1);
final ByteBuffer buffer = original.serializeBody(V1);
final ByteBuffer buffer = original.serialize(V1);
final DeleteAclsResponse result = DeleteAclsResponse.parse(buffer, V1);
assertEquals(original.filterResults(), result.filterResults());

View File

@ -59,7 +59,7 @@ public class DescribeAclsRequestTest {
@Test
public void shouldRoundTripLiteralV0() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(LITERAL_FILTER).build(V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serializeBody(), V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(original, result);
}
@ -74,28 +74,28 @@ public class DescribeAclsRequestTest {
PatternType.LITERAL),
ANY_FILTER.entryFilter())).build(V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serializeBody(), V0);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V0);
assertRequestEquals(expected, result);
}
@Test
public void shouldRoundTripLiteralV1() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(LITERAL_FILTER).build(V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serializeBody(), V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V1);
assertRequestEquals(original, result);
}
@Test
public void shouldRoundTripPrefixedV1() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(PREFIXED_FILTER).build(V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serializeBody(), V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V1);
assertRequestEquals(original, result);
}
@Test
public void shouldRoundTripAnyV1() {
final DescribeAclsRequest original = new DescribeAclsRequest.Builder(ANY_FILTER).build(V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serializeBody(), V1);
final DescribeAclsRequest result = DescribeAclsRequest.parse(original.serialize(), V1);
assertRequestEquals(original, result);
}

View File

@ -82,19 +82,19 @@ public class DescribeAclsResponseTest {
@Test(expected = UnsupportedVersionException.class)
public void shouldThrowOnV0IfNotLiteral() {
buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serializeBody(V0);
buildResponse(10, Errors.NONE, Collections.singletonList(PREFIXED_ACL1)).serialize(V0);
}
@Test(expected = IllegalArgumentException.class)
public void shouldThrowIfUnknown() {
buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serializeBody(V0);
buildResponse(10, Errors.NONE, Collections.singletonList(UNKNOWN_ACL)).serialize(V0);
}
@Test
public void shouldRoundTripV0() {
List<DescribeAclsResource> resources = Arrays.asList(LITERAL_ACL1, LITERAL_ACL2);
final DescribeAclsResponse original = buildResponse(10, Errors.NONE, resources);
final ByteBuffer buffer = original.serializeBody(V0);
final ByteBuffer buffer = original.serialize(V0);
final DescribeAclsResponse result = DescribeAclsResponse.parse(buffer, V0);
assertResponseEquals(original, result);
@ -108,7 +108,7 @@ public class DescribeAclsResponseTest {
public void shouldRoundTripV1() {
List<DescribeAclsResource> resources = Arrays.asList(LITERAL_ACL1, PREFIXED_ACL1);
final DescribeAclsResponse original = buildResponse(100, Errors.NONE, resources);
final ByteBuffer buffer = original.serializeBody(V1);
final ByteBuffer buffer = original.serialize(V1);
final DescribeAclsResponse result = DescribeAclsResponse.parse(buffer, V1);
assertResponseEquals(original, result);

View File

@ -44,7 +44,7 @@ public class EndTxnResponseTest {
assertEquals(throttleTimeMs, response.throttleTimeMs());
assertEquals(version >= 1, response.shouldClientThrottle(version));
response = EndTxnResponse.parse(response.serializeBody(version), version);
response = EndTxnResponse.parse(response.serialize(version), version);
assertEquals(expectedErrorCounts, response.errorCounts());
assertEquals(throttleTimeMs, response.throttleTimeMs());
assertEquals(version >= 1, response.shouldClientThrottle(version));

View File

@ -55,7 +55,7 @@ public class EnvelopeRequestTest {
InetAddress.getLocalHost().getAddress()
).build(version);
Send send = request.toSend("a", header);
Send send = request.toSend(header);
ByteBuffer buffer = TestUtils.toBuffer(send);
assertEquals(send.size() - 4, buffer.getInt());
assertEquals(header, RequestHeader.parse(buffer));

View File

@ -38,7 +38,7 @@ class EnvelopeResponseTest {
short headerVersion = ApiKeys.ENVELOPE.responseHeaderVersion(version);
ResponseHeader header = new ResponseHeader(15, headerVersion);
Send send = response.toSend("a", header, version);
Send send = response.toSend(header, version);
ByteBuffer buffer = TestUtils.toBuffer(send);
assertEquals(send.size() - 4, buffer.getInt());
assertEquals(header, ResponseHeader.parse(buffer, headerVersion));
@ -49,4 +49,4 @@ class EnvelopeResponseTest {
}
}
}
}

View File

@ -128,7 +128,7 @@ public class LeaderAndIsrRequestTest {
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
ByteBuffer byteBuffer = request.serializeBody();
ByteBuffer byteBuffer = request.serialize();
LeaderAndIsrRequest deserializedRequest = new LeaderAndIsrRequest(new LeaderAndIsrRequestData(
new ByteBufferAccessor(byteBuffer), version), version);

View File

@ -20,7 +20,7 @@ import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.MessageTestUtil;
import org.apache.kafka.common.protocol.MessageUtil;
import org.junit.Before;
import org.junit.Test;
@ -111,9 +111,9 @@ public class LeaveGroupResponseTest {
.setThrottleTimeMs(throttleTimeMs);
for (short version = 0; version <= ApiKeys.LEAVE_GROUP.latestVersion(); version++) {
LeaveGroupResponse primaryResponse = LeaveGroupResponse.parse(
MessageTestUtil.messageToByteBuffer(responseData, version), version);
MessageUtil.toByteBuffer(responseData, version), version);
LeaveGroupResponse secondaryResponse = LeaveGroupResponse.parse(
MessageTestUtil.messageToByteBuffer(responseData, version), version);
MessageUtil.toByteBuffer(responseData, version), version);
assertEquals(primaryResponse, primaryResponse);
assertEquals(primaryResponse, secondaryResponse);
@ -130,7 +130,7 @@ public class LeaveGroupResponseTest {
.setThrottleTimeMs(throttleTimeMs);
for (short version = 0; version <= ApiKeys.LEAVE_GROUP.latestVersion(); version++) {
ByteBuffer buffer = MessageTestUtil.messageToByteBuffer(data, version);
ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
LeaveGroupResponse leaveGroupResponse = LeaveGroupResponse.parse(buffer, version);
assertEquals(expectedErrorCounts, leaveGroupResponse.errorCounts());

View File

@ -35,7 +35,7 @@ import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitio
import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetTopicResponse;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.MessageTestUtil;
import org.apache.kafka.common.protocol.MessageUtil;
import org.junit.Test;
public class ListOffsetRequestTest {
@ -53,7 +53,7 @@ public class ListOffsetRequestTest {
ListOffsetRequestData data = new ListOffsetRequestData()
.setTopics(topics)
.setReplicaId(-1);
ListOffsetRequest request = ListOffsetRequest.parse(MessageTestUtil.messageToByteBuffer(data, (short) 0), (short) 0);
ListOffsetRequest request = ListOffsetRequest.parse(MessageUtil.toByteBuffer(data, (short) 0), (short) 0);
assertEquals(Collections.singleton(new TopicPartition("topic", 0)), request.duplicatePartitions());
}

View File

@ -22,7 +22,7 @@ import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResp
import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.MessageTestUtil;
import org.apache.kafka.common.protocol.MessageUtil;
import org.junit.Before;
import org.junit.Test;
@ -86,7 +86,7 @@ public class OffsetCommitResponseTest {
.setThrottleTimeMs(throttleTimeMs);
for (short version = 0; version <= ApiKeys.OFFSET_COMMIT.latestVersion(); version++) {
ByteBuffer buffer = MessageTestUtil.messageToByteBuffer(data, version);
ByteBuffer buffer = MessageUtil.toByteBuffer(data, version);
OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version);
assertEquals(expectedErrorCounts, response.errorCounts());

View File

@ -108,7 +108,7 @@ public class OffsetFetchResponseTest {
for (short version = 0; version <= ApiKeys.OFFSET_FETCH.latestVersion(); version++) {
Struct struct = latestResponse.data.toStruct(version);
OffsetFetchResponse oldResponse = OffsetFetchResponse.parse(latestResponse.serializeBody(version), version);
OffsetFetchResponse oldResponse = OffsetFetchResponse.parse(latestResponse.serialize(version), version);
if (version <= 1) {
assertFalse(struct.hasField(ERROR_CODE));

View File

@ -49,7 +49,7 @@ public class OffsetsForLeaderEpochRequestTest {
OffsetsForLeaderEpochRequest.Builder builder = OffsetsForLeaderEpochRequest.Builder.forFollower(
version, Collections.emptyMap(), replicaId);
OffsetsForLeaderEpochRequest request = builder.build();
OffsetsForLeaderEpochRequest parsed = OffsetsForLeaderEpochRequest.parse(request.serializeBody(), version);
OffsetsForLeaderEpochRequest parsed = OffsetsForLeaderEpochRequest.parse(request.serialize(), version);
if (version < 3)
assertEquals(OffsetsForLeaderEpochRequest.DEBUGGING_REPLICA_ID, parsed.replicaId());
else

View File

@ -299,7 +299,7 @@ public class ProduceRequestTest {
private void assertThrowsInvalidRecordException(ProduceRequest.Builder builder, short version) {
try {
builder.build(version).serializeBody();
builder.build(version).serialize();
fail("Builder did not raise " + InvalidRecordException.class.getName() + " as expected");
} catch (RuntimeException e) {
assertTrue("Unexpected exception type " + e.getClass().getName(),

View File

@ -46,8 +46,7 @@ public class ProduceResponseTest {
ProduceResponse v5Response = new ProduceResponse(responseData, 10);
short version = 5;
ByteBuffer buffer = v5Response.serializeWithHeader(version, 0);
buffer.rewind();
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(v5Response, version, 0);
ResponseHeader.parse(buffer, ApiKeys.PRODUCE.responseHeaderVersion(version)); // throw away.
ProduceResponse v5FromBytes = (ProduceResponse) AbstractResponse.parseResponse(ApiKeys.PRODUCE,
@ -92,7 +91,7 @@ public class ProduceResponseTest {
for (short ver = 0; ver <= PRODUCE.latestVersion(); ver++) {
ProduceResponse response = new ProduceResponse(responseData);
ProduceResponse.PartitionResponse deserialized = ProduceResponse.parse(response.serializeBody(ver), ver).responses().get(tp);
ProduceResponse.PartitionResponse deserialized = ProduceResponse.parse(response.serialize(ver), ver).responses().get(tp);
if (ver >= 8) {
assertEquals(1, deserialized.recordErrors.size());
assertEquals(3, deserialized.recordErrors.get(0).batchIndex);

View File

@ -99,6 +99,7 @@ public class RequestContextTest {
ClientInformation.EMPTY, true);
ByteBuffer buffer = context.buildResponseEnvelopePayload(new CreateTopicsResponse(expectedResponse));
assertEquals("Buffer limit and capacity should be the same", buffer.capacity(), buffer.limit());
CreateTopicsResponse parsedResponse = (CreateTopicsResponse) AbstractResponse.parseResponse(buffer, header);
assertEquals(expectedResponse, parsedResponse.data());
}

View File

@ -18,7 +18,6 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.nio.ByteBuffer;
@ -45,7 +44,7 @@ public class RequestHeaderTest {
assertEquals("", deserialized.clientId());
assertEquals(0, deserialized.headerVersion());
ByteBuffer serializedBuffer = TestUtils.serializeRequestHeader(deserialized);
ByteBuffer serializedBuffer = RequestTestUtils.serializeRequestHeader(deserialized);
assertEquals(ApiKeys.CONTROLLED_SHUTDOWN.id, serializedBuffer.getShort(0));
assertEquals(0, serializedBuffer.getShort(2));
@ -59,7 +58,7 @@ public class RequestHeaderTest {
RequestHeader header = new RequestHeader(ApiKeys.FIND_COORDINATOR, apiVersion, "", 10);
assertEquals(1, header.headerVersion());
ByteBuffer buffer = TestUtils.serializeRequestHeader(header);
ByteBuffer buffer = RequestTestUtils.serializeRequestHeader(header);
assertEquals(10, buffer.remaining());
RequestHeader deserialized = RequestHeader.parse(buffer);
assertEquals(header, deserialized);
@ -71,7 +70,7 @@ public class RequestHeaderTest {
RequestHeader header = new RequestHeader(ApiKeys.CREATE_DELEGATION_TOKEN, apiVersion, "", 10);
assertEquals(2, header.headerVersion());
ByteBuffer buffer = TestUtils.serializeRequestHeader(header);
ByteBuffer buffer = RequestTestUtils.serializeRequestHeader(header);
assertEquals(11, buffer.remaining());
RequestHeader deserialized = RequestHeader.parse(buffer);
assertEquals(header, deserialized);

View File

@ -183,7 +183,6 @@ import org.apache.kafka.common.security.token.delegation.DelegationToken;
import org.apache.kafka.common.security.token.delegation.TokenInformation;
import org.apache.kafka.common.utils.SecurityUtils;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import java.nio.BufferUnderflowException;
@ -571,7 +570,7 @@ public class RequestResponseTest {
short apiVersion = (short) version;
DescribeConfigsResponse response = createDescribeConfigsResponse(apiVersion);
DescribeConfigsResponse deserialized0 = (DescribeConfigsResponse) AbstractResponse.parseResponse(ApiKeys.DESCRIBE_CONFIGS,
response.serializeBody(apiVersion), apiVersion);
response.serialize(apiVersion), apiVersion);
verifyDescribeConfigsResponse(response, deserialized0, apiVersion);
}
}
@ -591,9 +590,9 @@ public class RequestResponseTest {
// Check for equality of the ByteBuffer only if indicated (it is likely to fail if any of the fields
// in the request is a HashMap with multiple elements since ordering of the elements may vary)
try {
ByteBuffer serializedBytes = req.serializeBody();
ByteBuffer serializedBytes = req.serialize();
AbstractRequest deserialized = AbstractRequest.parseRequest(req.apiKey(), req.version(), serializedBytes).request;
ByteBuffer serializedBytes2 = deserialized.serializeBody();
ByteBuffer serializedBytes2 = deserialized.serialize();
serializedBytes.rewind();
if (checkEquality)
assertEquals("Request " + req + "failed equality test", serializedBytes, serializedBytes2);
@ -607,9 +606,9 @@ public class RequestResponseTest {
// Check for equality and hashCode of the Struct only if indicated (it is likely to fail if any of the fields
// in the response is a HashMap with multiple elements since ordering of the elements may vary)
try {
ByteBuffer serializedBytes = response.serializeBody((short) version);
ByteBuffer serializedBytes = response.serialize((short) version);
AbstractResponse deserialized = AbstractResponse.parseResponse(response.apiKey(), serializedBytes, (short) version);
ByteBuffer serializedBytes2 = deserialized.serializeBody((short) version);
ByteBuffer serializedBytes2 = deserialized.serialize((short) version);
serializedBytes.rewind();
if (checkEquality)
assertEquals("Response " + response + "failed equality test", serializedBytes, serializedBytes2);
@ -734,7 +733,7 @@ public class RequestResponseTest {
6, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), emptyList(), records));
FetchResponse<MemoryRecords> response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID);
FetchResponse<MemoryRecords> deserialized = FetchResponse.parse(response.serializeBody((short) 4), (short) 4);
FetchResponse<MemoryRecords> deserialized = FetchResponse.parse(response.serialize((short) 4), (short) 4);
assertEquals(responseData, deserialized.responseData());
}
@ -752,7 +751,7 @@ public class RequestResponseTest {
int correlationId = 15;
short responseHeaderVersion = FETCH.responseHeaderVersion(apiVersion);
Send send = fetchResponse.toSend("1", new ResponseHeader(correlationId, responseHeaderVersion), apiVersion);
Send send = fetchResponse.toSend(new ResponseHeader(correlationId, responseHeaderVersion), apiVersion);
ByteBufferChannel channel = new ByteBufferChannel(send.size());
send.writeTo(channel);
channel.close();
@ -767,7 +766,7 @@ public class RequestResponseTest {
ResponseHeader responseHeader = ResponseHeader.parse(channel.buffer(), responseHeaderVersion);
assertEquals(correlationId, responseHeader.correlationId());
assertEquals(fetchResponse.serializeBody(apiVersion), buf);
assertEquals(fetchResponse.serialize(apiVersion), buf);
FetchResponseData deserialized = new FetchResponseData(new ByteBufferAccessor(buf), apiVersion);
ObjectSerializationCache serializationCache = new ObjectSerializationCache();
assertEquals(size, responseHeader.size(serializationCache) + deserialized.size(serializationCache, apiVersion));
@ -777,7 +776,7 @@ public class RequestResponseTest {
public void testControlledShutdownResponse() {
ControlledShutdownResponse response = createControlledShutdownResponse();
short version = ApiKeys.CONTROLLED_SHUTDOWN.latestVersion();
ByteBuffer buffer = response.serializeBody(version);
ByteBuffer buffer = response.serialize(version);
ControlledShutdownResponse deserialized = ControlledShutdownResponse.parse(buffer, version);
assertEquals(response.error(), deserialized.error());
assertEquals(response.data().remainingPartitions(), deserialized.data().remainingPartitions());
@ -814,7 +813,7 @@ public class RequestResponseTest {
public void testFetchRequestMaxBytesOldVersions() {
final short version = 1;
FetchRequest fr = createFetchRequest(version);
FetchRequest fr2 = FetchRequest.parse(fr.serializeBody(), version);
FetchRequest fr2 = FetchRequest.parse(fr.serialize(), version);
assertEquals(fr2.maxBytes(), fr.maxBytes());
}
@ -822,12 +821,12 @@ public class RequestResponseTest {
public void testFetchRequestIsolationLevel() throws Exception {
FetchRequest request = createFetchRequest(4, IsolationLevel.READ_COMMITTED);
FetchRequest deserialized = (FetchRequest) AbstractRequest.parseRequest(request.apiKey(), request.version(),
request.serializeBody()).request;
request.serialize()).request;
assertEquals(request.isolationLevel(), deserialized.isolationLevel());
request = createFetchRequest(4, IsolationLevel.READ_UNCOMMITTED);
deserialized = (FetchRequest) AbstractRequest.parseRequest(request.apiKey(), request.version(),
request.serializeBody()).request;
request.serialize()).request;
assertEquals(request.isolationLevel(), deserialized.isolationLevel());
}
@ -835,12 +834,12 @@ public class RequestResponseTest {
public void testFetchRequestWithMetadata() throws Exception {
FetchRequest request = createFetchRequest(4, IsolationLevel.READ_COMMITTED);
FetchRequest deserialized = (FetchRequest) AbstractRequest.parseRequest(ApiKeys.FETCH, request.version(),
request.serializeBody()).request;
request.serialize()).request;
assertEquals(request.isolationLevel(), deserialized.isolationLevel());
request = createFetchRequest(4, IsolationLevel.READ_UNCOMMITTED);
deserialized = (FetchRequest) AbstractRequest.parseRequest(ApiKeys.FETCH, request.version(),
request.serializeBody()).request;
request.serialize()).request;
assertEquals(request.isolationLevel(), deserialized.isolationLevel());
}
@ -866,7 +865,7 @@ public class RequestResponseTest {
public void testJoinGroupRequestVersion0RebalanceTimeout() {
final short version = 0;
JoinGroupRequest jgr = createJoinGroupRequest(version);
JoinGroupRequest jgr2 = JoinGroupRequest.parse(jgr.serializeBody(), version);
JoinGroupRequest jgr2 = JoinGroupRequest.parse(jgr.serialize(), version);
assertEquals(jgr2.data().rebalanceTimeoutMs(), jgr.data().rebalanceTimeoutMs());
}
@ -958,7 +957,7 @@ public class RequestResponseTest {
@Test
public void testApiVersionResponseParsingFallback() {
ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serializeBody((short) 0);
ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize((short) 0);
ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, ApiKeys.API_VERSIONS.latestVersion());
assertEquals(Errors.NONE.code(), response.data.errorCode());
@ -972,7 +971,7 @@ public class RequestResponseTest {
@Test
public void testApiVersionResponseParsing() {
ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serializeBody(ApiKeys.API_VERSIONS.latestVersion());
ByteBuffer buffer = ApiVersionsResponse.DEFAULT_API_VERSIONS_RESPONSE.serialize(ApiKeys.API_VERSIONS.latestVersion());
ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, ApiKeys.API_VERSIONS.latestVersion());
assertEquals(Errors.NONE.code(), response.data.errorCode());
@ -985,7 +984,7 @@ public class RequestResponseTest {
setTransactionalId("abracadabra").
setProducerId(123));
final UnsupportedVersionException exception = assertThrows(
UnsupportedVersionException.class, () -> bld.build((short) 2).serializeBody());
UnsupportedVersionException.class, () -> bld.build((short) 2).serialize());
assertTrue(exception.getMessage().contains("Attempted to write a non-default producerId at version 2"));
bld.build((short) 3);
}
@ -1349,7 +1348,7 @@ public class RequestResponseTest {
new TopicPartition("topic3", 0), Optional.empty(),
Optional.empty(), replicas, isr, offlineReplicas))));
return TestUtils.metadataResponse(asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata);
return RequestTestUtils.metadataResponse(asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata);
}
private OffsetCommitRequest createOffsetCommitRequest(int version) {

View File

@ -0,0 +1,193 @@
/*
* 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.common.requests;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.MetadataResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.record.RecordBatch;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
public class RequestTestUtils {
public static ByteBuffer serializeRequestHeader(RequestHeader header) {
ObjectSerializationCache serializationCache = new ObjectSerializationCache();
ByteBuffer buffer = ByteBuffer.allocate(header.size(serializationCache));
header.write(buffer, serializationCache);
buffer.flip();
return buffer;
}
public static ByteBuffer serializeRequestWithHeader(RequestHeader header, AbstractRequest request) {
return RequestUtils.serialize(header.data(), header.headerVersion(), request.data(), request.version());
}
public static ByteBuffer serializeResponseWithHeader(AbstractResponse response, short version, int correlationId) {
return response.serializeWithHeader(new ResponseHeader(correlationId,
response.apiKey().responseHeaderVersion(version)), version);
}
public static MetadataResponse metadataResponse(Collection<Node> brokers,
String clusterId, int controllerId,
List<MetadataResponse.TopicMetadata> topicMetadataList) {
return metadataResponse(brokers, clusterId, controllerId, topicMetadataList, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataResponse(Collection<Node> brokers,
String clusterId, int controllerId,
List<MetadataResponse.TopicMetadata> topicMetadataList,
short responseVersion) {
return metadataResponse(MetadataResponse.DEFAULT_THROTTLE_TIME, brokers, clusterId, controllerId,
topicMetadataList, MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED, responseVersion);
}
public static MetadataResponse metadataResponse(int throttleTimeMs, Collection<Node> brokers,
String clusterId, int controllerId,
List<MetadataResponse.TopicMetadata> topicMetadatas,
int clusterAuthorizedOperations,
short responseVersion) {
List<MetadataResponseData.MetadataResponseTopic> topics = new ArrayList<>();
topicMetadatas.forEach(topicMetadata -> {
MetadataResponseData.MetadataResponseTopic metadataResponseTopic = new MetadataResponseData.MetadataResponseTopic();
metadataResponseTopic
.setErrorCode(topicMetadata.error().code())
.setName(topicMetadata.topic())
.setIsInternal(topicMetadata.isInternal())
.setTopicAuthorizedOperations(topicMetadata.authorizedOperations());
for (MetadataResponse.PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata()) {
metadataResponseTopic.partitions().add(new MetadataResponseData.MetadataResponsePartition()
.setErrorCode(partitionMetadata.error.code())
.setPartitionIndex(partitionMetadata.partition())
.setLeaderId(partitionMetadata.leaderId.orElse(MetadataResponse.NO_LEADER_ID))
.setLeaderEpoch(partitionMetadata.leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
.setReplicaNodes(partitionMetadata.replicaIds)
.setIsrNodes(partitionMetadata.inSyncReplicaIds)
.setOfflineReplicas(partitionMetadata.offlineReplicaIds));
}
topics.add(metadataResponseTopic);
});
return MetadataResponse.prepareResponse(responseVersion, throttleTimeMs, brokers, clusterId, controllerId,
topics, clusterAuthorizedOperations); }
public static MetadataResponse metadataUpdateWith(final int numNodes,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith("kafka-cluster", numNodes, topicPartitionCounts);
}
public static MetadataResponse metadataUpdateWith(final int numNodes,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier) {
return metadataUpdateWith("kafka-cluster", numNodes, Collections.emptyMap(),
topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith(clusterId, numNodes, Collections.emptyMap(),
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final short responseVersion) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, responseVersion);
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier,
final PartitionMetadataSupplier partitionSupplier,
final short responseVersion) {
final List<Node> nodes = new ArrayList<>(numNodes);
for (int i = 0; i < numNodes; i++)
nodes.add(new Node(i, "localhost", 1969 + i));
List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>();
for (Map.Entry<String, Integer> topicPartitionCountEntry : topicPartitionCounts.entrySet()) {
String topic = topicPartitionCountEntry.getKey();
int numPartitions = topicPartitionCountEntry.getValue();
List<MetadataResponse.PartitionMetadata> partitionMetadata = new ArrayList<>(numPartitions);
for (int i = 0; i < numPartitions; i++) {
TopicPartition tp = new TopicPartition(topic, i);
Node leader = nodes.get(i % nodes.size());
List<Integer> replicaIds = Collections.singletonList(leader.id());
partitionMetadata.add(partitionSupplier.supply(
Errors.NONE, tp, Optional.of(leader.id()), Optional.ofNullable(epochSupplier.apply(tp)),
replicaIds, replicaIds, replicaIds));
}
topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, topic,
Topic.isInternal(topic), partitionMetadata));
}
for (Map.Entry<String, Errors> topicErrorEntry : topicErrors.entrySet()) {
String topic = topicErrorEntry.getKey();
topicMetadata.add(new MetadataResponse.TopicMetadata(topicErrorEntry.getValue(), topic,
Topic.isInternal(topic), Collections.emptyList()));
}
return metadataResponse(nodes, clusterId, 0, topicMetadata, responseVersion);
}
@FunctionalInterface
public interface PartitionMetadataSupplier {
MetadataResponse.PartitionMetadata supply(Errors error,
TopicPartition partition,
Optional<Integer> leaderId,
Optional<Integer> leaderEpoch,
List<Integer> replicas,
List<Integer> isr,
List<Integer> offlineReplicas);
}
}

View File

@ -134,7 +134,7 @@ public class StopReplicaRequestTest {
true, topicStates).build(version);
// Construct the request from the buffer
StopReplicaRequest request = StopReplicaRequest.parse(baseRequest.serializeBody(), version);
StopReplicaRequest request = StopReplicaRequest.parse(baseRequest.serialize(), version);
Map<TopicPartition, StopReplicaPartitionState> partitionStates =
StopReplicaRequestTest.partitionStates(request.topicStates());
@ -169,7 +169,7 @@ public class StopReplicaRequestTest {
true, topicStates).build(version);
// Construct the request from the buffer
StopReplicaRequest request = StopReplicaRequest.parse(baseRequest.serializeBody(), version);
StopReplicaRequest request = StopReplicaRequest.parse(baseRequest.serialize(), version);
Map<TopicPartition, StopReplicaPartitionState> partitionStates = request.partitionStates();
assertEquals(6, partitionStates.size());

View File

@ -18,7 +18,7 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.message.TxnOffsetCommitResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.MessageTestUtil;
import org.apache.kafka.common.protocol.MessageUtil;
import org.junit.Test;
import java.util.Arrays;
@ -56,7 +56,7 @@ public class TxnOffsetCommitResponseTest extends OffsetCommitResponseTest {
for (short version = 0; version <= ApiKeys.TXN_OFFSET_COMMIT.latestVersion(); version++) {
TxnOffsetCommitResponse response = TxnOffsetCommitResponse.parse(
MessageTestUtil.messageToByteBuffer(data, version), version);
MessageUtil.toByteBuffer(data, version), version);
assertEquals(expectedErrorCounts, response.errorCounts());
assertEquals(throttleTimeMs, response.throttleTimeMs());
assertEquals(version >= 1, response.shouldClientThrottle(version));

View File

@ -156,7 +156,7 @@ public class UpdateMetadataRequestTest {
assertEquals(2, request.controllerEpoch());
assertEquals(3, request.brokerEpoch());
ByteBuffer byteBuffer = request.serializeBody();
ByteBuffer byteBuffer = request.serialize();
UpdateMetadataRequest deserializedRequest = new UpdateMetadataRequest(new UpdateMetadataRequestData(
new ByteBufferAccessor(byteBuffer), version), version);

View File

@ -66,6 +66,7 @@ import org.apache.kafka.common.message.ListOffsetResponseData.ListOffsetPartitio
import org.apache.kafka.common.message.RequestHeaderData;
import org.apache.kafka.common.message.SaslAuthenticateRequestData;
import org.apache.kafka.common.message.SaslHandshakeRequestData;
import org.apache.kafka.common.network.ByteBufferSend;
import org.apache.kafka.common.network.CertStores;
import org.apache.kafka.common.network.ChannelBuilder;
import org.apache.kafka.common.network.ChannelBuilders;
@ -78,12 +79,12 @@ import org.apache.kafka.common.network.NetworkTestUtils;
import org.apache.kafka.common.network.NioEchoServer;
import org.apache.kafka.common.network.SaslChannelBuilder;
import org.apache.kafka.common.network.Selector;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.network.TransportLayer;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.SchemaException;
import org.apache.kafka.common.requests.ListOffsetResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.security.auth.Login;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.requests.AbstractRequest;
@ -706,7 +707,7 @@ public class SaslAuthenticatorTest {
setCorrelationId(1),
(short) 2);
ApiVersionsRequest request = new ApiVersionsRequest.Builder().build();
selector.send(request.toSend(node, header));
selector.send(new NetworkSend(node, request.toSend(header)));
ByteBuffer responseBuffer = waitForResponse();
ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion((short) 0));
ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0);
@ -764,7 +765,7 @@ public class SaslAuthenticatorTest {
ApiVersionsRequest request = new ApiVersionsRequest(new ApiVersionsRequestData().
setClientSoftwareName(" ").
setClientSoftwareVersion(" "), version);
selector.send(request.toSend(node, header));
selector.send(new NetworkSend(node, request.toSend(header)));
ByteBuffer responseBuffer = waitForResponse();
ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version));
ApiVersionsResponse response =
@ -806,7 +807,7 @@ public class SaslAuthenticatorTest {
createClientConnection(SecurityProtocol.PLAINTEXT, node);
RequestHeader header = new RequestHeader(ApiKeys.API_VERSIONS, version, "someclient", 1);
ApiVersionsRequest request = new ApiVersionsRequest.Builder().build(version);
selector.send(request.toSend(node, header));
selector.send(new NetworkSend(node, request.toSend(header)));
ByteBuffer responseBuffer = waitForResponse();
ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version));
ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, version);
@ -836,7 +837,7 @@ public class SaslAuthenticatorTest {
SaslHandshakeRequest request = buildSaslHandshakeRequest("PLAIN", ApiKeys.SASL_HANDSHAKE.latestVersion());
RequestHeader header = new RequestHeader(ApiKeys.SASL_HANDSHAKE, Short.MAX_VALUE, "someclient", 2);
selector.send(request.toSend(node1, header));
selector.send(new NetworkSend(node1, request.toSend(header)));
// This test uses a non-SASL PLAINTEXT client in order to do manual handshake.
// So the channel is in READY state.
NetworkTestUtils.waitForChannelClose(selector, node1, ChannelState.READY.state());
@ -864,7 +865,7 @@ public class SaslAuthenticatorTest {
Random random = new Random();
byte[] bytes = new byte[1024];
random.nextBytes(bytes);
selector.send(new NetworkSend(node1, ByteBuffer.wrap(bytes)));
selector.send(new NetworkSend(node1, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(bytes))));
NetworkTestUtils.waitForChannelClose(selector, node1, ChannelState.READY.state());
selector.close();
@ -875,7 +876,7 @@ public class SaslAuthenticatorTest {
String node2 = "invalid2";
createClientConnection(SecurityProtocol.PLAINTEXT, node2);
random.nextBytes(bytes);
selector.send(new NetworkSend(node2, ByteBuffer.wrap(bytes)));
selector.send(new NetworkSend(node2, ByteBufferSend.sizePrefixed(ByteBuffer.wrap(bytes))));
NetworkTestUtils.waitForChannelClose(selector, node2, ChannelState.READY.state());
selector.close();
@ -903,7 +904,7 @@ public class SaslAuthenticatorTest {
ApiVersionsRequest request = createApiVersionsRequestV0();
RequestHeader versionsHeader = new RequestHeader(ApiKeys.API_VERSIONS, request.version(), "someclient", 2);
selector.send(request.toSend(node1, versionsHeader));
selector.send(new NetworkSend(node1, request.toSend(versionsHeader)));
NetworkTestUtils.waitForChannelClose(selector, node1, ChannelState.READY.state());
selector.close();
@ -930,7 +931,7 @@ public class SaslAuthenticatorTest {
buffer.putInt(Integer.MAX_VALUE);
buffer.put(new byte[buffer.capacity() - 4]);
buffer.rewind();
selector.send(new NetworkSend(node1, buffer));
selector.send(new NetworkSend(node1, ByteBufferSend.sizePrefixed(buffer)));
NetworkTestUtils.waitForChannelClose(selector, node1, ChannelState.READY.state());
selector.close();
@ -944,7 +945,7 @@ public class SaslAuthenticatorTest {
buffer.putInt(Integer.MAX_VALUE);
buffer.put(new byte[buffer.capacity() - 4]);
buffer.rewind();
selector.send(new NetworkSend(node2, buffer));
selector.send(new NetworkSend(node2, ByteBufferSend.sizePrefixed(buffer)));
NetworkTestUtils.waitForChannelClose(selector, node2, ChannelState.READY.state());
selector.close();
@ -969,7 +970,7 @@ public class SaslAuthenticatorTest {
true).build();
RequestHeader metadataRequestHeader1 = new RequestHeader(ApiKeys.METADATA, metadataRequest1.version(),
"someclient", 1);
selector.send(metadataRequest1.toSend(node1, metadataRequestHeader1));
selector.send(new NetworkSend(node1, metadataRequest1.toSend(metadataRequestHeader1)));
NetworkTestUtils.waitForChannelClose(selector, node1, ChannelState.READY.state());
selector.close();
@ -983,7 +984,7 @@ public class SaslAuthenticatorTest {
MetadataRequest metadataRequest2 = new MetadataRequest.Builder(Collections.singletonList("sometopic"), true).build();
RequestHeader metadataRequestHeader2 = new RequestHeader(ApiKeys.METADATA,
metadataRequest2.version(), "someclient", 2);
selector.send(metadataRequest2.toSend(node2, metadataRequestHeader2));
selector.send(new NetworkSend(node2, metadataRequest2.toSend(metadataRequestHeader2)));
NetworkTestUtils.waitForChannelClose(selector, node2, ChannelState.READY.state());
selector.close();
@ -1603,7 +1604,7 @@ public class SaslAuthenticatorTest {
.setOffset(0)
.setTimestamp(0)))));
ListOffsetResponse response = new ListOffsetResponse(data);
ByteBuffer buffer = response.serializeWithHeader(LIST_OFFSETS.latestVersion(), 0);
ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(response, LIST_OFFSETS.latestVersion(), 0);
final RequestHeader header0 = new RequestHeader(LIST_OFFSETS, LIST_OFFSETS.latestVersion(), "id", SaslClientAuthenticator.MIN_RESERVED_CORRELATION_ID);
Assert.assertThrows(SchemaException.class, () -> NetworkClient.parseResponse(buffer.duplicate(), header0));
final RequestHeader header1 = new RequestHeader(LIST_OFFSETS, LIST_OFFSETS.latestVersion(), "id", 1);
@ -2042,7 +2043,7 @@ public class SaslAuthenticatorTest {
SaslAuthenticateRequest request = new SaslAuthenticateRequest.Builder(data).build();
sendKafkaRequestReceiveResponse(node, ApiKeys.SASL_AUTHENTICATE, request);
} else {
selector.send(new NetworkSend(node, authBuf));
selector.send(new NetworkSend(node, ByteBufferSend.sizePrefixed(authBuf)));
waitForResponse();
}
@ -2166,7 +2167,7 @@ public class SaslAuthenticatorTest {
private AbstractResponse sendKafkaRequestReceiveResponse(String node, ApiKeys apiKey, AbstractRequest request) throws IOException {
RequestHeader header = new RequestHeader(apiKey, request.version(), "someclient", nextCorrelationId++);
Send send = request.toSend(node, header);
NetworkSend send = new NetworkSend(node, request.toSend(header));
selector.send(send);
ByteBuffer responseBuffer = waitForResponse();
return NetworkClient.parseResponse(responseBuffer, header);

View File

@ -27,13 +27,13 @@ import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.network.TransportLayer;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.requests.ApiVersionsRequest;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.security.plain.PlainLoginModule;
import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.test.TestUtils;
import org.junit.Test;
import javax.security.auth.Subject;
@ -80,7 +80,7 @@ public class SaslServerAuthenticatorTest {
SCRAM_SHA_256.mechanismName(), new DefaultChannelMetadataRegistry());
RequestHeader header = new RequestHeader(ApiKeys.METADATA, (short) 0, "clientId", 13243);
ByteBuffer headerBuffer = TestUtils.serializeRequestHeader(header);
ByteBuffer headerBuffer = RequestTestUtils.serializeRequestHeader(header);
when(transportLayer.read(any(ByteBuffer.class))).then(invocation -> {
invocation.<ByteBuffer>getArgument(0).putInt(headerBuffer.remaining());
@ -123,10 +123,10 @@ public class SaslServerAuthenticatorTest {
SCRAM_SHA_256.mechanismName(), metadataRegistry);
RequestHeader header = new RequestHeader(ApiKeys.API_VERSIONS, version, "clientId", 0);
ByteBuffer headerBuffer = TestUtils.serializeRequestHeader(header);
ByteBuffer headerBuffer = RequestTestUtils.serializeRequestHeader(header);
ApiVersionsRequest request = new ApiVersionsRequest.Builder().build(version);
ByteBuffer requestBuffer = request.serializeBody();
ByteBuffer requestBuffer = request.serialize();
requestBuffer.rewind();
when(transportLayer.socketChannel().socket().getInetAddress()).thenReturn(InetAddress.getLoopbackAddress());

View File

@ -19,8 +19,8 @@ package org.apache.kafka.test;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.network.ChannelState;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.NetworkSend;
import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.requests.ByteBufferChannel;
import org.apache.kafka.common.utils.Time;
@ -39,8 +39,8 @@ import java.util.Map;
public class MockSelector implements Selectable {
private final Time time;
private final List<Send> initiatedSends = new ArrayList<>();
private final List<Send> completedSends = new ArrayList<>();
private final List<NetworkSend> initiatedSends = new ArrayList<>();
private final List<NetworkSend> completedSends = new ArrayList<>();
private final List<ByteBufferChannel> completedSendBuffers = new ArrayList<>();
private final List<NetworkReceive> completedReceives = new ArrayList<>();
private final Map<String, ChannelState> disconnected = new HashMap<>();
@ -107,8 +107,8 @@ public class MockSelector implements Selectable {
close(id);
}
private void removeSendsForNode(String id, Collection<Send> sends) {
sends.removeIf(send -> id.equals(send.destination()));
private void removeSendsForNode(String id, Collection<NetworkSend> sends) {
sends.removeIf(send -> id.equals(send.destinationId()));
}
public void clear() {
@ -120,7 +120,7 @@ public class MockSelector implements Selectable {
}
@Override
public void send(Send send) {
public void send(NetworkSend send) {
this.initiatedSends.add(send);
}
@ -132,13 +132,13 @@ public class MockSelector implements Selectable {
}
private void completeInitiatedSends() throws IOException {
for (Send send : initiatedSends) {
for (NetworkSend send : initiatedSends) {
completeSend(send);
}
this.initiatedSends.clear();
}
private void completeSend(Send send) throws IOException {
private void completeSend(NetworkSend send) throws IOException {
// Consume the send so that we will be able to send more requests to the destination
try (ByteBufferChannel discardChannel = new ByteBufferChannel(send.size())) {
while (!send.completed()) {
@ -150,11 +150,11 @@ public class MockSelector implements Selectable {
}
private void completeDelayedReceives() {
for (Send completedSend : completedSends) {
for (NetworkSend completedSend : completedSends) {
Iterator<DelayedReceive> delayedReceiveIterator = delayedReceives.iterator();
while (delayedReceiveIterator.hasNext()) {
DelayedReceive delayedReceive = delayedReceiveIterator.next();
if (delayedReceive.source().equals(completedSend.destination())) {
if (delayedReceive.source().equals(completedSend.destinationId())) {
completedReceives.add(delayedReceive.receive());
delayedReceiveIterator.remove();
}
@ -163,7 +163,7 @@ public class MockSelector implements Selectable {
}
@Override
public List<Send> completedSends() {
public List<NetworkSend> completedSends() {
return completedSends;
}

View File

@ -23,16 +23,10 @@ import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.internals.Topic;
import org.apache.kafka.common.message.MetadataResponseData;
import org.apache.kafka.common.network.NetworkReceive;
import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.requests.ByteBufferChannel;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
@ -62,7 +56,6 @@ import java.util.UUID;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -118,144 +111,6 @@ public class TestUtils {
return new Cluster("kafka-cluster", asList(ns), parts, Collections.emptySet(), Collections.emptySet());
}
public static MetadataResponse metadataResponse(Collection<Node> brokers,
String clusterId, int controllerId,
List<MetadataResponse.TopicMetadata> topicMetadataList) {
return metadataResponse(MetadataResponse.DEFAULT_THROTTLE_TIME, brokers, clusterId, controllerId,
topicMetadataList, MetadataResponse.AUTHORIZED_OPERATIONS_OMITTED);
}
public static MetadataResponse metadataResponse(int throttleTimeMs, Collection<Node> brokers,
String clusterId, int controllerId,
List<MetadataResponse.TopicMetadata> topicMetadatas,
int clusterAuthorizedOperations) {
List<MetadataResponseData.MetadataResponseTopic> topics = new ArrayList<>();
topicMetadatas.forEach(topicMetadata -> {
MetadataResponseData.MetadataResponseTopic metadataResponseTopic = new MetadataResponseData.MetadataResponseTopic();
metadataResponseTopic
.setErrorCode(topicMetadata.error().code())
.setName(topicMetadata.topic())
.setIsInternal(topicMetadata.isInternal())
.setTopicAuthorizedOperations(topicMetadata.authorizedOperations());
for (MetadataResponse.PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata()) {
metadataResponseTopic.partitions().add(new MetadataResponseData.MetadataResponsePartition()
.setErrorCode(partitionMetadata.error.code())
.setPartitionIndex(partitionMetadata.partition())
.setLeaderId(partitionMetadata.leaderId.orElse(MetadataResponse.NO_LEADER_ID))
.setLeaderEpoch(partitionMetadata.leaderEpoch.orElse(RecordBatch.NO_PARTITION_LEADER_EPOCH))
.setReplicaNodes(partitionMetadata.replicaIds)
.setIsrNodes(partitionMetadata.inSyncReplicaIds)
.setOfflineReplicas(partitionMetadata.offlineReplicaIds));
}
topics.add(metadataResponseTopic);
});
return MetadataResponse.prepareResponse(true, throttleTimeMs, brokers, clusterId, controllerId,
topics, clusterAuthorizedOperations); }
public static MetadataResponse metadataUpdateWith(final int numNodes,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith("kafka-cluster", numNodes, topicPartitionCounts);
}
public static MetadataResponse metadataUpdateWith(final int numNodes,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier) {
return metadataUpdateWith("kafka-cluster", numNodes, Collections.emptyMap(),
topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith(clusterId, numNodes, Collections.emptyMap(),
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final short responseVersion) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, tp -> null, MetadataResponse.PartitionMetadata::new, responseVersion);
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier) {
return metadataUpdateWith(clusterId, numNodes, topicErrors,
topicPartitionCounts, epochSupplier, MetadataResponse.PartitionMetadata::new, ApiKeys.METADATA.latestVersion());
}
public static MetadataResponse metadataUpdateWith(final String clusterId,
final int numNodes,
final Map<String, Errors> topicErrors,
final Map<String, Integer> topicPartitionCounts,
final Function<TopicPartition, Integer> epochSupplier,
final PartitionMetadataSupplier partitionSupplier,
final short responseVersion) {
final List<Node> nodes = new ArrayList<>(numNodes);
for (int i = 0; i < numNodes; i++)
nodes.add(new Node(i, "localhost", 1969 + i));
List<MetadataResponse.TopicMetadata> topicMetadata = new ArrayList<>();
for (Map.Entry<String, Integer> topicPartitionCountEntry : topicPartitionCounts.entrySet()) {
String topic = topicPartitionCountEntry.getKey();
int numPartitions = topicPartitionCountEntry.getValue();
List<MetadataResponse.PartitionMetadata> partitionMetadata = new ArrayList<>(numPartitions);
for (int i = 0; i < numPartitions; i++) {
TopicPartition tp = new TopicPartition(topic, i);
Node leader = nodes.get(i % nodes.size());
List<Integer> replicaIds = Collections.singletonList(leader.id());
partitionMetadata.add(partitionSupplier.supply(
Errors.NONE, tp, Optional.of(leader.id()), Optional.ofNullable(epochSupplier.apply(tp)),
replicaIds, replicaIds, replicaIds));
}
topicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, topic,
Topic.isInternal(topic), partitionMetadata));
}
for (Map.Entry<String, Errors> topicErrorEntry : topicErrors.entrySet()) {
String topic = topicErrorEntry.getKey();
topicMetadata.add(new MetadataResponse.TopicMetadata(topicErrorEntry.getValue(), topic,
Topic.isInternal(topic), Collections.emptyList()));
}
return metadataResponse(nodes, clusterId, 0, topicMetadata);
}
public static ByteBuffer serializeRequestHeader(RequestHeader header) {
ObjectSerializationCache serializationCache = new ObjectSerializationCache();
ByteBuffer buffer = ByteBuffer.allocate(header.size(serializationCache));
header.write(buffer, serializationCache);
buffer.rewind();
return buffer;
}
@FunctionalInterface
public interface PartitionMetadataSupplier {
MetadataResponse.PartitionMetadata supply(Errors error,
TopicPartition partition,
Optional<Integer> leaderId,
Optional<Integer> leaderEpoch,
List<Integer> replicas,
List<Integer> isr,
List<Integer> offlineReplicas);
}
public static Cluster clusterWith(final int nodes, final String topic, final int partitions) {
return clusterWith(nodes, Collections.singletonMap(topic, partitions));
}

View File

@ -23,11 +23,11 @@ import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.connect.storage.KafkaConfigBackingStore;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
@ -136,7 +136,7 @@ public class WorkerCoordinatorIncrementalTest {
this.time = new MockTime();
this.metadata = new Metadata(0, Long.MAX_VALUE, loggerFactory, new ClusterResourceListeners());
this.client = new MockClient(time, metadata);
this.client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.node = metadata.fetch().nodes().get(0);
this.consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time,
retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs);

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FindCoordinatorResponse;
import org.apache.kafka.common.requests.JoinGroupResponse;
import org.apache.kafka.common.requests.RequestTestUtils;
import org.apache.kafka.common.requests.SyncGroupRequest;
import org.apache.kafka.common.requests.SyncGroupResponse;
import org.apache.kafka.common.utils.LogContext;
@ -38,7 +39,6 @@ import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.runtime.TargetState;
import org.apache.kafka.connect.storage.KafkaConfigBackingStore;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.test.TestUtils;
import org.easymock.EasyMock;
import org.easymock.Mock;
import org.junit.After;
@ -122,7 +122,7 @@ public class WorkerCoordinatorTest {
this.time = new MockTime();
this.metadata = new Metadata(0, Long.MAX_VALUE, logContext, new ClusterResourceListeners());
this.client = new MockClient(time, metadata);
this.client.updateMetadata(TestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.node = metadata.fetch().nodes().get(0);
this.consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time, 100, 1000, heartbeatIntervalMs);
this.metrics = new Metrics(time);

View File

@ -1004,7 +1004,7 @@ object GroupMetadataManager {
* @return key for offset commit message
*/
def offsetCommitKey(groupId: String, topicPartition: TopicPartition): Array[Byte] = {
MessageUtil.toBytes(OffsetCommitKey.HIGHEST_SUPPORTED_VERSION,
MessageUtil.toVersionPrefixedBytes(OffsetCommitKey.HIGHEST_SUPPORTED_VERSION,
new OffsetCommitKey()
.setGroup(groupId)
.setTopic(topicPartition.topic)
@ -1018,7 +1018,7 @@ object GroupMetadataManager {
* @return key bytes for group metadata message
*/
def groupMetadataKey(groupId: String): Array[Byte] = {
MessageUtil.toBytes(GroupMetadataKeyData.HIGHEST_SUPPORTED_VERSION,
MessageUtil.toVersionPrefixedBytes(GroupMetadataKeyData.HIGHEST_SUPPORTED_VERSION,
new GroupMetadataKeyData()
.setGroup(groupId))
}
@ -1036,7 +1036,7 @@ object GroupMetadataManager {
if (apiVersion < KAFKA_2_1_IV0 || offsetAndMetadata.expireTimestamp.nonEmpty) 1.toShort
else if (apiVersion < KAFKA_2_1_IV1) 2.toShort
else 3.toShort
MessageUtil.toBytes(version, new OffsetCommitValue()
MessageUtil.toVersionPrefixedBytes(version, new OffsetCommitValue()
.setOffset(offsetAndMetadata.offset)
.setMetadata(offsetAndMetadata.metadata)
.setCommitTimestamp(offsetAndMetadata.commitTimestamp)
@ -1065,7 +1065,7 @@ object GroupMetadataManager {
else if (apiVersion < KAFKA_2_3_IV0) 2.toShort
else 3.toShort
MessageUtil.toBytes(version, new GroupMetadataValue()
MessageUtil.toVersionPrefixedBytes(version, new GroupMetadataValue()
.setProtocolType(groupMetadata.protocolType.getOrElse(""))
.setGeneration(groupMetadata.generationId)
.setProtocol(groupMetadata.protocolName.orNull)

View File

@ -60,7 +60,7 @@ object TransactionLog {
* @return key bytes
*/
private[transaction] def keyToBytes(transactionalId: String): Array[Byte] = {
MessageUtil.toBytes(TransactionLogKey.HIGHEST_SUPPORTED_VERSION,
MessageUtil.toVersionPrefixedBytes(TransactionLogKey.HIGHEST_SUPPORTED_VERSION,
new TransactionLogKey().setTransactionalId(transactionalId))
}
@ -82,7 +82,7 @@ object TransactionLog {
.setPartitionIds(partitions.map(tp => Integer.valueOf(tp.partition)).toList.asJava)
}.toList.asJava
MessageUtil.toBytes(TransactionLogValue.HIGHEST_SUPPORTED_VERSION,
MessageUtil.toVersionPrefixedBytes(TransactionLogValue.HIGHEST_SUPPORTED_VERSION,
new TransactionLogValue()
.setProducerId(txnMetadata.producerId)
.setProducerEpoch(txnMetadata.producerEpoch)

View File

@ -25,7 +25,6 @@ import java.util
import java.util.Optional
import java.util.concurrent._
import java.util.concurrent.atomic._
import kafka.cluster.{BrokerEndPoint, EndPoint}
import kafka.metrics.KafkaMetricsGroup
import kafka.network.ConnectionQuotas._
@ -42,7 +41,7 @@ import org.apache.kafka.common.memory.{MemoryPool, SimpleMemoryPool}
import org.apache.kafka.common.metrics._
import org.apache.kafka.common.metrics.stats.{Avg, CumulativeSum, Meter, Rate}
import org.apache.kafka.common.network.KafkaChannel.ChannelMuteEvent
import org.apache.kafka.common.network.{ChannelBuilder, ChannelBuilders, ClientInformation, KafkaChannel, ListenerName, ListenerReconfigurable, Selectable, Send, Selector => KSelector}
import org.apache.kafka.common.network.{ChannelBuilder, ChannelBuilders, ClientInformation, NetworkSend, KafkaChannel, ListenerName, ListenerReconfigurable, Selectable, Send, Selector => KSelector}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{ApiVersionsRequest, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol}
@ -964,7 +963,7 @@ private[kafka] class Processor(val id: Int,
// removed from the Selector after discarding any pending staged receives.
// `openOrClosingChannel` can be None if the selector closed the connection because it was idle for too long
if (openOrClosingChannel(connectionId).isDefined) {
selector.send(responseSend)
selector.send(new NetworkSend(connectionId, responseSend))
inflightResponses += (connectionId -> response)
}
}
@ -1123,8 +1122,8 @@ private[kafka] class Processor(val id: Int,
private def processCompletedSends(): Unit = {
selector.completedSends.forEach { send =>
try {
val response = inflightResponses.remove(send.destination).getOrElse {
throw new IllegalStateException(s"Send for ${send.destination} completed, but not in `inflightResponses`")
val response = inflightResponses.remove(send.destinationId).getOrElse {
throw new IllegalStateException(s"Send for ${send.destinationId} completed, but not in `inflightResponses`")
}
updateRequestMetrics(response)
@ -1134,11 +1133,11 @@ private[kafka] class Processor(val id: Int,
// Try unmuting the channel. If there was no quota violation and the channel has not been throttled,
// it will be unmuted immediately. If the channel has been throttled, it will unmuted only if the throttling
// delay has already passed by now.
handleChannelMuteEvent(send.destination, ChannelMuteEvent.RESPONSE_SENT)
tryUnmuteChannel(send.destination)
handleChannelMuteEvent(send.destinationId, ChannelMuteEvent.RESPONSE_SENT)
tryUnmuteChannel(send.destinationId)
} catch {
case e: Throwable => processChannelException(send.destination,
s"Exception while processing completed send to ${send.destination}", e)
case e: Throwable => processChannelException(send.destinationId,
s"Exception while processing completed send to ${send.destinationId}", e)
}
}
selector.clearCompletedSends()

View File

@ -19,10 +19,8 @@ package kafka.server
import java.util.concurrent.{CountDownLatch, LinkedBlockingDeque, TimeUnit}
import java.util.Collections
import kafka.cluster.{Broker, EndPoint}
import kafka.utils.TestUtils
import org.apache.kafka.test.{TestUtils => ClientsTestUtils}
import org.apache.kafka.clients.{ClientResponse, ManualMetadataUpdater, Metadata, MockClient}
import org.apache.kafka.common.feature.Features
import org.apache.kafka.common.feature.Features.emptySupportedFeatures
@ -30,7 +28,7 @@ import org.apache.kafka.common.utils.{MockTime, SystemTime}
import org.apache.kafka.common.message.MetadataRequestData
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractRequest, MetadataRequest, MetadataResponse}
import org.apache.kafka.common.requests.{AbstractRequest, MetadataRequest, MetadataResponse, RequestTestUtils}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
import org.junit.Test
@ -58,7 +56,7 @@ class BrokerToControllerRequestThreadTest {
when(metadataCache.getAliveBrokers).thenReturn(Seq(activeController))
when(metadataCache.getAliveBroker(controllerId)).thenReturn(Some(activeController))
val expectedResponse = ClientsTestUtils.metadataUpdateWith(2, Collections.singletonMap("a", new Integer(2)))
val expectedResponse = RequestTestUtils.metadataUpdateWith(2, Collections.singletonMap("a", new Integer(2)))
val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), requestQueue, metadataCache,
config, listenerName, time, "")
mockClient.prepareResponse(expectedResponse)
@ -102,7 +100,7 @@ class BrokerToControllerRequestThreadTest {
when(metadataCache.getAliveBroker(newControllerId)).thenReturn(Some(newController))
when(metadataCache.getAliveBrokers).thenReturn(Seq(oldController, newController))
val expectedResponse = ClientsTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", new Integer(2)))
val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", new Integer(2)))
val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(),
requestQueue, metadataCache, config, listenerName, time, "")
@ -157,10 +155,10 @@ class BrokerToControllerRequestThreadTest {
when(metadataCache.getAliveBroker(oldControllerId)).thenReturn(Some(oldController))
when(metadataCache.getAliveBroker(newControllerId)).thenReturn(Some(newController))
val responseWithNotControllerError = ClientsTestUtils.metadataUpdateWith("cluster1", 2,
val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2,
Collections.singletonMap("a", Errors.NOT_CONTROLLER),
Collections.singletonMap("a", new Integer(2)))
val expectedResponse = ClientsTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", new Integer(2)))
val expectedResponse = RequestTestUtils.metadataUpdateWith(3, Collections.singletonMap("a", new Integer(2)))
val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), requestQueue, metadataCache,
config, listenerName, time, "")
@ -207,7 +205,7 @@ class BrokerToControllerRequestThreadTest {
when(metadataCache.getAliveBrokers).thenReturn(Seq(controller))
when(metadataCache.getAliveBroker(controllerId)).thenReturn(Some(controller))
val responseWithNotControllerError = ClientsTestUtils.metadataUpdateWith("cluster1", 2,
val responseWithNotControllerError = RequestTestUtils.metadataUpdateWith("cluster1", 2,
Collections.singletonMap("a", Errors.NOT_CONTROLLER),
Collections.singletonMap("a", new Integer(2)))
val testRequestThread = new BrokerToControllerRequestThread(mockClient, new ManualMetadataUpdater(), requestQueue, metadataCache,

View File

@ -165,7 +165,8 @@ class RequestChannelTest {
}
def request(req: AbstractRequest): RequestChannel.Request = {
val buffer = req.serializeWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1))
val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1),
req)
val requestContext = newRequestContext(buffer)
new network.RequestChannel.Request(processor = 1,
requestContext,

View File

@ -25,8 +25,8 @@ import java.nio.charset.StandardCharsets
import java.util
import java.util.concurrent.{CompletableFuture, ConcurrentLinkedQueue, Executors, TimeUnit}
import java.util.{Properties, Random}
import com.yammer.metrics.core.{Gauge, Meter}
import javax.net.ssl._
import kafka.metrics.KafkaYammerMetrics
import kafka.security.CredentialProvider
@ -43,7 +43,7 @@ import org.apache.kafka.common.requests
import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, MockTime, Time}
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, MockTime, Time, Utils}
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
import org.apache.log4j.Level
import org.junit.Assert._
@ -111,11 +111,8 @@ class SocketServerTest {
outgoing.flush()
}
def sendApiRequest(socket: Socket, request: AbstractRequest, header: RequestHeader) = {
val byteBuffer = request.serializeWithHeader(header)
byteBuffer.rewind()
val serializedBytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(serializedBytes)
def sendApiRequest(socket: Socket, request: AbstractRequest, header: RequestHeader): Unit = {
val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
sendRequest(socket, serializedBytes)
}
@ -141,10 +138,8 @@ class SocketServerTest {
}
def processRequest(channel: RequestChannel, request: RequestChannel.Request): Unit = {
val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header)
byteBuffer.rewind()
val send = new NetworkSend(request.context.connectionId, byteBuffer)
val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest])
val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer))
channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(request.header.toString), None))
}
@ -214,22 +209,13 @@ class SocketServerTest {
.setTransactionalId(null))
.build()
val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId)
val byteBuffer = emptyRequest.serializeWithHeader(emptyHeader)
byteBuffer.rewind()
val serializedBytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(serializedBytes)
serializedBytes
Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest))
}
private def apiVersionRequestBytes(clientId: String, version: Short): Array[Byte] = {
val request = new ApiVersionsRequest.Builder().build(version)
val header = new RequestHeader(ApiKeys.API_VERSIONS, request.version(), clientId, -1)
val buffer = request.serializeWithHeader(header)
buffer.rewind()
val bytes = new Array[Byte](buffer.remaining())
buffer.get(bytes)
bytes
Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
}
@Test
@ -378,12 +364,9 @@ class SocketServerTest {
val correlationId = 57
val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId)
val request = new VoteRequest.Builder(new VoteRequestData()).build()
val byteBuffer = request.serializeWithHeader(header)
byteBuffer.rewind()
val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
val socket = connect()
val serializedBytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(serializedBytes)
val outgoing = new DataOutputStream(socket.getOutputStream)
try {
@ -687,8 +670,8 @@ class SocketServerTest {
// Mimic a primitive request handler that fetches the request from RequestChannel and place a response with a
// throttled channel.
val request = receiveRequest(server.dataPlaneRequestChannel)
val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header)
val send = new NetworkSend(request.context.connectionId, byteBuffer)
val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest])
val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer))
def channelThrottlingCallback(response: RequestChannel.Response): Unit = {
server.dataPlaneRequestChannel.sendResponse(response)
}
@ -966,11 +949,7 @@ class SocketServerTest {
.setTransactionalId(null))
.build()
val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId)
val byteBuffer = emptyRequest.serializeWithHeader(emptyHeader)
byteBuffer.rewind()
val serializedBytes = new Array[Byte](byteBuffer.remaining)
byteBuffer.get(serializedBytes)
val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest))
sendRequest(sslSocket, serializedBytes)
processRequest(overrideServer.dataPlaneRequestChannel)
@ -1113,7 +1092,7 @@ class SocketServerTest {
val requestMetrics = channel.metrics(request.header.apiKey.name)
def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count
val send = new NetworkSend(request.context.connectionId, ByteBuffer.allocate(responseBufferSize))
val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(ByteBuffer.allocate(responseBufferSize)))
channel.sendResponse(new RequestChannel.SendResponse(request, send, Some("someResponse"), None))
val expectedTotalTimeCount = totalTimeHistCount() + 1
@ -2002,7 +1981,7 @@ class SocketServerTest {
}
}
class CompletedSendsPollData(selector: TestableSelector) extends PollData[Send] {
class CompletedSendsPollData(selector: TestableSelector) extends PollData[NetworkSend] {
override def updateResults(): Unit = {
val currentSends = update(selector.completedSends.asScala)
selector.completedSends.clear()
@ -2059,8 +2038,8 @@ class SocketServerTest {
}
}
override def send(s: Send): Unit = {
runOp(SelectorOperation.Send, Some(s.destination)) {
override def send(s: NetworkSend): Unit = {
runOp(SelectorOperation.Send, Some(s.destinationId)) {
super.send(s)
}
}

View File

@ -19,7 +19,6 @@ package kafka.server
import java.net.InetAddress
import java.util
import java.util.Collections
import kafka.network.RequestChannel
import kafka.network.RequestChannel.EndThrottlingResponse
import kafka.network.RequestChannel.Session
@ -30,11 +29,8 @@ import org.apache.kafka.common.metrics.MetricConfig
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ClientInformation
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.requests.AbstractRequest
import org.apache.kafka.common.requests.FetchRequest
import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader, RequestTestUtils}
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.RequestContext
import org.apache.kafka.common.requests.RequestHeader
import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.MockTime
@ -60,10 +56,11 @@ class BaseClientQuotaManagerTest {
}
protected def buildRequest[T <: AbstractRequest](builder: AbstractRequest.Builder[T],
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
val request = builder.build()
val buffer = request.serializeWithHeader(new RequestHeader(builder.apiKey, request.version, "", 0))
val buffer = RequestTestUtils.serializeRequestWithHeader(
new RequestHeader(builder.apiKey, request.version, "", 0), request)
val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics])
// read the header from the buffer first so that the body can be read next from the Request constructor

View File

@ -21,13 +21,13 @@ import java.io.{DataInputStream, DataOutputStream}
import java.net.Socket
import java.nio.ByteBuffer
import java.util.Properties
import kafka.api.IntegrationTestHarness
import kafka.network.SocketServer
import kafka.utils.NotNothing
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader}
import org.apache.kafka.common.utils.Utils
import scala.annotation.nowarn
import scala.collection.Seq
@ -134,7 +134,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness {
}
def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = {
val serializedBytes = request.serializeWithHeader(header).array
val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
sendRequest(socket, serializedBytes)
}

View File

@ -139,7 +139,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
.setTimeoutMs(10000)
.setTransactionalId(null))
.build()
val bodyBytes = request.serializeBody
val bodyBytes = request.serialize
val byteBuffer = ByteBuffer.allocate(headerBytes.length + bodyBytes.remaining())
byteBuffer.put(headerBytes)
byteBuffer.put(bodyBytes)

View File

@ -19,7 +19,6 @@ package kafka.server
import java.net.InetAddress
import java.nio.ByteBuffer
import java.util.Optional
import kafka.network
import kafka.network.RequestChannel
import kafka.utils.MockTime
@ -29,7 +28,7 @@ import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.AlterConfigsResponseData
import org.apache.kafka.common.network.{ClientInformation, ListenerName}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AlterConfigsRequest, AlterConfigsResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AlterConfigsRequest, AlterConfigsResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader, RequestTestUtils}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
import org.junit.Assert._
@ -60,7 +59,8 @@ class ForwardingManagerTest {
val request = buildRequest(requestHeader, requestBuffer, clientPrincipal)
val responseBody = new AlterConfigsResponse(new AlterConfigsResponseData())
val responseBuffer = responseBody.serializeWithHeader(requestBody.version, requestCorrelationId + 1)
val responseBuffer = RequestTestUtils.serializeResponseWithHeader(responseBody, requestHeader.apiVersion,
requestCorrelationId + 1)
Mockito.when(brokerToController.sendRequest(
any(classOf[EnvelopeRequest.Builder]),
@ -118,7 +118,7 @@ class ForwardingManagerTest {
"clientId",
correlationId
)
val buffer = body.serializeWithHeader(header)
val buffer = RequestTestUtils.serializeRequestWithHeader(header, body)
// Fast-forward buffer to start of the request as `RequestChannel.Request` expects
RequestHeader.parse(buffer)

View File

@ -353,7 +353,7 @@ class KafkaApisTest {
configResource -> new AlterConfigsRequest.Config(
Seq(new AlterConfigsRequest.ConfigEntry("foo", "bar")).asJava))
val alterConfigsRequest = new AlterConfigsRequest.Builder(configs.asJava, false).build(requestHeader.apiVersion)
val serializedRequestData = alterConfigsRequest.serializeWithHeader(requestHeader)
val serializedRequestData = RequestTestUtils.serializeRequestWithHeader(requestHeader, alterConfigsRequest)
val capturedResponse = expectNoThrottling()
@ -376,7 +376,7 @@ class KafkaApisTest {
clientId, 0)
val leaveGroupRequest = new LeaveGroupRequest.Builder("group",
Collections.singletonList(new MemberIdentity())).build(requestHeader.apiVersion)
val serializedRequestData = leaveGroupRequest.serializeWithHeader(requestHeader)
val serializedRequestData = RequestTestUtils.serializeRequestWithHeader(requestHeader, leaveGroupRequest)
val capturedResponse = expectNoThrottling()
@ -2893,17 +2893,17 @@ class KafkaApisTest {
val listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val requestHeader = new RequestHeader(request.apiKey, request.version, clientId, 0)
val requestBuffer = request.serializeWithHeader(requestHeader)
val requestBuffer = RequestTestUtils.serializeRequestWithHeader(requestHeader, request)
val requestContext = new RequestContext(requestHeader, "1", InetAddress.getLocalHost,
KafkaPrincipal.ANONYMOUS, listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY,
fromPrivilegedListener)
val envelopeHeader = new RequestHeader(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion(), clientId, 0)
val envelopeBuffer = new EnvelopeRequest.Builder(
val envelopeBuffer = RequestTestUtils.serializeRequestWithHeader(envelopeHeader, new EnvelopeRequest.Builder(
requestBuffer,
new Array[Byte](0),
InetAddress.getLocalHost.getAddress
).build().serializeWithHeader(envelopeHeader)
).build())
val envelopeContext = new RequestContext(envelopeHeader, "1", InetAddress.getLocalHost,
KafkaPrincipal.ANONYMOUS, listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY,
fromPrivilegedListener, principalSerde.asJava)
@ -2934,8 +2934,8 @@ class KafkaApisTest {
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
fromPrivilegedListener: Boolean = false,
requestHeader: Option[RequestHeader] = None): RequestChannel.Request = {
val buffer = request.serializeWithHeader(requestHeader.getOrElse(
new RequestHeader(request.apiKey, request.version, clientId, 0)))
val buffer = RequestTestUtils.serializeRequestWithHeader(requestHeader.getOrElse(
new RequestHeader(request.apiKey, request.version, clientId, 0)), request)
// read the header from the buffer first so that the body can be read next from the Request constructor
val header = RequestHeader.parse(buffer)

View File

@ -609,8 +609,8 @@ class RequestQuotaTest extends BaseRequestTest {
"client-id",
0
)
val embedRequestData = new AlterClientQuotasRequest.Builder(
List.empty.asJava, false).build().serializeWithHeader(requestHeader)
val embedRequestData = RequestTestUtils.serializeRequestWithHeader(requestHeader,
new AlterClientQuotasRequest.Builder(List.empty.asJava, false).build())
new EnvelopeRequest.Builder(embedRequestData, new Array[Byte](0),
InetAddress.getByName("192.168.1.1").getAddress)

View File

@ -22,7 +22,6 @@ import java.net.InetAddress
import java.util
import java.util.Collections
import java.util.concurrent.{DelayQueue, TimeUnit}
import kafka.network.RequestChannel
import kafka.network.RequestChannel.{EndThrottlingResponse, Response, StartThrottlingResponse}
import org.apache.kafka.common.TopicPartition
@ -31,7 +30,7 @@ import org.apache.kafka.common.metrics.MetricConfig
import org.apache.kafka.common.network.ClientInformation
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader}
import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader, RequestTestUtils}
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.MockTime
import org.easymock.EasyMock
@ -50,7 +49,8 @@ class ThrottledChannelExpirationTest {
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
val request = builder.build()
val buffer = request.serializeWithHeader(new RequestHeader(builder.apiKey, request.version, "", 0))
val buffer = RequestTestUtils.serializeRequestWithHeader(
new RequestHeader(builder.apiKey, request.version, "", 0), request)
val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics])
// read the header from the buffer first so that the body can be read next from the Request constructor

View File

@ -86,7 +86,7 @@ public class FetchRequestBenchmark {
.build(ApiKeys.FETCH.latestVersion());
this.replicaRequest = FetchRequest.Builder.forReplica(ApiKeys.FETCH.latestVersion(), 1, 0, 0, fetchData)
.build(ApiKeys.FETCH.latestVersion());
this.requestBuffer = this.consumerRequest.serializeBody();
this.requestBuffer = this.consumerRequest.serialize();
}
@ -112,7 +112,7 @@ public class FetchRequestBenchmark {
@Benchmark
public int testSerializeFetchRequestForConsumer() throws IOException {
Send send = consumerRequest.toSend("dest", header);
Send send = consumerRequest.toSend(header);
ByteBufferChannel channel = new ByteBufferChannel(send.size());
send.writeTo(channel);
return channel.buffer().limit();
@ -120,7 +120,7 @@ public class FetchRequestBenchmark {
@Benchmark
public int testSerializeFetchRequestForReplica() throws IOException {
Send send = replicaRequest.toSend("dest", header);
Send send = replicaRequest.toSend(header);
ByteBufferChannel channel = new ByteBufferChannel(send.size());
send.writeTo(channel);
return channel.buffer().limit();

View File

@ -96,7 +96,7 @@ public class FetchResponseBenchmark {
@Benchmark
public int testSerializeFetchResponse() throws IOException {
Send send = fetchResponse.toSend("dest", header, ApiKeys.FETCH.latestVersion());
Send send = fetchResponse.toSend(header, ApiKeys.FETCH.latestVersion());
ByteBufferChannel channel = new ByteBufferChannel(send.size());
send.writeTo(channel);
return channel.buffer().limit();

View File

@ -173,7 +173,7 @@ public class ReplicaFetcherThreadBenchmark {
}
@Override
public RecordsSend<? extends BaseRecords> toSend(String destination) {
public RecordsSend<? extends BaseRecords> toSend() {
return null;
}
};

View File

@ -200,14 +200,13 @@ public class MetadataRequestBenchmark {
private RequestChannel.Request buildAllTopicMetadataRequest() {
MetadataRequest metadataRequest = MetadataRequest.Builder.allTopics().build();
ByteBuffer buffer = metadataRequest.serializeWithHeader(new RequestHeader(metadataRequest.apiKey(),
metadataRequest.version(), "", 0));
RequestHeader header = RequestHeader.parse(buffer);
RequestHeader header = new RequestHeader(metadataRequest.apiKey(), metadataRequest.version(), "", 0);
ByteBuffer bodyBuffer = metadataRequest.serialize();
RequestContext context = new RequestContext(header, "1", null, principal,
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY, false);
return new RequestChannel.Request(1, context, 0, MemoryPool.NONE, buffer, requestChannelMetrics, Option.empty());
return new RequestChannel.Request(1, context, 0, MemoryPool.NONE, bodyBuffer, requestChannelMetrics, Option.empty());
}
@Benchmark