MINOR: remove FetchResponse.AbortedTransaction and redundant construc… (#9758)

1. rename INVALID_HIGHWATERMARK to INVALID_HIGH_WATERMARK
2. replace FetchResponse.AbortedTransaction by FetchResponseData.AbortedTransaction
3. remove redundant constructors from FetchResponse.PartitionData
4. rename recordSet to records
5. add helpers "recordsOrFail" and "recordsSize" to FetchResponse to process record casting

Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Chia-Ping Tsai 2021-03-04 18:06:50 +08:00 committed by GitHub
parent 3ef39e1365
commit 8205051e90
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
45 changed files with 1102 additions and 1047 deletions

View File

@ -318,7 +318,7 @@ public class FetchSessionHandler {
* @param response The response. * @param response The response.
* @return True if the full fetch response partitions are valid. * @return True if the full fetch response partitions are valid.
*/ */
String verifyFullFetchResponsePartitions(FetchResponse<?> response) { String verifyFullFetchResponsePartitions(FetchResponse response) {
StringBuilder bld = new StringBuilder(); StringBuilder bld = new StringBuilder();
Set<TopicPartition> extra = Set<TopicPartition> extra =
findMissing(response.responseData().keySet(), sessionPartitions.keySet()); findMissing(response.responseData().keySet(), sessionPartitions.keySet());
@ -343,7 +343,7 @@ public class FetchSessionHandler {
* @param response The response. * @param response The response.
* @return True if the incremental fetch response partitions are valid. * @return True if the incremental fetch response partitions are valid.
*/ */
String verifyIncrementalFetchResponsePartitions(FetchResponse<?> response) { String verifyIncrementalFetchResponsePartitions(FetchResponse response) {
Set<TopicPartition> extra = Set<TopicPartition> extra =
findMissing(response.responseData().keySet(), sessionPartitions.keySet()); findMissing(response.responseData().keySet(), sessionPartitions.keySet());
if (!extra.isEmpty()) { if (!extra.isEmpty()) {
@ -362,7 +362,7 @@ public class FetchSessionHandler {
* @param response The FetchResponse. * @param response The FetchResponse.
* @return The string to log. * @return The string to log.
*/ */
private String responseDataToLogString(FetchResponse<?> response) { private String responseDataToLogString(FetchResponse response) {
if (!log.isTraceEnabled()) { if (!log.isTraceEnabled()) {
int implied = sessionPartitions.size() - response.responseData().size(); int implied = sessionPartitions.size() - response.responseData().size();
if (implied > 0) { if (implied > 0) {
@ -398,7 +398,7 @@ public class FetchSessionHandler {
* @return True if the response is well-formed; false if it can't be processed * @return True if the response is well-formed; false if it can't be processed
* because of missing or unexpected partitions. * because of missing or unexpected partitions.
*/ */
public boolean handleResponse(FetchResponse<?> response) { public boolean handleResponse(FetchResponse response) {
if (response.error() != Errors.NONE) { if (response.error() != Errors.NONE) {
log.info("Node {} was unable to process the fetch request with {}: {}.", log.info("Node {} was unable to process the fetch request with {}: {}.",
node, nextMetadata, response.error()); node, nextMetadata, response.error());

View File

@ -47,6 +47,7 @@ import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
@ -66,7 +67,6 @@ import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.record.ControlRecordType;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
@ -277,7 +277,7 @@ public class Fetcher<K, V> implements Closeable {
synchronized (Fetcher.this) { synchronized (Fetcher.this) {
try { try {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
FetchResponse<Records> response = (FetchResponse<Records>) resp.responseBody(); FetchResponse response = (FetchResponse) resp.responseBody();
FetchSessionHandler handler = sessionHandler(fetchTarget.id()); FetchSessionHandler handler = sessionHandler(fetchTarget.id());
if (handler == null) { if (handler == null) {
log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.", log.error("Unable to find FetchSessionHandler for node {}. Ignoring fetch response.",
@ -291,7 +291,7 @@ public class Fetcher<K, V> implements Closeable {
Set<TopicPartition> partitions = new HashSet<>(response.responseData().keySet()); Set<TopicPartition> partitions = new HashSet<>(response.responseData().keySet());
FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions); FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions);
for (Map.Entry<TopicPartition, FetchResponse.PartitionData<Records>> entry : response.responseData().entrySet()) { for (Map.Entry<TopicPartition, FetchResponseData.PartitionData> entry : response.responseData().entrySet()) {
TopicPartition partition = entry.getKey(); TopicPartition partition = entry.getKey();
FetchRequest.PartitionData requestData = data.sessionPartitions().get(partition); FetchRequest.PartitionData requestData = data.sessionPartitions().get(partition);
if (requestData == null) { if (requestData == null) {
@ -310,12 +310,12 @@ public class Fetcher<K, V> implements Closeable {
throw new IllegalStateException(message); throw new IllegalStateException(message);
} else { } else {
long fetchOffset = requestData.fetchOffset; long fetchOffset = requestData.fetchOffset;
FetchResponse.PartitionData<Records> partitionData = entry.getValue(); FetchResponseData.PartitionData partitionData = entry.getValue();
log.debug("Fetch {} at offset {} for partition {} returned fetch data {}", log.debug("Fetch {} at offset {} for partition {} returned fetch data {}",
isolationLevel, fetchOffset, partition, partitionData); isolationLevel, fetchOffset, partition, partitionData);
Iterator<? extends RecordBatch> batches = partitionData.records().batches().iterator(); Iterator<? extends RecordBatch> batches = FetchResponse.recordsOrFail(partitionData).batches().iterator();
short responseVersion = resp.requestHeader().apiVersion(); short responseVersion = resp.requestHeader().apiVersion();
completedFetches.add(new CompletedFetch(partition, partitionData, completedFetches.add(new CompletedFetch(partition, partitionData,
@ -618,8 +618,8 @@ public class Fetcher<K, V> implements Closeable {
// The first condition ensures that the completedFetches is not stuck with the same completedFetch // The first condition ensures that the completedFetches is not stuck with the same completedFetch
// in cases such as the TopicAuthorizationException, and the second condition ensures that no // in cases such as the TopicAuthorizationException, and the second condition ensures that no
// potential data loss due to an exception in a following record. // potential data loss due to an exception in a following record.
FetchResponse.PartitionData<Records> partition = records.partitionData; FetchResponseData.PartitionData partition = records.partitionData;
if (fetched.isEmpty() && (partition.records() == null || partition.records().sizeInBytes() == 0)) { if (fetched.isEmpty() && FetchResponse.recordsOrFail(partition).sizeInBytes() == 0) {
completedFetches.poll(); completedFetches.poll();
} }
throw e; throw e;
@ -1229,10 +1229,10 @@ public class Fetcher<K, V> implements Closeable {
*/ */
private CompletedFetch initializeCompletedFetch(CompletedFetch nextCompletedFetch) { private CompletedFetch initializeCompletedFetch(CompletedFetch nextCompletedFetch) {
TopicPartition tp = nextCompletedFetch.partition; TopicPartition tp = nextCompletedFetch.partition;
FetchResponse.PartitionData<Records> partition = nextCompletedFetch.partitionData; FetchResponseData.PartitionData partition = nextCompletedFetch.partitionData;
long fetchOffset = nextCompletedFetch.nextFetchOffset; long fetchOffset = nextCompletedFetch.nextFetchOffset;
CompletedFetch completedFetch = null; CompletedFetch completedFetch = null;
Errors error = partition.error(); Errors error = Errors.forCode(partition.errorCode());
try { try {
if (!subscriptions.hasValidPosition(tp)) { if (!subscriptions.hasValidPosition(tp)) {
@ -1249,11 +1249,11 @@ public class Fetcher<K, V> implements Closeable {
} }
log.trace("Preparing to read {} bytes of data for partition {} with offset {}", log.trace("Preparing to read {} bytes of data for partition {} with offset {}",
partition.records().sizeInBytes(), tp, position); FetchResponse.recordsSize(partition), tp, position);
Iterator<? extends RecordBatch> batches = partition.records().batches().iterator(); Iterator<? extends RecordBatch> batches = FetchResponse.recordsOrFail(partition).batches().iterator();
completedFetch = nextCompletedFetch; completedFetch = nextCompletedFetch;
if (!batches.hasNext() && partition.records().sizeInBytes() > 0) { if (!batches.hasNext() && FetchResponse.recordsSize(partition) > 0) {
if (completedFetch.responseVersion < 3) { if (completedFetch.responseVersion < 3) {
// Implement the pre KIP-74 behavior of throwing a RecordTooLargeException. // Implement the pre KIP-74 behavior of throwing a RecordTooLargeException.
Map<TopicPartition, Long> recordTooLargePartitions = Collections.singletonMap(tp, fetchOffset); Map<TopicPartition, Long> recordTooLargePartitions = Collections.singletonMap(tp, fetchOffset);
@ -1286,11 +1286,11 @@ public class Fetcher<K, V> implements Closeable {
subscriptions.updateLastStableOffset(tp, partition.lastStableOffset()); subscriptions.updateLastStableOffset(tp, partition.lastStableOffset());
} }
if (partition.preferredReadReplica().isPresent()) { if (FetchResponse.isPreferredReplica(partition)) {
subscriptions.updatePreferredReadReplica(completedFetch.partition, partition.preferredReadReplica().get(), () -> { subscriptions.updatePreferredReadReplica(completedFetch.partition, partition.preferredReadReplica(), () -> {
long expireTimeMs = time.milliseconds() + metadata.metadataExpireMs(); long expireTimeMs = time.milliseconds() + metadata.metadataExpireMs();
log.debug("Updating preferred read replica for partition {} to {}, set to expire at {}", log.debug("Updating preferred read replica for partition {} to {}, set to expire at {}",
tp, partition.preferredReadReplica().get(), expireTimeMs); tp, partition.preferredReadReplica(), expireTimeMs);
return expireTimeMs; return expireTimeMs;
}); });
} }
@ -1455,8 +1455,8 @@ public class Fetcher<K, V> implements Closeable {
private final TopicPartition partition; private final TopicPartition partition;
private final Iterator<? extends RecordBatch> batches; private final Iterator<? extends RecordBatch> batches;
private final Set<Long> abortedProducerIds; private final Set<Long> abortedProducerIds;
private final PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions; private final PriorityQueue<FetchResponseData.AbortedTransaction> abortedTransactions;
private final FetchResponse.PartitionData<Records> partitionData; private final FetchResponseData.PartitionData partitionData;
private final FetchResponseMetricAggregator metricAggregator; private final FetchResponseMetricAggregator metricAggregator;
private final short responseVersion; private final short responseVersion;
@ -1473,7 +1473,7 @@ public class Fetcher<K, V> implements Closeable {
private boolean initialized = false; private boolean initialized = false;
private CompletedFetch(TopicPartition partition, private CompletedFetch(TopicPartition partition,
FetchResponse.PartitionData<Records> partitionData, FetchResponseData.PartitionData partitionData,
FetchResponseMetricAggregator metricAggregator, FetchResponseMetricAggregator metricAggregator,
Iterator<? extends RecordBatch> batches, Iterator<? extends RecordBatch> batches,
Long fetchOffset, Long fetchOffset,
@ -1641,9 +1641,9 @@ public class Fetcher<K, V> implements Closeable {
if (abortedTransactions == null) if (abortedTransactions == null)
return; return;
while (!abortedTransactions.isEmpty() && abortedTransactions.peek().firstOffset <= offset) { while (!abortedTransactions.isEmpty() && abortedTransactions.peek().firstOffset() <= offset) {
FetchResponse.AbortedTransaction abortedTransaction = abortedTransactions.poll(); FetchResponseData.AbortedTransaction abortedTransaction = abortedTransactions.poll();
abortedProducerIds.add(abortedTransaction.producerId); abortedProducerIds.add(abortedTransaction.producerId());
} }
} }
@ -1651,12 +1651,12 @@ public class Fetcher<K, V> implements Closeable {
return batch.isTransactional() && abortedProducerIds.contains(batch.producerId()); return batch.isTransactional() && abortedProducerIds.contains(batch.producerId());
} }
private PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions(FetchResponse.PartitionData<?> partition) { private PriorityQueue<FetchResponseData.AbortedTransaction> abortedTransactions(FetchResponseData.PartitionData partition) {
if (partition.abortedTransactions() == null || partition.abortedTransactions().isEmpty()) if (partition.abortedTransactions() == null || partition.abortedTransactions().isEmpty())
return null; return null;
PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions = new PriorityQueue<>( PriorityQueue<FetchResponseData.AbortedTransaction> abortedTransactions = new PriorityQueue<>(
partition.abortedTransactions().size(), Comparator.comparingLong(o -> o.firstOffset) partition.abortedTransactions().size(), Comparator.comparingLong(FetchResponseData.AbortedTransaction::firstOffset)
); );
abortedTransactions.addAll(partition.abortedTransactions()); abortedTransactions.addAll(partition.abortedTransactions());
return abortedTransactions; return abortedTransactions;

View File

@ -19,10 +19,10 @@ package org.apache.kafka.common.requests;
import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchRequestData;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -296,14 +296,11 @@ public class FetchRequest extends AbstractRequest {
// may not be any partitions at all in the response. For this reason, the top-level error code // may not be any partitions at all in the response. For this reason, the top-level error code
// is essential for them. // is essential for them.
Errors error = Errors.forException(e); Errors error = Errors.forException(e);
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
for (Map.Entry<TopicPartition, PartitionData> entry : fetchData.entrySet()) { for (Map.Entry<TopicPartition, PartitionData> entry : fetchData.entrySet()) {
FetchResponse.PartitionData<MemoryRecords> partitionResponse = new FetchResponse.PartitionData<>(error, responseData.put(entry.getKey(), FetchResponse.partitionResponse(entry.getKey().partition(), error));
FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET,
FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), null, MemoryRecords.EMPTY);
responseData.put(entry.getKey(), partitionResponse);
} }
return new FetchResponse<>(error, responseData, throttleTimeMs, data.sessionId()); return FetchResponse.of(error, throttleTimeMs, data.sessionId(), responseData);
} }
public int replicaId() { public int replicaId() {

View File

@ -22,8 +22,8 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.ObjectSerializationCache;
import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.Records;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -33,7 +33,6 @@ import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors;
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
@ -57,238 +56,43 @@ import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
* the fetch offset after the index lookup * the fetch offset after the index lookup
* - {@link Errors#UNKNOWN_SERVER_ERROR} For any unexpected errors * - {@link Errors#UNKNOWN_SERVER_ERROR} For any unexpected errors
*/ */
public class FetchResponse<T extends BaseRecords> extends AbstractResponse { public class FetchResponse extends AbstractResponse {
public static final long INVALID_HIGH_WATERMARK = -1L;
public static final long INVALID_HIGHWATERMARK = -1L;
public static final long INVALID_LAST_STABLE_OFFSET = -1L; public static final long INVALID_LAST_STABLE_OFFSET = -1L;
public static final long INVALID_LOG_START_OFFSET = -1L; public static final long INVALID_LOG_START_OFFSET = -1L;
public static final int INVALID_PREFERRED_REPLICA_ID = -1; public static final int INVALID_PREFERRED_REPLICA_ID = -1;
private final FetchResponseData data; private final FetchResponseData data;
private final LinkedHashMap<TopicPartition, PartitionData<T>> responseDataMap; // we build responseData when needed.
private volatile LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = null;
@Override @Override
public FetchResponseData data() { public FetchResponseData data() {
return data; return data;
} }
public static final class AbortedTransaction {
public final long producerId;
public final long firstOffset;
public AbortedTransaction(long producerId, long firstOffset) {
this.producerId = producerId;
this.firstOffset = firstOffset;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
AbortedTransaction that = (AbortedTransaction) o;
return producerId == that.producerId && firstOffset == that.firstOffset;
}
@Override
public int hashCode() {
int result = Long.hashCode(producerId);
result = 31 * result + Long.hashCode(firstOffset);
return result;
}
@Override
public String toString() {
return "(producerId=" + producerId + ", firstOffset=" + firstOffset + ")";
}
static AbortedTransaction fromMessage(FetchResponseData.AbortedTransaction abortedTransaction) {
return new AbortedTransaction(abortedTransaction.producerId(), abortedTransaction.firstOffset());
}
}
public static final class PartitionData<T extends BaseRecords> {
private final FetchResponseData.FetchablePartitionResponse partitionResponse;
// Derived fields
private final Optional<Integer> preferredReplica;
private final List<AbortedTransaction> abortedTransactions;
private final Errors error;
private PartitionData(FetchResponseData.FetchablePartitionResponse partitionResponse) {
// We partially construct FetchablePartitionResponse since we don't know the partition ID at this point
// When we convert the PartitionData (and other fields) into FetchResponseData down in toMessage, we
// set the partition IDs.
this.partitionResponse = partitionResponse;
this.preferredReplica = Optional.of(partitionResponse.preferredReadReplica())
.filter(replicaId -> replicaId != INVALID_PREFERRED_REPLICA_ID);
if (partitionResponse.abortedTransactions() == null) {
this.abortedTransactions = null;
} else {
this.abortedTransactions = partitionResponse.abortedTransactions().stream()
.map(AbortedTransaction::fromMessage)
.collect(Collectors.toList());
}
this.error = Errors.forCode(partitionResponse.errorCode());
}
public PartitionData(Errors error,
long highWatermark,
long lastStableOffset,
long logStartOffset,
Optional<Integer> preferredReadReplica,
List<AbortedTransaction> abortedTransactions,
Optional<FetchResponseData.EpochEndOffset> divergingEpoch,
T records) {
this.preferredReplica = preferredReadReplica;
this.abortedTransactions = abortedTransactions;
this.error = error;
FetchResponseData.FetchablePartitionResponse partitionResponse =
new FetchResponseData.FetchablePartitionResponse();
partitionResponse.setErrorCode(error.code())
.setHighWatermark(highWatermark)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(logStartOffset);
if (abortedTransactions != null) {
partitionResponse.setAbortedTransactions(abortedTransactions.stream().map(
aborted -> new FetchResponseData.AbortedTransaction()
.setProducerId(aborted.producerId)
.setFirstOffset(aborted.firstOffset))
.collect(Collectors.toList()));
} else {
partitionResponse.setAbortedTransactions(null);
}
partitionResponse.setPreferredReadReplica(preferredReadReplica.orElse(INVALID_PREFERRED_REPLICA_ID));
partitionResponse.setRecordSet(records);
divergingEpoch.ifPresent(partitionResponse::setDivergingEpoch);
this.partitionResponse = partitionResponse;
}
public PartitionData(Errors error,
long highWatermark,
long lastStableOffset,
long logStartOffset,
Optional<Integer> preferredReadReplica,
List<AbortedTransaction> abortedTransactions,
T records) {
this(error, highWatermark, lastStableOffset, logStartOffset, preferredReadReplica,
abortedTransactions, Optional.empty(), records);
}
public PartitionData(Errors error,
long highWatermark,
long lastStableOffset,
long logStartOffset,
List<AbortedTransaction> abortedTransactions,
T records) {
this(error, highWatermark, lastStableOffset, logStartOffset, Optional.empty(), abortedTransactions, records);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
PartitionData that = (PartitionData) o;
return this.partitionResponse.equals(that.partitionResponse);
}
@Override
public int hashCode() {
return this.partitionResponse.hashCode();
}
@Override
public String toString() {
return "(error=" + error() +
", highWaterMark=" + highWatermark() +
", lastStableOffset = " + lastStableOffset() +
", logStartOffset = " + logStartOffset() +
", preferredReadReplica = " + preferredReadReplica().map(Object::toString).orElse("absent") +
", abortedTransactions = " + abortedTransactions() +
", divergingEpoch =" + divergingEpoch() +
", recordsSizeInBytes=" + records().sizeInBytes() + ")";
}
public Errors error() {
return error;
}
public long highWatermark() {
return partitionResponse.highWatermark();
}
public long lastStableOffset() {
return partitionResponse.lastStableOffset();
}
public long logStartOffset() {
return partitionResponse.logStartOffset();
}
public Optional<Integer> preferredReadReplica() {
return preferredReplica;
}
public List<AbortedTransaction> abortedTransactions() {
return abortedTransactions;
}
public Optional<FetchResponseData.EpochEndOffset> divergingEpoch() {
FetchResponseData.EpochEndOffset epochEndOffset = partitionResponse.divergingEpoch();
if (epochEndOffset.epoch() < 0) {
return Optional.empty();
} else {
return Optional.of(epochEndOffset);
}
}
@SuppressWarnings("unchecked")
public T records() {
return (T) partitionResponse.recordSet();
}
}
/**
* From version 3 or later, the entries in `responseData` should be in the same order as the entries in
* `FetchRequest.fetchData`.
*
* @param error The top-level error code.
* @param responseData The fetched data grouped by partition.
* @param throttleTimeMs The time in milliseconds that the response was throttled
* @param sessionId The fetch session id.
*/
public FetchResponse(Errors error,
LinkedHashMap<TopicPartition, PartitionData<T>> responseData,
int throttleTimeMs,
int sessionId) {
super(ApiKeys.FETCH);
this.data = toMessage(throttleTimeMs, error, responseData.entrySet().iterator(), sessionId);
this.responseDataMap = responseData;
}
public FetchResponse(FetchResponseData fetchResponseData) { public FetchResponse(FetchResponseData fetchResponseData) {
super(ApiKeys.FETCH); super(ApiKeys.FETCH);
this.data = fetchResponseData; this.data = fetchResponseData;
this.responseDataMap = toResponseDataMap(fetchResponseData);
} }
public Errors error() { public Errors error() {
return Errors.forCode(data.errorCode()); return Errors.forCode(data.errorCode());
} }
public LinkedHashMap<TopicPartition, PartitionData<T>> responseData() { public LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData() {
return responseDataMap; if (responseData == null) {
synchronized (this) {
if (responseData == null) {
responseData = new LinkedHashMap<>();
data.responses().forEach(topicResponse ->
topicResponse.partitions().forEach(partition ->
responseData.put(new TopicPartition(topicResponse.topic(), partition.partitionIndex()), partition))
);
}
}
}
return responseData;
} }
@Override @Override
@ -304,58 +108,15 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> errorCounts = new HashMap<>(); Map<Errors, Integer> errorCounts = new HashMap<>();
updateErrorCounts(errorCounts, error()); updateErrorCounts(errorCounts, error());
responseDataMap.values().forEach(response -> data.responses().forEach(topicResponse ->
updateErrorCounts(errorCounts, response.error()) topicResponse.partitions().forEach(partition ->
updateErrorCounts(errorCounts, Errors.forCode(partition.errorCode())))
); );
return errorCounts; return errorCounts;
} }
public static FetchResponse<MemoryRecords> parse(ByteBuffer buffer, short version) { public static FetchResponse parse(ByteBuffer buffer, short version) {
return new FetchResponse<>(new FetchResponseData(new ByteBufferAccessor(buffer), version)); return new FetchResponse(new FetchResponseData(new ByteBufferAccessor(buffer), version));
}
@SuppressWarnings("unchecked")
private static <T extends BaseRecords> LinkedHashMap<TopicPartition, PartitionData<T>> toResponseDataMap(
FetchResponseData message) {
LinkedHashMap<TopicPartition, PartitionData<T>> responseMap = new LinkedHashMap<>();
message.responses().forEach(topicResponse -> {
topicResponse.partitionResponses().forEach(partitionResponse -> {
TopicPartition tp = new TopicPartition(topicResponse.topic(), partitionResponse.partition());
PartitionData<T> partitionData = new PartitionData<>(partitionResponse);
responseMap.put(tp, partitionData);
});
});
return responseMap;
}
private static <T extends BaseRecords> FetchResponseData toMessage(int throttleTimeMs, Errors error,
Iterator<Map.Entry<TopicPartition, PartitionData<T>>> partIterator,
int sessionId) {
List<FetchResponseData.FetchableTopicResponse> topicResponseList = new ArrayList<>();
partIterator.forEachRemaining(entry -> {
PartitionData<T> partitionData = entry.getValue();
// Since PartitionData alone doesn't know the partition ID, we set it here
partitionData.partitionResponse.setPartition(entry.getKey().partition());
// We have to keep the order of input topic-partition. Hence, we batch the partitions only if the last
// batch is in the same topic group.
FetchResponseData.FetchableTopicResponse previousTopic = topicResponseList.isEmpty() ? null
: topicResponseList.get(topicResponseList.size() - 1);
if (previousTopic != null && previousTopic.topic().equals(entry.getKey().topic()))
previousTopic.partitionResponses().add(partitionData.partitionResponse);
else {
List<FetchResponseData.FetchablePartitionResponse> partitionResponses = new ArrayList<>();
partitionResponses.add(partitionData.partitionResponse);
topicResponseList.add(new FetchResponseData.FetchableTopicResponse()
.setTopic(entry.getKey().topic())
.setPartitionResponses(partitionResponses));
}
});
return new FetchResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode(error.code())
.setSessionId(sessionId)
.setResponses(topicResponseList);
} }
/** /**
@ -365,11 +126,11 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
* @param partIterator The partition iterator. * @param partIterator The partition iterator.
* @return The response size in bytes. * @return The response size in bytes.
*/ */
public static <T extends BaseRecords> int sizeOf(short version, public static int sizeOf(short version,
Iterator<Map.Entry<TopicPartition, PartitionData<T>>> partIterator) { Iterator<Map.Entry<TopicPartition, FetchResponseData.PartitionData>> partIterator) {
// Since the throttleTimeMs and metadata field sizes are constant and fixed, we can // Since the throttleTimeMs and metadata field sizes are constant and fixed, we can
// use arbitrary values here without affecting the result. // use arbitrary values here without affecting the result.
FetchResponseData data = toMessage(0, Errors.NONE, partIterator, INVALID_SESSION_ID); FetchResponseData data = toMessage(Errors.NONE, 0, INVALID_SESSION_ID, partIterator);
ObjectSerializationCache cache = new ObjectSerializationCache(); ObjectSerializationCache cache = new ObjectSerializationCache();
return 4 + data.size(cache, version); return 4 + data.size(cache, version);
} }
@ -378,4 +139,91 @@ public class FetchResponse<T extends BaseRecords> extends AbstractResponse {
public boolean shouldClientThrottle(short version) { public boolean shouldClientThrottle(short version) {
return version >= 8; return version >= 8;
} }
public static Optional<FetchResponseData.EpochEndOffset> divergingEpoch(FetchResponseData.PartitionData partitionResponse) {
return partitionResponse.divergingEpoch().epoch() < 0 ? Optional.empty()
: Optional.of(partitionResponse.divergingEpoch());
}
public static boolean isDivergingEpoch(FetchResponseData.PartitionData partitionResponse) {
return partitionResponse.divergingEpoch().epoch() >= 0;
}
public static Optional<Integer> preferredReadReplica(FetchResponseData.PartitionData partitionResponse) {
return partitionResponse.preferredReadReplica() == INVALID_PREFERRED_REPLICA_ID ? Optional.empty()
: Optional.of(partitionResponse.preferredReadReplica());
}
public static boolean isPreferredReplica(FetchResponseData.PartitionData partitionResponse) {
return partitionResponse.preferredReadReplica() != INVALID_PREFERRED_REPLICA_ID;
}
public static FetchResponseData.PartitionData partitionResponse(int partition, Errors error) {
return new FetchResponseData.PartitionData()
.setPartitionIndex(partition)
.setErrorCode(error.code())
.setHighWatermark(FetchResponse.INVALID_HIGH_WATERMARK);
}
/**
* Returns `partition.records` as `Records` (instead of `BaseRecords`). If `records` is `null`, returns `MemoryRecords.EMPTY`.
*
* If this response was deserialized after a fetch, this method should never fail. An example where this would
* fail is a down-converted response (e.g. LazyDownConversionRecords) on the broker (before it's serialized and
* sent on the wire).
*
* @param partition partition data
* @return Records or empty record if the records in PartitionData is null.
*/
public static Records recordsOrFail(FetchResponseData.PartitionData partition) {
if (partition.records() == null) return MemoryRecords.EMPTY;
if (partition.records() instanceof Records) return (Records) partition.records();
throw new ClassCastException("The record type is " + partition.records().getClass().getSimpleName() + ", which is not a subtype of " +
Records.class.getSimpleName() + ". This method is only safe to call if the `FetchResponse` was deserialized from bytes.");
}
/**
* @return The size in bytes of the records. 0 is returned if records of input partition is null.
*/
public static int recordsSize(FetchResponseData.PartitionData partition) {
return partition.records() == null ? 0 : partition.records().sizeInBytes();
}
public static FetchResponse of(Errors error,
int throttleTimeMs,
int sessionId,
LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData) {
return new FetchResponse(toMessage(error, throttleTimeMs, sessionId, responseData.entrySet().iterator()));
}
private static FetchResponseData toMessage(Errors error,
int throttleTimeMs,
int sessionId,
Iterator<Map.Entry<TopicPartition, FetchResponseData.PartitionData>> partIterator) {
List<FetchResponseData.FetchableTopicResponse> topicResponseList = new ArrayList<>();
partIterator.forEachRemaining(entry -> {
FetchResponseData.PartitionData partitionData = entry.getValue();
// Since PartitionData alone doesn't know the partition ID, we set it here
partitionData.setPartitionIndex(entry.getKey().partition());
// We have to keep the order of input topic-partition. Hence, we batch the partitions only if the last
// batch is in the same topic group.
FetchResponseData.FetchableTopicResponse previousTopic = topicResponseList.isEmpty() ? null
: topicResponseList.get(topicResponseList.size() - 1);
if (previousTopic != null && previousTopic.topic().equals(entry.getKey().topic()))
previousTopic.partitions().add(partitionData);
else {
List<FetchResponseData.PartitionData> partitionResponses = new ArrayList<>();
partitionResponses.add(partitionData);
topicResponseList.add(new FetchResponseData.FetchableTopicResponse()
.setTopic(entry.getKey().topic())
.setPartitions(partitionResponses));
}
});
return new FetchResponseData()
.setThrottleTimeMs(throttleTimeMs)
.setErrorCode(error.code())
.setSessionId(sessionId)
.setResponses(topicResponseList);
}
} }

View File

@ -53,9 +53,9 @@
"about": "The response topics.", "fields": [ "about": "The response topics.", "fields": [
{ "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName", { "name": "Topic", "type": "string", "versions": "0+", "entityType": "topicName",
"about": "The topic name." }, "about": "The topic name." },
{ "name": "PartitionResponses", "type": "[]FetchablePartitionResponse", "versions": "0+", { "name": "Partitions", "type": "[]PartitionData", "versions": "0+",
"about": "The topic partitions.", "fields": [ "about": "The topic partitions.", "fields": [
{ "name": "Partition", "type": "int32", "versions": "0+", { "name": "PartitionIndex", "type": "int32", "versions": "0+",
"about": "The partition index." }, "about": "The partition index." },
{ "name": "ErrorCode", "type": "int16", "versions": "0+", { "name": "ErrorCode", "type": "int16", "versions": "0+",
"about": "The error code, or 0 if there was no fetch error." }, "about": "The error code, or 0 if there was no fetch error." },
@ -94,7 +94,7 @@
]}, ]},
{ "name": "PreferredReadReplica", "type": "int32", "versions": "11+", "default": "-1", "ignorable": false, { "name": "PreferredReadReplica", "type": "int32", "versions": "11+", "default": "-1", "ignorable": false,
"about": "The preferred read replica for the consumer to use on its next fetch request"}, "about": "The preferred read replica for the consumer to use on its next fetch request"},
{ "name": "RecordSet", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."} { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}
]} ]}
]} ]}
] ]

View File

@ -17,8 +17,8 @@
package org.apache.kafka.clients; package org.apache.kafka.clients;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
@ -150,22 +150,21 @@ public class FetchSessionHandlerTest {
private static final class RespEntry { private static final class RespEntry {
final TopicPartition part; final TopicPartition part;
final FetchResponse.PartitionData<MemoryRecords> data; final FetchResponseData.PartitionData data;
RespEntry(String topic, int partition, long highWatermark, long lastStableOffset) { RespEntry(String topic, int partition, long highWatermark, long lastStableOffset) {
this.part = new TopicPartition(topic, partition); this.part = new TopicPartition(topic, partition);
this.data = new FetchResponse.PartitionData<>(
Errors.NONE, this.data = new FetchResponseData.PartitionData()
highWatermark, .setPartitionIndex(partition)
lastStableOffset, .setHighWatermark(highWatermark)
0, .setLastStableOffset(lastStableOffset)
null, .setLogStartOffset(0);
null);
} }
} }
private static LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> respMap(RespEntry... entries) { private static LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> respMap(RespEntry... entries) {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> map = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> map = new LinkedHashMap<>();
for (RespEntry entry : entries) { for (RespEntry entry : entries) {
map.put(entry.part, entry.data); map.put(entry.part, entry.data);
} }
@ -191,10 +190,10 @@ public class FetchSessionHandlerTest {
assertEquals(INVALID_SESSION_ID, data.metadata().sessionId()); assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
assertEquals(INITIAL_EPOCH, data.metadata().epoch()); assertEquals(INITIAL_EPOCH, data.metadata().epoch());
FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE, FetchResponse resp = FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID,
respMap(new RespEntry("foo", 0, 0, 0), respMap(new RespEntry("foo", 0, 0, 0),
new RespEntry("foo", 1, 0, 0)), new RespEntry("foo", 1, 0, 0))
0, INVALID_SESSION_ID); );
handler.handleResponse(resp); handler.handleResponse(resp);
FetchSessionHandler.Builder builder2 = handler.newBuilder(); FetchSessionHandler.Builder builder2 = handler.newBuilder();
@ -225,10 +224,9 @@ public class FetchSessionHandlerTest {
assertEquals(INVALID_SESSION_ID, data.metadata().sessionId()); assertEquals(INVALID_SESSION_ID, data.metadata().sessionId());
assertEquals(INITIAL_EPOCH, data.metadata().epoch()); assertEquals(INITIAL_EPOCH, data.metadata().epoch());
FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE, FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
respMap(new RespEntry("foo", 0, 10, 20), respMap(new RespEntry("foo", 0, 10, 20),
new RespEntry("foo", 1, 10, 20)), new RespEntry("foo", 1, 10, 20)));
0, 123);
handler.handleResponse(resp); handler.handleResponse(resp);
// Test an incremental fetch request which adds one partition and modifies another. // Test an incremental fetch request which adds one partition and modifies another.
@ -249,15 +247,14 @@ public class FetchSessionHandlerTest {
new ReqEntry("foo", 1, 10, 120, 210)), new ReqEntry("foo", 1, 10, 120, 210)),
data2.toSend()); data2.toSend());
FetchResponse<MemoryRecords> resp2 = new FetchResponse<>(Errors.NONE, FetchResponse resp2 = FetchResponse.of(Errors.NONE, 0, 123,
respMap(new RespEntry("foo", 1, 20, 20)), respMap(new RespEntry("foo", 1, 20, 20)));
0, 123);
handler.handleResponse(resp2); handler.handleResponse(resp2);
// Skip building a new request. Test that handling an invalid fetch session epoch response results // Skip building a new request. Test that handling an invalid fetch session epoch response results
// in a request which closes the session. // in a request which closes the session.
FetchResponse<MemoryRecords> resp3 = new FetchResponse<>(Errors.INVALID_FETCH_SESSION_EPOCH, respMap(), FetchResponse resp3 = FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH,
0, INVALID_SESSION_ID); 0, INVALID_SESSION_ID, respMap());
handler.handleResponse(resp3); handler.handleResponse(resp3);
FetchSessionHandler.Builder builder4 = handler.newBuilder(); FetchSessionHandler.Builder builder4 = handler.newBuilder();
@ -312,11 +309,10 @@ public class FetchSessionHandlerTest {
data.toSend(), data.sessionPartitions()); data.toSend(), data.sessionPartitions());
assertTrue(data.metadata().isFull()); assertTrue(data.metadata().isFull());
FetchResponse<MemoryRecords> resp = new FetchResponse<>(Errors.NONE, FetchResponse resp = FetchResponse.of(Errors.NONE, 0, 123,
respMap(new RespEntry("foo", 0, 10, 20), respMap(new RespEntry("foo", 0, 10, 20),
new RespEntry("foo", 1, 10, 20), new RespEntry("foo", 1, 10, 20),
new RespEntry("bar", 0, 10, 20)), new RespEntry("bar", 0, 10, 20)));
0, 123);
handler.handleResponse(resp); handler.handleResponse(resp);
// Test an incremental fetch request which removes two partitions. // Test an incremental fetch request which removes two partitions.
@ -337,8 +333,8 @@ public class FetchSessionHandlerTest {
// A FETCH_SESSION_ID_NOT_FOUND response triggers us to close the session. // A FETCH_SESSION_ID_NOT_FOUND response triggers us to close the session.
// The next request is a session establishing FULL request. // The next request is a session establishing FULL request.
FetchResponse<MemoryRecords> resp2 = new FetchResponse<>(Errors.FETCH_SESSION_ID_NOT_FOUND, FetchResponse resp2 = FetchResponse.of(Errors.FETCH_SESSION_ID_NOT_FOUND,
respMap(), 0, INVALID_SESSION_ID); 0, INVALID_SESSION_ID, respMap());
handler.handleResponse(resp2); handler.handleResponse(resp2);
FetchSessionHandler.Builder builder3 = handler.newBuilder(); FetchSessionHandler.Builder builder3 = handler.newBuilder();
builder3.add(new TopicPartition("foo", 0), builder3.add(new TopicPartition("foo", 0),
@ -354,11 +350,10 @@ public class FetchSessionHandlerTest {
@Test @Test
public void testVerifyFullFetchResponsePartitions() throws Exception { public void testVerifyFullFetchResponsePartitions() throws Exception {
FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1); FetchSessionHandler handler = new FetchSessionHandler(LOG_CONTEXT, 1);
String issue = handler.verifyFullFetchResponsePartitions(new FetchResponse<>(Errors.NONE, String issue = handler.verifyFullFetchResponsePartitions(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID,
respMap(new RespEntry("foo", 0, 10, 20), respMap(new RespEntry("foo", 0, 10, 20),
new RespEntry("foo", 1, 10, 20), new RespEntry("foo", 1, 10, 20),
new RespEntry("bar", 0, 10, 20)), new RespEntry("bar", 0, 10, 20))));
0, INVALID_SESSION_ID));
assertTrue(issue.contains("extra")); assertTrue(issue.contains("extra"));
assertFalse(issue.contains("omitted")); assertFalse(issue.contains("omitted"));
FetchSessionHandler.Builder builder = handler.newBuilder(); FetchSessionHandler.Builder builder = handler.newBuilder();
@ -369,16 +364,14 @@ public class FetchSessionHandlerTest {
builder.add(new TopicPartition("bar", 0), builder.add(new TopicPartition("bar", 0),
new FetchRequest.PartitionData(20, 120, 220, Optional.empty())); new FetchRequest.PartitionData(20, 120, 220, Optional.empty()));
builder.build(); builder.build();
String issue2 = handler.verifyFullFetchResponsePartitions(new FetchResponse<>(Errors.NONE, String issue2 = handler.verifyFullFetchResponsePartitions(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID,
respMap(new RespEntry("foo", 0, 10, 20), respMap(new RespEntry("foo", 0, 10, 20),
new RespEntry("foo", 1, 10, 20), new RespEntry("foo", 1, 10, 20),
new RespEntry("bar", 0, 10, 20)), new RespEntry("bar", 0, 10, 20))));
0, INVALID_SESSION_ID));
assertTrue(issue2 == null); assertTrue(issue2 == null);
String issue3 = handler.verifyFullFetchResponsePartitions(new FetchResponse<>(Errors.NONE, String issue3 = handler.verifyFullFetchResponsePartitions(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID,
respMap(new RespEntry("foo", 0, 10, 20), respMap(new RespEntry("foo", 0, 10, 20),
new RespEntry("foo", 1, 10, 20)), new RespEntry("foo", 1, 10, 20))));
0, INVALID_SESSION_ID));
assertFalse(issue3.contains("extra")); assertFalse(issue3.contains("extra"));
assertTrue(issue3.contains("omitted")); assertTrue(issue3.contains("omitted"));
} }

View File

@ -45,18 +45,20 @@ import org.apache.kafka.common.errors.InvalidGroupIdException;
import org.apache.kafka.common.errors.InvalidTopicException; import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.HeartbeatResponseData; import org.apache.kafka.common.message.HeartbeatResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupRequestData;
import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.JoinGroupResponseData;
import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData;
import org.apache.kafka.common.message.ListOffsetsResponseData;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.message.SyncGroupResponseData;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
import org.apache.kafka.common.message.ListOffsetsResponseData;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
import org.apache.kafka.common.message.SyncGroupResponseData;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
@ -91,8 +93,6 @@ import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.MockConsumerInterceptor; import org.apache.kafka.test.MockConsumerInterceptor;
import org.apache.kafka.test.MockMetricsReporter; import org.apache.kafka.test.MockMetricsReporter;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.apache.kafka.common.metrics.stats.Avg;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import javax.management.MBeanServer; import javax.management.MBeanServer;
@ -2276,8 +2276,8 @@ public class KafkaConsumerTest {
return new ListOffsetsResponse(data); return new ListOffsetsResponse(data);
} }
private FetchResponse<MemoryRecords> fetchResponse(Map<TopicPartition, FetchInfo> fetches) { private FetchResponse fetchResponse(Map<TopicPartition, FetchInfo> fetches) {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> tpResponses = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> tpResponses = new LinkedHashMap<>();
for (Map.Entry<TopicPartition, FetchInfo> fetchEntry : fetches.entrySet()) { for (Map.Entry<TopicPartition, FetchInfo> fetchEntry : fetches.entrySet()) {
TopicPartition partition = fetchEntry.getKey(); TopicPartition partition = fetchEntry.getKey();
long fetchOffset = fetchEntry.getValue().offset; long fetchOffset = fetchEntry.getValue().offset;
@ -2294,14 +2294,17 @@ public class KafkaConsumerTest {
builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes()); builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
records = builder.build(); records = builder.build();
} }
tpResponses.put(partition, new FetchResponse.PartitionData<>( tpResponses.put(partition,
Errors.NONE, highWatermark, FetchResponse.INVALID_LAST_STABLE_OFFSET, new FetchResponseData.PartitionData()
logStartOffset, null, records)); .setPartitionIndex(partition.partition())
.setHighWatermark(highWatermark)
.setLogStartOffset(logStartOffset)
.setRecords(records));
} }
return new FetchResponse<>(Errors.NONE, tpResponses, 0, INVALID_SESSION_ID); return FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, tpResponses);
} }
private FetchResponse<MemoryRecords> fetchResponse(TopicPartition partition, long fetchOffset, int count) { private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset, int count) {
FetchInfo fetchInfo = new FetchInfo(fetchOffset, count); FetchInfo fetchInfo = new FetchInfo(fetchOffset, count);
return fetchResponse(Collections.singletonMap(partition, fetchInfo)); return fetchResponse(Collections.singletonMap(partition, fetchInfo));
} }

View File

@ -48,6 +48,7 @@ import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.ApiMessageType; import org.apache.kafka.common.message.ApiMessageType;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic; import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
@ -1270,13 +1271,18 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = new LinkedHashMap<>();
partitions.put(tp1, new FetchResponse.PartitionData<>(Errors.NONE, 100, Map<TopicPartition, FetchResponseData.PartitionData> partitions = new LinkedHashMap<>();
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, records)); partitions.put(tp1, new FetchResponseData.PartitionData()
partitions.put(tp0, new FetchResponse.PartitionData<>(Errors.OFFSET_OUT_OF_RANGE, 100, .setPartitionIndex(tp1.partition())
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY)); .setHighWatermark(100)
client.prepareResponse(new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), .setRecords(records));
0, INVALID_SESSION_ID)); partitions.put(tp0, new FetchResponseData.PartitionData()
.setPartitionIndex(tp0.partition())
.setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code())
.setHighWatermark(100));
client.prepareResponse(FetchResponse.of(Errors.NONE,
0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
List<ConsumerRecord<byte[], byte[]>> allFetchedRecords = new ArrayList<>(); List<ConsumerRecord<byte[], byte[]>> allFetchedRecords = new ArrayList<>();
@ -1316,17 +1322,29 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = new LinkedHashMap<>(); Map<TopicPartition, FetchResponseData.PartitionData> partitions = new LinkedHashMap<>();
partitions.put(tp1, new FetchResponse.PartitionData<>(Errors.NONE, 100, FetchResponse.INVALID_LAST_STABLE_OFFSET, partitions.put(tp1, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LOG_START_OFFSET, null, records)); .setPartitionIndex(tp1.partition())
partitions.put(tp0, new FetchResponse.PartitionData<>(Errors.OFFSET_OUT_OF_RANGE, 100, .setHighWatermark(100)
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY)); .setRecords(records));
partitions.put(tp2, new FetchResponse.PartitionData<>(Errors.NONE, 100L, 4, partitions.put(tp0, new FetchResponseData.PartitionData()
0L, null, nextRecords)); .setPartitionIndex(tp0.partition())
partitions.put(tp3, new FetchResponse.PartitionData<>(Errors.NONE, 100L, 4, .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code())
0L, null, partialRecords)); .setHighWatermark(100));
client.prepareResponse(new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), partitions.put(tp2, new FetchResponseData.PartitionData()
0, INVALID_SESSION_ID)); .setPartitionIndex(tp2.partition())
.setHighWatermark(100)
.setLastStableOffset(4)
.setLogStartOffset(0)
.setRecords(nextRecords));
partitions.put(tp3, new FetchResponseData.PartitionData()
.setPartitionIndex(tp3.partition())
.setHighWatermark(100)
.setLastStableOffset(4)
.setLogStartOffset(0)
.setRecords(partialRecords));
client.prepareResponse(FetchResponse.of(Errors.NONE,
0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
List<ConsumerRecord<byte[], byte[]>> fetchedRecords = new ArrayList<>(); List<ConsumerRecord<byte[], byte[]>> fetchedRecords = new ArrayList<>();
@ -1384,9 +1402,11 @@ public class FetcherTest {
assignFromUser(Utils.mkSet(tp0)); assignFromUser(Utils.mkSet(tp0));
subscriptions.seek(tp0, 1); subscriptions.seek(tp0, 1);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = new HashMap<>(); Map<TopicPartition, FetchResponseData.PartitionData> partitions = new HashMap<>();
partitions.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, 100, partitions.put(tp0, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), null, records)); .setPartitionIndex(tp0.partition())
.setHighWatermark(100)
.setRecords(records));
client.prepareResponse(fullFetchResponse(tp0, this.records, Errors.NONE, 100L, 0)); client.prepareResponse(fullFetchResponse(tp0, this.records, Errors.NONE, 100L, 0));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
@ -1397,9 +1417,11 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
partitions = new HashMap<>(); partitions = new HashMap<>();
partitions.put(tp1, new FetchResponse.PartitionData<>(Errors.OFFSET_OUT_OF_RANGE, 100, partitions.put(tp1, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), null, MemoryRecords.EMPTY)); .setPartitionIndex(tp1.partition())
client.prepareResponse(new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), 0, INVALID_SESSION_ID)); .setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code())
.setHighWatermark(100));
client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
assertEquals(1, fetcher.fetchedRecords().get(tp0).size()); assertEquals(1, fetcher.fetchedRecords().get(tp0).size());
@ -2094,7 +2116,7 @@ public class FetcherTest {
ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true); ClientRequest request = client.newClientRequest(node.idString(), builder, time.milliseconds(), true);
client.send(request, time.milliseconds()); client.send(request, time.milliseconds());
client.poll(1, time.milliseconds()); client.poll(1, time.milliseconds());
FetchResponse<MemoryRecords> response = fullFetchResponse(tp0, nextRecords, Errors.NONE, i, throttleTimeMs); FetchResponse response = fullFetchResponse(tp0, nextRecords, Errors.NONE, i, throttleTimeMs);
buffer = RequestTestUtils.serializeResponseWithHeader(response, ApiKeys.FETCH.latestVersion(), request.correlationId()); buffer = RequestTestUtils.serializeResponseWithHeader(response, ApiKeys.FETCH.latestVersion(), request.correlationId());
selector.completeReceive(new NetworkReceive(node.idString(), buffer)); selector.completeReceive(new NetworkReceive(node.idString(), buffer));
client.poll(1, time.milliseconds()); client.poll(1, time.milliseconds());
@ -2256,7 +2278,7 @@ public class FetcherTest {
client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, partitionCounts, tp -> validLeaderEpoch)); client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, partitionCounts, tp -> validLeaderEpoch));
int expectedBytes = 0; int expectedBytes = 0;
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> fetchPartitionData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> fetchPartitionData = new LinkedHashMap<>();
for (TopicPartition tp : Utils.mkSet(tp1, tp2)) { for (TopicPartition tp : Utils.mkSet(tp1, tp2)) {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
@ -2269,12 +2291,15 @@ public class FetcherTest {
for (Record record : records.records()) for (Record record : records.records())
expectedBytes += record.sizeInBytes(); expectedBytes += record.sizeInBytes();
fetchPartitionData.put(tp, new FetchResponse.PartitionData<>(Errors.NONE, 15L, fetchPartitionData.put(tp, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records)); .setPartitionIndex(tp.partition())
.setHighWatermark(15)
.setLogStartOffset(0)
.setRecords(records));
} }
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(new FetchResponse<>(Errors.NONE, fetchPartitionData, 0, INVALID_SESSION_ID)); client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, fetchPartitionData));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchedRecords(); Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords = fetchedRecords();
@ -2333,15 +2358,21 @@ public class FetcherTest {
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
MemoryRecords records = builder.build(); MemoryRecords records = builder.build();
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = new HashMap<>(); Map<TopicPartition, FetchResponseData.PartitionData> partitions = new HashMap<>();
partitions.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, 100, partitions.put(tp0, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records)); .setPartitionIndex(tp0.partition())
partitions.put(tp1, new FetchResponse.PartitionData<>(Errors.OFFSET_OUT_OF_RANGE, 100, .setHighWatermark(100)
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, MemoryRecords.EMPTY)); .setLogStartOffset(0)
.setRecords(records));
partitions.put(tp1, new FetchResponseData.PartitionData()
.setPartitionIndex(tp1.partition())
.setErrorCode(Errors.OFFSET_OUT_OF_RANGE.code())
.setHighWatermark(100)
.setLogStartOffset(0));
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), client.prepareResponse(FetchResponse.of(Errors.NONE,
0, INVALID_SESSION_ID)); 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
fetcher.fetchedRecords(); fetcher.fetchedRecords();
@ -2375,15 +2406,19 @@ public class FetcherTest {
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
MemoryRecords records = builder.build(); MemoryRecords records = builder.build();
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = new HashMap<>(); Map<TopicPartition, FetchResponseData.PartitionData> partitions = new HashMap<>();
partitions.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, 100, partitions.put(tp0, new FetchResponseData.PartitionData()
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, records)); .setPartitionIndex(tp0.partition())
partitions.put(tp1, new FetchResponse.PartitionData<>(Errors.NONE, 100, .setHighWatermark(100)
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, .setLogStartOffset(0)
MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("val".getBytes())))); .setRecords(records));
partitions.put(tp1, new FetchResponseData.PartitionData()
.setPartitionIndex(tp1.partition())
.setHighWatermark(100)
.setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("val".getBytes()))));
client.prepareResponse(new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
0, INVALID_SESSION_ID));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
fetcher.fetchedRecords(); fetcher.fetchedRecords();
@ -2778,8 +2813,8 @@ public class FetcherTest {
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 0)); new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0));
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -2811,7 +2846,6 @@ public class FetcherTest {
commitTransaction(buffer, 1L, currentOffset); commitTransaction(buffer, 1L, currentOffset);
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -2824,7 +2858,7 @@ public class FetcherTest {
FetchRequest request = (FetchRequest) body; FetchRequest request = (FetchRequest) body;
assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel()); assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
return true; return true;
}, fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); }, fullFetchResponseWithAbortedTransactions(records, Collections.emptyList(), Errors.NONE, 100L, 100L, 0));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
assertTrue(fetcher.hasCompletedFetches()); assertTrue(fetcher.hasCompletedFetches());
@ -2840,7 +2874,7 @@ public class FetcherTest {
new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED); new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_COMMITTED);
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
long pid1 = 1L; long pid1 = 1L;
long pid2 = 2L; long pid2 = 2L;
@ -2863,7 +2897,7 @@ public class FetcherTest {
// abort producer 2 // abort producer 2
abortTransaction(buffer, pid2, 5L); abortTransaction(buffer, pid2, 5L);
abortedTransactions.add(new FetchResponse.AbortedTransaction(pid2, 2L)); abortedTransactions.add(new FetchResponseData.AbortedTransaction().setProducerId(pid2).setFirstOffset(2L));
// New transaction for producer 1 (eventually aborted) // New transaction for producer 1 (eventually aborted)
appendTransactionalRecords(buffer, pid1, 6L, appendTransactionalRecords(buffer, pid1, 6L,
@ -2879,7 +2913,7 @@ public class FetcherTest {
// abort producer 1 // abort producer 1
abortTransaction(buffer, pid1, 9L); abortTransaction(buffer, pid1, 9L);
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 6)); abortedTransactions.add(new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(6));
// commit producer 2 // commit producer 2
commitTransaction(buffer, pid2, 10L); commitTransaction(buffer, pid2, 10L);
@ -2931,8 +2965,9 @@ public class FetcherTest {
commitTransaction(buffer, 1L, currentOffset); commitTransaction(buffer, 1L, currentOffset);
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 0)); new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0)
);
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -2983,8 +3018,8 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
// prepare the response. the aborted transactions begin at offsets which are no longer in the log // prepare the response. the aborted transactions begin at offsets which are no longer in the log
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(producerId, 0L)); new FetchResponseData.AbortedTransaction().setProducerId(producerId).setFirstOffset(0L));
client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer), client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer),
abortedTransactions, Errors.NONE, 100L, 100L, 0)); abortedTransactions, Errors.NONE, 100L, 100L, 0));
@ -3120,9 +3155,10 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
// prepare the response. the aborted transactions begin at offsets which are no longer in the log // prepare the response. the aborted transactions begin at offsets which are no longer in the log
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Arrays.asList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(pid2, 6L)); new FetchResponseData.AbortedTransaction().setProducerId(pid2).setFirstOffset(6),
abortedTransactions.add(new FetchResponse.AbortedTransaction(pid1, 0L)); new FetchResponseData.AbortedTransaction().setProducerId(pid1).setFirstOffset(0)
);
client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer), client.prepareResponse(fullFetchResponseWithAbortedTransactions(MemoryRecords.readableRecords(buffer),
abortedTransactions, Errors.NONE, 100L, 100L, 0)); abortedTransactions, Errors.NONE, 100L, 100L, 0));
@ -3151,8 +3187,8 @@ public class FetcherTest {
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 0)); new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0));
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -3184,8 +3220,8 @@ public class FetcherTest {
currentOffset += abortTransaction(buffer, 1L, currentOffset); currentOffset += abortTransaction(buffer, 1L, currentOffset);
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>(); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 0)); new FetchResponseData.AbortedTransaction().setProducerId(1).setFirstOffset(0));
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -3216,12 +3252,19 @@ public class FetcherTest {
subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1))); subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1)));
// Fetch some records and establish an incremental fetch session. // Fetch some records and establish an incremental fetch session.
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions1 = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> partitions1 = new LinkedHashMap<>();
partitions1.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, 2L, partitions1.put(tp0, new FetchResponseData.PartitionData()
2, 0L, null, this.records)); .setPartitionIndex(tp0.partition())
partitions1.put(tp1, new FetchResponse.PartitionData<>(Errors.NONE, 100L, .setHighWatermark(2)
FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, null, emptyRecords)); .setLastStableOffset(2)
FetchResponse<MemoryRecords> resp1 = new FetchResponse<>(Errors.NONE, partitions1, 0, 123); .setLogStartOffset(0)
.setRecords(this.records));
partitions1.put(tp1, new FetchResponseData.PartitionData()
.setPartitionIndex(tp1.partition())
.setHighWatermark(100)
.setLogStartOffset(0)
.setRecords(emptyRecords));
FetchResponse resp1 = FetchResponse.of(Errors.NONE, 0, 123, partitions1);
client.prepareResponse(resp1); client.prepareResponse(resp1);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches()); assertFalse(fetcher.hasCompletedFetches());
@ -3246,8 +3289,8 @@ public class FetcherTest {
assertEquals(4L, subscriptions.position(tp0).offset); assertEquals(4L, subscriptions.position(tp0).offset);
// The second response contains no new records. // The second response contains no new records.
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions2 = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> partitions2 = new LinkedHashMap<>();
FetchResponse<MemoryRecords> resp2 = new FetchResponse<>(Errors.NONE, partitions2, 0, 123); FetchResponse resp2 = FetchResponse.of(Errors.NONE, 0, 123, partitions2);
client.prepareResponse(resp2); client.prepareResponse(resp2);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
@ -3257,10 +3300,14 @@ public class FetcherTest {
assertEquals(1L, subscriptions.position(tp1).offset); assertEquals(1L, subscriptions.position(tp1).offset);
// The third response contains some new records for tp0. // The third response contains some new records for tp0.
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions3 = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> partitions3 = new LinkedHashMap<>();
partitions3.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, 100L, partitions3.put(tp0, new FetchResponseData.PartitionData()
4, 0L, null, this.nextRecords)); .setPartitionIndex(tp0.partition())
FetchResponse<MemoryRecords> resp3 = new FetchResponse<>(Errors.NONE, partitions3, 0, 123); .setHighWatermark(100)
.setLastStableOffset(4)
.setLogStartOffset(0)
.setRecords(this.nextRecords));
FetchResponse resp3 = FetchResponse.of(Errors.NONE, 0, 123, partitions3);
client.prepareResponse(resp3); client.prepareResponse(resp3);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
@ -3319,7 +3366,7 @@ public class FetcherTest {
} }
@Override @Override
public boolean handleResponse(FetchResponse<?> response) { public boolean handleResponse(FetchResponse response) {
verifySessionPartitions(); verifySessionPartitions();
return handler.handleResponse(response); return handler.handleResponse(response);
} }
@ -3367,14 +3414,18 @@ public class FetcherTest {
if (!client.requests().isEmpty()) { if (!client.requests().isEmpty()) {
ClientRequest request = client.requests().peek(); ClientRequest request = client.requests().peek();
FetchRequest fetchRequest = (FetchRequest) request.requestBuilder().build(); FetchRequest fetchRequest = (FetchRequest) request.requestBuilder().build();
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseMap = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseMap = new LinkedHashMap<>();
for (Map.Entry<TopicPartition, FetchRequest.PartitionData> entry : fetchRequest.fetchData().entrySet()) { for (Map.Entry<TopicPartition, FetchRequest.PartitionData> entry : fetchRequest.fetchData().entrySet()) {
TopicPartition tp = entry.getKey(); TopicPartition tp = entry.getKey();
long offset = entry.getValue().fetchOffset; long offset = entry.getValue().fetchOffset;
responseMap.put(tp, new FetchResponse.PartitionData<>(Errors.NONE, offset + 2L, offset + 2, responseMap.put(tp, new FetchResponseData.PartitionData()
0L, null, buildRecords(offset, 2, offset))); .setPartitionIndex(tp.partition())
.setHighWatermark(offset + 2)
.setLastStableOffset(offset + 2)
.setLogStartOffset(0)
.setRecords(buildRecords(offset, 2, offset)));
} }
client.respondToRequest(request, new FetchResponse<>(Errors.NONE, responseMap, 0, 123)); client.respondToRequest(request, FetchResponse.of(Errors.NONE, 0, 123, responseMap));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
} }
} }
@ -3429,11 +3480,15 @@ public class FetcherTest {
assertTrue(epoch == 0 || epoch == nextEpoch, assertTrue(epoch == 0 || epoch == nextEpoch,
String.format("Unexpected epoch expected %d got %d", nextEpoch, epoch)); String.format("Unexpected epoch expected %d got %d", nextEpoch, epoch));
nextEpoch++; nextEpoch++;
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseMap = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseMap = new LinkedHashMap<>();
responseMap.put(tp0, new FetchResponse.PartitionData<>(Errors.NONE, nextOffset + 2L, nextOffset + 2, responseMap.put(tp0, new FetchResponseData.PartitionData()
0L, null, buildRecords(nextOffset, 2, nextOffset))); .setPartitionIndex(tp0.partition())
.setHighWatermark(nextOffset + 2)
.setLastStableOffset(nextOffset + 2)
.setLogStartOffset(0)
.setRecords(buildRecords(nextOffset, 2, nextOffset)));
nextOffset += 2; nextOffset += 2;
client.respondToRequest(request, new FetchResponse<>(Errors.NONE, responseMap, 0, 123)); client.respondToRequest(request, FetchResponse.of(Errors.NONE, 0, 123, responseMap));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
} }
} }
@ -3483,7 +3538,6 @@ public class FetcherTest {
commitTransaction(buffer, 1L, currentOffset); commitTransaction(buffer, 1L, currentOffset);
buffer.flip(); buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
MemoryRecords records = MemoryRecords.readableRecords(buffer); MemoryRecords records = MemoryRecords.readableRecords(buffer);
assignFromUser(singleton(tp0)); assignFromUser(singleton(tp0));
@ -3496,7 +3550,7 @@ public class FetcherTest {
FetchRequest request = (FetchRequest) body; FetchRequest request = (FetchRequest) body;
assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel()); assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
return true; return true;
}, fullFetchResponseWithAbortedTransactions(records, abortedTransactions, Errors.NONE, 100L, 100L, 0)); }, fullFetchResponseWithAbortedTransactions(records, Collections.emptyList(), Errors.NONE, 100L, 100L, 0));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
assertTrue(fetcher.hasCompletedFetches()); assertTrue(fetcher.hasCompletedFetches());
@ -4463,41 +4517,66 @@ public class FetcherTest {
return new ListOffsetsResponse(data); return new ListOffsetsResponse(data);
} }
private FetchResponse<MemoryRecords> fullFetchResponseWithAbortedTransactions(MemoryRecords records, private FetchResponse fullFetchResponseWithAbortedTransactions(MemoryRecords records,
List<FetchResponse.AbortedTransaction> abortedTransactions, List<FetchResponseData.AbortedTransaction> abortedTransactions,
Errors error, Errors error,
long lastStableOffset, long lastStableOffset,
long hw, long hw,
int throttleTime) { int throttleTime) {
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = Collections.singletonMap(tp0, Map<TopicPartition, FetchResponseData.PartitionData> partitions = Collections.singletonMap(tp0,
new FetchResponse.PartitionData<>(error, hw, lastStableOffset, 0L, abortedTransactions, records)); new FetchResponseData.PartitionData()
return new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), throttleTime, INVALID_SESSION_ID); .setPartitionIndex(tp0.partition())
.setErrorCode(error.code())
.setHighWatermark(hw)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(0)
.setAbortedTransactions(abortedTransactions)
.setRecords(records));
return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions));
} }
private FetchResponse<MemoryRecords> fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw, int throttleTime) { private FetchResponse fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw, int throttleTime) {
return fullFetchResponse(tp, records, error, hw, FetchResponse.INVALID_LAST_STABLE_OFFSET, throttleTime); return fullFetchResponse(tp, records, error, hw, FetchResponse.INVALID_LAST_STABLE_OFFSET, throttleTime);
} }
private FetchResponse<MemoryRecords> fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw, private FetchResponse fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw,
long lastStableOffset, int throttleTime) { long lastStableOffset, int throttleTime) {
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = Collections.singletonMap(tp, Map<TopicPartition, FetchResponseData.PartitionData> partitions = Collections.singletonMap(tp,
new FetchResponse.PartitionData<>(error, hw, lastStableOffset, 0L, null, records)); new FetchResponseData.PartitionData()
return new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), throttleTime, INVALID_SESSION_ID); .setPartitionIndex(tp.partition())
.setErrorCode(error.code())
.setHighWatermark(hw)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(0)
.setRecords(records));
return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions));
} }
private FetchResponse<MemoryRecords> fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw, private FetchResponse fullFetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw,
long lastStableOffset, int throttleTime, Optional<Integer> preferredReplicaId) { long lastStableOffset, int throttleTime, Optional<Integer> preferredReplicaId) {
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = Collections.singletonMap(tp, Map<TopicPartition, FetchResponseData.PartitionData> partitions = Collections.singletonMap(tp,
new FetchResponse.PartitionData<>(error, hw, lastStableOffset, 0L, new FetchResponseData.PartitionData()
preferredReplicaId, null, records)); .setPartitionIndex(tp.partition())
return new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), throttleTime, INVALID_SESSION_ID); .setErrorCode(error.code())
.setHighWatermark(hw)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(0)
.setRecords(records)
.setPreferredReadReplica(preferredReplicaId.orElse(FetchResponse.INVALID_PREFERRED_REPLICA_ID)));
return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions));
} }
private FetchResponse<MemoryRecords> fetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw, private FetchResponse fetchResponse(TopicPartition tp, MemoryRecords records, Errors error, long hw,
long lastStableOffset, long logStartOffset, int throttleTime) { long lastStableOffset, long logStartOffset, int throttleTime) {
Map<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> partitions = Collections.singletonMap(tp, Map<TopicPartition, FetchResponseData.PartitionData> partitions = Collections.singletonMap(tp,
new FetchResponse.PartitionData<>(error, hw, lastStableOffset, logStartOffset, null, records)); new FetchResponseData.PartitionData()
return new FetchResponse<>(Errors.NONE, new LinkedHashMap<>(partitions), throttleTime, INVALID_SESSION_ID); .setPartitionIndex(tp.partition())
.setErrorCode(error.code())
.setHighWatermark(hw)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(logStartOffset)
.setRecords(records));
return FetchResponse.of(Errors.NONE, throttleTime, INVALID_SESSION_ID, new LinkedHashMap<>(partitions));
} }
private MetadataResponse newMetadataResponse(String topic, Errors error) { private MetadataResponse newMetadataResponse(String topic, Errors error) {

View File

@ -803,15 +803,18 @@ public class RequestResponseTest {
@Test @Test
public void fetchResponseVersionTest() { public void fetchResponseVersionTest() {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10)); MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10));
responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>( responseData.put(new TopicPartition("test", 0),
Errors.NONE, 1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, new FetchResponseData.PartitionData()
0L, Optional.empty(), Collections.emptyList(), records)); .setPartitionIndex(0)
.setHighWatermark(1000000)
.setLogStartOffset(0)
.setRecords(records));
FetchResponse<MemoryRecords> v0Response = new FetchResponse<>(Errors.NONE, responseData, 0, INVALID_SESSION_ID); FetchResponse v0Response = FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, responseData);
FetchResponse<MemoryRecords> v1Response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID); FetchResponse v1Response = FetchResponse.of(Errors.NONE, 10, INVALID_SESSION_ID, responseData);
assertEquals(0, v0Response.throttleTimeMs(), "Throttle time must be zero"); assertEquals(0, v0Response.throttleTimeMs(), "Throttle time must be zero");
assertEquals(10, v1Response.throttleTimeMs(), "Throttle time must be 10"); assertEquals(10, v1Response.throttleTimeMs(), "Throttle time must be 10");
assertEquals(responseData, v0Response.responseData(), "Response data does not match"); assertEquals(responseData, v0Response.responseData(), "Response data does not match");
@ -820,22 +823,34 @@ public class RequestResponseTest {
@Test @Test
public void testFetchResponseV4() { public void testFetchResponseV4() {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10)); MemoryRecords records = MemoryRecords.readableRecords(ByteBuffer.allocate(10));
List<FetchResponse.AbortedTransaction> abortedTransactions = asList( List<FetchResponseData.AbortedTransaction> abortedTransactions = asList(
new FetchResponse.AbortedTransaction(10, 100), new FetchResponseData.AbortedTransaction().setProducerId(10).setFirstOffset(100),
new FetchResponse.AbortedTransaction(15, 50) new FetchResponseData.AbortedTransaction().setProducerId(15).setFirstOffset(50)
); );
responseData.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData<>(Errors.NONE, 100000, responseData.put(new TopicPartition("bar", 0),
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), abortedTransactions, records)); new FetchResponseData.PartitionData()
responseData.put(new TopicPartition("bar", 1), new FetchResponse.PartitionData<>(Errors.NONE, 900000, .setPartitionIndex(0)
5, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), null, records)); .setHighWatermark(1000000)
responseData.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData<>(Errors.NONE, 70000, .setAbortedTransactions(abortedTransactions)
6, FetchResponse.INVALID_LOG_START_OFFSET, Optional.empty(), emptyList(), records)); .setRecords(records));
responseData.put(new TopicPartition("bar", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(900000)
.setLastStableOffset(5)
.setRecords(records));
responseData.put(new TopicPartition("foo", 0),
new FetchResponseData.PartitionData()
.setPartitionIndex(0)
.setHighWatermark(70000)
.setLastStableOffset(6)
.setRecords(records));
FetchResponse<MemoryRecords> response = new FetchResponse<>(Errors.NONE, responseData, 10, INVALID_SESSION_ID); FetchResponse response = FetchResponse.of(Errors.NONE, 10, INVALID_SESSION_ID, responseData);
FetchResponse<MemoryRecords> deserialized = FetchResponse.parse(response.serialize((short) 4), (short) 4); FetchResponse deserialized = FetchResponse.parse(response.serialize((short) 4), (short) 4);
assertEquals(responseData, deserialized.responseData()); assertEquals(responseData, deserialized.responseData());
} }
@ -849,7 +864,7 @@ public class RequestResponseTest {
} }
} }
private void verifyFetchResponseFullWrite(short apiVersion, FetchResponse<MemoryRecords> fetchResponse) throws Exception { private void verifyFetchResponseFullWrite(short apiVersion, FetchResponse fetchResponse) throws Exception {
int correlationId = 15; int correlationId = 15;
short responseHeaderVersion = FETCH.responseHeaderVersion(apiVersion); short responseHeaderVersion = FETCH.responseHeaderVersion(apiVersion);
@ -1158,38 +1173,49 @@ public class RequestResponseTest {
return FetchRequest.Builder.forConsumer(100, 100000, fetchData).setMaxBytes(1000).build((short) version); return FetchRequest.Builder.forConsumer(100, 100000, fetchData).setMaxBytes(1000).build((short) version);
} }
private FetchResponse<MemoryRecords> createFetchResponse(Errors error, int sessionId) { private FetchResponse createFetchResponse(Errors error, int sessionId) {
return new FetchResponse<>(error, new LinkedHashMap<>(), 25, sessionId); return FetchResponse.of(error, 25, sessionId, new LinkedHashMap<>());
} }
private FetchResponse<MemoryRecords> createFetchResponse(int sessionId) { private FetchResponse createFetchResponse(int sessionId) {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes()));
responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>(Errors.NONE, responseData.put(new TopicPartition("test", 0), new FetchResponseData.PartitionData()
1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, Optional.empty(), Collections.emptyList(), records)); .setPartitionIndex(0)
List<FetchResponse.AbortedTransaction> abortedTransactions = Collections.singletonList( .setHighWatermark(1000000)
new FetchResponse.AbortedTransaction(234L, 999L)); .setLogStartOffset(0)
responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData<>(Errors.NONE, .setRecords(records));
1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, Optional.empty(), abortedTransactions, MemoryRecords.EMPTY)); List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.singletonList(
return new FetchResponse<>(Errors.NONE, responseData, 25, sessionId); new FetchResponseData.AbortedTransaction().setProducerId(234L).setFirstOffset(999L));
responseData.put(new TopicPartition("test", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(1000000)
.setLogStartOffset(0)
.setAbortedTransactions(abortedTransactions));
return FetchResponse.of(Errors.NONE, 25, sessionId, responseData);
} }
private FetchResponse<MemoryRecords> createFetchResponse(boolean includeAborted) { private FetchResponse createFetchResponse(boolean includeAborted) {
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes()));
responseData.put(new TopicPartition("test", 0), new FetchResponseData.PartitionData()
.setPartitionIndex(0)
.setHighWatermark(1000000)
.setLogStartOffset(0)
.setRecords(records));
responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData<>(Errors.NONE, List<FetchResponseData.AbortedTransaction> abortedTransactions = Collections.emptyList();
1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, Optional.empty(), Collections.emptyList(), records));
List<FetchResponse.AbortedTransaction> abortedTransactions = Collections.emptyList();
if (includeAborted) { if (includeAborted) {
abortedTransactions = Collections.singletonList( abortedTransactions = Collections.singletonList(
new FetchResponse.AbortedTransaction(234L, 999L)); new FetchResponseData.AbortedTransaction().setProducerId(234L).setFirstOffset(999L));
} }
responseData.put(new TopicPartition("test", 1), new FetchResponse.PartitionData<>(Errors.NONE, responseData.put(new TopicPartition("test", 1), new FetchResponseData.PartitionData()
1000000, FetchResponse.INVALID_LAST_STABLE_OFFSET, 0L, Optional.empty(), abortedTransactions, MemoryRecords.EMPTY)); .setPartitionIndex(1)
.setHighWatermark(1000000)
.setLogStartOffset(0)
.setAbortedTransactions(abortedTransactions));
return new FetchResponse<>(Errors.NONE, responseData, 25, INVALID_SESSION_ID); return FetchResponse.of(Errors.NONE, 25, INVALID_SESSION_ID, responseData);
} }
private HeartbeatRequest createHeartBeatRequest() { private HeartbeatRequest createHeartBeatRequest() {

View File

@ -40,7 +40,6 @@ import org.apache.kafka.common.errors._
import org.apache.kafka.common.message.{DescribeProducersResponseData, FetchResponseData} import org.apache.kafka.common.message.{DescribeProducersResponseData, FetchResponseData}
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.requests.ListOffsetsRequest
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET
import org.apache.kafka.common.requests.ProduceResponse.RecordError import org.apache.kafka.common.requests.ProduceResponse.RecordError
@ -1572,7 +1571,7 @@ class Log(@volatile private var _dir: File,
private def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, private def emptyFetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
includeAbortedTxns: Boolean): FetchDataInfo = { includeAbortedTxns: Boolean): FetchDataInfo = {
val abortedTransactions = val abortedTransactions =
if (includeAbortedTxns) Some(List.empty[AbortedTransaction]) if (includeAbortedTxns) Some(List.empty[FetchResponseData.AbortedTransaction])
else None else None
FetchDataInfo(fetchOffsetMetadata, FetchDataInfo(fetchOffsetMetadata,
MemoryRecords.EMPTY, MemoryRecords.EMPTY,
@ -1676,7 +1675,7 @@ class Log(@volatile private var _dir: File,
logEndOffset logEndOffset
} }
val abortedTransactions = ListBuffer.empty[AbortedTransaction] val abortedTransactions = ListBuffer.empty[FetchResponseData.AbortedTransaction]
def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction) def accumulator(abortedTxns: List[AbortedTxn]): Unit = abortedTransactions ++= abortedTxns.map(_.asAbortedTransaction)
collectAbortedTransactions(startOffset, upperBoundOffset, segmentEntry, accumulator) collectAbortedTransactions(startOffset, upperBoundOffset, segmentEntry, accumulator)

View File

@ -20,10 +20,9 @@ import java.io.{File, IOException}
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels.FileChannel import java.nio.channels.FileChannel
import java.nio.file.{Files, StandardOpenOption} import java.nio.file.{Files, StandardOpenOption}
import kafka.utils.{Logging, nonthreadsafe} import kafka.utils.{Logging, nonthreadsafe}
import org.apache.kafka.common.KafkaException import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
@ -245,7 +244,9 @@ private[log] class AbortedTxn(val buffer: ByteBuffer) {
def lastStableOffset: Long = buffer.getLong(LastStableOffsetOffset) def lastStableOffset: Long = buffer.getLong(LastStableOffsetOffset)
def asAbortedTransaction: AbortedTransaction = new AbortedTransaction(producerId, firstOffset) def asAbortedTransaction: FetchResponseData.AbortedTransaction = new FetchResponseData.AbortedTransaction()
.setProducerId(producerId)
.setFirstOffset(firstOffset)
override def toString: String = override def toString: String =
s"AbortedTxn(version=$version, producerId=$producerId, firstOffset=$firstOffset, " + s"AbortedTxn(version=$version, producerId=$producerId, firstOffset=$firstOffset, " +

View File

@ -192,7 +192,7 @@ object RequestChannel extends Logging {
resources.add(newResource) resources.add(newResource)
} }
val data = new IncrementalAlterConfigsRequestData() val data = new IncrementalAlterConfigsRequestData()
.setValidateOnly(alterConfigs.data().validateOnly()) .setValidateOnly(alterConfigs.data.validateOnly())
.setResources(resources) .setResources(resources)
new IncrementalAlterConfigsRequest.Builder(data).build(alterConfigs.version) new IncrementalAlterConfigsRequest.Builder(data).build(alterConfigs.version)

View File

@ -135,7 +135,7 @@ object RequestConvertToJson {
case res: EndQuorumEpochResponse => EndQuorumEpochResponseDataJsonConverter.write(res.data, version) case res: EndQuorumEpochResponse => EndQuorumEpochResponseDataJsonConverter.write(res.data, version)
case res: EnvelopeResponse => EnvelopeResponseDataJsonConverter.write(res.data, version) case res: EnvelopeResponse => EnvelopeResponseDataJsonConverter.write(res.data, version)
case res: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version) case res: ExpireDelegationTokenResponse => ExpireDelegationTokenResponseDataJsonConverter.write(res.data, version)
case res: FetchResponse[_] => FetchResponseDataJsonConverter.write(res.data, version, false) case res: FetchResponse => FetchResponseDataJsonConverter.write(res.data, version, false)
case res: FindCoordinatorResponse => FindCoordinatorResponseDataJsonConverter.write(res.data, version) case res: FindCoordinatorResponse => FindCoordinatorResponseDataJsonConverter.write(res.data, version)
case res: HeartbeatResponse => HeartbeatResponseDataJsonConverter.write(res.data, version) case res: HeartbeatResponse => HeartbeatResponseDataJsonConverter.write(res.data, version)
case res: IncrementalAlterConfigsResponse => IncrementalAlterConfigsResponseDataJsonConverter.write(res.data, version) case res: IncrementalAlterConfigsResponse => IncrementalAlterConfigsResponseDataJsonConverter.write(res.data, version)

View File

@ -17,37 +17,33 @@
package kafka.server package kafka.server
import kafka.cluster.BrokerEndPoint
import kafka.common.ClientIdAndBroker
import kafka.log.LogAppendInfo
import kafka.metrics.KafkaMetricsGroup
import kafka.server.AbstractFetcherThread.{ReplicaFetch, ResultWithPartitions}
import kafka.utils.CoreUtils.inLock
import kafka.utils.Implicits._
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.PartitionStates
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.message.{FetchResponseData, OffsetForLeaderEpochRequestData}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Records}
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.{InvalidRecordException, TopicPartition}
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util import java.util
import java.util.Optional import java.util.Optional
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import kafka.cluster.BrokerEndPoint
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
import kafka.utils.Implicits._
import org.apache.kafka.common.errors._
import kafka.common.ClientIdAndBroker
import kafka.metrics.KafkaMetricsGroup
import kafka.utils.CoreUtils.inLock
import org.apache.kafka.common.protocol.Errors
import scala.collection.{Map, Set, mutable} import scala.collection.{Map, Set, mutable}
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicLong
import kafka.log.LogAppendInfo
import kafka.server.AbstractFetcherThread.ReplicaFetch
import kafka.server.AbstractFetcherThread.ResultWithPartitions
import org.apache.kafka.common.{InvalidRecordException, TopicPartition}
import org.apache.kafka.common.internals.PartitionStates
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Records}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import scala.math._ import scala.math._
/** /**
@ -62,7 +58,7 @@ abstract class AbstractFetcherThread(name: String,
val brokerTopicStats: BrokerTopicStats) //BrokerTopicStats's lifecycle managed by ReplicaManager val brokerTopicStats: BrokerTopicStats) //BrokerTopicStats's lifecycle managed by ReplicaManager
extends ShutdownableThread(name, isInterruptible) { extends ShutdownableThread(name, isInterruptible) {
type FetchData = FetchResponse.PartitionData[Records] type FetchData = FetchResponseData.PartitionData
type EpochData = OffsetForLeaderEpochRequestData.OffsetForLeaderPartition type EpochData = OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
private val partitionStates = new PartitionStates[PartitionFetchState] private val partitionStates = new PartitionStates[PartitionFetchState]
@ -340,7 +336,7 @@ abstract class AbstractFetcherThread(name: String,
// the current offset is the same as the offset requested. // the current offset is the same as the offset requested.
val fetchPartitionData = sessionPartitions.get(topicPartition) val fetchPartitionData = sessionPartitions.get(topicPartition)
if (fetchPartitionData != null && fetchPartitionData.fetchOffset == currentFetchState.fetchOffset && currentFetchState.isReadyForFetch) { if (fetchPartitionData != null && fetchPartitionData.fetchOffset == currentFetchState.fetchOffset && currentFetchState.isReadyForFetch) {
partitionData.error match { Errors.forCode(partitionData.errorCode) match {
case Errors.NONE => case Errors.NONE =>
try { try {
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
@ -364,7 +360,7 @@ abstract class AbstractFetcherThread(name: String,
} }
} }
if (isTruncationOnFetchSupported) { if (isTruncationOnFetchSupported) {
partitionData.divergingEpoch.ifPresent { divergingEpoch => FetchResponse.divergingEpoch(partitionData).ifPresent { divergingEpoch =>
divergingEndOffsets += topicPartition -> new EpochEndOffset() divergingEndOffsets += topicPartition -> new EpochEndOffset()
.setPartition(topicPartition.partition) .setPartition(topicPartition.partition)
.setErrorCode(Errors.NONE.code) .setErrorCode(Errors.NONE.code)
@ -416,9 +412,8 @@ abstract class AbstractFetcherThread(name: String,
"expected to persist.") "expected to persist.")
partitionsWithError += topicPartition partitionsWithError += topicPartition
case _ => case partitionError =>
error(s"Error for partition $topicPartition at offset ${currentFetchState.fetchOffset}", error(s"Error for partition $topicPartition at offset ${currentFetchState.fetchOffset}", partitionError.exception)
partitionData.error.exception)
partitionsWithError += topicPartition partitionsWithError += topicPartition
} }
} }

View File

@ -34,7 +34,6 @@ import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicRe
import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBroker import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBroker
import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, BrokerHeartbeatResponseData, BrokerRegistrationResponseData, CreateTopicsResponseData, DescribeQuorumResponseData, EndQuorumEpochResponseData, FetchResponseData, MetadataResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, UnregisterBrokerResponseData, VoteResponseData} import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, BrokerHeartbeatResponseData, BrokerRegistrationResponseData, CreateTopicsResponseData, DescribeQuorumResponseData, EndQuorumEpochResponseData, FetchResponseData, MetadataResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, UnregisterBrokerResponseData, VoteResponseData}
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors}
import org.apache.kafka.common.record.BaseRecords
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.Resource import org.apache.kafka.common.resource.Resource
import org.apache.kafka.common.resource.Resource.CLUSTER_NAME import org.apache.kafka.common.resource.Resource.CLUSTER_NAME
@ -116,7 +115,7 @@ class ControllerApis(val requestChannel: RequestChannel,
private def handleFetch(request: RequestChannel.Request): Unit = { private def handleFetch(request: RequestChannel.Request): Unit = {
authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
handleRaftRequest(request, response => new FetchResponse[BaseRecords](response.asInstanceOf[FetchResponseData])) handleRaftRequest(request, response => new FetchResponse(response.asInstanceOf[FetchResponseData]))
} }
def handleMetadataRequest(request: RequestChannel.Request): Unit = { def handleMetadataRequest(request: RequestChannel.Request): Unit = {

View File

@ -17,8 +17,8 @@
package kafka.server package kafka.server
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.record.Records import org.apache.kafka.common.record.Records
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
sealed trait FetchIsolation sealed trait FetchIsolation
case object FetchLogEnd extends FetchIsolation case object FetchLogEnd extends FetchIsolation
@ -28,4 +28,4 @@ case object FetchTxnCommitted extends FetchIsolation
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
records: Records, records: Records,
firstEntryIncomplete: Boolean = false, firstEntryIncomplete: Boolean = false,
abortedTransactions: Option[List[AbortedTransaction]] = None) abortedTransactions: Option[List[FetchResponseData.AbortedTransaction]] = None)

View File

@ -17,27 +17,26 @@
package kafka.server package kafka.server
import java.util
import java.util.Optional
import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.Records
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INITIAL_EPOCH, INVALID_SESSION_ID} import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INITIAL_EPOCH, INVALID_SESSION_ID}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata}
import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, Time, Utils} import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, Time, Utils}
import scala.math.Ordered.orderingToOrdered import java.util
import java.util.Optional
import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
import scala.collection.{mutable, _} import scala.collection.{mutable, _}
import scala.math.Ordered.orderingToOrdered
object FetchSession { object FetchSession {
type REQ_MAP = util.Map[TopicPartition, FetchRequest.PartitionData] type REQ_MAP = util.Map[TopicPartition, FetchRequest.PartitionData]
type RESP_MAP = util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] type RESP_MAP = util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
type CACHE_MAP = ImplicitLinkedHashCollection[CachedPartition] type CACHE_MAP = ImplicitLinkedHashCollection[CachedPartition]
type RESP_MAP_ITER = util.Iterator[util.Map.Entry[TopicPartition, FetchResponse.PartitionData[Records]]] type RESP_MAP_ITER = util.Iterator[util.Map.Entry[TopicPartition, FetchResponseData.PartitionData]]
val NUM_INCREMENTAL_FETCH_SESSISONS = "NumIncrementalFetchSessions" val NUM_INCREMENTAL_FETCH_SESSISONS = "NumIncrementalFetchSessions"
val NUM_INCREMENTAL_FETCH_PARTITIONS_CACHED = "NumIncrementalFetchPartitionsCached" val NUM_INCREMENTAL_FETCH_PARTITIONS_CACHED = "NumIncrementalFetchPartitionsCached"
@ -100,7 +99,7 @@ class CachedPartition(val topic: String,
reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch) reqData.currentLeaderEpoch, reqData.logStartOffset, -1, reqData.lastFetchedEpoch)
def this(part: TopicPartition, reqData: FetchRequest.PartitionData, def this(part: TopicPartition, reqData: FetchRequest.PartitionData,
respData: FetchResponse.PartitionData[Records]) = respData: FetchResponseData.PartitionData) =
this(part.topic, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark, this(part.topic, part.partition, reqData.maxBytes, reqData.fetchOffset, respData.highWatermark,
reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch) reqData.currentLeaderEpoch, reqData.logStartOffset, respData.logStartOffset, reqData.lastFetchedEpoch)
@ -125,10 +124,10 @@ class CachedPartition(val topic: String,
* @param updateResponseData if set to true, update this CachedPartition with new request and response data. * @param updateResponseData if set to true, update this CachedPartition with new request and response data.
* @return True if this partition should be included in the response; false if it can be omitted. * @return True if this partition should be included in the response; false if it can be omitted.
*/ */
def maybeUpdateResponseData(respData: FetchResponse.PartitionData[Records], updateResponseData: Boolean): Boolean = { def maybeUpdateResponseData(respData: FetchResponseData.PartitionData, updateResponseData: Boolean): Boolean = {
// Check the response data. // Check the response data.
var mustRespond = false var mustRespond = false
if ((respData.records != null) && (respData.records.sizeInBytes > 0)) { if (FetchResponse.recordsSize(respData) > 0) {
// Partitions with new data are always included in the response. // Partitions with new data are always included in the response.
mustRespond = true mustRespond = true
} }
@ -142,11 +141,11 @@ class CachedPartition(val topic: String,
if (updateResponseData) if (updateResponseData)
localLogStartOffset = respData.logStartOffset localLogStartOffset = respData.logStartOffset
} }
if (respData.preferredReadReplica.isPresent) { if (FetchResponse.isPreferredReplica(respData)) {
// If the broker computed a preferred read replica, we need to include it in the response // If the broker computed a preferred read replica, we need to include it in the response
mustRespond = true mustRespond = true
} }
if (respData.error.code != 0) { if (respData.errorCode != Errors.NONE.code) {
// Partitions with errors are always included in the response. // Partitions with errors are always included in the response.
// We also set the cached highWatermark to an invalid offset, -1. // We also set the cached highWatermark to an invalid offset, -1.
// This ensures that when the error goes away, we re-send the partition. // This ensures that when the error goes away, we re-send the partition.
@ -154,7 +153,8 @@ class CachedPartition(val topic: String,
highWatermark = -1 highWatermark = -1
mustRespond = true mustRespond = true
} }
if (respData.divergingEpoch.isPresent) {
if (FetchResponse.isDivergingEpoch(respData)) {
// Partitions with diverging epoch are always included in response to trigger truncation. // Partitions with diverging epoch are always included in response to trigger truncation.
mustRespond = true mustRespond = true
} }
@ -163,7 +163,7 @@ class CachedPartition(val topic: String,
override def hashCode: Int = (31 * partition) + topic.hashCode override def hashCode: Int = (31 * partition) + topic.hashCode
def canEqual(that: Any) = that.isInstanceOf[CachedPartition] def canEqual(that: Any): Boolean = that.isInstanceOf[CachedPartition]
override def equals(that: Any): Boolean = override def equals(that: Any): Boolean =
that match { that match {
@ -292,7 +292,7 @@ trait FetchContext extends Logging {
* Updates the fetch context with new partition information. Generates response data. * Updates the fetch context with new partition information. Generates response data.
* The response data may require subsequent down-conversion. * The response data may require subsequent down-conversion.
*/ */
def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse
def partitionsToLogString(partitions: util.Collection[TopicPartition]): String = def partitionsToLogString(partitions: util.Collection[TopicPartition]): String =
FetchSession.partitionsToLogString(partitions, isTraceEnabled) FetchSession.partitionsToLogString(partitions, isTraceEnabled)
@ -300,8 +300,8 @@ trait FetchContext extends Logging {
/** /**
* Return an empty throttled response due to quota violation. * Return an empty throttled response due to quota violation.
*/ */
def getThrottledResponse(throttleTimeMs: Int): FetchResponse[Records] = def getThrottledResponse(throttleTimeMs: Int): FetchResponse =
new FetchResponse(Errors.NONE, new FetchSession.RESP_MAP, throttleTimeMs, INVALID_SESSION_ID) FetchResponse.of(Errors.NONE, throttleTimeMs, INVALID_SESSION_ID, new FetchSession.RESP_MAP)
} }
/** /**
@ -318,9 +318,9 @@ class SessionErrorContext(val error: Errors,
} }
// Because of the fetch session error, we don't know what partitions were supposed to be in this request. // Because of the fetch session error, we don't know what partitions were supposed to be in this request.
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
debug(s"Session error fetch context returning $error") debug(s"Session error fetch context returning $error")
new FetchResponse(error, new FetchSession.RESP_MAP, 0, INVALID_SESSION_ID) FetchResponse.of(error, 0, INVALID_SESSION_ID, new FetchSession.RESP_MAP)
} }
} }
@ -341,9 +341,9 @@ class SessionlessFetchContext(val fetchData: util.Map[TopicPartition, FetchReque
FetchResponse.sizeOf(versionId, updates.entrySet.iterator) FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet)}") debug(s"Sessionless fetch context returning ${partitionsToLogString(updates.keySet)}")
new FetchResponse(Errors.NONE, updates, 0, INVALID_SESSION_ID) FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, updates)
} }
} }
@ -372,7 +372,7 @@ class FullFetchContext(private val time: Time,
FetchResponse.sizeOf(versionId, updates.entrySet.iterator) FetchResponse.sizeOf(versionId, updates.entrySet.iterator)
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
def createNewSession: FetchSession.CACHE_MAP = { def createNewSession: FetchSession.CACHE_MAP = {
val cachedPartitions = new FetchSession.CACHE_MAP(updates.size) val cachedPartitions = new FetchSession.CACHE_MAP(updates.size)
updates.forEach { (part, respData) => updates.forEach { (part, respData) =>
@ -385,7 +385,7 @@ class FullFetchContext(private val time: Time,
updates.size, () => createNewSession) updates.size, () => createNewSession)
debug(s"Full fetch context with session id $responseSessionId returning " + debug(s"Full fetch context with session id $responseSessionId returning " +
s"${partitionsToLogString(updates.keySet)}") s"${partitionsToLogString(updates.keySet)}")
new FetchResponse(Errors.NONE, updates, 0, responseSessionId) FetchResponse.of(Errors.NONE, 0, responseSessionId, updates)
} }
} }
@ -417,7 +417,7 @@ class IncrementalFetchContext(private val time: Time,
private class PartitionIterator(val iter: FetchSession.RESP_MAP_ITER, private class PartitionIterator(val iter: FetchSession.RESP_MAP_ITER,
val updateFetchContextAndRemoveUnselected: Boolean) val updateFetchContextAndRemoveUnselected: Boolean)
extends FetchSession.RESP_MAP_ITER { extends FetchSession.RESP_MAP_ITER {
var nextElement: util.Map.Entry[TopicPartition, FetchResponse.PartitionData[Records]] = null var nextElement: util.Map.Entry[TopicPartition, FetchResponseData.PartitionData] = null
override def hasNext: Boolean = { override def hasNext: Boolean = {
while ((nextElement == null) && iter.hasNext) { while ((nextElement == null) && iter.hasNext) {
@ -441,7 +441,7 @@ class IncrementalFetchContext(private val time: Time,
nextElement != null nextElement != null
} }
override def next(): util.Map.Entry[TopicPartition, FetchResponse.PartitionData[Records]] = { override def next(): util.Map.Entry[TopicPartition, FetchResponseData.PartitionData] = {
if (!hasNext) throw new NoSuchElementException if (!hasNext) throw new NoSuchElementException
val element = nextElement val element = nextElement
nextElement = null nextElement = null
@ -463,7 +463,7 @@ class IncrementalFetchContext(private val time: Time,
} }
} }
override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse[Records] = { override def updateAndGenerateResponseData(updates: FetchSession.RESP_MAP): FetchResponse = {
session.synchronized { session.synchronized {
// Check to make sure that the session epoch didn't change in between // Check to make sure that the session epoch didn't change in between
// creating this fetch context and generating this response. // creating this fetch context and generating this response.
@ -471,7 +471,7 @@ class IncrementalFetchContext(private val time: Time,
if (session.epoch != expectedEpoch) { if (session.epoch != expectedEpoch) {
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
s"got ${session.epoch}. Possible duplicate request.") s"got ${session.epoch}. Possible duplicate request.")
new FetchResponse(Errors.INVALID_FETCH_SESSION_EPOCH, new FetchSession.RESP_MAP, 0, session.id) FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, 0, session.id, new FetchSession.RESP_MAP)
} else { } else {
// Iterate over the update list using PartitionIterator. This will prune updates which don't need to be sent // Iterate over the update list using PartitionIterator. This will prune updates which don't need to be sent
val partitionIter = new PartitionIterator(updates.entrySet.iterator, true) val partitionIter = new PartitionIterator(updates.entrySet.iterator, true)
@ -480,12 +480,12 @@ class IncrementalFetchContext(private val time: Time,
} }
debug(s"Incremental fetch context with session id ${session.id} returning " + debug(s"Incremental fetch context with session id ${session.id} returning " +
s"${partitionsToLogString(updates.keySet)}") s"${partitionsToLogString(updates.keySet)}")
new FetchResponse(Errors.NONE, updates, 0, session.id) FetchResponse.of(Errors.NONE, 0, session.id, updates)
} }
} }
} }
override def getThrottledResponse(throttleTimeMs: Int): FetchResponse[Records] = { override def getThrottledResponse(throttleTimeMs: Int): FetchResponse = {
session.synchronized { session.synchronized {
// Check to make sure that the session epoch didn't change in between // Check to make sure that the session epoch didn't change in between
// creating this fetch context and generating this response. // creating this fetch context and generating this response.
@ -493,9 +493,9 @@ class IncrementalFetchContext(private val time: Time,
if (session.epoch != expectedEpoch) { if (session.epoch != expectedEpoch) {
info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " + info(s"Incremental fetch session ${session.id} expected epoch $expectedEpoch, but " +
s"got ${session.epoch}. Possible duplicate request.") s"got ${session.epoch}. Possible duplicate request.")
new FetchResponse(Errors.INVALID_FETCH_SESSION_EPOCH, new FetchSession.RESP_MAP, throttleTimeMs, session.id) FetchResponse.of(Errors.INVALID_FETCH_SESSION_EPOCH, throttleTimeMs, session.id, new FetchSession.RESP_MAP)
} else { } else {
new FetchResponse(Errors.NONE, new FetchSession.RESP_MAP, throttleTimeMs, session.id) FetchResponse.of(Errors.NONE, throttleTimeMs, session.id, new FetchSession.RESP_MAP)
} }
} }
} }

View File

@ -17,24 +17,18 @@
package kafka.server package kafka.server
import java.lang.{Long => JLong}
import java.nio.ByteBuffer
import java.util
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicInteger
import java.util.{Collections, Optional}
import kafka.admin.AdminUtils import kafka.admin.AdminUtils
import kafka.api.{ApiVersion, ElectLeadersRequestOps, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0} import kafka.api.{ApiVersion, ElectLeadersRequestOps, KAFKA_0_11_0_IV0, KAFKA_2_3_IV0}
import kafka.common.OffsetAndMetadata import kafka.common.OffsetAndMetadata
import kafka.controller.ReplicaAssignment import kafka.controller.ReplicaAssignment
import kafka.coordinator.group.{GroupCoordinator, JoinGroupResult, LeaveGroupResult, SyncGroupResult} import kafka.coordinator.group._
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
import kafka.log.AppendOrigin import kafka.log.AppendOrigin
import kafka.message.ZStdCompressionCodec import kafka.message.ZStdCompressionCodec
import kafka.network.RequestChannel import kafka.network.RequestChannel
import kafka.security.authorizer.AuthorizerUtils import kafka.security.authorizer.AuthorizerUtils
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.server.metadata.ConfigRepository
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.AlterConfigOp.OpType
@ -61,7 +55,7 @@ import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsParti
import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic} import org.apache.kafka.common.message.MetadataResponseData.{MetadataResponsePartition, MetadataResponseTopic}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection} import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult, OffsetForLeaderTopicResultCollection}
import org.apache.kafka.common.message.{AddOffsetsToTxnResponseData, AlterClientQuotasResponseData, AlterConfigsResponseData, AlterPartitionReassignmentsResponseData, AlterReplicaLogDirsResponseData, CreateAclsResponseData, CreatePartitionsResponseData, CreateTopicsResponseData, DeleteAclsResponseData, DeleteGroupsResponseData, DeleteRecordsResponseData, DeleteTopicsResponseData, DescribeAclsResponseData, DescribeClientQuotasResponseData, DescribeClusterResponseData, DescribeConfigsResponseData, DescribeGroupsResponseData, DescribeLogDirsResponseData, DescribeProducersResponseData, DescribeTransactionsResponseData, EndTxnResponseData, ExpireDelegationTokenResponseData, FindCoordinatorResponseData, HeartbeatResponseData, InitProducerIdResponseData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsResponseData, ListOffsetsResponseData, ListPartitionReassignmentsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteResponseData, OffsetForLeaderEpochResponseData, RenewDelegationTokenResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, StopReplicaResponseData, SyncGroupResponseData, UpdateMetadataResponseData} import org.apache.kafka.common.message._
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.network.{ListenerName, Send}
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
@ -80,14 +74,18 @@ import org.apache.kafka.common.utils.{ProducerIdAndEpoch, Time}
import org.apache.kafka.common.{Node, TopicPartition, Uuid} import org.apache.kafka.common.{Node, TopicPartition, Uuid}
import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.authorizer._
import java.lang.{Long => JLong}
import java.nio.ByteBuffer
import java.util
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicInteger
import java.util.{Collections, Optional}
import scala.annotation.nowarn import scala.annotation.nowarn
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.collection.{Map, Seq, Set, immutable, mutable} import scala.collection.{Map, Seq, Set, immutable, mutable}
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.util.{Failure, Success, Try} import scala.util.{Failure, Success, Try}
import kafka.coordinator.group.GroupOverview
import kafka.server.metadata.ConfigRepository
/** /**
* Logic to handle the various Kafka requests * Logic to handle the various Kafka requests
@ -681,25 +679,20 @@ class KafkaApis(val requestChannel: RequestChannel,
None None
} }
def errorResponse[T >: MemoryRecords <: BaseRecords](error: Errors): FetchResponse.PartitionData[T] = { val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponseData.PartitionData)]()
new FetchResponse.PartitionData[T](error, FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET,
FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY)
}
val erroneous = mutable.ArrayBuffer[(TopicPartition, FetchResponse.PartitionData[Records])]()
val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]() val interesting = mutable.ArrayBuffer[(TopicPartition, FetchRequest.PartitionData)]()
if (fetchRequest.isFromFollower) { if (fetchRequest.isFromFollower) {
// The follower must have ClusterAction on ClusterResource in order to fetch partition data. // The follower must have ClusterAction on ClusterResource in order to fetch partition data.
if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) { if (authHelper.authorize(request.context, CLUSTER_ACTION, CLUSTER, CLUSTER_NAME)) {
fetchContext.foreachPartition { (topicPartition, data) => fetchContext.foreachPartition { (topicPartition, data) =>
if (!metadataCache.contains(topicPartition)) if (!metadataCache.contains(topicPartition))
erroneous += topicPartition -> errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
else else
interesting += (topicPartition -> data) interesting += (topicPartition -> data)
} }
} else { } else {
fetchContext.foreachPartition { (part, _) => fetchContext.foreachPartition { (part, _) =>
erroneous += part -> errorResponse(Errors.TOPIC_AUTHORIZATION_FAILED) erroneous += part -> FetchResponse.partitionResponse(part.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
} }
} }
} else { } else {
@ -711,9 +704,9 @@ class KafkaApis(val requestChannel: RequestChannel,
val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic) val authorizedTopics = authHelper.filterByAuthorized(request.context, READ, TOPIC, partitionDatas)(_._1.topic)
partitionDatas.foreach { case (topicPartition, data) => partitionDatas.foreach { case (topicPartition, data) =>
if (!authorizedTopics.contains(topicPartition.topic)) if (!authorizedTopics.contains(topicPartition.topic))
erroneous += topicPartition -> errorResponse(Errors.TOPIC_AUTHORIZATION_FAILED) erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.TOPIC_AUTHORIZATION_FAILED)
else if (!metadataCache.contains(topicPartition)) else if (!metadataCache.contains(topicPartition))
erroneous += topicPartition -> errorResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION) erroneous += topicPartition -> FetchResponse.partitionResponse(topicPartition.partition, Errors.UNKNOWN_TOPIC_OR_PARTITION)
else else
interesting += (topicPartition -> data) interesting += (topicPartition -> data)
} }
@ -732,12 +725,12 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
def maybeConvertFetchedData(tp: TopicPartition, def maybeConvertFetchedData(tp: TopicPartition,
partitionData: FetchResponse.PartitionData[Records]): FetchResponse.PartitionData[BaseRecords] = { partitionData: FetchResponseData.PartitionData): FetchResponseData.PartitionData = {
val logConfig = replicaManager.getLogConfig(tp) val logConfig = replicaManager.getLogConfig(tp)
if (logConfig.exists(_.compressionType == ZStdCompressionCodec.name) && versionId < 10) { if (logConfig.exists(_.compressionType == ZStdCompressionCodec.name) && versionId < 10) {
trace(s"Fetching messages is disabled for ZStandard compressed partition $tp. Sending unsupported version response to $clientId.") trace(s"Fetching messages is disabled for ZStandard compressed partition $tp. Sending unsupported version response to $clientId.")
errorResponse(Errors.UNSUPPORTED_COMPRESSION_TYPE) FetchResponse.partitionResponse(tp.partition, Errors.UNSUPPORTED_COMPRESSION_TYPE)
} else { } else {
// Down-conversion of the fetched records is needed when the stored magic version is // Down-conversion of the fetched records is needed when the stored magic version is
// greater than that supported by the client (as indicated by the fetch request version). If the // greater than that supported by the client (as indicated by the fetch request version). If the
@ -746,7 +739,7 @@ class KafkaApis(val requestChannel: RequestChannel,
// know it must be supported. However, if the magic version is changed from a higher version back to a // know it must be supported. However, if the magic version is changed from a higher version back to a
// lower version, this check will no longer be valid and we will fail to down-convert the messages // lower version, this check will no longer be valid and we will fail to down-convert the messages
// which were written in the new format prior to the version downgrade. // which were written in the new format prior to the version downgrade.
val unconvertedRecords = partitionData.records val unconvertedRecords = FetchResponse.recordsOrFail(partitionData)
val downConvertMagic = val downConvertMagic =
logConfig.map(_.messageFormatVersion.recordVersion.value).flatMap { magic => logConfig.map(_.messageFormatVersion.recordVersion.value).flatMap { magic =>
if (magic > RecordBatch.MAGIC_VALUE_V0 && versionId <= 1 && !unconvertedRecords.hasCompatibleMagic(RecordBatch.MAGIC_VALUE_V0)) if (magic > RecordBatch.MAGIC_VALUE_V0 && versionId <= 1 && !unconvertedRecords.hasCompatibleMagic(RecordBatch.MAGIC_VALUE_V0))
@ -762,7 +755,7 @@ class KafkaApis(val requestChannel: RequestChannel,
// For fetch requests from clients, check if down-conversion is disabled for the particular partition // For fetch requests from clients, check if down-conversion is disabled for the particular partition
if (!fetchRequest.isFromFollower && !logConfig.forall(_.messageDownConversionEnable)) { if (!fetchRequest.isFromFollower && !logConfig.forall(_.messageDownConversionEnable)) {
trace(s"Conversion to message format ${downConvertMagic.get} is disabled for partition $tp. Sending unsupported version response to $clientId.") trace(s"Conversion to message format ${downConvertMagic.get} is disabled for partition $tp. Sending unsupported version response to $clientId.")
errorResponse(Errors.UNSUPPORTED_VERSION) FetchResponse.partitionResponse(tp.partition, Errors.UNSUPPORTED_VERSION)
} else { } else {
try { try {
trace(s"Down converting records from partition $tp to message format version $magic for fetch request from $clientId") trace(s"Down converting records from partition $tp to message format version $magic for fetch request from $clientId")
@ -770,71 +763,77 @@ class KafkaApis(val requestChannel: RequestChannel,
// as possible. With KIP-283, we have the ability to lazily down-convert in a chunked manner. The lazy, chunked // as possible. With KIP-283, we have the ability to lazily down-convert in a chunked manner. The lazy, chunked
// down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the // down-conversion always guarantees that at least one batch of messages is down-converted and sent out to the
// client. // client.
val error = maybeDownConvertStorageError(partitionData.error) new FetchResponseData.PartitionData()
new FetchResponse.PartitionData[BaseRecords](error, partitionData.highWatermark, .setPartitionIndex(tp.partition)
partitionData.lastStableOffset, partitionData.logStartOffset, .setErrorCode(maybeDownConvertStorageError(Errors.forCode(partitionData.errorCode)).code)
partitionData.preferredReadReplica, partitionData.abortedTransactions, .setHighWatermark(partitionData.highWatermark)
new LazyDownConversionRecords(tp, unconvertedRecords, magic, fetchContext.getFetchOffset(tp).get, time)) .setLastStableOffset(partitionData.lastStableOffset)
.setLogStartOffset(partitionData.logStartOffset)
.setAbortedTransactions(partitionData.abortedTransactions)
.setRecords(new LazyDownConversionRecords(tp, unconvertedRecords, magic, fetchContext.getFetchOffset(tp).get, time))
.setPreferredReadReplica(partitionData.preferredReadReplica())
} catch { } catch {
case e: UnsupportedCompressionTypeException => case e: UnsupportedCompressionTypeException =>
trace("Received unsupported compression type error during down-conversion", e) trace("Received unsupported compression type error during down-conversion", e)
errorResponse(Errors.UNSUPPORTED_COMPRESSION_TYPE) FetchResponse.partitionResponse(tp.partition, Errors.UNSUPPORTED_COMPRESSION_TYPE)
} }
} }
case None => case None =>
val error = maybeDownConvertStorageError(partitionData.error) new FetchResponseData.PartitionData()
new FetchResponse.PartitionData[BaseRecords](error, .setPartitionIndex(tp.partition)
partitionData.highWatermark, .setErrorCode(maybeDownConvertStorageError(Errors.forCode(partitionData.errorCode)).code)
partitionData.lastStableOffset, .setHighWatermark(partitionData.highWatermark)
partitionData.logStartOffset, .setLastStableOffset(partitionData.lastStableOffset)
partitionData.preferredReadReplica, .setLogStartOffset(partitionData.logStartOffset)
partitionData.abortedTransactions, .setAbortedTransactions(partitionData.abortedTransactions)
partitionData.divergingEpoch, .setRecords(unconvertedRecords)
unconvertedRecords) .setPreferredReadReplica(partitionData.preferredReadReplica)
.setDivergingEpoch(partitionData.divergingEpoch)
} }
} }
} }
// the callback for process a fetch response, invoked before throttling // the callback for process a fetch response, invoked before throttling
def processResponseCallback(responsePartitionData: Seq[(TopicPartition, FetchPartitionData)]): Unit = { def processResponseCallback(responsePartitionData: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
val partitions = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val partitions = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
val reassigningPartitions = mutable.Set[TopicPartition]() val reassigningPartitions = mutable.Set[TopicPartition]()
responsePartitionData.foreach { case (tp, data) => responsePartitionData.foreach { case (tp, data) =>
val abortedTransactions = data.abortedTransactions.map(_.asJava).orNull val abortedTransactions = data.abortedTransactions.map(_.asJava).orNull
val lastStableOffset = data.lastStableOffset.getOrElse(FetchResponse.INVALID_LAST_STABLE_OFFSET) val lastStableOffset = data.lastStableOffset.getOrElse(FetchResponse.INVALID_LAST_STABLE_OFFSET)
if (data.isReassignmentFetch) if (data.isReassignmentFetch) reassigningPartitions.add(tp)
reassigningPartitions.add(tp) val partitionData = new FetchResponseData.PartitionData()
val error = maybeDownConvertStorageError(data.error) .setPartitionIndex(tp.partition)
partitions.put(tp, new FetchResponse.PartitionData( .setErrorCode(maybeDownConvertStorageError(data.error).code)
error, .setHighWatermark(data.highWatermark)
data.highWatermark, .setLastStableOffset(lastStableOffset)
lastStableOffset, .setLogStartOffset(data.logStartOffset)
data.logStartOffset, .setAbortedTransactions(abortedTransactions)
data.preferredReadReplica.map(int2Integer).asJava, .setRecords(data.records)
abortedTransactions, .setPreferredReadReplica(data.preferredReadReplica.getOrElse(FetchResponse.INVALID_PREFERRED_REPLICA_ID))
data.divergingEpoch.asJava, data.divergingEpoch.foreach(partitionData.setDivergingEpoch)
data.records)) partitions.put(tp, partitionData)
} }
erroneous.foreach { case (tp, data) => partitions.put(tp, data) } erroneous.foreach { case (tp, data) => partitions.put(tp, data) }
var unconvertedFetchResponse: FetchResponse[Records] = null var unconvertedFetchResponse: FetchResponse = null
def createResponse(throttleTimeMs: Int): FetchResponse[BaseRecords] = { def createResponse(throttleTimeMs: Int): FetchResponse = {
// Down-convert messages for each partition if required // Down-convert messages for each partition if required
val convertedData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[BaseRecords]] val convertedData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
unconvertedFetchResponse.responseData.forEach { (tp, unconvertedPartitionData) => unconvertedFetchResponse.responseData.forEach { (tp, unconvertedPartitionData) =>
if (unconvertedPartitionData.error != Errors.NONE) val error = Errors.forCode(unconvertedPartitionData.errorCode)
if (error != Errors.NONE)
debug(s"Fetch request with correlation id ${request.header.correlationId} from client $clientId " + debug(s"Fetch request with correlation id ${request.header.correlationId} from client $clientId " +
s"on partition $tp failed due to ${unconvertedPartitionData.error.exceptionName}") s"on partition $tp failed due to ${error.exceptionName}")
convertedData.put(tp, maybeConvertFetchedData(tp, unconvertedPartitionData)) convertedData.put(tp, maybeConvertFetchedData(tp, unconvertedPartitionData))
} }
// Prepare fetch response from converted data // Prepare fetch response from converted data
val response = new FetchResponse(unconvertedFetchResponse.error, convertedData, throttleTimeMs, val response = FetchResponse.of(unconvertedFetchResponse.error, throttleTimeMs, unconvertedFetchResponse.sessionId, convertedData)
unconvertedFetchResponse.sessionId)
// record the bytes out metrics only when the response is being sent // record the bytes out metrics only when the response is being sent
response.responseData.forEach { (tp, data) => response.responseData.forEach { (tp, data) =>
brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower, reassigningPartitions.contains(tp), data.records.sizeInBytes) brokerTopicStats.updateBytesOut(tp.topic, fetchRequest.isFromFollower,
reassigningPartitions.contains(tp), FetchResponse.recordsSize(data))
} }
response response
} }
@ -3367,7 +3366,7 @@ object KafkaApis {
// Traffic from both in-sync and out of sync replicas are accounted for in replication quota to ensure total replication // Traffic from both in-sync and out of sync replicas are accounted for in replication quota to ensure total replication
// traffic doesn't exceed quota. // traffic doesn't exceed quota.
private[server] def sizeOfThrottledPartitions(versionId: Short, private[server] def sizeOfThrottledPartitions(versionId: Short,
unconvertedResponse: FetchResponse[Records], unconvertedResponse: FetchResponse,
quota: ReplicationQuotaManager): Int = { quota: ReplicationQuotaManager): Int = {
FetchResponse.sizeOf(versionId, unconvertedResponse.responseData.entrySet FetchResponse.sizeOf(versionId, unconvertedResponse.responseData.entrySet
.iterator.asScala.filter(element => quota.isThrottled(element.getKey)).asJava) .iterator.asScala.filter(element => quota.isThrottled(element.getKey)).asJava)

View File

@ -17,28 +17,24 @@
package kafka.server package kafka.server
import java.util
import java.util.Optional
import kafka.api.Request import kafka.api.Request
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.log.{LeaderOffsetIncremented, LogAppendInfo} import kafka.log.{LeaderOffsetIncremented, LogAppendInfo}
import kafka.server.AbstractFetcherThread.ReplicaFetch import kafka.server.AbstractFetcherThread.{ReplicaFetch, ResultWithPartitions}
import kafka.server.AbstractFetcherThread.ResultWithPartitions
import kafka.server.QuotaFactory.UnboundedQuota import kafka.server.QuotaFactory.UnboundedQuota
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.Records
import org.apache.kafka.common.requests.FetchResponse.PartitionData
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH
import org.apache.kafka.common.requests.RequestUtils import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, RequestUtils}
import scala.jdk.CollectionConverters._ import java.util
import java.util.Optional
import scala.collection.{Map, Seq, Set, mutable} import scala.collection.{Map, Seq, Set, mutable}
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
class ReplicaAlterLogDirsThread(name: String, class ReplicaAlterLogDirsThread(name: String,
sourceBroker: BrokerEndPoint, sourceBroker: BrokerEndPoint,
@ -77,15 +73,21 @@ class ReplicaAlterLogDirsThread(name: String,
} }
def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = { def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = {
var partitionData: Seq[(TopicPartition, FetchResponse.PartitionData[Records])] = null var partitionData: Seq[(TopicPartition, FetchData)] = null
val request = fetchRequest.build() val request = fetchRequest.build()
def processResponseCallback(responsePartitionData: Seq[(TopicPartition, FetchPartitionData)]): Unit = { def processResponseCallback(responsePartitionData: Seq[(TopicPartition, FetchPartitionData)]): Unit = {
partitionData = responsePartitionData.map { case (tp, data) => partitionData = responsePartitionData.map { case (tp, data) =>
val abortedTransactions = data.abortedTransactions.map(_.asJava).orNull val abortedTransactions = data.abortedTransactions.map(_.asJava).orNull
val lastStableOffset = data.lastStableOffset.getOrElse(FetchResponse.INVALID_LAST_STABLE_OFFSET) val lastStableOffset = data.lastStableOffset.getOrElse(FetchResponse.INVALID_LAST_STABLE_OFFSET)
tp -> new FetchResponse.PartitionData(data.error, data.highWatermark, lastStableOffset, tp -> new FetchResponseData.PartitionData()
data.logStartOffset, abortedTransactions, data.records) .setPartitionIndex(tp.partition)
.setErrorCode(data.error.code)
.setHighWatermark(data.highWatermark)
.setLastStableOffset(lastStableOffset)
.setLogStartOffset(data.logStartOffset)
.setAbortedTransactions(abortedTransactions)
.setRecords(data.records)
} }
} }
@ -110,10 +112,10 @@ class ReplicaAlterLogDirsThread(name: String,
// process fetched data // process fetched data
override def processPartitionData(topicPartition: TopicPartition, override def processPartitionData(topicPartition: TopicPartition,
fetchOffset: Long, fetchOffset: Long,
partitionData: PartitionData[Records]): Option[LogAppendInfo] = { partitionData: FetchData): Option[LogAppendInfo] = {
val partition = replicaMgr.getPartitionOrException(topicPartition) val partition = replicaMgr.getPartitionOrException(topicPartition)
val futureLog = partition.futureLocalLogOrException val futureLog = partition.futureLocalLogOrException
val records = toMemoryRecords(partitionData.records) val records = toMemoryRecords(FetchResponse.recordsOrFail(partitionData))
if (fetchOffset != futureLog.logEndOffset) if (fetchOffset != futureLog.logEndOffset)
throw new IllegalStateException("Offset mismatch for the future replica %s: fetched offset = %d, log end offset = %d.".format( throw new IllegalStateException("Offset mismatch for the future replica %s: fetched offset = %d, log end offset = %d.".format(

View File

@ -35,7 +35,7 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetFor
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.utils.{LogContext, Time}
@ -162,7 +162,7 @@ class ReplicaFetcherThread(name: String,
val logTrace = isTraceEnabled val logTrace = isTraceEnabled
val partition = replicaMgr.getPartitionOrException(topicPartition) val partition = replicaMgr.getPartitionOrException(topicPartition)
val log = partition.localLogOrException val log = partition.localLogOrException
val records = toMemoryRecords(partitionData.records) val records = toMemoryRecords(FetchResponse.recordsOrFail(partitionData))
maybeWarnIfOversizedRecords(records, topicPartition) maybeWarnIfOversizedRecords(records, topicPartition)
@ -215,7 +215,7 @@ class ReplicaFetcherThread(name: String,
override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = { override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = {
try { try {
val clientResponse = leaderEndpoint.sendRequest(fetchRequest) val clientResponse = leaderEndpoint.sendRequest(fetchRequest)
val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse[Records]] val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
if (!fetchSessionHandler.handleResponse(fetchResponse)) { if (!fetchSessionHandler.handleResponse(fetchResponse)) {
Map.empty Map.empty
} else { } else {

View File

@ -57,7 +57,6 @@ import org.apache.kafka.common.replica.PartitionView.DefaultPartitionView
import org.apache.kafka.common.replica.ReplicaView.DefaultReplicaView import org.apache.kafka.common.replica.ReplicaView.DefaultReplicaView
import org.apache.kafka.common.replica.{ClientMetadata, _} import org.apache.kafka.common.replica.{ClientMetadata, _}
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
@ -150,7 +149,7 @@ case class FetchPartitionData(error: Errors = Errors.NONE,
records: Records, records: Records,
divergingEpoch: Option[FetchResponseData.EpochEndOffset], divergingEpoch: Option[FetchResponseData.EpochEndOffset],
lastStableOffset: Option[Long], lastStableOffset: Option[Long],
abortedTransactions: Option[List[AbortedTransaction]], abortedTransactions: Option[List[FetchResponseData.AbortedTransaction]],
preferredReadReplica: Option[Int], preferredReadReplica: Option[Int],
isReassignmentFetch: Boolean) isReassignmentFetch: Boolean)

View File

@ -17,6 +17,22 @@
package kafka.tools package kafka.tools
import joptsimple.OptionParser
import kafka.api._
import kafka.utils.{IncludeList, _}
import org.apache.kafka.clients._
import org.apache.kafka.clients.admin.{Admin, ListTopicsOptions, TopicDescription}
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{NetworkReceive, Selectable, Selector}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.AbstractRequest.Builder
import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetsRequest, FetchRequest => JFetchRequest}
import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{Node, TopicPartition}
import java.net.SocketTimeoutException import java.net.SocketTimeoutException
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util import java.util
@ -24,26 +40,8 @@ import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
import java.util.regex.{Pattern, PatternSyntaxException} import java.util.regex.{Pattern, PatternSyntaxException}
import java.util.{Date, Optional, Properties} import java.util.{Date, Optional, Properties}
import joptsimple.OptionParser
import kafka.api._
import kafka.utils.IncludeList
import kafka.utils._
import org.apache.kafka.clients._
import org.apache.kafka.clients.admin.{Admin, ListTopicsOptions, TopicDescription}
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.{NetworkReceive, Selectable, Selector}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.AbstractRequest.Builder
import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetsRequest, FetchRequest => JFetchRequest}
import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{Node, TopicPartition}
import scala.jdk.CollectionConverters._
import scala.collection.Seq import scala.collection.Seq
import scala.jdk.CollectionConverters._
/** /**
* For verifying the consistency among replicas. * For verifying the consistency among replicas.
@ -261,7 +259,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: collection.Map[To
expectedNumFetchers: Int, expectedNumFetchers: Int,
reportInterval: Long) extends Logging { reportInterval: Long) extends Logging {
private val fetchOffsetMap = new Pool[TopicPartition, Long] private val fetchOffsetMap = new Pool[TopicPartition, Long]
private val recordsCache = new Pool[TopicPartition, Pool[Int, FetchResponse.PartitionData[MemoryRecords]]] private val recordsCache = new Pool[TopicPartition, Pool[Int, FetchResponseData.PartitionData]]
private val fetcherBarrier = new AtomicReference(new CountDownLatch(expectedNumFetchers)) private val fetcherBarrier = new AtomicReference(new CountDownLatch(expectedNumFetchers))
private val verificationBarrier = new AtomicReference(new CountDownLatch(1)) private val verificationBarrier = new AtomicReference(new CountDownLatch(1))
@volatile private var lastReportTime = Time.SYSTEM.milliseconds @volatile private var lastReportTime = Time.SYSTEM.milliseconds
@ -284,7 +282,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: collection.Map[To
private def initialize(): Unit = { private def initialize(): Unit = {
for (topicPartition <- expectedReplicasPerTopicPartition.keySet) for (topicPartition <- expectedReplicasPerTopicPartition.keySet)
recordsCache.put(topicPartition, new Pool[Int, FetchResponse.PartitionData[MemoryRecords]]) recordsCache.put(topicPartition, new Pool[Int, FetchResponseData.PartitionData])
setInitialOffsets() setInitialOffsets()
} }
@ -294,7 +292,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: collection.Map[To
fetchOffsetMap.put(tp, offset) fetchOffsetMap.put(tp, offset)
} }
def addFetchedData(topicAndPartition: TopicPartition, replicaId: Int, partitionData: FetchResponse.PartitionData[MemoryRecords]): Unit = { def addFetchedData(topicAndPartition: TopicPartition, replicaId: Int, partitionData: FetchResponseData.PartitionData): Unit = {
recordsCache.get(topicAndPartition).put(replicaId, partitionData) recordsCache.get(topicAndPartition).put(replicaId, partitionData)
} }
@ -311,7 +309,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: collection.Map[To
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicPartition + ", but expected " "fetched " + fetchResponsePerReplica.size + " replicas for " + topicPartition + ", but expected "
+ expectedReplicasPerTopicPartition(topicPartition) + " replicas") + expectedReplicasPerTopicPartition(topicPartition) + " replicas")
val recordBatchIteratorMap = fetchResponsePerReplica.map { case (replicaId, fetchResponse) => val recordBatchIteratorMap = fetchResponsePerReplica.map { case (replicaId, fetchResponse) =>
replicaId -> fetchResponse.records.batches.iterator replicaId -> FetchResponse.recordsOrFail(fetchResponse).batches.iterator
} }
val maxHw = fetchResponsePerReplica.values.map(_.highWatermark).max val maxHw = fetchResponsePerReplica.values.map(_.highWatermark).max
@ -403,10 +401,10 @@ private class ReplicaFetcher(name: String, sourceBroker: Node, topicPartitions:
debug("Issuing fetch request ") debug("Issuing fetch request ")
var fetchResponse: FetchResponse[MemoryRecords] = null var fetchResponse: FetchResponse = null
try { try {
val clientResponse = fetchEndpoint.sendRequest(fetchRequestBuilder) val clientResponse = fetchEndpoint.sendRequest(fetchRequestBuilder)
fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse[MemoryRecords]] fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
} catch { } catch {
case t: Throwable => case t: Throwable =>
if (!isRunning) if (!isRunning)
@ -414,14 +412,13 @@ private class ReplicaFetcher(name: String, sourceBroker: Node, topicPartitions:
} }
if (fetchResponse != null) { if (fetchResponse != null) {
fetchResponse.responseData.forEach { (tp, partitionData) => fetchResponse.data.responses.forEach(topicResponse =>
replicaBuffer.addFetchedData(tp, sourceBroker.id, partitionData) topicResponse.partitions.forEach(partitionResponse =>
} replicaBuffer.addFetchedData(new TopicPartition(topicResponse.topic, partitionResponse.partitionIndex),
sourceBroker.id, partitionResponse)))
} else { } else {
val emptyResponse = new FetchResponse.PartitionData(Errors.NONE, FetchResponse.INVALID_HIGHWATERMARK,
FetchResponse.INVALID_LAST_STABLE_OFFSET, FetchResponse.INVALID_LOG_START_OFFSET, null, MemoryRecords.EMPTY)
for (topicAndPartition <- topicPartitions) for (topicAndPartition <- topicPartitions)
replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, emptyResponse) replicaBuffer.addFetchedData(topicAndPartition, sourceBroker.id, FetchResponse.partitionResponse(topicAndPartition.partition, Errors.NONE))
} }
fetcherBarrier.countDown() fetcherBarrier.countDown()

View File

@ -24,7 +24,6 @@ import kafka.utils.Logging
import org.apache.kafka.common.internals.FatalExitError import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, EndQuorumEpochResponseData, FetchResponseData, FetchSnapshotResponseData, VoteResponseData} import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, EndQuorumEpochResponseData, FetchResponseData, FetchSnapshotResponseData, VoteResponseData}
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage} import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage}
import org.apache.kafka.common.record.BaseRecords
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
@ -81,7 +80,7 @@ class TestRaftRequestHandler(
} }
private def handleFetch(request: RequestChannel.Request): Unit = { private def handleFetch(request: RequestChannel.Request): Unit = {
handle(request, response => new FetchResponse[BaseRecords](response.asInstanceOf[FetchResponseData])) handle(request, response => new FetchResponse(response.asInstanceOf[FetchResponseData]))
} }
private def handleFetchSnapshot(request: RequestChannel.Request): Unit = { private def handleFetchSnapshot(request: RequestChannel.Request): Unit = {

View File

@ -49,7 +49,7 @@ import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadata
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData} import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, Records, SimpleRecord} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
import org.apache.kafka.common.resource.ResourceType._ import org.apache.kafka.common.resource.ResourceType._
@ -159,7 +159,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
val requestKeyToError = (topicNames: Map[Uuid, String]) => Map[ApiKeys, Nothing => Errors]( val requestKeyToError = (topicNames: Map[Uuid, String]) => Map[ApiKeys, Nothing => Errors](
ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2), ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors.asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2),
ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses.asScala.find(_._1 == tp).get._2.error), ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses.asScala.find(_._1 == tp).get._2.error),
ApiKeys.FETCH -> ((resp: requests.FetchResponse[Records]) => resp.responseData.asScala.find(_._1 == tp).get._2.error), ApiKeys.FETCH -> ((resp: requests.FetchResponse) => Errors.forCode(resp.responseData.asScala.find(_._1 == tp).get._2.errorCode)),
ApiKeys.LIST_OFFSETS -> ((resp: ListOffsetsResponse) => { ApiKeys.LIST_OFFSETS -> ((resp: ListOffsetsResponse) => {
Errors.forCode( Errors.forCode(
resp.data resp.data
@ -169,12 +169,12 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
) )
}), }),
ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode( ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode(
resp.data().topics().get(0).partitions().get(0).errorCode())), resp.data.topics().get(0).partitions().get(0).errorCode)),
ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.error), ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.error),
ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => resp.error), ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => resp.error),
ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error), ApiKeys.UPDATE_METADATA -> ((resp: requests.UpdateMetadataResponse) => resp.error),
ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error), ApiKeys.JOIN_GROUP -> ((resp: JoinGroupResponse) => resp.error),
ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => Errors.forCode(resp.data.errorCode())), ApiKeys.SYNC_GROUP -> ((resp: SyncGroupResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.DESCRIBE_GROUPS -> ((resp: DescribeGroupsResponse) => { ApiKeys.DESCRIBE_GROUPS -> ((resp: DescribeGroupsResponse) => {
Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode) Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)
}), }),
@ -187,8 +187,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => Errors.forCode( ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => Errors.forCode(
resp.partitionErrors.asScala.find(pe => pe.topicName == tp.topic && pe.partitionIndex == tp.partition).get.errorCode)), resp.partitionErrors.asScala.find(pe => pe.topicName == tp.topic && pe.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error), ApiKeys.CONTROLLED_SHUTDOWN -> ((resp: requests.ControlledShutdownResponse) => resp.error),
ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => Errors.forCode(resp.data.topics.find(topic).errorCode())), ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => Errors.forCode(resp.data.topics.find(topic).errorCode)),
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(topic).errorCode())), ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => Errors.forCode(resp.data.responses.find(topic).errorCode)),
ApiKeys.DELETE_RECORDS -> ((resp: requests.DeleteRecordsResponse) => Errors.forCode( ApiKeys.DELETE_RECORDS -> ((resp: requests.DeleteRecordsResponse) => Errors.forCode(
resp.data.topics.find(tp.topic).partitions.find(tp.partition).errorCode)), resp.data.topics.find(tp.topic).partitions.find(tp.partition).errorCode)),
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => Errors.forCode( ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => Errors.forCode(
@ -211,17 +211,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
.find(p => p.partitionIndex == tp.partition).get.errorCode)), .find(p => p.partitionIndex == tp.partition).get.errorCode)),
ApiKeys.DESCRIBE_LOG_DIRS -> ((resp: DescribeLogDirsResponse) => ApiKeys.DESCRIBE_LOG_DIRS -> ((resp: DescribeLogDirsResponse) =>
if (resp.data.results.size() > 0) Errors.forCode(resp.data.results.get(0).errorCode) else Errors.CLUSTER_AUTHORIZATION_FAILED), if (resp.data.results.size() > 0) Errors.forCode(resp.data.results.get(0).errorCode) else Errors.CLUSTER_AUTHORIZATION_FAILED),
ApiKeys.CREATE_PARTITIONS -> ((resp: CreatePartitionsResponse) => Errors.forCode(resp.data.results.asScala.head.errorCode())), ApiKeys.CREATE_PARTITIONS -> ((resp: CreatePartitionsResponse) => Errors.forCode(resp.data.results.asScala.head.errorCode)),
ApiKeys.ELECT_LEADERS -> ((resp: ElectLeadersResponse) => Errors.forCode(resp.data().errorCode())), ApiKeys.ELECT_LEADERS -> ((resp: ElectLeadersResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.INCREMENTAL_ALTER_CONFIGS -> ((resp: IncrementalAlterConfigsResponse) => { ApiKeys.INCREMENTAL_ALTER_CONFIGS -> ((resp: IncrementalAlterConfigsResponse) => {
val topicResourceError = IncrementalAlterConfigsResponse.fromResponseData(resp.data()).get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)) val topicResourceError = IncrementalAlterConfigsResponse.fromResponseData(resp.data).get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic))
if (topicResourceError == null) if (topicResourceError == null)
IncrementalAlterConfigsResponse.fromResponseData(resp.data()).get(new ConfigResource(ConfigResource.Type.BROKER_LOGGER, brokerId.toString)).error IncrementalAlterConfigsResponse.fromResponseData(resp.data).get(new ConfigResource(ConfigResource.Type.BROKER_LOGGER, brokerId.toString)).error
else else
topicResourceError.error() topicResourceError.error()
}), }),
ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> ((resp: AlterPartitionReassignmentsResponse) => Errors.forCode(resp.data().errorCode())), ApiKeys.ALTER_PARTITION_REASSIGNMENTS -> ((resp: AlterPartitionReassignmentsResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.LIST_PARTITION_REASSIGNMENTS -> ((resp: ListPartitionReassignmentsResponse) => Errors.forCode(resp.data().errorCode())), ApiKeys.LIST_PARTITION_REASSIGNMENTS -> ((resp: ListPartitionReassignmentsResponse) => Errors.forCode(resp.data.errorCode)),
ApiKeys.OFFSET_DELETE -> ((resp: OffsetDeleteResponse) => { ApiKeys.OFFSET_DELETE -> ((resp: OffsetDeleteResponse) => {
Errors.forCode( Errors.forCode(
resp.data resp.data
@ -326,9 +326,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
requests.ProduceRequest.forCurrentMagic(new ProduceRequestData() requests.ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic()).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition()) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
@ -363,9 +363,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
private def offsetsForLeaderEpochRequest: OffsetsForLeaderEpochRequest = { private def offsetsForLeaderEpochRequest: OffsetsForLeaderEpochRequest = {
val epochs = new OffsetForLeaderTopicCollection() val epochs = new OffsetForLeaderTopicCollection()
epochs.add(new OffsetForLeaderTopic() epochs.add(new OffsetForLeaderTopic()
.setTopic(tp.topic()) .setTopic(tp.topic)
.setPartitions(List(new OffsetForLeaderPartition() .setPartitions(List(new OffsetForLeaderPartition()
.setPartition(tp.partition()) .setPartition(tp.partition)
.setLeaderEpoch(7) .setLeaderEpoch(7)
.setCurrentLeaderEpoch(27)).asJava)) .setCurrentLeaderEpoch(27)).asJava))
OffsetsForLeaderEpochRequest.Builder.forConsumer(epochs).build() OffsetsForLeaderEpochRequest.Builder.forConsumer(epochs).build()
@ -509,9 +509,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
private def stopReplicaRequest: StopReplicaRequest = { private def stopReplicaRequest: StopReplicaRequest = {
val topicStates = Seq( val topicStates = Seq(
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(tp.topic()) .setTopicName(tp.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp.partition()) .setPartitionIndex(tp.partition)
.setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.initialLeaderEpoch + 2)
.setDeletePartition(true)).asJava) .setDeletePartition(true)).asJava)
).asJava ).asJava
@ -658,7 +658,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
List(new AlterPartitionReassignmentsRequestData.ReassignableTopic() List(new AlterPartitionReassignmentsRequestData.ReassignableTopic()
.setName(topic) .setName(topic)
.setPartitions( .setPartitions(
List(new AlterPartitionReassignmentsRequestData.ReassignablePartition().setPartitionIndex(tp.partition())).asJava List(new AlterPartitionReassignmentsRequestData.ReassignablePartition().setPartitionIndex(tp.partition)).asJava
)).asJava )).asJava
) )
).build() ).build()
@ -1625,7 +1625,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
@Test @Test
def testUnauthorizedCreatePartitions(): Unit = { def testUnauthorizedCreatePartitions(): Unit = {
val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest)
assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code(), createPartitionsResponse.data.results.asScala.head.errorCode()) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, createPartitionsResponse.data.results.asScala.head.errorCode)
} }
@Test @Test
@ -1633,7 +1633,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
createTopic(topic) createTopic(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, ALTER, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL))
val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest)
assertEquals(Errors.NONE.code(), createPartitionsResponse.data.results.asScala.head.errorCode()) assertEquals(Errors.NONE.code, createPartitionsResponse.data.results.asScala.head.errorCode)
} }
@Test @Test
@ -2133,7 +2133,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
numRecords: Int, numRecords: Int,
tp: TopicPartition): Unit = { tp: TopicPartition): Unit = {
val futures = (0 until numRecords).map { i => val futures = (0 until numRecords).map { i =>
producer.send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes)) producer.send(new ProducerRecord(tp.topic, tp.partition, i.toString.getBytes, i.toString.getBytes))
} }
try { try {
futures.foreach(_.get) futures.foreach(_.get)

View File

@ -18,9 +18,8 @@
package kafka.tools package kafka.tools
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.FetchResponse
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Assertions.assertTrue
@ -44,7 +43,12 @@ class ReplicaVerificationToolTest {
} }
val initialOffset = 4 val initialOffset = 4
val memoryRecords = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, records: _*) val memoryRecords = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, records: _*)
val partitionData = new FetchResponse.PartitionData(Errors.NONE, 20, 20, 0L, null, memoryRecords) val partitionData = new FetchResponseData.PartitionData()
.setPartitionIndex(tp.partition)
.setHighWatermark(20)
.setLastStableOffset(20)
.setLogStartOffset(0)
.setRecords(memoryRecords)
replicaBuffer.addFetchedData(tp, replicaId, partitionData) replicaBuffer.addFetchedData(tp, replicaId, partitionData)
} }

View File

@ -34,11 +34,11 @@ import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchI
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid}
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.record.FileRecords.TimestampAndOffset import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter
import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils}
import org.easymock.EasyMock import org.easymock.EasyMock
@ -4725,7 +4725,8 @@ class LogTest {
assertEquals(1, fetchDataInfo.abortedTransactions.size) assertEquals(1, fetchDataInfo.abortedTransactions.size)
assertTrue(fetchDataInfo.abortedTransactions.isDefined) assertTrue(fetchDataInfo.abortedTransactions.isDefined)
assertEquals(new AbortedTransaction(pid, 0), fetchDataInfo.abortedTransactions.get.head) assertEquals(new FetchResponseData.AbortedTransaction().setProducerId(pid).setFirstOffset(0),
fetchDataInfo.abortedTransactions.get.head)
} }
@Test @Test

View File

@ -17,7 +17,7 @@
package kafka.log package kafka.log
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction import org.apache.kafka.common.message.FetchResponseData
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
@ -136,7 +136,7 @@ class TransactionIndexTest {
assertEquals(abortedTransactions.take(3), index.collectAbortedTxns(0L, 100L).abortedTransactions) assertEquals(abortedTransactions.take(3), index.collectAbortedTxns(0L, 100L).abortedTransactions)
index.reset() index.reset()
assertEquals(List.empty[AbortedTransaction], index.collectAbortedTxns(0L, 100L).abortedTransactions) assertEquals(List.empty[FetchResponseData.AbortedTransaction], index.collectAbortedTxns(0L, 100L).abortedTransactions)
} }
@Test @Test

View File

@ -20,7 +20,6 @@ package kafka.server
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.Optional import java.util.Optional
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.log.LogAppendInfo import kafka.log.LogAppendInfo
import kafka.message.NoCompressionCodec import kafka.message.NoCompressionCodec
@ -37,7 +36,7 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.{BeforeEach, Test}
@ -907,8 +906,8 @@ class AbstractFetcherThreadTest {
partitionData: FetchData): Option[LogAppendInfo] = { partitionData: FetchData): Option[LogAppendInfo] = {
val state = replicaPartitionState(topicPartition) val state = replicaPartitionState(topicPartition)
if (isTruncationOnFetchSupported && partitionData.divergingEpoch.isPresent) { if (isTruncationOnFetchSupported && FetchResponse.isDivergingEpoch(partitionData)) {
val divergingEpoch = partitionData.divergingEpoch.get val divergingEpoch = partitionData.divergingEpoch
truncateOnFetchResponse(Map(topicPartition -> new EpochEndOffset() truncateOnFetchResponse(Map(topicPartition -> new EpochEndOffset()
.setPartition(topicPartition.partition) .setPartition(topicPartition.partition)
.setErrorCode(Errors.NONE.code) .setErrorCode(Errors.NONE.code)
@ -923,7 +922,7 @@ class AbstractFetcherThreadTest {
s"fetched offset = $fetchOffset, log end offset = ${state.logEndOffset}.") s"fetched offset = $fetchOffset, log end offset = ${state.logEndOffset}.")
// Now check message's crc // Now check message's crc
val batches = partitionData.records.batches.asScala val batches = FetchResponse.recordsOrFail(partitionData).batches.asScala
var maxTimestamp = RecordBatch.NO_TIMESTAMP var maxTimestamp = RecordBatch.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L var offsetOfMaxTimestamp = -1L
var lastOffset = state.logEndOffset var lastOffset = state.logEndOffset
@ -955,7 +954,7 @@ class AbstractFetcherThreadTest {
sourceCodec = NoCompressionCodec, sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec, targetCodec = NoCompressionCodec,
shallowCount = batches.size, shallowCount = batches.size,
validBytes = partitionData.records.sizeInBytes, validBytes = FetchResponse.recordsSize(partitionData),
offsetsMonotonic = true, offsetsMonotonic = true,
lastOffsetOfFirstBatch = batches.headOption.map(_.lastOffset).getOrElse(-1))) lastOffsetOfFirstBatch = batches.headOption.map(_.lastOffset).getOrElse(-1)))
} }
@ -1143,9 +1142,16 @@ class AbstractFetcherThreadTest {
(Errors.NONE, records) (Errors.NONE, records)
} }
val partitionData = new FetchData()
.setPartitionIndex(partition.partition)
.setErrorCode(error.code)
.setHighWatermark(leaderState.highWatermark)
.setLastStableOffset(leaderState.highWatermark)
.setLogStartOffset(leaderState.logStartOffset)
.setRecords(records)
divergingEpoch.foreach(partitionData.setDivergingEpoch)
(partition, new FetchData(error, leaderState.highWatermark, leaderState.highWatermark, leaderState.logStartOffset, (partition, partitionData)
Optional.empty[Integer], List.empty.asJava, divergingEpoch.asJava, records))
}.toMap }.toMap
} }

View File

@ -23,7 +23,6 @@ import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.common.serialization.StringSerializer
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -79,8 +78,8 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
partitionMap partitionMap
} }
private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse[MemoryRecords] = { private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = {
connectAndReceive[FetchResponse[MemoryRecords]](request, destination = brokerSocketServer(leaderId)) connectAndReceive[FetchResponse](request, destination = brokerSocketServer(leaderId))
} }
/** /**
@ -90,11 +89,11 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
def testV1FetchWithDownConversionDisabled(): Unit = { def testV1FetchWithDownConversionDisabled(): Unit = {
val topicMap = createTopics(numTopics = 5, numPartitions = 1) val topicMap = createTopics(numTopics = 5, numPartitions = 1)
val topicPartitions = topicMap.keySet.toSeq val topicPartitions = topicMap.keySet.toSeq
topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic, "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024,
topicPartitions)).build(1) topicPartitions)).build(1)
val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest) val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest)
topicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, fetchResponse.responseData().get(tp).error)) topicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, Errors.forCode(fetchResponse.responseData.get(tp).errorCode)))
} }
/** /**
@ -104,11 +103,11 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
def testLatestFetchWithDownConversionDisabled(): Unit = { def testLatestFetchWithDownConversionDisabled(): Unit = {
val topicMap = createTopics(numTopics = 5, numPartitions = 1) val topicMap = createTopics(numTopics = 5, numPartitions = 1)
val topicPartitions = topicMap.keySet.toSeq val topicPartitions = topicMap.keySet.toSeq
topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) topicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic, "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024,
topicPartitions)).build() topicPartitions)).build()
val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest) val fetchResponse = sendFetchRequest(topicMap.head._2, fetchRequest)
topicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) topicPartitions.foreach(tp => assertEquals(Errors.NONE, Errors.forCode(fetchResponse.responseData.get(tp).errorCode)))
} }
/** /**
@ -129,13 +128,13 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
val allTopics = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions val allTopics = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions
val leaderId = conversionDisabledTopicsMap.head._2 val leaderId = conversionDisabledTopicsMap.head._2
allTopics.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) allTopics.foreach(tp => producer.send(new ProducerRecord(tp.topic, "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024, val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(1024,
allTopics)).build(1) allTopics)).build(1)
val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
conversionDisabledTopicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION, fetchResponse.responseData().get(tp).error)) conversionDisabledTopicPartitions.foreach(tp => assertEquals(Errors.UNSUPPORTED_VERSION.code, fetchResponse.responseData.get(tp).errorCode))
conversionEnabledTopicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) conversionEnabledTopicPartitions.foreach(tp => assertEquals(Errors.NONE.code, fetchResponse.responseData.get(tp).errorCode))
} }
/** /**
@ -155,11 +154,11 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
val allTopicPartitions = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions val allTopicPartitions = conversionDisabledTopicPartitions ++ conversionEnabledTopicPartitions
val leaderId = conversionDisabledTopicsMap.head._2 val leaderId = conversionDisabledTopicsMap.head._2
allTopicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic(), "key", "value")).get()) allTopicPartitions.foreach(tp => producer.send(new ProducerRecord(tp.topic, "key", "value")).get())
val fetchRequest = FetchRequest.Builder.forReplica(1, 1, Int.MaxValue, 0, val fetchRequest = FetchRequest.Builder.forReplica(1, 1, Int.MaxValue, 0,
createPartitionMap(1024, allTopicPartitions)).build() createPartitionMap(1024, allTopicPartitions)).build()
val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
allTopicPartitions.foreach(tp => assertEquals(Errors.NONE, fetchResponse.responseData().get(tp).error)) allTopicPartitions.foreach(tp => assertEquals(Errors.NONE.code, fetchResponse.responseData.get(tp).errorCode))
} }
} }

View File

@ -17,17 +17,16 @@
package kafka.server package kafka.server
import java.util.{Optional, Properties}
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import java.util.{Optional, Properties}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
/** /**
@ -92,8 +91,8 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
}) })
} }
private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse[MemoryRecords] = { private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = {
connectAndReceive[FetchResponse[MemoryRecords]](request, destination = brokerSocketServer(leaderId)) connectAndReceive[FetchResponse](request, destination = brokerSocketServer(leaderId))
} }
/** /**
@ -117,7 +116,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
FetchRequest.Builder.forConsumer(Int.MaxValue, 0, FetchRequest.Builder.forConsumer(Int.MaxValue, 0,
Map(testTopicPartition -> Map(testTopicPartition ->
new PartitionData(fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(3)) new PartitionData(fetchOffset, 0, Integer.MAX_VALUE, Optional.empty())).asJava).build(3))
val records = response.responseData().get(testTopicPartition).records.records() val records = FetchResponse.recordsOrFail(response.responseData.get(testTopicPartition)).records()
assertNotNull(records) assertNotNull(records)
val recordsList = records.asScala.toList val recordsList = records.asScala.toList
assertEquals(expected.size, recordsList.size) assertEquals(expected.size, recordsList.size)

View File

@ -16,24 +16,25 @@
*/ */
package kafka.server package kafka.server
import java.io.DataInputStream
import java.util
import java.util.{Optional, Properties}
import kafka.api.KAFKA_0_11_0_IV2 import kafka.api.KAFKA_0_11_0_IV2
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.message.{GZIPCompressionCodec, ProducerCompressionCodec, ZStdCompressionCodec} import kafka.message.{GZIPCompressionCodec, ProducerCompressionCodec, ZStdCompressionCodec}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{MemoryRecords, Record, RecordBatch} import org.apache.kafka.common.record.{Record, RecordBatch}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata}
import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer} import org.apache.kafka.common.serialization.{ByteArraySerializer, StringSerializer}
import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.apache.kafka.common.{IsolationLevel, TopicPartition}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.{AfterEach, Test}
import scala.jdk.CollectionConverters._ import java.io.DataInputStream
import java.util
import java.util.{Optional, Properties}
import scala.collection.Seq import scala.collection.Seq
import scala.jdk.CollectionConverters._
import scala.util.Random import scala.util.Random
/** /**
@ -70,8 +71,8 @@ class FetchRequestTest extends BaseRequestTest {
partitionMap partitionMap
} }
private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse[MemoryRecords] = { private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = {
connectAndReceive[FetchResponse[MemoryRecords]](request, destination = brokerSocketServer(leaderId)) connectAndReceive[FetchResponse](request, destination = brokerSocketServer(leaderId))
} }
private def initProducer(): Unit = { private def initProducer(): Unit = {
@ -133,12 +134,12 @@ class FetchRequestTest extends BaseRequestTest {
}.sum }.sum
assertTrue(responseSize3 <= maxResponseBytes) assertTrue(responseSize3 <= maxResponseBytes)
val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1) val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
assertEquals(Errors.NONE, partitionData3.error) assertEquals(Errors.NONE.code, partitionData3.errorCode)
assertTrue(partitionData3.highWatermark > 0) assertTrue(partitionData3.highWatermark > 0)
val size3 = records(partitionData3).map(_.sizeInBytes).sum val size3 = records(partitionData3).map(_.sizeInBytes).sum
assertTrue(size3 <= maxResponseBytes, s"Expected $size3 to be smaller than $maxResponseBytes") assertTrue(size3 <= maxResponseBytes, s"Expected $size3 to be smaller than $maxResponseBytes")
assertTrue(size3 > maxPartitionBytes, s"Expected $size3 to be larger than $maxPartitionBytes") assertTrue(size3 > maxPartitionBytes, s"Expected $size3 to be larger than $maxPartitionBytes")
assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes) assertTrue(maxPartitionBytes < FetchResponse.recordsSize(partitionData3))
// 4. Partition with message larger than the response limit at the start of the list // 4. Partition with message larger than the response limit at the start of the list
val shuffledTopicPartitions4 = Seq(partitionWithLargeMessage2, partitionWithLargeMessage1) ++ val shuffledTopicPartitions4 = Seq(partitionWithLargeMessage2, partitionWithLargeMessage1) ++
@ -151,11 +152,11 @@ class FetchRequestTest extends BaseRequestTest {
} }
assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4) assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4)
val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2) val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
assertEquals(Errors.NONE, partitionData4.error) assertEquals(Errors.NONE.code, partitionData4.errorCode)
assertTrue(partitionData4.highWatermark > 0) assertTrue(partitionData4.highWatermark > 0)
val size4 = records(partitionData4).map(_.sizeInBytes).sum val size4 = records(partitionData4).map(_.sizeInBytes).sum
assertTrue(size4 > maxResponseBytes, s"Expected $size4 to be larger than $maxResponseBytes") assertTrue(size4 > maxResponseBytes, s"Expected $size4 to be larger than $maxResponseBytes")
assertTrue(maxResponseBytes < partitionData4.records.sizeInBytes) assertTrue(maxResponseBytes < FetchResponse.recordsSize(partitionData4))
} }
@Test @Test
@ -169,9 +170,9 @@ class FetchRequestTest extends BaseRequestTest {
Seq(topicPartition))).build(2) Seq(topicPartition))).build(2)
val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0) assertTrue(partitionData.highWatermark > 0)
assertEquals(maxPartitionBytes, partitionData.records.sizeInBytes) assertEquals(maxPartitionBytes, FetchResponse.recordsSize(partitionData))
assertEquals(0, records(partitionData).map(_.sizeInBytes).sum) assertEquals(0, records(partitionData).map(_.sizeInBytes).sum)
} }
@ -186,7 +187,7 @@ class FetchRequestTest extends BaseRequestTest {
Seq(topicPartition))).isolationLevel(IsolationLevel.READ_COMMITTED).build(4) Seq(topicPartition))).isolationLevel(IsolationLevel.READ_COMMITTED).build(4)
val fetchResponse = sendFetchRequest(leaderId, fetchRequest) val fetchResponse = sendFetchRequest(leaderId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.lastStableOffset > 0) assertTrue(partitionData.lastStableOffset > 0)
assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0) assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0)
} }
@ -209,7 +210,7 @@ class FetchRequestTest extends BaseRequestTest {
Seq(topicPartition))).build() Seq(topicPartition))).build()
val fetchResponse = sendFetchRequest(nonReplicaId, fetchRequest) val fetchResponse = sendFetchRequest(nonReplicaId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, partitionData.error) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode)
} }
@Test @Test
@ -243,11 +244,11 @@ class FetchRequestTest extends BaseRequestTest {
// Validate the expected truncation // Validate the expected truncation
val fetchResponse = sendFetchRequest(secondLeaderId, fetchRequest) val fetchResponse = sendFetchRequest(secondLeaderId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertEquals(0L, partitionData.records.sizeInBytes()) assertEquals(0L, FetchResponse.recordsSize(partitionData))
assertTrue(partitionData.divergingEpoch.isPresent) assertTrue(FetchResponse.isDivergingEpoch(partitionData))
val divergingEpoch = partitionData.divergingEpoch.get() val divergingEpoch = partitionData.divergingEpoch
assertEquals(firstLeaderEpoch, divergingEpoch.epoch) assertEquals(firstLeaderEpoch, divergingEpoch.epoch)
assertEquals(firstEpochEndOffset, divergingEpoch.endOffset) assertEquals(firstEpochEndOffset, divergingEpoch.endOffset)
} }
@ -265,7 +266,7 @@ class FetchRequestTest extends BaseRequestTest {
val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, partitionMap).build() val fetchRequest = FetchRequest.Builder.forConsumer(0, 1, partitionMap).build()
val fetchResponse = sendFetchRequest(brokerId, fetchRequest) val fetchResponse = sendFetchRequest(brokerId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(error, partitionData.error) assertEquals(error.code, partitionData.errorCode)
} }
// We need a leader change in order to check epoch fencing since the first epoch is 0 and // We need a leader change in order to check epoch fencing since the first epoch is 0 and
@ -329,7 +330,7 @@ class FetchRequestTest extends BaseRequestTest {
.build() .build()
val fetchResponse = sendFetchRequest(destinationBrokerId, fetchRequest) val fetchResponse = sendFetchRequest(destinationBrokerId, fetchRequest)
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(expectedError, partitionData.error) assertEquals(expectedError.code, partitionData.errorCode)
} }
// We only check errors because we do not expect the partition in the response otherwise // We only check errors because we do not expect the partition in the response otherwise
@ -366,7 +367,7 @@ class FetchRequestTest extends BaseRequestTest {
// batch is not complete, but sent when the producer is closed // batch is not complete, but sent when the producer is closed
futures.foreach(_.get) futures.foreach(_.get)
def fetch(version: Short, maxPartitionBytes: Int, closeAfterPartialResponse: Boolean): Option[FetchResponse[MemoryRecords]] = { def fetch(version: Short, maxPartitionBytes: Int, closeAfterPartialResponse: Boolean): Option[FetchResponse] = {
val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, val fetchRequest = FetchRequest.Builder.forConsumer(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes,
Seq(topicPartition))).build(version) Seq(topicPartition))).build(version)
@ -383,7 +384,7 @@ class FetchRequestTest extends BaseRequestTest {
s"Fetch size too small $size, broker may have run out of memory") s"Fetch size too small $size, broker may have run out of memory")
None None
} else { } else {
Some(receive[FetchResponse[MemoryRecords]](socket, ApiKeys.FETCH, version)) Some(receive[FetchResponse](socket, ApiKeys.FETCH, version))
} }
} finally { } finally {
socket.close() socket.close()
@ -396,8 +397,8 @@ class FetchRequestTest extends BaseRequestTest {
val response = fetch(version, maxPartitionBytes = batchSize, closeAfterPartialResponse = false) val response = fetch(version, maxPartitionBytes = batchSize, closeAfterPartialResponse = false)
val fetchResponse = response.getOrElse(throw new IllegalStateException("No fetch response")) val fetchResponse = response.getOrElse(throw new IllegalStateException("No fetch response"))
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
val batches = partitionData.records.batches.asScala.toBuffer val batches = FetchResponse.recordsOrFail(partitionData).batches.asScala.toBuffer
assertEquals(3, batches.size) // size is 3 (not 4) since maxPartitionBytes=msgValueSize*4, excluding key and headers assertEquals(3, batches.size) // size is 3 (not 4) since maxPartitionBytes=msgValueSize*4, excluding key and headers
} }
@ -442,9 +443,9 @@ class FetchRequestTest extends BaseRequestTest {
// validate response // validate response
val partitionData = fetchResponse.responseData.get(topicPartition) val partitionData = fetchResponse.responseData.get(topicPartition)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0) assertTrue(partitionData.highWatermark > 0)
val batches = partitionData.records.batches.asScala.toBuffer val batches = FetchResponse.recordsOrFail(partitionData).batches.asScala.toBuffer
val batch = batches.head val batch = batches.head
assertEquals(expectedMagic, batch.magic) assertEquals(expectedMagic, batch.magic)
assertEquals(currentExpectedOffset, batch.baseOffset) assertEquals(currentExpectedOffset, batch.baseOffset)
@ -504,35 +505,34 @@ class FetchRequestTest extends BaseRequestTest {
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() > 0, "Expected the broker to create a new incremental fetch session") assertTrue(resp1.sessionId() > 0, "Expected the broker to create a new incremental fetch session")
debug(s"Test created an incremental fetch session ${resp1.sessionId}") debug(s"Test created an incremental fetch session ${resp1.sessionId}")
assertTrue(resp1.responseData().containsKey(foo0)) assertTrue(resp1.responseData.containsKey(foo0))
assertTrue(resp1.responseData().containsKey(foo1)) assertTrue(resp1.responseData.containsKey(foo1))
assertTrue(resp1.responseData().containsKey(bar0)) assertTrue(resp1.responseData.containsKey(bar0))
assertEquals(Errors.NONE, resp1.responseData().get(foo0).error) assertEquals(Errors.NONE.code, resp1.responseData.get(foo0).errorCode)
assertEquals(Errors.NONE, resp1.responseData().get(foo1).error) assertEquals(Errors.NONE.code, resp1.responseData.get(foo1).errorCode)
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, resp1.responseData().get(bar0).error) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, resp1.responseData.get(bar0).errorCode)
val req2 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 1), Nil) val req2 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 1), Nil)
val resp2 = sendFetchRequest(0, req2) val resp2 = sendFetchRequest(0, req2)
assertEquals(Errors.NONE, resp2.error()) assertEquals(Errors.NONE, resp2.error())
assertEquals(resp1.sessionId(), assertEquals(resp1.sessionId(), resp2.sessionId(), "Expected the broker to continue the incremental fetch session")
resp2.sessionId(), "Expected the broker to continue the incremental fetch session")
assertFalse(resp2.responseData().containsKey(foo0)) assertFalse(resp2.responseData().containsKey(foo0))
assertFalse(resp2.responseData().containsKey(foo1)) assertFalse(resp2.responseData().containsKey(foo1))
assertTrue(resp2.responseData().containsKey(bar0)) assertTrue(resp2.responseData().containsKey(bar0))
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, resp2.responseData().get(bar0).error) assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), resp2.responseData().get(bar0).errorCode())
createTopic("bar", Map(0 -> List(0, 1))) createTopic("bar", Map(0 -> List(0, 1)))
val req3 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 2), Nil) val req3 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 2), Nil)
val resp3 = sendFetchRequest(0, req3) val resp3 = sendFetchRequest(0, req3)
assertEquals(Errors.NONE, resp3.error()) assertEquals(Errors.NONE, resp3.error())
assertFalse(resp3.responseData().containsKey(foo0)) assertFalse(resp3.responseData.containsKey(foo0))
assertFalse(resp3.responseData().containsKey(foo1)) assertFalse(resp3.responseData.containsKey(foo1))
assertTrue(resp3.responseData().containsKey(bar0)) assertTrue(resp3.responseData.containsKey(bar0))
assertEquals(Errors.NONE, resp3.responseData().get(bar0).error) assertEquals(Errors.NONE.code, resp3.responseData.get(bar0).errorCode)
val req4 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 3), Nil) val req4 = createFetchRequest(Nil, new JFetchMetadata(resp1.sessionId(), 3), Nil)
val resp4 = sendFetchRequest(0, req4) val resp4 = sendFetchRequest(0, req4)
assertEquals(Errors.NONE, resp4.error()) assertEquals(Errors.NONE, resp4.error())
assertFalse(resp4.responseData().containsKey(foo0)) assertFalse(resp4.responseData.containsKey(foo0))
assertFalse(resp4.responseData().containsKey(foo1)) assertFalse(resp4.responseData.containsKey(foo1))
assertFalse(resp4.responseData().containsKey(bar0)) assertFalse(resp4.responseData.containsKey(bar0))
} }
@Test @Test
@ -560,7 +560,7 @@ class FetchRequestTest extends BaseRequestTest {
val res0 = sendFetchRequest(leaderId, req0) val res0 = sendFetchRequest(leaderId, req0)
val data0 = res0.responseData.get(topicPartition) val data0 = res0.responseData.get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, data0.error) assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data0.errorCode)
// fetch request with version 10: works fine! // fetch request with version 10: works fine!
val req1= new FetchRequest.Builder(0, 10, -1, Int.MaxValue, 0, val req1= new FetchRequest.Builder(0, 10, -1, Int.MaxValue, 0,
@ -568,14 +568,14 @@ class FetchRequestTest extends BaseRequestTest {
.setMaxBytes(800).build() .setMaxBytes(800).build()
val res1 = sendFetchRequest(leaderId, req1) val res1 = sendFetchRequest(leaderId, req1)
val data1 = res1.responseData.get(topicPartition) val data1 = res1.responseData.get(topicPartition)
assertEquals(Errors.NONE, data1.error) assertEquals(Errors.NONE.code, data1.errorCode)
assertEquals(3, records(data1).size) assertEquals(3, records(data1).size)
} }
@Test @Test
def testPartitionDataEquals(): Unit = { def testPartitionDataEquals(): Unit = {
assertEquals(new FetchRequest.PartitionData(300, 0L, 300, Optional.of(300)), assertEquals(new FetchRequest.PartitionData(300, 0L, 300, Optional.of(300)),
new FetchRequest.PartitionData(300, 0L, 300, Optional.of(300))); new FetchRequest.PartitionData(300, 0L, 300, Optional.of(300)))
} }
@Test @Test
@ -614,7 +614,7 @@ class FetchRequestTest extends BaseRequestTest {
val res0 = sendFetchRequest(leaderId, req0) val res0 = sendFetchRequest(leaderId, req0)
val data0 = res0.responseData.get(topicPartition) val data0 = res0.responseData.get(topicPartition)
assertEquals(Errors.NONE, data0.error) assertEquals(Errors.NONE.code, data0.errorCode)
assertEquals(1, records(data0).size) assertEquals(1, records(data0).size)
val req1 = new FetchRequest.Builder(0, 1, -1, Int.MaxValue, 0, val req1 = new FetchRequest.Builder(0, 1, -1, Int.MaxValue, 0,
@ -623,7 +623,7 @@ class FetchRequestTest extends BaseRequestTest {
val res1 = sendFetchRequest(leaderId, req1) val res1 = sendFetchRequest(leaderId, req1)
val data1 = res1.responseData.get(topicPartition) val data1 = res1.responseData.get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, data1.error) assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data1.errorCode)
// fetch request with fetch version v3 (magic 1): // fetch request with fetch version v3 (magic 1):
// gzip compressed record is returned with down-conversion. // gzip compressed record is returned with down-conversion.
@ -634,7 +634,7 @@ class FetchRequestTest extends BaseRequestTest {
val res2 = sendFetchRequest(leaderId, req2) val res2 = sendFetchRequest(leaderId, req2)
val data2 = res2.responseData.get(topicPartition) val data2 = res2.responseData.get(topicPartition)
assertEquals(Errors.NONE, data2.error) assertEquals(Errors.NONE.code, data2.errorCode)
assertEquals(1, records(data2).size) assertEquals(1, records(data2).size)
val req3 = new FetchRequest.Builder(0, 1, -1, Int.MaxValue, 0, val req3 = new FetchRequest.Builder(0, 1, -1, Int.MaxValue, 0,
@ -643,7 +643,7 @@ class FetchRequestTest extends BaseRequestTest {
val res3 = sendFetchRequest(leaderId, req3) val res3 = sendFetchRequest(leaderId, req3)
val data3 = res3.responseData.get(topicPartition) val data3 = res3.responseData.get(topicPartition)
assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE, data3.error) assertEquals(Errors.UNSUPPORTED_COMPRESSION_TYPE.code, data3.errorCode)
// fetch request with version 10: works fine! // fetch request with version 10: works fine!
val req4= new FetchRequest.Builder(0, 10, -1, Int.MaxValue, 0, val req4= new FetchRequest.Builder(0, 10, -1, Int.MaxValue, 0,
@ -651,15 +651,15 @@ class FetchRequestTest extends BaseRequestTest {
.setMaxBytes(800).build() .setMaxBytes(800).build()
val res4 = sendFetchRequest(leaderId, req4) val res4 = sendFetchRequest(leaderId, req4)
val data4 = res4.responseData.get(topicPartition) val data4 = res4.responseData.get(topicPartition)
assertEquals(Errors.NONE, data4.error) assertEquals(Errors.NONE.code, data4.errorCode)
assertEquals(3, records(data4).size) assertEquals(3, records(data4).size)
} }
private def records(partitionData: FetchResponse.PartitionData[MemoryRecords]): Seq[Record] = { private def records(partitionData: FetchResponseData.PartitionData): Seq[Record] = {
partitionData.records.records.asScala.toBuffer FetchResponse.recordsOrFail(partitionData).records.asScala.toBuffer
} }
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse[MemoryRecords], private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
maxPartitionBytes: Int, maxResponseBytes: Int, numMessagesPerPartition: Int): Unit = { maxPartitionBytes: Int, maxResponseBytes: Int, numMessagesPerPartition: Int): Unit = {
assertEquals(expectedPartitions, fetchResponse.responseData.keySet.asScala.toSeq) assertEquals(expectedPartitions, fetchResponse.responseData.keySet.asScala.toSeq)
var emptyResponseSeen = false var emptyResponseSeen = false
@ -668,10 +668,10 @@ class FetchRequestTest extends BaseRequestTest {
expectedPartitions.foreach { tp => expectedPartitions.foreach { tp =>
val partitionData = fetchResponse.responseData.get(tp) val partitionData = fetchResponse.responseData.get(tp)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0) assertTrue(partitionData.highWatermark > 0)
val records = partitionData.records val records = FetchResponse.recordsOrFail(partitionData)
responseBufferSize += records.sizeInBytes responseBufferSize += records.sizeInBytes
val batches = records.batches.asScala.toBuffer val batches = records.batches.asScala.toBuffer

View File

@ -16,26 +16,26 @@
*/ */
package kafka.server package kafka.server
import java.util
import java.util.{Collections, Optional}
import kafka.utils.MockTime import kafka.utils.MockTime
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.Records
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID} import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.requests.{FetchRequest, FetchMetadata => JFetchMetadata}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
import java.util
import java.util.{Collections, Optional}
@Timeout(120) @Timeout(120)
class FetchSessionTest { class FetchSessionTest {
@Test @Test
def testNewSessionId(): Unit = { def testNewSessionId(): Unit = {
val cache = new FetchSessionCache(3, 100) val cache = new FetchSessionCache(3, 100)
for (i <- 0 to 10000) { for (_ <- 0 to 10000) {
val id = cache.newSessionId() val id = cache.newSessionId()
assertTrue(id > 0) assertTrue(id > 0)
} }
@ -125,7 +125,7 @@ class FetchSessionTest {
assertEquals(3, cache.totalPartitions) assertEquals(3, cache.totalPartitions)
} }
val EMPTY_PART_LIST = Collections.unmodifiableList(new util.ArrayList[TopicPartition]()) private val EMPTY_PART_LIST = Collections.unmodifiableList(new util.ArrayList[TopicPartition]())
@Test @Test
@ -155,13 +155,22 @@ class FetchSessionTest {
assertEquals(Optional.of(1), epochs1(tp1)) assertEquals(Optional.of(1), epochs1(tp1))
assertEquals(Optional.of(2), epochs1(tp2)) assertEquals(Optional.of(2), epochs1(tp2))
val response = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val response = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
response.put(tp0, new FetchResponse.PartitionData(Errors.NONE, 100, 100, response.put(tp0, new FetchResponseData.PartitionData()
100, null, null)) .setPartitionIndex(tp0.partition)
response.put(tp1, new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
response.put(tp2, new FetchResponse.PartitionData( .setLogStartOffset(100))
Errors.NONE, 5, 5, 5, null, null)) response.put(tp1, new FetchResponseData.PartitionData()
.setPartitionIndex(tp1.partition)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
response.put(tp2, new FetchResponseData.PartitionData()
.setPartitionIndex(tp2.partition)
.setHighWatermark(5)
.setLastStableOffset(5)
.setLogStartOffset(5))
val sessionId = context1.updateAndGenerateResponseData(response).sessionId() val sessionId = context1.updateAndGenerateResponseData(response).sessionId()
@ -220,10 +229,22 @@ class FetchSessionTest {
assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.empty, tp2 -> Optional.of(1)), assertEquals(Map(tp0 -> Optional.empty, tp1 -> Optional.empty, tp2 -> Optional.of(1)),
cachedLastFetchedEpochs(context1)) cachedLastFetchedEpochs(context1))
val response = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val response = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
response.put(tp0, new FetchResponse.PartitionData(Errors.NONE, 100, 100, 100, null, null)) response.put(tp0, new FetchResponseData.PartitionData()
response.put(tp1, new FetchResponse.PartitionData(Errors.NONE, 10, 10, 10, null, null)) .setPartitionIndex(tp0.partition)
response.put(tp2, new FetchResponse.PartitionData(Errors.NONE, 5, 5, 5, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
response.put(tp1, new FetchResponseData.PartitionData()
.setPartitionIndex(tp1.partition)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
response.put(tp2, new FetchResponseData.PartitionData()
.setPartitionIndex(tp2.partition)
.setHighWatermark(5)
.setLastStableOffset(5)
.setLogStartOffset(5))
val sessionId = context1.updateAndGenerateResponseData(response).sessionId() val sessionId = context1.updateAndGenerateResponseData(response).sessionId()
@ -275,15 +296,23 @@ class FetchSessionTest {
}) })
assertEquals(0, context2.getFetchOffset(new TopicPartition("foo", 0)).get) assertEquals(0, context2.getFetchOffset(new TopicPartition("foo", 0)).get)
assertEquals(10, context2.getFetchOffset(new TopicPartition("foo", 1)).get) assertEquals(10, context2.getFetchOffset(new TopicPartition("foo", 1)).get)
val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData2.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData2.put(new TopicPartition("foo", 0),
Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
respData2.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setPartitionIndex(0)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
respData2.put(new TopicPartition("foo", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2)
assertEquals(Errors.NONE, resp2.error()) assertEquals(Errors.NONE, resp2.error())
assertTrue(resp2.sessionId() != INVALID_SESSION_ID) assertTrue(resp2.sessionId() != INVALID_SESSION_ID)
assertEquals(respData2, resp2.responseData()) assertEquals(respData2, resp2.responseData)
// Test trying to create a new session with an invalid epoch // Test trying to create a new session with an invalid epoch
val context3 = fetchManager.newContext( val context3 = fetchManager.newContext(
@ -314,7 +343,7 @@ class FetchSessionTest {
val resp5 = context5.updateAndGenerateResponseData(respData2) val resp5 = context5.updateAndGenerateResponseData(respData2)
assertEquals(Errors.NONE, resp5.error()) assertEquals(Errors.NONE, resp5.error())
assertEquals(resp2.sessionId(), resp5.sessionId()) assertEquals(resp2.sessionId(), resp5.sessionId())
assertEquals(0, resp5.responseData().size()) assertEquals(0, resp5.responseData.size())
// Test setting an invalid fetch session epoch. // Test setting an invalid fetch session epoch.
val context6 = fetchManager.newContext( val context6 = fetchManager.newContext(
@ -345,11 +374,19 @@ class FetchSessionTest {
new JFetchMetadata(prevSessionId, FINAL_EPOCH), reqData8, EMPTY_PART_LIST, false) new JFetchMetadata(prevSessionId, FINAL_EPOCH), reqData8, EMPTY_PART_LIST, false)
assertEquals(classOf[SessionlessFetchContext], context8.getClass) assertEquals(classOf[SessionlessFetchContext], context8.getClass)
assertEquals(0, cache.size) assertEquals(0, cache.size)
val respData8 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData8 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData8.put(new TopicPartition("bar", 0), respData8.put(new TopicPartition("bar", 0),
new FetchResponse.PartitionData(Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
.setPartitionIndex(0)
.setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
respData8.put(new TopicPartition("bar", 1), respData8.put(new TopicPartition("bar", 1),
new FetchResponse.PartitionData(Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
val resp8 = context8.updateAndGenerateResponseData(respData8) val resp8 = context8.updateAndGenerateResponseData(respData8)
assertEquals(Errors.NONE, resp8.error) assertEquals(Errors.NONE, resp8.error)
nextSessionId = resp8.sessionId nextSessionId = resp8.sessionId
@ -370,15 +407,21 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false) val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], context1.getClass) assertEquals(classOf[FullFetchContext], context1.getClass)
val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData1.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
Errors.NONE, 100, 100, 100, null, null)) .setPartitionIndex(0)
respData1.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
.setLogStartOffset(100))
respData1.put(new TopicPartition("foo", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, resp1.error()) assertEquals(Errors.NONE, resp1.error())
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
assertEquals(2, resp1.responseData().size()) assertEquals(2, resp1.responseData.size())
// Create an incremental fetch request that removes foo-0 and adds bar-0 // Create an incremental fetch request that removes foo-0 and adds bar-0
val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] val reqData2 = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData]
@ -391,18 +434,26 @@ class FetchSessionTest {
assertEquals(classOf[IncrementalFetchContext], context2.getClass) assertEquals(classOf[IncrementalFetchContext], context2.getClass)
val parts2 = Set(new TopicPartition("foo", 1), new TopicPartition("bar", 0)) val parts2 = Set(new TopicPartition("foo", 1), new TopicPartition("bar", 0))
val reqData2Iter = parts2.iterator val reqData2Iter = parts2.iterator
context2.foreachPartition((topicPart, data) => { context2.foreachPartition((topicPart, _) => {
assertEquals(reqData2Iter.next(), topicPart) assertEquals(reqData2Iter.next(), topicPart)
}) })
assertEquals(None, context2.getFetchOffset(new TopicPartition("foo", 0))) assertEquals(None, context2.getFetchOffset(new TopicPartition("foo", 0)))
assertEquals(10, context2.getFetchOffset(new TopicPartition("foo", 1)).get) assertEquals(10, context2.getFetchOffset(new TopicPartition("foo", 1)).get)
assertEquals(15, context2.getFetchOffset(new TopicPartition("bar", 0)).get) assertEquals(15, context2.getFetchOffset(new TopicPartition("bar", 0)).get)
assertEquals(None, context2.getFetchOffset(new TopicPartition("bar", 2))) assertEquals(None, context2.getFetchOffset(new TopicPartition("bar", 2)))
val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData2.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( respData2.put(new TopicPartition("foo", 1),
Errors.NONE, 10, 10, 10, null, null)) new FetchResponseData.PartitionData()
respData2.put(new TopicPartition("bar", 0), new FetchResponse.PartitionData( .setPartitionIndex(1)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
respData2.put(new TopicPartition("bar", 0),
new FetchResponseData.PartitionData()
.setPartitionIndex(0)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2)
assertEquals(Errors.NONE, resp2.error) assertEquals(Errors.NONE, resp2.error)
assertEquals(1, resp2.responseData.size) assertEquals(1, resp2.responseData.size)
@ -424,15 +475,21 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session1context1 = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false) val session1context1 = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], session1context1.getClass) assertEquals(classOf[FullFetchContext], session1context1.getClass)
val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData1.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
Errors.NONE, 100, 100, 100, null, null)) .setPartitionIndex(0)
respData1.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
.setLogStartOffset(100))
respData1.put(new TopicPartition("foo", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session1resp = session1context1.updateAndGenerateResponseData(respData1) val session1resp = session1context1.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, session1resp.error()) assertEquals(Errors.NONE, session1resp.error())
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID) assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session1resp.responseData().size()) assertEquals(2, session1resp.responseData.size)
// check session entered into case // check session entered into case
assertTrue(cache.get(session1resp.sessionId()).isDefined) assertTrue(cache.get(session1resp.sessionId()).isDefined)
@ -446,15 +503,22 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session2context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false) val session2context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], session2context.getClass) assertEquals(classOf[FullFetchContext], session2context.getClass)
val session2RespData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val session2RespData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
session2RespData.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( session2RespData.put(new TopicPartition("foo", 0),
Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
session2RespData.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setPartitionIndex(0)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
session2RespData.put(new TopicPartition("foo", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session2resp = session2context.updateAndGenerateResponseData(respData1) val session2resp = session2context.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, session2resp.error()) assertEquals(Errors.NONE, session2resp.error())
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID) assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session2resp.responseData().size()) assertEquals(2, session2resp.responseData.size)
// both newly created entries are present in cache // both newly created entries are present in cache
assertTrue(cache.get(session1resp.sessionId()).isDefined) assertTrue(cache.get(session1resp.sessionId()).isDefined)
@ -481,19 +545,25 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session3context = fetchManager.newContext(JFetchMetadata.INITIAL, session3req, EMPTY_PART_LIST, false) val session3context = fetchManager.newContext(JFetchMetadata.INITIAL, session3req, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], session3context.getClass) assertEquals(classOf[FullFetchContext], session3context.getClass)
val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData3.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData3.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
Errors.NONE, 100, 100, 100, null, null)) .setPartitionIndex(0)
respData3.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
.setLogStartOffset(100))
respData3.put(new TopicPartition("foo", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session3resp = session3context.updateAndGenerateResponseData(respData3) val session3resp = session3context.updateAndGenerateResponseData(respData3)
assertEquals(Errors.NONE, session3resp.error()) assertEquals(Errors.NONE, session3resp.error())
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID) assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session3resp.responseData().size()) assertEquals(2, session3resp.responseData.size)
assertTrue(cache.get(session1resp.sessionId()).isDefined) assertTrue(cache.get(session1resp.sessionId()).isDefined)
assertFalse(cache.get(session2resp.sessionId()).isDefined, assertFalse(cache.get(session2resp.sessionId()).isDefined, "session 2 should have been evicted by latest session, as session 1 was used more recently")
"session 2 should have been evicted by latest session, as session 1 was used more recently")
assertTrue(cache.get(session3resp.sessionId()).isDefined) assertTrue(cache.get(session3resp.sessionId()).isDefined)
} }
@ -512,15 +582,21 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session1context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, true) val session1context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, true)
assertEquals(classOf[FullFetchContext], session1context.getClass) assertEquals(classOf[FullFetchContext], session1context.getClass)
val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData1.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
Errors.NONE, 100, 100, 100, null, null)) .setPartitionIndex(0)
respData1.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
.setLogStartOffset(100))
respData1.put(new TopicPartition("foo", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session1resp = session1context.updateAndGenerateResponseData(respData1) val session1resp = session1context.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, session1resp.error()) assertEquals(Errors.NONE, session1resp.error())
assertTrue(session1resp.sessionId() != INVALID_SESSION_ID) assertTrue(session1resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session1resp.responseData().size()) assertEquals(2, session1resp.responseData.size)
assertEquals(1, cache.size) assertEquals(1, cache.size)
// move time forward to age session 1 a little compared to session 2 // move time forward to age session 1 a little compared to session 2
@ -534,15 +610,23 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session2context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false) val session2context = fetchManager.newContext(JFetchMetadata.INITIAL, session1req, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], session2context.getClass) assertEquals(classOf[FullFetchContext], session2context.getClass)
val session2RespData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val session2RespData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
session2RespData.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( session2RespData.put(new TopicPartition("foo", 0),
Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
session2RespData.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setPartitionIndex(0)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
session2RespData.put(new TopicPartition("foo", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session2resp = session2context.updateAndGenerateResponseData(respData1) val session2resp = session2context.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, session2resp.error()) assertEquals(Errors.NONE, session2resp.error())
assertTrue(session2resp.sessionId() != INVALID_SESSION_ID) assertTrue(session2resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session2resp.responseData().size()) assertEquals(2, session2resp.responseData.size)
// both newly created entries are present in cache // both newly created entries are present in cache
assertTrue(cache.get(session1resp.sessionId()).isDefined) assertTrue(cache.get(session1resp.sessionId()).isDefined)
@ -558,21 +642,28 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session3context = fetchManager.newContext(JFetchMetadata.INITIAL, session3req, EMPTY_PART_LIST, true) val session3context = fetchManager.newContext(JFetchMetadata.INITIAL, session3req, EMPTY_PART_LIST, true)
assertEquals(classOf[FullFetchContext], session3context.getClass) assertEquals(classOf[FullFetchContext], session3context.getClass)
val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData3 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData3.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData3.put(new TopicPartition("foo", 0),
Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
respData3.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setPartitionIndex(0)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
respData3.put(new TopicPartition("foo", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session3resp = session3context.updateAndGenerateResponseData(respData3) val session3resp = session3context.updateAndGenerateResponseData(respData3)
assertEquals(Errors.NONE, session3resp.error()) assertEquals(Errors.NONE, session3resp.error())
assertTrue(session3resp.sessionId() != INVALID_SESSION_ID) assertTrue(session3resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session3resp.responseData().size()) assertEquals(2, session3resp.responseData.size)
assertTrue(cache.get(session1resp.sessionId()).isDefined) assertTrue(cache.get(session1resp.sessionId()).isDefined)
// even though session 2 is more recent than session 1, and has not reached expiry time, it is less // even though session 2 is more recent than session 1, and has not reached expiry time, it is less
// privileged than session 2, and thus session 3 should be entered and session 2 evicted. // privileged than session 2, and thus session 3 should be entered and session 2 evicted.
assertFalse(cache.get(session2resp.sessionId()).isDefined, assertFalse(cache.get(session2resp.sessionId()).isDefined, "session 2 should have been evicted by session 3")
"session 2 should have been evicted by session 3")
assertTrue(cache.get(session3resp.sessionId()).isDefined) assertTrue(cache.get(session3resp.sessionId()).isDefined)
assertEquals(2, cache.size) assertEquals(2, cache.size)
@ -586,18 +677,25 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val session4context = fetchManager.newContext(JFetchMetadata.INITIAL, session4req, EMPTY_PART_LIST, true) val session4context = fetchManager.newContext(JFetchMetadata.INITIAL, session4req, EMPTY_PART_LIST, true)
assertEquals(classOf[FullFetchContext], session4context.getClass) assertEquals(classOf[FullFetchContext], session4context.getClass)
val respData4 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData4 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData4.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData4.put(new TopicPartition("foo", 0),
Errors.NONE, 100, 100, 100, null, null)) new FetchResponseData.PartitionData()
respData4.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setPartitionIndex(0)
Errors.NONE, 10, 10, 10, null, null)) .setHighWatermark(100)
.setLastStableOffset(100)
.setLogStartOffset(100))
respData4.put(new TopicPartition("foo", 1),
new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val session4resp = session3context.updateAndGenerateResponseData(respData4) val session4resp = session3context.updateAndGenerateResponseData(respData4)
assertEquals(Errors.NONE, session4resp.error()) assertEquals(Errors.NONE, session4resp.error())
assertTrue(session4resp.sessionId() != INVALID_SESSION_ID) assertTrue(session4resp.sessionId() != INVALID_SESSION_ID)
assertEquals(2, session4resp.responseData().size()) assertEquals(2, session4resp.responseData.size)
assertFalse(cache.get(session1resp.sessionId()).isDefined, assertFalse(cache.get(session1resp.sessionId()).isDefined, "session 1 should have been evicted by session 4 even though it is privileged as it has hit eviction time")
"session 1 should have been evicted by session 4 even though it is privileged as it has hit eviction time")
assertTrue(cache.get(session3resp.sessionId()).isDefined) assertTrue(cache.get(session3resp.sessionId()).isDefined)
assertTrue(cache.get(session4resp.sessionId()).isDefined) assertTrue(cache.get(session4resp.sessionId()).isDefined)
assertEquals(2, cache.size) assertEquals(2, cache.size)
@ -617,11 +715,17 @@ class FetchSessionTest {
Optional.empty())) Optional.empty()))
val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false) val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData1, EMPTY_PART_LIST, false)
assertEquals(classOf[FullFetchContext], context1.getClass) assertEquals(classOf[FullFetchContext], context1.getClass)
val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData1 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData1.put(new TopicPartition("foo", 0), new FetchResponse.PartitionData( respData1.put(new TopicPartition("foo", 0), new FetchResponseData.PartitionData()
Errors.NONE, 100, 100, 100, null, null)) .setPartitionIndex(0)
respData1.put(new TopicPartition("foo", 1), new FetchResponse.PartitionData( .setHighWatermark(100)
Errors.NONE, 10, 10, 10, null, null)) .setLastStableOffset(100)
.setLogStartOffset(100))
respData1.put(new TopicPartition("foo", 1), new FetchResponseData.PartitionData()
.setPartitionIndex(1)
.setHighWatermark(10)
.setLastStableOffset(10)
.setLogStartOffset(10))
val resp1 = context1.updateAndGenerateResponseData(respData1) val resp1 = context1.updateAndGenerateResponseData(respData1)
assertEquals(Errors.NONE, resp1.error) assertEquals(Errors.NONE, resp1.error)
assertTrue(resp1.sessionId() != INVALID_SESSION_ID) assertTrue(resp1.sessionId() != INVALID_SESSION_ID)
@ -636,10 +740,10 @@ class FetchSessionTest {
val context2 = fetchManager.newContext( val context2 = fetchManager.newContext(
new JFetchMetadata(resp1.sessionId, 1), reqData2, removed2, false) new JFetchMetadata(resp1.sessionId, 1), reqData2, removed2, false)
assertEquals(classOf[SessionlessFetchContext], context2.getClass) assertEquals(classOf[SessionlessFetchContext], context2.getClass)
val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData2 = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
val resp2 = context2.updateAndGenerateResponseData(respData2) val resp2 = context2.updateAndGenerateResponseData(respData2)
assertEquals(INVALID_SESSION_ID, resp2.sessionId) assertEquals(INVALID_SESSION_ID, resp2.sessionId)
assertTrue(resp2.responseData().isEmpty) assertTrue(resp2.responseData.isEmpty)
assertEquals(0, cache.size) assertEquals(0, cache.size)
} }
@ -658,12 +762,19 @@ class FetchSessionTest {
// Full fetch context returns all partitions in the response // Full fetch context returns all partitions in the response
val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData, EMPTY_PART_LIST, isFollower = false) val context1 = fetchManager.newContext(JFetchMetadata.INITIAL, reqData, EMPTY_PART_LIST, isFollower = false)
assertEquals(classOf[FullFetchContext], context1.getClass) assertEquals(classOf[FullFetchContext], context1.getClass)
val respData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val respData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
respData.put(tp1, new FetchResponse.PartitionData(Errors.NONE, respData.put(tp1, new FetchResponseData.PartitionData()
105, 105, 0, Optional.empty(), Collections.emptyList(), Optional.empty(), null)) .setPartitionIndex(tp1.partition)
val divergingEpoch = Optional.of(new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(90)) .setHighWatermark(105)
respData.put(tp2, new FetchResponse.PartitionData(Errors.NONE, .setLastStableOffset(105)
105, 105, 0, Optional.empty(), Collections.emptyList(), divergingEpoch, null)) .setLogStartOffset(0))
val divergingEpoch = new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(90)
respData.put(tp2, new FetchResponseData.PartitionData()
.setPartitionIndex(tp2.partition)
.setHighWatermark(105)
.setLastStableOffset(105)
.setLogStartOffset(0)
.setDivergingEpoch(divergingEpoch))
val resp1 = context1.updateAndGenerateResponseData(respData) val resp1 = context1.updateAndGenerateResponseData(respData)
assertEquals(Errors.NONE, resp1.error) assertEquals(Errors.NONE, resp1.error)
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId) assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
@ -679,8 +790,12 @@ class FetchSessionTest {
assertEquals(Collections.singleton(tp2), resp2.responseData.keySet) assertEquals(Collections.singleton(tp2), resp2.responseData.keySet)
// All partitions with divergent epoch should be returned. // All partitions with divergent epoch should be returned.
respData.put(tp1, new FetchResponse.PartitionData(Errors.NONE, respData.put(tp1, new FetchResponseData.PartitionData()
105, 105, 0, Optional.empty(), Collections.emptyList(), divergingEpoch, null)) .setPartitionIndex(tp1.partition)
.setHighWatermark(105)
.setLastStableOffset(105)
.setLogStartOffset(0)
.setDivergingEpoch(divergingEpoch))
val resp3 = context2.updateAndGenerateResponseData(respData) val resp3 = context2.updateAndGenerateResponseData(respData)
assertEquals(Errors.NONE, resp3.error) assertEquals(Errors.NONE, resp3.error)
assertEquals(resp1.sessionId, resp3.sessionId) assertEquals(resp1.sessionId, resp3.sessionId)
@ -688,8 +803,11 @@ class FetchSessionTest {
// Partitions that meet other conditions should be returned regardless of whether // Partitions that meet other conditions should be returned regardless of whether
// divergingEpoch is set or not. // divergingEpoch is set or not.
respData.put(tp1, new FetchResponse.PartitionData(Errors.NONE, respData.put(tp1, new FetchResponseData.PartitionData()
110, 110, 0, Optional.empty(), Collections.emptyList(), Optional.empty(), null)) .setPartitionIndex(tp1.partition)
.setHighWatermark(110)
.setLastStableOffset(110)
.setLogStartOffset(0))
val resp4 = context2.updateAndGenerateResponseData(respData) val resp4 = context2.updateAndGenerateResponseData(respData)
assertEquals(Errors.NONE, resp4.error) assertEquals(Errors.NONE, resp4.error)
assertEquals(resp1.sessionId, resp4.sessionId) assertEquals(resp1.sessionId, resp4.sessionId)

View File

@ -1072,7 +1072,7 @@ class KafkaApisTest {
val response = capturedResponse.getValue.asInstanceOf[OffsetCommitResponse] val response = capturedResponse.getValue.asInstanceOf[OffsetCommitResponse]
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION,
Errors.forCode(response.data().topics().get(0).partitions().get(0).errorCode())) Errors.forCode(response.data.topics().get(0).partitions().get(0).errorCode))
} }
checkInvalidPartition(-1) checkInvalidPartition(-1)
@ -1425,9 +1425,9 @@ class KafkaApisTest {
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(
Collections.singletonList(new ProduceRequestData.TopicProduceData() Collections.singletonList(new ProduceRequestData.TopicProduceData()
.setName(tp.topic()).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition()) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
@ -1632,21 +1632,21 @@ class KafkaApisTest {
val topicStates = Seq( val topicStates = Seq(
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(groupMetadataPartition.topic()) .setTopicName(groupMetadataPartition.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(groupMetadataPartition.partition()) .setPartitionIndex(groupMetadataPartition.partition)
.setLeaderEpoch(leaderEpoch) .setLeaderEpoch(leaderEpoch)
.setDeletePartition(deletePartition)).asJava), .setDeletePartition(deletePartition)).asJava),
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(txnStatePartition.topic()) .setTopicName(txnStatePartition.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(txnStatePartition.partition()) .setPartitionIndex(txnStatePartition.partition)
.setLeaderEpoch(leaderEpoch) .setLeaderEpoch(leaderEpoch)
.setDeletePartition(deletePartition)).asJava), .setDeletePartition(deletePartition)).asJava),
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(fooPartition.topic()) .setTopicName(fooPartition.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(fooPartition.partition()) .setPartitionIndex(fooPartition.partition)
.setLeaderEpoch(leaderEpoch) .setLeaderEpoch(leaderEpoch)
.setDeletePartition(deletePartition)).asJava) .setDeletePartition(deletePartition)).asJava)
).asJava ).asJava
@ -1806,8 +1806,8 @@ class KafkaApisTest {
val response = capturedResponse.getValue.asInstanceOf[DescribeGroupsResponse] val response = capturedResponse.getValue.asInstanceOf[DescribeGroupsResponse]
val group = response.data().groups().get(0) val group = response.data.groups().get(0)
assertEquals(Errors.NONE, Errors.forCode(group.errorCode())) assertEquals(Errors.NONE, Errors.forCode(group.errorCode))
assertEquals(groupId, group.groupId()) assertEquals(groupId, group.groupId())
assertEquals(groupSummary.state, group.groupState()) assertEquals(groupSummary.state, group.groupState())
assertEquals(groupSummary.protocolType, group.protocolType()) assertEquals(groupSummary.protocolType, group.protocolType())
@ -1873,7 +1873,7 @@ class KafkaApisTest {
val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse] val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse]
def errorForPartition(topic: String, partition: Int): Errors = { def errorForPartition(topic: String, partition: Int): Errors = {
Errors.forCode(response.data.topics.find(topic).partitions.find(partition).errorCode()) Errors.forCode(response.data.topics.find(topic).partitions.find(partition).errorCode)
} }
assertEquals(2, response.data.topics.size) assertEquals(2, response.data.topics.size)
@ -1914,7 +1914,7 @@ class KafkaApisTest {
val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse] val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse]
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION,
Errors.forCode(response.data.topics.find(topic).partitions.find(invalidPartitionId).errorCode())) Errors.forCode(response.data.topics.find(topic).partitions.find(invalidPartitionId).errorCode))
} }
checkInvalidPartition(-1) checkInvalidPartition(-1)
@ -1942,7 +1942,7 @@ class KafkaApisTest {
val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse] val response = capturedResponse.getValue.asInstanceOf[OffsetDeleteResponse]
assertEquals(Errors.GROUP_ID_NOT_FOUND, Errors.forCode(response.data.errorCode())) assertEquals(Errors.GROUP_ID_NOT_FOUND, Errors.forCode(response.data.errorCode))
} }
private def testListOffsetFailedGetLeaderReplica(error: Errors): Unit = { private def testListOffsetFailedGetLeaderReplica(error: Errors): Unit = {
@ -2130,16 +2130,15 @@ class KafkaApisTest {
EasyMock.replay(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, fetchManager) EasyMock.replay(replicaManager, clientQuotaManager, clientRequestQuotaManager, requestChannel, fetchManager)
createKafkaApis().handleFetchRequest(request) createKafkaApis().handleFetchRequest(request)
val response = capturedResponse.getValue.asInstanceOf[FetchResponse[BaseRecords]] val response = capturedResponse.getValue.asInstanceOf[FetchResponse]
assertTrue(response.responseData.containsKey(tp)) assertTrue(response.responseData.containsKey(tp))
val partitionData = response.responseData.get(tp) val partitionData = response.responseData.get(tp)
assertEquals(Errors.NONE, partitionData.error) assertEquals(Errors.NONE.code, partitionData.errorCode)
assertEquals(hw, partitionData.highWatermark) assertEquals(hw, partitionData.highWatermark)
assertEquals(-1, partitionData.lastStableOffset) assertEquals(-1, partitionData.lastStableOffset)
assertEquals(0, partitionData.logStartOffset) assertEquals(0, partitionData.logStartOffset)
assertEquals(timestamp, assertEquals(timestamp, FetchResponse.recordsOrFail(partitionData).batches.iterator.next.maxTimestamp)
partitionData.records.asInstanceOf[MemoryRecords].batches.iterator.next.maxTimestamp)
assertNull(partitionData.abortedTransactions) assertNull(partitionData.abortedTransactions)
} }
@ -2563,7 +2562,7 @@ class KafkaApisTest {
.setPartitions(Collections.singletonList( .setPartitions(Collections.singletonList(
new OffsetCommitResponseData.OffsetCommitResponsePartition() new OffsetCommitResponseData.OffsetCommitResponsePartition()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(Errors.UNSUPPORTED_VERSION.code()) .setErrorCode(Errors.UNSUPPORTED_VERSION.code)
)) ))
) )
val response = capturedResponse.getValue.asInstanceOf[OffsetCommitResponse] val response = capturedResponse.getValue.asInstanceOf[OffsetCommitResponse]
@ -2871,9 +2870,9 @@ class KafkaApisTest {
val fooPartition = new TopicPartition("foo", 0) val fooPartition = new TopicPartition("foo", 0)
val topicStates = Seq( val topicStates = Seq(
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(fooPartition.topic()) .setTopicName(fooPartition.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(fooPartition.partition()) .setPartitionIndex(fooPartition.partition)
.setLeaderEpoch(1) .setLeaderEpoch(1)
.setDeletePartition(false)).asJava) .setDeletePartition(false)).asJava)
).asJava ).asJava
@ -3246,15 +3245,18 @@ class KafkaApisTest {
@Test @Test
def testSizeOfThrottledPartitions(): Unit = { def testSizeOfThrottledPartitions(): Unit = {
def fetchResponse(data: Map[TopicPartition, String]): FetchResponse[Records] = {
val responseData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]]( def fetchResponse(data: Map[TopicPartition, String]): FetchResponse = {
val responseData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData](
data.map { case (tp, raw) => data.map { case (tp, raw) =>
tp -> new FetchResponse.PartitionData(Errors.NONE, tp -> new FetchResponseData.PartitionData()
105, 105, 0, Optional.empty(), Collections.emptyList(), Optional.empty(), .setPartitionIndex(tp.partition)
MemoryRecords.withRecords(CompressionType.NONE, .setHighWatermark(105)
new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))).asInstanceOf[Records]) .setLastStableOffset(105)
.setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
}.toMap.asJava) }.toMap.asJava)
new FetchResponse(Errors.NONE, responseData, 100, 100) FetchResponse.of(Errors.NONE, 100, 100, responseData)
} }
val throttledPartition = new TopicPartition("throttledData", 0) val throttledPartition = new TopicPartition("throttledData", 0)

View File

@ -17,26 +17,22 @@
package kafka.server package kafka.server
import java.io.File
import java.util.concurrent.atomic.AtomicInteger
import java.util.{Optional, Properties, Random}
import kafka.log.{ClientRecordDeletion, Log, LogSegment} import kafka.log.{ClientRecordDeletion, Log, LogSegment}
import kafka.utils.{MockTime, TestUtils} import kafka.utils.{MockTime, TestUtils}
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic}
import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition import org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse, ListOffsetsTopicResponse}
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse
import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse}
import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.apache.kafka.common.{IsolationLevel, TopicPartition}
import org.easymock.{EasyMock, IAnswer} import org.easymock.{EasyMock, IAnswer}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import scala.jdk.CollectionConverters._ import java.io.File
import java.util.concurrent.atomic.AtomicInteger
import java.util.{Optional, Properties, Random}
import scala.collection.mutable.Buffer import scala.collection.mutable.Buffer
import scala.jdk.CollectionConverters._
class LogOffsetTest extends BaseRequestTest { class LogOffsetTest extends BaseRequestTest {
@ -127,7 +123,7 @@ class LogOffsetTest extends BaseRequestTest {
Map(topicPartition -> new FetchRequest.PartitionData(consumerOffsets.head, FetchRequest.INVALID_LOG_START_OFFSET, Map(topicPartition -> new FetchRequest.PartitionData(consumerOffsets.head, FetchRequest.INVALID_LOG_START_OFFSET,
300 * 1024, Optional.empty())).asJava).build() 300 * 1024, Optional.empty())).asJava).build()
val fetchResponse = sendFetchRequest(fetchRequest) val fetchResponse = sendFetchRequest(fetchRequest)
assertFalse(fetchResponse.responseData.get(topicPartition).records.batches.iterator.hasNext) assertFalse(FetchResponse.recordsOrFail(fetchResponse.responseData.get(topicPartition)).batches.iterator.hasNext)
} }
@Test @Test
@ -251,8 +247,8 @@ class LogOffsetTest extends BaseRequestTest {
connectAndReceive[ListOffsetsResponse](request) connectAndReceive[ListOffsetsResponse](request)
} }
private def sendFetchRequest(request: FetchRequest): FetchResponse[MemoryRecords] = { private def sendFetchRequest(request: FetchRequest): FetchResponse = {
connectAndReceive[FetchResponse[MemoryRecords]](request) connectAndReceive[FetchResponse](request)
} }
private def buildTargetTimes(tp: TopicPartition, timestamp: Long, maxNumOffsets: Int): List[ListOffsetsTopic] = { private def buildTargetTimes(tp: TopicPartition, timestamp: Long, maxNumOffsets: Int): List[ListOffsetsTopic] = {

View File

@ -16,9 +16,6 @@
*/ */
package kafka.server package kafka.server
import java.nio.charset.StandardCharsets
import java.util.{Collections, Optional}
import kafka.api.{ApiVersion, KAFKA_2_6_IV0} import kafka.api.{ApiVersion, KAFKA_2_6_IV0}
import kafka.cluster.{BrokerEndPoint, Partition} import kafka.cluster.{BrokerEndPoint, Partition}
import kafka.log.{Log, LogAppendInfo, LogManager} import kafka.log.{Log, LogAppendInfo, LogManager}
@ -32,17 +29,18 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Records, SimpleRecord} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.requests.FetchResponse
import org.apache.kafka.common.utils.SystemTime import org.apache.kafka.common.utils.SystemTime
import org.easymock.EasyMock._ import org.easymock.EasyMock._
import org.easymock.{Capture, CaptureType} import org.easymock.{Capture, CaptureType}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.{AfterEach, Test}
import scala.jdk.CollectionConverters._ import java.nio.charset.StandardCharsets
import java.util.Collections
import scala.collection.{Map, mutable} import scala.collection.{Map, mutable}
import scala.jdk.CollectionConverters._
class ReplicaFetcherThreadTest { class ReplicaFetcherThreadTest {
@ -531,16 +529,18 @@ class ReplicaFetcherThreadTest {
assertEquals(1, mockNetwork.fetchCount) assertEquals(1, mockNetwork.fetchCount)
partitions.foreach { tp => assertEquals(Fetching, thread.fetchState(tp).get.state) } partitions.foreach { tp => assertEquals(Fetching, thread.fetchState(tp).get.state) }
def partitionData(divergingEpoch: FetchResponseData.EpochEndOffset): FetchResponse.PartitionData[Records] = { def partitionData(partition: Int, divergingEpoch: FetchResponseData.EpochEndOffset): FetchResponseData.PartitionData = {
new FetchResponse.PartitionData[Records]( new FetchResponseData.PartitionData()
Errors.NONE, 0, 0, 0, Optional.empty(), Collections.emptyList(), .setPartitionIndex(partition)
Optional.of(divergingEpoch), MemoryRecords.EMPTY) .setLastStableOffset(0)
.setLogStartOffset(0)
.setDivergingEpoch(divergingEpoch)
} }
// Loop 2 should truncate based on diverging epoch and continue to send fetch requests. // Loop 2 should truncate based on diverging epoch and continue to send fetch requests.
mockNetwork.setFetchPartitionDataForNextResponse(Map( mockNetwork.setFetchPartitionDataForNextResponse(Map(
t1p0 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(4).setEndOffset(140)), t1p0 -> partitionData(t1p0.partition, new FetchResponseData.EpochEndOffset().setEpoch(4).setEndOffset(140)),
t1p1 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(4).setEndOffset(141)) t1p1 -> partitionData(t1p1.partition, new FetchResponseData.EpochEndOffset().setEpoch(4).setEndOffset(141))
)) ))
latestLogEpoch = Some(4) latestLogEpoch = Some(4)
thread.doWork() thread.doWork()
@ -555,8 +555,8 @@ class ReplicaFetcherThreadTest {
// Loop 3 should truncate because of diverging epoch. Offset truncation is not complete // Loop 3 should truncate because of diverging epoch. Offset truncation is not complete
// because divergent epoch is not known to follower. We truncate and stay in Fetching state. // because divergent epoch is not known to follower. We truncate and stay in Fetching state.
mockNetwork.setFetchPartitionDataForNextResponse(Map( mockNetwork.setFetchPartitionDataForNextResponse(Map(
t1p0 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(130)), t1p0 -> partitionData(t1p0.partition, new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(130)),
t1p1 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(131)) t1p1 -> partitionData(t1p1.partition, new FetchResponseData.EpochEndOffset().setEpoch(3).setEndOffset(131))
)) ))
thread.doWork() thread.doWork()
assertEquals(0, mockNetwork.epochFetchCount) assertEquals(0, mockNetwork.epochFetchCount)
@ -569,8 +569,8 @@ class ReplicaFetcherThreadTest {
// because divergent epoch is not known to follower. Last fetched epoch cannot be determined // because divergent epoch is not known to follower. Last fetched epoch cannot be determined
// from the log. We truncate and stay in Fetching state. // from the log. We truncate and stay in Fetching state.
mockNetwork.setFetchPartitionDataForNextResponse(Map( mockNetwork.setFetchPartitionDataForNextResponse(Map(
t1p0 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(2).setEndOffset(120)), t1p0 -> partitionData(t1p0.partition, new FetchResponseData.EpochEndOffset().setEpoch(2).setEndOffset(120)),
t1p1 -> partitionData(new FetchResponseData.EpochEndOffset().setEpoch(2).setEndOffset(121)) t1p1 -> partitionData(t1p1.partition, new FetchResponseData.EpochEndOffset().setEpoch(2).setEndOffset(121))
)) ))
latestLogEpoch = None latestLogEpoch = None
thread.doWork() thread.doWork()
@ -963,9 +963,11 @@ class ReplicaFetcherThreadTest {
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(CompressionType.NONE,
new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
val partitionData: thread.FetchData = new FetchResponseData.PartitionData()
val partitionData: thread.FetchData = new FetchResponse.PartitionData[Records]( .setPartitionIndex(t1p0.partition)
Errors.NONE, 0, 0, 0, Optional.empty(), Collections.emptyList(), records) .setLastStableOffset(0)
.setLogStartOffset(0)
.setRecords(records)
thread.processPartitionData(t1p0, 0, partitionData) thread.processPartitionData(t1p0, 0, partitionData)
if (isReassigning) if (isReassigning)

View File

@ -17,24 +17,17 @@
package kafka.server package kafka.server
import java.io.File
import java.net.InetAddress
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.{Collections, Optional, Properties}
import kafka.api._ import kafka.api._
import kafka.log.{AppendOrigin, Log, LogConfig, LogManager, ProducerStateManager}
import kafka.cluster.{BrokerEndPoint, Partition} import kafka.cluster.{BrokerEndPoint, Partition}
import kafka.log.LeaderOffsetIncremented import kafka.log._
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.server.checkpoints.{LazyOffsetCheckpoints, OffsetCheckpointFile}
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend
import kafka.server.metadata.CachedConfigRepository import kafka.server.metadata.CachedConfigRepository
import kafka.utils.TestUtils.createBroker import kafka.utils.TestUtils.createBroker
import kafka.utils.timer.MockTimer import kafka.utils.timer.MockTimer
import kafka.utils.{MockScheduler, MockTime, TestUtils} import kafka.utils.{MockScheduler, MockTime, TestUtils}
import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.message.LeaderAndIsrRequestData import org.apache.kafka.common.message.LeaderAndIsrRequestData
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
@ -45,21 +38,23 @@ import org.apache.kafka.common.record._
import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata
import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{IsolationLevel, Node, TopicPartition, Uuid} import org.apache.kafka.common.{IsolationLevel, Node, TopicPartition, Uuid}
import org.easymock.EasyMock import org.easymock.EasyMock
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.Mockito import org.mockito.Mockito
import scala.collection.mutable import java.io.File
import java.net.InetAddress
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.{Collections, Optional, Properties}
import scala.collection.{Map, Seq, mutable}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq}
class ReplicaManagerTest { class ReplicaManagerTest {
@ -403,7 +398,7 @@ class ReplicaManagerTest {
assertEquals(Errors.NONE, fetchData.error) assertEquals(Errors.NONE, fetchData.error)
assertTrue(fetchData.records.batches.asScala.isEmpty) assertTrue(fetchData.records.batches.asScala.isEmpty)
assertEquals(Some(0), fetchData.lastStableOffset) assertEquals(Some(0), fetchData.lastStableOffset)
assertEquals(Some(List.empty[AbortedTransaction]), fetchData.abortedTransactions) assertEquals(Some(List.empty[FetchResponseData.AbortedTransaction]), fetchData.abortedTransactions)
// delayed fetch should timeout and return nothing // delayed fetch should timeout and return nothing
consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0), consumerFetchResult = fetchAsConsumer(replicaManager, new TopicPartition(topic, 0),
@ -416,7 +411,7 @@ class ReplicaManagerTest {
assertEquals(Errors.NONE, fetchData.error) assertEquals(Errors.NONE, fetchData.error)
assertTrue(fetchData.records.batches.asScala.isEmpty) assertTrue(fetchData.records.batches.asScala.isEmpty)
assertEquals(Some(0), fetchData.lastStableOffset) assertEquals(Some(0), fetchData.lastStableOffset)
assertEquals(Some(List.empty[AbortedTransaction]), fetchData.abortedTransactions) assertEquals(Some(List.empty[FetchResponseData.AbortedTransaction]), fetchData.abortedTransactions)
// now commit the transaction // now commit the transaction
val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0) val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
@ -448,7 +443,7 @@ class ReplicaManagerTest {
fetchData = consumerFetchResult.assertFired fetchData = consumerFetchResult.assertFired
assertEquals(Errors.NONE, fetchData.error) assertEquals(Errors.NONE, fetchData.error)
assertEquals(Some(numRecords + 1), fetchData.lastStableOffset) assertEquals(Some(numRecords + 1), fetchData.lastStableOffset)
assertEquals(Some(List.empty[AbortedTransaction]), fetchData.abortedTransactions) assertEquals(Some(List.empty[FetchResponseData.AbortedTransaction]), fetchData.abortedTransactions)
assertEquals(numRecords + 1, fetchData.records.batches.asScala.size) assertEquals(numRecords + 1, fetchData.records.batches.asScala.size)
} finally { } finally {
replicaManager.shutdown(checkpointHW = false) replicaManager.shutdown(checkpointHW = false)

View File

@ -193,7 +193,7 @@ class UpdateFeaturesTest extends BaseRequestTest {
new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(), new UpdateFeaturesRequest.Builder(new UpdateFeaturesRequestData().setFeatureUpdates(validUpdates)).build(),
notControllerSocketServer) notControllerSocketServer)
assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode())) assertEquals(Errors.NOT_CONTROLLER, Errors.forCode(response.data.errorCode))
assertNotNull(response.data.errorMessage()) assertNotNull(response.data.errorMessage())
assertEquals(0, response.data.results.size) assertEquals(0, response.data.results.size)
checkFeatures( checkFeatures(

View File

@ -18,14 +18,12 @@ package kafka.server.epoch.util
import java.net.SocketTimeoutException import java.net.SocketTimeoutException
import java.util import java.util
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.server.BlockingSend import kafka.server.BlockingSend
import org.apache.kafka.clients.{ClientRequest, ClientResponse, MockClient, NetworkClientUtils} import org.apache.kafka.clients.{ClientRequest, ClientResponse, MockClient, NetworkClientUtils}
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData import org.apache.kafka.common.message.{FetchResponseData, OffsetForLeaderEpochResponseData}
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{OffsetForLeaderTopicResult, EpochEndOffset} import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.{EpochEndOffset, OffsetForLeaderTopicResult}
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.Records
import org.apache.kafka.common.requests.AbstractRequest.Builder import org.apache.kafka.common.requests.AbstractRequest.Builder
import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, OffsetsForLeaderEpochResponse, FetchMetadata => JFetchMetadata} import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, OffsetsForLeaderEpochResponse, FetchMetadata => JFetchMetadata}
import org.apache.kafka.common.utils.{SystemTime, Time} import org.apache.kafka.common.utils.{SystemTime, Time}
@ -52,7 +50,7 @@ class ReplicaFetcherMockBlockingSend(offsets: java.util.Map[TopicPartition, Epoc
var lastUsedOffsetForLeaderEpochVersion = -1 var lastUsedOffsetForLeaderEpochVersion = -1
var callback: Option[() => Unit] = None var callback: Option[() => Unit] = None
var currentOffsets: util.Map[TopicPartition, EpochEndOffset] = offsets var currentOffsets: util.Map[TopicPartition, EpochEndOffset] = offsets
var fetchPartitionData: Map[TopicPartition, FetchResponse.PartitionData[Records]] = Map.empty var fetchPartitionData: Map[TopicPartition, FetchResponseData.PartitionData] = Map.empty
private val sourceNode = new Node(sourceBroker.id, sourceBroker.host, sourceBroker.port) private val sourceNode = new Node(sourceBroker.id, sourceBroker.host, sourceBroker.port)
def setEpochRequestCallback(postEpochFunction: () => Unit): Unit = { def setEpochRequestCallback(postEpochFunction: () => Unit): Unit = {
@ -63,7 +61,7 @@ class ReplicaFetcherMockBlockingSend(offsets: java.util.Map[TopicPartition, Epoc
currentOffsets = newOffsets currentOffsets = newOffsets
} }
def setFetchPartitionDataForNextResponse(partitionData: Map[TopicPartition, FetchResponse.PartitionData[Records]]): Unit = { def setFetchPartitionDataForNextResponse(partitionData: Map[TopicPartition, FetchResponseData.PartitionData]): Unit = {
fetchPartitionData = partitionData fetchPartitionData = partitionData
} }
@ -97,11 +95,11 @@ class ReplicaFetcherMockBlockingSend(offsets: java.util.Map[TopicPartition, Epoc
case ApiKeys.FETCH => case ApiKeys.FETCH =>
fetchCount += 1 fetchCount += 1
val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData[Records]] val partitionData = new util.LinkedHashMap[TopicPartition, FetchResponseData.PartitionData]
fetchPartitionData.foreach { case (tp, data) => partitionData.put(tp, data) } fetchPartitionData.foreach { case (tp, data) => partitionData.put(tp, data) }
fetchPartitionData = Map.empty fetchPartitionData = Map.empty
new FetchResponse(Errors.NONE, partitionData, 0, FetchResponse.of(Errors.NONE, 0,
if (partitionData.isEmpty) JFetchMetadata.INVALID_SESSION_ID else 1) if (partitionData.isEmpty) JFetchMetadata.INVALID_SESSION_ID else 1, partitionData)
case _ => case _ =>
throw new UnsupportedOperationException throw new UnsupportedOperationException

View File

@ -18,6 +18,7 @@
package org.apache.kafka.jmh.common; package org.apache.kafka.jmh.common;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
@ -42,9 +43,7 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.Optional;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -61,11 +60,11 @@ public class FetchResponseBenchmark {
@Param({"3", "10", "20"}) @Param({"3", "10", "20"})
private int partitionCount; private int partitionCount;
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> responseData; LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> responseData;
ResponseHeader header; ResponseHeader header;
FetchResponse<MemoryRecords> fetchResponse; FetchResponse fetchResponse;
@Setup(Level.Trial) @Setup(Level.Trial)
public void setup() { public void setup() {
@ -78,19 +77,22 @@ public class FetchResponseBenchmark {
for (int topicIdx = 0; topicIdx < topicCount; topicIdx++) { for (int topicIdx = 0; topicIdx < topicCount; topicIdx++) {
String topic = UUID.randomUUID().toString(); String topic = UUID.randomUUID().toString();
for (int partitionId = 0; partitionId < partitionCount; partitionId++) { for (int partitionId = 0; partitionId < partitionCount; partitionId++) {
FetchResponse.PartitionData<MemoryRecords> partitionData = new FetchResponse.PartitionData<>( FetchResponseData.PartitionData partitionData = new FetchResponseData.PartitionData()
Errors.NONE, 0, 0, 0, Optional.empty(), Collections.emptyList(), records); .setPartitionIndex(partitionId)
.setLastStableOffset(0)
.setLogStartOffset(0)
.setRecords(records);
responseData.put(new TopicPartition(topic, partitionId), partitionData); responseData.put(new TopicPartition(topic, partitionId), partitionData);
} }
} }
this.header = new ResponseHeader(100, ApiKeys.FETCH.responseHeaderVersion(ApiKeys.FETCH.latestVersion())); this.header = new ResponseHeader(100, ApiKeys.FETCH.responseHeaderVersion(ApiKeys.FETCH.latestVersion()));
this.fetchResponse = new FetchResponse<>(Errors.NONE, responseData, 0, 0); this.fetchResponse = FetchResponse.of(Errors.NONE, 0, 0, responseData);
} }
@Benchmark @Benchmark
public int testConstructFetchResponse() { public int testConstructFetchResponse() {
FetchResponse<MemoryRecords> fetchResponse = new FetchResponse<>(Errors.NONE, responseData, 0, 0); FetchResponse fetchResponse = FetchResponse.of(Errors.NONE, 0, 0, responseData);
return fetchResponse.responseData().size(); return fetchResponse.responseData().size();
} }

View File

@ -44,13 +44,13 @@ import kafka.server.metadata.CachedConfigRepository;
import kafka.utils.KafkaScheduler; import kafka.utils.KafkaScheduler;
import kafka.utils.Pool; import kafka.utils.Pool;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.message.LeaderAndIsrRequestData; import org.apache.kafka.common.message.LeaderAndIsrRequestData;
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition;
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset; import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.BaseRecords; import org.apache.kafka.common.record.BaseRecords;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.RecordsSend; import org.apache.kafka.common.record.RecordsSend;
import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
@ -82,7 +82,6 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.Properties; import java.util.Properties;
@ -137,7 +136,7 @@ public class ReplicaFetcherThreadBenchmark {
Time.SYSTEM, Time.SYSTEM,
true); true);
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<BaseRecords>> initialFetched = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> initialFetched = new LinkedHashMap<>();
scala.collection.mutable.Map<TopicPartition, InitialFetchState> initialFetchStates = new scala.collection.mutable.HashMap<>(); scala.collection.mutable.Map<TopicPartition, InitialFetchState> initialFetchStates = new scala.collection.mutable.HashMap<>();
for (int i = 0; i < partitionCount; i++) { for (int i = 0; i < partitionCount; i++) {
TopicPartition tp = new TopicPartition("topic", i); TopicPartition tp = new TopicPartition("topic", i);
@ -174,8 +173,11 @@ public class ReplicaFetcherThreadBenchmark {
return null; return null;
} }
}; };
initialFetched.put(tp, new FetchResponse.PartitionData<>(Errors.NONE, 0, 0, 0, initialFetched.put(tp, new FetchResponseData.PartitionData()
new LinkedList<>(), fetched)); .setPartitionIndex(tp.partition())
.setLastStableOffset(0)
.setLogStartOffset(0)
.setRecords(fetched));
} }
ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class); ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class);
@ -186,7 +188,7 @@ public class ReplicaFetcherThreadBenchmark {
// so that we do not measure this time as part of the steady state work // so that we do not measure this time as part of the steady state work
fetcher.doWork(); fetcher.doWork();
// handle response to engage the incremental fetch session handler // handle response to engage the incremental fetch session handler
fetcher.fetchSessionHandler().handleResponse(new FetchResponse<>(Errors.NONE, initialFetched, 0, 999)); fetcher.fetchSessionHandler().handleResponse(FetchResponse.of(Errors.NONE, 0, 999, initialFetched));
} }
@TearDown(Level.Trial) @TearDown(Level.Trial)
@ -292,7 +294,8 @@ public class ReplicaFetcherThreadBenchmark {
} }
@Override @Override
public Option<LogAppendInfo> processPartitionData(TopicPartition topicPartition, long fetchOffset, FetchResponse.PartitionData partitionData) { public Option<LogAppendInfo> processPartitionData(TopicPartition topicPartition, long fetchOffset,
FetchResponseData.PartitionData partitionData) {
return Option.empty(); return Option.empty();
} }
@ -317,7 +320,7 @@ public class ReplicaFetcherThreadBenchmark {
} }
@Override @Override
public Map<TopicPartition, FetchResponse.PartitionData<Records>> fetchFromLeader(FetchRequest.Builder fetchRequest) { public Map<TopicPartition, FetchResponseData.PartitionData> fetchFromLeader(FetchRequest.Builder fetchRequest) {
return new scala.collection.mutable.HashMap<>(); return new scala.collection.mutable.HashMap<>();
} }
} }

View File

@ -19,8 +19,8 @@ package org.apache.kafka.jmh.fetchsession;
import org.apache.kafka.clients.FetchSessionHandler; import org.apache.kafka.clients.FetchSessionHandler;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.LogContext;
@ -70,24 +70,21 @@ public class FetchSessionBenchmark {
handler = new FetchSessionHandler(LOG_CONTEXT, 1); handler = new FetchSessionHandler(LOG_CONTEXT, 1);
FetchSessionHandler.Builder builder = handler.newBuilder(); FetchSessionHandler.Builder builder = handler.newBuilder();
LinkedHashMap<TopicPartition, FetchResponse.PartitionData<MemoryRecords>> respMap = new LinkedHashMap<>(); LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> respMap = new LinkedHashMap<>();
for (int i = 0; i < partitionCount; i++) { for (int i = 0; i < partitionCount; i++) {
TopicPartition tp = new TopicPartition("foo", i); TopicPartition tp = new TopicPartition("foo", i);
FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(0, 0, 200, FetchRequest.PartitionData partitionData = new FetchRequest.PartitionData(0, 0, 200,
Optional.empty()); Optional.empty());
fetches.put(tp, partitionData); fetches.put(tp, partitionData);
builder.add(tp, partitionData); builder.add(tp, partitionData);
respMap.put(tp, new FetchResponse.PartitionData<>( respMap.put(tp, new FetchResponseData.PartitionData()
Errors.NONE, .setPartitionIndex(tp.partition())
0L, .setLastStableOffset(0)
0L, .setLogStartOffset(0));
0,
null,
null));
} }
builder.build(); builder.build();
// build and handle an initial response so that the next fetch will be incremental // build and handle an initial response so that the next fetch will be incremental
handler.handleResponse(new FetchResponse<>(Errors.NONE, respMap, 0, 1)); handler.handleResponse(FetchResponse.of(Errors.NONE, 0, 1, respMap));
int counter = 0; int counter = 0;
for (TopicPartition topicPartition: new ArrayList<>(fetches.keySet())) { for (TopicPartition topicPartition: new ArrayList<>(fetches.keySet())) {

View File

@ -40,6 +40,7 @@ import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
@ -906,7 +907,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
) { ) {
return RaftUtil.singletonFetchResponse(log.topicPartition(), Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(log.topicPartition(), Errors.NONE, partitionData -> {
partitionData partitionData
.setRecordSet(records) .setRecords(records)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setLogStartOffset(log.startOffset()) .setLogStartOffset(log.startOffset())
.setHighWatermark(highWatermark .setHighWatermark(highWatermark
@ -991,11 +992,11 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
} }
FetchResponseData response = tryCompleteFetchRequest(request.replicaId(), fetchPartition, currentTimeMs); FetchResponseData response = tryCompleteFetchRequest(request.replicaId(), fetchPartition, currentTimeMs);
FetchResponseData.FetchablePartitionResponse partitionResponse = FetchResponseData.PartitionData partitionResponse =
response.responses().get(0).partitionResponses().get(0); response.responses().get(0).partitions().get(0);
if (partitionResponse.errorCode() != Errors.NONE.code() if (partitionResponse.errorCode() != Errors.NONE.code()
|| partitionResponse.recordSet().sizeInBytes() > 0 || FetchResponse.recordsSize(partitionResponse) > 0
|| request.maxWaitMs() == 0) { || request.maxWaitMs() == 0) {
return completedFuture(response); return completedFuture(response);
} }
@ -1084,8 +1085,8 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
return false; return false;
} }
FetchResponseData.FetchablePartitionResponse partitionResponse = FetchResponseData.PartitionData partitionResponse =
response.responses().get(0).partitionResponses().get(0); response.responses().get(0).partitions().get(0);
FetchResponseData.LeaderIdAndEpoch currentLeaderIdAndEpoch = partitionResponse.currentLeader(); FetchResponseData.LeaderIdAndEpoch currentLeaderIdAndEpoch = partitionResponse.currentLeader();
OptionalInt responseLeaderId = optionalLeaderId(currentLeaderIdAndEpoch.leaderId()); OptionalInt responseLeaderId = optionalLeaderId(currentLeaderIdAndEpoch.leaderId());
@ -1143,7 +1144,7 @@ public class KafkaRaftClient<T> implements RaftClient<T> {
state.setFetchingSnapshot(Optional.of(log.createSnapshot(snapshotId))); state.setFetchingSnapshot(Optional.of(log.createSnapshot(snapshotId)));
} }
} else { } else {
Records records = (Records) partitionResponse.recordSet(); Records records = FetchResponse.recordsOrFail(partitionResponse);
if (records.sizeInBytes() > 0) { if (records.sizeInBytes() > 0) {
appendAsFollower(records); appendAsFollower(records);
} }

View File

@ -73,19 +73,19 @@ public class RaftUtil {
public static FetchResponseData singletonFetchResponse( public static FetchResponseData singletonFetchResponse(
TopicPartition topicPartition, TopicPartition topicPartition,
Errors topLevelError, Errors topLevelError,
Consumer<FetchResponseData.FetchablePartitionResponse> partitionConsumer Consumer<FetchResponseData.PartitionData> partitionConsumer
) { ) {
FetchResponseData.FetchablePartitionResponse fetchablePartition = FetchResponseData.PartitionData fetchablePartition =
new FetchResponseData.FetchablePartitionResponse(); new FetchResponseData.PartitionData();
fetchablePartition.setPartition(topicPartition.partition()); fetchablePartition.setPartitionIndex(topicPartition.partition());
partitionConsumer.accept(fetchablePartition); partitionConsumer.accept(fetchablePartition);
FetchResponseData.FetchableTopicResponse fetchableTopic = FetchResponseData.FetchableTopicResponse fetchableTopic =
new FetchResponseData.FetchableTopicResponse() new FetchResponseData.FetchableTopicResponse()
.setTopic(topicPartition.topic()) .setTopic(topicPartition.topic())
.setPartitionResponses(Collections.singletonList(fetchablePartition)); .setPartitions(Collections.singletonList(fetchablePartition));
return new FetchResponseData() return new FetchResponseData()
.setErrorCode(topLevelError.code()) .setErrorCode(topLevelError.code())
@ -102,8 +102,8 @@ public class RaftUtil {
static boolean hasValidTopicPartition(FetchResponseData data, TopicPartition topicPartition) { static boolean hasValidTopicPartition(FetchResponseData data, TopicPartition topicPartition) {
return data.responses().size() == 1 && return data.responses().size() == 1 &&
data.responses().get(0).topic().equals(topicPartition.topic()) && data.responses().get(0).topic().equals(topicPartition.topic()) &&
data.responses().get(0).partitionResponses().size() == 1 && data.responses().get(0).partitions().size() == 1 &&
data.responses().get(0).partitionResponses().get(0).partition() == topicPartition.partition(); data.responses().get(0).partitions().get(0).partitionIndex() == topicPartition.partition();
} }
static boolean hasValidTopicPartition(VoteResponseData data, TopicPartition topicPartition) { static boolean hasValidTopicPartition(VoteResponseData data, TopicPartition topicPartition) {

View File

@ -89,7 +89,7 @@ final public class KafkaRaftClientSnapshotTest {
// Send Fetch request less than start offset // Send Fetch request less than start offset
context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, epoch, 0)); context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0, epoch, 0));
context.pollUntilResponse(); context.pollUntilResponse();
FetchResponseData.FetchablePartitionResponse partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId()); assertEquals(localId, partitionResponse.currentLeader().leaderId());
@ -176,7 +176,7 @@ final public class KafkaRaftClientSnapshotTest {
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset + 1, oldestSnapshotId.epoch + 1, 0) context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset + 1, oldestSnapshotId.epoch + 1, 0)
); );
context.pollUntilResponse(); context.pollUntilResponse();
FetchResponseData.FetchablePartitionResponse partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId()); assertEquals(localId, partitionResponse.currentLeader().leaderId());
@ -265,7 +265,7 @@ final public class KafkaRaftClientSnapshotTest {
context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset, oldestSnapshotId.epoch + 1, 0) context.fetchRequest(epoch, otherNodeId, oldestSnapshotId.offset, oldestSnapshotId.epoch + 1, 0)
); );
context.pollUntilResponse(); context.pollUntilResponse();
FetchResponseData.FetchablePartitionResponse partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId()); assertEquals(localId, partitionResponse.currentLeader().leaderId());
@ -318,7 +318,7 @@ final public class KafkaRaftClientSnapshotTest {
) )
); );
context.pollUntilResponse(); context.pollUntilResponse();
FetchResponseData.FetchablePartitionResponse partitionResponse = context.assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = context.assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(localId, partitionResponse.currentLeader().leaderId()); assertEquals(localId, partitionResponse.currentLeader().leaderId());
@ -1329,9 +1329,7 @@ final public class KafkaRaftClientSnapshotTest {
long highWatermark long highWatermark
) { ) {
return RaftUtil.singletonFetchResponse(topicPartition, Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(topicPartition, Errors.NONE, partitionData -> {
partitionData partitionData.setHighWatermark(highWatermark);
.setErrorCode(Errors.NONE.code())
.setHighWatermark(highWatermark);
partitionData.currentLeader() partitionData.currentLeader()
.setLeaderEpoch(epoch) .setLeaderEpoch(epoch)

View File

@ -606,7 +606,7 @@ public final class RaftClientTestContext {
return raftMessage.correlationId(); return raftMessage.correlationId();
} }
FetchResponseData.FetchablePartitionResponse assertSentFetchPartitionResponse() { FetchResponseData.PartitionData assertSentFetchPartitionResponse() {
List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH); List<RaftResponse.Outbound> sentMessages = drainSentResponses(ApiKeys.FETCH);
assertEquals( assertEquals(
1, sentMessages.size(), "Found unexpected sent messages " + sentMessages); 1, sentMessages.size(), "Found unexpected sent messages " + sentMessages);
@ -617,8 +617,8 @@ public final class RaftClientTestContext {
assertEquals(1, response.responses().size()); assertEquals(1, response.responses().size());
assertEquals(metadataPartition.topic(), response.responses().get(0).topic()); assertEquals(metadataPartition.topic(), response.responses().get(0).topic());
assertEquals(1, response.responses().get(0).partitionResponses().size()); assertEquals(1, response.responses().get(0).partitions().size());
return response.responses().get(0).partitionResponses().get(0); return response.responses().get(0).partitions().get(0);
} }
void assertSentFetchPartitionResponse(Errors error) { void assertSentFetchPartitionResponse(Errors error) {
@ -637,7 +637,7 @@ public final class RaftClientTestContext {
int epoch, int epoch,
OptionalInt leaderId OptionalInt leaderId
) { ) {
FetchResponseData.FetchablePartitionResponse partitionResponse = assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = assertSentFetchPartitionResponse();
assertEquals(error, Errors.forCode(partitionResponse.errorCode())); assertEquals(error, Errors.forCode(partitionResponse.errorCode()));
assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(epoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(leaderId.orElse(-1), partitionResponse.currentLeader().leaderId()); assertEquals(leaderId.orElse(-1), partitionResponse.currentLeader().leaderId());
@ -645,14 +645,14 @@ public final class RaftClientTestContext {
assertEquals(-1, partitionResponse.divergingEpoch().epoch()); assertEquals(-1, partitionResponse.divergingEpoch().epoch());
assertEquals(-1, partitionResponse.snapshotId().endOffset()); assertEquals(-1, partitionResponse.snapshotId().endOffset());
assertEquals(-1, partitionResponse.snapshotId().epoch()); assertEquals(-1, partitionResponse.snapshotId().epoch());
return (MemoryRecords) partitionResponse.recordSet(); return (MemoryRecords) partitionResponse.records();
} }
MemoryRecords assertSentFetchPartitionResponse( MemoryRecords assertSentFetchPartitionResponse(
long highWatermark, long highWatermark,
int leaderEpoch int leaderEpoch
) { ) {
FetchResponseData.FetchablePartitionResponse partitionResponse = assertSentFetchPartitionResponse(); FetchResponseData.PartitionData partitionResponse = assertSentFetchPartitionResponse();
assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode())); assertEquals(Errors.NONE, Errors.forCode(partitionResponse.errorCode()));
assertEquals(leaderEpoch, partitionResponse.currentLeader().leaderEpoch()); assertEquals(leaderEpoch, partitionResponse.currentLeader().leaderEpoch());
assertEquals(highWatermark, partitionResponse.highWatermark()); assertEquals(highWatermark, partitionResponse.highWatermark());
@ -660,7 +660,7 @@ public final class RaftClientTestContext {
assertEquals(-1, partitionResponse.divergingEpoch().epoch()); assertEquals(-1, partitionResponse.divergingEpoch().epoch());
assertEquals(-1, partitionResponse.snapshotId().endOffset()); assertEquals(-1, partitionResponse.snapshotId().endOffset());
assertEquals(-1, partitionResponse.snapshotId().epoch()); assertEquals(-1, partitionResponse.snapshotId().epoch());
return (MemoryRecords) partitionResponse.recordSet(); return (MemoryRecords) partitionResponse.records();
} }
RaftRequest.Outbound assertSentFetchSnapshotRequest() { RaftRequest.Outbound assertSentFetchSnapshotRequest() {
@ -928,7 +928,7 @@ public final class RaftClientTestContext {
) { ) {
return RaftUtil.singletonFetchResponse(metadataPartition, Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(metadataPartition, Errors.NONE, partitionData -> {
partitionData partitionData
.setRecordSet(records) .setRecords(records)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setHighWatermark(highWatermark); .setHighWatermark(highWatermark);
@ -946,9 +946,7 @@ public final class RaftClientTestContext {
long highWatermark long highWatermark
) { ) {
return RaftUtil.singletonFetchResponse(metadataPartition, Errors.NONE, partitionData -> { return RaftUtil.singletonFetchResponse(metadataPartition, Errors.NONE, partitionData -> {
partitionData partitionData.setHighWatermark(highWatermark);
.setErrorCode(Errors.NONE.code())
.setHighWatermark(highWatermark);
partitionData.currentLeader() partitionData.currentLeader()
.setLeaderEpoch(epoch) .setLeaderEpoch(epoch)